From e312cfa79400c5ce9ad63236672fc253ee324494 Mon Sep 17 00:00:00 2001 From: attack204 Date: Thu, 19 Jan 2023 10:19:04 +0800 Subject: [PATCH 0001/2223] feature:urlCluster --- src/Storages/StorageURL.cpp | 573 +++++++++--------- src/Storages/StorageURL.h | 93 ++- src/Storages/StorageURLCluster.cpp | 161 +++++ src/Storages/StorageURLCluster.h | 56 ++ src/TableFunctions/TableFunctionURL.h | 4 +- .../TableFunctionURLCluster.cpp | 119 ++++ src/TableFunctions/TableFunctionURLCluster.h | 49 ++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + 9 files changed, 780 insertions(+), 277 deletions(-) create mode 100644 src/Storages/StorageURLCluster.cpp create mode 100644 src/Storages/StorageURLCluster.h create mode 100644 src/TableFunctions/TableFunctionURLCluster.cpp create mode 100644 src/TableFunctions/TableFunctionURLCluster.h diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index dbe99d0ce9a..9002e27ae46 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -37,6 +37,8 @@ #include #include #include +#include +#include namespace DB @@ -121,284 +123,220 @@ IStorageURLBase::IStorageURLBase( setInMemoryMetadata(storage_metadata); } - -namespace +HTTPHeaderEntries getHeaders(const HTTPHeaderEntries & headers_) { - HTTPHeaderEntries getHeaders(const HTTPHeaderEntries & headers_) + HTTPHeaderEntries headers(headers_.begin(), headers_.end()); + + // Propagate OpenTelemetry trace context, if any, downstream. + const auto ¤t_trace_context = OpenTelemetry::CurrentContext(); + if (current_trace_context.isTraceEnabled()) { - HTTPHeaderEntries headers(headers_.begin(), headers_.end()); + headers.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); - // Propagate OpenTelemetry trace context, if any, downstream. - const auto ¤t_trace_context = OpenTelemetry::CurrentContext(); - if (current_trace_context.isTraceEnabled()) + if (!current_trace_context.tracestate.empty()) { - headers.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); - - if (!current_trace_context.tracestate.empty()) - { - headers.emplace_back("tracestate", current_trace_context.tracestate); - } + headers.emplace_back("tracestate", current_trace_context.tracestate); } - - return headers; } + return headers; +} - class StorageURLSource : public ISource +std::vector getPathsList(const String & uri, ContextPtr context) +{ + context->getRemoteHostFilter().checkURL(Poco::URI(uri)); + + Poco::Net::HTTPBasicCredentials credentials; + + std::vector urls_to_check; + if (urlWithGlobs(uri)) { - using URIParams = std::vector>; - - public: - struct URIInfo + size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; + auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); + for (const auto & description : uri_descriptions) { - using FailoverOptions = std::vector; - std::vector uri_list_to_read; - std::atomic next_uri_to_read = 0; + auto options = parseRemoteDescription(description, 0, description.size(), '|', max_addresses); + urls_to_check.insert(urls_to_check.end(), options.begin(), options.end()); + } + } + else + { + urls_to_check = {uri}; + } + + return urls_to_check; +} + +void StorageURLSource::setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri) +{ + const auto & user_info = request_uri.getUserInfo(); + if (!user_info.empty()) + { + std::size_t n = user_info.find(':'); + if (n != std::string::npos) + { + credentials.setUsername(user_info.substr(0, n)); + credentials.setPassword(user_info.substr(n + 1)); + } + } +} + +StorageURLSource::StorageURLSource( + URIInfoPtr uri_info_, + const std::string & http_method, + std::function callback, + const String & format, + const std::optional & format_settings, + String name_, + const Block & sample_block, + ContextPtr context, + const ColumnsDescription & columns, + UInt64 max_block_size, + const ConnectionTimeouts & timeouts, + CompressionMethod compression_method, + size_t download_threads, + const HTTPHeaderEntries & headers_, + const URIParams & params, + bool glob_url) + : ISource(sample_block), name(std::move(name_)), uri_info(uri_info_) + { + auto headers = getHeaders(headers_); + + /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. + initialize = [=, this](const URIInfo::FailoverOptions & uri_options) + { + if (uri_options.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); + + auto first_option = uri_options.begin(); + read_buf = getFirstAvailableURLReadBuffer( + first_option, + uri_options.end(), + context, + params, + http_method, + callback, + timeouts, + compression_method, + credentials, + headers, + glob_url, + uri_options.size() == 1, + download_threads); + + auto input_format + = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size, format_settings); + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); + + builder.addSimpleTransform( + [&](const Block & cur_header) + { return std::make_shared(cur_header, columns, *input_format, context); }); + + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + reader = std::make_unique(*pipeline); }; - using URIInfoPtr = std::shared_ptr; + } - void onCancel() override - { - std::lock_guard lock(reader_mutex); - if (reader) - reader->cancel(); - } +std::unique_ptr StorageURLSource::getFirstAvailableURLReadBuffer( + std::vector::const_iterator & option, + const std::vector::const_iterator & end, + ContextPtr context, + const URIParams & params, + const String & http_method, + std::function callback, + const ConnectionTimeouts & timeouts, + CompressionMethod compression_method, + Poco::Net::HTTPBasicCredentials & credentials, + const HTTPHeaderEntries & headers, + bool glob_url, + bool delay_initialization, + size_t download_threads) +{ + String first_exception_message; + ReadSettings read_settings = context->getReadSettings(); - static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri) + size_t options = std::distance(option, end); + for (; option != end; ++option) + { + bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); + auto request_uri = Poco::URI(*option); + + for (const auto & [param, value] : params) + request_uri.addQueryParameter(param, value); + + setCredentials(credentials, request_uri); + + const auto settings = context->getSettings(); + int zstd_window_log_max = static_cast(settings.zstd_window_log_max); + try { - const auto & user_info = request_uri.getUserInfo(); - if (!user_info.empty()) + if (download_threads > 1) { - std::size_t n = user_info.find(':'); - if (n != std::string::npos) - { - credentials.setUsername(user_info.substr(0, n)); - credentials.setPassword(user_info.substr(n + 1)); - } - } - } - - StorageURLSource( - URIInfoPtr uri_info_, - const std::string & http_method, - std::function callback, - const String & format, - const std::optional & format_settings, - String name_, - const Block & sample_block, - ContextPtr context, - const ColumnsDescription & columns, - UInt64 max_block_size, - const ConnectionTimeouts & timeouts, - CompressionMethod compression_method, - size_t download_threads, - const HTTPHeaderEntries & headers_ = {}, - const URIParams & params = {}, - bool glob_url = false) - : ISource(sample_block), name(std::move(name_)), uri_info(uri_info_) - { - auto headers = getHeaders(headers_); - - /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. - initialize = [=, this](const URIInfo::FailoverOptions & uri_options) - { - if (uri_options.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); - - auto first_option = uri_options.begin(); - read_buf = getFirstAvailableURLReadBuffer( - first_option, - uri_options.end(), - context, - params, - http_method, - callback, - timeouts, - compression_method, - credentials, - headers, - glob_url, - uri_options.size() == 1, - download_threads); - - auto input_format - = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size, format_settings); - QueryPipelineBuilder builder; - builder.init(Pipe(input_format)); - - builder.addSimpleTransform( - [&](const Block & cur_header) - { return std::make_shared(cur_header, columns, *input_format, context); }); - - pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - reader = std::make_unique(*pipeline); - }; - } - - String getName() const override { return name; } - - Chunk generate() override - { - while (true) - { - if (!reader) - { - auto current_uri_pos = uri_info->next_uri_to_read.fetch_add(1); - if (current_uri_pos >= uri_info->uri_list_to_read.size()) - return {}; - - auto current_uri = uri_info->uri_list_to_read[current_uri_pos]; - - std::lock_guard lock(reader_mutex); - initialize(current_uri); - } - - Chunk chunk; - std::lock_guard lock(reader_mutex); - if (reader->pull(chunk)) - return chunk; - - pipeline->reset(); - reader.reset(); - } - } - - static std::unique_ptr getFirstAvailableURLReadBuffer( - std::vector::const_iterator & option, - const std::vector::const_iterator & end, - ContextPtr context, - const URIParams & params, - const String & http_method, - std::function callback, - const ConnectionTimeouts & timeouts, - CompressionMethod compression_method, - Poco::Net::HTTPBasicCredentials & credentials, - const HTTPHeaderEntries & headers, - bool glob_url, - bool delay_initialization, - size_t download_threads) - { - String first_exception_message; - ReadSettings read_settings = context->getReadSettings(); - - size_t options = std::distance(option, end); - for (; option != end; ++option) - { - bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); - auto request_uri = Poco::URI(*option); - - for (const auto & [param, value] : params) - request_uri.addQueryParameter(param, value); - - setCredentials(credentials, request_uri); - - const auto settings = context->getSettings(); - int zstd_window_log_max = static_cast(settings.zstd_window_log_max); try { - if (download_threads > 1) + ReadWriteBufferFromHTTP buffer( + request_uri, + Poco::Net::HTTPRequest::HTTP_HEAD, + callback, + timeouts, + credentials, + settings.max_http_get_redirects, + settings.max_read_buffer_size, + read_settings, + headers, + ReadWriteBufferFromHTTP::Range{0, std::nullopt}, + &context->getRemoteHostFilter(), + true, + /* use_external_buffer */ false, + /* skip_url_not_found_error */ skip_url_not_found_error); + + Poco::Net::HTTPResponse res; + + for (size_t i = 0; i < settings.http_max_tries; ++i) { try { - ReadWriteBufferFromHTTP buffer( - request_uri, - Poco::Net::HTTPRequest::HTTP_HEAD, - callback, - timeouts, - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, - ReadWriteBufferFromHTTP::Range{0, std::nullopt}, - &context->getRemoteHostFilter(), - true, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error); - - Poco::Net::HTTPResponse res; - - for (size_t i = 0; i < settings.http_max_tries; ++i) - { - try - { - buffer.callWithRedirects(res, Poco::Net::HTTPRequest::HTTP_HEAD, true); - break; - } - catch (const Poco::Exception & e) - { - LOG_TRACE( - &Poco::Logger::get("StorageURLSource"), - "HTTP HEAD request to `{}` failed at try {}/{}. " - "Error: {}.", - request_uri.toString(), - i + 1, - settings.http_max_tries, - e.displayText()); - if (!ReadWriteBufferFromHTTP::isRetriableError(res.getStatus())) - { - throw; - } - } - } - - // to check if Range header is supported, we need to send a request with it set - const bool supports_ranges = (res.has("Accept-Ranges") && res.get("Accept-Ranges") == "bytes") - || (res.has("Content-Range") && res.get("Content-Range").starts_with("bytes")); - LOG_TRACE( - &Poco::Logger::get("StorageURLSource"), - fmt::runtime(supports_ranges ? "HTTP Range is supported" : "HTTP Range is not supported")); - - - if (supports_ranges && res.getStatus() == Poco::Net::HTTPResponse::HTTP_PARTIAL_CONTENT - && res.hasContentLength()) - { - LOG_TRACE( - &Poco::Logger::get("StorageURLSource"), - "Using ParallelReadBuffer with {} workers with chunks of {} bytes", - download_threads, - settings.max_download_buffer_size); - - auto read_buffer_factory = std::make_unique( - res.getContentLength(), - settings.max_download_buffer_size, - request_uri, - http_method, - callback, - timeouts, - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, - &context->getRemoteHostFilter(), - delay_initialization, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error); - - return wrapReadBufferWithCompressionMethod( - std::make_unique( - std::move(read_buffer_factory), - threadPoolCallbackRunner(IOThreadPool::get(), "URLParallelRead"), - download_threads), - compression_method, - zstd_window_log_max); - } + buffer.callWithRedirects(res, Poco::Net::HTTPRequest::HTTP_HEAD, true); + break; } catch (const Poco::Exception & e) { LOG_TRACE( &Poco::Logger::get("StorageURLSource"), - "Failed to setup ParallelReadBuffer because of an exception:\n{}.\nFalling back to the single-threaded " - "buffer", + "HTTP HEAD request to `{}` failed at try {}/{}. " + "Error: {}.", + request_uri.toString(), + i + 1, + settings.http_max_tries, e.displayText()); + if (!ReadWriteBufferFromHTTP::isRetriableError(res.getStatus())) + { + throw; + } } } - LOG_TRACE(&Poco::Logger::get("StorageURLSource"), "Using single-threaded read buffer"); + // to check if Range header is supported, we need to send a request with it set + const bool supports_ranges = (res.has("Accept-Ranges") && res.get("Accept-Ranges") == "bytes") + || (res.has("Content-Range") && res.get("Content-Range").starts_with("bytes")); + LOG_TRACE( + &Poco::Logger::get("StorageURLSource"), + fmt::runtime(supports_ranges ? "HTTP Range is supported" : "HTTP Range is not supported")); - return wrapReadBufferWithCompressionMethod( - std::make_unique( + + if (supports_ranges && res.getStatus() == Poco::Net::HTTPResponse::HTTP_PARTIAL_CONTENT + && res.hasContentLength()) + { + LOG_TRACE( + &Poco::Logger::get("StorageURLSource"), + "Using ParallelReadBuffer with {} workers with chunks of {} bytes", + download_threads, + settings.max_download_buffer_size); + + auto read_buffer_factory = std::make_unique( + res.getContentLength(), + settings.max_download_buffer_size, request_uri, http_method, callback, @@ -408,45 +346,130 @@ namespace settings.max_read_buffer_size, read_settings, headers, - ReadWriteBufferFromHTTP::Range{}, &context->getRemoteHostFilter(), delay_initialization, /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error), + /* skip_url_not_found_error */ skip_url_not_found_error); + + return wrapReadBufferWithCompressionMethod( + std::make_unique( + std::move(read_buffer_factory), + threadPoolCallbackRunner(IOThreadPool::get(), "URLParallelRead"), + download_threads), compression_method, - zstd_window_log_max); + zstd_window_log_max); + } } - catch (...) + catch (const Poco::Exception & e) { - if (first_exception_message.empty()) - first_exception_message = getCurrentExceptionMessage(false); - - if (options == 1) - throw; - - tryLogCurrentException(__PRETTY_FUNCTION__); + LOG_TRACE( + &Poco::Logger::get("StorageURLSource"), + "Failed to setup ParallelReadBuffer because of an exception:\n{}.\nFalling back to the single-threaded " + "buffer", + e.displayText()); } } - throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message); + LOG_TRACE(&Poco::Logger::get("StorageURLSource"), "Using single-threaded read buffer"); + + return wrapReadBufferWithCompressionMethod( + std::make_unique( + request_uri, + http_method, + callback, + timeouts, + credentials, + settings.max_http_get_redirects, + settings.max_read_buffer_size, + read_settings, + headers, + ReadWriteBufferFromHTTP::Range{}, + &context->getRemoteHostFilter(), + delay_initialization, + /* use_external_buffer */ false, + /* skip_url_not_found_error */ skip_url_not_found_error), + compression_method, + zstd_window_log_max); + } + catch (...) + { + if (first_exception_message.empty()) + first_exception_message = getCurrentExceptionMessage(false); + + if (options == 1) + throw; + + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message); +} + +class StorageURLSource::DisclosedGlobIterator::Impl +{ +public: + Impl(ContextPtr context_, const String & uri) + { + uris = getPathsList(uri, context_); + uris_iter = uris.begin(); + } + + String next() + { + std::lock_guard lock(mutex); + if (uris_iter != uris.end()) + { + auto answer = *uris_iter; + ++uris_iter; + return answer; + } + return {}; + } +private: + std::mutex mutex; + Strings uris; + Strings::iterator uris_iter; +}; + +StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(ContextPtr context_, const String & uri) + : pimpl(std::make_shared(context_, uri)) {} + +String StorageURLSource::DisclosedGlobIterator::next() +{ + return pimpl->next(); +} + +void StorageURLSource::onCancel() +{ + std::lock_guard lock(reader_mutex); + if (reader) + reader->cancel(); +} +Chunk StorageURLSource::generate() +{ + while (true) + { + if (!reader) + { + auto current_uri_pos = uri_info->next_uri_to_read.fetch_add(1); + if (current_uri_pos >= uri_info->uri_list_to_read.size()) + return {}; + + auto current_uri = uri_info->uri_list_to_read[current_uri_pos]; + + std::lock_guard lock(reader_mutex); + initialize(current_uri); } - private: - using InitializeFunc = std::function; - InitializeFunc initialize; + Chunk chunk; + std::lock_guard lock(reader_mutex); + if (reader->pull(chunk)) + return chunk; - String name; - URIInfoPtr uri_info; - - std::unique_ptr read_buf; - std::unique_ptr pipeline; - std::unique_ptr reader; - /// onCancell and generate can be called concurrently and both of them - /// have R/W access to reader pointer. - std::mutex reader_mutex; - - Poco::Net::HTTPBasicCredentials credentials; - }; + pipeline->reset(); + reader.reset(); + } } StorageURLSink::StorageURLSink( diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 6666e2a716e..f8075d8047e 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -200,7 +201,6 @@ public: static void processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection); }; - /// StorageURLWithFailover is allowed only for URL table function, not as a separate storage. class StorageURLWithFailover final : public StorageURL { @@ -234,4 +234,95 @@ public: private: std::vector uri_options; }; + +class PullingPipelineExecutor; + +class StorageURLSource : public ISource +{ + using URIParams = std::vector>; + +public: + + class DisclosedGlobIterator + { + public: + DisclosedGlobIterator(ContextPtr context_, const String & uri_); + String next(); + private: + class Impl; + /// shared_ptr to have copy constructor + std::shared_ptr pimpl; + }; + + struct URIInfo + { + using FailoverOptions = std::vector; + std::vector uri_list_to_read; + std::atomic next_uri_to_read = 0; + }; + + using IteratorWrapper = std::function; + using URIInfoPtr = std::shared_ptr; + + void onCancel() override; + + static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri); + + StorageURLSource( + URIInfoPtr uri_info_, + const std::string & http_method, + std::function callback, + const String & format, + const std::optional & format_settings, + String name_, + const Block & sample_block, + ContextPtr context, + const ColumnsDescription & columns, + UInt64 max_block_size, + const ConnectionTimeouts & timeouts, + CompressionMethod compression_method, + size_t download_threads, + const HTTPHeaderEntries & headers_ = {}, + const URIParams & params = {}, + bool glob_url = false); + + String getName() const override { return name; } + + Chunk generate() override; + + static std::unique_ptr getFirstAvailableURLReadBuffer( + std::vector::const_iterator & option, + const std::vector::const_iterator & end, + ContextPtr context, + const URIParams & params, + const String & http_method, + std::function callback, + const ConnectionTimeouts & timeouts, + CompressionMethod compression_method, + Poco::Net::HTTPBasicCredentials & credentials, + const HTTPHeaderEntries & headers, + bool glob_url, + bool delay_initialization, + size_t download_threads); + +private: + using InitializeFunc = std::function; + InitializeFunc initialize; + + String name; + URIInfoPtr uri_info; + + std::unique_ptr read_buf; + std::unique_ptr pipeline; + std::unique_ptr reader; + /// onCancell and generate can be called concurrently and both of them + /// have R/W access to reader pointer. + std::mutex reader_mutex; + + Poco::Net::HTTPBasicCredentials credentials; + +}; + + + } diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp new file mode 100644 index 00000000000..f133e62e367 --- /dev/null +++ b/src/Storages/StorageURLCluster.cpp @@ -0,0 +1,161 @@ +#include "config.h" +#include "Interpreters/Context_fwd.h" + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include + +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +StorageURLCluster::StorageURLCluster( + ContextPtr context_, + String cluster_name_, + const String & uri_, + const StorageID & table_id_, + const String & format_name_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & compression_method_, + const StorageURL::Configuration &configuration_) + : IStorageCluster(table_id_) + , cluster_name(cluster_name_) + , uri(uri_) + , format_name(format_name_) + , compression_method(compression_method_) +{ + context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); + + StorageInMemoryMetadata storage_metadata; + + if (columns_.empty()) + { + auto columns = StorageURL::getTableStructureFromData(format_name_, + uri_, + chooseCompressionMethod(Poco::URI(uri_).getPath(), compression_method), + configuration_.headers, + std::nullopt, + context_); + storage_metadata.setColumns(columns); + add_columns_structure_to_query = true; + } + else + storage_metadata.setColumns(columns_); + + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); +} + +/// The code executes on initiator +Pipe StorageURLCluster::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t /*max_block_size*/, + size_t /*num_streams*/) +{ + auto cluster = getCluster(context); + auto extension = getTaskIteratorExtension(query_info.query, context); + + /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) + Block header = + InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + + const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; + + Pipes pipes; + + const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; + + auto query_to_send = query_info.original_query->clone(); + if (add_columns_structure_to_query) + addColumnsStructureToQueryWithClusterEngine( + query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 3, getName()); + + const auto & current_settings = context->getSettingsRef(); + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); + for (const auto & shard_info : cluster->getShardsInfo()) + { + auto try_results = shard_info.pool->getMany(timeouts, ¤t_settings, PoolMode::GET_MANY); + for (auto & try_result : try_results) + { + auto remote_query_executor = std::make_shared( + shard_info.pool, + std::vector{try_result}, + queryToString(query_to_send), + header, + context, + /*throttler=*/nullptr, + scalars, + Tables(), + processed_stage, + extension); + + pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false)); + } + } + + storage_snapshot->check(column_names); + return Pipe::unitePipes(std::move(pipes)); +} + +QueryProcessingStage::Enum StorageURLCluster::getQueryProcessingStage( + ContextPtr context, QueryProcessingStage::Enum to_stage, const StorageSnapshotPtr &, SelectQueryInfo &) const +{ + /// Initiator executes query on remote node. + if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) + if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) + return QueryProcessingStage::Enum::WithMergeableState; + + /// Follower just reads the data. + return QueryProcessingStage::Enum::FetchColumns; +} + + +ClusterPtr StorageURLCluster::getCluster(ContextPtr context) const +{ + return context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); +} + +RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(ASTPtr, ContextPtr context) const +{ + auto iterator = std::make_shared(context, uri); + auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); + return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; +} + +NamesAndTypesList StorageURLCluster::getVirtuals() const +{ + return NamesAndTypesList{ + {"_path", std::make_shared(std::make_shared())}, + {"_file", std::make_shared(std::make_shared())}}; +} + + +} diff --git a/src/Storages/StorageURLCluster.h b/src/Storages/StorageURLCluster.h new file mode 100644 index 00000000000..f2266668ef6 --- /dev/null +++ b/src/Storages/StorageURLCluster.h @@ -0,0 +1,56 @@ +#pragma once + +#include "config.h" + +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +class Context; + +class StorageURLCluster : public IStorageCluster +{ +public: + StorageURLCluster( + ContextPtr context_, + String cluster_name_, + const String & uri_, + const StorageID & table_id_, + const String & format_name_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & compression_method_, + const StorageURL::Configuration &configuration_); + + std::string getName() const override { return "URLCluster"; } + + Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, + ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, size_t /*num_streams*/) override; + + QueryProcessingStage::Enum + getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; + + NamesAndTypesList getVirtuals() const override; + + ClusterPtr getCluster(ContextPtr context) const override; + RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; + +private: + String cluster_name; + String uri; + String format_name; + String compression_method; + bool add_columns_structure_to_query = false; + +}; + + +} + diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index a670bdc0682..5322a12040c 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -26,6 +26,8 @@ public: protected: void parseArguments(const ASTPtr & ast, ContextPtr context) override; + StorageURL::Configuration configuration; + private: StoragePtr getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, @@ -35,7 +37,7 @@ private: String getFormatFromFirstArgument() override; - StorageURL::Configuration configuration; + }; } diff --git a/src/TableFunctions/TableFunctionURLCluster.cpp b/src/TableFunctions/TableFunctionURLCluster.cpp new file mode 100644 index 00000000000..59a4288893b --- /dev/null +++ b/src/TableFunctions/TableFunctionURLCluster.cpp @@ -0,0 +1,119 @@ +#include + +#include "registerTableFunctions.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +void TableFunctionURLCluster::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + auto ast_copy = ast_function->clone(); + /// Parse args + ASTs & args_func = ast_copy->children; + + if (args_func.size() != 1) + throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + ASTs & args = args_func.at(0)->children; + + const auto message = fmt::format( + "The signature of table function {} shall be the following:\n" \ + " - cluster, uri\n",\ + " - cluster, uri, format\n",\ + " - cluster, uri, format, structure\n",\ + " - cluster, uri, format, structure, compression_method", + getName()); + + if (args.size() < 2 || args.size() > 5) + throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + /// This argument is always the first + cluster_name = checkAndGetLiteralArgument(args[0], "cluster_name"); + + if (!context->tryGetCluster(cluster_name)) + throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", cluster_name); + + /// Just cut the first arg (cluster_name) and try to parse other table function arguments as is + args.erase(args.begin()); + + TableFunctionURL::parseArguments(ast_copy, context); +} + + +ColumnsDescription TableFunctionURLCluster::getActualTableStructure(ContextPtr context) const +{ + if (structure == "auto") + { + context->checkAccess(getSourceAccessType()); + return StorageURL::getTableStructureFromData(format, + filename, + chooseCompressionMethod(Poco::URI(filename).getPath(), compression_method), + configuration.headers, + std::nullopt, + context); + } + + return parseColumnsListFromString(structure, context); +} + + +StoragePtr TableFunctionURLCluster::getStorage( + const String & /*source*/, const String & /*format_*/, const ColumnsDescription &, ContextPtr context, + const std::string & table_name, const String & /*compression_method_*/) const +{ + StoragePtr storage; + if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) + { + //On worker node this uri won't contains globs + storage = std::make_shared( + filename, + StorageID(getDatabaseName(), table_name), + format, + std::nullopt /*format settings*/, + getActualTableStructure(context), + ConstraintsDescription{}, + String{}, + context, + compression_method, + configuration.headers, + configuration.http_method); + } + else + { + storage = std::make_shared( + context, + cluster_name, filename, StorageID(getDatabaseName(), table_name), + format, getActualTableStructure(context), ConstraintsDescription{}, + compression_method, configuration); + } + return storage; +} + +void registerTableFunctionURLCluster(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/TableFunctions/TableFunctionURLCluster.h b/src/TableFunctions/TableFunctionURLCluster.h new file mode 100644 index 00000000000..3402c8f8167 --- /dev/null +++ b/src/TableFunctions/TableFunctionURLCluster.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +class Context; + +/** + * urlCluster(cluster, URI, format, structure, compression_method) + * A table function, which allows to process many files from url on a specific cluster + * On initiator it creates a connection to _all_ nodes in cluster, discloses asterics + * in url file path and dispatch each file dynamically. + * On worker node it asks initiator about next task to process, processes it. + * This is repeated until the tasks are finished. + */ +class TableFunctionURLCluster : public TableFunctionURL +{ +public: + static constexpr auto name = "urlCluster"; + std::string getName() const override + { + return name; + } + bool hasStaticStructure() const override { return true; } + +protected: + StoragePtr getStorage( + const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, + const std::string & table_name, const String & compression_method_) const override; + + const char * getStorageTypeName() const override { return "URLCluster"; } + + AccessType getSourceAccessType() const override { return AccessType::URL; } + + ColumnsDescription getActualTableStructure(ContextPtr) const override; + void parseArguments(const ASTPtr &, ContextPtr) override; + + String cluster_name; + +}; + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 95892d36b18..7613d9924b9 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -16,6 +16,7 @@ void registerTableFunctions() registerTableFunctionExecutable(factory); registerTableFunctionFile(factory); registerTableFunctionURL(factory); + registerTableFunctionURLCluster(factory); registerTableFunctionValues(factory); registerTableFunctionInput(factory); registerTableFunctionGenerate(factory); diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 5f91205474e..1e11739dede 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -13,6 +13,7 @@ void registerTableFunctionZeros(TableFunctionFactory & factory); void registerTableFunctionExecutable(TableFunctionFactory & factory); void registerTableFunctionFile(TableFunctionFactory & factory); void registerTableFunctionURL(TableFunctionFactory & factory); +void registerTableFunctionURLCluster(TableFunctionFactory & factory); void registerTableFunctionValues(TableFunctionFactory & factory); void registerTableFunctionInput(TableFunctionFactory & factory); void registerTableFunctionGenerate(TableFunctionFactory & factory); From 8caa2a0ee9bd14f30e5511397d75fa075d364dfd Mon Sep 17 00:00:00 2001 From: attack204 Date: Thu, 19 Jan 2023 12:28:13 +0800 Subject: [PATCH 0002/2223] update version --- contrib/azure | 2 +- contrib/poco | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/azure b/contrib/azure index ef75afc075f..ea8c3044f43 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit ef75afc075fc71fbcd8fe28dcda3794ae265fd1c +Subproject commit ea8c3044f43f5afa7016d2d580ed201f495d7e94 diff --git a/contrib/poco b/contrib/poco index 79923422618..7fefdf30244 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 799234226187c0ae0b8c90f23465b25ed7956e56 +Subproject commit 7fefdf30244a9bf8eb58562a9b2a51cc59a8877a From 7bd0010c5088d7b4cf6b7a2e1bdfbf8b5a527811 Mon Sep 17 00:00:00 2001 From: attack204 Date: Thu, 19 Jan 2023 13:18:07 +0800 Subject: [PATCH 0003/2223] fix conflict --- src/Storages/StorageURL.cpp | 162 +++++++++--------------------------- 1 file changed, 40 insertions(+), 122 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f2e339e2c7f..fd1b7bfc484 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -199,44 +199,44 @@ StorageURLSource::StorageURLSource( const URIParams & params, bool glob_url) : ISource(sample_block), name(std::move(name_)), uri_info(uri_info_) +{ + auto headers = getHeaders(headers_); + + /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. + initialize = [=, this](const URIInfo::FailoverOptions & uri_options) { - auto headers = getHeaders(headers_); + if (uri_options.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); - /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. - initialize = [=, this](const URIInfo::FailoverOptions & uri_options) - { - if (uri_options.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); + auto first_option = uri_options.begin(); + read_buf = getFirstAvailableURLReadBuffer( + first_option, + uri_options.end(), + context, + params, + http_method, + callback, + timeouts, + compression_method, + credentials, + headers, + glob_url, + uri_options.size() == 1, + download_threads); - auto first_option = uri_options.begin(); - read_buf = getFirstAvailableURLReadBuffer( - first_option, - uri_options.end(), - context, - params, - http_method, - callback, - timeouts, - compression_method, - credentials, - headers, - glob_url, - uri_options.size() == 1, - download_threads); + auto input_format + = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size, format_settings); + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); - auto input_format - = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size, format_settings); - QueryPipelineBuilder builder; - builder.init(Pipe(input_format)); + builder.addSimpleTransform( + [&](const Block & cur_header) + { return std::make_shared(cur_header, columns, *input_format, context); }); - builder.addSimpleTransform( - [&](const Block & cur_header) - { return std::make_shared(cur_header, columns, *input_format, context); }); - - pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - reader = std::make_unique(*pipeline); - }; - } + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + reader = std::make_unique(*pipeline); + }; +} std::unique_ptr StorageURLSource::getFirstAvailableURLReadBuffer( std::vector::const_iterator & option, @@ -297,92 +297,8 @@ std::unique_ptr StorageURLSource::getFirstAvailableURLReadBuffer( { try { - ReadWriteBufferFromHTTP buffer( - request_uri, - Poco::Net::HTTPRequest::HTTP_HEAD, - callback, - timeouts, - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, - ReadWriteBufferFromHTTP::Range{0, std::nullopt}, - &context->getRemoteHostFilter(), - true, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error); - - Poco::Net::HTTPResponse res; - - for (size_t i = 0; i < settings.http_max_tries; ++i) - { - try - { - buffer.callWithRedirects(res, Poco::Net::HTTPRequest::HTTP_HEAD, true); - break; - } - catch (const Poco::Exception & e) - { - LOG_TRACE( - &Poco::Logger::get("StorageURLSource"), - "HTTP HEAD request to `{}` failed at try {}/{}. " - "Error: {}.", - request_uri.toString(), - i + 1, - settings.http_max_tries, - e.displayText()); - if (!ReadWriteBufferFromHTTP::isRetriableError(res.getStatus())) - { - throw; - } - } - } - - // to check if Range header is supported, we need to send a request with it set - const bool supports_ranges = (res.has("Accept-Ranges") && res.get("Accept-Ranges") == "bytes") - || (res.has("Content-Range") && res.get("Content-Range").starts_with("bytes")); - - if (supports_ranges) - LOG_TRACE(&Poco::Logger::get("StorageURLSource"), "HTTP Range is supported"); - else - LOG_TRACE(&Poco::Logger::get("StorageURLSource"), "HTTP Range is not supported"); - - - if (supports_ranges && res.getStatus() == Poco::Net::HTTPResponse::HTTP_PARTIAL_CONTENT - && res.hasContentLength()) - { - LOG_TRACE( - &Poco::Logger::get("StorageURLSource"), - "Using ParallelReadBuffer with {} workers with chunks of {} bytes", - download_threads, - settings.max_download_buffer_size); - - auto read_buffer_factory = std::make_unique( - res.getContentLength(), - settings.max_download_buffer_size, - request_uri, - http_method, - callback, - timeouts, - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, - &context->getRemoteHostFilter(), - delay_initialization, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error); - - return wrapReadBufferWithCompressionMethod( - std::make_unique( - std::move(read_buffer_factory), - threadPoolCallbackRunner(IOThreadPool::get(), "URLParallelRead"), - download_threads), - compression_method, - zstd_window_log_max); - } + buffer.callWithRedirects(res, Poco::Net::HTTPRequest::HTTP_HEAD, true); + break; } catch (const Poco::Exception & e) { @@ -404,9 +320,11 @@ std::unique_ptr StorageURLSource::getFirstAvailableURLReadBuffer( // to check if Range header is supported, we need to send a request with it set const bool supports_ranges = (res.has("Accept-Ranges") && res.get("Accept-Ranges") == "bytes") || (res.has("Content-Range") && res.get("Content-Range").starts_with("bytes")); - LOG_TRACE( - &Poco::Logger::get("StorageURLSource"), - fmt::runtime(supports_ranges ? "HTTP Range is supported" : "HTTP Range is not supported")); + + if (supports_ranges) + LOG_TRACE(&Poco::Logger::get("StorageURLSource"), "HTTP Range is supported"); + else + LOG_TRACE(&Poco::Logger::get("StorageURLSource"), "HTTP Range is not supported"); if (supports_ranges && res.getStatus() == Poco::Net::HTTPResponse::HTTP_PARTIAL_CONTENT From 0844384544abdd384ab7b1b82766e6aefb77c5bc Mon Sep 17 00:00:00 2001 From: attack204 Date: Thu, 19 Jan 2023 13:25:12 +0800 Subject: [PATCH 0004/2223] fix version conflict --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 7fefdf30244..0ab9bba7cca 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 7fefdf30244a9bf8eb58562a9b2a51cc59a8877a +Subproject commit 0ab9bba7ccad3c8dacce04a35cb3b78218547ab4 From 6eee6bcd868465a0377ebd4678bd57be9467dcd5 Mon Sep 17 00:00:00 2001 From: attack204 Date: Thu, 19 Jan 2023 13:51:32 +0800 Subject: [PATCH 0005/2223] doc:add urlCluster --- .../table-functions/urlCluster.md | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) create mode 100644 docs/en/sql-reference/table-functions/urlCluster.md diff --git a/docs/en/sql-reference/table-functions/urlCluster.md b/docs/en/sql-reference/table-functions/urlCluster.md new file mode 100644 index 00000000000..1fd35890a8b --- /dev/null +++ b/docs/en/sql-reference/table-functions/urlCluster.md @@ -0,0 +1,44 @@ +--- +slug: /en/sql-reference/table-functions/urlCluster +sidebar_position: 55 +sidebar_label: urlCluster +--- + +# urlCluster Table Function + +Allows processing files from URL in parallel from many nodes in a specified cluster. On initiator it creates a connection to all nodes in the cluster, discloses asterics in URL file path, and dispatches each file dynamically. On the worker node it asks the initiator about the next task to process and processes it. This is repeated until all tasks are finished. + +**Syntax** + +``` sql +urlCluster(cluster_name, URL, format, structure) +``` + +**Arguments** + +- `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. +- `URL` — HTTP or HTTPS server address, which can accept `GET` requests. Type: [String](../../sql-reference/data-types/string.md). +- `format` — [Format](../../interfaces/formats.md#formats) of the data. Type: [String](../../sql-reference/data-types/string.md). +- `structure` — Table structure in `'UserID UInt64, Name String'` format. Determines column names and types. Type: [String](../../sql-reference/data-types/string.md). + +**Returned value** + +A table with the specified format and structure and with data from the defined `URL`. + +**Examples** + +Getting the first 3 lines of a table that contains columns of `String` and [UInt32](../../sql-reference/data-types/int-uint.md) type from HTTP-server which answers in [CSV](../../interfaces/formats.md#csv) format. + +``` sql +SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3; +``` + +## Globs in URL + +Patterns in curly brackets `{ }` are used to generate a set of shards or to specify failover addresses. Supported pattern types and examples see in the description of the [remote](remote.md#globs-in-addresses) function. +Character `|` inside patterns is used to specify failover addresses. They are iterated in the same order as listed in the pattern. The number of generated addresses is limited by [glob_expansion_max_elements](../../operations/settings/settings.md#glob_expansion_max_elements) setting. + +**See Also** + +- [HDFS engine](../../engines/table-engines/special/url.md) +- [URL table function](../../sql-reference/table-functions/url.md) \ No newline at end of file From ef0bed7705b173d355b6a98effcde654746970bc Mon Sep 17 00:00:00 2001 From: attack204 Date: Thu, 19 Jan 2023 14:19:47 +0800 Subject: [PATCH 0006/2223] add test --- .../table-functions/urlCluster.md | 20 ++++++++++++++++++- tests/integration/test_storage_url/test.py | 13 ++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/urlCluster.md b/docs/en/sql-reference/table-functions/urlCluster.md index 1fd35890a8b..8f19632c433 100644 --- a/docs/en/sql-reference/table-functions/urlCluster.md +++ b/docs/en/sql-reference/table-functions/urlCluster.md @@ -29,8 +29,26 @@ A table with the specified format and structure and with data from the defined ` Getting the first 3 lines of a table that contains columns of `String` and [UInt32](../../sql-reference/data-types/int-uint.md) type from HTTP-server which answers in [CSV](../../interfaces/formats.md#csv) format. +1. Create a basic HTTP server using the standard Python 3 tools and start it: + +```python +from http.server import BaseHTTPRequestHandler, HTTPServer + +class CSVHTTPServer(BaseHTTPRequestHandler): + def do_GET(self): + self.send_response(200) + self.send_header('Content-type', 'text/csv') + self.end_headers() + + self.wfile.write(bytes('Hello,1\nWorld,2\n', "utf-8")) + +if __name__ == "__main__": + server_address = ('127.0.0.1', 12345) + HTTPServer(server_address, CSVHTTPServer).serve_forever() +``` + ``` sql -SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3; +SELECT * FROM urlCluster('cluster_simple','http://127.0.0.1:12345', CSV, 'column1 String, column2 UInt32') ``` ## Globs in URL diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index 5591e63400c..dbb4abe277c 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -34,6 +34,19 @@ def test_partition_by(): ) assert result.strip() == "1\t2\t3" +def test_urlCluster(): + result = node1.query( + f"select * from urlCluster('test_cluster_two_shards', 'http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')" + ) + assert result.strip() == "3\t2\t1" + result = node1.query( + f"select * from urlCluster('test_cluster_two_shards', 'http://nginx:80/test_2', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')" + ) + assert result.strip() == "1\t3\t2" + result = node1.query( + f"select * from urlCluster('test_cluster_two_shards', 'http://nginx:80/test_3', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')" + ) + assert result.strip() == "1\t2\t3" def test_table_function_url_access_rights(): node1.query("CREATE USER OR REPLACE u1") From 99bdef71c826f9fa2e68bf921fa38875c5498ef9 Mon Sep 17 00:00:00 2001 From: attack204 Date: Thu, 19 Jan 2023 15:12:47 +0800 Subject: [PATCH 0007/2223] fix:style --- src/TableFunctions/TableFunctionURLCluster.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionURLCluster.cpp b/src/TableFunctions/TableFunctionURLCluster.cpp index 59a4288893b..f1d5954f47c 100644 --- a/src/TableFunctions/TableFunctionURLCluster.cpp +++ b/src/TableFunctions/TableFunctionURLCluster.cpp @@ -20,8 +20,8 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_GET; } void TableFunctionURLCluster::parseArguments(const ASTPtr & ast_function, ContextPtr context) From f549380867848fcc7a87441ecc59315cd9d2d68e Mon Sep 17 00:00:00 2001 From: attack204 Date: Thu, 19 Jan 2023 16:10:59 +0800 Subject: [PATCH 0008/2223] fix:style --- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageURL.h | 2 -- src/Storages/StorageURLCluster.cpp | 2 -- src/TableFunctions/TableFunctionURL.h | 1 - 4 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index fd1b7bfc484..9a8c70344a0 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -448,6 +448,7 @@ void StorageURLSource::onCancel() if (reader) reader->cancel(); } + Chunk StorageURLSource::generate() { while (true) @@ -495,7 +496,6 @@ StorageURLSink::StorageURLSink( writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, format_settings); } - void StorageURLSink::consume(Chunk chunk) { std::lock_guard lock(cancel_mutex); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index f8075d8047e..acee582fbc5 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -323,6 +323,4 @@ private: }; - - } diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index f133e62e367..9e088ab69dc 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -137,7 +137,6 @@ QueryProcessingStage::Enum StorageURLCluster::getQueryProcessingStage( return QueryProcessingStage::Enum::FetchColumns; } - ClusterPtr StorageURLCluster::getCluster(ContextPtr context) const { return context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); @@ -157,5 +156,4 @@ NamesAndTypesList StorageURLCluster::getVirtuals() const {"_file", std::make_shared(std::make_shared())}}; } - } diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index 5322a12040c..a444fb0c3d7 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -37,7 +37,6 @@ private: String getFormatFromFirstArgument() override; - }; } From 1f4139718ab5b4f00b8623fad241c25310d724bf Mon Sep 17 00:00:00 2001 From: attack204 Date: Thu, 19 Jan 2023 16:19:39 +0800 Subject: [PATCH 0009/2223] fix:style --- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageURL.h | 2 +- src/Storages/StorageURLCluster.cpp | 2 +- src/TableFunctions/TableFunctionURL.h | 2 +- src/TableFunctions/TableFunctionURLCluster.h | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9a8c70344a0..f1ba7b63e7b 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -449,7 +449,7 @@ void StorageURLSource::onCancel() reader->cancel(); } -Chunk StorageURLSource::generate() +Chunk StorageURLSource::generate() { while (true) { diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index acee582fbc5..c35d3aebdc7 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -304,7 +304,7 @@ public: bool glob_url, bool delay_initialization, size_t download_threads); - + private: using InitializeFunc = std::function; InitializeFunc initialize; diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 9e088ab69dc..7ff786bf5cd 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -49,7 +49,7 @@ StorageURLCluster::StorageURLCluster( , compression_method(compression_method_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); - + StorageInMemoryMetadata storage_metadata; if (columns_.empty()) diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index a444fb0c3d7..c0aae1666f1 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -27,7 +27,7 @@ protected: void parseArguments(const ASTPtr & ast, ContextPtr context) override; StorageURL::Configuration configuration; - + private: StoragePtr getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, diff --git a/src/TableFunctions/TableFunctionURLCluster.h b/src/TableFunctions/TableFunctionURLCluster.h index 3402c8f8167..7deda443b05 100644 --- a/src/TableFunctions/TableFunctionURLCluster.h +++ b/src/TableFunctions/TableFunctionURLCluster.h @@ -43,7 +43,7 @@ protected: void parseArguments(const ASTPtr &, ContextPtr) override; String cluster_name; - + }; } From 5d387f27f8928266d5e5900a11bf881c83aa8b58 Mon Sep 17 00:00:00 2001 From: attack204 Date: Thu, 19 Jan 2023 16:49:44 +0800 Subject: [PATCH 0010/2223] fix:style --- tests/integration/test_storage_url/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index dbb4abe277c..96772613ded 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -34,6 +34,7 @@ def test_partition_by(): ) assert result.strip() == "1\t2\t3" + def test_urlCluster(): result = node1.query( f"select * from urlCluster('test_cluster_two_shards', 'http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')" @@ -48,6 +49,7 @@ def test_urlCluster(): ) assert result.strip() == "1\t2\t3" + def test_table_function_url_access_rights(): node1.query("CREATE USER OR REPLACE u1") From 0c92c9917e87804035be1f104abddf87f2b2b231 Mon Sep 17 00:00:00 2001 From: attack204 Date: Sat, 21 Jan 2023 09:33:37 +0800 Subject: [PATCH 0011/2223] Fix schema inference from insertion table in urlCluster --- src/Storages/StorageURLCluster.cpp | 7 ++++--- src/Storages/StorageURLCluster.h | 5 +++-- src/TableFunctions/TableFunctionURLCluster.cpp | 6 +++--- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 7ff786bf5cd..116a08ce3a3 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -41,12 +41,14 @@ StorageURLCluster::StorageURLCluster( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & compression_method_, - const StorageURL::Configuration &configuration_) + const StorageURL::Configuration &configuration_, + bool structure_argument_was_provided_) : IStorageCluster(table_id_) , cluster_name(cluster_name_) , uri(uri_) , format_name(format_name_) , compression_method(compression_method_) + , structure_argument_was_provided(structure_argument_was_provided_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); @@ -61,7 +63,6 @@ StorageURLCluster::StorageURLCluster( std::nullopt, context_); storage_metadata.setColumns(columns); - add_columns_structure_to_query = true; } else storage_metadata.setColumns(columns_); @@ -94,7 +95,7 @@ Pipe StorageURLCluster::read( const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; auto query_to_send = query_info.original_query->clone(); - if (add_columns_structure_to_query) + if (!structure_argument_was_provided) addColumnsStructureToQueryWithClusterEngine( query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 3, getName()); diff --git a/src/Storages/StorageURLCluster.h b/src/Storages/StorageURLCluster.h index f2266668ef6..b90af8df5b8 100644 --- a/src/Storages/StorageURLCluster.h +++ b/src/Storages/StorageURLCluster.h @@ -27,7 +27,8 @@ public: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & compression_method_, - const StorageURL::Configuration &configuration_); + const StorageURL::Configuration &configuration_, + bool structure_argument_was_provided_); std::string getName() const override { return "URLCluster"; } @@ -47,7 +48,7 @@ private: String uri; String format_name; String compression_method; - bool add_columns_structure_to_query = false; + bool structure_argument_was_provided; }; diff --git a/src/TableFunctions/TableFunctionURLCluster.cpp b/src/TableFunctions/TableFunctionURLCluster.cpp index f1d5954f47c..00aadab5e01 100644 --- a/src/TableFunctions/TableFunctionURLCluster.cpp +++ b/src/TableFunctions/TableFunctionURLCluster.cpp @@ -80,7 +80,7 @@ ColumnsDescription TableFunctionURLCluster::getActualTableStructure(ContextPtr c StoragePtr TableFunctionURLCluster::getStorage( - const String & /*source*/, const String & /*format_*/, const ColumnsDescription &, ContextPtr context, + const String & /*source*/, const String & /*format_*/, const ColumnsDescription & columns, ContextPtr context, const std::string & table_name, const String & /*compression_method_*/) const { StoragePtr storage; @@ -92,7 +92,7 @@ StoragePtr TableFunctionURLCluster::getStorage( StorageID(getDatabaseName(), table_name), format, std::nullopt /*format settings*/, - getActualTableStructure(context), + columns, ConstraintsDescription{}, String{}, context, @@ -106,7 +106,7 @@ StoragePtr TableFunctionURLCluster::getStorage( context, cluster_name, filename, StorageID(getDatabaseName(), table_name), format, getActualTableStructure(context), ConstraintsDescription{}, - compression_method, configuration); + compression_method, configuration, structure != "auto"); } return storage; } From 6e7ed238ca48f5ea8290c0fc48224ab67bfab7b7 Mon Sep 17 00:00:00 2001 From: attack204 Date: Sat, 21 Jan 2023 10:48:15 +0800 Subject: [PATCH 0012/2223] add urlCluster to undocumented.sql --- .../02414_all_new_table_functions_must_be_documented.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql index a73993f6a5a..ef339b760aa 100644 --- a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql @@ -2,5 +2,5 @@ -- Please help shorten this list down to zero elements. SELECT name FROM system.table_functions WHERE length(description) < 10 AND name NOT IN ( - 'cosn', 'oss', 'hdfs', 'hdfsCluster', 'hive', 'mysql', 'postgresql', 's3', 's3Cluster', 'sqlite' -- these functions are not enabled in fast test + 'cosn', 'oss', 'hdfs', 'hdfsCluster', 'hive', 'mysql', 'postgresql', 's3', 's3Cluster', 'sqlite', 'urlCluster' -- these functions are not enabled in fast test ) ORDER BY name; From be37730906230fcc4255a9802231cc4d3fe11679 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 27 Jan 2023 16:23:27 +0300 Subject: [PATCH 0013/2223] upd --- src/Common/DateLUT.cpp | 10 +++++++++- src/Common/DateLUT.h | 24 ++++++++++++++++++++++-- src/Core/Settings.h | 1 + 3 files changed, 32 insertions(+), 3 deletions(-) diff --git a/src/Common/DateLUT.cpp b/src/Common/DateLUT.cpp index ae6127670e5..fd10b799b66 100644 --- a/src/Common/DateLUT.cpp +++ b/src/Common/DateLUT.cpp @@ -7,6 +7,8 @@ #include #include +#include +#include namespace @@ -147,7 +149,7 @@ DateLUT::DateLUT() } -const DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const +const ALWAYS_INLINE DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const { std::lock_guard lock(mutex); @@ -163,3 +165,9 @@ DateLUT & DateLUT::getInstance() static DateLUT ret; return ret; } + +std::string DateLUT::extractTimezoneFromContext(const DB::ContextPtr query_context) +{ + std::string ret = query_context->getSettingsRef().implicit_timezone.value; + return ret; +} diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index b7ba37c2bec..bd7969bffa6 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -5,6 +5,10 @@ #include #include +// +//#include "Interpreters/Context_fwd.h" +//#include "Interpreters/Context.h" +#include "Common/CurrentThread.h" #include #include @@ -20,16 +24,30 @@ public: static ALWAYS_INLINE const DateLUTImpl & instance() // -V1071 { const auto & date_lut = getInstance(); + + if (DB::CurrentThread::isInitialized()) + { + const auto query_context = DB::CurrentThread::get().getQueryContext(); + + if (query_context) + { + auto implicit_timezone = extractTimezoneFromContext(query_context); + + if (!implicit_timezone.empty()) + return instance(implicit_timezone); + } + } + return *date_lut.default_impl.load(std::memory_order_acquire); } /// Return singleton DateLUTImpl instance for a given time zone. static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { - const auto & date_lut = getInstance(); if (time_zone.empty()) - return *date_lut.default_impl.load(std::memory_order_acquire); + return instance(); + const auto & date_lut = getInstance(); return date_lut.getImplementation(time_zone); } static void setDefaultTimezone(const std::string & time_zone) @@ -45,6 +63,8 @@ protected: private: static DateLUT & getInstance(); + static std::string extractTimezoneFromContext(const DB::ContextPtr query_context); + const DateLUTImpl & getImplementation(const std::string & time_zone) const; using DateLUTImplPtr = std::unique_ptr; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1948a6da012..2da5791ff81 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -688,6 +688,7 @@ class IColumn; M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ M(Bool, force_aggregation_in_order, false, "Force use of aggregation in order on remote nodes during distributed aggregation. PLEASE, NEVER CHANGE THIS SETTING VALUE MANUALLY!", IMPORTANT) \ + M(String, implicit_timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. From 537721e297e4ba94f30d5bd76ca0a7b01a080a5c Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 27 Jan 2023 23:40:43 +0300 Subject: [PATCH 0014/2223] re-add tests --- .../0_stateless/02538_implicit_timezone.reference | 3 +++ tests/queries/0_stateless/02538_implicit_timezone.sql | 9 +++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/02538_implicit_timezone.reference create mode 100644 tests/queries/0_stateless/02538_implicit_timezone.sql diff --git a/tests/queries/0_stateless/02538_implicit_timezone.reference b/tests/queries/0_stateless/02538_implicit_timezone.reference new file mode 100644 index 00000000000..8ed8024f652 --- /dev/null +++ b/tests/queries/0_stateless/02538_implicit_timezone.reference @@ -0,0 +1,3 @@ +1999-12-12 18:23:23.123 +1999-12-12 23:23:23.123 +1999-12-13 04:23:23.123 diff --git a/tests/queries/0_stateless/02538_implicit_timezone.sql b/tests/queries/0_stateless/02538_implicit_timezone.sql new file mode 100644 index 00000000000..663b218d235 --- /dev/null +++ b/tests/queries/0_stateless/02538_implicit_timezone.sql @@ -0,0 +1,9 @@ +SET implicit_timezone = 'Asia/Novosibirsk'; + +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); + +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'Europe/Zurich'; + +SET implicit_timezone = 'Europe/Zurich'; + +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); \ No newline at end of file From f29700bd2fe1fc79cb6fd5bf2b31129119e6978c Mon Sep 17 00:00:00 2001 From: LiuYangkuan Date: Wed, 1 Feb 2023 18:50:43 +0800 Subject: [PATCH 0015/2223] use IDisk to do IO in Keeper's snapshots and logs --- programs/keeper-converter/KeeperConverter.cpp | 3 +- src/Coordination/Changelog.cpp | 151 +++++++++++------- src/Coordination/Changelog.h | 6 +- src/Coordination/KeeperLogStore.cpp | 11 +- src/Coordination/KeeperLogStore.h | 3 + src/Coordination/KeeperServer.cpp | 23 ++- src/Coordination/KeeperSnapshotManager.cpp | 91 ++++++----- src/Coordination/KeeperSnapshotManager.h | 24 ++- src/Coordination/KeeperStateMachine.cpp | 4 +- src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStateManager.cpp | 71 ++++---- src/Coordination/KeeperStateManager.h | 8 +- src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 2 +- src/IO/ZstdDeflatingAppendableWriteBuffer.h | 4 +- 14 files changed, 244 insertions(+), 159 deletions(-) diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index 7d25c1d5017..c81e61685fd 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -9,6 +9,7 @@ #include #include #include +#include int mainEntryClickHouseKeeperConverter(int argc, char ** argv) @@ -51,7 +52,7 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(storage.getZXID(), 1, std::make_shared()); DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); - DB::KeeperSnapshotManager manager(options["output-dir"].as(), 1, keeper_context); + DB::KeeperSnapshotManager manager(std::make_shared("Keeper-snapshots", options["output-dir"].as(), 0), 1, keeper_context); auto snp = manager.serializeSnapshotToBuffer(snapshot); auto path = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); std::cout << "Snapshot serialized to path:" << path << std::endl; diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 142a7209b42..2d3fb9cba47 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -31,12 +31,9 @@ namespace constexpr auto DEFAULT_PREFIX = "changelog"; -std::string formatChangelogPath( - const std::string & prefix, const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) +inline std::string formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) { - std::filesystem::path path(prefix); - path /= std::filesystem::path(fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension)); - return path; + return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension); } ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path) @@ -88,11 +85,11 @@ class ChangelogWriter public: ChangelogWriter( std::map & existing_changelogs_, - const std::filesystem::path & changelogs_dir_, + DiskPtr disk_, LogFileSettings log_file_settings_) : existing_changelogs(existing_changelogs_) , log_file_settings(log_file_settings_) - , changelogs_dir(changelogs_dir_) + , disk(disk_) , log(&Poco::Logger::get("Changelog")) { } @@ -109,7 +106,7 @@ public: file_description->expectedEntriesCountInLog()); // we have a file we need to finalize first - if (tryGetFileBuffer() && prealloc_done) + if (tryGetFileBaseBuffer() && prealloc_done) { finalizeCurrentFile(); @@ -121,18 +118,16 @@ public: && *last_index_written != current_file_description->to_log_index) { auto new_path = formatChangelogPath( - changelogs_dir, current_file_description->prefix, current_file_description->from_log_index, *last_index_written, current_file_description->extension); - std::filesystem::rename(current_file_description->path, new_path); + disk->moveFile(current_file_description->path, new_path); current_file_description->path = std::move(new_path); } } - file_buf = std::make_unique( - file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode == WriteMode::Rewrite ? -1 : (O_APPEND | O_CREAT | O_WRONLY)); + file_buf = disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode); last_index_written.reset(); current_file_description = std::move(file_description); @@ -148,12 +143,15 @@ public: } } - bool isFileSet() const { return tryGetFileBuffer() != nullptr; } - + /// There is bug when compressed_buffer has value, file_buf's ownership transfer to compressed_buffer + bool isFileSet() const + { + return compressed_buffer.get() != nullptr || file_buf.get() != nullptr; + } bool appendRecord(ChangelogRecord && record) { - const auto * file_buffer = tryGetFileBuffer(); + const auto * file_buffer = tryGetFileBaseBuffer(); assert(file_buffer && current_file_description); assert(record.header.index - getStartIndex() <= current_file_description->expectedEntriesCountInLog()); @@ -207,7 +205,7 @@ public: void flush() { - auto * file_buffer = tryGetFileBuffer(); + auto * file_buffer = tryGetFileBaseBuffer(); /// Fsync file system if needed if (file_buffer && log_file_settings.force_sync) file_buffer->sync(); @@ -232,7 +230,6 @@ public: new_description->extension += "." + toContentEncodingName(CompressionMethod::Zstd); new_description->path = formatChangelogPath( - changelogs_dir, new_description->prefix, new_start_log_index, new_start_log_index + log_file_settings.rotate_interval - 1, @@ -254,14 +251,13 @@ private: void finalizeCurrentFile() { - const auto * file_buffer = tryGetFileBuffer(); - assert(file_buffer && prealloc_done); + assert(prealloc_done); assert(current_file_description); // compact can delete the file and we don't need to do anything if (current_file_description->deleted) { - LOG_WARNING(log, "Log {} is already deleted", file_buffer->getFileName()); + LOG_WARNING(log, "Log {} is already deleted", current_file_description->path); return; } @@ -270,7 +266,8 @@ private: flush(); - if (log_file_settings.max_size != 0) + const auto * file_buffer = tryGetFileBuffer(); + if (log_file_settings.max_size != 0 && file_buffer) ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); if (log_file_settings.compress_logs) @@ -281,6 +278,8 @@ private: WriteBuffer & getBuffer() { + /// TODO: unify compressed_buffer and file_buf, + /// compressed_buffer can use its NestedBuffer directly if compress_logs=false if (compressed_buffer) return *compressed_buffer; @@ -310,10 +309,15 @@ private: if (compressed_buffer) return dynamic_cast(compressed_buffer->getNestedBuffer()); - if (file_buf) - return file_buf.get(); + return dynamic_cast(file_buf.get()); + } - return nullptr; + WriteBufferFromFileBase * tryGetFileBaseBuffer() + { + if (compressed_buffer) + return dynamic_cast(compressed_buffer->getNestedBuffer()); + + return file_buf.get(); } void tryPreallocateForFile() @@ -325,13 +329,22 @@ private: return; } - const auto & file_buffer = getFileBuffer(); + const auto * file_buffer = tryGetFileBuffer(); + + if (!file_buffer) + { + initial_file_size = 0; + prealloc_done = true; + LOG_WARNING(log, "Could not preallocate space on disk {} using fallocate", disk->getName()); + return; + } + #ifdef OS_LINUX { int res = -1; do { - res = fallocate(file_buffer.getFD(), FALLOC_FL_KEEP_SIZE, 0, log_file_settings.max_size + log_file_settings.overallocate_size); + res = fallocate(file_buffer->getFD(), FALLOC_FL_KEEP_SIZE, 0, log_file_settings.max_size + log_file_settings.overallocate_size); } while (res < 0 && errno == EINTR); if (res != 0) @@ -346,7 +359,7 @@ private: } } #endif - initial_file_size = getSizeFromFileDescriptor(file_buffer.getFD()); + initial_file_size = getSizeFromFileDescriptor(file_buffer->getFD()); prealloc_done = true; } @@ -354,7 +367,7 @@ private: std::map & existing_changelogs; ChangelogFileDescriptionPtr current_file_description{nullptr}; - std::unique_ptr file_buf; + std::unique_ptr file_buf; std::optional last_index_written; size_t initial_file_size{0}; @@ -364,7 +377,7 @@ private: LogFileSettings log_file_settings; - const std::filesystem::path changelogs_dir; + DiskPtr disk; Poco::Logger * const log; }; @@ -394,10 +407,12 @@ struct ChangelogReadResult class ChangelogReader { public: - explicit ChangelogReader(const std::string & filepath_) : filepath(filepath_) + explicit ChangelogReader(DiskPtr disk_, const std::string & filepath_) + : disk(disk_) + , filepath(filepath_) { auto compression_method = chooseCompressionMethod(filepath, ""); - auto read_buffer_from_file = std::make_unique(filepath); + auto read_buffer_from_file = disk->readFile(filepath); read_buf = wrapReadBufferWithCompressionMethod(std::move(read_buffer_from_file), compression_method); } @@ -493,37 +508,35 @@ public: } private: + DiskPtr disk; std::string filepath; std::unique_ptr read_buf; }; Changelog::Changelog( - const std::string & changelogs_dir_, + DiskPtr disk_, Poco::Logger * log_, LogFileSettings log_file_settings) - : changelogs_dir(changelogs_dir_) - , changelogs_detached_dir(changelogs_dir / "detached") + : disk(disk_) + , changelogs_detached_dir("detached") , rotate_interval(log_file_settings.rotate_interval) , log(log_) , write_operations(std::numeric_limits::max()) , append_completion_queue(std::numeric_limits::max()) { /// Load all files in changelog directory - namespace fs = std::filesystem; - if (!fs::exists(changelogs_dir)) - fs::create_directories(changelogs_dir); - for (const auto & p : fs::directory_iterator(changelogs_dir)) + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { - if (p == changelogs_detached_dir) + if (it->name() == changelogs_detached_dir) continue; - auto file_description = getChangelogFileDescription(p.path()); + auto file_description = getChangelogFileDescription(it->path()); existing_changelogs[file_description->from_log_index] = std::move(file_description); } if (existing_changelogs.empty()) - LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", changelogs_dir.generic_string()); + LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", disk->getPath()); clean_log_thread = ThreadFromGlobalPool([this] { cleanLogThread(); }); @@ -532,7 +545,7 @@ Changelog::Changelog( append_completion_thread = ThreadFromGlobalPool([this] { appendCompletionThread(); }); current_writer = std::make_unique( - existing_changelogs, changelogs_dir, log_file_settings); + existing_changelogs, disk, log_file_settings); } void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) @@ -604,7 +617,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin break; } - ChangelogReader reader(changelog_description.path); + ChangelogReader reader(disk, changelog_description.path); last_log_read_result = reader.readChangelog(logs, start_to_read_from, log); last_log_read_result->log_start_index = changelog_description.from_log_index; @@ -671,7 +684,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin if (last_log_read_result->last_read_index == 0 || last_log_read_result->error) /// If it's broken log then remove it { LOG_INFO(log, "Removing chagelog {} because it's empty or read finished with error", description->path); - std::filesystem::remove(description->path); + disk->removeFile(description->path); existing_changelogs.erase(last_log_read_result->log_start_index); std::erase_if(logs, [last_log_read_result](const auto & item) { return item.first >= last_log_read_result->log_start_index; }); } @@ -691,6 +704,9 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin void Changelog::initWriter(ChangelogFileDescriptionPtr description) { + if (description->expectedEntriesCountInLog() != rotate_interval) + LOG_TRACE(log, "Looks like rotate_logs_interval was changed, current {}, expected entries in last log {}", rotate_interval, description->expectedEntriesCountInLog()); + LOG_TRACE(log, "Continue to write into {}", description->path); current_writer->setFile(std::move(description), WriteMode::Append); } @@ -715,20 +731,20 @@ std::string getCurrentTimestampFolder() void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end) { - const auto timestamp_folder = changelogs_detached_dir / getCurrentTimestampFolder(); + const auto timestamp_folder = (fs::path(changelogs_detached_dir) / getCurrentTimestampFolder()).generic_string(); for (auto itr = begin; itr != end;) { - if (!std::filesystem::exists(timestamp_folder)) + if (!disk->exists(timestamp_folder)) { - LOG_WARNING(log, "Moving broken logs to {}", timestamp_folder.generic_string()); - std::filesystem::create_directories(timestamp_folder); + LOG_WARNING(log, "Moving broken logs to {}", timestamp_folder); + disk->createDirectories(timestamp_folder); } LOG_WARNING(log, "Removing changelog {}", itr->second->path); const std::filesystem::path & path = itr->second->path; const auto new_path = timestamp_folder / path.filename(); - std::filesystem::rename(path, new_path); + disk->moveFile(path.generic_string(), new_path.generic_string()); itr = existing_changelogs.erase(itr); } } @@ -885,7 +901,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) auto to_remove_itr = existing_changelogs.upper_bound(index); for (auto itr = to_remove_itr; itr != existing_changelogs.end();) { - std::filesystem::remove(itr->second->path); + disk->removeFile(itr->second->path); itr = existing_changelogs.erase(itr); } } @@ -937,12 +953,19 @@ void Changelog::compact(uint64_t up_to_log_index) /// If failed to push to queue for background removing, then we will remove it now if (!log_files_to_delete_queue.tryPush(changelog_description.path, 1)) { - std::error_code ec; - std::filesystem::remove(changelog_description.path, ec); - if (ec) - LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", changelog_description.path, ec.message()); - else - LOG_INFO(log, "Removed changelog {} because of compaction", changelog_description.path); + try + { + disk->removeFile(itr->second->path); + LOG_INFO(log, "Removed changelog {} because of compaction.", itr->second->path); + } + catch (Exception & e) + { + LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", itr->second->path, e.message()); + } + catch (...) + { + tryLogCurrentException(log); + } } changelog_description.deleted = true; @@ -1135,11 +1158,19 @@ void Changelog::cleanLogThread() std::string path; while (log_files_to_delete_queue.pop(path)) { - std::error_code ec; - if (std::filesystem::remove(path, ec)) + try + { + disk->removeFile(path); LOG_INFO(log, "Removed changelog {} because of compaction.", path); - else - LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", path, ec.message()); + } + catch (Exception & e) + { + LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", path, e.message()); + } + catch (...) + { + tryLogCurrentException(log); + } } } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 288f71bb915..052eba45858 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -86,7 +86,7 @@ class Changelog { public: Changelog( - const std::string & changelogs_dir_, + DiskPtr disk_, Poco::Logger * log_, LogFileSettings log_file_settings); @@ -168,8 +168,8 @@ private: /// Clean useless log files in a background thread void cleanLogThread(); - const std::filesystem::path changelogs_dir; - const std::filesystem::path changelogs_detached_dir; + DiskPtr disk; + const String changelogs_detached_dir; const uint64_t rotate_interval; Poco::Logger * log; diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index d1bd2f9db18..fcf9400a1d9 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -1,13 +1,13 @@ #include #include +#include namespace DB { -KeeperLogStore::KeeperLogStore( - const std::string & changelogs_path, LogFileSettings log_file_settings) +KeeperLogStore::KeeperLogStore(DiskPtr disk_, LogFileSettings log_file_settings) : log(&Poco::Logger::get("KeeperLogStore")) - , changelog(changelogs_path, log, log_file_settings) + , changelog(disk_, log, log_file_settings) { if (log_file_settings.force_sync) LOG_INFO(log, "force_sync enabled"); @@ -15,6 +15,11 @@ KeeperLogStore::KeeperLogStore( LOG_INFO(log, "force_sync disabled"); } +KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, LogFileSettings log_file_settings) + : KeeperLogStore(std::make_shared("Keeper-logs", changelogs_path, 0), log_file_settings) +{ +} + uint64_t KeeperLogStore::start_index() const { std::lock_guard lock(changelog_lock); diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 108241e024e..c2428413753 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -14,6 +14,9 @@ namespace DB class KeeperLogStore : public nuraft::log_store { public: + KeeperLogStore(DiskPtr disk_, LogFileSettings log_file_settings); + + /// For gtest KeeperLogStore(const std::string & changelogs_path, LogFileSettings log_file_settings); /// Read log storage from filesystem starting from last_commited_log_index diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b823fbc697c..4de194fe9ae 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -25,6 +25,7 @@ #include #include #include +#include namespace DB { @@ -121,20 +122,36 @@ KeeperServer::KeeperServer( keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); keeper_context->ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); + if (!fs::exists(configuration_and_settings_->snapshot_storage_path)) + fs::create_directories(configuration_and_settings_->snapshot_storage_path); + auto snapshots_disk = std::make_shared("Keeper-snapshots", configuration_and_settings_->snapshot_storage_path, 0); + state_machine = nuraft::cs_new( responses_queue_, snapshots_queue_, - configuration_and_settings_->snapshot_storage_path, + snapshots_disk, coordination_settings, keeper_context, config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr, checkAndGetSuperdigest(configuration_and_settings_->super_digest)); + auto state_path = fs::path(configuration_and_settings_->state_file_path).parent_path().generic_string(); + auto state_file_name = fs::path(configuration_and_settings_->state_file_path).filename().generic_string(); + + if (!fs::exists(state_path)) + fs::create_directories(state_path); + auto state_disk = std::make_shared("Keeper-state", state_path, 0); + + if (!fs::exists(configuration_and_settings_->log_storage_path)) + fs::create_directories(configuration_and_settings_->log_storage_path); + auto logs_disk = std::make_shared("Keeper-logs", configuration_and_settings_->log_storage_path, 0); + state_manager = nuraft::cs_new( server_id, "keeper_server", - configuration_and_settings_->log_storage_path, - configuration_and_settings_->state_file_path, + logs_disk, + state_disk, + state_file_name, config, coordination_settings); } diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a3f7dbc2c6a..3d308fbabda 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -507,39 +508,45 @@ KeeperSnapshotManager::KeeperSnapshotManager( bool compress_snapshots_zstd_, const std::string & superdigest_, size_t storage_tick_time_) - : snapshots_path(snapshots_path_) + : KeeperSnapshotManager( + std::make_shared("Keeper-snapshots", snapshots_path_, 0), + snapshots_to_keep_, + keeper_context_, + compress_snapshots_zstd_, + superdigest_, + storage_tick_time_) +{ +} + +KeeperSnapshotManager::KeeperSnapshotManager( + DiskPtr disk_, + size_t snapshots_to_keep_, + const KeeperContextPtr & keeper_context_, + bool compress_snapshots_zstd_, + const std::string & superdigest_, + size_t storage_tick_time_) + : disk(disk_) , snapshots_to_keep(snapshots_to_keep_) , compress_snapshots_zstd(compress_snapshots_zstd_) , superdigest(superdigest_) , storage_tick_time(storage_tick_time_) , keeper_context(keeper_context_) { - namespace fs = std::filesystem; - - if (!fs::exists(snapshots_path)) - fs::create_directories(snapshots_path); - - for (const auto & p : fs::directory_iterator(snapshots_path)) + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { - const auto & path = p.path(); - - if (!path.has_filename()) + const auto & name = it->name(); + if (name.empty()) continue; - - if (startsWith(path.filename(), "tmp_")) /// Unfinished tmp files + if (startsWith(name, "tmp_")) { - std::filesystem::remove(p); + disk->removeFile(it->path()); continue; } - /// Not snapshot file - if (!startsWith(path.filename(), "snapshot_")) - { + if (!startsWith(name, "snapshot_")) continue; - } - - size_t snapshot_up_to = getSnapshotPathUpToLogIdx(p.path()); - existing_snapshots[snapshot_up_to] = p.path(); + size_t snapshot_up_to = getSnapshotPathUpToLogIdx(name); + existing_snapshots[snapshot_up_to] = it->path(); } removeOutdatedSnapshotsIfNeeded(); @@ -552,19 +559,17 @@ std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - std::string tmp_snapshot_path = std::filesystem::path{snapshots_path} / tmp_snapshot_file_name; - std::string new_snapshot_path = std::filesystem::path{snapshots_path} / snapshot_file_name; - WriteBufferFromFile plain_buf(tmp_snapshot_path); - copyData(reader, plain_buf); - plain_buf.sync(); + auto plain_buf = disk->writeFile(tmp_snapshot_file_name); + copyData(reader, *plain_buf); + plain_buf->sync(); - std::filesystem::rename(tmp_snapshot_path, new_snapshot_path); + disk->moveFile(tmp_snapshot_file_name, snapshot_file_name); - existing_snapshots.emplace(up_to_log_idx, new_snapshot_path); + existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); removeOutdatedSnapshotsIfNeeded(); - return new_snapshot_path; + return snapshot_file_name; } nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk() @@ -578,7 +583,7 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff } catch (const DB::Exception &) { - std::filesystem::remove(latest_itr->second); + disk->removeFile(latest_itr->second); existing_snapshots.erase(latest_itr->first); tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -591,8 +596,8 @@ nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFrom { const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx); WriteBufferFromNuraftBuffer writer; - ReadBufferFromFile reader(snapshot_path); - copyData(reader, writer); + auto reader = disk->readFile(snapshot_path); + copyData(*reader, writer); return writer.getBuffer(); } @@ -664,7 +669,7 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) auto itr = existing_snapshots.find(log_idx); if (itr == existing_snapshots.end()) throw Exception(ErrorCodes::UNKNOWN_SNAPSHOT, "Unknown snapshot with log index {}", log_idx); - std::filesystem::remove(itr->second); + disk->removeFile(itr->second); existing_snapshots.erase(itr); } @@ -673,10 +678,8 @@ std::pair KeeperSnapshotManager::serializeSnapshot auto up_to_log_idx = snapshot.snapshot_meta->get_last_log_idx(); auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - std::string tmp_snapshot_path = std::filesystem::path{snapshots_path} / tmp_snapshot_file_name; - std::string new_snapshot_path = std::filesystem::path{snapshots_path} / snapshot_file_name; - auto writer = std::make_unique(tmp_snapshot_path, O_WRONLY | O_TRUNC | O_CREAT | O_CLOEXEC | O_APPEND); + auto writer = disk->writeFile(tmp_snapshot_file_name); std::unique_ptr compressed_writer; if (compress_snapshots_zstd) compressed_writer = wrapWriteBufferWithCompressionMethod(std::move(writer), CompressionMethod::Zstd, 3); @@ -688,13 +691,21 @@ std::pair KeeperSnapshotManager::serializeSnapshot compressed_writer->sync(); std::error_code ec; - std::filesystem::rename(tmp_snapshot_path, new_snapshot_path, ec); - if (!ec) + + try { - existing_snapshots.emplace(up_to_log_idx, new_snapshot_path); - removeOutdatedSnapshotsIfNeeded(); + disk->moveFile(tmp_snapshot_file_name, snapshot_file_name); } - return {new_snapshot_path, ec}; + catch (fs::filesystem_error & e) + { + ec = e.code(); + return {snapshot_file_name, ec}; + } + + existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); + removeOutdatedSnapshotsIfNeeded(); + + return {snapshot_file_name, ec}; } } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 52647712083..9babad9ed98 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -97,6 +98,15 @@ using SnapshotMetaAndStorage = std::pair; class KeeperSnapshotManager { public: + KeeperSnapshotManager( + DiskPtr disk_, + size_t snapshots_to_keep_, + const KeeperContextPtr & keeper_context_, + bool compress_snapshots_zstd_ = true, + const std::string & superdigest_ = "", + size_t storage_tick_time_ = 500); + + /// For gtest KeeperSnapshotManager( const std::string & snapshots_path_, size_t snapshots_to_keep_, @@ -144,9 +154,15 @@ public: if (!existing_snapshots.empty()) { const auto & path = existing_snapshots.at(getLatestSnapshotIndex()); - std::error_code ec; - if (std::filesystem::exists(path, ec)) - return path; + + try + { + if (disk->exists(path)) + return path; + } + catch (...) + { + } } return ""; } @@ -158,7 +174,7 @@ private: /// ZSTD codec. static bool isZstdCompressed(nuraft::ptr buffer); - const std::string snapshots_path; + DiskPtr disk; /// How many snapshots to keep before remove const size_t snapshots_to_keep; /// All existing snapshots in our path (log_index -> path) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e591f87c6f1..56b8c7e5f2e 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -41,14 +41,14 @@ namespace KeeperStateMachine::KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - const std::string & snapshots_path_, + DiskPtr disk_, const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, const std::string & superdigest_) : coordination_settings(coordination_settings_) , snapshot_manager( - snapshots_path_, + disk_, coordination_settings->snapshots_to_keep, keeper_context_, coordination_settings->compress_snapshots_with_zstd_format, diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index ffc7fce1cfe..ba209f63a0c 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -25,7 +25,7 @@ public: KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - const std::string & snapshots_path_, + DiskPtr disk_, const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index cfb3519e597..36a88f5591f 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -214,8 +215,8 @@ KeeperStateManager::KeeperStateManager( int server_id_, const std::string & host, int port, const std::string & logs_path, const std::string & state_file_path) : my_server_id(server_id_) , secure(false) - , log_store(nuraft::cs_new(logs_path, LogFileSettings{.force_sync =false, .compress_logs = false, .rotate_interval = 5000})) - , server_state_path(state_file_path) + , log_store(nuraft::cs_new(std::make_shared("Keeper-logs", logs_path, 0), LogFileSettings{.force_sync =false, .compress_logs = false, .rotate_interval = 5000})) + , server_state_file_name(fs::path(state_file_path).filename().generic_string()) , logger(&Poco::Logger::get("KeeperStateManager")) { auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); @@ -228,8 +229,9 @@ KeeperStateManager::KeeperStateManager( KeeperStateManager::KeeperStateManager( int my_server_id_, const std::string & config_prefix_, - const std::string & log_storage_path, - const std::string & state_file_path, + DiskPtr log_disk_, + DiskPtr state_disk_, + const std::string & server_state_file_name_, const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings) : my_server_id(my_server_id_) @@ -237,7 +239,7 @@ KeeperStateManager::KeeperStateManager( , config_prefix(config_prefix_) , configuration_wrapper(parseServersConfiguration(config, false)) , log_store(nuraft::cs_new( - log_storage_path, + log_disk_, LogFileSettings { .force_sync = coordination_settings->force_sync, @@ -246,7 +248,8 @@ KeeperStateManager::KeeperStateManager( .max_size = coordination_settings->max_log_file_size, .overallocate_size = coordination_settings->log_file_overallocate_size })) - , server_state_path(state_file_path) + , disk(state_disk_) + , server_state_file_name(server_state_file_name_) , logger(&Poco::Logger::get("KeeperStateManager")) { } @@ -285,11 +288,11 @@ void KeeperStateManager::save_config(const nuraft::cluster_config & config) configuration_wrapper.cluster_config = nuraft::cluster_config::deserialize(*buf); } -const std::filesystem::path & KeeperStateManager::getOldServerStatePath() +const String & KeeperStateManager::getOldServerStatePath() { static auto old_path = [this] { - return server_state_path.parent_path() / (server_state_path.filename().generic_string() + "-OLD"); + return server_state_file_name + "-OLD"; }(); return old_path; @@ -310,25 +313,24 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state) { const auto & old_path = getOldServerStatePath(); - if (std::filesystem::exists(server_state_path)) - std::filesystem::rename(server_state_path, old_path); + if (disk->exists(server_state_file_name)) + disk->moveFile(server_state_file_name, old_path); - WriteBufferFromFile server_state_file(server_state_path, DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); + auto server_state_file = disk->writeFile(server_state_file_name); auto buf = state.serialize(); // calculate checksum SipHash hash; hash.update(current_server_state_version); hash.update(reinterpret_cast(buf->data_begin()), buf->size()); - writeIntBinary(hash.get64(), server_state_file); + writeIntBinary(hash.get64(), *server_state_file); - writeIntBinary(static_cast(current_server_state_version), server_state_file); + writeIntBinary(static_cast(current_server_state_version), *server_state_file); - server_state_file.write(reinterpret_cast(buf->data_begin()), buf->size()); - server_state_file.sync(); - server_state_file.close(); + server_state_file->write(reinterpret_cast(buf->data_begin()), buf->size()); + server_state_file->sync(); - std::filesystem::remove(old_path); + disk->removeFileIfExists(old_path); } nuraft::ptr KeeperStateManager::read_state() @@ -339,22 +341,22 @@ nuraft::ptr KeeperStateManager::read_state() { try { - ReadBufferFromFile read_buf(path); - auto content_size = read_buf.getFileSize(); + auto read_buf = disk->readFile(path); + auto content_size = read_buf->getFileSize(); if (content_size == 0) return nullptr; uint64_t read_checksum{0}; - readIntBinary(read_checksum, read_buf); + readIntBinary(read_checksum, *read_buf); uint8_t version; - readIntBinary(version, read_buf); + readIntBinary(version, *read_buf); auto buffer_size = content_size - sizeof read_checksum - sizeof version; auto state_buf = nuraft::buffer::alloc(buffer_size); - read_buf.readStrict(reinterpret_cast(state_buf->data_begin()), buffer_size); + read_buf->readStrict(reinterpret_cast(state_buf->data_begin()), buffer_size); SipHash hash; hash.update(version); @@ -364,15 +366,15 @@ nuraft::ptr KeeperStateManager::read_state() { constexpr auto error_format = "Invalid checksum while reading state from {}. Got {}, expected {}"; #ifdef NDEBUG - LOG_ERROR(logger, error_format, path.generic_string(), hash.get64(), read_checksum); + LOG_ERROR(logger, error_format, path, hash.get64(), read_checksum); return nullptr; #else - throw Exception(ErrorCodes::CORRUPTED_DATA, error_format, path.generic_string(), hash.get64(), read_checksum); + throw Exception(ErrorCodes::CORRUPTED_DATA, error_format, disk->getPath() + path, hash.get64(), read_checksum); #endif } auto state = nuraft::srv_state::deserialize(*state_buf); - LOG_INFO(logger, "Read state from {}", path.generic_string()); + LOG_INFO(logger, "Read state from {}", disk->getPath() + path); return state; } catch (const std::exception & e) @@ -383,37 +385,34 @@ nuraft::ptr KeeperStateManager::read_state() throw; } - LOG_ERROR(logger, "Failed to deserialize state from {}", path.generic_string()); + LOG_ERROR(logger, "Failed to deserialize state from {}", disk->getPath() + path); return nullptr; } }; - if (std::filesystem::exists(server_state_path)) + if (disk->exists(server_state_file_name)) { - auto state = try_read_file(server_state_path); + auto state = try_read_file(server_state_file_name); if (state) { - if (std::filesystem::exists(old_path)) - std::filesystem::remove(old_path); + disk->removeFileIfExists(old_path); return state; } - std::filesystem::remove(server_state_path); + disk->removeFile(server_state_file_name); } - if (std::filesystem::exists(old_path)) + if (disk->exists(old_path)) { auto state = try_read_file(old_path); - if (state) { - std::filesystem::rename(old_path, server_state_path); + disk->moveFile(old_path, server_state_file_name); return state; } - - std::filesystem::remove(old_path); + disk->removeFile(old_path); } LOG_WARNING(logger, "No state was read"); diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index 5d210f8c0ea..d8369100d1c 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -39,7 +39,8 @@ public: KeeperStateManager( int server_id_, const std::string & config_prefix_, - const std::string & log_storage_path, + DiskPtr logs_disk_, + DiskPtr state_disk_, const std::string & state_file_path, const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings); @@ -111,7 +112,7 @@ public: ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; private: - const std::filesystem::path & getOldServerStatePath(); + const String & getOldServerStatePath(); /// Wrapper struct for Keeper cluster config. We parse this /// info from XML files. @@ -136,7 +137,8 @@ private: nuraft::ptr log_store; - const std::filesystem::path server_state_path; + DiskPtr disk; + const String server_state_file_name; Poco::Logger * logger; diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index f8c4d0e2bac..d079a0d230d 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } ZstdDeflatingAppendableWriteBuffer::ZstdDeflatingAppendableWriteBuffer( - std::unique_ptr out_, + std::unique_ptr out_, int compression_level, bool append_to_existing_file_, size_t buf_size, diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.h b/src/IO/ZstdDeflatingAppendableWriteBuffer.h index a0715480737..b64e5d5c4cf 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.h +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.h @@ -29,7 +29,7 @@ public: static inline constexpr ZSTDLastBlock ZSTD_CORRECT_TERMINATION_LAST_BLOCK = {0x01, 0x00, 0x00}; ZstdDeflatingAppendableWriteBuffer( - std::unique_ptr out_, + std::unique_ptr out_, int compression_level, bool append_to_existing_file_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, @@ -68,7 +68,7 @@ private: /// Adding zstd empty block (ZSTD_CORRECT_TERMINATION_LAST_BLOCK) to out.working_buffer void addEmptyBlock(); - std::unique_ptr out; + std::unique_ptr out; bool append_to_existing_file = false; ZSTD_CCtx * cctx; From 3a6dea5e163a9dfeac0a0e2be6ba2898d939b058 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 21 Feb 2023 17:03:00 +0100 Subject: [PATCH 0016/2223] When reading from multiple files displace parallel parsing Implementation: * Added a new parameter to getInput & getInputFormat to disable parallel parsing. * Currently this is used only by StorageFile as we have not seen degradation for other storages reading from multiple paths. --- src/Formats/FormatFactory.cpp | 5 +++-- src/Formats/FormatFactory.h | 3 ++- src/Interpreters/Context.cpp | 4 ++-- src/Interpreters/Context.h | 3 ++- src/Storages/StorageFile.cpp | 5 ++++- 5 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 5bee13075fb..79fcb8a1da9 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -216,7 +216,8 @@ InputFormatPtr FormatFactory::getInput( const Block & sample, ContextPtr context, UInt64 max_block_size, - const std::optional & _format_settings) const + const std::optional & _format_settings, + const bool disable_parallel_parsing) const { auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); @@ -231,7 +232,7 @@ InputFormatPtr FormatFactory::getInput( // Doesn't make sense to use parallel parsing with less than four threads // (segmentator + two parsers + reader). - bool parallel_parsing = settings.input_format_parallel_parsing && file_segmentation_engine && settings.max_threads >= 4; + bool parallel_parsing = !disable_parallel_parsing && settings.input_format_parallel_parsing && file_segmentation_engine && settings.max_threads >= 4; if (settings.max_memory_usage && settings.min_chunk_bytes_for_parallel_parsing * settings.max_threads * 2 > settings.max_memory_usage) parallel_parsing = false; diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 398548e4b22..69b1fd74bdc 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -127,7 +127,8 @@ public: const Block & sample, ContextPtr context, UInt64 max_block_size, - const std::optional & format_settings = std::nullopt) const; + const std::optional & format_settings = std::nullopt, + const bool disable_parallel_parsing = false) const; InputFormatPtr getInputFormat( const String & name, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0feac593acd..dbadf79700d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3260,9 +3260,9 @@ void Context::checkPartitionCanBeDropped(const String & database, const String & } -InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings) const +InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings, const bool disable_parallel_parsing) const { - return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings); + return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings, disable_parallel_parsing); } OutputFormatPtr Context::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 19bb6868331..ad5e4570d15 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -724,7 +724,8 @@ public: BackupsWorker & getBackupsWorker() const; /// I/O formats. - InputFormatPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings = std::nullopt) const; + InputFormatPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, + const std::optional & format_settings = std::nullopt, const bool disable_parallel_parsing = false) const; OutputFormatPtr getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const; OutputFormatPtr getOutputFormatParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e2a2f84bc72..344c04edf0d 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -565,6 +565,7 @@ public: , block_for_format(block_for_format_) , context(context_) , max_block_size(max_block_size_) + , disable_parallel_parsing(storage->paths.size() > 1) { if (!storage->use_table_fd) { @@ -607,7 +608,7 @@ public: read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context); auto format - = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings); + = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, disable_parallel_parsing); QueryPipelineBuilder builder; builder.init(Pipe(format)); @@ -691,6 +692,8 @@ private: UInt64 total_rows_approx_accumulated = 0; size_t total_rows_count_times = 0; UInt64 total_rows_approx_max = 0; + + bool disable_parallel_parsing = false; }; From 010edbf2aad3508402e82b8ce62f90ce62fc9f09 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 22 Feb 2023 10:39:23 +0100 Subject: [PATCH 0017/2223] do another way and logs work --- src/Common/DateLUT.h | 38 ++++++++++++++++++-------------------- 1 file changed, 18 insertions(+), 20 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index bd7969bffa6..6ec9cf1646d 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -5,9 +5,6 @@ #include #include -// -//#include "Interpreters/Context_fwd.h" -//#include "Interpreters/Context.h" #include "Common/CurrentThread.h" #include @@ -24,30 +21,31 @@ public: static ALWAYS_INLINE const DateLUTImpl & instance() // -V1071 { const auto & date_lut = getInstance(); - - if (DB::CurrentThread::isInitialized()) - { - const auto query_context = DB::CurrentThread::get().getQueryContext(); - - if (query_context) - { - auto implicit_timezone = extractTimezoneFromContext(query_context); - - if (!implicit_timezone.empty()) - return instance(implicit_timezone); - } - } - return *date_lut.default_impl.load(std::memory_order_acquire); } /// Return singleton DateLUTImpl instance for a given time zone. static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { - if (time_zone.empty()) - return instance(); - const auto & date_lut = getInstance(); + + if (time_zone.empty()) + { + if (DB::CurrentThread::isInitialized()) + { + const auto query_context = DB::CurrentThread::get().getQueryContext(); + + if (query_context) + { + auto implicit_timezone = extractTimezoneFromContext(query_context); + + if (!implicit_timezone.empty()) + return instance(implicit_timezone); + } + } + return *date_lut.default_impl.load(std::memory_order_acquire); + } + return date_lut.getImplementation(time_zone); } static void setDefaultTimezone(const std::string & time_zone) From 1cf6c3a9c0fa4867684dee56c651d4131aa3b0fe Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 22 Feb 2023 10:51:32 +0100 Subject: [PATCH 0018/2223] update test names --- ...licit_timezone.reference => 02668_implicit_timezone.reference} | 0 .../{02538_implicit_timezone.sql => 02668_implicit_timezone.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02538_implicit_timezone.reference => 02668_implicit_timezone.reference} (100%) rename tests/queries/0_stateless/{02538_implicit_timezone.sql => 02668_implicit_timezone.sql} (100%) diff --git a/tests/queries/0_stateless/02538_implicit_timezone.reference b/tests/queries/0_stateless/02668_implicit_timezone.reference similarity index 100% rename from tests/queries/0_stateless/02538_implicit_timezone.reference rename to tests/queries/0_stateless/02668_implicit_timezone.reference diff --git a/tests/queries/0_stateless/02538_implicit_timezone.sql b/tests/queries/0_stateless/02668_implicit_timezone.sql similarity index 100% rename from tests/queries/0_stateless/02538_implicit_timezone.sql rename to tests/queries/0_stateless/02668_implicit_timezone.sql From 3e9ab1276a914abaa36d5a230a3c81327cdbc37c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 22 Feb 2023 14:56:44 +0100 Subject: [PATCH 0019/2223] Updated to reduce parallel parsing based on number of files - When reading from multiple files reduce parallel parsing --- src/Formats/FormatFactory.cpp | 16 +++++++++++----- src/Formats/FormatFactory.h | 2 +- src/Interpreters/Context.cpp | 4 ++-- src/Interpreters/Context.h | 5 ++++- src/Storages/StorageFile.cpp | 5 +---- 5 files changed, 19 insertions(+), 13 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 79fcb8a1da9..e696f7e3f6a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -217,7 +217,7 @@ InputFormatPtr FormatFactory::getInput( ContextPtr context, UInt64 max_block_size, const std::optional & _format_settings, - const bool disable_parallel_parsing) const + const std::optional number_of_streams) const { auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); @@ -230,14 +230,18 @@ InputFormatPtr FormatFactory::getInput( const Settings & settings = context->getSettingsRef(); const auto & file_segmentation_engine = getCreators(name).file_segmentation_engine; + size_t max_threads = settings.max_threads; + if (number_of_streams.has_value() && number_of_streams.value()) + max_threads = std::max(max_threads/number_of_streams.value(),1UL); + // Doesn't make sense to use parallel parsing with less than four threads // (segmentator + two parsers + reader). - bool parallel_parsing = !disable_parallel_parsing && settings.input_format_parallel_parsing && file_segmentation_engine && settings.max_threads >= 4; + bool parallel_parsing = settings.input_format_parallel_parsing && file_segmentation_engine && max_threads >= 4; - if (settings.max_memory_usage && settings.min_chunk_bytes_for_parallel_parsing * settings.max_threads * 2 > settings.max_memory_usage) + if (settings.max_memory_usage && settings.min_chunk_bytes_for_parallel_parsing * max_threads * 2 > settings.max_memory_usage) parallel_parsing = false; - if (settings.max_memory_usage_for_user && settings.min_chunk_bytes_for_parallel_parsing * settings.max_threads * 2 > settings.max_memory_usage_for_user) + if (settings.max_memory_usage_for_user && settings.min_chunk_bytes_for_parallel_parsing * max_threads * 2 > settings.max_memory_usage_for_user) parallel_parsing = false; if (parallel_parsing) @@ -248,6 +252,7 @@ InputFormatPtr FormatFactory::getInput( parallel_parsing = false; } + if (parallel_parsing) { const auto & input_getter = getCreators(name).input_creator; @@ -264,8 +269,9 @@ InputFormatPtr FormatFactory::getInput( (ReadBuffer & input) -> InputFormatPtr { return input_getter(input, sample, row_input_format_params, format_settings); }; + ParallelParsingInputFormat::Params params{ - buf, sample, parser_creator, file_segmentation_engine, name, settings.max_threads, + buf, sample, parser_creator, file_segmentation_engine, name, max_threads, settings.min_chunk_bytes_for_parallel_parsing, max_block_size, context->getApplicationType() == Context::ApplicationType::SERVER}; auto format = std::make_shared(params); if (!settings.input_format_record_errors_file_path.toString().empty()) diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 69b1fd74bdc..6787a2a4038 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -128,7 +128,7 @@ public: ContextPtr context, UInt64 max_block_size, const std::optional & format_settings = std::nullopt, - const bool disable_parallel_parsing = false) const; + const std::optional number_of_streams = std::nullopt) const; InputFormatPtr getInputFormat( const String & name, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index dbadf79700d..de9c3821861 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3260,9 +3260,9 @@ void Context::checkPartitionCanBeDropped(const String & database, const String & } -InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings, const bool disable_parallel_parsing) const +InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings, const std::optional number_of_streams) const { - return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings, disable_parallel_parsing); + return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings, number_of_streams); } OutputFormatPtr Context::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ad5e4570d15..3d89853ee86 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -724,8 +724,11 @@ public: BackupsWorker & getBackupsWorker() const; /// I/O formats. + /// If number_of_streams is set then the number of threads are divided among the streams + /// Eg: max_threads = 16, number_of_streams = 8, then each stream will use 2 threads + /// This is done to improve performance InputFormatPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, - const std::optional & format_settings = std::nullopt, const bool disable_parallel_parsing = false) const; + const std::optional & format_settings = std::nullopt, const std::optional number_of_streams = std::nullopt) const; OutputFormatPtr getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const; OutputFormatPtr getOutputFormatParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 344c04edf0d..fd42a72c151 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -565,7 +565,6 @@ public: , block_for_format(block_for_format_) , context(context_) , max_block_size(max_block_size_) - , disable_parallel_parsing(storage->paths.size() > 1) { if (!storage->use_table_fd) { @@ -608,7 +607,7 @@ public: read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context); auto format - = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, disable_parallel_parsing); + = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, storage->paths.size()); QueryPipelineBuilder builder; builder.init(Pipe(format)); @@ -692,8 +691,6 @@ private: UInt64 total_rows_approx_accumulated = 0; size_t total_rows_count_times = 0; UInt64 total_rows_approx_max = 0; - - bool disable_parallel_parsing = false; }; From 393830ecdc78cd2745cc439d7ac95c3421fe9044 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 22 Feb 2023 16:30:46 +0100 Subject: [PATCH 0020/2223] add docs + tiny cleanup --- docs/en/operations/settings/settings.md | 27 ++++++++++++++++++++++++ docs/ru/operations/settings/settings.md | 28 +++++++++++++++++++++++++ src/Common/DateLUT.cpp | 3 +-- 3 files changed, 56 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1060eae1b0e..4e105124086 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3939,3 +3939,30 @@ Default value: `0`. :::note Use this setting only for backward compatibility if your use cases depend on old syntax. ::: + +## implicit_timezone {#implicit_timezone} + +If specified, sets a implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. +Examples: +``` +SELECT timeZone(), timeZoneOf(now()) +┌─timeZone()────┬─timeZoneOf(now())─┐ +│ Europe/Berlin │ Europe/Berlin │ +└───────────────┴───────────────────┘ + +:) SELECT timeZone(), timeZoneOf(now()) SETTINGS implicit_timezone = 'Asia/Novosibirsk' +┌─timeZone()────┬─timeZoneOf(now())─┐ +│ Europe/Berlin │ Asia/Novosibirsk │ +└───────────────┴───────────────────┘ + +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'America/Denver'; +┌─toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich')─┐ +│ 1999-12-13 07:23:23.123 │ +└──────────────────────────────────────────────────────────────────────────────┘ +``` + +Possible values: + +- Any valid timezone in `Region/Place` notation, e.g. `Europe/Berlin` + +Default value: `''`. \ No newline at end of file diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 4025966ac21..8d3f2706585 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4084,3 +4084,31 @@ ALTER TABLE test FREEZE SETTINGS alter_partition_verbose_result = 1; Задает символ, который интерпретируется как суффикс после результирующего набора данных формата [CustomSeparated](../../interfaces/formats.md#format-customseparated). Значение по умолчанию: `''`. + +## implicit_timezone {#implicit_timezone} + +Задаёт значение часового пояса (timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. + +Примеры: +``` +SELECT timeZone(), timeZoneOf(now()) +┌─timeZone()────┬─timeZoneOf(now())─┐ +│ Europe/Berlin │ Europe/Berlin │ +└───────────────┴───────────────────┘ + +:) SELECT timeZone(), timeZoneOf(now()) SETTINGS implicit_timezone = 'Asia/Novosibirsk' +┌─timeZone()────┬─timeZoneOf(now())─┐ +│ Europe/Berlin │ Asia/Novosibirsk │ +└───────────────┴───────────────────┘ + +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'America/Denver'; +┌─toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich')─┐ +│ 1999-12-13 07:23:23.123 │ +└──────────────────────────────────────────────────────────────────────────────┘ +``` + +Возможные значения: + +- Строка вида `Регион/Город`, например `Europe/Zurich` + +Значение по умолчанию: `''`. \ No newline at end of file diff --git a/src/Common/DateLUT.cpp b/src/Common/DateLUT.cpp index fd10b799b66..e309b0cb28a 100644 --- a/src/Common/DateLUT.cpp +++ b/src/Common/DateLUT.cpp @@ -8,7 +8,6 @@ #include #include #include -#include namespace @@ -149,7 +148,7 @@ DateLUT::DateLUT() } -const ALWAYS_INLINE DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const +const DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const { std::lock_guard lock(mutex); From c61aff7cac2e5cc79dc4591d9228308e017e5b28 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 23 Feb 2023 13:38:13 +0100 Subject: [PATCH 0021/2223] Added standalone function to get server's own timezone Fix missing global_context --- src/Common/DateLUT.h | 10 ++++++++++ src/Functions/serverConstants.cpp | 19 +++++++++++++++++-- 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index b66821989e3..18ef5ee2e30 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -42,6 +42,16 @@ public: if (!implicit_timezone.empty()) return instance(implicit_timezone); } + + const auto global_context = DB::CurrentThread::get().getGlobalContext(); + if (global_context) + { + auto implicit_timezone = extractTimezoneFromContext(global_context); + + if (!implicit_timezone.empty()) + return instance(implicit_timezone); + } + } return *date_lut.default_impl.load(std::memory_order_acquire); } diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 96615d0a4c9..a89e1564f28 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -60,13 +60,22 @@ namespace }; - /// Returns the server time zone. + /// Returns default timezone for current session. class FunctionTimezone : public FunctionConstantBase { public: static constexpr auto name = "timezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {} + explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance("").getTimeZone()}, context->isDistributed()) {} + }; + + /// Returns the server time zone (timezone in which server runs). + class FunctionServerTimezone : public FunctionConstantBase + { + public: + static constexpr auto name = "serverTimezone"; + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {} }; @@ -155,6 +164,12 @@ REGISTER_FUNCTION(Timezone) factory.registerAlias("timeZone", "timezone"); } +REGISTER_FUNCTION(ServerTimezone) +{ + factory.registerFunction(); + factory.registerAlias("serverTimeZone", "serverTimezone"); +} + REGISTER_FUNCTION(Uptime) { factory.registerFunction(); From a9d0f7e7dbb281def311b22e4ae6300c73b5e979 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 23 Feb 2023 19:14:49 +0100 Subject: [PATCH 0022/2223] Added docs for new serverTimeZone function Updated tests and docs --- docs/en/operations/settings/settings.md | 28 ++++++++++--------- .../functions/date-time-functions.md | 21 +++++++++++++- docs/ru/operations/settings/settings.md | 27 +++++++++--------- .../functions/date-time-functions.md | 21 +++++++++++++- src/Common/DateLUT.cpp | 3 +- src/Common/DateLUT.h | 8 ++++-- src/Core/Settings.h | 2 +- src/Functions/serverConstants.cpp | 23 +++++++++++++-- ...rence => 02668_timezone_setting.reference} | 0 ...imezone.sql => 02668_timezone_setting.sql} | 6 ++-- 10 files changed, 101 insertions(+), 38 deletions(-) rename tests/queries/0_stateless/{02668_implicit_timezone.reference => 02668_timezone_setting.reference} (100%) rename tests/queries/0_stateless/{02668_implicit_timezone.sql => 02668_timezone_setting.sql} (61%) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4e105124086..c1c4483d341 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3940,25 +3940,27 @@ Default value: `0`. Use this setting only for backward compatibility if your use cases depend on old syntax. ::: -## implicit_timezone {#implicit_timezone} +## timezone {#timezone} If specified, sets a implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. Examples: + +```clickhouse +SELECT timeZone(), serverTimezone() FORMAT TSV + +Europe/Berlin Europe/Berlin ``` -SELECT timeZone(), timeZoneOf(now()) -┌─timeZone()────┬─timeZoneOf(now())─┐ -│ Europe/Berlin │ Europe/Berlin │ -└───────────────┴───────────────────┘ -:) SELECT timeZone(), timeZoneOf(now()) SETTINGS implicit_timezone = 'Asia/Novosibirsk' -┌─timeZone()────┬─timeZoneOf(now())─┐ -│ Europe/Berlin │ Asia/Novosibirsk │ -└───────────────┴───────────────────┘ +```clickhouse +SELECT timeZone(), serverTimezone() SETTINGS timezone = 'Asia/Novosibirsk' FORMAT TSV -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'America/Denver'; -┌─toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich')─┐ -│ 1999-12-13 07:23:23.123 │ -└──────────────────────────────────────────────────────────────────────────────┘ +Asia/Novosibirsk Europe/Berlin +``` + +```clickhouse +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'America/Denver' FORMAT TSV + +1999-12-13 07:23:23.123 ``` Possible values: diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f6af8abcbaf..8d31cb3872f 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -26,7 +26,7 @@ SELECT ## timeZone -Returns the timezone of the server. +Returns the default timezone of the server for current session. This can be modified using `SET timezone = 'New/Value'` If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. **Syntax** @@ -43,6 +43,25 @@ Alias: `timezone`. Type: [String](../../sql-reference/data-types/string.md). +## serverTimeZone + +Returns the actual timezone in which the server runs in. +If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. + +**Syntax** + +``` sql +timeZone() +``` + +Alias: `ServerTimezone`, `servertimezone`. + +**Returned value** + +- Timezone. + +Type: [String](../../sql-reference/data-types/string.md). + ## toTimeZone Converts time or date and time to the specified time zone. The time zone is an attribute of the `Date` and `DateTime` data types. The internal value (number of seconds) of the table field or of the resultset's column does not change, the column's type changes and its string representation changes accordingly. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 8d3f2706585..dd1e9d98427 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4085,26 +4085,27 @@ ALTER TABLE test FREEZE SETTINGS alter_partition_verbose_result = 1; Значение по умолчанию: `''`. -## implicit_timezone {#implicit_timezone} +## timezone {#timezone} Задаёт значение часового пояса (timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. Примеры: +```clickhouse +SELECT timeZone(), serverTimezone() FORMAT TSV + +Europe/Berlin Europe/Berlin ``` -SELECT timeZone(), timeZoneOf(now()) -┌─timeZone()────┬─timeZoneOf(now())─┐ -│ Europe/Berlin │ Europe/Berlin │ -└───────────────┴───────────────────┘ -:) SELECT timeZone(), timeZoneOf(now()) SETTINGS implicit_timezone = 'Asia/Novosibirsk' -┌─timeZone()────┬─timeZoneOf(now())─┐ -│ Europe/Berlin │ Asia/Novosibirsk │ -└───────────────┴───────────────────┘ +```clickhouse +SELECT timeZone(), serverTimezone() SETTINGS timezone = 'Asia/Novosibirsk' FORMAT TSV -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'America/Denver'; -┌─toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich')─┐ -│ 1999-12-13 07:23:23.123 │ -└──────────────────────────────────────────────────────────────────────────────┘ +Asia/Novosibirsk Europe/Berlin +``` + +```clickhouse +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'America/Denver' FORMAT TSV + +1999-12-13 07:23:23.123 ``` Возможные значения: diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 8fbcaf9568b..77188ea2797 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -26,7 +26,7 @@ SELECT ## timeZone {#timezone} -Возвращает часовой пояс сервера. +Возвращает часовой пояс сервера, считающийся умолчанием для текущей сессии. Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. **Синтаксис** @@ -43,6 +43,25 @@ timeZone() Тип: [String](../../sql-reference/data-types/string.md). +## serverTimeZone {#servertimezone} + +Возвращает (истинный) часовой пояс сервера, в котором тот работает. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. + +**Синтаксис** + +``` sql +serverTimeZone() +``` + +Синонимы: `servertimezone`, `serverTimezone`. + +**Возвращаемое значение** + +- Часовой пояс. + +Тип: [String](../../sql-reference/data-types/string.md). + ## toTimeZone {#totimezone} Переводит дату или дату с временем в указанный часовой пояс. Часовой пояс - это атрибут типов `Date` и `DateTime`. Внутреннее значение (количество секунд) поля таблицы или результирующего столбца не изменяется, изменяется тип поля и, соответственно, его текстовое отображение. diff --git a/src/Common/DateLUT.cpp b/src/Common/DateLUT.cpp index e309b0cb28a..3698fe45aa7 100644 --- a/src/Common/DateLUT.cpp +++ b/src/Common/DateLUT.cpp @@ -167,6 +167,5 @@ DateLUT & DateLUT::getInstance() std::string DateLUT::extractTimezoneFromContext(const DB::ContextPtr query_context) { - std::string ret = query_context->getSettingsRef().implicit_timezone.value; - return ret; + return query_context->getSettingsRef().timezone.value; } diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 18ef5ee2e30..a9ee61dc8ab 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -17,14 +17,18 @@ class DateLUT : private boost::noncopyable { public: - /// Return singleton DateLUTImpl instance for the default time zone. + /// Return singleton DateLUTImpl instance for server's timezone (the one which server has). static ALWAYS_INLINE const DateLUTImpl & instance() { const auto & date_lut = getInstance(); return *date_lut.default_impl.load(std::memory_order_acquire); } - /// Return singleton DateLUTImpl instance for a given time zone. + /* + Return singleton DateLUTImpl instance for a given time zone. If timezone is an empty string, + timezone set by `timezone` setting for current session is used. If it is not set, server's timezone is used, + and return is the same as calling instance(). + */ static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { const auto & date_lut = getInstance(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e3ed1b26269..e70b8c131b3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -712,7 +712,7 @@ class IColumn; M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ M(Bool, force_aggregation_in_order, false, "Force use of aggregation in order on remote nodes during distributed aggregation. PLEASE, NEVER CHANGE THIS SETTING VALUE MANUALLY!", IMPORTANT) \ - M(String, implicit_timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ + M(String, timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index a89e1564f28..b57f7e40e07 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -160,14 +160,33 @@ REGISTER_FUNCTION(TcpPort) REGISTER_FUNCTION(Timezone) { - factory.registerFunction(); + factory.registerFunction({ + R"( +Returns the default timezone for current session. +Used as default timezone for parsing DateTime|DateTime64 without explicitly specified timezone. +Can be changed with SET timezone = 'New/Tz' + +[example:timezone] +)", + Documentation::Examples{{"serverTimezone", "SELECT timezone();"}}, + Documentation::Categories{"Constant", "Miscellaneous"} + }); factory.registerAlias("timeZone", "timezone"); } REGISTER_FUNCTION(ServerTimezone) { - factory.registerFunction(); + factory.registerFunction({ + R"( +Returns the timezone name in which server operates. + +[example:serverTimezone] +)", + Documentation::Examples{{"serverTimezone", "SELECT serverTimezone();"}}, + Documentation::Categories{"Constant", "Miscellaneous"} + }); factory.registerAlias("serverTimeZone", "serverTimezone"); + factory.registerAlias("servertimezone", "serverTimezone"); } REGISTER_FUNCTION(Uptime) diff --git a/tests/queries/0_stateless/02668_implicit_timezone.reference b/tests/queries/0_stateless/02668_timezone_setting.reference similarity index 100% rename from tests/queries/0_stateless/02668_implicit_timezone.reference rename to tests/queries/0_stateless/02668_timezone_setting.reference diff --git a/tests/queries/0_stateless/02668_implicit_timezone.sql b/tests/queries/0_stateless/02668_timezone_setting.sql similarity index 61% rename from tests/queries/0_stateless/02668_implicit_timezone.sql rename to tests/queries/0_stateless/02668_timezone_setting.sql index 663b218d235..3748b536614 100644 --- a/tests/queries/0_stateless/02668_implicit_timezone.sql +++ b/tests/queries/0_stateless/02668_timezone_setting.sql @@ -1,9 +1,9 @@ -SET implicit_timezone = 'Asia/Novosibirsk'; +SET timezone = 'Asia/Novosibirsk'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'Europe/Zurich'; +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; -SET implicit_timezone = 'Europe/Zurich'; +SET timezone = 'Europe/Zurich'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); \ No newline at end of file From c3a6efe0310ec23521eb97c1b53c6616f72ba7a0 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 24 Feb 2023 20:51:12 +0100 Subject: [PATCH 0023/2223] update --- src/Client/ClientBase.cpp | 2 +- src/Common/DateLUT.h | 13 +++++++------ src/Functions/serverConstants.cpp | 2 +- src/IO/ReadHelpers.h | 8 ++++---- src/IO/WriteHelpers.h | 6 +++--- .../queries/0_stateless/02668_timezone_setting.sql | 6 +++--- 6 files changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index bc8c43af8c6..9ebe115d408 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2139,7 +2139,7 @@ void ClientBase::runInteractive() initQueryIdFormats(); /// Initialize DateLUT here to avoid counting time spent here as query execution time. - const auto local_tz = DateLUT::instance().getTimeZone(); + const auto local_tz = DateLUT::instance("").getTimeZone(); suggest.emplace(); if (load_suggestions) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index a9ee61dc8ab..29a4ee13d87 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -32,6 +32,7 @@ public: static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { const auto & date_lut = getInstance(); + std::string effective_time_zone; if (time_zone.empty()) { @@ -41,19 +42,19 @@ public: if (query_context) { - auto implicit_timezone = extractTimezoneFromContext(query_context); + effective_time_zone = extractTimezoneFromContext(query_context); - if (!implicit_timezone.empty()) - return instance(implicit_timezone); + if (!effective_time_zone.empty()) + return date_lut.getImplementation(effective_time_zone); } const auto global_context = DB::CurrentThread::get().getGlobalContext(); if (global_context) { - auto implicit_timezone = extractTimezoneFromContext(global_context); + effective_time_zone = extractTimezoneFromContext(global_context); - if (!implicit_timezone.empty()) - return instance(implicit_timezone); + if (!effective_time_zone.empty()) + return date_lut.getImplementation(effective_time_zone); } } diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index b57f7e40e07..ea74d7a89bb 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -168,7 +168,7 @@ Can be changed with SET timezone = 'New/Tz' [example:timezone] )", - Documentation::Examples{{"serverTimezone", "SELECT timezone();"}}, + Documentation::Examples{{"timezone", "SELECT timezone();"}}, Documentation::Categories{"Constant", "Miscellaneous"} }); factory.registerAlias("timeZone", "timezone"); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index f8931a7f622..9f5358ee141 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1032,22 +1032,22 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re return ReturnType(is_ok); } -inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) { readDateTimeTextImpl(datetime, buf, time_zone); } -inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) +inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance("")) { readDateTimeTextImpl(datetime64, scale, buf, date_lut); } -inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) { return readDateTimeTextImpl(datetime, buf, time_zone); } -inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) +inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance("")) { return readDateTimeTextImpl(datetime64, scale, buf, date_lut); } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 1c0b48c53c3..d408e2bed42 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -755,14 +755,14 @@ inline void writeDateTimeText(const LocalDateTime & datetime, WriteBuffer & buf) /// In the format YYYY-MM-DD HH:MM:SS, according to the specified time zone. template -inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) { writeDateTimeText(LocalDateTime(datetime, time_zone), buf); } /// In the format YYYY-MM-DD HH:MM:SS.NNNNNNNNN, according to the specified time zone. template -inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) { static constexpr UInt32 MaxScale = DecimalUtils::max_precision; scale = scale > MaxScale ? MaxScale : scale; @@ -796,7 +796,7 @@ inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & /// In the RFC 1123 format: "Tue, 03 Dec 2019 00:11:50 GMT". You must provide GMT DateLUT. /// This is needed for HTTP requests. -inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) { const auto & values = time_zone.getValues(datetime); diff --git a/tests/queries/0_stateless/02668_timezone_setting.sql b/tests/queries/0_stateless/02668_timezone_setting.sql index 3748b536614..f331ab58307 100644 --- a/tests/queries/0_stateless/02668_timezone_setting.sql +++ b/tests/queries/0_stateless/02668_timezone_setting.sql @@ -1,9 +1,9 @@ SET timezone = 'Asia/Novosibirsk'; - SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); - SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; SET timezone = 'Europe/Zurich'; +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); \ No newline at end of file +SET timezone = 'Абырвалг'; +select now(); -- { serverError POCO_EXCEPTION } \ No newline at end of file From 6a996f552b9cf70f88d2a6c7c8f1ef2780268666 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 26 Feb 2023 10:06:27 +0100 Subject: [PATCH 0024/2223] update undocumented funcs reference --- .../02415_all_new_functions_must_be_documented.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index e41249af54c..ce14ee871f5 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -683,7 +683,6 @@ throwIf tid timeSlot timeSlots -timezone timezoneOf timezoneOffset toBool From a69425326de20dcf5814c39a0962023cabec27ec Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 27 Feb 2023 01:40:00 +0100 Subject: [PATCH 0025/2223] upd --- src/Client/ClientBase.cpp | 2 +- src/Common/DateLUT.h | 59 +++++++++++++++---------------- src/DataTypes/TimezoneMixin.h | 2 +- src/Functions/serverConstants.cpp | 4 +-- src/IO/ReadHelpers.h | 8 ++--- src/IO/WriteHelpers.h | 6 ++-- 6 files changed, 39 insertions(+), 42 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a335dca0602..96aff9aa304 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2139,7 +2139,7 @@ void ClientBase::runInteractive() initQueryIdFormats(); /// Initialize DateLUT here to avoid counting time spent here as query execution time. - const auto local_tz = DateLUT::instance("").getTimeZone(); + const auto local_tz = DateLUT::instance().getTimeZone(); suggest.emplace(); if (load_suggestions) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 29a4ee13d87..efbf56b59b2 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -17,49 +17,46 @@ class DateLUT : private boost::noncopyable { public: - /// Return singleton DateLUTImpl instance for server's timezone (the one which server has). + /// Return singleton DateLUTImpl instance for timezone set by `timezone` setting for current session is used. + /// If it is not set, server's timezone (the one which server has) is being used. static ALWAYS_INLINE const DateLUTImpl & instance() { + std::string effective_time_zone; const auto & date_lut = getInstance(); + + if (DB::CurrentThread::isInitialized()) + { + const auto query_context = DB::CurrentThread::get().getQueryContext(); + + if (query_context) + { + effective_time_zone = extractTimezoneFromContext(query_context); + + if (!effective_time_zone.empty()) + return date_lut.getImplementation(effective_time_zone); + } + + const auto global_context = DB::CurrentThread::get().getGlobalContext(); + if (global_context) + { + effective_time_zone = extractTimezoneFromContext(global_context); + + if (!effective_time_zone.empty()) + return date_lut.getImplementation(effective_time_zone); + } + + } return *date_lut.default_impl.load(std::memory_order_acquire); } - /* - Return singleton DateLUTImpl instance for a given time zone. If timezone is an empty string, - timezone set by `timezone` setting for current session is used. If it is not set, server's timezone is used, - and return is the same as calling instance(). - */ + /// Return singleton DateLUTImpl instance for a given time zone. If timezone is an empty string, + /// server's timezone is used. The `timezone` setting is not considered here. static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { const auto & date_lut = getInstance(); - std::string effective_time_zone; if (time_zone.empty()) - { - if (DB::CurrentThread::isInitialized()) - { - const auto query_context = DB::CurrentThread::get().getQueryContext(); - - if (query_context) - { - effective_time_zone = extractTimezoneFromContext(query_context); - - if (!effective_time_zone.empty()) - return date_lut.getImplementation(effective_time_zone); - } - - const auto global_context = DB::CurrentThread::get().getGlobalContext(); - if (global_context) - { - effective_time_zone = extractTimezoneFromContext(global_context); - - if (!effective_time_zone.empty()) - return date_lut.getImplementation(effective_time_zone); - } - - } return *date_lut.default_impl.load(std::memory_order_acquire); - } return date_lut.getImplementation(time_zone); } diff --git a/src/DataTypes/TimezoneMixin.h b/src/DataTypes/TimezoneMixin.h index 03ecde5dd0a..5b7870c7b9a 100644 --- a/src/DataTypes/TimezoneMixin.h +++ b/src/DataTypes/TimezoneMixin.h @@ -15,7 +15,7 @@ public: explicit TimezoneMixin(const String & time_zone_name = "") : has_explicit_time_zone(!time_zone_name.empty()) - , time_zone(DateLUT::instance(time_zone_name)) + , time_zone(time_zone_name.empty() ? DateLUT::instance() : DateLUT::instance(time_zone_name)) , utc_time_zone(DateLUT::instance("UTC")) { } diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index ea74d7a89bb..57a6279bd7a 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -66,7 +66,7 @@ namespace public: static constexpr auto name = "timezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance("").getTimeZone()}, context->isDistributed()) {} + explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {} }; /// Returns the server time zone (timezone in which server runs). @@ -75,7 +75,7 @@ namespace public: static constexpr auto name = "serverTimezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {} + explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance("").getTimeZone()}, context->isDistributed()) {} }; diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 9f5358ee141..f8931a7f622 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1032,22 +1032,22 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re return ReturnType(is_ok); } -inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) +inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readDateTimeTextImpl(datetime, buf, time_zone); } -inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance("")) +inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { readDateTimeTextImpl(datetime64, scale, buf, date_lut); } -inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) +inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { return readDateTimeTextImpl(datetime, buf, time_zone); } -inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance("")) +inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { return readDateTimeTextImpl(datetime64, scale, buf, date_lut); } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index d408e2bed42..1c0b48c53c3 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -755,14 +755,14 @@ inline void writeDateTimeText(const LocalDateTime & datetime, WriteBuffer & buf) /// In the format YYYY-MM-DD HH:MM:SS, according to the specified time zone. template -inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) +inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { writeDateTimeText(LocalDateTime(datetime, time_zone), buf); } /// In the format YYYY-MM-DD HH:MM:SS.NNNNNNNNN, according to the specified time zone. template -inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) +inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { static constexpr UInt32 MaxScale = DecimalUtils::max_precision; scale = scale > MaxScale ? MaxScale : scale; @@ -796,7 +796,7 @@ inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & /// In the RFC 1123 format: "Tue, 03 Dec 2019 00:11:50 GMT". You must provide GMT DateLUT. /// This is needed for HTTP requests. -inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) +inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { const auto & values = time_zone.getValues(datetime); From f3e19144d81449c1e2bdec52ebc38e85ea1e8ee9 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 27 Feb 2023 14:38:15 +0100 Subject: [PATCH 0026/2223] update --- src/Interpreters/executeQuery.cpp | 3 ++- src/Loggers/OwnPatternFormatter.cpp | 2 +- src/Loggers/OwnPatternFormatter.h | 2 ++ 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 435401796a0..cda7ec2b0d3 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -49,6 +49,7 @@ #include #include #include +#include #include #include #include @@ -1279,7 +1280,7 @@ void executeQuery( QueryResultDetails result_details { .query_id = context->getClientInfo().current_query_id, - .timezone = DateLUT::instance().getTimeZone(), + .timezone = DateLUT::instance("").getTimeZone(), }; std::unique_ptr compressed_buffer; diff --git a/src/Loggers/OwnPatternFormatter.cpp b/src/Loggers/OwnPatternFormatter.cpp index 02a2c2e510b..54d2b995d15 100644 --- a/src/Loggers/OwnPatternFormatter.cpp +++ b/src/Loggers/OwnPatternFormatter.cpp @@ -22,7 +22,7 @@ void OwnPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ext, const Poco::Message & msg = msg_ext.base; /// Change delimiters in date for compatibility with old logs. - DB::writeDateTimeText<'.', ':'>(msg_ext.time_seconds, wb); + DB::writeDateTimeText<'.', ':'>(msg_ext.time_seconds, wb, server_timezone); DB::writeChar('.', wb); DB::writeChar('0' + ((msg_ext.time_microseconds / 100000) % 10), wb); diff --git a/src/Loggers/OwnPatternFormatter.h b/src/Loggers/OwnPatternFormatter.h index d776b097cb2..07d0409b0ae 100644 --- a/src/Loggers/OwnPatternFormatter.h +++ b/src/Loggers/OwnPatternFormatter.h @@ -2,6 +2,7 @@ #include +#include #include "ExtendedLogChannel.h" @@ -30,5 +31,6 @@ public: virtual void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text) const; private: + const DateLUTImpl & server_timezone = DateLUT::instance(""); bool color; }; From e435748473d33a578625597faaadb5c0a1db08cb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Mar 2023 18:13:30 +0100 Subject: [PATCH 0027/2223] Update ubuntu_ami_for_ci.sh --- tests/ci/worker/ubuntu_ami_for_ci.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/worker/ubuntu_ami_for_ci.sh b/tests/ci/worker/ubuntu_ami_for_ci.sh index a43eac828c6..58797502419 100644 --- a/tests/ci/worker/ubuntu_ami_for_ci.sh +++ b/tests/ci/worker/ubuntu_ami_for_ci.sh @@ -75,6 +75,9 @@ cat < /etc/docker/daemon.json } EOT +# Increase the limit on number of virtual memory mappings to aviod 'Cannot mmap' error +sysctl -w vm.max_map_count=2097152 + systemctl restart docker # buildx builder is user-specific From 47cd46f590dfc3660e17b91a4ce8f5755734f2f9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Mar 2023 18:19:19 +0100 Subject: [PATCH 0028/2223] Update tests/ci/worker/ubuntu_ami_for_ci.sh Co-authored-by: Mikhail f. Shiryaev --- tests/ci/worker/ubuntu_ami_for_ci.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/ubuntu_ami_for_ci.sh b/tests/ci/worker/ubuntu_ami_for_ci.sh index 58797502419..f8d1645a54d 100644 --- a/tests/ci/worker/ubuntu_ami_for_ci.sh +++ b/tests/ci/worker/ubuntu_ami_for_ci.sh @@ -76,7 +76,7 @@ cat < /etc/docker/daemon.json EOT # Increase the limit on number of virtual memory mappings to aviod 'Cannot mmap' error -sysctl -w vm.max_map_count=2097152 +echo "vm.max_map_count = 2097152" > /etc/sysctl.d/01-increase-map-counts.conf systemctl restart docker From f2fbf2d61e8ede663ba37065d8ea8fe9b430de3e Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 6 Mar 2023 02:52:05 +0100 Subject: [PATCH 0029/2223] tcp protocol modification (min revision to be updated) --- src/Client/ClientBase.cpp | 4 ++++ src/Client/Connection.cpp | 5 +++++ src/Client/IServerConnection.h | 2 ++ src/Client/MultiplexedConnections.cpp | 2 ++ src/Client/Suggest.cpp | 1 + src/Common/DateLUT.h | 1 + src/Core/Protocol.h | 4 +++- src/Core/ProtocolDefines.h | 2 ++ src/Interpreters/executeQuery.cpp | 2 +- src/Server/TCPHandler.cpp | 12 ++++++++++++ src/Server/TCPHandler.h | 1 + 11 files changed, 34 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 96aff9aa304..65d04a6bb9d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1020,6 +1020,10 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_) onProfileEvents(packet.block); return true; + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + return true; + default: throw Exception( ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}", packet.type, connection->getDescription()); diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index eea007a8608..87e9e20e8f7 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -972,6 +972,11 @@ Packet Connection::receivePacket() res.block = receiveProfileEvents(); return res; + case Protocol::Server::TimezoneUpdate: + readStringBinary(server_timezone, *in); + res.server_timezone = server_timezone; + return res; + default: /// In unknown state, disconnect - to not leave unsynchronised connection. disconnect(); diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index cd4db8f5258..52382ff9d45 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -38,6 +38,8 @@ struct Packet ParallelReadRequest request; ParallelReadResponse response; + std::string server_timezone; + Packet() : type(Protocol::Server::Hello) {} }; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index cc260353339..668833b2a84 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -258,6 +258,7 @@ Packet MultiplexedConnections::drain() switch (packet.type) { + case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: @@ -339,6 +340,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac switch (packet.type) { + case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 7027f35d21a..4a29bead540 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -158,6 +158,7 @@ void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & t fillWordsFromBlock(packet.block); continue; + case Protocol::Server::TimezoneUpdate: case Protocol::Server::Progress: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index efbf56b59b2..f17fe772dbc 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -60,6 +60,7 @@ public: return date_lut.getImplementation(time_zone); } + static void setDefaultTimezone(const std::string & time_zone) { auto & date_lut = getInstance(); diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 86c0a851c60..97a2831ffe8 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -83,7 +83,8 @@ namespace Protocol ProfileEvents = 14, /// Packet with profile events from server. MergeTreeAllRangesAnnounecement = 15, MergeTreeReadTaskRequest = 16, /// Request from a MergeTree replica to a coordinator - MAX = MergeTreeReadTaskRequest, + TimezoneUpdate = 17, /// Receive server's (session-wide) default timezone + MAX = TimezoneUpdate, }; @@ -111,6 +112,7 @@ namespace Protocol "ProfileEvents", "MergeTreeAllRangesAnnounecement", "MergeTreeReadTaskRequest", + "TimezoneUpdate", }; return packet <= MAX ? data[packet] diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 3bbfb95f020..e56ae0305cc 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -72,3 +72,5 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS 54460 #define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461 + +#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54461 diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cda7ec2b0d3..85e623dc17d 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1280,7 +1280,7 @@ void executeQuery( QueryResultDetails result_details { .query_id = context->getClientInfo().current_query_id, - .timezone = DateLUT::instance("").getTimeZone(), + .timezone = DateLUT::instance().getTimeZone(), }; std::unique_ptr compressed_buffer; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a307b472a64..9bb11f34916 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -446,6 +446,7 @@ void TCPHandler::runImpl() sendSelectProfileEvents(); sendLogs(); + return false; }; @@ -483,6 +484,9 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); sendLogs(); + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES + && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) + sendTimezone(); sendEndOfStream(); } @@ -1035,6 +1039,14 @@ void TCPHandler::sendInsertProfileEvents() sendProfileEvents(); } +void TCPHandler::sendTimezone() +{ + writeVarUInt(Protocol::Server::TimezoneUpdate, *out); + writeStringBinary(DateLUT::instance().getTimeZone(), *out); + out->next(); +} + + bool TCPHandler::receiveProxyHeader() { if (in->eof()) diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index f06b0b060b3..b19f908bc27 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -262,6 +262,7 @@ private: void sendProfileEvents(); void sendSelectProfileEvents(); void sendInsertProfileEvents(); + void sendTimezone(); /// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled. void initBlockInput(); From 0706108b683ab5d67885b81a16b24a76c4d59513 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 6 Mar 2023 11:16:53 +0100 Subject: [PATCH 0030/2223] typo fix --- src/Interpreters/executeQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 85e623dc17d..435401796a0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -49,7 +49,6 @@ #include #include #include -#include #include #include #include From 57c5a637217779fbcc999cbaa5bd965f8892d092 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 6 Mar 2023 16:39:47 +0100 Subject: [PATCH 0031/2223] fix receive of timezone update on processing --- src/Client/ClientBase.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 65d04a6bb9d..7ca6bbed6ba 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1188,6 +1188,10 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de columns_description = ColumnsDescription::parse(packet.multistring_message[1]); return receiveSampleBlock(out, columns_description, parsed_query); + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + break; + default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Data, Exception or Log, got {})", @@ -1533,6 +1537,10 @@ bool ClientBase::receiveEndOfQuery() onProfileEvents(packet.block); break; + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + break; + default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Exception, EndOfStream, Log, Progress or ProfileEvents. Got {})", From d93937cc5e92ae4612259e9e57bca15489aabc8f Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 6 Mar 2023 16:45:38 +0100 Subject: [PATCH 0032/2223] increment protocol version --- src/Core/ProtocolDefines.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index e56ae0305cc..5483489d5c4 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -54,7 +54,7 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54461 +#define DBMS_TCP_PROTOCOL_VERSION 54462 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 @@ -73,4 +73,4 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461 -#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54461 +#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54462 From 3a918ae66a984451e0db0f56ffa6232b897ad62f Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 02:33:46 +0100 Subject: [PATCH 0033/2223] revert protocol changes, found better way --- src/Client/ClientBase.cpp | 25 ++++++++++++------------- src/Client/Connection.cpp | 5 ----- src/Client/MultiplexedConnections.cpp | 2 -- src/Client/Suggest.cpp | 1 - src/Core/Protocol.h | 4 +--- src/Core/ProtocolDefines.h | 4 +--- src/Server/TCPHandler.cpp | 10 ---------- src/Server/TCPHandler.h | 1 - 8 files changed, 14 insertions(+), 38 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 7ca6bbed6ba..09c510f01f3 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1020,10 +1020,6 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_) onProfileEvents(packet.block); return true; - case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); - return true; - default: throw Exception( ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}", packet.type, connection->getDescription()); @@ -1188,10 +1184,6 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de columns_description = ColumnsDescription::parse(packet.multistring_message[1]); return receiveSampleBlock(out, columns_description, parsed_query); - case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); - break; - default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Data, Exception or Log, got {})", @@ -1500,7 +1492,7 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents)) + while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents )) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); @@ -1537,10 +1529,6 @@ bool ClientBase::receiveEndOfQuery() onProfileEvents(packet.block); break; - case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); - break; - default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Exception, EndOfStream, Log, Progress or ProfileEvents. Got {})", @@ -1611,6 +1599,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.resetProgress(); profile_events.watch.restart(); + const std::string old_timezone = DateLUT::instance().getTimeZone(); + { /// Temporarily apply query settings to context. std::optional old_settings; @@ -1659,6 +1649,9 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin bool is_async_insert = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); + if (!global_context->getSettingsRef().timezone.toString().empty()) + DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); + /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert) { @@ -1693,6 +1686,10 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin query_parameters.insert_or_assign(name, value); global_context->addQueryParameters(set_query->query_parameters); + + if (!global_context->getSettingsRef().timezone.toString().empty()) + DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); + } if (const auto * use_query = parsed_query->as()) { @@ -1703,6 +1700,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin connection->setDefaultDatabase(new_database); } } + else + DateLUT::setDefaultTimezone(old_timezone); /// Always print last block (if it was not printed already) if (profile_events.last_block) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 87e9e20e8f7..eea007a8608 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -972,11 +972,6 @@ Packet Connection::receivePacket() res.block = receiveProfileEvents(); return res; - case Protocol::Server::TimezoneUpdate: - readStringBinary(server_timezone, *in); - res.server_timezone = server_timezone; - return res; - default: /// In unknown state, disconnect - to not leave unsynchronised connection. disconnect(); diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 668833b2a84..cc260353339 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -258,7 +258,6 @@ Packet MultiplexedConnections::drain() switch (packet.type) { - case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: @@ -340,7 +339,6 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac switch (packet.type) { - case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 4a29bead540..7027f35d21a 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -158,7 +158,6 @@ void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & t fillWordsFromBlock(packet.block); continue; - case Protocol::Server::TimezoneUpdate: case Protocol::Server::Progress: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 97a2831ffe8..86c0a851c60 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -83,8 +83,7 @@ namespace Protocol ProfileEvents = 14, /// Packet with profile events from server. MergeTreeAllRangesAnnounecement = 15, MergeTreeReadTaskRequest = 16, /// Request from a MergeTree replica to a coordinator - TimezoneUpdate = 17, /// Receive server's (session-wide) default timezone - MAX = TimezoneUpdate, + MAX = MergeTreeReadTaskRequest, }; @@ -112,7 +111,6 @@ namespace Protocol "ProfileEvents", "MergeTreeAllRangesAnnounecement", "MergeTreeReadTaskRequest", - "TimezoneUpdate", }; return packet <= MAX ? data[packet] diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 5483489d5c4..3bbfb95f020 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -54,7 +54,7 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54462 +#define DBMS_TCP_PROTOCOL_VERSION 54461 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 @@ -72,5 +72,3 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS 54460 #define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461 - -#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54462 diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9bb11f34916..617b084a149 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -484,9 +484,6 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); sendLogs(); - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES - && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) - sendTimezone(); sendEndOfStream(); } @@ -1039,13 +1036,6 @@ void TCPHandler::sendInsertProfileEvents() sendProfileEvents(); } -void TCPHandler::sendTimezone() -{ - writeVarUInt(Protocol::Server::TimezoneUpdate, *out); - writeStringBinary(DateLUT::instance().getTimeZone(), *out); - out->next(); -} - bool TCPHandler::receiveProxyHeader() { diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index b19f908bc27..f06b0b060b3 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -262,7 +262,6 @@ private: void sendProfileEvents(); void sendSelectProfileEvents(); void sendInsertProfileEvents(); - void sendTimezone(); /// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled. void initBlockInput(); From c859478db3a3964c49457e49bab62bdf975bed7f Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 02:36:02 +0100 Subject: [PATCH 0034/2223] upd --- src/Client/IServerConnection.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 52382ff9d45..cd4db8f5258 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -38,8 +38,6 @@ struct Packet ParallelReadRequest request; ParallelReadResponse response; - std::string server_timezone; - Packet() : type(Protocol::Server::Hello) {} }; From 5e7a861e688dea04ecfe9c54d30c642f65a28569 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 02:45:47 +0100 Subject: [PATCH 0035/2223] fix --- src/Server/TCPHandler.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 617b084a149..a307b472a64 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -446,7 +446,6 @@ void TCPHandler::runImpl() sendSelectProfileEvents(); sendLogs(); - return false; }; @@ -1036,7 +1035,6 @@ void TCPHandler::sendInsertProfileEvents() sendProfileEvents(); } - bool TCPHandler::receiveProxyHeader() { if (in->eof()) From a7a3c9d1a675743e776fde32c96ccd9bbfc94e46 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 02:52:42 +0100 Subject: [PATCH 0036/2223] fix style --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 09c510f01f3..25442c89f99 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1492,7 +1492,7 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents )) + while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents)) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); From e92501d5dd7a9f3a77ad38f8750432a2286e9f0b Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 13:02:02 +0100 Subject: [PATCH 0037/2223] update tests + exception --- src/Client/ClientBase.cpp | 28 +++++++++++++++++-- ...rence => 02674_timezone_setting.reference} | 0 ...setting.sql => 02674_timezone_setting.sql} | 4 +-- 3 files changed, 27 insertions(+), 5 deletions(-) rename tests/queries/0_stateless/{02668_timezone_setting.reference => 02674_timezone_setting.reference} (100%) rename tests/queries/0_stateless/{02668_timezone_setting.sql => 02674_timezone_setting.sql} (73%) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 25442c89f99..13f28806066 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -107,6 +107,7 @@ namespace ErrorCodes extern const int UNRECOGNIZED_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_PARSE_DATETIME; } } @@ -1599,6 +1600,9 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.resetProgress(); profile_events.watch.restart(); + /// A query may contain timezone setting. To handle this, old client-wide tz is saved here. + /// If timezone was set for a query, after its execution client tz will be back to old one. + /// If it was a settings query, new setting will be applied to client. const std::string old_timezone = DateLUT::instance().getTimeZone(); { @@ -1649,8 +1653,18 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin bool is_async_insert = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); - if (!global_context->getSettingsRef().timezone.toString().empty()) - DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); + /// pre-load timezone from (query) settings -- new timezone may also be specified in query. + try + { + if (!global_context->getSettingsRef().timezone.toString().empty()) + DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); + } + catch (Poco::Exception &) + { + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, + "Invalid time zone {} in client settings. Use `SET timezone = \'New/TZ\'` to set a proper timezone.", + global_context->getSettingsRef().timezone.toString()); + } /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert) @@ -1687,9 +1701,17 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin global_context->addQueryParameters(set_query->query_parameters); + try + { if (!global_context->getSettingsRef().timezone.toString().empty()) DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); - + } + catch (Poco::Exception &) + { + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, + "Invalid time zone {} in client settings. Use `SET timezone = \'New/TZ\'` to set a proper timezone.", + global_context->getSettingsRef().timezone.toString()); + } } if (const auto * use_query = parsed_query->as()) { diff --git a/tests/queries/0_stateless/02668_timezone_setting.reference b/tests/queries/0_stateless/02674_timezone_setting.reference similarity index 100% rename from tests/queries/0_stateless/02668_timezone_setting.reference rename to tests/queries/0_stateless/02674_timezone_setting.reference diff --git a/tests/queries/0_stateless/02668_timezone_setting.sql b/tests/queries/0_stateless/02674_timezone_setting.sql similarity index 73% rename from tests/queries/0_stateless/02668_timezone_setting.sql rename to tests/queries/0_stateless/02674_timezone_setting.sql index f331ab58307..51820fc2dca 100644 --- a/tests/queries/0_stateless/02668_timezone_setting.sql +++ b/tests/queries/0_stateless/02674_timezone_setting.sql @@ -5,5 +5,5 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric SET timezone = 'Europe/Zurich'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); -SET timezone = 'Абырвалг'; -select now(); -- { serverError POCO_EXCEPTION } \ No newline at end of file +SET timezone = 'Абырвалг'; -- { clientError CANNOT_PARSE_DATETIME } +select now(); -- { clientError CANNOT_PARSE_DATETIME } \ No newline at end of file From 1fd6e3f23b41dac6fde5b238e1a5da11a976b5ae Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 16:02:30 +0100 Subject: [PATCH 0038/2223] Revert "fix style" This reverts commit a7a3c9d1a675743e776fde32c96ccd9bbfc94e46. --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 13f28806066..cfef1a5d3fe 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1493,7 +1493,7 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents)) + while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents )) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); From 1ce697d8c06ce7f44e078f9b8809dcaa3e3ba8f8 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 16:05:23 +0100 Subject: [PATCH 0039/2223] Revert "revert protocol changes, found better way" This reverts commit 3a918ae66a984451e0db0f56ffa6232b897ad62f. --- src/Client/ClientBase.cpp | 47 +++++-------------- src/Client/Connection.cpp | 5 ++ src/Client/IServerConnection.h | 2 + src/Client/MultiplexedConnections.cpp | 2 + src/Client/Suggest.cpp | 1 + src/Core/Protocol.h | 4 +- src/Core/ProtocolDefines.h | 4 +- src/Server/TCPHandler.cpp | 12 +++++ src/Server/TCPHandler.h | 1 + ...rence => 02668_timezone_setting.reference} | 0 ...setting.sql => 02668_timezone_setting.sql} | 4 +- 11 files changed, 44 insertions(+), 38 deletions(-) rename tests/queries/0_stateless/{02674_timezone_setting.reference => 02668_timezone_setting.reference} (100%) rename tests/queries/0_stateless/{02674_timezone_setting.sql => 02668_timezone_setting.sql} (73%) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index cfef1a5d3fe..7ca6bbed6ba 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -107,7 +107,6 @@ namespace ErrorCodes extern const int UNRECOGNIZED_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_PARSE_DATETIME; } } @@ -1021,6 +1020,10 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_) onProfileEvents(packet.block); return true; + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + return true; + default: throw Exception( ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}", packet.type, connection->getDescription()); @@ -1185,6 +1188,10 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de columns_description = ColumnsDescription::parse(packet.multistring_message[1]); return receiveSampleBlock(out, columns_description, parsed_query); + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + break; + default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Data, Exception or Log, got {})", @@ -1493,7 +1500,7 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents )) + while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents)) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); @@ -1530,6 +1537,10 @@ bool ClientBase::receiveEndOfQuery() onProfileEvents(packet.block); break; + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + break; + default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Exception, EndOfStream, Log, Progress or ProfileEvents. Got {})", @@ -1600,11 +1611,6 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.resetProgress(); profile_events.watch.restart(); - /// A query may contain timezone setting. To handle this, old client-wide tz is saved here. - /// If timezone was set for a query, after its execution client tz will be back to old one. - /// If it was a settings query, new setting will be applied to client. - const std::string old_timezone = DateLUT::instance().getTimeZone(); - { /// Temporarily apply query settings to context. std::optional old_settings; @@ -1653,19 +1659,6 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin bool is_async_insert = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); - /// pre-load timezone from (query) settings -- new timezone may also be specified in query. - try - { - if (!global_context->getSettingsRef().timezone.toString().empty()) - DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); - } - catch (Poco::Exception &) - { - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, - "Invalid time zone {} in client settings. Use `SET timezone = \'New/TZ\'` to set a proper timezone.", - global_context->getSettingsRef().timezone.toString()); - } - /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert) { @@ -1700,18 +1693,6 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin query_parameters.insert_or_assign(name, value); global_context->addQueryParameters(set_query->query_parameters); - - try - { - if (!global_context->getSettingsRef().timezone.toString().empty()) - DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); - } - catch (Poco::Exception &) - { - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, - "Invalid time zone {} in client settings. Use `SET timezone = \'New/TZ\'` to set a proper timezone.", - global_context->getSettingsRef().timezone.toString()); - } } if (const auto * use_query = parsed_query->as()) { @@ -1722,8 +1703,6 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin connection->setDefaultDatabase(new_database); } } - else - DateLUT::setDefaultTimezone(old_timezone); /// Always print last block (if it was not printed already) if (profile_events.last_block) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index eea007a8608..87e9e20e8f7 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -972,6 +972,11 @@ Packet Connection::receivePacket() res.block = receiveProfileEvents(); return res; + case Protocol::Server::TimezoneUpdate: + readStringBinary(server_timezone, *in); + res.server_timezone = server_timezone; + return res; + default: /// In unknown state, disconnect - to not leave unsynchronised connection. disconnect(); diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index cd4db8f5258..52382ff9d45 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -38,6 +38,8 @@ struct Packet ParallelReadRequest request; ParallelReadResponse response; + std::string server_timezone; + Packet() : type(Protocol::Server::Hello) {} }; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index cc260353339..668833b2a84 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -258,6 +258,7 @@ Packet MultiplexedConnections::drain() switch (packet.type) { + case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: @@ -339,6 +340,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac switch (packet.type) { + case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 7027f35d21a..4a29bead540 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -158,6 +158,7 @@ void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & t fillWordsFromBlock(packet.block); continue; + case Protocol::Server::TimezoneUpdate: case Protocol::Server::Progress: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 86c0a851c60..97a2831ffe8 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -83,7 +83,8 @@ namespace Protocol ProfileEvents = 14, /// Packet with profile events from server. MergeTreeAllRangesAnnounecement = 15, MergeTreeReadTaskRequest = 16, /// Request from a MergeTree replica to a coordinator - MAX = MergeTreeReadTaskRequest, + TimezoneUpdate = 17, /// Receive server's (session-wide) default timezone + MAX = TimezoneUpdate, }; @@ -111,6 +112,7 @@ namespace Protocol "ProfileEvents", "MergeTreeAllRangesAnnounecement", "MergeTreeReadTaskRequest", + "TimezoneUpdate", }; return packet <= MAX ? data[packet] diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 3bbfb95f020..5483489d5c4 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -54,7 +54,7 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54461 +#define DBMS_TCP_PROTOCOL_VERSION 54462 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 @@ -72,3 +72,5 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS 54460 #define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461 + +#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54462 diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a307b472a64..9bb11f34916 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -446,6 +446,7 @@ void TCPHandler::runImpl() sendSelectProfileEvents(); sendLogs(); + return false; }; @@ -483,6 +484,9 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); sendLogs(); + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES + && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) + sendTimezone(); sendEndOfStream(); } @@ -1035,6 +1039,14 @@ void TCPHandler::sendInsertProfileEvents() sendProfileEvents(); } +void TCPHandler::sendTimezone() +{ + writeVarUInt(Protocol::Server::TimezoneUpdate, *out); + writeStringBinary(DateLUT::instance().getTimeZone(), *out); + out->next(); +} + + bool TCPHandler::receiveProxyHeader() { if (in->eof()) diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index f06b0b060b3..b19f908bc27 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -262,6 +262,7 @@ private: void sendProfileEvents(); void sendSelectProfileEvents(); void sendInsertProfileEvents(); + void sendTimezone(); /// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled. void initBlockInput(); diff --git a/tests/queries/0_stateless/02674_timezone_setting.reference b/tests/queries/0_stateless/02668_timezone_setting.reference similarity index 100% rename from tests/queries/0_stateless/02674_timezone_setting.reference rename to tests/queries/0_stateless/02668_timezone_setting.reference diff --git a/tests/queries/0_stateless/02674_timezone_setting.sql b/tests/queries/0_stateless/02668_timezone_setting.sql similarity index 73% rename from tests/queries/0_stateless/02674_timezone_setting.sql rename to tests/queries/0_stateless/02668_timezone_setting.sql index 51820fc2dca..f331ab58307 100644 --- a/tests/queries/0_stateless/02674_timezone_setting.sql +++ b/tests/queries/0_stateless/02668_timezone_setting.sql @@ -5,5 +5,5 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric SET timezone = 'Europe/Zurich'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); -SET timezone = 'Абырвалг'; -- { clientError CANNOT_PARSE_DATETIME } -select now(); -- { clientError CANNOT_PARSE_DATETIME } \ No newline at end of file +SET timezone = 'Абырвалг'; +select now(); -- { serverError POCO_EXCEPTION } \ No newline at end of file From 067bfb884406d05722edec2d4fef82c2b65844ec Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 17:47:14 +0000 Subject: [PATCH 0040/2223] Add new function generateRandomStructure --- .../functions/other-functions.md | 70 ++++ .../sql-reference/table-functions/generate.md | 14 + src/DataTypes/NestedUtils.cpp | 15 +- src/DataTypes/NestedUtils.h | 3 + src/Functions/generateRandomStructure.cpp | 343 ++++++++++++++++++ src/Storages/StorageGenerateRandom.cpp | 4 +- .../TableFunctionGenerateRandom.cpp | 6 +- .../02584_compressor_codecs.reference | 14 + .../0_stateless/02584_compressor_codecs.sh | 34 ++ .../02586_generate_random_structure.reference | 4 + .../02586_generate_random_structure.sql | 10 + 11 files changed, 512 insertions(+), 5 deletions(-) create mode 100644 src/Functions/generateRandomStructure.cpp create mode 100644 tests/queries/0_stateless/02584_compressor_codecs.reference create mode 100755 tests/queries/0_stateless/02584_compressor_codecs.sh create mode 100644 tests/queries/0_stateless/02586_generate_random_structure.reference create mode 100644 tests/queries/0_stateless/02586_generate_random_structure.sql diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 7146484361e..2f46df07b0a 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2609,3 +2609,73 @@ Result: │ 286 │ └──────────────────────────┘ ``` + +## generateRandomStructure + +Generates random table structure in a format `column1_name column1_type, column2_name column2_type, ...`. + +**Syntax** + +``` sql +generateRandomStructure([number_of_columns, seed]) +``` + +**Arguments** + +- `number_of_columns` — The desired number of columns in the result table structure. If set to 0, the number of columns will be random from 1 to 128. Default value - 0. +- `seed` - Random seed to produce stable results. If seed is not specified, it is randomly generated. + +All arguments must be constant. + +**Returned value** + +- Randomly generated table structure. + +Type: [String](../../sql-reference/data-types/string.md). + +**Examples** + +Query: + +``` sql +SELECT generateRandomStructure() +``` + +Result: + +``` text +┌─generateRandomStructure()─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ c1 Decimal32(5), c2 Date, c3 Tuple(LowCardinality(String), Int128, UInt64, UInt16, UInt8, IPv6), c4 Array(UInt128), c5 UInt32, c6 IPv4, c7 Decimal256(64), c8 Decimal128(3), c9 UInt256, c10 UInt64, c11 DateTime │ +└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT generateRandomStructure(1) +``` + +Result: + +``` text +┌─generateRandomStructure(1)─┐ +│ c1 Map(UInt256, UInt16) │ +└────────────────────────────┘ +``` + +Query: + +``` sql +SELECT generateRandomStructure(0, 11) +``` + +Result: + +``` text +┌─generateRandomStructure(0, 11)──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ c1 Date32, c2 String, c3 IPv6, c4 DateTime, c5 UInt16, c6 Tuple(e1 UInt32, e2 Date, e3 Date, e4 IPv6, e5 Nested(e1 DateTime, e2 FixedString(110), e3 Int256, e4 Array(Decimal64(4)), e5 Decimal128(18), e6 Enum16('v0' = 0, 'v1' = 1, 'v2' = 2, 'v3' = 3, 'v4' = 4)), e6 DateTime64(4)), c7 DateTime, c8 DateTime64(6), c9 Bool │ +└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. + diff --git a/docs/en/sql-reference/table-functions/generate.md b/docs/en/sql-reference/table-functions/generate.md index b53ccdd42b5..7e39786c256 100644 --- a/docs/en/sql-reference/table-functions/generate.md +++ b/docs/en/sql-reference/table-functions/generate.md @@ -53,5 +53,19 @@ SELECT * FROM random; └──────────────────────────────┴──────────────┴────────────────────────────────────────────────────────────────────┘ ``` +In combination with [generateRandomStructure](../../sql-reference/functions/other-functions.md#generateRandomStructure): + +```sql +SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 3; +``` + +```text +┌─────────────────────────c1─┬─────c2─┬───────────────────c3─┬───────────────────────────────────────c4─┐ +│ 2085-07-05 23:48:43.345759 │ -20656 │ 1632406185424686785 │ -210464718903845545171230673454802.15238 │ +│ 1971-07-17 16:32:36.390777 │ -27071 │ -1553021742787219162 │ 1095158319964381336405161704296125.08074 │ +│ 2024-02-19 13:14:32.902513 │ 24913 │ 7727442383333447640 │ 1090748832613398997057187200834127.07109 │ +└────────────────────────────┴────────┴──────────────────────┴──────────────────────────────────────────┘ +``` + ## Related content - Blog: [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse) diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index f029ac6ba27..9ee803c4235 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -71,7 +71,7 @@ std::string extractTableName(const std::string & nested_name) } -Block flatten(const Block & block) +static Block flattenImpl(const Block & block, bool flatten_named_tuple) { Block res; @@ -114,7 +114,7 @@ Block flatten(const Block & block) else res.insert(elem); } - else if (const DataTypeTuple * type_tuple = typeid_cast(elem.type.get())) + else if (const DataTypeTuple * type_tuple = typeid_cast(elem.type.get()); type_tuple && flatten_named_tuple) { if (type_tuple->haveExplicitNames()) { @@ -143,6 +143,17 @@ Block flatten(const Block & block) return res; } +Block flatten(const Block & block) +{ + return flattenImpl(block, true); +} + + +Block flattenArrayOfTuples(const Block & block) +{ + return flattenImpl(block, false); +} + namespace { diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index 90fdd683493..e009ceb18fe 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -23,6 +23,9 @@ namespace Nested /// 2) For an Array with named Tuple element column, a Array(Tuple(x ..., y ..., ...)), replace it with multiple Array Columns, a.x ..., a.y ..., ... Block flatten(const Block & block); + /// Same as flatten but only for Array with named Tuple element column. + Block flattenArrayOfTuples(const Block & block); + /// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column. NamesAndTypesList collect(const NamesAndTypesList & names_and_types); diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp new file mode 100644 index 00000000000..c27c8428cb0 --- /dev/null +++ b/src/Functions/generateRandomStructure.cpp @@ -0,0 +1,343 @@ +#include "config.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; +} + +class FunctionGenerateRandomStructure : public IFunction +{ +private: + enum class SimpleTypes + { + Int8, + UInt8, + Bool, + Int16, + UInt16, + Int32, + UInt32, + Int64, + UInt64, + Int128, + UInt128, + Int256, + UInt256, + Float32, + Float64, + DateTime64, + Decimal32, + Decimal64, + Decimal128, + Decimal256, + Date, + Date32, + DateTime, + String, + FixedString, + Enum8, + Enum16, + IPv4, + IPv6, + }; + + enum class ComplexTypes + { + Nullable, + LowCardinality, + Array, + Tuple, + Map, + Nested, + }; + + enum class MapKeyTypes + { + Int8, + UInt8, + Bool, + Int16, + UInt16, + Int32, + UInt32, + Int64, + UInt64, + Int128, + UInt128, + Int256, + UInt256, + Date, + Date32, + DateTime, + String, + FixedString, + }; + + static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; + static constexpr size_t MAX_TUPLE_ELEMENTS = 16; + static constexpr size_t MAX_DATETIME64_PRECISION = 9; + static constexpr size_t MAX_DECIMAL32_PRECISION = 9; + static constexpr size_t MAX_DECIMAL64_PRECISION = 18; + static constexpr size_t MAX_DECIMAL128_PRECISION = 38; + static constexpr size_t MAX_DECIMAL256_PRECISION = 76; + static constexpr size_t MAX_DEPTH = 32; + +public: + static constexpr auto name = "generateRandomStructure"; + + static FunctionPtr create(ContextPtr /*context*/) + { + return std::make_shared(); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isVariadic() const override { return true; } + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() > 2) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 0, 1 or 2.", + getName(), arguments.size()); + + if (arguments.size() > 1 && !isUnsignedInteger(arguments[0])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the first argument of function {}, expected unsigned integer", + arguments[0]->getName(), + getName()); + } + + if (arguments.size() > 2 && !isUnsignedInteger(arguments[1])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the second argument of function {}, expected unsigned integer", + arguments[1]->getName(), + getName()); + } + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + size_t seed = randomSeed(); + size_t number_of_columns = 0; + + if (!arguments.empty()) + { + const auto & first_arg = arguments[0]; + + if (!isUnsignedInteger(first_arg.type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the first argument of function {}, expected unsigned integer", + first_arg.type->getName(), + getName()); + + number_of_columns = first_arg.column->getUInt(0); + if (number_of_columns > MAX_NUMBER_OF_COLUMNS) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Maximum allowed number of columns is {}, got {}", MAX_NUMBER_OF_COLUMNS, number_of_columns); + + if (arguments.size() == 2) + { + const auto & second_arg = arguments[1]; + + if (!isUnsignedInteger(second_arg.type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the second argument of function {}, expected unsigned integer", + second_arg.type->getName(), + getName()); + + seed = second_arg.column->getUInt(0); + } + } + + pcg64 rng(seed); + if (number_of_columns == 0) + number_of_columns = generateNumberOfColumns(rng); + + auto col_res = ColumnString::create(); + String generated_structure = ""; + for (size_t i = 0; i != number_of_columns; ++i) + { + if (i != 0) + generated_structure += ", "; + auto type = generateRandomType(rng); + generated_structure += "c" + std::to_string(i + 1) + " " + type; + } + col_res->insert(generated_structure); + return ColumnConst::create(std::move(col_res), input_rows_count); + } + +private: + + size_t generateNumberOfColumns(pcg64 & rng) const + { + return rng() % MAX_NUMBER_OF_COLUMNS + 1; + } + + String generateRandomType(pcg64 & rng, bool allow_complex_types = true, size_t depth = 0) const + { + constexpr size_t simple_types_size = magic_enum::enum_count(); + constexpr size_t complex_types_size = magic_enum::enum_count(); + size_t type_index; + if (allow_complex_types) + type_index = rng() % (simple_types_size + complex_types_size); + else + type_index = rng() % simple_types_size; + + if (type_index < simple_types_size) + { + auto type = magic_enum::enum_value(type_index); + switch (type) + { + case SimpleTypes::FixedString: + return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; + case SimpleTypes::DateTime64: + return "DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")"; + case SimpleTypes::Decimal32: + return "Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")"; + case SimpleTypes::Decimal64: + return "Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")"; + case SimpleTypes::Decimal128: + return "Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")"; + case SimpleTypes::Decimal256: + return "Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")"; + case SimpleTypes::Enum8: + return "Enum8(" + generateEnumValues(rng) + ")"; + case SimpleTypes::Enum16: + return "Enum16(" + generateEnumValues(rng) + ")"; + default: + return String(magic_enum::enum_name(type)); + } + } + + auto complex_type = magic_enum::enum_value(type_index - simple_types_size); + switch (complex_type) + { + case ComplexTypes::LowCardinality: + return "LowCardinality(" + generateLowCardinalityNestedType(rng) + ")"; + case ComplexTypes::Nullable: + return "Nullable(" + generateRandomType(rng, false, depth + 1) + ")"; + case ComplexTypes::Array: + return "Array(" + generateRandomType(rng, true, depth + 1) + ")"; + case ComplexTypes::Map: + return "Map(" + generateMapKeyType(rng) + ", " + generateRandomType(rng, true, depth + 1) + ")"; + case ComplexTypes::Tuple: + { + size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; + bool named_tuple = rng() % 2; + String tuple_type = "Tuple("; + for (size_t i = 0; i != elements; ++i) + { + if (i != 0) + tuple_type += ", "; + if (named_tuple) + tuple_type += "e" + std::to_string(i + 1) + " "; + tuple_type += generateRandomType(rng, true, depth + 1); + } + return tuple_type + ")"; + } + case ComplexTypes::Nested: + { + size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; + String nested_type = "Nested("; + for (size_t i = 0; i != elements; ++i) + { + if (i != 0) + nested_type += ", "; + nested_type += "e" + std::to_string(i + 1) + " " + generateRandomType(rng, true, depth + 1); + } + return nested_type + ")"; + } + } + } + + String generateMapKeyType(pcg64 & rng) const + { + constexpr size_t map_keys_types_size = magic_enum::enum_count(); + auto type = magic_enum::enum_value(rng() % map_keys_types_size); + if (type == MapKeyTypes::FixedString) + return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; + return String(magic_enum::enum_name(type)); + } + + String generateLowCardinalityNestedType(pcg64 & rng) const + { + /// Support only String and FixedString. + String nested_type; + if (rng() % 2) + nested_type = "String"; + else + nested_type = "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; + return rng() % 2 ? nested_type : "Nullable(" + nested_type + ")"; + } + + String generateEnumValues(pcg64 & rng) const + { + /// Don't generate big enums, because it will lead to really big strings + /// and slowness of this function, and it can lead to `Max query size exceeded` + /// while using this function with generateRandom. + ssize_t num_values = rng() % 16 + 1; + String result; + for (ssize_t i = 0; i != num_values; ++i) + { + if (i != 0) + result += ", "; + result += "'v" + std::to_string(i) + "' = " + std::to_string(i); + } + return result; + } +}; + + +REGISTER_FUNCTION(GenerateRandomStructure) +{ + factory.registerFunction( + { + R"( +Generates a random table structure. +This function takes an optional constant argument, the number of column in the result structure. +If argument is now specified, the number of columns is random. The maximum number of columns is 1024. +The function returns a value of type String. +)", + Documentation::Examples{ + {"random", "SELECT generateRandomStructure()"}, + {"with specified number of arguments", "SELECT generateRandomStructure(10)"}}, + Documentation::Categories{"Random"} + }, + FunctionFactory::CaseSensitive); +} + +} diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 601306bd1bf..f616313a595 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -422,7 +422,7 @@ class GenerateSource : public ISource { public: GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, ContextPtr context_) - : ISource(Nested::flatten(prepareBlockToFill(block_header_))) + : ISource(Nested::flattenArrayOfTuples(prepareBlockToFill(block_header_))) , block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_) , block_to_fill(std::move(block_header_)), rng(random_seed_), context(context_) {} @@ -437,7 +437,7 @@ protected: for (const auto & elem : block_to_fill) columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context)); - columns = Nested::flatten(block_to_fill.cloneWithColumns(columns)).getColumns(); + columns = Nested::flattenArrayOfTuples(block_to_fill.cloneWithColumns(columns)).getColumns(); return {std::move(columns), block_size}; } diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 5f1a13d8857..12cbda334a3 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include "registerTableFunctions.h" @@ -28,7 +29,7 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } -void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr /*context*/) +void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr context) { ASTs & args_func = ast_function->children; @@ -45,6 +46,9 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co "Table function '{}' requires at most four arguments: " " structure, [random_seed, max_string_length, max_array_length].", getName()); + /// Allow constant expression for structure argument, it can be generated using generateRandomStructure function. + args[0] = evaluateConstantExpressionAsLiteral(args[0], context); + // All the arguments must be literals. for (const auto & arg : args) { diff --git a/tests/queries/0_stateless/02584_compressor_codecs.reference b/tests/queries/0_stateless/02584_compressor_codecs.reference new file mode 100644 index 00000000000..23751ef6c1f --- /dev/null +++ b/tests/queries/0_stateless/02584_compressor_codecs.reference @@ -0,0 +1,14 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02584_compressor_codecs.sh b/tests/queries/0_stateless/02584_compressor_codecs.sh new file mode 100755 index 00000000000..930d101466b --- /dev/null +++ b/tests/queries/0_stateless/02584_compressor_codecs.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "Hello, World!" > 02584_test_data + +$CLICKHOUSE_COMPRESSOR --codec 'Delta' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; +$CLICKHOUSE_COMPRESSOR --codec 'Delta(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'Delta([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'Delta(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; + +$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; +$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; + +$CLICKHOUSE_COMPRESSOR --codec 'Gorilla' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; +$CLICKHOUSE_COMPRESSOR --codec 'Gorilla(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'Gorilla([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'Gorilla(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; + +$CLICKHOUSE_COMPRESSOR --codec 'FPC' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; +$CLICKHOUSE_COMPRESSOR --codec 'FPC(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; +$CLICKHOUSE_COMPRESSOR --codec 'FPC(5, 1)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'FPC([1,2,3])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; +$CLICKHOUSE_COMPRESSOR --codec 'FPC(5, 4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; + + +$CLICKHOUSE_COMPRESSOR --codec 'T64' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_SYNTAX_FOR_CODEC_TYPE"; + +rm 02584_test_data 02584_test_out + diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference new file mode 100644 index 00000000000..335c5807c35 --- /dev/null +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -0,0 +1,4 @@ +c1 Int256, c2 Bool, c3 Int16, c4 Map(Int64, Array(Bool)), c5 Enum16(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4, \'v5\' = 5, \'v6\' = 6, \'v7\' = 7, \'v8\' = 8, \'v9\' = 9, \'v10\' = 10) +String +Const(String) +2085-07-05 23:48:43.345759 10105 1535011673144902513 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql new file mode 100644 index 00000000000..c67196569af --- /dev/null +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -0,0 +1,10 @@ +select generateRandomStructure(5, 42); +select toTypeName(generateRandomStructure(5, 42)); +select toColumnTypeName(generateRandomStructure(5, 42)); +SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 1; + +select generateRandomStructure(5, 42, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select generateRandomStructure('5'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN} From e5f6ced3d7d87a11e38f1a1499544fc7c2ad2e05 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 18:01:37 +0000 Subject: [PATCH 0041/2223] Fix style --- src/Functions/generateRandomStructure.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index c27c8428cb0..11b6f7877e1 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -18,7 +18,6 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; } @@ -242,7 +241,7 @@ private: return String(magic_enum::enum_name(type)); } } - + auto complex_type = magic_enum::enum_value(type_index - simple_types_size); switch (complex_type) { From 66eb06d8399144b40ac94a4b0944bb15b37ce2c9 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 20:15:32 +0000 Subject: [PATCH 0042/2223] Better --- .../functions/other-functions.md | 23 +- src/Functions/generateRandomStructure.cpp | 340 ++++++++++++------ .../02586_generate_random_structure.reference | 7 +- .../02586_generate_random_structure.sql | 13 +- 4 files changed, 267 insertions(+), 116 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 2f46df07b0a..5f6b6e5687d 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2617,13 +2617,15 @@ Generates random table structure in a format `column1_name column1_type, column2 **Syntax** ``` sql -generateRandomStructure([number_of_columns, seed]) +generateRandomStructure([number_of_columns, seed, allow_big_numbers, allow_enums]) ``` **Arguments** -- `number_of_columns` — The desired number of columns in the result table structure. If set to 0, the number of columns will be random from 1 to 128. Default value - 0. -- `seed` - Random seed to produce stable results. If seed is not specified, it is randomly generated. +- `number_of_columns` — The desired number of columns in the result table structure. If set to 0 or `Null`, the number of columns will be random from 1 to 128. Default value: `Null`. +- `seed` - Random seed to produce stable results. If seed is not specified or set to `Null`, it is randomly generated. +- `allow_big_numbers` - Indicates if big number types (`Int128/UInt128/Int256/UInt256/Decimal128/Decinal256`) can be generated. Default value: true. +- `allow_enums` - Indicates if enum types can be generated. Default - true. All arguments must be constant. @@ -2666,7 +2668,7 @@ Result: Query: ``` sql -SELECT generateRandomStructure(0, 11) +SELECT generateRandomStructure(Null, 11) ``` Result: @@ -2677,5 +2679,18 @@ Result: └─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +``` sql +SELECT generateRandomStructure(6, Null, false, false) +``` + +Result: + +``` text +┌─generateRandomStructure(6, NULL, false, false)───────────────────────────────────────────────────────┐ +│ c1 Float32, c2 Tuple(DateTime), c3 UInt8, c4 UInt16, c5 Int64, c6 Array(Map(FixedString(108), Date)) │ +└──────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + + This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index 11b6f7877e1..f95b4a279de 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes class FunctionGenerateRandomStructure : public IFunction { private: - enum class SimpleTypes + enum class Type { Int8, UInt8, @@ -35,30 +35,26 @@ private: UInt32, Int64, UInt64, - Int128, - UInt128, - Int256, - UInt256, Float32, Float64, DateTime64, Decimal32, Decimal64, - Decimal128, - Decimal256, Date, Date32, DateTime, String, FixedString, - Enum8, - Enum16, IPv4, IPv6, - }; - - enum class ComplexTypes - { + Int128, + UInt128, + Int256, + UInt256, + Decimal128, + Decimal256, + Enum8, + Enum16, Nullable, LowCardinality, Array, @@ -66,27 +62,74 @@ private: Map, Nested, }; - - enum class MapKeyTypes + + static constexpr std::array simple_types { - Int8, - UInt8, - Bool, - Int16, - UInt16, - Int32, - UInt32, - Int64, - UInt64, - Int128, - UInt128, - Int256, - UInt256, - Date, - Date32, - DateTime, - String, - FixedString, + Type::Int8, + Type::UInt8, + Type::Bool, + Type::Int16, + Type::UInt16, + Type::Int32, + Type::UInt32, + Type::Int64, + Type::UInt64, + Type::Float32, + Type::Float64, + Type::DateTime64, + Type::Decimal32, + Type::Decimal64, + Type::Date, + Type::Date32, + Type::DateTime, + Type::String, + Type::FixedString, + Type::IPv4, + Type::IPv6, + }; + + static constexpr std::array big_number_types + { + Type::Int128, + Type::UInt128, + Type::Int256, + Type::UInt256, + Type::Decimal128, + Type::Decimal256, + }; + + static constexpr std::array enum_types + { + Type::Enum8, + Type::Enum16, + }; + + static constexpr std::array complex_types + { + Type::Nullable, + Type::LowCardinality, + Type::Array, + Type::Tuple, + Type::Map, + Type::Nested, + }; + + static constexpr std::array map_key_types + { + Type::Int8, + Type::UInt8, + Type::Bool, + Type::Int16, + Type::UInt16, + Type::Int32, + Type::UInt32, + Type::Int64, + Type::UInt64, + Type::Date, + Type::Date32, + Type::DateTime, + Type::String, + Type::FixedString, }; static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; @@ -114,48 +157,66 @@ public: bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1, 2, 3}; } + bool useDefaultImplementationForConstants() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() > 2) + if (arguments.size() > 4) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 0, 1 or 2.", + "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 4", getName(), arguments.size()); - if (arguments.size() > 1 && !isUnsignedInteger(arguments[0])) + if (!arguments.empty() && !isUnsignedInteger(arguments[0]) && !arguments[0]->onlyNull()) { throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the first argument of function {}, expected unsigned integer", + "Illegal type {} of the first argument of function {}, expected unsigned integer or Null", arguments[0]->getName(), getName()); } - if (arguments.size() > 2 && !isUnsignedInteger(arguments[1])) + if (arguments.size() > 1 && !isUnsignedInteger(arguments[1]) && !arguments[1]->onlyNull()) { throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {}, expected unsigned integer", + "Illegal type {} of the second argument of function {}, expected unsigned integer or Null", arguments[1]->getName(), getName()); } + if (arguments.size() > 2 && !isUInt8(arguments[2])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the third argument of function {}, expected UInt8", + arguments[2]->getName(), + getName()); + } + + if (arguments.size() > 3 && !isUInt8(arguments[3])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the fourth argument of function {}, expected UInt8", + arguments[3]->getName(), + getName()); + } + return std::make_shared(); } - bool useDefaultImplementationForConstants() const override { return false; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { size_t seed = randomSeed(); size_t number_of_columns = 0; - if (!arguments.empty()) + if (!arguments.empty() && !arguments[0].column->onlyNull()) { const auto & first_arg = arguments[0]; - if (!isUnsignedInteger(first_arg.type)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, @@ -165,21 +226,55 @@ public: number_of_columns = first_arg.column->getUInt(0); if (number_of_columns > MAX_NUMBER_OF_COLUMNS) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Maximum allowed number of columns is {}, got {}", MAX_NUMBER_OF_COLUMNS, number_of_columns); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Maximum allowed number of columns is {}, got {}", + MAX_NUMBER_OF_COLUMNS, + number_of_columns); + } - if (arguments.size() == 2) - { - const auto & second_arg = arguments[1]; + if (arguments.size() > 1 && !arguments[1].column->onlyNull()) + { + const auto & second_arg = arguments[1]; - if (!isUnsignedInteger(second_arg.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {}, expected unsigned integer", - second_arg.type->getName(), - getName()); + if (!isUnsignedInteger(second_arg.type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the second argument of function {}, expected unsigned integer", + second_arg.type->getName(), + getName()); - seed = second_arg.column->getUInt(0); - } + seed = second_arg.column->getUInt(0); + } + + bool allow_big_numbers = true; + if (arguments.size() > 2) + { + const auto & third_arg = arguments[2]; + + if (!isUInt8(third_arg.type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the second argument of function {}, expected UInt8", + third_arg.type->getName(), + getName()); + + allow_big_numbers = third_arg.column->getBool(0); + } + + bool allow_enums = true; + if (arguments.size() > 3) + { + const auto & fourth_arg = arguments[3]; + + if (!isUInt8(fourth_arg.type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the fourth argument of function {}, expected UInt8", + fourth_arg.type->getName(), + getName()); + + allow_enums = fourth_arg.column->getBool(0); } pcg64 rng(seed); @@ -192,7 +287,7 @@ public: { if (i != 0) generated_structure += ", "; - auto type = generateRandomType(rng); + auto type = generateRandomType(rng, allow_big_numbers, allow_enums); generated_structure += "c" + std::to_string(i + 1) + " " + type; } col_res->insert(generated_structure); @@ -205,55 +300,55 @@ private: { return rng() % MAX_NUMBER_OF_COLUMNS + 1; } - - String generateRandomType(pcg64 & rng, bool allow_complex_types = true, size_t depth = 0) const + + String generateRandomType(pcg64 & rng, bool allow_big_numbers, bool allow_enums) const { - constexpr size_t simple_types_size = magic_enum::enum_count(); - constexpr size_t complex_types_size = magic_enum::enum_count(); - size_t type_index; - if (allow_complex_types) - type_index = rng() % (simple_types_size + complex_types_size); - else - type_index = rng() % simple_types_size; - - if (type_index < simple_types_size) + if (allow_big_numbers) { - auto type = magic_enum::enum_value(type_index); - switch (type) - { - case SimpleTypes::FixedString: - return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - case SimpleTypes::DateTime64: - return "DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")"; - case SimpleTypes::Decimal32: - return "Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")"; - case SimpleTypes::Decimal64: - return "Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")"; - case SimpleTypes::Decimal128: - return "Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")"; - case SimpleTypes::Decimal256: - return "Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")"; - case SimpleTypes::Enum8: - return "Enum8(" + generateEnumValues(rng) + ")"; - case SimpleTypes::Enum16: - return "Enum16(" + generateEnumValues(rng) + ")"; - default: - return String(magic_enum::enum_name(type)); - } + if (allow_enums) + return generateRandomTypeImpl(rng); + return generateRandomTypeImpl(rng); } - auto complex_type = magic_enum::enum_value(type_index - simple_types_size); - switch (complex_type) + if (allow_enums) + return generateRandomTypeImpl(rng); + return generateRandomTypeImpl(rng); + } + + + template + String generateRandomTypeImpl(pcg64 & rng, size_t depth = 0) const + { + constexpr auto all_types = getAllTypes(); + auto type = all_types[rng() % all_types.size()]; + + switch (type) { - case ComplexTypes::LowCardinality: + case Type::FixedString: + return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; + case Type::DateTime64: + return "DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")"; + case Type::Decimal32: + return "Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")"; + case Type::Decimal64: + return "Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")"; + case Type::Decimal128: + return "Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")"; + case Type::Decimal256: + return "Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")"; + case Type::Enum8: + return "Enum8(" + generateEnumValues(rng) + ")"; + case Type::Enum16: + return "Enum16(" + generateEnumValues(rng) + ")"; + case Type::LowCardinality: return "LowCardinality(" + generateLowCardinalityNestedType(rng) + ")"; - case ComplexTypes::Nullable: - return "Nullable(" + generateRandomType(rng, false, depth + 1) + ")"; - case ComplexTypes::Array: - return "Array(" + generateRandomType(rng, true, depth + 1) + ")"; - case ComplexTypes::Map: - return "Map(" + generateMapKeyType(rng) + ", " + generateRandomType(rng, true, depth + 1) + ")"; - case ComplexTypes::Tuple: + case Type::Nullable: + return "Nullable(" + generateRandomTypeImpl(rng, depth + 1) + ")"; + case Type::Array: + return "Array(" + generateRandomTypeImpl(rng, depth + 1) + ")"; + case Type::Map: + return "Map(" + generateMapKeyType(rng) + ", " + generateRandomTypeImpl(rng, depth + 1) + ")"; + case Type::Tuple: { size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; bool named_tuple = rng() % 2; @@ -264,11 +359,11 @@ private: tuple_type += ", "; if (named_tuple) tuple_type += "e" + std::to_string(i + 1) + " "; - tuple_type += generateRandomType(rng, true, depth + 1); + tuple_type += generateRandomTypeImpl(rng, depth + 1); } return tuple_type + ")"; } - case ComplexTypes::Nested: + case Type::Nested: { size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; String nested_type = "Nested("; @@ -276,20 +371,21 @@ private: { if (i != 0) nested_type += ", "; - nested_type += "e" + std::to_string(i + 1) + " " + generateRandomType(rng, true, depth + 1); + nested_type += "e" + std::to_string(i + 1) + " " + generateRandomTypeImpl(rng, depth + 1); } return nested_type + ")"; } + default: + return String(magic_enum::enum_name(type)); } } String generateMapKeyType(pcg64 & rng) const { - constexpr size_t map_keys_types_size = magic_enum::enum_count(); - auto type = magic_enum::enum_value(rng() % map_keys_types_size); - if (type == MapKeyTypes::FixedString) + auto type = map_key_types[rng() % map_key_types.size()]; + if (type == Type::FixedString) return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - return String(magic_enum::enum_name(type)); + return String(magic_enum::enum_name(type)); } String generateLowCardinalityNestedType(pcg64 & rng) const @@ -318,6 +414,36 @@ private: } return result; } + + template + static constexpr auto getAllTypes() + { + constexpr size_t result_size = simple_types.size() + big_number_types.size() * allow_big_numbers + enum_types.size() * allow_enums + complex_types.size() * allow_complex_types; + std::array result; + size_t index = 0; + for (size_t i = 0; i != simple_types.size(); ++i, ++index) + result[index] = simple_types[i]; + + if constexpr (allow_big_numbers) + { + for (size_t i = 0; i != big_number_types.size(); ++i, ++index) + result[index] = big_number_types[i]; + } + + if constexpr (allow_enums) + { + for (size_t i = 0; i != enum_types.size(); ++i, ++index) + result[index] = enum_types[i]; + } + + if constexpr (allow_complex_types) + { + for (size_t i = 0; i != complex_types.size(); ++i, ++index) + result[index] = complex_types[i]; + } + + return result; + } }; diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index 335c5807c35..181b7f0ec6f 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -1,4 +1,7 @@ -c1 Int256, c2 Bool, c3 Int16, c4 Map(Int64, Array(Bool)), c5 Enum16(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4, \'v5\' = 5, \'v6\' = 6, \'v7\' = 7, \'v8\' = 8, \'v9\' = 9, \'v10\' = 10) +c1 DateTime64(1), c2 Int16, c3 Map(Int64, Array(Bool)), c4 Decimal256(30), c5 Int128 +c1 Date, c2 Float64, c3 DateTime, c4 Map(Int64, DateTime64(4)), c5 Nested(e1 LowCardinality(String), e2 UInt32, e3 Enum16(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4, \'v5\' = 5, \'v6\' = 6, \'v7\' = 7), e4 Float32, e5 Date32, e6 Int64, e7 Decimal64(0), e8 UInt16, e9 Date32, e10 Int64, e11 DateTime64(0)) +c1 Nested(e1 Int64, e2 Int16, e3 Map(Int16, LowCardinality(Nullable(String))), e4 UInt8, e5 Nested(e1 Array(Nullable(Decimal64(12))), e2 DateTime64(1), e3 UInt64, e4 FixedString(61), e5 Decimal64(13), e6 UInt8), e6 Int8), c2 DateTime64(5), c3 IPv4, c4 String, c5 String +c1 DateTime64(1), c2 IPv4, c3 Nullable(Decimal128(37)), c4 UInt128, c5 Date String Const(String) -2085-07-05 23:48:43.345759 10105 1535011673144902513 +2106-02-02 121 17265 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index c67196569af..b524f6a5ff1 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -1,10 +1,17 @@ select generateRandomStructure(5, 42); +select generateRandomStructure(5, 42, false); +select generateRandomStructure(5, 42, false, false); +select generateRandomStructure(5, 42, true, false); select toTypeName(generateRandomStructure(5, 42)); select toColumnTypeName(generateRandomStructure(5, 42)); SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 1; -select generateRandomStructure(5, 42, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select generateRandomStructure(5, 42, false, false, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} select generateRandomStructure('5'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(5, 42, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(materialize(5), 42, false, false); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, materialize(42), false, false); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, materialize(false), false); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} From 0d430de54fef53e61f821d8b4bae684c816d4f2a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 20:18:17 +0000 Subject: [PATCH 0043/2223] Better --- src/Functions/generateRandomStructure.cpp | 31 ----------------------- 1 file changed, 31 deletions(-) diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index f95b4a279de..3b42fd99fb4 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -217,13 +217,6 @@ public: if (!arguments.empty() && !arguments[0].column->onlyNull()) { const auto & first_arg = arguments[0]; - if (!isUnsignedInteger(first_arg.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the first argument of function {}, expected unsigned integer", - first_arg.type->getName(), - getName()); - number_of_columns = first_arg.column->getUInt(0); if (number_of_columns > MAX_NUMBER_OF_COLUMNS) throw Exception( @@ -236,14 +229,6 @@ public: if (arguments.size() > 1 && !arguments[1].column->onlyNull()) { const auto & second_arg = arguments[1]; - - if (!isUnsignedInteger(second_arg.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {}, expected unsigned integer", - second_arg.type->getName(), - getName()); - seed = second_arg.column->getUInt(0); } @@ -251,14 +236,6 @@ public: if (arguments.size() > 2) { const auto & third_arg = arguments[2]; - - if (!isUInt8(third_arg.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {}, expected UInt8", - third_arg.type->getName(), - getName()); - allow_big_numbers = third_arg.column->getBool(0); } @@ -266,14 +243,6 @@ public: if (arguments.size() > 3) { const auto & fourth_arg = arguments[3]; - - if (!isUInt8(fourth_arg.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the fourth argument of function {}, expected UInt8", - fourth_arg.type->getName(), - getName()); - allow_enums = fourth_arg.column->getBool(0); } From 746d12e7ccf4780d6887aa3f961605b7f313b77b Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 20:19:28 +0000 Subject: [PATCH 0044/2223] Remove wrong tests --- .../02584_compressor_codecs.reference | 14 -------- .../0_stateless/02584_compressor_codecs.sh | 34 ------------------- 2 files changed, 48 deletions(-) delete mode 100644 tests/queries/0_stateless/02584_compressor_codecs.reference delete mode 100755 tests/queries/0_stateless/02584_compressor_codecs.sh diff --git a/tests/queries/0_stateless/02584_compressor_codecs.reference b/tests/queries/0_stateless/02584_compressor_codecs.reference deleted file mode 100644 index 23751ef6c1f..00000000000 --- a/tests/queries/0_stateless/02584_compressor_codecs.reference +++ /dev/null @@ -1,14 +0,0 @@ -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/02584_compressor_codecs.sh b/tests/queries/0_stateless/02584_compressor_codecs.sh deleted file mode 100755 index 930d101466b..00000000000 --- a/tests/queries/0_stateless/02584_compressor_codecs.sh +++ /dev/null @@ -1,34 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -echo "Hello, World!" > 02584_test_data - -$CLICKHOUSE_COMPRESSOR --codec 'Delta' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; -$CLICKHOUSE_COMPRESSOR --codec 'Delta(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'Delta([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'Delta(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; - -$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; -$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; - -$CLICKHOUSE_COMPRESSOR --codec 'Gorilla' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; -$CLICKHOUSE_COMPRESSOR --codec 'Gorilla(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'Gorilla([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'Gorilla(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; - -$CLICKHOUSE_COMPRESSOR --codec 'FPC' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; -$CLICKHOUSE_COMPRESSOR --codec 'FPC(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS"; -$CLICKHOUSE_COMPRESSOR --codec 'FPC(5, 1)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'FPC([1,2,3])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER"; -$CLICKHOUSE_COMPRESSOR --codec 'FPC(5, 4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out'; - - -$CLICKHOUSE_COMPRESSOR --codec 'T64' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_SYNTAX_FOR_CODEC_TYPE"; - -rm 02584_test_data 02584_test_out - From 48d701f0e70a10807d64eb041dcdc17562cf6731 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 20:27:14 +0000 Subject: [PATCH 0045/2223] Better docs --- src/Functions/generateRandomStructure.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index 3b42fd99fb4..0fb9ef0f2f3 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -422,13 +422,19 @@ REGISTER_FUNCTION(GenerateRandomStructure) { R"( Generates a random table structure. -This function takes an optional constant argument, the number of column in the result structure. -If argument is now specified, the number of columns is random. The maximum number of columns is 1024. +This function takes 4 optional constant arguments: the number of column in the result structure (random by default), +random seed (random by default), flag that indicates if big number types can be used (true by default), +flag that indicates if enum types can be used (true by default). +The maximum number of columns is 128. The function returns a value of type String. )", Documentation::Examples{ {"random", "SELECT generateRandomStructure()"}, - {"with specified number of arguments", "SELECT generateRandomStructure(10)"}}, + {"with specified number of arguments", "SELECT generateRandomStructure(10)"}, + {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, + {"without big number types", "SELECT generateRandomStructure(10, NULL, false)"}, + {"without enum types", "SELECT generateRandomStructure(10, NULL, false, false)"}, + }, Documentation::Categories{"Random"} }, FunctionFactory::CaseSensitive); From f9d9b1ee2379b608a3f13bb6e0087e2e4b292ba4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 10 Mar 2023 16:16:28 +0000 Subject: [PATCH 0046/2223] Add more options --- .../functions/other-functions.md | 19 +- src/Functions/generateRandomStructure.cpp | 262 +++++++++++------- .../02586_generate_random_structure.reference | 14 +- .../02586_generate_random_structure.sql | 16 +- 4 files changed, 201 insertions(+), 110 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 5f6b6e5687d..b4664f75e67 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2617,7 +2617,7 @@ Generates random table structure in a format `column1_name column1_type, column2 **Syntax** ``` sql -generateRandomStructure([number_of_columns, seed, allow_big_numbers, allow_enums]) +generateRandomStructure([number_of_columns, seed, allow_big_numbers, allow_enums, allow_decimals, allow_ip, allow_only_string_map_keys]) ``` **Arguments** @@ -2625,7 +2625,10 @@ generateRandomStructure([number_of_columns, seed, allow_big_numbers, allow_enums - `number_of_columns` — The desired number of columns in the result table structure. If set to 0 or `Null`, the number of columns will be random from 1 to 128. Default value: `Null`. - `seed` - Random seed to produce stable results. If seed is not specified or set to `Null`, it is randomly generated. - `allow_big_numbers` - Indicates if big number types (`Int128/UInt128/Int256/UInt256/Decimal128/Decinal256`) can be generated. Default value: true. -- `allow_enums` - Indicates if enum types can be generated. Default - true. +- `allow_enums` - Indicates if enum types (`Enum8/Enum16`) can be generated. Default - true. +- `allow_decimals` - Indicates if decimal types (`Decimal(P, S)`) can be generated. Default - true. +- `allow_ip` - Indicates if ip types (`IPv4/IPv6`) can be generated. Default - true. +- `allow_only_string_map_keys` - Indicates if Map key type can be only `String/FixedString`. Default - false. All arguments must be constant. @@ -2691,6 +2694,18 @@ Result: └──────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +``` sql +SELECT generateRandomStructure(6, Null, false, false, false, false, true) +``` + +Result: + +``` text +┌─generateRandomStructure(6, NULL, false, false, false, false, true)─────────────────────────────────────────────────┐ +│ c1 String, c2 UInt32, c3 Int32, c4 Int32, c5 Tuple(LowCardinality(Nullable(FixedString(101))), UInt8), c6 DateTime │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index 0fb9ef0f2f3..e6766e731b2 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -62,8 +62,8 @@ private: Map, Nested, }; - - static constexpr std::array simple_types + + static constexpr std::array simple_types { Type::Int8, Type::UInt8, @@ -76,24 +76,30 @@ private: Type::UInt64, Type::Float32, Type::Float64, - Type::DateTime64, - Type::Decimal32, - Type::Decimal64, Type::Date, Type::Date32, Type::DateTime, Type::String, Type::FixedString, - Type::IPv4, - Type::IPv6, }; - static constexpr std::array big_number_types + static constexpr std::array big_integer_types { Type::Int128, Type::UInt128, Type::Int256, Type::UInt256, + }; + + static constexpr std::array decimal_types + { + Type::DateTime64, + Type::Decimal32, + Type::Decimal64, + }; + + static constexpr std::array big_decimal_types + { Type::Decimal128, Type::Decimal256, }; @@ -104,6 +110,12 @@ private: Type::Enum16, }; + static constexpr std::array ip_types + { + Type::IPv4, + Type::IPv6, + }; + static constexpr std::array complex_types { Type::Nullable, @@ -132,6 +144,12 @@ private: Type::FixedString, }; + static constexpr std::array map_key_string_types + { + Type::String, + Type::FixedString + }; + static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; static constexpr size_t MAX_TUPLE_ELEMENTS = 16; static constexpr size_t MAX_DATETIME64_PRECISION = 9; @@ -157,53 +175,48 @@ public: bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1, 2, 3}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1, 2, 3, 4, 5, 6}; } bool useDefaultImplementationForConstants() const override { return false; } bool useDefaultImplementationForNulls() const override { return false; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() > 4) + if (arguments.size() > 7) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 4", + "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 7", getName(), arguments.size()); - if (!arguments.empty() && !isUnsignedInteger(arguments[0]) && !arguments[0]->onlyNull()) + for (size_t i = 0; i != 2; ++i) { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the first argument of function {}, expected unsigned integer or Null", - arguments[0]->getName(), - getName()); + if (arguments.size() == i) + break; + + if (!isUnsignedInteger(arguments[i]) && !arguments[i]->onlyNull()) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the {} argument of function {}, expected unsigned integer or Null", + i + 1, + arguments[i]->getName(), + getName()); + } } - if (arguments.size() > 1 && !isUnsignedInteger(arguments[1]) && !arguments[1]->onlyNull()) + for (size_t i = 2; i != 7; ++i) { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {}, expected unsigned integer or Null", - arguments[1]->getName(), - getName()); - } + if (arguments.size() <= i) + break; - if (arguments.size() > 2 && !isUInt8(arguments[2])) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the third argument of function {}, expected UInt8", - arguments[2]->getName(), - getName()); - } - - if (arguments.size() > 3 && !isUInt8(arguments[3])) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the fourth argument of function {}, expected UInt8", - arguments[3]->getName(), - getName()); + if (!isUInt8(arguments[i])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the {} argument of function {}, expected UInt8", + i + 1, + arguments[i]->getName(), + getName()); + } } return std::make_shared(); @@ -216,8 +229,7 @@ public: if (!arguments.empty() && !arguments[0].column->onlyNull()) { - const auto & first_arg = arguments[0]; - number_of_columns = first_arg.column->getUInt(0); + number_of_columns = arguments[0].column->getUInt(0); if (number_of_columns > MAX_NUMBER_OF_COLUMNS) throw Exception( ErrorCodes::BAD_ARGUMENTS, @@ -227,36 +239,39 @@ public: } if (arguments.size() > 1 && !arguments[1].column->onlyNull()) - { - const auto & second_arg = arguments[1]; - seed = second_arg.column->getUInt(0); - } + seed = arguments[1].column->getUInt(0); bool allow_big_numbers = true; if (arguments.size() > 2) - { - const auto & third_arg = arguments[2]; - allow_big_numbers = third_arg.column->getBool(0); - } + allow_big_numbers = arguments[2].column->getBool(0); bool allow_enums = true; if (arguments.size() > 3) - { - const auto & fourth_arg = arguments[3]; - allow_enums = fourth_arg.column->getBool(0); - } + allow_enums = arguments[3].column->getBool(0); + + bool allow_decimals = true; + if (arguments.size() > 4) + allow_decimals = arguments[4].column->getBool(0); + + bool allow_ip = true; + if (arguments.size() > 5) + allow_ip = arguments[5].column->getBool(0); + + bool only_string_map_key = false; + if (arguments.size() > 6) + only_string_map_key = arguments[6].column->getBool(0); pcg64 rng(seed); if (number_of_columns == 0) number_of_columns = generateNumberOfColumns(rng); auto col_res = ColumnString::create(); - String generated_structure = ""; + String generated_structure; for (size_t i = 0; i != number_of_columns; ++i) { if (i != 0) generated_structure += ", "; - auto type = generateRandomType(rng, allow_big_numbers, allow_enums); + auto type = generateRandomType(rng, allow_big_numbers, allow_enums, allow_decimals, allow_ip, only_string_map_key); generated_structure += "c" + std::to_string(i + 1) + " " + type; } col_res->insert(generated_structure); @@ -269,26 +284,37 @@ private: { return rng() % MAX_NUMBER_OF_COLUMNS + 1; } - - String generateRandomType(pcg64 & rng, bool allow_big_numbers, bool allow_enums) const + + /// Helper struct to call generateRandomTypeImpl with lots of bool template arguments without writing big if/else over all bool variables. + template + struct Dispatcher { - if (allow_big_numbers) + static auto call(const FunctionGenerateRandomStructure * f, pcg64 & rng) { - if (allow_enums) - return generateRandomTypeImpl(rng); - return generateRandomTypeImpl(rng); + return f->generateRandomTypeImpl(rng); } - if (allow_enums) - return generateRandomTypeImpl(rng); - return generateRandomTypeImpl(rng); - } - + template + static auto call(const FunctionGenerateRandomStructure * f, pcg64 & rng, bool b, Args1... ar1) + { + if (b) + return Dispatcher::call(f, rng, ar1...); + else + return Dispatcher::call(f, rng, ar1...); + } - template + friend FunctionGenerateRandomStructure; + }; + + String generateRandomType(pcg64 & rng, bool allow_big_numbers, bool allow_enums, bool allow_decimals, bool allow_ip, bool allow_only_string_map_keys) const + { + return Dispatcher<>::call(this, rng, allow_big_numbers, allow_enums, allow_decimals, allow_ip, allow_only_string_map_keys, true); + } + + template String generateRandomTypeImpl(pcg64 & rng, size_t depth = 0) const { - constexpr auto all_types = getAllTypes(); + constexpr auto all_types = getAllTypes(); auto type = all_types[rng() % all_types.size()]; switch (type) @@ -312,11 +338,21 @@ private: case Type::LowCardinality: return "LowCardinality(" + generateLowCardinalityNestedType(rng) + ")"; case Type::Nullable: - return "Nullable(" + generateRandomTypeImpl(rng, depth + 1) + ")"; + { + auto nested_type = generateRandomTypeImpl(rng, depth + 1); + return "Nullable(" + nested_type + ")"; + } case Type::Array: - return "Array(" + generateRandomTypeImpl(rng, depth + 1) + ")"; + { + auto nested_type = generateRandomTypeImpl(rng, depth + 1); + return "Array(" + nested_type + ")"; + } case Type::Map: - return "Map(" + generateMapKeyType(rng) + ", " + generateRandomTypeImpl(rng, depth + 1) + ")"; + { + auto key_type = generateMapKeyType(rng); + auto value_type = generateRandomTypeImpl(rng, depth + 1); + return "Map(" + key_type + ", " + value_type + ")"; + } case Type::Tuple: { size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; @@ -328,7 +364,7 @@ private: tuple_type += ", "; if (named_tuple) tuple_type += "e" + std::to_string(i + 1) + " "; - tuple_type += generateRandomTypeImpl(rng, depth + 1); + tuple_type += generateRandomTypeImpl(rng, depth + 1); } return tuple_type + ")"; } @@ -340,7 +376,8 @@ private: { if (i != 0) nested_type += ", "; - nested_type += "e" + std::to_string(i + 1) + " " + generateRandomTypeImpl(rng, depth + 1); + auto element_type = generateRandomTypeImpl(rng, depth + 1); + nested_type += "e" + std::to_string(i + 1) + " " + element_type; } return nested_type + ")"; } @@ -349,9 +386,15 @@ private: } } + template String generateMapKeyType(pcg64 & rng) const { - auto type = map_key_types[rng() % map_key_types.size()]; + Type type; + if constexpr (allow_only_string_map_keys) + type = map_key_string_types[rng() % map_key_string_types.size()]; + else + type = map_key_types[rng() % map_key_types.size()]; + if (type == Type::FixedString) return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; return String(magic_enum::enum_name(type)); @@ -384,33 +427,42 @@ private: return result; } - template + template static constexpr auto getAllTypes() { - constexpr size_t result_size = simple_types.size() + big_number_types.size() * allow_big_numbers + enum_types.size() * allow_enums + complex_types.size() * allow_complex_types; + constexpr size_t big_integer_types_size = big_integer_types.size() * allow_big_numbers; + constexpr size_t enum_types_size = enum_types.size() * allow_enums; + constexpr size_t decimal_types_size = decimal_types.size() * allow_decimals; + constexpr size_t big_decimal_types_size = big_decimal_types.size() * allow_big_numbers * allow_decimals; + constexpr size_t ip_types_size = ip_types.size() * allow_ip; + constexpr size_t complex_types_size = complex_types.size() * allow_complex_types; + + constexpr size_t result_size = simple_types.size() + big_integer_types_size + enum_types_size + decimal_types_size + + big_decimal_types_size + ip_types_size + complex_types_size; std::array result; size_t index = 0; + for (size_t i = 0; i != simple_types.size(); ++i, ++index) result[index] = simple_types[i]; - - if constexpr (allow_big_numbers) - { - for (size_t i = 0; i != big_number_types.size(); ++i, ++index) - result[index] = big_number_types[i]; - } - - if constexpr (allow_enums) - { - for (size_t i = 0; i != enum_types.size(); ++i, ++index) - result[index] = enum_types[i]; - } - - if constexpr (allow_complex_types) - { - for (size_t i = 0; i != complex_types.size(); ++i, ++index) - result[index] = complex_types[i]; - } - + + for (size_t i = 0; i != big_integer_types_size; ++i, ++index) + result[index] = big_integer_types[i]; + + for (size_t i = 0; i != enum_types_size; ++i, ++index) + result[index] = enum_types[i]; + + for (size_t i = 0; i != decimal_types_size; ++i, ++index) + result[index] = decimal_types[i]; + + for (size_t i = 0; i != big_decimal_types_size; ++i, ++index) + result[index] = big_decimal_types[i]; + + for (size_t i = 0; i != ip_types_size; ++i, ++index) + result[index] = ip_types[i]; + + for (size_t i = 0; i != complex_types_size; ++i, ++index) + result[index] = complex_types[i]; + return result; } }; @@ -422,9 +474,14 @@ REGISTER_FUNCTION(GenerateRandomStructure) { R"( Generates a random table structure. -This function takes 4 optional constant arguments: the number of column in the result structure (random by default), -random seed (random by default), flag that indicates if big number types can be used (true by default), -flag that indicates if enum types can be used (true by default). +This function takes 4 optional constant arguments: +1) the number of column in the result structure (random by default) +2) random seed (random by default) +3) flag that indicates if big number types can be used (true by default) +4) flag that indicates if enum types can be used (true by default) +5) flag that indicates if decimal types can be used (true by default) +6) flag that indicates if ip types (IPv4, IPv6) can be used (true by default) +7) flag that indicates if map keys should be only String or FixedString (false by default) The maximum number of columns is 128. The function returns a value of type String. )", @@ -433,7 +490,10 @@ The function returns a value of type String. {"with specified number of arguments", "SELECT generateRandomStructure(10)"}, {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, {"without big number types", "SELECT generateRandomStructure(10, NULL, false)"}, - {"without enum types", "SELECT generateRandomStructure(10, NULL, false, false)"}, + {"without enum types", "SELECT generateRandomStructure(10, NULL, true, false)"}, + {"without decimal types", "SELECT generateRandomStructure(10, NULL, true, true, false)"}, + {"without ip types", "SELECT generateRandomStructure(10, NULL, true, true, true, false)"}, + {"with only string mak key types", "SELECT generateRandomStructure(10, NULL, true, true, true, true, true)"}, }, Documentation::Categories{"Random"} }, diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index 181b7f0ec6f..76d89828071 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -1,7 +1,11 @@ -c1 DateTime64(1), c2 Int16, c3 Map(Int64, Array(Bool)), c4 Decimal256(30), c5 Int128 -c1 Date, c2 Float64, c3 DateTime, c4 Map(Int64, DateTime64(4)), c5 Nested(e1 LowCardinality(String), e2 UInt32, e3 Enum16(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4, \'v5\' = 5, \'v6\' = 6, \'v7\' = 7), e4 Float32, e5 Date32, e6 Int64, e7 Decimal64(0), e8 UInt16, e9 Date32, e10 Int64, e11 DateTime64(0)) -c1 Nested(e1 Int64, e2 Int16, e3 Map(Int16, LowCardinality(Nullable(String))), e4 UInt8, e5 Nested(e1 Array(Nullable(Decimal64(12))), e2 DateTime64(1), e3 UInt64, e4 FixedString(61), e5 Decimal64(13), e6 UInt8), e6 Int8), c2 DateTime64(5), c3 IPv4, c4 String, c5 String -c1 DateTime64(1), c2 IPv4, c3 Nullable(Decimal128(37)), c4 UInt128, c5 Date +c1 Date, c2 Bool, c3 Int16, c4 Map(Int64, Array(Bool)), c5 Decimal256(30) +c1 String, c2 Float64, c3 Enum8(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4), c4 UInt64, c5 Date +c1 Nested(e1 Int64, e2 Int16, e3 Map(Int16, LowCardinality(Nullable(String))), e4 UInt8, e5 Nested(e1 Array(Nullable(DateTime)), e2 Nullable(Bool), e3 UInt8, e4 UInt64, e5 Decimal64(6), e6 DateTime), e6 LowCardinality(Nullable(String))), c2 Date, c3 Int32, c4 IPv4, c5 Decimal32(8) +c1 Date, c2 UInt16, c3 UInt256, c4 Nullable(IPv4), c5 Nullable(Decimal64(17)) +c1 Array(Int64), c2 Map(String, LowCardinality(String)), c3 Date, c4 Map(Int64, UInt128), c5 UInt8 +c1 Date, c2 UInt16, c3 UInt256, c4 Nullable(Decimal128(37)), c5 DateTime64(8) +c1 Date, c2 Bool, c3 Int16, c4 Map(FixedString(120), Bool), c5 Decimal256(30) +c1 String, c2 Float64, c3 Enum8(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4), c4 UInt64, c5 Date String Const(String) -2106-02-02 121 17265 +1977-07-28 true 5389 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index b524f6a5ff1..061fbc24219 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -2,16 +2,28 @@ select generateRandomStructure(5, 42); select generateRandomStructure(5, 42, false); select generateRandomStructure(5, 42, false, false); select generateRandomStructure(5, 42, true, false); +select generateRandomStructure(5, 42, true, true, false); +select generateRandomStructure(5, 42, true, true, true, false); +select generateRandomStructure(5, 42, true, true, true, true, true); +select generateRandomStructure(5, 42, false, true, true); select toTypeName(generateRandomStructure(5, 42)); select toColumnTypeName(generateRandomStructure(5, 42)); -SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 1; +SELECT * FROM generateRandom(generateRandomStructure(3, 42), 42) LIMIT 1; -select generateRandomStructure(5, 42, false, false, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select generateRandomStructure(5, 42, false, false, false, false, true, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} select generateRandomStructure('5'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(5, 42, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(5, 42, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(5, 42, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(5, 42, false, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select generateRandomStructure(5, 42, false, false, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(materialize(5), 42, false, false); -- {serverError ILLEGAL_COLUMN} select generateRandomStructure(5, materialize(42), false, false); -- {serverError ILLEGAL_COLUMN} select generateRandomStructure(5, 42, materialize(false), false); -- {serverError ILLEGAL_COLUMN} select generateRandomStructure(5, 42, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, 42, false, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} + From 74bb7625a33d959e6bf93f918c8998ff3425faf6 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sun, 12 Mar 2023 18:09:49 +0800 Subject: [PATCH 0047/2223] allow empty column names in CSVWithNames/TSVWithNames --- src/Interpreters/TreeRewriter.cpp | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 4c134e175dc..66a08257a46 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -919,6 +919,26 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select RequiredSourceColumnsVisitor::Data columns_context; columns_context.visit_index_hint = visit_index_hint; + + if (auto * t = query->as()) + { + LOG_INFO(&Poco::Logger::get("TreeRewriter"), "lmx_00 enter"); + NameSet select_columns; + std::vector empty_name; + auto & select_query = *t; + for (size_t i = 0; i < select_query.select()->children.size(); i++) { + auto node = select_query.select()->children[i]; + if (auto* identifier = node->as()) { + LOG_INFO(&Poco::Logger::get("TreeRewriter"), "lmx_1 {}", identifier->name()); + if (identifier->name().empty()) { + empty_name.push_back(i); + select_query.select()->children.erase(select_query.select()->children.begin()+i); + } else { + select_columns.insert(identifier->name()); + } + } + } + } RequiredSourceColumnsVisitor(columns_context).visit(query); NameSet source_column_names; From 0ef6aea8280c3f8e7606dcb8e6d4b2186adabb68 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sun, 12 Mar 2023 18:22:10 +0800 Subject: [PATCH 0048/2223] fix --- src/Interpreters/TreeRewriter.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 66a08257a46..1ac8b62c11c 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -922,19 +922,12 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (auto * t = query->as()) { - LOG_INFO(&Poco::Logger::get("TreeRewriter"), "lmx_00 enter"); - NameSet select_columns; - std::vector empty_name; auto & select_query = *t; for (size_t i = 0; i < select_query.select()->children.size(); i++) { auto node = select_query.select()->children[i]; if (auto* identifier = node->as()) { - LOG_INFO(&Poco::Logger::get("TreeRewriter"), "lmx_1 {}", identifier->name()); if (identifier->name().empty()) { - empty_name.push_back(i); select_query.select()->children.erase(select_query.select()->children.begin()+i); - } else { - select_columns.insert(identifier->name()); } } } From b78e9dcc05d247d1ce9b88456e1edd808cc79c25 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sun, 12 Mar 2023 22:45:25 +0800 Subject: [PATCH 0049/2223] fix style --- src/Interpreters/TreeRewriter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 1ac8b62c11c..81da53756ec 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -923,9 +923,9 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (auto * t = query->as()) { auto & select_query = *t; - for (size_t i = 0; i < select_query.select()->children.size(); i++) { + for (size_t i = 0; i < select_query.select()->children.size(); ++i) { auto node = select_query.select()->children[i]; - if (auto* identifier = node->as()) { + if (auto * identifier = node->as()) { if (identifier->name().empty()) { select_query.select()->children.erase(select_query.select()->children.begin()+i); } From 76f4ca0e8e60052801f24e5b3ff659b750cef04e Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sun, 12 Mar 2023 23:14:56 +0800 Subject: [PATCH 0050/2223] fix style --- src/Interpreters/TreeRewriter.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 81da53756ec..95d277b490d 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -926,9 +926,8 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select for (size_t i = 0; i < select_query.select()->children.size(); ++i) { auto node = select_query.select()->children[i]; if (auto * identifier = node->as()) { - if (identifier->name().empty()) { + if (identifier->name().empty()) select_query.select()->children.erase(select_query.select()->children.begin()+i); - } } } } From 48a1f1c838ae57e04ec8fb1f84dcd50ca2649908 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sun, 12 Mar 2023 23:49:23 +0800 Subject: [PATCH 0051/2223] fix style --- src/Interpreters/TreeRewriter.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 95d277b490d..83886dd1cb0 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -923,13 +923,10 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (auto * t = query->as()) { auto & select_query = *t; - for (size_t i = 0; i < select_query.select()->children.size(); ++i) { - auto node = select_query.select()->children[i]; - if (auto * identifier = node->as()) { + for (size_t i = 0; i < select_query.select()->children.size(); ++i) + if (auto * identifier = select_query.select()->children[i]->as()) if (identifier->name().empty()) select_query.select()->children.erase(select_query.select()->children.begin()+i); - } - } } RequiredSourceColumnsVisitor(columns_context).visit(query); From bbb31cf8913527eb21823216821ca536c2779563 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 15 Mar 2023 18:37:23 +0100 Subject: [PATCH 0052/2223] added validation on setting modification --- src/Client/ClientBase.cpp | 17 ++++++++++++----- src/Client/ClientBase.h | 1 + src/Core/Settings.h | 2 +- src/Core/SettingsFields.cpp | 11 +++++++++++ src/Core/SettingsFields.h | 38 +++++++++++++++++++++++++++++++++++++ src/Server/TCPHandler.cpp | 17 +++++++++++++---- 6 files changed, 76 insertions(+), 10 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 7ca6bbed6ba..5b086d675ba 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1021,7 +1021,7 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_) return true; case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); + onTimezoneUpdate(packet.server_timezone); return true; default: @@ -1046,6 +1046,11 @@ void ClientBase::onProgress(const Progress & value) progress_indication.writeProgress(*tty_buf); } +void ClientBase::onTimezoneUpdate(const String & tz) +{ + DateLUT::setDefaultTimezone(tz); +} + void ClientBase::onEndOfStream() { @@ -1189,12 +1194,12 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de return receiveSampleBlock(out, columns_description, parsed_query); case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); + onTimezoneUpdate(packet.server_timezone); break; default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, - "Unexpected packet from server (expected Data, Exception or Log, got {})", + "Unexpected packet from server (expected Data, Exception, Log or TimezoneUpdate, got {})", String(Protocol::Server::toString(packet.type))); } } @@ -1500,7 +1505,9 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents)) + while (packet_type && (*packet_type == Protocol::Server::Log || + *packet_type == Protocol::Server::ProfileEvents || + *packet_type == Protocol::Server::TimezoneUpdate)) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); @@ -1538,7 +1545,7 @@ bool ClientBase::receiveEndOfQuery() break; case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); + onTimezoneUpdate(packet.server_timezone); break; default: diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 52e15a1a075..18d9a30cac0 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -139,6 +139,7 @@ private: void cancelQuery(); void onProgress(const Progress & value); + void onTimezoneUpdate(const String & tz); void onData(Block & block, ASTPtr parsed_query); void onLogData(Block & block); void onTotals(Block & block, ASTPtr parsed_query); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e508818a26a..ced59219a5b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -712,7 +712,7 @@ class IColumn; M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ M(Bool, force_aggregation_in_order, false, "Force use of aggregation in order on remote nodes during distributed aggregation. PLEASE, NEVER CHANGE THIS SETTING VALUE MANUALLY!", IMPORTANT) \ - M(String, timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ + M(Timezone, timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 4164bf1e27e..44369c7c8a0 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -445,6 +445,17 @@ String SettingFieldEnumHelpers::readBinary(ReadBuffer & in) return str; } +void SettingFieldTimezone::writeBinary(WriteBuffer & out) const +{ + writeStringBinary(value, out); +} + +void SettingFieldTimezone::readBinary(ReadBuffer & in) +{ + String str; + readStringBinary(str, in); + *this = std::move(str); +} String SettingFieldCustom::toString() const { diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index c6fe46c9f6b..b580122d3db 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -539,6 +540,43 @@ void SettingFieldMultiEnum::readBinary(ReadBuffer & in) return std::initializer_list> __VA_ARGS__ .size();\ } +/* Setting field for specifying user-defined timezone. It is basically a string, but it needs validation. + */ +struct SettingFieldTimezone +{ + String value; + bool changed = false; + + explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } + explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; } + explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } + explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; } + explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet(); validateTimezone(str); value = str; } + + SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } + SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; } + SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; } + SettingFieldTimezone & operator =(const char * str) { *this = std::string_view{str}; return *this; } + SettingFieldTimezone & operator =(const Field & f) { *this = f.safeGet(); return *this; } + + operator const String &() const { return value; } /// NOLINT + explicit operator Field() const { return value; } + + const String & toString() const { return value; } + void parseFromString(const String & str) { *this = str; } + + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); + +private: + cctz::time_zone validated_tz; + void validateTimezone(const std::string & str) + { + if (str != "" && !cctz::load_time_zone(str, &validated_tz)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); + } +}; + /// Can keep a value of any type. Used for user-defined settings. struct SettingFieldCustom { diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9bb11f34916..f43982c5133 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -442,6 +442,9 @@ void TCPHandler::runImpl() if (isQueryCancelled()) return true; + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES + && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) + sendTimezone(); sendProgress(); sendSelectProfileEvents(); sendLogs(); @@ -483,10 +486,10 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); - sendLogs(); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) sendTimezone(); + sendLogs(); sendEndOfStream(); } @@ -1041,9 +1044,15 @@ void TCPHandler::sendInsertProfileEvents() void TCPHandler::sendTimezone() { - writeVarUInt(Protocol::Server::TimezoneUpdate, *out); - writeStringBinary(DateLUT::instance().getTimeZone(), *out); - out->next(); +// const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString(); + const String & tz = query_context->getSettingsRef().timezone.toString(); + if (!tz.empty()) + { + LOG_DEBUG(log, "Sent timezone: {}", tz); + writeVarUInt(Protocol::Server::TimezoneUpdate, *out); + writeStringBinary(tz, *out); + out->next(); + } } From c93202cca4cec2e83c51cb6b3cb56dc820965caa Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 9 Mar 2023 21:23:57 -0500 Subject: [PATCH 0053/2223] Keeper Client MVP --- programs/CMakeLists.txt | 21 +++ programs/config_tools.h.in | 1 + programs/keeper-client/CMakeLists.txt | 9 + programs/keeper-client/KeeperClient.cpp | 224 ++++++++++++++++++++++++ programs/keeper-client/KeeperClient.h | 44 +++++ programs/main.cpp | 6 + src/Client/ClientBase.cpp | 8 - src/Client/ClientBase.h | 8 + 8 files changed, 313 insertions(+), 8 deletions(-) create mode 100644 programs/keeper-client/CMakeLists.txt create mode 100644 programs/keeper-client/KeeperClient.cpp create mode 100644 programs/keeper-client/KeeperClient.h diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 47017a94cb5..c00d1f5349f 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -55,6 +55,8 @@ option (ENABLE_CLICKHOUSE_KEEPER "ClickHouse alternative to ZooKeeper" ${ENABLE_ option (ENABLE_CLICKHOUSE_KEEPER_CONVERTER "Util allows to convert ZooKeeper logs and snapshots into clickhouse-keeper snapshot" ${ENABLE_CLICKHOUSE_ALL}) +option (ENABLE_CLICKHOUSE_KEEPER_CLIENT "ClickHouse Keeper Client" ${ENABLE_CLICKHOUSE_ALL}) + option (ENABLE_CLICKHOUSE_SU "A tool similar to 'su'" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_DISKS "A tool to manage disks" ${ENABLE_CLICKHOUSE_ALL}) @@ -169,6 +171,13 @@ else() message(STATUS "ClickHouse keeper-converter mode: OFF") endif() +if (ENABLE_CLICKHOUSE_KEEPER_CLIENT) + message(STATUS "ClickHouse keeper-client mode: ON") +else() + message(STATUS "ClickHouse keeper-client mode: OFF") +endif() + + if (ENABLE_CLICKHOUSE_DISKS) message(STATUS "Clickhouse disks mode: ON") else() @@ -237,6 +246,10 @@ if (ENABLE_CLICKHOUSE_KEEPER_CONVERTER) add_subdirectory (keeper-converter) endif() +if (ENABLE_CLICKHOUSE_KEEPER_CLIENT) + add_subdirectory (keeper-client) +endif() + if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) add_subdirectory (odbc-bridge) endif () @@ -301,6 +314,9 @@ endif() if (ENABLE_CLICKHOUSE_KEEPER_CONVERTER) clickhouse_target_link_split_lib(clickhouse keeper-converter) endif() +if (ENABLE_CLICKHOUSE_KEEPER_CLIENT) + clickhouse_target_link_split_lib(clickhouse keeper-client) +endif() if (ENABLE_CLICKHOUSE_INSTALL) clickhouse_target_link_split_lib(clickhouse install) endif () @@ -392,6 +408,11 @@ if (ENABLE_CLICKHOUSE_KEEPER_CONVERTER) install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-keeper-converter" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) list(APPEND CLICKHOUSE_BUNDLE clickhouse-keeper-converter) endif () +if (ENABLE_CLICKHOUSE_KEEPER_CLIENT) + add_custom_target (clickhouse-keeper-client ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-keeper-client DEPENDS clickhouse) + install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-keeper-client" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) + list(APPEND CLICKHOUSE_BUNDLE clickhouse-keeper-client) +endif () if (ENABLE_CLICKHOUSE_DISKS) add_custom_target (clickhouse-disks ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-disks DEPENDS clickhouse) install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-disks" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) diff --git a/programs/config_tools.h.in b/programs/config_tools.h.in index 30444e8c84e..65ef3ca762b 100644 --- a/programs/config_tools.h.in +++ b/programs/config_tools.h.in @@ -17,6 +17,7 @@ #cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE #cmakedefine01 ENABLE_CLICKHOUSE_LIBRARY_BRIDGE #cmakedefine01 ENABLE_CLICKHOUSE_KEEPER +#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CLIENT #cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CONVERTER #cmakedefine01 ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER #cmakedefine01 ENABLE_CLICKHOUSE_SU diff --git a/programs/keeper-client/CMakeLists.txt b/programs/keeper-client/CMakeLists.txt new file mode 100644 index 00000000000..06055d6d820 --- /dev/null +++ b/programs/keeper-client/CMakeLists.txt @@ -0,0 +1,9 @@ +set (CLICKHOUSE_KEEPER_CLIENT_SOURCES KeeperClient.cpp) + +set (CLICKHOUSE_KEEPER_CLIENT_LINK + PRIVATE + boost::program_options + dbms +) + +clickhouse_program_add(keeper-client) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp new file mode 100644 index 00000000000..05011f064fb --- /dev/null +++ b/programs/keeper-client/KeeperClient.cpp @@ -0,0 +1,224 @@ +#include "KeeperClient.h" +#include +#include +#include +#include +#include +#include + + +namespace po = boost::program_options; +namespace fs = std::filesystem; + +namespace DB +{ + +String KeeperClient::getAbsolutePath(const String & relative) +{ + String result; + if (relative.starts_with('/')) + result = fs::weakly_canonical(relative); + else + result = fs::weakly_canonical(cwd / relative); + + if (result.ends_with('/') && result.size() > 1) + result.pop_back(); + + return result; +} + +void KeeperClient::loadCommands(std::vector> && new_commands) +{ + for (auto & [name, args_count, callback] : new_commands) { + commands.insert({{name, args_count}, callback}); + suggest.addWords({name}); + } +} + +void KeeperClient::defineOptions(Poco::Util::OptionSet & options) +{ + Poco::Util::Application::defineOptions(options); + + options.addOption( + Poco::Util::Option("help", "h", "show help and exit") + .binding("help")); + + options.addOption( + Poco::Util::Option("connection-timeout", "", "set connection timeout in seconds. default 10s.") + .argument("connection-timeout") + .binding("connection-timeout")); + + options.addOption( + Poco::Util::Option("session-timeout", "", "set session timeout in seconds. default 10s.") + .argument("session-timeout") + .binding("session-timeout")); + + options.addOption( + Poco::Util::Option("operation-timeout", "", "set operation timeout in seconds. default 10s.") + .argument("operation-timeout") + .binding("operation-timeout")); + + options.addOption( + Poco::Util::Option("history-file", "", "set path of history file. default `~/.keeper-client-history`") + .argument("history-file") + .binding("history-file")); +} + +void KeeperClient::initialize(Poco::Util::Application & /* self */) +{ + loadCommands({ + {"set", 2, [](KeeperClient * client, const std::vector & args) { + client->zookeeper->set(client->getAbsolutePath(args[1]), args[2]); + }}, + + {"create", 2, [](KeeperClient * client, const std::vector & args) { + client->zookeeper->create(client->getAbsolutePath(args[1]), args[2], zkutil::CreateMode::Persistent); + }}, + + {"get", 1, [](KeeperClient * client, const std::vector & args) { + std::cout << client->zookeeper->get(client->getAbsolutePath(args[1])) << "\n"; + }}, + + {"ls", 0, [](KeeperClient * client, const std::vector & /* args */) { + auto children = client->zookeeper->getChildren(client->cwd); + for (auto & child : children) + std::cout << child << " "; + std::cout << "\n"; + }}, + + {"ls", 1, [](KeeperClient * client, const std::vector & args) { + auto children = client->zookeeper->getChildren(client->getAbsolutePath(args[1])); + for (auto & child : children) + std::cout << child << " "; + std::cout << "\n"; + }}, + + {"cd", 0, [](KeeperClient * /* client */, const std::vector & /* args */) { + }}, + + {"cd", 1, [](KeeperClient * client, const std::vector & args) { + auto new_path = client->getAbsolutePath(args[1]); + if (!client->zookeeper->exists(new_path)) + std::cerr << "Path " << new_path << " does not exists\n"; + else + client->cwd = new_path; + }}, + + {"rm", 1, [](KeeperClient * client, const std::vector & args) { + client->zookeeper->remove(client->getAbsolutePath(args[1])); + }}, + + {"rmr", 1, [](KeeperClient * client, const std::vector & args) { + client->zookeeper->removeRecursive(client->getAbsolutePath(args[1])); + }}, + }); + + String home_path; + const char * home_path_cstr = getenv("HOME"); // NOLINT(concurrency-mt-unsafe) + if (home_path_cstr) + home_path = home_path_cstr; + + if (config().has("history-file")) + history_file = config().getString("history-file"); + else + history_file = home_path + "/.keeper-client-history"; + + if (!history_file.empty() && !fs::exists(history_file)) + { + try + { + FS::createFile(history_file); + } + catch (const ErrnoException & e) + { + if (e.getErrno() != EEXIST) + throw; + } + } + + EventNotifier::init(); +} + +bool KeeperClient::processQueryText(const String & text) +{ + if (exit_strings.find(text) != exit_strings.end()) + return false; + + std::vector tokens; + boost::algorithm::split(tokens, text, boost::is_any_of(" ")); + + try + { + auto callback = commands.find({tokens[0], tokens.size() - 1}); + if (callback == commands.end()) + std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; + else + callback->second(this, tokens); + } + catch (Coordination::Exception & err) + { + std::cerr << err.message() << "\n"; + } + return true; +} + +void KeeperClient::runInteractive() +{ + + LineReader::Patterns query_extenders = {"\\"}; + LineReader::Patterns query_delimiters = {}; + + ReplxxLineReader lr(suggest, history_file, false, query_extenders, query_delimiters, {}); + lr.enableBracketedPaste(); + + while (true) + { + auto input = lr.readLine( cwd.string() + " :) ", ":-] "); + if (input.empty()) + break; + + if (!processQueryText(input)) + break; + } +} + +int KeeperClient::main(const std::vector & args) +{ + zkutil::ZooKeeperArgs zk_args(args[0]); + zk_args.connection_timeout_ms = config().getInt("connection-timeout", 10) * 1000; + zk_args.session_timeout_ms = config().getInt("session-timeout", 10) * 1000; + zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; + zookeeper = std::make_unique(zk_args); + + runInteractive(); + + return 0; +} + +} + + +int mainEntryClickHouseKeeperClient(int argc, char ** argv) +{ + try + { + DB::KeeperClient client; + client.init(argc, argv); + return client.run(); + } + catch (const DB::Exception & e) + { + std::cerr << DB::getExceptionMessage(e, false) << std::endl; + return 1; + } + catch (const boost::program_options::error & e) + { + std::cerr << "Bad arguments: " << e.what() << std::endl; + return DB::ErrorCodes::BAD_ARGUMENTS; + } + catch (...) + { + std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; + return 1; + } +} diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h new file mode 100644 index 00000000000..8d96ade7659 --- /dev/null +++ b/programs/keeper-client/KeeperClient.h @@ -0,0 +1,44 @@ +#pragma once + + +#include +#include +#include +#include + + +namespace DB +{ + +class KeeperClient; + +class KeeperClient: public Poco::Util::Application +{ +public: + using Callback = std::function &)>; + + KeeperClient() = default; + + void initialize(Poco::Util::Application & self) override; + + int main(const std::vector & args) override; + + void defineOptions(Poco::Util::OptionSet & options) override; + +protected: + void runInteractive(); + void loadCommands(std::vector> &&); + bool processQueryText(const String & text); + + String getAbsolutePath(const String & relative); + + std::map, Callback> commands; + + String history_file; + LineReader::Suggest suggest; + + zkutil::ZooKeeperPtr zookeeper; + std::filesystem::path cwd = "/"; +}; + +} diff --git a/programs/main.cpp b/programs/main.cpp index 83e64b8c932..9a3ad47a86e 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -62,6 +62,9 @@ int mainEntryClickHouseKeeper(int argc, char ** argv); #if ENABLE_CLICKHOUSE_KEEPER_CONVERTER int mainEntryClickHouseKeeperConverter(int argc, char ** argv); #endif +#if ENABLE_CLICKHOUSE_KEEPER_CLIENT +int mainEntryClickHouseKeeperClient(int argc, char ** argv); +#endif #if ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER int mainEntryClickHouseStaticFilesDiskUploader(int argc, char ** argv); #endif @@ -133,6 +136,9 @@ std::pair clickhouse_applications[] = #if ENABLE_CLICKHOUSE_KEEPER_CONVERTER {"keeper-converter", mainEntryClickHouseKeeperConverter}, #endif +#if ENABLE_CLICKHOUSE_KEEPER_CLIENT + {"keeper-client", mainEntryClickHouseKeeperClient}, +#endif #if ENABLE_CLICKHOUSE_INSTALL {"install", mainEntryClickHouseInstall}, {"start", mainEntryClickHouseStart}, diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 120d273aa62..c931fb426ec 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -86,14 +86,6 @@ namespace CurrentMetrics namespace DB { -static const NameSet exit_strings -{ - "exit", "quit", "logout", "учше", "йгше", "дщпщге", - "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", - "q", "й", "\\q", "\\Q", "\\й", "\\Й", ":q", "Жй" -}; - - namespace ErrorCodes { extern const int BAD_ARGUMENTS; diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 5926f73f51a..566c5aefa04 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -24,6 +24,14 @@ namespace po = boost::program_options; namespace DB { + +static const NameSet exit_strings +{ + "exit", "quit", "logout", "учше", "йгше", "дщпщге", + "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", + "q", "й", "\\q", "\\Q", "\\й", "\\Й", ":q", "Жй" +}; + namespace ErrorCodes { extern const int NOT_IMPLEMENTED; From 9051ddb1743730f4504abdf2209c33a0b03de9b5 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 9 Mar 2023 21:45:58 -0500 Subject: [PATCH 0054/2223] Style fix --- programs/keeper-client/KeeperClient.cpp | 37 +++++++++++++++++-------- 1 file changed, 26 insertions(+), 11 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 05011f064fb..0a4cdd4286b 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -13,6 +13,11 @@ namespace fs = std::filesystem; namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + String KeeperClient::getAbsolutePath(const String & relative) { String result; @@ -29,7 +34,8 @@ String KeeperClient::getAbsolutePath(const String & relative) void KeeperClient::loadCommands(std::vector> && new_commands) { - for (auto & [name, args_count, callback] : new_commands) { + for (auto & [name, args_count, callback] : new_commands) + { commands.insert({{name, args_count}, callback}); suggest.addWords({name}); } @@ -67,36 +73,43 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) void KeeperClient::initialize(Poco::Util::Application & /* self */) { loadCommands({ - {"set", 2, [](KeeperClient * client, const std::vector & args) { + {"set", 2, [](KeeperClient * client, const std::vector & args) + { client->zookeeper->set(client->getAbsolutePath(args[1]), args[2]); }}, - {"create", 2, [](KeeperClient * client, const std::vector & args) { + {"create", 2, [](KeeperClient * client, const std::vector & args) + { client->zookeeper->create(client->getAbsolutePath(args[1]), args[2], zkutil::CreateMode::Persistent); }}, - {"get", 1, [](KeeperClient * client, const std::vector & args) { + {"get", 1, [](KeeperClient * client, const std::vector & args) + { std::cout << client->zookeeper->get(client->getAbsolutePath(args[1])) << "\n"; }}, - {"ls", 0, [](KeeperClient * client, const std::vector & /* args */) { + {"ls", 0, [](KeeperClient * client, const std::vector & /* args */) + { auto children = client->zookeeper->getChildren(client->cwd); for (auto & child : children) std::cout << child << " "; std::cout << "\n"; }}, - {"ls", 1, [](KeeperClient * client, const std::vector & args) { + {"ls", 1, [](KeeperClient * client, const std::vector & args) + { auto children = client->zookeeper->getChildren(client->getAbsolutePath(args[1])); for (auto & child : children) std::cout << child << " "; std::cout << "\n"; }}, - {"cd", 0, [](KeeperClient * /* client */, const std::vector & /* args */) { + {"cd", 0, [](KeeperClient * /* client */, const std::vector & /* args */) + { }}, - {"cd", 1, [](KeeperClient * client, const std::vector & args) { + {"cd", 1, [](KeeperClient * client, const std::vector & args) + { auto new_path = client->getAbsolutePath(args[1]); if (!client->zookeeper->exists(new_path)) std::cerr << "Path " << new_path << " does not exists\n"; @@ -104,11 +117,13 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) client->cwd = new_path; }}, - {"rm", 1, [](KeeperClient * client, const std::vector & args) { + {"rm", 1, [](KeeperClient * client, const std::vector & args) + { client->zookeeper->remove(client->getAbsolutePath(args[1])); }}, - {"rmr", 1, [](KeeperClient * client, const std::vector & args) { + {"rmr", 1, [](KeeperClient * client, const std::vector & args) + { client->zookeeper->removeRecursive(client->getAbsolutePath(args[1])); }}, }); @@ -173,7 +188,7 @@ void KeeperClient::runInteractive() while (true) { - auto input = lr.readLine( cwd.string() + " :) ", ":-] "); + auto input = lr.readLine(cwd.string() + " :) ", ":-] "); if (input.empty()) break; From dfea87d24888eb3d223ab5c020ff5a3cdd029409 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 10 Mar 2023 23:12:16 -0500 Subject: [PATCH 0055/2223] Added confirmation for rmr operation. Implemented support for four-letter-word commands. --- programs/keeper-client/KeeperClient.cpp | 73 ++++++++++++++++++++++--- programs/keeper-client/KeeperClient.h | 13 ++++- 2 files changed, 78 insertions(+), 8 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 0a4cdd4286b..752c44bd8f4 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -13,11 +13,47 @@ namespace fs = std::filesystem; namespace DB { +static const NameSet four_letter_word_commands +{ + "ruok", "mntr", "srvr", "stat", "srst", "conf", + "cons", "crst", "envi", "dirs", "isro", "wchs", + "wchc", "wchp", "dump", "csnp", "lgif", "rqld", +}; + namespace ErrorCodes { extern const int BAD_ARGUMENTS; } +String KeeperClient::executeFourLetterCommand(const String & command) +{ + // We need create new socket every time because ZooKeeper forcefully shut down connection after four-letter-word command. + Poco::Net::StreamSocket socket; + socket.connect(Poco::Net::SocketAddress{zk_args.hosts[0]}, zk_args.connection_timeout_ms * 1000); + + socket.setReceiveTimeout(zk_args.operation_timeout_ms * 1000); + socket.setSendTimeout(zk_args.operation_timeout_ms * 1000); + socket.setNoDelay(true); + + ReadBufferFromPocoSocket in(socket); + WriteBufferFromPocoSocket out(socket); + + out.write(command.data(), command.size()); + out.next(); + + String result; + readStringUntilEOF(result, in); + in.next(); + return result; +} + +void KeeperClient::askConfirmation(const String & prompt, std::function && callback) +{ + std::cout << prompt << " Continue?\n"; + need_confirmation = true; + confirmation_callback = callback; +} + String KeeperClient::getAbsolutePath(const String & relative) { String result; @@ -124,7 +160,9 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) {"rmr", 1, [](KeeperClient * client, const std::vector & args) { - client->zookeeper->removeRecursive(client->getAbsolutePath(args[1])); + String path = client->getAbsolutePath(args[1]); + client->askConfirmation("You are going to recursively delete path " + path, + [client, path]{ client->zookeeper->removeRecursive(path); }); }}, }); @@ -164,11 +202,26 @@ bool KeeperClient::processQueryText(const String & text) try { - auto callback = commands.find({tokens[0], tokens.size() - 1}); - if (callback == commands.end()) - std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; + if (need_confirmation) + { + if (tokens.size() == 1 && (tokens[0] == "y" || tokens[0] == "Y")) + { + need_confirmation = false; + confirmation_callback(); + } + + need_confirmation = false; + } + else if (tokens.size() == 1 && tokens[0].size() == 4 && four_letter_word_commands.find(tokens[0]) != four_letter_word_commands.end()) + std::cout << executeFourLetterCommand(tokens[0]) << "\n"; else - callback->second(this, tokens); + { + auto callback = commands.find({tokens[0], tokens.size() - 1}); + if (callback == commands.end()) + std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; + else + callback->second(this, tokens); + } } catch (Coordination::Exception & err) { @@ -188,7 +241,13 @@ void KeeperClient::runInteractive() while (true) { - auto input = lr.readLine(cwd.string() + " :) ", ":-] "); + String prompt; + if (need_confirmation) + prompt = "[y/n] "; + else + prompt = cwd.string() + " :) "; + + auto input = lr.readLine(prompt, ":-] "); if (input.empty()) break; @@ -199,7 +258,7 @@ void KeeperClient::runInteractive() int KeeperClient::main(const std::vector & args) { - zkutil::ZooKeeperArgs zk_args(args[0]); + zk_args.hosts = {args[0]}; zk_args.connection_timeout_ms = config().getInt("connection-timeout", 10) * 1000; zk_args.session_timeout_ms = config().getInt("session-timeout", 10) * 1000; zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 8d96ade7659..0634d3e4b37 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -3,6 +3,9 @@ #include #include +#include +#include +#include #include #include @@ -27,10 +30,13 @@ public: protected: void runInteractive(); - void loadCommands(std::vector> &&); + void loadCommands(std::vector> && callback); bool processQueryText(const String & text); + String executeFourLetterCommand(const String & command); + String getAbsolutePath(const String & relative); + void askConfirmation(const String & prompt, std::function && callback); std::map, Callback> commands; @@ -38,7 +44,12 @@ protected: LineReader::Suggest suggest; zkutil::ZooKeeperPtr zookeeper; + zkutil::ZooKeeperArgs zk_args; + std::filesystem::path cwd = "/"; + + bool need_confirmation = false; + std::function confirmation_callback; }; } From 72769d468ea55f419286caf417ed985af9e4069b Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 10 Mar 2023 23:15:15 -0500 Subject: [PATCH 0056/2223] comment fix --- programs/keeper-client/KeeperClient.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 752c44bd8f4..43a9527a3ca 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -27,7 +27,7 @@ namespace ErrorCodes String KeeperClient::executeFourLetterCommand(const String & command) { - // We need create new socket every time because ZooKeeper forcefully shut down connection after four-letter-word command. + /// We need to create a new socket every time because ZooKeeper forcefully shuts down the connection after a four-letter-word command. Poco::Net::StreamSocket socket; socket.connect(Poco::Net::SocketAddress{zk_args.hosts[0]}, zk_args.connection_timeout_ms * 1000); From 18fada7028c7997f25814201ce7c89c05efb9e3d Mon Sep 17 00:00:00 2001 From: pufit Date: Sun, 12 Mar 2023 12:54:42 -0400 Subject: [PATCH 0057/2223] fix duplication --- programs/keeper-client/KeeperClient.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 43a9527a3ca..74eae042b4c 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -204,13 +204,9 @@ bool KeeperClient::processQueryText(const String & text) { if (need_confirmation) { - if (tokens.size() == 1 && (tokens[0] == "y" || tokens[0] == "Y")) - { - need_confirmation = false; - confirmation_callback(); - } - need_confirmation = false; + if (tokens.size() == 1 && (tokens[0] == "y" || tokens[0] == "Y")) + confirmation_callback(); } else if (tokens.size() == 1 && tokens[0].size() == 4 && four_letter_word_commands.find(tokens[0]) != four_letter_word_commands.end()) std::cout << executeFourLetterCommand(tokens[0]) << "\n"; From 752eab501ce9da992f7f4a1dc3600521c7c65723 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 14 Mar 2023 14:24:36 -0400 Subject: [PATCH 0058/2223] default host to connect --- programs/keeper-client/KeeperClient.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 74eae042b4c..1f68cbd05bf 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -254,7 +254,11 @@ void KeeperClient::runInteractive() int KeeperClient::main(const std::vector & args) { - zk_args.hosts = {args[0]}; + if (args.empty()) + zk_args.hosts = {"localhost:2181"}; + else + zk_args.hosts = {args[0]}; + zk_args.connection_timeout_ms = config().getInt("connection-timeout", 10) * 1000; zk_args.session_timeout_ms = config().getInt("session-timeout", 10) * 1000; zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; From 7adc442fedba0617f322cdecdaee7b21904bf51a Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 14 Mar 2023 15:32:48 -0400 Subject: [PATCH 0059/2223] support run in non-interactive mode --- programs/keeper-client/KeeperClient.cpp | 35 ++++++++++++++++++++++--- programs/keeper-client/KeeperClient.h | 1 + 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 1f68cbd05bf..9ed60dd0d4b 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -70,11 +70,15 @@ String KeeperClient::getAbsolutePath(const String & relative) void KeeperClient::loadCommands(std::vector> && new_commands) { - for (auto & [name, args_count, callback] : new_commands) + for (const auto & [name, args_count, callback] : new_commands) { commands.insert({{name, args_count}, callback}); suggest.addWords({name}); } + + for (const auto & command : four_letter_word_commands) { + suggest.addWords({command}); + } } void KeeperClient::defineOptions(Poco::Util::OptionSet & options) @@ -85,6 +89,11 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) Poco::Util::Option("help", "h", "show help and exit") .binding("help")); + options.addOption( + Poco::Util::Option("query", "q", "will execute given query, then exit.") + .argument("query") + .binding("query")); + options.addOption( Poco::Util::Option("connection-timeout", "", "set connection timeout in seconds. default 10s.") .argument("connection-timeout") @@ -192,6 +201,18 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) EventNotifier::init(); } +void KeeperClient::executeQuery(const String & query) +{ + std::vector queries; + boost::algorithm::split(queries, query, boost::is_any_of(";")); + + for (const auto & query_text : queries) + { + if (!query_text.empty()) + processQueryText(query_text); + } +} + bool KeeperClient::processQueryText(const String & text) { if (exit_strings.find(text) != exit_strings.end()) @@ -214,7 +235,12 @@ bool KeeperClient::processQueryText(const String & text) { auto callback = commands.find({tokens[0], tokens.size() - 1}); if (callback == commands.end()) - std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; + { + if (tokens[0].size() == 4 && tokens.size() == 1) /// Treat it like unrecognized four-letter command + std::cout << executeFourLetterCommand(tokens[0]) << "\n"; + else + std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; + } else callback->second(this, tokens); } @@ -264,7 +290,10 @@ int KeeperClient::main(const std::vector & args) zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; zookeeper = std::make_unique(zk_args); - runInteractive(); + if (config().has("query")) + executeQuery(config().getString("query")); + else + runInteractive(); return 0; } diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 0634d3e4b37..10099b06021 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -32,6 +32,7 @@ protected: void runInteractive(); void loadCommands(std::vector> && callback); bool processQueryText(const String & text); + void executeQuery(const String & query); String executeFourLetterCommand(const String & command); From bfdc2b58b421718550b586fab6806e24a18cc960 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 14 Mar 2023 15:34:08 -0400 Subject: [PATCH 0060/2223] style fix --- programs/keeper-client/KeeperClient.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 9ed60dd0d4b..c427c6fc8ef 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -203,7 +203,7 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) void KeeperClient::executeQuery(const String & query) { - std::vector queries; + std::vector queries; boost::algorithm::split(queries, query, boost::is_any_of(";")); for (const auto & query_text : queries) @@ -218,7 +218,7 @@ bool KeeperClient::processQueryText(const String & text) if (exit_strings.find(text) != exit_strings.end()) return false; - std::vector tokens; + std::vector tokens; boost::algorithm::split(tokens, text, boost::is_any_of(" ")); try From 7dc6ff02c33b3d84fdaf4ff2f16a74c8b2edbd3e Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 14 Mar 2023 17:50:09 -0400 Subject: [PATCH 0061/2223] use keeper-client in integration tests --- programs/keeper-client/KeeperClient.cpp | 7 +++++++ tests/integration/helpers/keeper_utils.py | 22 ++++------------------ 2 files changed, 11 insertions(+), 18 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index c427c6fc8ef..10aa4b1dedd 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -113,6 +113,11 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) Poco::Util::Option("history-file", "", "set path of history file. default `~/.keeper-client-history`") .argument("history-file") .binding("history-file")); + + options.addOption( + Poco::Util::Option("log-level", "", "set log level") + .argument("log-level") + .binding("log-level")); } void KeeperClient::initialize(Poco::Util::Application & /* self */) @@ -198,6 +203,8 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) } } + Poco::Logger::root().setLevel(config().getString("log-level", "error")); + EventNotifier::init(); } diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3b909194b63..c6cd9dfa18a 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -1,26 +1,12 @@ import socket import time - - -def get_keeper_socket(cluster, node, port=9181): - hosts = cluster.get_instance_ip(node.name) - client = socket.socket() - client.settimeout(10) - client.connect((hosts, port)) - return client +from helper.client import CommandRequest def send_4lw_cmd(cluster, node, cmd="ruok", port=9181): - client = None - try: - client = get_keeper_socket(cluster, node, port) - client.send(cmd.encode()) - data = client.recv(100_000) - data = data.decode() - return data - finally: - if client is not None: - client.close() + return CommandRequest( + ["cluster.server_bin_path", "keeper-client", f"{cluster.get_instance_ip(node.name)}:{port}", "-q", cmd] + ).get_answer() NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests" From e6d01c617b16117d314dbeea43660ece5c4b54d2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 14 Mar 2023 22:14:43 +0000 Subject: [PATCH 0062/2223] Automatic style fix --- tests/integration/helpers/keeper_utils.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index c6cd9dfa18a..f9cc2fb29fb 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -5,7 +5,13 @@ from helper.client import CommandRequest def send_4lw_cmd(cluster, node, cmd="ruok", port=9181): return CommandRequest( - ["cluster.server_bin_path", "keeper-client", f"{cluster.get_instance_ip(node.name)}:{port}", "-q", cmd] + [ + "cluster.server_bin_path", + "keeper-client", + f"{cluster.get_instance_ip(node.name)}:{port}", + "-q", + cmd, + ] ).get_answer() From ef9f66e36fed106f59bafaf971a02799fbecb75f Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 14 Mar 2023 18:30:23 -0400 Subject: [PATCH 0063/2223] style fix --- programs/keeper-client/KeeperClient.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 10aa4b1dedd..54b5cf1d6cf 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -76,9 +76,8 @@ void KeeperClient::loadCommands(std::vector suggest.addWords({name}); } - for (const auto & command : four_letter_word_commands) { + for (const auto & command : four_letter_word_commands) suggest.addWords({command}); - } } void KeeperClient::defineOptions(Poco::Util::OptionSet & options) From 65f2516079f3a2b53af8224078119fc4062b6ef7 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 16 Mar 2023 15:37:06 -0400 Subject: [PATCH 0064/2223] separate integration tests for keeper-client --- tests/integration/helpers/keeper_utils.py | 28 +++++---- .../test_keeper_client/__init__.py | 0 .../configs/keeper_config.xml | 3 + tests/integration/test_keeper_client/test.py | 57 +++++++++++++++++++ 4 files changed, 78 insertions(+), 10 deletions(-) create mode 100644 tests/integration/test_keeper_client/__init__.py create mode 100644 tests/integration/test_keeper_client/configs/keeper_config.xml create mode 100644 tests/integration/test_keeper_client/test.py diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index f9cc2fb29fb..3b909194b63 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -1,18 +1,26 @@ import socket import time -from helper.client import CommandRequest + + +def get_keeper_socket(cluster, node, port=9181): + hosts = cluster.get_instance_ip(node.name) + client = socket.socket() + client.settimeout(10) + client.connect((hosts, port)) + return client def send_4lw_cmd(cluster, node, cmd="ruok", port=9181): - return CommandRequest( - [ - "cluster.server_bin_path", - "keeper-client", - f"{cluster.get_instance_ip(node.name)}:{port}", - "-q", - cmd, - ] - ).get_answer() + client = None + try: + client = get_keeper_socket(cluster, node, port) + client.send(cmd.encode()) + data = client.recv(100_000) + data = data.decode() + return data + finally: + if client is not None: + client.close() NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests" diff --git a/tests/integration/test_keeper_client/__init__.py b/tests/integration/test_keeper_client/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_client/configs/keeper_config.xml b/tests/integration/test_keeper_client/configs/keeper_config.xml new file mode 100644 index 00000000000..7e912283ac0 --- /dev/null +++ b/tests/integration/test_keeper_client/configs/keeper_config.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py new file mode 100644 index 00000000000..64ef62b6243 --- /dev/null +++ b/tests/integration/test_keeper_client/test.py @@ -0,0 +1,57 @@ +import pytest +from helpers.client import CommandRequest +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/keeper_config.xml"], + with_zookeeper=True, + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_base_commands(started_cluster): + _ = started_cluster + + command = CommandRequest( + [ + started_cluster.server_bin_path, + "keeper-client", + f"{cluster.get_instance_ip('zoo1')}:{cluster.zookeeper_port}", + "-q", + "create test_create_zk_node1 testvalue1;create test_create_zk_node_2 testvalue2;get test_create_zk_node1;", + ], + stdin="", + ) + + assert command.get_answer() == "testvalue1\n" + + +def test_four_letter_word_commands(started_cluster): + _ = started_cluster + + command = CommandRequest( + [ + started_cluster.server_bin_path, + "keeper-client", + f"{cluster.get_instance_ip('zoo1')}:{cluster.zookeeper_port}", + "-q", + "ruok", + ], + stdin="", + ) + + assert command.get_answer() == "imok\n" From 3380e467d91ca8797135b860342944b46505efd3 Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 27 Mar 2023 11:12:46 -0400 Subject: [PATCH 0065/2223] fix typo --- programs/keeper-client/KeeperClient.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 10099b06021..50a8b35f6c5 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -30,7 +30,7 @@ public: protected: void runInteractive(); - void loadCommands(std::vector> && callback); + void loadCommands(std::vector> && new_commands); bool processQueryText(const String & text); void executeQuery(const String & query); From 17efdbf6251e51d945dd1df598a6526c465fe99b Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Fri, 31 Mar 2023 21:56:35 +0800 Subject: [PATCH 0066/2223] change --- src/Formats/ReadSchemaUtils.cpp | 374 +++++++++++++++--------------- src/Interpreters/TreeRewriter.cpp | 9 - 2 files changed, 182 insertions(+), 201 deletions(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 653efd4f5c1..f12eb8cb71e 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -9,223 +9,213 @@ #include #include -namespace DB -{ +namespace DB { -namespace ErrorCodes -{ - extern const int EMPTY_DATA_PASSED; - extern const int BAD_ARGUMENTS; - extern const int ONLY_NULLS_WHILE_READING_SCHEMA; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; -} + namespace ErrorCodes { + extern const int EMPTY_DATA_PASSED; + extern const int BAD_ARGUMENTS; + extern const int ONLY_NULLS_WHILE_READING_SCHEMA; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + } -static std::optional getOrderedColumnsList( - const NamesAndTypesList & columns_list, const Names & columns_order_hint) -{ - if (columns_list.size() != columns_order_hint.size()) - return {}; - - std::unordered_map available_columns; - for (const auto & [name, type] : columns_list) - available_columns.emplace(name, type); - - NamesAndTypesList res; - for (const auto & name : columns_order_hint) - { - auto it = available_columns.find(name); - if (it == available_columns.end()) + static std::optional getOrderedColumnsList( + const NamesAndTypesList &columns_list, const Names &columns_order_hint) { + if (columns_list.size() != columns_order_hint.size()) return {}; - res.emplace_back(name, it->second); - } - return res; -} + std::unordered_map available_columns; + for (const auto &[name, type]: columns_list) + available_columns.emplace(name, type); -bool isRetryableSchemaInferenceError(int code) -{ - return code == ErrorCodes::EMPTY_DATA_PASSED || code == ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA; -} + NamesAndTypesList res; + for (const auto &name: columns_order_hint) { + auto it = available_columns.find(name); + if (it == available_columns.end()) + return {}; -ColumnsDescription readSchemaFromFormat( - const String & format_name, - const std::optional & format_settings, - ReadBufferIterator & read_buffer_iterator, - bool retry, - ContextPtr & context, - std::unique_ptr & buf) -{ - NamesAndTypesList names_and_types; - if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) - { - auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, format_settings); - try - { - names_and_types = external_schema_reader->readSchema(); - } - catch (Exception & e) - { - e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); - throw; + res.emplace_back(name, it->second); } + return res; } - else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) - { - std::string exception_messages; - SchemaReaderPtr schema_reader; - size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; - size_t iterations = 0; - ColumnsDescription cached_columns; - while (true) - { - bool is_eof = false; - try - { - buf = read_buffer_iterator(cached_columns); - if (!buf) - break; - is_eof = buf->eof(); + + bool isRetryableSchemaInferenceError(int code) { + return code == ErrorCodes::EMPTY_DATA_PASSED || code == ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA; + } + + ColumnsDescription readSchemaFromFormat( + const String &format_name, + const std::optional &format_settings, + ReadBufferIterator &read_buffer_iterator, + bool retry, + ContextPtr &context, + std::unique_ptr &buf) { + NamesAndTypesList names_and_types; + if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) { + auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, + format_settings); + try { + names_and_types = external_schema_reader->readSchema(); } - catch (Exception & e) - { + catch (Exception &e) { e.addMessage(fmt::format( - "Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + "Cannot extract table structure from {} format file. You can specify the structure manually", + format_name)); throw; } - catch (...) - { - auto exception_message = getCurrentExceptionMessage(false); - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file:\n{}\nYou can specify the structure manually", - format_name, - exception_message); - } - - ++iterations; - - if (is_eof) - { - auto exception_message = fmt::format("Cannot extract table structure from {} format file, file is empty", format_name); - - if (!retry) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message); - - exception_messages += "\n" + exception_message; - continue; - } - - try - { - schema_reader = FormatFactory::instance().getSchemaReader(format_name, *buf, context, format_settings); - schema_reader->setMaxRowsToRead(max_rows_to_read); - names_and_types = schema_reader->readSchema(); - break; - } - catch (...) - { - auto exception_message = getCurrentExceptionMessage(false); - if (schema_reader) - { - size_t rows_read = schema_reader->getNumRowsRead(); - assert(rows_read <= max_rows_to_read); - max_rows_to_read -= schema_reader->getNumRowsRead(); - if (rows_read != 0 && max_rows_to_read == 0) - { - exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; - if (iterations > 1) - { - exception_messages += "\n" + exception_message; - break; - } - retry = false; - } + } else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) { + std::string exception_messages; + SchemaReaderPtr schema_reader; + size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference + : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; + size_t iterations = 0; + ColumnsDescription cached_columns; + while (true) { + bool is_eof = false; + try { + buf = read_buffer_iterator(cached_columns); + if (!buf) + break; + is_eof = buf->eof(); + } + catch (Exception &e) { + e.addMessage(fmt::format( + "Cannot extract table structure from {} format file. You can specify the structure manually", + format_name)); + throw; + } + catch (...) { + auto exception_message = getCurrentExceptionMessage(false); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file:\n{}\nYou can specify the structure manually", + format_name, + exception_message); } - if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) - { - try - { - throw; - } - catch (Exception & e) - { - e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); - throw; - } - catch (...) - { + ++iterations; + + if (is_eof) { + auto exception_message = fmt::format( + "Cannot extract table structure from {} format file, file is empty", format_name); + + if (!retry) throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file. " - "Error: {}. You can specify the structure manually", - format_name, exception_message); - } + "{}. You can specify the structure manually", exception_message); + + exception_messages += "\n" + exception_message; + continue; } - exception_messages += "\n" + exception_message; + try { + schema_reader = FormatFactory::instance().getSchemaReader(format_name, *buf, context, + format_settings); + schema_reader->setMaxRowsToRead(max_rows_to_read); + names_and_types = schema_reader->readSchema(); + break; + } + catch (...) { + auto exception_message = getCurrentExceptionMessage(false); + if (schema_reader) { + size_t rows_read = schema_reader->getNumRowsRead(); + assert(rows_read <= max_rows_to_read); + max_rows_to_read -= schema_reader->getNumRowsRead(); + if (rows_read != 0 && max_rows_to_read == 0) { + exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; + if (iterations > 1) { + exception_messages += "\n" + exception_message; + break; + } + retry = false; + } + } + + if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) { + try { + throw; + } + catch (Exception &e) { + e.addMessage(fmt::format( + "Cannot extract table structure from {} format file. You can specify the structure manually", + format_name)); + throw; + } + catch (...) { + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file. " + "Error: {}. You can specify the structure manually", + format_name, exception_message); + } + } + + exception_messages += "\n" + exception_message; + } } - } - if (!cached_columns.empty()) - return cached_columns; + if (!cached_columns.empty()) + return cached_columns; - if (names_and_types.empty()) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "All attempts to extract table structure from files failed. " - "Errors:{}\nYou can specify the structure manually", exception_messages); + if (names_and_types.empty()) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "All attempts to extract table structure from files failed. " + "Errors:{}\nYou can specify the structure manually", exception_messages); - /// If we have "INSERT SELECT" query then try to order - /// columns as they are ordered in table schema for formats - /// without strict column order (like JSON and TSKV). - /// It will allow to execute simple data loading with query - /// "INSERT INTO table SELECT * FROM ..." - const auto & insertion_table = context->getInsertionTable(); - if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty()) - { - auto storage = DatabaseCatalog::instance().getTable(insertion_table, context); - auto metadata = storage->getInMemoryMetadataPtr(); - auto names_in_storage = metadata->getColumns().getNamesOfPhysical(); - auto ordered_list = getOrderedColumnsList(names_and_types, names_in_storage); - if (ordered_list) - names_and_types = *ordered_list; - } + /// If we have "INSERT SELECT" query then try to order + /// columns as they are ordered in table schema for formats + /// without strict column order (like JSON and TSKV). + /// It will allow to execute simple data loading with query + /// "INSERT INTO table SELECT * FROM ..." + const auto &insertion_table = context->getInsertionTable(); + if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty()) { + auto storage = DatabaseCatalog::instance().getTable(insertion_table, context); + auto metadata = storage->getInMemoryMetadataPtr(); + auto names_in_storage = metadata->getColumns().getNamesOfPhysical(); + auto ordered_list = getOrderedColumnsList(names_and_types, names_in_storage); + if (ordered_list) + names_and_types = *ordered_list; + } + } else + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{} file format doesn't support schema inference. You must specify the structure manually", + format_name); + names_and_types.erase(std::remove_if(names_and_types.begin(), names_and_types.end(), + [](const NameAndTypePair &pair) { return pair.name.empty(); }), + names_and_types.end()); + return ColumnsDescription(names_and_types); } - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "{} file format doesn't support schema inference. You must specify the structure manually", - format_name); - return ColumnsDescription(names_and_types); -} + ColumnsDescription + readSchemaFromFormat(const String &format_name, const std::optional &format_settings, + ReadBufferIterator &read_buffer_iterator, bool retry, ContextPtr &context) { + std::unique_ptr buf_out; + return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out); + } -ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional & format_settings, ReadBufferIterator & read_buffer_iterator, bool retry, ContextPtr & context) -{ - std::unique_ptr buf_out; - return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out); -} + SchemaCache::Key getKeyForSchemaCache(const String &source, const String &format, + const std::optional &format_settings, + const ContextPtr &context) { + return getKeysForSchemaCache({source}, format, format_settings, context).front(); + } -SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context) -{ - return getKeysForSchemaCache({source}, format, format_settings, context).front(); -} + static SchemaCache::Key + makeSchemaCacheKey(const String &source, const String &format, const String &additional_format_info) { + return SchemaCache::Key{source, format, additional_format_info}; + } -static SchemaCache::Key makeSchemaCacheKey(const String & source, const String & format, const String & additional_format_info) -{ - return SchemaCache::Key{source, format, additional_format_info}; -} - -SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & format, const std::optional & format_settings, const ContextPtr & context) -{ - /// For some formats data schema depends on some settings, so it's possible that - /// two queries to the same source will get two different schemas. To process this - /// case we add some additional information specific for the format to the cache key. - /// For example, for Protobuf format additional information is the path to the schema - /// and message name. - String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, format_settings); - SchemaCache::Keys cache_keys; - cache_keys.reserve(sources.size()); - std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), [&](const auto & source){ return makeSchemaCacheKey(source, format, additional_format_info); }); - return cache_keys; -} + SchemaCache::Keys getKeysForSchemaCache(const Strings &sources, const String &format, + const std::optional &format_settings, + const ContextPtr &context) { + /// For some formats data schema depends on some settings, so it's possible that + /// two queries to the same source will get two different schemas. To process this + /// case we add some additional information specific for the format to the cache key. + /// For example, for Protobuf format additional information is the path to the schema + /// and message name. + String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, + format_settings); + SchemaCache::Keys cache_keys; + cache_keys.reserve(sources.size()); + std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), + [&](const auto &source) { return makeSchemaCacheKey(source, format, additional_format_info); }); + return cache_keys; + } } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 83886dd1cb0..4c134e175dc 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -919,15 +919,6 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select RequiredSourceColumnsVisitor::Data columns_context; columns_context.visit_index_hint = visit_index_hint; - - if (auto * t = query->as()) - { - auto & select_query = *t; - for (size_t i = 0; i < select_query.select()->children.size(); ++i) - if (auto * identifier = select_query.select()->children[i]->as()) - if (identifier->name().empty()) - select_query.select()->children.erase(select_query.select()->children.begin()+i); - } RequiredSourceColumnsVisitor(columns_context).visit(query); NameSet source_column_names; From 3b756ef0261d33a9c8771cad7906f690db11bf75 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Fri, 31 Mar 2023 21:58:20 +0800 Subject: [PATCH 0067/2223] rollback --- src/Formats/ReadSchemaUtils.cpp | 375 ++++++++++++++++---------------- 1 file changed, 193 insertions(+), 182 deletions(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index f12eb8cb71e..7f1c3bf3e63 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -9,213 +9,224 @@ #include #include -namespace DB { +namespace DB +{ - namespace ErrorCodes { - extern const int EMPTY_DATA_PASSED; - extern const int BAD_ARGUMENTS; - extern const int ONLY_NULLS_WHILE_READING_SCHEMA; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - } +namespace ErrorCodes +{ + extern const int EMPTY_DATA_PASSED; + extern const int BAD_ARGUMENTS; + extern const int ONLY_NULLS_WHILE_READING_SCHEMA; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; +} - static std::optional getOrderedColumnsList( - const NamesAndTypesList &columns_list, const Names &columns_order_hint) { - if (columns_list.size() != columns_order_hint.size()) +static std::optional getOrderedColumnsList( + const NamesAndTypesList & columns_list, const Names & columns_order_hint) +{ + if (columns_list.size() != columns_order_hint.size()) + return {}; + + std::unordered_map available_columns; + for (const auto & [name, type] : columns_list) + available_columns.emplace(name, type); + + NamesAndTypesList res; + for (const auto & name : columns_order_hint) + { + auto it = available_columns.find(name); + if (it == available_columns.end()) return {}; - std::unordered_map available_columns; - for (const auto &[name, type]: columns_list) - available_columns.emplace(name, type); + res.emplace_back(name, it->second); + } + return res; +} - NamesAndTypesList res; - for (const auto &name: columns_order_hint) { - auto it = available_columns.find(name); - if (it == available_columns.end()) - return {}; +bool isRetryableSchemaInferenceError(int code) +{ + return code == ErrorCodes::EMPTY_DATA_PASSED || code == ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA; +} - res.emplace_back(name, it->second); +ColumnsDescription readSchemaFromFormat( + const String & format_name, + const std::optional & format_settings, + ReadBufferIterator & read_buffer_iterator, + bool retry, + ContextPtr & context, + std::unique_ptr & buf) +{ + NamesAndTypesList names_and_types; + if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) + { + auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, format_settings); + try + { + names_and_types = external_schema_reader->readSchema(); + } + catch (Exception & e) + { + e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + throw; } - return res; } - - bool isRetryableSchemaInferenceError(int code) { - return code == ErrorCodes::EMPTY_DATA_PASSED || code == ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA; - } - - ColumnsDescription readSchemaFromFormat( - const String &format_name, - const std::optional &format_settings, - ReadBufferIterator &read_buffer_iterator, - bool retry, - ContextPtr &context, - std::unique_ptr &buf) { - NamesAndTypesList names_and_types; - if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) { - auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, - format_settings); - try { - names_and_types = external_schema_reader->readSchema(); + else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) + { + std::string exception_messages; + SchemaReaderPtr schema_reader; + size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; + size_t iterations = 0; + ColumnsDescription cached_columns; + while (true) + { + bool is_eof = false; + try + { + buf = read_buffer_iterator(cached_columns); + if (!buf) + break; + is_eof = buf->eof(); } - catch (Exception &e) { + catch (Exception & e) + { e.addMessage(fmt::format( - "Cannot extract table structure from {} format file. You can specify the structure manually", - format_name)); + "Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); throw; } - } else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) { - std::string exception_messages; - SchemaReaderPtr schema_reader; - size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference - : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; - size_t iterations = 0; - ColumnsDescription cached_columns; - while (true) { - bool is_eof = false; - try { - buf = read_buffer_iterator(cached_columns); - if (!buf) - break; - is_eof = buf->eof(); - } - catch (Exception &e) { - e.addMessage(fmt::format( - "Cannot extract table structure from {} format file. You can specify the structure manually", - format_name)); - throw; - } - catch (...) { - auto exception_message = getCurrentExceptionMessage(false); - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file:\n{}\nYou can specify the structure manually", - format_name, - exception_message); + catch (...) + { + auto exception_message = getCurrentExceptionMessage(false); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file:\n{}\nYou can specify the structure manually", + format_name, + exception_message); + } + + ++iterations; + + if (is_eof) + { + auto exception_message = fmt::format("Cannot extract table structure from {} format file, file is empty", format_name); + + if (!retry) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message); + + exception_messages += "\n" + exception_message; + continue; + } + + try + { + schema_reader = FormatFactory::instance().getSchemaReader(format_name, *buf, context, format_settings); + schema_reader->setMaxRowsToRead(max_rows_to_read); + names_and_types = schema_reader->readSchema(); + break; + } + catch (...) + { + auto exception_message = getCurrentExceptionMessage(false); + if (schema_reader) + { + size_t rows_read = schema_reader->getNumRowsRead(); + assert(rows_read <= max_rows_to_read); + max_rows_to_read -= schema_reader->getNumRowsRead(); + if (rows_read != 0 && max_rows_to_read == 0) + { + exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; + if (iterations > 1) + { + exception_messages += "\n" + exception_message; + break; + } + retry = false; + } } - ++iterations; - - if (is_eof) { - auto exception_message = fmt::format( - "Cannot extract table structure from {} format file, file is empty", format_name); - - if (!retry) + if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) + { + try + { + throw; + } + catch (Exception & e) + { + e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + throw; + } + catch (...) + { throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "{}. You can specify the structure manually", exception_message); - - exception_messages += "\n" + exception_message; - continue; - } - - try { - schema_reader = FormatFactory::instance().getSchemaReader(format_name, *buf, context, - format_settings); - schema_reader->setMaxRowsToRead(max_rows_to_read); - names_and_types = schema_reader->readSchema(); - break; - } - catch (...) { - auto exception_message = getCurrentExceptionMessage(false); - if (schema_reader) { - size_t rows_read = schema_reader->getNumRowsRead(); - assert(rows_read <= max_rows_to_read); - max_rows_to_read -= schema_reader->getNumRowsRead(); - if (rows_read != 0 && max_rows_to_read == 0) { - exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; - if (iterations > 1) { - exception_messages += "\n" + exception_message; - break; - } - retry = false; - } + "Cannot extract table structure from {} format file. " + "Error: {}. You can specify the structure manually", + format_name, exception_message); } - - if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) { - try { - throw; - } - catch (Exception &e) { - e.addMessage(fmt::format( - "Cannot extract table structure from {} format file. You can specify the structure manually", - format_name)); - throw; - } - catch (...) { - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file. " - "Error: {}. You can specify the structure manually", - format_name, exception_message); - } - } - - exception_messages += "\n" + exception_message; } + + exception_messages += "\n" + exception_message; } + } - if (!cached_columns.empty()) - return cached_columns; + if (!cached_columns.empty()) + return cached_columns; - if (names_and_types.empty()) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "All attempts to extract table structure from files failed. " - "Errors:{}\nYou can specify the structure manually", exception_messages); + if (names_and_types.empty()) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "All attempts to extract table structure from files failed. " + "Errors:{}\nYou can specify the structure manually", exception_messages); - /// If we have "INSERT SELECT" query then try to order - /// columns as they are ordered in table schema for formats - /// without strict column order (like JSON and TSKV). - /// It will allow to execute simple data loading with query - /// "INSERT INTO table SELECT * FROM ..." - const auto &insertion_table = context->getInsertionTable(); - if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty()) { - auto storage = DatabaseCatalog::instance().getTable(insertion_table, context); - auto metadata = storage->getInMemoryMetadataPtr(); - auto names_in_storage = metadata->getColumns().getNamesOfPhysical(); - auto ordered_list = getOrderedColumnsList(names_and_types, names_in_storage); - if (ordered_list) - names_and_types = *ordered_list; - } - } else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "{} file format doesn't support schema inference. You must specify the structure manually", - format_name); - names_and_types.erase(std::remove_if(names_and_types.begin(), names_and_types.end(), - [](const NameAndTypePair &pair) { return pair.name.empty(); }), - names_and_types.end()); - return ColumnsDescription(names_and_types); + /// If we have "INSERT SELECT" query then try to order + /// columns as they are ordered in table schema for formats + /// without strict column order (like JSON and TSKV). + /// It will allow to execute simple data loading with query + /// "INSERT INTO table SELECT * FROM ..." + const auto & insertion_table = context->getInsertionTable(); + if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty()) + { + auto storage = DatabaseCatalog::instance().getTable(insertion_table, context); + auto metadata = storage->getInMemoryMetadataPtr(); + auto names_in_storage = metadata->getColumns().getNamesOfPhysical(); + auto ordered_list = getOrderedColumnsList(names_and_types, names_in_storage); + if (ordered_list) + names_and_types = *ordered_list; + } } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{} file format doesn't support schema inference. You must specify the structure manually", + format_name); + names_and_types.erase(std::remove_if(names_and_types.begin(), names_and_types.end(), + [](const NameAndTypePair& pair) { return pair.name.empty(); }), names_and_types.end()); + return ColumnsDescription(names_and_types); +} - ColumnsDescription - readSchemaFromFormat(const String &format_name, const std::optional &format_settings, - ReadBufferIterator &read_buffer_iterator, bool retry, ContextPtr &context) { - std::unique_ptr buf_out; - return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out); - } +ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional & format_settings, ReadBufferIterator & read_buffer_iterator, bool retry, ContextPtr & context) +{ + std::unique_ptr buf_out; + return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out); +} - SchemaCache::Key getKeyForSchemaCache(const String &source, const String &format, - const std::optional &format_settings, - const ContextPtr &context) { - return getKeysForSchemaCache({source}, format, format_settings, context).front(); - } +SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context) +{ + return getKeysForSchemaCache({source}, format, format_settings, context).front(); +} - static SchemaCache::Key - makeSchemaCacheKey(const String &source, const String &format, const String &additional_format_info) { - return SchemaCache::Key{source, format, additional_format_info}; - } +static SchemaCache::Key makeSchemaCacheKey(const String & source, const String & format, const String & additional_format_info) +{ + return SchemaCache::Key{source, format, additional_format_info}; +} - SchemaCache::Keys getKeysForSchemaCache(const Strings &sources, const String &format, - const std::optional &format_settings, - const ContextPtr &context) { - /// For some formats data schema depends on some settings, so it's possible that - /// two queries to the same source will get two different schemas. To process this - /// case we add some additional information specific for the format to the cache key. - /// For example, for Protobuf format additional information is the path to the schema - /// and message name. - String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, - format_settings); - SchemaCache::Keys cache_keys; - cache_keys.reserve(sources.size()); - std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), - [&](const auto &source) { return makeSchemaCacheKey(source, format, additional_format_info); }); - return cache_keys; - } +SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & format, const std::optional & format_settings, const ContextPtr & context) +{ + /// For some formats data schema depends on some settings, so it's possible that + /// two queries to the same source will get two different schemas. To process this + /// case we add some additional information specific for the format to the cache key. + /// For example, for Protobuf format additional information is the path to the schema + /// and message name. + String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, format_settings); + SchemaCache::Keys cache_keys; + cache_keys.reserve(sources.size()); + std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), [&](const auto & source){ return makeSchemaCacheKey(source, format, additional_format_info); }); + return cache_keys; +} } From b869572a5411ce22519152a9a650d7fa65c8c517 Mon Sep 17 00:00:00 2001 From: laimuxi <1721261216@qq.com> Date: Sat, 1 Apr 2023 15:20:26 +0800 Subject: [PATCH 0068/2223] reformat code --- src/Formats/ReadSchemaUtils.cpp | 81 +++++++++++++++++++++------------ 1 file changed, 51 insertions(+), 30 deletions(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 7f1c3bf3e63..d185b938ed6 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -1,13 +1,13 @@ -#include #include -#include -#include #include +#include +#include +#include #include -#include -#include #include +#include #include +#include namespace DB { @@ -20,8 +20,7 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } -static std::optional getOrderedColumnsList( - const NamesAndTypesList & columns_list, const Names & columns_order_hint) +static std::optional getOrderedColumnsList(const NamesAndTypesList & columns_list, const Names & columns_order_hint) { if (columns_list.size() != columns_order_hint.size()) return {}; @@ -65,7 +64,8 @@ ColumnsDescription readSchemaFromFormat( } catch (Exception & e) { - e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + e.addMessage( + fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); throw; } } @@ -73,7 +73,8 @@ ColumnsDescription readSchemaFromFormat( { std::string exception_messages; SchemaReaderPtr schema_reader; - size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; + size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference + : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; size_t iterations = 0; ColumnsDescription cached_columns; while (true) @@ -88,8 +89,8 @@ ColumnsDescription readSchemaFromFormat( } catch (Exception & e) { - e.addMessage(fmt::format( - "Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + e.addMessage( + fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); throw; } catch (...) @@ -109,7 +110,8 @@ ColumnsDescription readSchemaFromFormat( auto exception_message = fmt::format("Cannot extract table structure from {} format file, file is empty", format_name); if (!retry) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message); exception_messages += "\n" + exception_message; continue; @@ -132,7 +134,8 @@ ColumnsDescription readSchemaFromFormat( max_rows_to_read -= schema_reader->getNumRowsRead(); if (rows_read != 0 && max_rows_to_read == 0) { - exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; + exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting " + "input_format_max_rows_to_read_for_schema_inference"; if (iterations > 1) { exception_messages += "\n" + exception_message; @@ -150,15 +153,18 @@ ColumnsDescription readSchemaFromFormat( } catch (Exception & e) { - e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); + e.addMessage(fmt::format( + "Cannot extract table structure from {} format file. You can specify the structure manually", format_name)); throw; } catch (...) { - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file. " - "Error: {}. You can specify the structure manually", - format_name, exception_message); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file. " + "Error: {}. You can specify the structure manually", + format_name, + exception_message); } } @@ -170,9 +176,11 @@ ColumnsDescription readSchemaFromFormat( return cached_columns; if (names_and_types.empty()) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "All attempts to extract table structure from files failed. " - "Errors:{}\nYou can specify the structure manually", exception_messages); + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "All attempts to extract table structure from files failed. " + "Errors:{}\nYou can specify the structure manually", + exception_messages); /// If we have "INSERT SELECT" query then try to order /// columns as they are ordered in table schema for formats @@ -191,21 +199,29 @@ ColumnsDescription readSchemaFromFormat( } } else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "{} file format doesn't support schema inference. You must specify the structure manually", - format_name); - names_and_types.erase(std::remove_if(names_and_types.begin(), names_and_types.end(), - [](const NameAndTypePair& pair) { return pair.name.empty(); }), names_and_types.end()); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "{} file format doesn't support schema inference. You must specify the structure manually", + format_name); + names_and_types.erase( + std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }), + names_and_types.end()); return ColumnsDescription(names_and_types); } -ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional & format_settings, ReadBufferIterator & read_buffer_iterator, bool retry, ContextPtr & context) +ColumnsDescription readSchemaFromFormat( + const String & format_name, + const std::optional & format_settings, + ReadBufferIterator & read_buffer_iterator, + bool retry, + ContextPtr & context) { std::unique_ptr buf_out; return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out); } -SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context) +SchemaCache::Key getKeyForSchemaCache( + const String & source, const String & format, const std::optional & format_settings, const ContextPtr & context) { return getKeysForSchemaCache({source}, format, format_settings, context).front(); } @@ -215,7 +231,8 @@ static SchemaCache::Key makeSchemaCacheKey(const String & source, const String & return SchemaCache::Key{source, format, additional_format_info}; } -SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & format, const std::optional & format_settings, const ContextPtr & context) +SchemaCache::Keys getKeysForSchemaCache( + const Strings & sources, const String & format, const std::optional & format_settings, const ContextPtr & context) { /// For some formats data schema depends on some settings, so it's possible that /// two queries to the same source will get two different schemas. To process this @@ -225,7 +242,11 @@ SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, format_settings); SchemaCache::Keys cache_keys; cache_keys.reserve(sources.size()); - std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), [&](const auto & source){ return makeSchemaCacheKey(source, format, additional_format_info); }); + std::transform( + sources.begin(), + sources.end(), + std::back_inserter(cache_keys), + [&](const auto & source) { return makeSchemaCacheKey(source, format, additional_format_info); }); return cache_keys; } From 892e436046f3f7bb135c5df8b18a1951833dd29f Mon Sep 17 00:00:00 2001 From: pufit Date: Sun, 2 Apr 2023 16:51:10 -0400 Subject: [PATCH 0069/2223] Move host and port to options --- programs/keeper-client/KeeperClient.cpp | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 54b5cf1d6cf..52a31a388cc 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -85,9 +85,19 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) Poco::Util::Application::defineOptions(options); options.addOption( - Poco::Util::Option("help", "h", "show help and exit") + Poco::Util::Option("help", "", "show help and exit") .binding("help")); + options.addOption( + Poco::Util::Option("host", "h", "server hostname. default `localhost`") + .argument("host") + .binding("host")); + + options.addOption( + Poco::Util::Option("port", "p", "server port. default `2181`") + .argument("port") + .binding("port")); + options.addOption( Poco::Util::Option("query", "q", "will execute given query, then exit.") .argument("query") @@ -284,13 +294,12 @@ void KeeperClient::runInteractive() } } -int KeeperClient::main(const std::vector & args) +int KeeperClient::main(const std::vector & /* args */) { - if (args.empty()) - zk_args.hosts = {"localhost:2181"}; - else - zk_args.hosts = {args[0]}; + auto host = config().getString("host", "localhost"); + auto port = config().getString("port", "2181"); + zk_args.hosts = {host + ":" + port}; zk_args.connection_timeout_ms = config().getInt("connection-timeout", 10) * 1000; zk_args.session_timeout_ms = config().getInt("session-timeout", 10) * 1000; zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; From e36addb96a7eaaba8f9a90383d3e77020a1a61e8 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 11 Apr 2023 13:03:03 +0200 Subject: [PATCH 0070/2223] Hackish way of setting up timezone on the client Warning: lots of debug logging --- programs/client/Client.cpp | 12 ++++- src/Client/ClientBase.cpp | 47 +++++++++++++++++++ src/Functions/timezoneOf.cpp | 13 +++++ src/Interpreters/Context.cpp | 9 ++++ src/Server/TCPHandler.cpp | 21 ++++++--- .../0_stateless/02668_timezone_setting.sql | 8 ++-- 6 files changed, 99 insertions(+), 11 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 660b8d7c00a..2aa75e60294 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -4,8 +4,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -307,7 +309,7 @@ int Client::main(const std::vector & /*args*/) try { UseSSL use_ssl; - MainThreadStatus::getInstance(); + auto & thread_status = MainThreadStatus::getInstance(); setupSignalHandler(); std::cout << std::fixed << std::setprecision(3); @@ -320,6 +322,14 @@ try processConfig(); initTtyBuffer(toProgressOption(config().getString("progress", "default"))); + { + // All that just to set DB::CurrentThread::get().getGlobalContext() + // which is required for client timezone (pushed as from server) to work. + auto thread_group = std::make_shared(); + thread_group->global_context = global_context; + thread_status.attachQuery(thread_group, false); + } + /// Includes delayed_interactive. if (is_interactive) { diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index ca32b9b97d7..e3e0364523a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -65,6 +65,7 @@ #include #include +#include #include #include #include @@ -73,11 +74,44 @@ #include "config_version.h" #include "config.h" +#include + namespace fs = std::filesystem; using namespace std::literals; +namespace +{ +using namespace DB; +using ContetGetterFunc = std::function const; +const void* getContextPtrOrNull(ContetGetterFunc contextFunc) +{ + try + { + return contextFunc().get(); + } + catch(...) + { + } + return nullptr; +} + +void LogContextes(const std::string_view scope, const ContextPtr global_context) +{ + const auto * context = global_context.get(); + std::cerr << scope << " contextes" + << "\n\tglobal: " << reinterpret_cast(context) + << "\n\tsession: " << getContextPtrOrNull([&]() { return context ? context->getSessionContext() : nullptr; }) + << "\n\tquery: " << getContextPtrOrNull([&]() { return context ? context->getQueryContext() : nullptr; }) + << "\n\tcurrent T query: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getQueryContext(); }) + << "\n\tcurrent T global: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getGlobalContext(); }) +// << "\n\tbuffer: " << getContextPtrOrNull(context, &Context::getBufferContext) + << std::endl; +} + +} + namespace CurrentMetrics { extern const Metric MemoryTracking; @@ -438,7 +472,12 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) /// output_format, do not output it. /// Also do not output too much data if we're fuzzing. if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) + { + LogContextes("ClientBase::onData header", global_context); return; + } + + LogContextes("ClientBase::onData DATA block", global_context); /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout)) @@ -1048,7 +1087,15 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { + std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; DateLUT::setDefaultTimezone(tz); + + Settings settings; + settings.timezone = tz; + global_context->applySettingsChanges(settings.changes()); +// DB::CurrentThread::get().getQueryContext()->applySettingsChanges(settings.changes()); + + LogContextes("ClientBase::onTimezoneUpdate", global_context); } diff --git a/src/Functions/timezoneOf.cpp b/src/Functions/timezoneOf.cpp index 6454b1cd735..ce419b7b4cd 100644 --- a/src/Functions/timezoneOf.cpp +++ b/src/Functions/timezoneOf.cpp @@ -5,7 +5,11 @@ #include #include #include +#include "Poco/Logger.h" +#include +#include +#include namespace DB { @@ -52,6 +56,15 @@ public: { DataTypePtr type_no_nullable = removeNullable(arguments[0].type); + { + const auto query_context = DB::CurrentThread::get().getQueryContext(); + + LOG_DEBUG(&Poco::Logger::get("Function timezoneOf"), "query context: {}, timezone: {} ({})", + reinterpret_cast(query_context.get()), + query_context->getSettingsRef().timezone.toString(), + (query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED")); + } + return DataTypeString().createColumnConst(input_rows_count, dynamic_cast(*type_no_nullable).getTimeZone().getTimeZone()); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cf1d5203bf7..e27889702c5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -118,6 +118,8 @@ #include #endif +#include + namespace fs = std::filesystem; namespace ProfileEvents @@ -1543,6 +1545,13 @@ void Context::applySettingChange(const SettingChange & change) void Context::applySettingsChanges(const SettingsChanges & changes) { auto lock = getLock(); + LOG_DEBUG(shared->log, "Context::applySettingsChanges {} applying settings changes: {}", reinterpret_cast(this), + fmt::join(std::ranges::transform_view(changes, + [](const SettingChange & change) + { + return change.name + ": " + change.value.dump(); + }), ", ")); + for (const SettingChange & change : changes) applySettingChange(change); applySettingsQuirks(settings); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ef4bf81a5c1..4d5402d65d5 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -345,6 +345,7 @@ void TCPHandler::runImpl() /// Send block to the client - input storage structure. state.input_header = metadata_snapshot->getSampleBlock(); sendData(state.input_header); + sendTimezone(); }); query_context->setInputBlocksReaderCallback([this] (ContextPtr context) -> Block @@ -452,9 +453,7 @@ void TCPHandler::runImpl() if (isQueryCancelled()) return true; - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES - && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) - sendTimezone(); +// sendTimezone(); sendProgress(); sendSelectProfileEvents(); sendLogs(); @@ -496,9 +495,7 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES - && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) - sendTimezone(); +// sendTimezone(); sendLogs(); sendEndOfStream(); } @@ -764,7 +761,7 @@ void TCPHandler::processInsertQuery() /// Send block to the client - table structure. sendData(executor.getHeader()); - + sendTimezone(); sendLogs(); while (readDataNext()) @@ -809,6 +806,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors() { std::lock_guard lock(task_callback_mutex); sendData(header); + sendTimezone(); } } @@ -1061,7 +1059,16 @@ void TCPHandler::sendInsertProfileEvents() void TCPHandler::sendTimezone() { +// if (client_tcp_protocol_version <= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES +// || client_tcp_protocol_version <= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) +// return; + // const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString(); + LOG_DEBUG(log, "TCPHandler::sendTimezone() query context: {}, timezone: {} ({})", + reinterpret_cast(query_context.get()), + query_context->getSettingsRef().timezone.toString(), + (query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED")); + const String & tz = query_context->getSettingsRef().timezone.toString(); if (!tz.empty()) { diff --git a/tests/queries/0_stateless/02668_timezone_setting.sql b/tests/queries/0_stateless/02668_timezone_setting.sql index f331ab58307..d85efaa8a39 100644 --- a/tests/queries/0_stateless/02668_timezone_setting.sql +++ b/tests/queries/0_stateless/02668_timezone_setting.sql @@ -1,9 +1,11 @@ +SET timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} + SET timezone = 'Asia/Novosibirsk'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; -SET timezone = 'Europe/Zurich'; +SET timezone = 'Asia/Manila'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); -SET timezone = 'Абырвалг'; -select now(); -- { serverError POCO_EXCEPTION } \ No newline at end of file +SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich'; +SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn'; From f087f0e87733eeb2672dc989e792f3c85d462601 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 11 Apr 2023 14:18:16 +0200 Subject: [PATCH 0071/2223] Update src/Formats/ReadSchemaUtils.cpp --- src/Formats/ReadSchemaUtils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index d185b938ed6..f80d9ee41d2 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -203,6 +203,7 @@ ColumnsDescription readSchemaFromFormat( ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference. You must specify the structure manually", format_name); + /// Some formats like CSVWithNames can contain empty column names. We don't support empty column names and futher processing can fail with an exception. Let's just remove columns with empty names from the structure. names_and_types.erase( std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }), names_and_types.end()); From 5d18343fb8ac1b0cae8085a660b8c995b9e33ea2 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 00:15:07 +0200 Subject: [PATCH 0072/2223] fixed delay --- src/Client/ClientBase.cpp | 36 +++++++++++++++++------------------- src/Server/TCPHandler.cpp | 12 ++++++------ 2 files changed, 23 insertions(+), 25 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e3e0364523a..8da4ac200d9 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -65,7 +65,7 @@ #include #include -#include +//#include #include #include #include @@ -74,7 +74,7 @@ #include "config_version.h" #include "config.h" -#include +//#include namespace fs = std::filesystem; @@ -97,18 +97,18 @@ const void* getContextPtrOrNull(ContetGetterFunc contextFunc) return nullptr; } -void LogContextes(const std::string_view scope, const ContextPtr global_context) -{ - const auto * context = global_context.get(); - std::cerr << scope << " contextes" - << "\n\tglobal: " << reinterpret_cast(context) - << "\n\tsession: " << getContextPtrOrNull([&]() { return context ? context->getSessionContext() : nullptr; }) - << "\n\tquery: " << getContextPtrOrNull([&]() { return context ? context->getQueryContext() : nullptr; }) - << "\n\tcurrent T query: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getQueryContext(); }) - << "\n\tcurrent T global: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getGlobalContext(); }) -// << "\n\tbuffer: " << getContextPtrOrNull(context, &Context::getBufferContext) - << std::endl; -} +//void LogContextes(const std::string_view scope, const ContextPtr global_context) +//{ +// const auto * context = global_context.get(); +// std::cerr << scope << " contextes" +// << "\n\tglobal: " << reinterpret_cast(context) +// << "\n\tsession: " << getContextPtrOrNull([&]() { return context ? context->getSessionContext() : nullptr; }) +// << "\n\tquery: " << getContextPtrOrNull([&]() { return context ? context->getQueryContext() : nullptr; }) +// << "\n\tcurrent T query: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getQueryContext(); }) +// << "\n\tcurrent T global: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getGlobalContext(); }) +//// << "\n\tbuffer: " << getContextPtrOrNull(context, &Context::getBufferContext) +// << std::endl; +//} } @@ -473,11 +473,11 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) /// Also do not output too much data if we're fuzzing. if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) { - LogContextes("ClientBase::onData header", global_context); +// LogContextes("ClientBase::onData header", global_context); return; } - LogContextes("ClientBase::onData DATA block", global_context); +// LogContextes("ClientBase::onData DATA block", global_context); /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout)) @@ -1088,14 +1088,12 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; - DateLUT::setDefaultTimezone(tz); Settings settings; settings.timezone = tz; global_context->applySettingsChanges(settings.changes()); -// DB::CurrentThread::get().getQueryContext()->applySettingsChanges(settings.changes()); - LogContextes("ClientBase::onTimezoneUpdate", global_context); +// LogContextes("ClientBase::onTimezoneUpdate", global_context); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4d5402d65d5..6ff7acf025a 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -453,7 +453,7 @@ void TCPHandler::runImpl() if (isQueryCancelled()) return true; -// sendTimezone(); + sendTimezone(); sendProgress(); sendSelectProfileEvents(); sendLogs(); @@ -495,7 +495,7 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); -// sendTimezone(); + sendTimezone(); sendLogs(); sendEndOfStream(); } @@ -1069,14 +1069,14 @@ void TCPHandler::sendTimezone() query_context->getSettingsRef().timezone.toString(), (query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED")); - const String & tz = query_context->getSettingsRef().timezone.toString(); - if (!tz.empty()) - { + const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString(); +// if (!tz.empty()) +// { LOG_DEBUG(log, "Sent timezone: {}", tz); writeVarUInt(Protocol::Server::TimezoneUpdate, *out); writeStringBinary(tz, *out); out->next(); - } +// } } From a9499eed794731a3fed2305e4d5f0e3607815816 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 12:47:05 +0200 Subject: [PATCH 0073/2223] moved getting server TZ DateLUT to separate place, upd tests and fix --- programs/copier/ClusterCopierApp.cpp | 2 +- programs/keeper/Keeper.cpp | 4 +-- programs/obfuscator/Obfuscator.cpp | 2 +- programs/server/Server.cpp | 4 +-- src/Client/ClientBase.cpp | 26 +++++++++---------- src/Common/DateLUT.h | 7 +++++ src/Daemon/BaseDaemon.cpp | 2 +- src/Functions/serverConstants.cpp | 2 +- src/Loggers/OwnPatternFormatter.h | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- .../MergeTree/MergeFromLogEntryTask.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 6 ++--- .../MergeTree/MergeTreeMutationEntry.cpp | 2 +- src/Storages/MergeTree/MergeTreePartInfo.cpp | 4 +-- src/Storages/MergeTree/MergeTreePartition.cpp | 4 +-- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 2 +- .../ReplicatedMergeTreeMutationEntry.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 2 +- .../02668_timezone_setting.reference | 3 --- .../0_stateless/02668_timezone_setting.sql | 11 -------- .../02681_timezone_setting.reference | 5 ++++ .../0_stateless/02681_timezone_setting.sql | 11 ++++++++ 23 files changed, 59 insertions(+), 50 deletions(-) delete mode 100644 tests/queries/0_stateless/02668_timezone_setting.reference delete mode 100644 tests/queries/0_stateless/02668_timezone_setting.sql create mode 100644 tests/queries/0_stateless/02681_timezone_setting.reference create mode 100644 tests/queries/0_stateless/02681_timezone_setting.sql diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index b2994b90e23..92657f81c2a 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -43,7 +43,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) time_t timestamp = Poco::Timestamp().epochTime(); auto curr_pid = Poco::Process::id(); - process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); + process_id = std::to_string(DateLUT::serverTimezoneInstance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; process_path = fs::weakly_canonical(fs::path(base_dir) / ("clickhouse-copier_" + process_id)); fs::create_directories(process_path); diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index ed3297ed7cb..58a87057363 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -399,8 +399,8 @@ try /// Initialize DateLUT early, to not interfere with running time of first query. LOG_DEBUG(log, "Initializing DateLUT."); - DateLUT::instance(); - LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); + DateLUT::serverTimezoneInstance(); + LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::serverTimezoneInstance().getTimeZone()); /// Don't want to use DNS cache DNSResolver::instance().setDisableCacheFlag(); diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 274ad29a174..9b7f2c424d3 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -492,7 +492,7 @@ private: const DateLUTImpl & date_lut; public: - explicit DateTimeModel(UInt64 seed_) : seed(seed_), date_lut(DateLUT::instance()) {} + explicit DateTimeModel(UInt64 seed_) : seed(seed_), date_lut(DateLUT::serverTimezoneInstance()) {} void train(const IColumn &) override {} void finalize() override {} diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 711dfb3820a..23113686aa1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1041,8 +1041,8 @@ try /// Initialize DateLUT early, to not interfere with running time of first query. LOG_DEBUG(log, "Initializing DateLUT."); - DateLUT::instance(); - LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); + DateLUT::serverTimezoneInstance(); + LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::serverTimezoneInstance().getTimeZone()); /// Storage with temporary data for processing of heavy queries. if (!server_settings.tmp_policy.value.empty()) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8da4ac200d9..7a91a382787 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -84,18 +84,18 @@ using namespace std::literals; namespace { using namespace DB; -using ContetGetterFunc = std::function const; -const void* getContextPtrOrNull(ContetGetterFunc contextFunc) -{ - try - { - return contextFunc().get(); - } - catch(...) - { - } - return nullptr; -} +//using ContetGetterFunc = std::function const; +//const void* getContextPtrOrNull(ContetGetterFunc contextFunc) +//{ +// try +// { +// return contextFunc().get(); +// } +// catch(...) +// { +// } +// return nullptr; +//} //void LogContextes(const std::string_view scope, const ContextPtr global_context) //{ @@ -1087,7 +1087,7 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { - std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; +// std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; Settings settings; settings.timezone = tz; diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index f17fe772dbc..810810edb6c 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -17,6 +17,13 @@ class DateLUT : private boost::noncopyable { public: + /// Return singleton DateLUTImpl instance for server's (native) time zone. + static ALWAYS_INLINE const DateLUTImpl & serverTimezoneInstance() + { + const auto & date_lut = getInstance(); + return *date_lut.default_impl.load(std::memory_order_acquire); + } + /// Return singleton DateLUTImpl instance for timezone set by `timezone` setting for current session is used. /// If it is not set, server's timezone (the one which server has) is being used. static ALWAYS_INLINE const DateLUTImpl & instance() diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 18c4c0d97a0..c6b5be3ea87 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -1003,7 +1003,7 @@ void BaseDaemon::shouldSetupWatchdog(char * argv0_) void BaseDaemon::setupWatchdog() { /// Initialize in advance to avoid double initialization in forked processes. - DateLUT::instance(); + DateLUT::serverTimezoneInstance(); std::string original_process_name; if (argv0) diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 57a6279bd7a..5d54815818d 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -75,7 +75,7 @@ namespace public: static constexpr auto name = "serverTimezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance("").getTimeZone()}, context->isDistributed()) {} + explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::serverTimezoneInstance().getTimeZone()}, context->isDistributed()) {} }; diff --git a/src/Loggers/OwnPatternFormatter.h b/src/Loggers/OwnPatternFormatter.h index 07d0409b0ae..8b0d11bcec1 100644 --- a/src/Loggers/OwnPatternFormatter.h +++ b/src/Loggers/OwnPatternFormatter.h @@ -31,6 +31,6 @@ public: virtual void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text) const; private: - const DateLUTImpl & server_timezone = DateLUT::instance(""); + const DateLUTImpl & server_timezone = DateLUT::serverTimezoneInstance(); bool color; }; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 85420cabb8d..f50a7169d39 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1030,7 +1030,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() DayNum max_date; MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date); - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); partition = MergeTreePartition(date_lut.toNumYYYYMM(min_date)); minmax_idx = std::make_shared(min_date, max_date); } diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index e017c9681e8..28e30b5f64f 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -57,7 +57,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() { LOG_INFO(log, "Will try to fetch part {} until '{}' because this part assigned to recompression merge. " "Source replica {} will try to merge this part first", entry.new_part_name, - DateLUT::instance().timeToString(entry.create_time + storage_settings_ptr->try_fetch_recompressed_part_timeout.totalSeconds()), entry.source_replica); + DateLUT::serverTimezoneInstance().timeToString(entry.create_time + storage_settings_ptr->try_fetch_recompressed_part_timeout.totalSeconds()), entry.source_replica); /// Waiting other replica to recompress part. No need to check it. return PrepareResult{ .prepared_successfully = false, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 37cfe4d065e..48a1cb97c89 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -115,7 +115,7 @@ void updateTTL( if (const ColumnUInt16 * column_date = typeid_cast(ttl_column.get())) { - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); for (const auto & val : column_date->getData()) ttl_info.update(date_lut.fromDayNum(DayNum(val))); } @@ -128,7 +128,7 @@ void updateTTL( { if (typeid_cast(&column_const->getDataColumn())) { - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); ttl_info.update(date_lut.fromDayNum(DayNum(column_const->getValue()))); } else if (typeid_cast(&column_const->getDataColumn())) @@ -369,7 +369,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( DayNum min_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].left.get()); DayNum max_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].right.get()); - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); auto min_month = date_lut.toNumYYYYMM(min_date); auto max_month = date_lut.toNumYYYYMM(max_date); diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 2e30a3f3986..2c0359b0f3f 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -127,7 +127,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat LocalDateTime create_time_dt; *buf >> "create time: " >> create_time_dt >> "\n"; - create_time = DateLUT::instance().makeDateTime( + create_time = DateLUT::serverTimezoneInstance().makeDateTime( create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); diff --git a/src/Storages/MergeTree/MergeTreePartInfo.cpp b/src/Storages/MergeTree/MergeTreePartInfo.cpp index 84432a293d7..e1b52d8a7b7 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -148,7 +148,7 @@ void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & part_name, D throw Exception(ErrorCodes::BAD_DATA_PART_NAME, "Unexpected part name: {}", part_name); } - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); min_date = date_lut.YYYYMMDDToDayNum(min_yyyymmdd); max_date = date_lut.YYYYMMDDToDayNum(max_yyyymmdd); @@ -219,7 +219,7 @@ String MergeTreePartInfo::getPartNameV1() const String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) const { - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); /// Directory name for the part has form: `YYYYMMDD_YYYYMMDD_N_N_L`. diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 3b28012e7d6..b0fc34ac2f7 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -239,7 +239,7 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const result += '-'; if (typeid_cast(partition_key_sample.getByPosition(i).type.get())) - result += toString(DateLUT::instance().toNumYYYYMMDD(DayNum(value[i].safeGet()))); + result += toString(DateLUT::serverTimezoneInstance().toNumYYYYMMDD(DayNum(value[i].safeGet()))); else if (typeid_cast(partition_key_sample.getByPosition(i).type.get())) result += toString(value[i].get().toUnderType()); else @@ -320,7 +320,7 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit throw Exception( ErrorCodes::INVALID_PARTITION_VALUE, "Cannot parse partition_id: got unexpected Date: {}", date_yyyymmdd); - UInt32 date = DateLUT::instance().YYYYMMDDToDayNum(date_yyyymmdd); + UInt32 date = DateLUT::serverTimezoneInstance().YYYYMMDDToDayNum(date_yyyymmdd); res.emplace_back(date); break; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 79b0beb0933..ac956433eab 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -199,7 +199,7 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in, MergeTreeDataFor { LocalDateTime create_time_dt; in >> "create_time: " >> create_time_dt >> "\n"; - create_time = DateLUT::instance().makeDateTime( + create_time = DateLUT::serverTimezoneInstance().makeDateTime( create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp index 1efb3f6826b..17f3637e722 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp @@ -38,7 +38,7 @@ void ReplicatedMergeTreeMutationEntry::readText(ReadBuffer & in) LocalDateTime create_time_dt; in >> "create time: " >> create_time_dt >> "\n"; - create_time = DateLUT::instance().makeDateTime( + create_time = DateLUT::serverTimezoneInstance().makeDateTime( create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fe4a144deaa..356663496a6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5318,7 +5318,7 @@ String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { /// The date range is all month long. - const auto & lut = DateLUT::instance(); + const auto & lut = DateLUT::serverTimezoneInstance(); time_t start_time = lut.YYYYMMDDToDate(parse(part_info.partition_id + "01")); DayNum left_date = DayNum(lut.toDayNum(start_time).toUnderType()); DayNum right_date = DayNum(static_cast(left_date) + lut.daysInMonth(start_time) - 1); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 3471e4ea6bf..8546fdd3c9f 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1335,7 +1335,7 @@ ASTPtr StorageWindowView::innerQueryParser(const ASTSelectQuery & query) time_zone = &DateLUT::instance(window_view_timezone); } else - time_zone = &DateLUT::instance(); + time_zone = &DateLUT::serverTimezoneInstance(); return result; } diff --git a/tests/queries/0_stateless/02668_timezone_setting.reference b/tests/queries/0_stateless/02668_timezone_setting.reference deleted file mode 100644 index 8ed8024f652..00000000000 --- a/tests/queries/0_stateless/02668_timezone_setting.reference +++ /dev/null @@ -1,3 +0,0 @@ -1999-12-12 18:23:23.123 -1999-12-12 23:23:23.123 -1999-12-13 04:23:23.123 diff --git a/tests/queries/0_stateless/02668_timezone_setting.sql b/tests/queries/0_stateless/02668_timezone_setting.sql deleted file mode 100644 index d85efaa8a39..00000000000 --- a/tests/queries/0_stateless/02668_timezone_setting.sql +++ /dev/null @@ -1,11 +0,0 @@ -SET timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} - -SET timezone = 'Asia/Novosibirsk'; -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; - -SET timezone = 'Asia/Manila'; -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); - -SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich'; -SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn'; diff --git a/tests/queries/0_stateless/02681_timezone_setting.reference b/tests/queries/0_stateless/02681_timezone_setting.reference new file mode 100644 index 00000000000..8850d77ab03 --- /dev/null +++ b/tests/queries/0_stateless/02681_timezone_setting.reference @@ -0,0 +1,5 @@ +2022-12-12 17:23:23.123 +2022-12-12 23:23:23.123 +2022-12-12 22:23:23.123 +Europe/Zurich Europe/Zurich +Pacific/Pitcairn Pacific/Pitcairn diff --git a/tests/queries/0_stateless/02681_timezone_setting.sql b/tests/queries/0_stateless/02681_timezone_setting.sql new file mode 100644 index 00000000000..73afb4c029b --- /dev/null +++ b/tests/queries/0_stateless/02681_timezone_setting.sql @@ -0,0 +1,11 @@ +SET timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} + +SET timezone = 'Asia/Novosibirsk'; +SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); +SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; + +SET timezone = 'Asia/Manila'; +SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); + +SELECT timezone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich' FORMAT TSV; +SELECT timezone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn' FORMAT TSV; From daef5d818a5cdc6e358efaa49e56d90ee530e6bf Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 15:31:58 +0200 Subject: [PATCH 0074/2223] fix according to updates in ThreadStatus.h --- programs/client/Client.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index aa563198c82..528c504e555 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -325,9 +325,8 @@ try { // All that just to set DB::CurrentThread::get().getGlobalContext() // which is required for client timezone (pushed as from server) to work. - auto thread_group = std::make_shared(); - thread_group->global_context = global_context; - thread_status.attachQuery(thread_group, false); + auto thread_group = std::make_shared(); + thread_status.attachToGroup(thread_group, false); } /// Includes delayed_interactive. From 3f8956f854253a5b17c6fa4163372f7e0f6cf664 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 17:45:11 +0200 Subject: [PATCH 0075/2223] remove additional logging --- src/Client/ClientBase.cpp | 30 ------------------------------ src/Functions/timezoneOf.cpp | 14 -------------- src/Interpreters/Context.cpp | 8 -------- src/Server/TCPHandler.cpp | 24 +++++++----------------- 4 files changed, 7 insertions(+), 69 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index d722d39e8f6..f4253ab90f6 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -66,7 +66,6 @@ #include #include -//#include #include #include #include @@ -75,9 +74,6 @@ #include "config_version.h" #include "config.h" -//#include - - namespace fs = std::filesystem; using namespace std::literals; @@ -85,32 +81,6 @@ using namespace std::literals; namespace { using namespace DB; -//using ContetGetterFunc = std::function const; -//const void* getContextPtrOrNull(ContetGetterFunc contextFunc) -//{ -// try -// { -// return contextFunc().get(); -// } -// catch(...) -// { -// } -// return nullptr; -//} - -//void LogContextes(const std::string_view scope, const ContextPtr global_context) -//{ -// const auto * context = global_context.get(); -// std::cerr << scope << " contextes" -// << "\n\tglobal: " << reinterpret_cast(context) -// << "\n\tsession: " << getContextPtrOrNull([&]() { return context ? context->getSessionContext() : nullptr; }) -// << "\n\tquery: " << getContextPtrOrNull([&]() { return context ? context->getQueryContext() : nullptr; }) -// << "\n\tcurrent T query: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getQueryContext(); }) -// << "\n\tcurrent T global: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getGlobalContext(); }) -//// << "\n\tbuffer: " << getContextPtrOrNull(context, &Context::getBufferContext) -// << std::endl; -//} - } namespace CurrentMetrics diff --git a/src/Functions/timezoneOf.cpp b/src/Functions/timezoneOf.cpp index ce419b7b4cd..7a5957a5dbc 100644 --- a/src/Functions/timezoneOf.cpp +++ b/src/Functions/timezoneOf.cpp @@ -5,11 +5,6 @@ #include #include #include -#include "Poco/Logger.h" - -#include -#include -#include namespace DB { @@ -56,15 +51,6 @@ public: { DataTypePtr type_no_nullable = removeNullable(arguments[0].type); - { - const auto query_context = DB::CurrentThread::get().getQueryContext(); - - LOG_DEBUG(&Poco::Logger::get("Function timezoneOf"), "query context: {}, timezone: {} ({})", - reinterpret_cast(query_context.get()), - query_context->getSettingsRef().timezone.toString(), - (query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED")); - } - return DataTypeString().createColumnConst(input_rows_count, dynamic_cast(*type_no_nullable).getTimeZone().getTimeZone()); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9e0b1dfd032..e888902ae29 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -119,8 +119,6 @@ #include #endif -#include - namespace fs = std::filesystem; namespace ProfileEvents @@ -1683,12 +1681,6 @@ void Context::applySettingChange(const SettingChange & change) void Context::applySettingsChanges(const SettingsChanges & changes) { auto lock = getLock(); - LOG_DEBUG(shared->log, "Context::applySettingsChanges {} applying settings changes: {}", reinterpret_cast(this), - fmt::join(std::ranges::transform_view(changes, - [](const SettingChange & change) - { - return change.name + ": " + change.value.dump(); - }), ", ")); for (const SettingChange & change : changes) applySettingChange(change); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a48097a649f..9a1b64eaf89 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -767,7 +767,6 @@ void TCPHandler::processInsertQuery() /// Send block to the client - table structure. sendData(executor.getHeader()); - sendTimezone(); sendLogs(); while (readDataNext()) @@ -1070,24 +1069,15 @@ void TCPHandler::sendInsertProfileEvents() void TCPHandler::sendTimezone() { -// if (client_tcp_protocol_version <= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES -// || client_tcp_protocol_version <= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) -// return; + if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES) + return; -// const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString(); - LOG_DEBUG(log, "TCPHandler::sendTimezone() query context: {}, timezone: {} ({})", - reinterpret_cast(query_context.get()), - query_context->getSettingsRef().timezone.toString(), - (query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED")); + const String & tz = query_context->getSettingsRef().timezone.toString(); - const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString(); -// if (!tz.empty()) -// { - LOG_DEBUG(log, "Sent timezone: {}", tz); - writeVarUInt(Protocol::Server::TimezoneUpdate, *out); - writeStringBinary(tz, *out); - out->next(); -// } + LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz); + writeVarUInt(Protocol::Server::TimezoneUpdate, *out); + writeStringBinary(tz, *out); + out->next(); } From 73675cd8d29ffd3c5e1b1a57a023ee1ac946ef8c Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 19:17:48 +0200 Subject: [PATCH 0076/2223] tryfix fasttest --- src/Server/TCPHandler.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9a1b64eaf89..152c7aba56e 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -458,7 +458,6 @@ void TCPHandler::runImpl() if (getQueryCancellationStatus() == CancellationStatus::FULLY_CANCELLED) return true; - sendTimezone(); sendProgress(); sendSelectProfileEvents(); sendLogs(); @@ -811,7 +810,6 @@ void TCPHandler::processOrdinaryQueryWithProcessors() { std::lock_guard lock(task_callback_mutex); sendData(header); - sendTimezone(); } } From 49c95a535ab5982a03b3dea731692893ed559806 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 12 Apr 2023 20:26:57 +0200 Subject: [PATCH 0077/2223] Updated to add error or completed status in zookeeper for a cluster for backup/restore, to avoid interpreting previously failed backup/restore when zookeeper is unable to remove nodes --- src/Backups/BackupCoordinationLocal.cpp | 2 +- src/Backups/BackupCoordinationLocal.h | 2 +- src/Backups/BackupCoordinationRemote.cpp | 12 ++++++---- src/Backups/BackupCoordinationRemote.h | 2 +- src/Backups/BackupCoordinationStage.h | 4 ++++ src/Backups/BackupCoordinationStageSync.cpp | 14 ++++++++++++ src/Backups/BackupCoordinationStageSync.h | 1 + src/Backups/BackupsWorker.cpp | 25 +++++++++++++-------- src/Backups/IBackupCoordination.h | 2 +- src/Backups/IRestoreCoordination.h | 2 +- src/Backups/RestoreCoordinationLocal.cpp | 2 +- src/Backups/RestoreCoordinationLocal.h | 2 +- src/Backups/RestoreCoordinationRemote.cpp | 12 ++++++---- src/Backups/RestoreCoordinationRemote.h | 2 +- 14 files changed, 59 insertions(+), 25 deletions(-) diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 27e0f173cf3..47b67693114 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -15,7 +15,7 @@ BackupCoordinationLocal::BackupCoordinationLocal(bool plain_backup_) BackupCoordinationLocal::~BackupCoordinationLocal() = default; -void BackupCoordinationLocal::setStage(const String &, const String &) +void BackupCoordinationLocal::setStage(const String &, const String &, const bool &) { } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 60fcc014720..1f6bb84972e 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -22,7 +22,7 @@ public: BackupCoordinationLocal(bool plain_backup_); ~BackupCoordinationLocal() override; - void setStage(const String & new_stage, const String & message) override; + void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 8e6b5db91b1..48f1ce3eef7 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -252,13 +252,17 @@ void BackupCoordinationRemote::removeAllNodes() } -void BackupCoordinationRemote::setStage(const String & new_stage, const String & message) +void BackupCoordinationRemote::setStage(const String & new_stage, const String & message, const bool & for_cluster) { - stage_sync->set(current_host, new_stage, message); + if (for_cluster) + stage_sync->setStageForCluster(new_stage); + else + stage_sync->set(current_host, new_stage, message); } void BackupCoordinationRemote::setError(const Exception & exception) { + stage_sync->setStageForCluster(Stage::ERROR); stage_sync->setError(current_host, exception); } @@ -777,8 +781,8 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) String status; if (zk->tryGet(root_zookeeper_path + "/" + existing_backup_path + "/stage", status)) { - /// If status is not COMPLETED it could be because the backup failed, check if 'error' exists - if (status != Stage::COMPLETED && !zk->exists(root_zookeeper_path + "/" + existing_backup_path + "/error")) + /// Check if some other restore is in progress + if (status == Stage::SCHEDULED_TO_START) { LOG_WARNING(log, "Found a concurrent backup: {}, current backup: {}", existing_backup_uuid, toString(backup_uuid)); result = true; diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index 949dd9c9bf0..40ce2ae6ccc 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -33,7 +33,7 @@ public: ~BackupCoordinationRemote() override; - void setStage(const String & new_stage, const String & message) override; + void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/BackupCoordinationStage.h b/src/Backups/BackupCoordinationStage.h index 40a4b262caa..41cd66346a2 100644 --- a/src/Backups/BackupCoordinationStage.h +++ b/src/Backups/BackupCoordinationStage.h @@ -43,6 +43,10 @@ namespace BackupCoordinationStage /// Coordination stage meaning that a host finished its work. constexpr const char * COMPLETED = "completed"; + + /// Coordination stage meaning that backup/restore has failed due to an error + /// Check '/error' for the error message + constexpr const char * ERROR = "error"; } } diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index effb00085c3..5cbeec0ec76 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -61,6 +61,20 @@ void BackupCoordinationStageSync::set(const String & current_host, const String }); } +void BackupCoordinationStageSync::setStageForCluster(const String & new_stage) +{ + auto holder = with_retries.createRetriesControlHolder("setStageForCluster"); + holder.retries_ctl.retryLoop( + [&, &zookeeper = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zookeeper); + zookeeper->trySet(zookeeper_path, new_stage); + auto code = zookeeper->trySet(zookeeper_path, new_stage); + if (code != Coordination::Error::ZOK) + throw zkutil::KeeperException(code, zookeeper_path); + }); +} + void BackupCoordinationStageSync::setError(const String & current_host, const Exception & exception) { auto holder = with_retries.createRetriesControlHolder("setError"); diff --git a/src/Backups/BackupCoordinationStageSync.h b/src/Backups/BackupCoordinationStageSync.h index 56081f8779c..9dde4e3095f 100644 --- a/src/Backups/BackupCoordinationStageSync.h +++ b/src/Backups/BackupCoordinationStageSync.h @@ -16,6 +16,7 @@ public: /// Sets the stage of the current host and signal other hosts if there were other hosts waiting for that. void set(const String & current_host, const String & new_stage, const String & message); + void setStageForCluster(const String & new_stage); void setError(const String & current_host, const Exception & exception); /// Sets the stage of the current host and waits until all hosts come to the same stage. diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 4b17174a8de..aae9cfd620f 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -368,6 +368,7 @@ void BackupsWorker::doBackup( /// Wait until all the hosts have written their backup entries. backup_coordination->waitForStage(Stage::COMPLETED); + backup_coordination->setStage(Stage::COMPLETED, /* message */ "", /* for_cluster */ true); } else { @@ -654,12 +655,26 @@ void BackupsWorker::doRestore( /// (If this isn't ON CLUSTER query RestorerFromBackup will check access rights later.) ClusterPtr cluster; bool on_cluster = !restore_query->cluster.empty(); + if (on_cluster) { restore_query->cluster = context->getMacros()->expand(restore_query->cluster); cluster = context->getCluster(restore_query->cluster); restore_settings.cluster_host_ids = cluster->getHostIDs(); + } + /// Make a restore coordination. + if (!restore_coordination) + restore_coordination = makeRestoreCoordination(context, restore_settings, /* remote= */ on_cluster); + + if (!allow_concurrent_restores && restore_coordination->hasConcurrentRestores(std::ref(num_active_restores))) + throw Exception( + ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, + "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'"); + + + if (on_cluster) + { /// We cannot just use access checking provided by the function executeDDLQueryOnCluster(): it would be incorrect /// because different replicas can contain different set of tables and so the required access rights can differ too. /// So the right way is pass through the entire cluster and check access for each host. @@ -676,15 +691,6 @@ void BackupsWorker::doRestore( } } - /// Make a restore coordination. - if (!restore_coordination) - restore_coordination = makeRestoreCoordination(context, restore_settings, /* remote= */ on_cluster); - - if (!allow_concurrent_restores && restore_coordination->hasConcurrentRestores(std::ref(num_active_restores))) - throw Exception( - ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, - "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'"); - /// Do RESTORE. if (on_cluster) { @@ -703,6 +709,7 @@ void BackupsWorker::doRestore( /// Wait until all the hosts have written their backup entries. restore_coordination->waitForStage(Stage::COMPLETED); + restore_coordination->setStage(Stage::COMPLETED, /* message */ "", /* for_cluster */ true); } else { diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 75d9202374b..614e6a16db8 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -21,7 +21,7 @@ public: virtual ~IBackupCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & new_stage, const String & message) = 0; + virtual void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) = 0; virtual void setError(const Exception & exception) = 0; virtual Strings waitForStage(const String & stage_to_wait) = 0; virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index 2f9e8d171f6..599a698a1f9 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -18,7 +18,7 @@ public: virtual ~IRestoreCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & new_stage, const String & message) = 0; + virtual void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) = 0; virtual void setError(const Exception & exception) = 0; virtual Strings waitForStage(const String & stage_to_wait) = 0; virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index 068c4fe7e52..f689277f5b6 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -11,7 +11,7 @@ RestoreCoordinationLocal::RestoreCoordinationLocal() : log(&Poco::Logger::get("R RestoreCoordinationLocal::~RestoreCoordinationLocal() = default; -void RestoreCoordinationLocal::setStage(const String &, const String &) +void RestoreCoordinationLocal::setStage(const String &, const String &, const bool &) { } diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index e27f0d1ef88..4456ad966d4 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -19,7 +19,7 @@ public: ~RestoreCoordinationLocal() override; /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & new_stage, const String & message) override; + void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index cc03f0c4a2a..0a89b1cd4e7 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -90,13 +90,17 @@ void RestoreCoordinationRemote::createRootNodes() }); } -void RestoreCoordinationRemote::setStage(const String & new_stage, const String & message) +void RestoreCoordinationRemote::setStage(const String & new_stage, const String & message, const bool & for_cluster) { - stage_sync->set(current_host, new_stage, message); + if (for_cluster) + stage_sync->setStageForCluster(new_stage); + else + stage_sync->set(current_host, new_stage, message); } void RestoreCoordinationRemote::setError(const Exception & exception) { + stage_sync->setStageForCluster(Stage::ERROR); stage_sync->setError(current_host, exception); } @@ -282,8 +286,8 @@ bool RestoreCoordinationRemote::hasConcurrentRestores(const std::atomic String status; if (zk->tryGet(root_zookeeper_path + "/" + existing_restore_path + "/stage", status)) { - /// If status is not COMPLETED it could be because the restore failed, check if 'error' exists - if (status != Stage::COMPLETED && !zk->exists(root_zookeeper_path + "/" + existing_restore_path + "/error")) + /// Check if some other restore is in progress + if (status == Stage::SCHEDULED_TO_START) { LOG_WARNING(log, "Found a concurrent restore: {}, current restore: {}", existing_restore_uuid, toString(restore_uuid)); result = true; diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index eb0fcff9c2d..21a38f01fa6 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -26,7 +26,7 @@ public: ~RestoreCoordinationRemote() override; /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & new_stage, const String & message) override; + void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; From 981a73cd867c435c74adf06cece8ec279fb8fde8 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 21:20:12 +0200 Subject: [PATCH 0078/2223] upd remotequeryexecutor on receive timezone --- src/QueryPipeline/RemoteQueryExecutor.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b7490a2ad9c..23c1412dc76 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -449,6 +449,9 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::processPacket(Packet packet throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push into profile queue"); break; + case Protocol::Server::TimezoneUpdate: + break; + default: got_unknown_packet_from_replica = true; throw Exception( @@ -546,6 +549,9 @@ void RemoteQueryExecutor::finish(std::unique_ptr * read_context) if (!profile_queue->emplace(std::move(packet.block))) throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push into profile queue"); break; + + case Protocol::Server::TimezoneUpdate: + break; default: got_unknown_packet_from_replica = true; From 0c616ac0a287ce017c18561013b87e576ac8e74b Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 21:31:13 +0200 Subject: [PATCH 0079/2223] fix style --- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 23c1412dc76..56b5357c522 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -549,7 +549,7 @@ void RemoteQueryExecutor::finish(std::unique_ptr * read_context) if (!profile_queue->emplace(std::move(packet.block))) throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push into profile queue"); break; - + case Protocol::Server::TimezoneUpdate: break; From 1ec32d374d8872aedd2f13bfdbbb263d98feed17 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 22:27:54 +0200 Subject: [PATCH 0080/2223] update Timezone packet processing --- src/Client/HedgedConnections.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 13e4fe75b3d..d11954f3838 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -418,6 +418,7 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli } replica_with_last_received_packet = replica_location; break; + case Protocol::Server::TimezoneUpdate: case Protocol::Server::PartUUIDs: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: From 16292eb5a18d1f410421217461a7b3e44b39dbec Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 22:54:51 +0200 Subject: [PATCH 0081/2223] update timezone packet handling in remote inserter --- src/QueryPipeline/RemoteInserter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteInserter.cpp b/src/QueryPipeline/RemoteInserter.cpp index b8a878b56c3..134c169e35f 100644 --- a/src/QueryPipeline/RemoteInserter.cpp +++ b/src/QueryPipeline/RemoteInserter.cpp @@ -130,7 +130,7 @@ void RemoteInserter::onFinish() break; else if (Protocol::Server::Exception == packet.type) packet.exception->rethrow(); - else if (Protocol::Server::Log == packet.type) + else if (Protocol::Server::Log == packet.type || Protocol::Server::TimezoneUpdate == packet.type) { // Do nothing } From d4b2297e9fa53336cd4d05919a1048ad742018cd Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 13 Apr 2023 09:53:39 +0200 Subject: [PATCH 0082/2223] Fixed comment --- src/Backups/BackupCoordinationRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 48f1ce3eef7..cd4901eb5ae 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -781,7 +781,7 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) String status; if (zk->tryGet(root_zookeeper_path + "/" + existing_backup_path + "/stage", status)) { - /// Check if some other restore is in progress + /// Check if some other backup is in progress if (status == Stage::SCHEDULED_TO_START) { LOG_WARNING(log, "Found a concurrent backup: {}, current backup: {}", existing_backup_uuid, toString(backup_uuid)); From bac5fbc3d2cf1b9606d66543244036797221a4b3 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Apr 2023 13:26:09 +0200 Subject: [PATCH 0083/2223] fix error on connection drop after 1st query --- src/Client/ClientBase.cpp | 4 ---- src/Client/Connection.cpp | 4 ++++ src/Server/TCPHandler.cpp | 4 +++- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f4253ab90f6..2e82144e64d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1074,13 +1074,9 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { -// std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; - Settings settings; settings.timezone = tz; global_context->applySettingsChanges(settings.changes()); - -// LogContextes("ClientBase::onTimezoneUpdate", global_context); } diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index a0025eafd64..08549265848 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -335,6 +335,10 @@ void Connection::receiveHello() nonce.emplace(read_nonce); } } + else if (packet_type == Protocol::Server::TimezoneUpdate) + { + // skip this packet at hello, will receive and process it later + } else if (packet_type == Protocol::Server::Exception) receiveException()->rethrow(); else diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 152c7aba56e..a875507d227 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -183,8 +183,11 @@ void TCPHandler::runImpl() /// User will be authenticated here. It will also set settings from user profile into connection_context. try { + LOG_DEBUG(log, "Before receiveHello"); receiveHello(); + LOG_DEBUG(log, "Before sendHello"); sendHello(); + LOG_DEBUG(log, "Before receiveAddendum"); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); @@ -499,7 +502,6 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); - sendTimezone(); sendLogs(); sendEndOfStream(); } From d5ea52e4b9b4005356f79e3eaadd4d6458fa116e Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 14 Apr 2023 01:28:59 +0200 Subject: [PATCH 0084/2223] optimize --- src/Common/DateLUT.h | 11 +++++------ src/Core/SettingsFields.h | 6 +++--- src/DataTypes/TimezoneMixin.h | 2 +- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 810810edb6c..59b280240ea 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -28,11 +28,11 @@ public: /// If it is not set, server's timezone (the one which server has) is being used. static ALWAYS_INLINE const DateLUTImpl & instance() { - std::string effective_time_zone; const auto & date_lut = getInstance(); if (DB::CurrentThread::isInitialized()) { + std::string effective_time_zone; const auto query_context = DB::CurrentThread::get().getQueryContext(); if (query_context) @@ -43,6 +43,8 @@ public: return date_lut.getImplementation(effective_time_zone); } + /// Timezone is passed in query_context, but on CH-Client we have no query context, + /// and each time we modify client's global context const auto global_context = DB::CurrentThread::get().getGlobalContext(); if (global_context) { @@ -56,15 +58,12 @@ public: return *date_lut.default_impl.load(std::memory_order_acquire); } - /// Return singleton DateLUTImpl instance for a given time zone. If timezone is an empty string, - /// server's timezone is used. The `timezone` setting is not considered here. static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { - const auto & date_lut = getInstance(); - if (time_zone.empty()) - return *date_lut.default_impl.load(std::memory_order_acquire); + return instance(); + const auto & date_lut = getInstance(); return date_lut.getImplementation(time_zone); } diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 8e9ffe03008..0ee3ddd4862 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -576,10 +576,10 @@ struct SettingFieldTimezone private: cctz::time_zone validated_tz; - void validateTimezone(const std::string & str) + void validateTimezone(const std::string & tz_str) { - if (str != "" && !cctz::load_time_zone(str, &validated_tz)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); + if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); } }; diff --git a/src/DataTypes/TimezoneMixin.h b/src/DataTypes/TimezoneMixin.h index 5b7870c7b9a..03ecde5dd0a 100644 --- a/src/DataTypes/TimezoneMixin.h +++ b/src/DataTypes/TimezoneMixin.h @@ -15,7 +15,7 @@ public: explicit TimezoneMixin(const String & time_zone_name = "") : has_explicit_time_zone(!time_zone_name.empty()) - , time_zone(time_zone_name.empty() ? DateLUT::instance() : DateLUT::instance(time_zone_name)) + , time_zone(DateLUT::instance(time_zone_name)) , utc_time_zone(DateLUT::instance("UTC")) { } From 74c6ca558b3301427368941f3e0df031b04cc10d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 14 Apr 2023 18:03:46 +0200 Subject: [PATCH 0085/2223] Removed line from test_disallow_concurrrency for CI checks --- .../test_backup_restore_on_cluster/test_disallow_concurrency.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index 0d8fad96438..a76af00d339 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -6,7 +6,6 @@ import concurrent from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV, assert_eq_with_retry - cluster = ClickHouseCluster(__file__) num_nodes = 10 From 93572ab42768195fccc77a809e149355a3f8065d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 15 Apr 2023 13:43:04 +0200 Subject: [PATCH 0086/2223] Removed parameter from setStage function and added function setStageForCluster --- src/Backups/BackupCoordinationLocal.cpp | 6 +++++- src/Backups/BackupCoordinationLocal.h | 3 ++- src/Backups/BackupCoordinationRemote.cpp | 12 +++++++----- src/Backups/BackupCoordinationRemote.h | 3 ++- src/Backups/BackupsWorker.cpp | 4 ++-- src/Backups/IBackupCoordination.h | 3 ++- src/Backups/IRestoreCoordination.h | 3 ++- src/Backups/RestoreCoordinationLocal.cpp | 6 +++++- src/Backups/RestoreCoordinationLocal.h | 3 ++- src/Backups/RestoreCoordinationRemote.cpp | 12 +++++++----- src/Backups/RestoreCoordinationRemote.h | 3 ++- 11 files changed, 38 insertions(+), 20 deletions(-) diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 47b67693114..5b7ee37618b 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -15,7 +15,11 @@ BackupCoordinationLocal::BackupCoordinationLocal(bool plain_backup_) BackupCoordinationLocal::~BackupCoordinationLocal() = default; -void BackupCoordinationLocal::setStage(const String &, const String &, const bool &) +void BackupCoordinationLocal::setStage(const String &, const String &) +{ +} + +void BackupCoordinationLocal::setStageForCluster(const String &) { } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 1f6bb84972e..f1ffa8e8517 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -22,7 +22,8 @@ public: BackupCoordinationLocal(bool plain_backup_); ~BackupCoordinationLocal() override; - void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) override; + void setStage(const String & new_stage, const String & message) override; + void setStageForCluster(const String & new_stage) override; /// Sets stage for cluster void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index cd4901eb5ae..c5c4efa3530 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -252,12 +252,14 @@ void BackupCoordinationRemote::removeAllNodes() } -void BackupCoordinationRemote::setStage(const String & new_stage, const String & message, const bool & for_cluster) +void BackupCoordinationRemote::setStage(const String & new_stage, const String & message) { - if (for_cluster) - stage_sync->setStageForCluster(new_stage); - else - stage_sync->set(current_host, new_stage, message); + stage_sync->set(current_host, new_stage, message); +} + +void BackupCoordinationRemote::setStageForCluster(const String & new_stage) +{ + stage_sync->setStageForCluster(new_stage); } void BackupCoordinationRemote::setError(const Exception & exception) diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index 40ce2ae6ccc..c659cb0d459 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -33,7 +33,8 @@ public: ~BackupCoordinationRemote() override; - void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) override; + void setStage(const String & new_stage, const String & message) override; + void setStageForCluster(const String & new_stage) override; /// Sets stage for cluster void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index aae9cfd620f..de05cc2b092 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -368,7 +368,7 @@ void BackupsWorker::doBackup( /// Wait until all the hosts have written their backup entries. backup_coordination->waitForStage(Stage::COMPLETED); - backup_coordination->setStage(Stage::COMPLETED, /* message */ "", /* for_cluster */ true); + backup_coordination->setStageForCluster(Stage::COMPLETED); } else { @@ -709,7 +709,7 @@ void BackupsWorker::doRestore( /// Wait until all the hosts have written their backup entries. restore_coordination->waitForStage(Stage::COMPLETED); - restore_coordination->setStage(Stage::COMPLETED, /* message */ "", /* for_cluster */ true); + restore_coordination->setStageForCluster(Stage::COMPLETED); } else { diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 614e6a16db8..6caae1dd741 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -21,7 +21,8 @@ public: virtual ~IBackupCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) = 0; + virtual void setStage(const String & new_stage, const String & message) = 0; + virtual void setStageForCluster(const String & new_stage) = 0; virtual void setError(const Exception & exception) = 0; virtual Strings waitForStage(const String & stage_to_wait) = 0; virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index 599a698a1f9..a5c8db84c86 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -18,7 +18,8 @@ public: virtual ~IRestoreCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) = 0; + virtual void setStage(const String & new_stage, const String & message) = 0; + virtual void setStageForCluster(const String & new_stage) = 0; /// Sets stage for cluster virtual void setError(const Exception & exception) = 0; virtual Strings waitForStage(const String & stage_to_wait) = 0; virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index f689277f5b6..513204c931c 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -11,7 +11,11 @@ RestoreCoordinationLocal::RestoreCoordinationLocal() : log(&Poco::Logger::get("R RestoreCoordinationLocal::~RestoreCoordinationLocal() = default; -void RestoreCoordinationLocal::setStage(const String &, const String &, const bool &) +void RestoreCoordinationLocal::setStage(const String &, const String &) +{ +} + +void RestoreCoordinationLocal::setStageForCluster(const String &) { } diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index 4456ad966d4..0e4f4f01846 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -19,7 +19,8 @@ public: ~RestoreCoordinationLocal() override; /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) override; + void setStage(const String & new_stage, const String & message) override; + void setStageForCluster(const String & new_stage) override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 0a89b1cd4e7..2c2187a1eb5 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -90,12 +90,14 @@ void RestoreCoordinationRemote::createRootNodes() }); } -void RestoreCoordinationRemote::setStage(const String & new_stage, const String & message, const bool & for_cluster) +void RestoreCoordinationRemote::setStage(const String & new_stage, const String & message) { - if (for_cluster) - stage_sync->setStageForCluster(new_stage); - else - stage_sync->set(current_host, new_stage, message); + stage_sync->set(current_host, new_stage, message); +} + +void RestoreCoordinationRemote::setStageForCluster(const String & new_stage) +{ + stage_sync->setStageForCluster(new_stage); } void RestoreCoordinationRemote::setError(const Exception & exception) diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 21a38f01fa6..947d08a66e5 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -26,7 +26,8 @@ public: ~RestoreCoordinationRemote() override; /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & new_stage, const String & message, const bool & for_cluster = false) override; + void setStage(const String & new_stage, const String & message) override; + void setStageForCluster(const String & new_stage) override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; From 267bbcab007d02748af2b2b18c63de73c4fa327b Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Fri, 14 Apr 2023 00:09:57 +0300 Subject: [PATCH 0087/2223] Added ability to implicitly use file table function in clickhouse-local --- programs/local/LocalServer.cpp | 3 +- src/Databases/DatabaseFactory.cpp | 21 +- src/Databases/DatabaseFileSystem.cpp | 132 +++++++++ src/Databases/DatabaseFileSystem.h | 51 ++++ src/Databases/DatabasesOverlay.cpp | 267 ++++++++++++++++++ src/Databases/DatabasesOverlay.h | 68 +++++ ...cal_implicit_file_table_function.reference | 9 + ...ouse_local_implicit_file_table_function.sh | 43 +++ 8 files changed, 591 insertions(+), 3 deletions(-) create mode 100644 src/Databases/DatabaseFileSystem.cpp create mode 100644 src/Databases/DatabaseFileSystem.h create mode 100644 src/Databases/DatabasesOverlay.cpp create mode 100644 src/Databases/DatabasesOverlay.h create mode 100644 tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.reference create mode 100755 tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 5768e744f94..566d11791ca 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -647,7 +648,7 @@ void LocalServer::processConfig() * if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons. */ std::string default_database = config().getString("default_database", "_local"); - DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, global_context)); + DatabaseCatalog::instance().attachDatabase(default_database, CreateClickHouseLocalDatabaseOverlay(default_database, global_context)); global_context->setCurrentDatabase(default_database); applyCmdOptions(global_context); diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 89a799349bf..b023bb06ad1 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -132,13 +133,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String static const std::unordered_set database_engines{"Ordinary", "Atomic", "Memory", "Dictionary", "Lazy", "Replicated", "MySQL", "MaterializeMySQL", "MaterializedMySQL", - "PostgreSQL", "MaterializedPostgreSQL", "SQLite"}; + "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "FileSystem"}; if (!database_engines.contains(engine_name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine name `{}` does not exist", engine_name); static const std::unordered_set engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL", - "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite"}; + "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "FileSystem"}; static const std::unordered_set engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"}; bool engine_may_have_arguments = engines_with_arguments.contains(engine_name); @@ -432,6 +433,22 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared(context, engine_define, create.attach, database_path); } #endif + else if (engine_name == "FileSystem") { + const ASTFunction * engine = engine_define->engine; + + // If init_path is empty, then the current path from Poco will be used + std::string init_path; + + if (engine->arguments && engine->arguments->children.size() > 0) { + if (engine->arguments->children.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "FileSystem database requires at most 1 argument: file_system_path"); + + const auto & arguments = engine->arguments->children; + init_path = safeGetLiteralValue(arguments[0], engine_name); + } + + return std::make_shared(database_name, init_path, context); + } throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", engine_name); } diff --git a/src/Databases/DatabaseFileSystem.cpp b/src/Databases/DatabaseFileSystem.cpp new file mode 100644 index 00000000000..9e2273970c3 --- /dev/null +++ b/src/Databases/DatabaseFileSystem.cpp @@ -0,0 +1,132 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +DatabaseFileSystem::DatabaseFileSystem(const String & name_, const String & path_, ContextPtr context_) + : IDatabase(name_), WithContext(context_->getGlobalContext()), path(path_), log(&Poco::Logger::get("DatabaseFileSystem(" + name_ + ")")) +{ + if (path.empty()) + path = Poco::Path::current(); +} + +std::string DatabaseFileSystem::getTablePath(const std::string& table_name) const +{ + return Poco::Path(path, table_name).toString(); +} + +void DatabaseFileSystem::addTable(const std::string& table_name, StoragePtr table_storage) const +{ + std::lock_guard lock(mutex); + loaded_tables.emplace(table_name, table_storage); +} + +bool DatabaseFileSystem::isTableExist(const String & name, ContextPtr) const +{ + { + std::lock_guard lock(mutex); + if (loaded_tables.find(name) != loaded_tables.end()) + return true; + } + + Poco::File table_file(getTablePath(name)); + return table_file.exists() && table_file.isFile(); +} + +StoragePtr DatabaseFileSystem::tryGetTable(const String & name, ContextPtr context_) const +{ + // Check if the table exists in the loaded tables map + { + std::lock_guard lock(mutex); + auto it = loaded_tables.find(name); + if (it != loaded_tables.end()) + return it->second; + } + + auto table_path = getTablePath(name); + + // If the table doesn't exist in the tables map, check if the corresponding file exists + Poco::File table_file(table_path); + if (!table_file.exists()) + return nullptr; + + // If the file exists, create a new table using TableFunctionFile and return it. + auto args = makeASTFunction("file", std::make_shared(table_path)); + + auto table_function = TableFunctionFactory::instance().get(args, context_); + if (!table_function) + return nullptr; + + auto table_storage = table_function->execute(args, context_, name); + if (table_storage) + addTable(name, table_storage); + + return table_storage; +} + +ASTPtr DatabaseFileSystem::getCreateDatabaseQuery() const +{ + auto settings = getContext()->getSettingsRef(); + ParserCreateQuery parser; + + String query = "CREATE DATABASE " + backQuoteIfNeed(getDatabaseName()) + " ENGINE = FileSystem(" + backQuoteIfNeed(path) + ")"; + ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth); + + if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) + { + auto & ast_create_query = ast->as(); + ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); + } + + return ast; +} + +void DatabaseFileSystem::shutdown() +{ + Tables tables_snapshot; + { + std::lock_guard lock(mutex); + tables_snapshot = loaded_tables; + } + + for (const auto & kv : tables_snapshot) + { + auto table_id = kv.second->getStorageID(); + kv.second->flushAndShutdown(); + } + + std::lock_guard lock(mutex); + loaded_tables.clear(); +} + +/** + * Returns an empty vector because the database is read-only and no tables can be backed up. + */ +std::vector> DatabaseFileSystem::getTablesForBackup(const FilterByNameFunction&, const ContextPtr&) const { + return {}; +} + +/** + * + * Returns an empty iterator because the database does not have its own tables + * But only caches them for quick access. + */ +DatabaseTablesIteratorPtr DatabaseFileSystem::getTablesIterator(ContextPtr, const FilterByNameFunction&) const { + return std::make_unique(Tables{}, getDatabaseName()); +} + +} // DB diff --git a/src/Databases/DatabaseFileSystem.h b/src/Databases/DatabaseFileSystem.h new file mode 100644 index 00000000000..474a7e78335 --- /dev/null +++ b/src/Databases/DatabaseFileSystem.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; + +/** + * DatabaseFileSystem allows to interact with files stored on the file system + * Uses TableFunctionFile to implicitly load file when a user requests the table, and provides read-only access to the data in the file + * Tables are cached inside the database for quick access + * + * Used in clickhouse-local to access local files + */ +class DatabaseFileSystem : public IDatabase, protected WithContext +{ +public: + DatabaseFileSystem(const String & name, const String & path, ContextPtr context); + + String getEngineName() const override { return "FileSystem"; } + + bool isTableExist(const String & name, ContextPtr context) const override; + + StoragePtr tryGetTable(const String & name, ContextPtr context) const override; + + bool empty() const override { return true; } + + ASTPtr getCreateDatabaseQuery() const override; + + void shutdown() override; + + std::vector> getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; + +protected: + std::string getTablePath(const std::string & table_name) const; + void addTable(const std::string & table_name, StoragePtr table_storage) const; + +private: + String path; + mutable Tables loaded_tables TSA_GUARDED_BY(mutex); + Poco::Logger * log; +}; + +} // DB diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp new file mode 100644 index 00000000000..9c3d802df73 --- /dev/null +++ b/src/Databases/DatabasesOverlay.cpp @@ -0,0 +1,267 @@ +#include + +#include +#include +#include + +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; + extern const int CANNOT_GET_CREATE_TABLE_QUERY; +} + +DatabasesOverlay::DatabasesOverlay(const String & name_, ContextPtr context_) + : IDatabase(name_), WithContext(context_->getGlobalContext()), log(&Poco::Logger::get("DatabaseOverlay(" + name_ + ")")) +{ +} + +DatabasesOverlay & DatabasesOverlay::registerNextDatabase(DatabasePtr database) +{ + databases.push_back(std::move(database)); + return *this; +} + +bool DatabasesOverlay::isTableExist(const String & table_name, ContextPtr context_) const +{ + for (const auto & db : databases) + { + if (db->isTableExist(table_name, context_)) + return true; + } + return false; +} + +StoragePtr DatabasesOverlay::tryGetTable(const String & table_name, ContextPtr context_) const +{ + StoragePtr result = nullptr; + for (const auto & db : databases) + { + result = db->tryGetTable(table_name, context_); + if (result) + break; + } + return result; +} + +void DatabasesOverlay::createTable(ContextPtr context_, const String & table_name, const StoragePtr & table, const ASTPtr & query) +{ + for (auto & db : databases) + { + try + { + db->createTable(context_, table_name, table, query); + return; + } + catch (...) + { + continue; + } + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no databases for CREATE TABLE {} query in Database{}", table_name, getEngineName()); +} + +void DatabasesOverlay::dropTable(ContextPtr context_, const String & table_name, bool sync) +{ + for (auto & db : databases) + { + try + { + db->dropTable(context_, table_name, sync); + return; + } + catch (...) + { + continue; + } + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no databases for DROP TABLE {} query in Database{}", table_name, getEngineName()); +} + +void DatabasesOverlay::attachTable( + ContextPtr context_, const String & table_name, const StoragePtr & table, const String & relative_table_path) +{ + for (auto & db : databases) + { + try + { + db->attachTable(context_, table_name, table, relative_table_path); + return; + } + catch (...) + { + continue; + } + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no databases for ATTACH TABLE query in Database{}", getEngineName()); +} + +StoragePtr DatabasesOverlay::detachTable(ContextPtr context_, const String & table_name) +{ + StoragePtr result = nullptr; + for (auto & db : databases) + { + try + { + result = db->detachTable(context_, table_name); + if (result) + return result; + } + catch (...) + { + continue; + } + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no databases for DETACH TABLE {} query in Database{}", table_name, getEngineName()); +} + +ASTPtr DatabasesOverlay::getCreateTableQueryImpl(const String & name, ContextPtr context_, bool throw_on_error) const +{ + ASTPtr result = nullptr; + for (const auto & db : databases) + { + result = db->tryGetCreateTableQuery(name, context_); + if (result) + break; + } + if (!result && throw_on_error) + throw Exception(ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY, "There is no metadata of table {} in Database{}", name, getEngineName()); + return result; +} + +/* + * DatabaseOverlay cannot be constructed by "CREATE DATABASE" query, as it is not a traditional ClickHouse database + * To use DatabaseOverlay, it must be constructed programmatically in code + */ +ASTPtr DatabasesOverlay::getCreateDatabaseQuery() const +{ + return std::make_shared(); +} + +String DatabasesOverlay::getTableDataPath(const String & table_name) const +{ + String result; + for (const auto & db : databases) + { + result = db->getTableDataPath(table_name); + if (!result.empty()) + break; + } + return result; +} + +String DatabasesOverlay::getTableDataPath(const ASTCreateQuery & query) const +{ + String result; + for (const auto & db : databases) + { + result = db->getTableDataPath(query); + if (!result.empty()) + break; + } + return result; +} + +UUID DatabasesOverlay::tryGetTableUUID(const String & table_name) const +{ + UUID result = UUIDHelpers::Nil; + for (const auto & db : databases) + { + result = db->tryGetTableUUID(table_name); + if (result != UUIDHelpers::Nil) + break; + } + return result; +} + +void DatabasesOverlay::drop(ContextPtr context_) +{ + for (auto & db : databases) + db->drop(context_); +} + +void DatabasesOverlay::alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) +{ + for (auto & db : databases) + { + try + { + db->alterTable(local_context, table_id, metadata); + return; + } + catch (...) + { + continue; + } + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no databases for alterTable in Database{}", getEngineName()); +} + +std::vector> +DatabasesOverlay::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const +{ + std::vector> result; + for (const auto & db : databases) + { + auto dbBackup = db->getTablesForBackup(filter, local_context); + result.insert(result.end(), std::make_move_iterator(dbBackup.begin()), std::make_move_iterator(dbBackup.end())); + } + return result; +} + +void DatabasesOverlay::createTableRestoredFromBackup( + const ASTPtr & create_table_query, + ContextMutablePtr local_context, + std::shared_ptr /*restore_coordination*/, + UInt64 /*timeout_ms*/) +{ + /// Creates a tables by executing a "CREATE TABLE" query. + InterpreterCreateQuery interpreter{create_table_query, local_context}; + interpreter.setInternal(true); + interpreter.execute(); +} + +bool DatabasesOverlay::empty() const +{ + for (const auto & db : databases) + { + if (!db->empty()) + return false; + } + return true; +} + +void DatabasesOverlay::shutdown() +{ + for (auto & db : databases) + db->shutdown(); +} + +DatabaseTablesIteratorPtr DatabasesOverlay::getTablesIterator(ContextPtr context_, const FilterByNameFunction & filter_by_table_name) const +{ + Tables tables; + for (const auto & db : databases) + { + for (auto table_it = db->getTablesIterator(context_, filter_by_table_name); table_it->isValid(); table_it->next()) + tables.insert({table_it->name(), table_it->table()}); + } + return std::make_unique(std::move(tables), getDatabaseName()); +} + +DatabasePtr CreateClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context_) +{ + auto databaseCombiner = std::make_shared(name_, context_); + databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); + databaseCombiner->registerNextDatabase(std::make_shared(name_, context_)); + return databaseCombiner; +} + +} diff --git a/src/Databases/DatabasesOverlay.h b/src/Databases/DatabasesOverlay.h new file mode 100644 index 00000000000..77f0085161b --- /dev/null +++ b/src/Databases/DatabasesOverlay.h @@ -0,0 +1,68 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/** + * Implements the IDatabase interface and combines multiple other databases + * Searches for tables in each database in order until found, and delegates operations to the appropriate database + * Useful for combining databases + * + * Used in clickhouse-local to combine DatabaseFileSystem and DatabaseMemory + */ +class DatabasesOverlay : public IDatabase, protected WithContext +{ +public: + DatabasesOverlay(const String & name_, ContextPtr context_); + + /// Not thread-safe. Use only as factory to initialize database + DatabasesOverlay & registerNextDatabase(DatabasePtr database); + + String getEngineName() const override { return "Overlay"; } + +public: + bool isTableExist(const String & table_name, ContextPtr context) const override; + + StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override; + + void createTable(ContextPtr context, const String & table_name, const StoragePtr & table, const ASTPtr & query) override; + + void dropTable(ContextPtr context, const String & table_name, bool sync) override; + + void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override; + + StoragePtr detachTable(ContextPtr context, const String & table_name) override; + + ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override; + ASTPtr getCreateDatabaseQuery() const override; + + String getTableDataPath(const String & table_name) const override; + String getTableDataPath(const ASTCreateQuery & query) const override; + + UUID tryGetTableUUID(const String & table_name) const override; + + void drop(ContextPtr context) override; + + void alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; + + std::vector> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override; + + void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr restore_coordination, UInt64 timeout_ms) override; + + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; + + bool empty() const override; + + void shutdown() override; + +protected: + std::vector databases; + Poco::Logger * log; +}; + +DatabasePtr CreateClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context_); + +} diff --git a/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.reference b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.reference new file mode 100644 index 00000000000..0fcd843e737 --- /dev/null +++ b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.reference @@ -0,0 +1,9 @@ +Test 1: check explicit and implicit call of the file table function +explicit: +4 +implicit: +4 +Test 2: check FileSystem database +4 +Test 3: check show database with FileSystem +test02707 diff --git a/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh new file mode 100755 index 00000000000..4d8d7b1395a --- /dev/null +++ b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +dir=02707_clickhouse_local_tmp +[[ -d $dir ]] && rm -r $dir +mkdir $dir +# Create temporary csv file for tests +echo '"id","str","int","text"' > $dir/tmp.csv +echo '1,"abc",123,"abacaba"' >> $dir/tmp.csv +echo '2,"def",456,"bacabaa"' >> $dir/tmp.csv +echo '3,"story",78912,"acabaab"' >> $dir/tmp.csv +echo '4,"history",21321321,"cabaaba"' >> $dir/tmp.csv + +################# +echo "Test 1: check explicit and implicit call of the file table function" + +echo "explicit:" +$CLICKHOUSE_LOCAL -q 'SELECT COUNT(*) FROM file("02707_clickhouse_local_tmp/tmp.csv")' +echo "implicit:" +$CLICKHOUSE_LOCAL -q 'SELECT COUNT(*) FROM "02707_clickhouse_local_tmp/tmp.csv"' + +################# +echo "Test 2: check FileSystem database" +$CLICKHOUSE_LOCAL --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test; +CREATE DATABASE test ENGINE = FileSystem('02707_clickhouse_local_tmp'); +SELECT COUNT(*) FROM test.\`tmp.csv\`; +DROP DATABASE test; +""" + +################# +echo "Test 3: check show database with FileSystem" +$CLICKHOUSE_LOCAL --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test02707; +CREATE DATABASE test02707 ENGINE = FileSystem('02707_clickhouse_local_tmp'); +SHOW DATABASES; +DROP DATABASE test02707; +""" | grep "test02707" + +rm -r $dir \ No newline at end of file From 22be85d9764d6ebe3511313c9dadcbdf070c53ad Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 18 Apr 2023 02:42:30 +0200 Subject: [PATCH 0088/2223] renamed setting --- docs/en/operations/settings/settings.md | 8 +++---- .../functions/date-time-functions.md | 2 +- docs/ru/operations/settings/settings.md | 8 +++---- .../functions/date-time-functions.md | 2 +- src/Client/ClientBase.cpp | 5 +--- src/Common/DateLUT.cpp | 2 +- src/Core/Settings.h | 2 +- src/Core/SettingsFields.cpp | 15 ++++++++++++ src/Core/SettingsFields.h | 23 +++++++++++-------- src/Interpreters/Context.cpp | 1 - src/Server/TCPHandler.cpp | 6 +---- .../0_stateless/02681_timezone_setting.sql | 12 +++++----- 12 files changed, 48 insertions(+), 38 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7caf3d4333f..dd81b07b9c0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4008,9 +4008,9 @@ Default value: `0`. Use this setting only for backward compatibility if your use cases depend on old syntax. ::: -## timezone {#timezone} +## session_timezone {#session_timezone} -If specified, sets a implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. +If specified, sets an implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. Examples: ```clickhouse @@ -4020,13 +4020,13 @@ Europe/Berlin Europe/Berlin ``` ```clickhouse -SELECT timeZone(), serverTimezone() SETTINGS timezone = 'Asia/Novosibirsk' FORMAT TSV +SELECT timeZone(), serverTimezone() SETTINGS session_timezone = 'Asia/Novosibirsk' FORMAT TSV Asia/Novosibirsk Europe/Berlin ``` ```clickhouse -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'America/Denver' FORMAT TSV +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver' FORMAT TSV 1999-12-13 07:23:23.123 ``` diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f96041996d4..c1b8d201745 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -26,7 +26,7 @@ SELECT ## timeZone -Returns the default timezone of the server for current session. This can be modified using `SET timezone = 'New/Value'` +Returns the default timezone of the server for current session. This can be modified using `SET session_timezone = 'New/Value'` If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. **Syntax** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 1687e37dba2..fd4d1e11df7 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4075,9 +4075,9 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca Значение по умолчанию: `false` -## timezone {#timezone} +## session_timezone {#session_timezone} -Задаёт значение часового пояса (timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. +Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. Примеры: ```clickhouse @@ -4087,13 +4087,13 @@ Europe/Berlin Europe/Berlin ``` ```clickhouse -SELECT timeZone(), serverTimezone() SETTINGS timezone = 'Asia/Novosibirsk' FORMAT TSV +SELECT timeZone(), serverTimezone() SETTINGS session_timezone = 'Asia/Novosibirsk' FORMAT TSV Asia/Novosibirsk Europe/Berlin ``` ```clickhouse -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'America/Denver' FORMAT TSV +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver' FORMAT TSV 1999-12-13 07:23:23.123 ``` diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 132b54c1040..3e378c08308 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -26,7 +26,7 @@ SELECT ## timeZone {#timezone} -Возвращает часовой пояс сервера, считающийся умолчанием для текущей сессии. +Возвращает часовой пояс сервера, считающийся умолчанием для текущей сессии. Можно изменить значение с помощью `SET session_timezone = 'New/Timezone''` Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. **Синтаксис** diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 2e82144e64d..5126777fa1e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -451,12 +451,9 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) /// Also do not output too much data if we're fuzzing. if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) { -// LogContextes("ClientBase::onData header", global_context); return; } -// LogContextes("ClientBase::onData DATA block", global_context); - /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout)) progress_indication.clearProgressOutput(*tty_buf); @@ -1075,7 +1072,7 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { Settings settings; - settings.timezone = tz; + settings.session_timezone = tz; global_context->applySettingsChanges(settings.changes()); } diff --git a/src/Common/DateLUT.cpp b/src/Common/DateLUT.cpp index 16bd69a20f8..f7e7df016cb 100644 --- a/src/Common/DateLUT.cpp +++ b/src/Common/DateLUT.cpp @@ -167,5 +167,5 @@ DateLUT & DateLUT::getInstance() std::string DateLUT::extractTimezoneFromContext(const DB::ContextPtr query_context) { - return query_context->getSettingsRef().timezone.value; + return query_context->getSettingsRef().session_timezone.value; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b3e9f3fb220..837958aa5b0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -734,7 +734,7 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(Bool, function_json_value_return_type_allow_nullable, false, "Allow function to return nullable type.", 0) \ M(Bool, function_json_value_return_type_allow_complex, false, "Allow function to return complex type, such as: struct, array, map.", 0) \ - M(Timezone, timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ + M(Timezone, session_timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone in current session.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index f4169aa0c64..65720056c8a 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -13,6 +13,7 @@ #include +extern const char * auto_time_zones[]; namespace DB { @@ -26,6 +27,14 @@ namespace ErrorCodes namespace { + bool checkIsExitingTimeZone(const std::string_view timezone) + { + for (auto * it = auto_time_zones; *it; ++it) + if (timezone == *it) + return true; + return false; + } + template T stringToNumber(const String & str) { @@ -463,6 +472,12 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) *this = std::move(str); } +void SettingFieldTimezone::validateTimezone(std::string_view str) +{ + if (str != "" && !checkIsExitingTimeZone(str)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); +} + String SettingFieldCustom::toString() const { return value.dump(); diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 0ee3ddd4862..e3b18a606a1 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -553,13 +552,16 @@ struct SettingFieldTimezone String value; bool changed = false; - explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } + explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(str); value = str; } +// explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; } - explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } +// explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } + explicit SettingFieldTimezone(String && str) { validateTimezone(str); value = std::move(str); } explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet(); validateTimezone(str); value = str; } - SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } +// SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } + SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(str); value = str; changed = true; return *this; } SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; } SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; } SettingFieldTimezone & operator =(const char * str) { *this = std::string_view{str}; return *this; } @@ -575,12 +577,13 @@ struct SettingFieldTimezone void readBinary(ReadBuffer & in); private: - cctz::time_zone validated_tz; - void validateTimezone(const std::string & tz_str) - { - if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); - } + static void validateTimezone(std::string_view str); +// cctz::time_zone validated_tz; +// void validateTimezone(const std::string & tz_str) +// { +// if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) +// throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); +// } }; /// Can keep a value of any type. Used for user-defined settings. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 82ecd87faa0..400eb570131 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1680,7 +1680,6 @@ void Context::applySettingChange(const SettingChange & change) void Context::applySettingsChanges(const SettingsChanges & changes) { auto lock = getLock(); - for (const SettingChange & change : changes) applySettingChange(change); applySettingsQuirks(settings); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a875507d227..e44609529ba 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -183,11 +183,8 @@ void TCPHandler::runImpl() /// User will be authenticated here. It will also set settings from user profile into connection_context. try { - LOG_DEBUG(log, "Before receiveHello"); receiveHello(); - LOG_DEBUG(log, "Before sendHello"); sendHello(); - LOG_DEBUG(log, "Before receiveAddendum"); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); @@ -465,7 +462,6 @@ void TCPHandler::runImpl() sendSelectProfileEvents(); sendLogs(); - return false; }; @@ -1072,7 +1068,7 @@ void TCPHandler::sendTimezone() if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES) return; - const String & tz = query_context->getSettingsRef().timezone.toString(); + const String & tz = query_context->getSettingsRef().session_timezone.toString(); LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz); writeVarUInt(Protocol::Server::TimezoneUpdate, *out); diff --git a/tests/queries/0_stateless/02681_timezone_setting.sql b/tests/queries/0_stateless/02681_timezone_setting.sql index 73afb4c029b..f66e8d2b646 100644 --- a/tests/queries/0_stateless/02681_timezone_setting.sql +++ b/tests/queries/0_stateless/02681_timezone_setting.sql @@ -1,11 +1,11 @@ -SET timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} +SET session_timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} -SET timezone = 'Asia/Novosibirsk'; +SET session_timezone = 'Asia/Novosibirsk'; SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); -SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; +SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'Europe/Zurich'; -SET timezone = 'Asia/Manila'; +SET session_timezone = 'Asia/Manila'; SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); -SELECT timezone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich' FORMAT TSV; -SELECT timezone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn' FORMAT TSV; +SELECT timezone(), timezoneOf(now()) SETTINGS session_timezone = 'Europe/Zurich' FORMAT TSV; +SELECT timezone(), timezoneOf(now()) SETTINGS session_timezone = 'Pacific/Pitcairn' FORMAT TSV; From 0550b0640ce0020d4e4f0015447631c4b742ab13 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 18 Apr 2023 03:35:29 +0200 Subject: [PATCH 0089/2223] fix linking issue --- src/Core/SettingsFields.cpp | 15 --------------- src/Core/SettingsFields.h | 23 ++++++++++------------- 2 files changed, 10 insertions(+), 28 deletions(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 65720056c8a..f4169aa0c64 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -13,7 +13,6 @@ #include -extern const char * auto_time_zones[]; namespace DB { @@ -27,14 +26,6 @@ namespace ErrorCodes namespace { - bool checkIsExitingTimeZone(const std::string_view timezone) - { - for (auto * it = auto_time_zones; *it; ++it) - if (timezone == *it) - return true; - return false; - } - template T stringToNumber(const String & str) { @@ -472,12 +463,6 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) *this = std::move(str); } -void SettingFieldTimezone::validateTimezone(std::string_view str) -{ - if (str != "" && !checkIsExitingTimeZone(str)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); -} - String SettingFieldCustom::toString() const { return value.dump(); diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index e3b18a606a1..0ee3ddd4862 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -552,16 +553,13 @@ struct SettingFieldTimezone String value; bool changed = false; - explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(str); value = str; } -// explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } + explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; } -// explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } - explicit SettingFieldTimezone(String && str) { validateTimezone(str); value = std::move(str); } + explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet(); validateTimezone(str); value = str; } -// SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } - SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(str); value = str; changed = true; return *this; } + SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; } SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; } SettingFieldTimezone & operator =(const char * str) { *this = std::string_view{str}; return *this; } @@ -577,13 +575,12 @@ struct SettingFieldTimezone void readBinary(ReadBuffer & in); private: - static void validateTimezone(std::string_view str); -// cctz::time_zone validated_tz; -// void validateTimezone(const std::string & tz_str) -// { -// if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) -// throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); -// } + cctz::time_zone validated_tz; + void validateTimezone(const std::string & tz_str) + { + if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); + } }; /// Can keep a value of any type. Used for user-defined settings. From 96553bc3d8e70d06e03191f4b848ed07c91e5c6a Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Sun, 16 Apr 2023 23:25:57 +0300 Subject: [PATCH 0090/2223] Fix style and tests --- src/Databases/DatabaseFactory.cpp | 6 ++- src/Databases/DatabaseFileSystem.cpp | 45 +++++++++++-------- src/Databases/DatabasesOverlay.cpp | 1 - ...ouse_local_implicit_file_table_function.sh | 14 +++--- 4 files changed, 38 insertions(+), 28 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index b023bb06ad1..9c13881fc7b 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -433,13 +433,15 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared(context, engine_define, create.attach, database_path); } #endif - else if (engine_name == "FileSystem") { + else if (engine_name == "FileSystem") + { const ASTFunction * engine = engine_define->engine; // If init_path is empty, then the current path from Poco will be used std::string init_path; - if (engine->arguments && engine->arguments->children.size() > 0) { + if (engine->arguments && !engine->arguments->children.empty()) + { if (engine->arguments->children.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "FileSystem database requires at most 1 argument: file_system_path"); diff --git a/src/Databases/DatabaseFileSystem.cpp b/src/Databases/DatabaseFileSystem.cpp index 9e2273970c3..8b92ad8080a 100644 --- a/src/Databases/DatabaseFileSystem.cpp +++ b/src/Databases/DatabaseFileSystem.cpp @@ -59,23 +59,30 @@ StoragePtr DatabaseFileSystem::tryGetTable(const String & name, ContextPtr conte auto table_path = getTablePath(name); - // If the table doesn't exist in the tables map, check if the corresponding file exists - Poco::File table_file(table_path); - if (!table_file.exists()) + try + { + // If the table doesn't exist in the tables map, check if the corresponding file exists + Poco::File table_file(table_path); + if (!table_file.exists()) + return nullptr; + + // If the file exists, create a new table using TableFunctionFile and return it. + auto args = makeASTFunction("file", std::make_shared(table_path)); + + auto table_function = TableFunctionFactory::instance().get(args, context_); + if (!table_function) + return nullptr; + + auto table_storage = table_function->execute(args, context_, name); + if (table_storage) + addTable(name, table_storage); + + return table_storage; + } + catch (...) + { return nullptr; - - // If the file exists, create a new table using TableFunctionFile and return it. - auto args = makeASTFunction("file", std::make_shared(table_path)); - - auto table_function = TableFunctionFactory::instance().get(args, context_); - if (!table_function) - return nullptr; - - auto table_storage = table_function->execute(args, context_, name); - if (table_storage) - addTable(name, table_storage); - - return table_storage; + } } ASTPtr DatabaseFileSystem::getCreateDatabaseQuery() const @@ -116,7 +123,8 @@ void DatabaseFileSystem::shutdown() /** * Returns an empty vector because the database is read-only and no tables can be backed up. */ -std::vector> DatabaseFileSystem::getTablesForBackup(const FilterByNameFunction&, const ContextPtr&) const { +std::vector> DatabaseFileSystem::getTablesForBackup(const FilterByNameFunction&, const ContextPtr&) const +{ return {}; } @@ -125,7 +133,8 @@ std::vector> DatabaseFileSystem::getTablesForBacku * Returns an empty iterator because the database does not have its own tables * But only caches them for quick access. */ -DatabaseTablesIteratorPtr DatabaseFileSystem::getTablesIterator(ContextPtr, const FilterByNameFunction&) const { +DatabaseTablesIteratorPtr DatabaseFileSystem::getTablesIterator(ContextPtr, const FilterByNameFunction&) const +{ return std::make_unique(Tables{}, getDatabaseName()); } diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 9c3d802df73..da26f9282a0 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; extern const int CANNOT_GET_CREATE_TABLE_QUERY; } diff --git a/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh index 4d8d7b1395a..eea1e47ba7f 100755 --- a/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh +++ b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh @@ -4,8 +4,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -dir=02707_clickhouse_local_tmp -[[ -d $dir ]] && rm -r $dir +dir=${CLICKHOUSE_TEST_UNIQUE_NAME} +[[ -d $dir ]] && rm -rd $dir mkdir $dir # Create temporary csv file for tests echo '"id","str","int","text"' > $dir/tmp.csv @@ -18,15 +18,15 @@ echo '4,"history",21321321,"cabaaba"' >> $dir/tmp.csv echo "Test 1: check explicit and implicit call of the file table function" echo "explicit:" -$CLICKHOUSE_LOCAL -q 'SELECT COUNT(*) FROM file("02707_clickhouse_local_tmp/tmp.csv")' +$CLICKHOUSE_LOCAL -q "SELECT COUNT(*) FROM file('${dir}/tmp.csv')" echo "implicit:" -$CLICKHOUSE_LOCAL -q 'SELECT COUNT(*) FROM "02707_clickhouse_local_tmp/tmp.csv"' +$CLICKHOUSE_LOCAL -q "SELECT COUNT(*) FROM \"${dir}/tmp.csv\"" ################# echo "Test 2: check FileSystem database" $CLICKHOUSE_LOCAL --multiline --multiquery -q """ DROP DATABASE IF EXISTS test; -CREATE DATABASE test ENGINE = FileSystem('02707_clickhouse_local_tmp'); +CREATE DATABASE test ENGINE = FileSystem('${dir}'); SELECT COUNT(*) FROM test.\`tmp.csv\`; DROP DATABASE test; """ @@ -35,9 +35,9 @@ DROP DATABASE test; echo "Test 3: check show database with FileSystem" $CLICKHOUSE_LOCAL --multiline --multiquery -q """ DROP DATABASE IF EXISTS test02707; -CREATE DATABASE test02707 ENGINE = FileSystem('02707_clickhouse_local_tmp'); +CREATE DATABASE test02707 ENGINE = FileSystem('${dir}'); SHOW DATABASES; DROP DATABASE test02707; """ | grep "test02707" -rm -r $dir \ No newline at end of file +rm -rd $dir From 1e8c0a2db9671f0862975499f16b923a49c3a2ec Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 00:06:15 +0200 Subject: [PATCH 0091/2223] Lighter timezone validation Reused external variable from src/Storages/System/StorageSystemTimeZones.generated.cpp Required changes to CMakeLists of some standalone modules to link properly --- programs/library-bridge/CMakeLists.txt | 8 +++++++- programs/odbc-bridge/CMakeLists.txt | 6 +++++- src/Core/SettingsFields.cpp | 15 +++++++++++++++ src/Core/SettingsFields.h | 24 ++++++++++++++---------- utils/check-marks/CMakeLists.txt | 6 +++++- utils/keeper-data-dumper/CMakeLists.txt | 8 +++++++- utils/wal-dump/CMakeLists.txt | 6 +++++- 7 files changed, 58 insertions(+), 15 deletions(-) diff --git a/programs/library-bridge/CMakeLists.txt b/programs/library-bridge/CMakeLists.txt index 1cacc391ca5..97af7c3b22e 100644 --- a/programs/library-bridge/CMakeLists.txt +++ b/programs/library-bridge/CMakeLists.txt @@ -13,11 +13,17 @@ set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES library-bridge.cpp ) +set(CLICKHOUSE_LIBRARY_BRIDGE_EXTERNAL_SOURCES + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp +) + if (OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () -clickhouse_add_executable(clickhouse-library-bridge ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) +clickhouse_add_executable(clickhouse-library-bridge + ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES} + ${CLICKHOUSE_LIBRARY_BRIDGE_EXTERNAL_SOURCES}) target_link_libraries(clickhouse-library-bridge PRIVATE daemon diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index 118610e4dcd..bf1b42df026 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -15,13 +15,17 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES validateODBCConnectionString.cpp ) +set(ODBC_BRIDGE_EXTERNAL_SOURCES + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp +) + if (OS_LINUX) # clickhouse-odbc-bridge is always a separate binary. # Reason: it must not export symbols from SSL, mariadb-client, etc. to not break ABI compatibility with ODBC drivers. set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () -clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) +clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES} ${ODBC_BRIDGE_EXTERNAL_SOURCES}) target_link_libraries(clickhouse-odbc-bridge PRIVATE daemon diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index f4169aa0c64..65720056c8a 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -13,6 +13,7 @@ #include +extern const char * auto_time_zones[]; namespace DB { @@ -26,6 +27,14 @@ namespace ErrorCodes namespace { + bool checkIsExitingTimeZone(const std::string_view timezone) + { + for (auto * it = auto_time_zones; *it; ++it) + if (timezone == *it) + return true; + return false; + } + template T stringToNumber(const String & str) { @@ -463,6 +472,12 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) *this = std::move(str); } +void SettingFieldTimezone::validateTimezone(std::string_view str) +{ + if (str != "" && !checkIsExitingTimeZone(str)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); +} + String SettingFieldCustom::toString() const { return value.dump(); diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 0ee3ddd4862..e78fef9f455 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -2,7 +2,7 @@ #include #include -#include +//#include #include #include #include @@ -553,13 +553,16 @@ struct SettingFieldTimezone String value; bool changed = false; - explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } +// explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } + explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; } - explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } +// explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } + explicit SettingFieldTimezone(String && str) { validateTimezone(str); value = std::move(str); } explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet(); validateTimezone(str); value = str; } - SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } +// SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } + SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(str); value = str; changed = true; return *this; } SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; } SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; } SettingFieldTimezone & operator =(const char * str) { *this = std::string_view{str}; return *this; } @@ -575,12 +578,13 @@ struct SettingFieldTimezone void readBinary(ReadBuffer & in); private: - cctz::time_zone validated_tz; - void validateTimezone(const std::string & tz_str) - { - if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); - } +// cctz::time_zone validated_tz; +// void validateTimezone(const std::string & str) +// { +// if (!str.empty() && !cctz::load_time_zone(str, &validated_tz)) +// throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); +// } + static void validateTimezone(std::string_view str); }; /// Can keep a value of any type. Used for user-defined settings. diff --git a/utils/check-marks/CMakeLists.txt b/utils/check-marks/CMakeLists.txt index 05546a2989b..456fb3d7112 100644 --- a/utils/check-marks/CMakeLists.txt +++ b/utils/check-marks/CMakeLists.txt @@ -1,2 +1,6 @@ -clickhouse_add_executable (check-marks main.cpp) +set(CHECK_MARKS_EXTERNAL_SOURCES + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp +) + +clickhouse_add_executable (check-marks ${CHECK_MARKS_EXTERNAL_SOURCES} main.cpp) target_link_libraries(check-marks PRIVATE dbms boost::program_options) diff --git a/utils/keeper-data-dumper/CMakeLists.txt b/utils/keeper-data-dumper/CMakeLists.txt index 1f55e50e68e..a6858a29e8b 100644 --- a/utils/keeper-data-dumper/CMakeLists.txt +++ b/utils/keeper-data-dumper/CMakeLists.txt @@ -1,2 +1,8 @@ -clickhouse_add_executable(keeper-data-dumper main.cpp) +set(KEEPER_DATA_DUMPER_EXTERNAL_SOURCES + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp +) + +clickhouse_add_executable(keeper-data-dumper + ${KEEPER_DATA_DUMPER_EXTERNAL_SOURCES} + main.cpp) target_link_libraries(keeper-data-dumper PRIVATE dbms) diff --git a/utils/wal-dump/CMakeLists.txt b/utils/wal-dump/CMakeLists.txt index 3d59e95b4ca..754799a6faf 100644 --- a/utils/wal-dump/CMakeLists.txt +++ b/utils/wal-dump/CMakeLists.txt @@ -1,2 +1,6 @@ -clickhouse_add_executable (wal-dump main.cpp) +set(WAL_DUMP_EXTERNAL_SOURCES + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp +) + +clickhouse_add_executable (wal-dump ${WAL_DUMP_EXTERNAL_SOURCES} main.cpp) target_link_libraries(wal-dump PRIVATE dbms boost::program_options) From 24be7203d931b57a35241fec1abe31a9099ba096 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 00:39:08 +0200 Subject: [PATCH 0092/2223] add errorcode reference --- src/Core/SettingsFields.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 65720056c8a..e952688a968 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; extern const int CANNOT_PARSE_BOOL; extern const int CANNOT_PARSE_NUMBER; + extern const int BAD_ARGUMENTS; } From 542c09cb518988cf54261edbab691c459efa9a88 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 01:35:49 +0200 Subject: [PATCH 0093/2223] fix keeper standalone linking --- programs/keeper/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 761335fb707..ff2de3f581c 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -107,6 +107,8 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/GraphiteWriter.cpp ${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp + Keeper.cpp clickhouse-keeper.cpp ) From ce7dc8b123502aee8af1578d87133f8283c66a5b Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 13:42:07 +0200 Subject: [PATCH 0094/2223] tidy --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index e952688a968..6af38586ed8 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -475,7 +475,7 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) void SettingFieldTimezone::validateTimezone(std::string_view str) { - if (str != "" && !checkIsExitingTimeZone(str)) + if (!str.empty() && !checkIsExitingTimeZone(str)) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); } From daae5025e8586156a016687672100a8ec0db6016 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 14:45:51 +0200 Subject: [PATCH 0095/2223] small updates due to review --- docs/en/operations/settings/settings.md | 1 + docs/en/sql-reference/functions/date-time-functions.md | 2 +- docs/ru/operations/settings/settings.md | 1 + programs/client/Client.cpp | 2 +- src/Client/ClientBase.cpp | 5 ----- 5 files changed, 4 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index dd81b07b9c0..2010f763c84 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4011,6 +4011,7 @@ Use this setting only for backward compatibility if your use cases depend on old ## session_timezone {#session_timezone} If specified, sets an implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. +Setting this to `''` (empty string) effectively resets implicit timezone to server timezone. Examples: ```clickhouse diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index c1b8d201745..265ce676ef7 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -51,7 +51,7 @@ If it is executed in the context of a distributed table, then it generates a nor **Syntax** ``` sql -timeZone() +serverTimeZone() ``` Alias: `ServerTimezone`, `servertimezone`. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index fd4d1e11df7..8180f5435b8 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4078,6 +4078,7 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca ## session_timezone {#session_timezone} Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. +При значении настройки `''` (пустая строка), будет совпадать с часовым поясом сервера. Примеры: ```clickhouse diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 528c504e555..b760efc21d1 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -324,7 +324,7 @@ try { // All that just to set DB::CurrentThread::get().getGlobalContext() - // which is required for client timezone (pushed as from server) to work. + // which is required for client timezone (pushed from server) to work. auto thread_group = std::make_shared(); thread_status.attachToGroup(thread_group, false); } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5126777fa1e..6df86db886b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -78,11 +78,6 @@ namespace fs = std::filesystem; using namespace std::literals; -namespace -{ -using namespace DB; -} - namespace CurrentMetrics { extern const Metric MemoryTracking; From 61f55930ceee99ce23cdab794ce77945f9a6ee1c Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Wed, 19 Apr 2023 15:46:17 +0300 Subject: [PATCH 0096/2223] Update docs/en/operations/settings/settings.md Co-authored-by: Vasily Nemkov --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2010f763c84..366e7de8d28 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4034,7 +4034,7 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric Possible values: -- Any valid timezone in `Region/Place` notation, e.g. `Europe/Berlin` +- Any timezone name from `system.time_zones`, e.g. `Europe/Berlin`, `UTC` or `Zulu` Default value: `''`. From b281ceacbb4cc8f66a56b9bc12d5ab521098ce08 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 14:47:57 +0200 Subject: [PATCH 0097/2223] Update docs/ru/operations/settings/settings.md --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 8180f5435b8..f0f497b6254 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4101,6 +4101,6 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric Возможные значения: -- Строка вида `Регион/Город`, например `Europe/Zurich` +- Любая зона из `system.time_zones`, например `Europe/Berlin`, `UTC` или `Zulu` Значение по умолчанию: `''`. From b06d7355d597abdf6692b5c232fb12449d57aa5b Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Wed, 19 Apr 2023 17:39:40 +0300 Subject: [PATCH 0098/2223] Update src/Core/SettingsFields.h Co-authored-by: Vasily Nemkov --- src/Core/SettingsFields.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index e78fef9f455..8bd7370c980 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -2,7 +2,6 @@ #include #include -//#include #include #include #include From b81ce64fa23cf2d05edd488eeb1adbf981784a54 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Wed, 19 Apr 2023 17:39:52 +0300 Subject: [PATCH 0099/2223] Update src/Client/ClientBase.cpp Co-authored-by: Vasily Nemkov --- src/Client/ClientBase.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6df86db886b..bd83246871b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1531,9 +1531,9 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || - *packet_type == Protocol::Server::ProfileEvents || - *packet_type == Protocol::Server::TimezoneUpdate)) + while (packet_type && (*packet_type == Protocol::Server::Log + || *packet_type == Protocol::Server::ProfileEvents + || *packet_type == Protocol::Server::TimezoneUpdate)) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); From 21d5846cabd0717184f44d98b8480fefc683e807 Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Tue, 18 Apr 2023 18:12:11 +0300 Subject: [PATCH 0100/2223] Fix test --- .../02707_clickhouse_local_implicit_file_table_function.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh index eea1e47ba7f..24de0ad579c 100755 --- a/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh +++ b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh @@ -7,6 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) dir=${CLICKHOUSE_TEST_UNIQUE_NAME} [[ -d $dir ]] && rm -rd $dir mkdir $dir + # Create temporary csv file for tests echo '"id","str","int","text"' > $dir/tmp.csv echo '1,"abc",123,"abacaba"' >> $dir/tmp.csv @@ -40,4 +41,5 @@ SHOW DATABASES; DROP DATABASE test02707; """ | grep "test02707" +# Remove temporary dir with files rm -rd $dir From bf55f43e1933fdcbbc2ec85e5b0823c6a7e3eb5e Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 20 Apr 2023 02:53:42 +0200 Subject: [PATCH 0101/2223] update cmakelists --- src/CMakeLists.txt | 5 +++++ src/Core/SettingsFields.cpp | 4 ++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 76e5ef83e41..5ac3f6e1654 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -191,6 +191,11 @@ list (APPEND dbms_headers Dictionaries/DictionaryStructure.h Dictionaries/getDictionaryConfigurationFromAST.h) +# Required for validation of Timezone in session_timezone setting. +# This way we don't need to create timezone via cctz each time, but check against pregenerated char** +list (APPEND dbms_sources + Storages/System/StorageSystemTimeZones.generated.cpp) + if (NOT ENABLE_SSL) list (REMOVE_ITEM clickhouse_common_io_sources Common/OpenSSLHelpers.cpp) list (REMOVE_ITEM clickhouse_common_io_headers Common/OpenSSLHelpers.h) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 6af38586ed8..c0556519563 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -28,7 +28,7 @@ namespace ErrorCodes namespace { - bool checkIsExitingTimeZone(const std::string_view timezone) + bool checkIsExistingTimeZone(const std::string_view timezone) { for (auto * it = auto_time_zones; *it; ++it) if (timezone == *it) @@ -475,7 +475,7 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) void SettingFieldTimezone::validateTimezone(std::string_view str) { - if (!str.empty() && !checkIsExitingTimeZone(str)) + if (!str.empty() && !checkIsExistingTimeZone(str)) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); } From f4af76ab8baee97c06cf1e53346da6107c7ccbbd Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 20 Apr 2023 11:58:51 +0200 Subject: [PATCH 0102/2223] cleanup cmakelists --- programs/library-bridge/CMakeLists.txt | 7 +------ programs/odbc-bridge/CMakeLists.txt | 6 +----- utils/check-marks/CMakeLists.txt | 6 +----- utils/keeper-data-dumper/CMakeLists.txt | 8 +------- utils/wal-dump/CMakeLists.txt | 6 +----- 5 files changed, 5 insertions(+), 28 deletions(-) diff --git a/programs/library-bridge/CMakeLists.txt b/programs/library-bridge/CMakeLists.txt index 97af7c3b22e..79497d5fb2e 100644 --- a/programs/library-bridge/CMakeLists.txt +++ b/programs/library-bridge/CMakeLists.txt @@ -13,17 +13,12 @@ set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES library-bridge.cpp ) -set(CLICKHOUSE_LIBRARY_BRIDGE_EXTERNAL_SOURCES - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp -) - if (OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () clickhouse_add_executable(clickhouse-library-bridge - ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES} - ${CLICKHOUSE_LIBRARY_BRIDGE_EXTERNAL_SOURCES}) + ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) target_link_libraries(clickhouse-library-bridge PRIVATE daemon diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index bf1b42df026..118610e4dcd 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -15,17 +15,13 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES validateODBCConnectionString.cpp ) -set(ODBC_BRIDGE_EXTERNAL_SOURCES - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp -) - if (OS_LINUX) # clickhouse-odbc-bridge is always a separate binary. # Reason: it must not export symbols from SSL, mariadb-client, etc. to not break ABI compatibility with ODBC drivers. set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () -clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES} ${ODBC_BRIDGE_EXTERNAL_SOURCES}) +clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) target_link_libraries(clickhouse-odbc-bridge PRIVATE daemon diff --git a/utils/check-marks/CMakeLists.txt b/utils/check-marks/CMakeLists.txt index 456fb3d7112..05546a2989b 100644 --- a/utils/check-marks/CMakeLists.txt +++ b/utils/check-marks/CMakeLists.txt @@ -1,6 +1,2 @@ -set(CHECK_MARKS_EXTERNAL_SOURCES - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp -) - -clickhouse_add_executable (check-marks ${CHECK_MARKS_EXTERNAL_SOURCES} main.cpp) +clickhouse_add_executable (check-marks main.cpp) target_link_libraries(check-marks PRIVATE dbms boost::program_options) diff --git a/utils/keeper-data-dumper/CMakeLists.txt b/utils/keeper-data-dumper/CMakeLists.txt index a6858a29e8b..1f55e50e68e 100644 --- a/utils/keeper-data-dumper/CMakeLists.txt +++ b/utils/keeper-data-dumper/CMakeLists.txt @@ -1,8 +1,2 @@ -set(KEEPER_DATA_DUMPER_EXTERNAL_SOURCES - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp -) - -clickhouse_add_executable(keeper-data-dumper - ${KEEPER_DATA_DUMPER_EXTERNAL_SOURCES} - main.cpp) +clickhouse_add_executable(keeper-data-dumper main.cpp) target_link_libraries(keeper-data-dumper PRIVATE dbms) diff --git a/utils/wal-dump/CMakeLists.txt b/utils/wal-dump/CMakeLists.txt index 754799a6faf..3d59e95b4ca 100644 --- a/utils/wal-dump/CMakeLists.txt +++ b/utils/wal-dump/CMakeLists.txt @@ -1,6 +1,2 @@ -set(WAL_DUMP_EXTERNAL_SOURCES - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp -) - -clickhouse_add_executable (wal-dump ${WAL_DUMP_EXTERNAL_SOURCES} main.cpp) +clickhouse_add_executable (wal-dump main.cpp) target_link_libraries(wal-dump PRIVATE dbms boost::program_options) From e37745811cd6000348655c7c42cdc25436a3090e Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 20 Apr 2023 12:04:12 +0200 Subject: [PATCH 0103/2223] style --- programs/library-bridge/CMakeLists.txt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/programs/library-bridge/CMakeLists.txt b/programs/library-bridge/CMakeLists.txt index 79497d5fb2e..1cacc391ca5 100644 --- a/programs/library-bridge/CMakeLists.txt +++ b/programs/library-bridge/CMakeLists.txt @@ -17,8 +17,7 @@ if (OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () -clickhouse_add_executable(clickhouse-library-bridge - ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) +clickhouse_add_executable(clickhouse-library-bridge ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) target_link_libraries(clickhouse-library-bridge PRIVATE daemon From b52646c5559fc6eaf820701788c95a8cd89b3285 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 21 Apr 2023 08:20:35 +0000 Subject: [PATCH 0104/2223] Eliminate double log serializing --- src/Coordination/KeeperServer.cpp | 36 ++++++++----------- src/Coordination/KeeperStateMachine.cpp | 2 +- .../WriteBufferFromNuraftBuffer.cpp | 1 - 3 files changed, 16 insertions(+), 23 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 56ed8f4eafe..ea97a2bc01d 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -471,17 +471,6 @@ void KeeperServer::shutdown() namespace { -// Serialize the request with all the necessary information for the leader -// we don't know ZXID and digest yet so we don't serialize it -nuraft::ptr getZooKeeperRequestMessage(const KeeperStorage::RequestForSession & request_for_session) -{ - DB::WriteBufferFromNuraftBuffer write_buf; - DB::writeIntBinary(request_for_session.session_id, write_buf); - request_for_session.request->write(write_buf); - DB::writeIntBinary(request_for_session.time, write_buf); - return write_buf.getBuffer(); -} - // Serialize the request for the log entry nuraft::ptr getZooKeeperLogEntry(const KeeperStorage::RequestForSession & request_for_session) { @@ -489,12 +478,9 @@ nuraft::ptr getZooKeeperLogEntry(const KeeperStorage::RequestFor DB::writeIntBinary(request_for_session.session_id, write_buf); request_for_session.request->write(write_buf); DB::writeIntBinary(request_for_session.time, write_buf); - DB::writeIntBinary(request_for_session.zxid, write_buf); - assert(request_for_session.digest); - DB::writeIntBinary(request_for_session.digest->version, write_buf); - if (request_for_session.digest->version != KeeperStorage::DigestVersion::NO_DIGEST) - DB::writeIntBinary(request_for_session.digest->value, write_buf); - + DB::writeIntBinary(static_cast(0), write_buf); + DB::writeIntBinary(KeeperStorage::DigestVersion::NO_DIGEST, write_buf); + DB::writeIntBinary(static_cast(0), write_buf); return write_buf.getBuffer(); } @@ -512,9 +498,7 @@ RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForS { std::vector> entries; for (const auto & request_for_session : requests_for_sessions) - { - entries.push_back(getZooKeeperRequestMessage(request_for_session)); - } + entries.push_back(getZooKeeperLogEntry(request_for_session)); std::lock_guard lock{server_write_mutex}; if (is_recovering) @@ -642,7 +626,17 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ return nuraft::cb_func::ReturnCode::ReturnNull; request_for_session.digest = state_machine->getNodesDigest(); - entry = nuraft::cs_new(entry->get_term(), getZooKeeperLogEntry(request_for_session), entry->get_val_type()); + + static constexpr size_t write_buffer_size + = sizeof(request_for_session.zxid) + sizeof(request_for_session.digest->version) + sizeof(request_for_session.digest->value); + auto * buffer_start = reinterpret_cast(entry_buf.data_begin() + entry_buf.size() - write_buffer_size); + + WriteBuffer write_buf(buffer_start, write_buffer_size); + writeIntBinary(request_for_session.zxid, write_buf); + writeIntBinary(request_for_session.digest->version, write_buf); + if (request_for_session.digest->version != KeeperStorage::NO_DIGEST) + writeIntBinary(request_for_session.digest->value, write_buf); + break; } case nuraft::cb_func::AppendLogFailed: diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 7c55739a96f..5a077ff85a3 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -190,7 +190,7 @@ KeeperStorage::RequestForSession KeeperStateMachine::parseRequest(nuraft::buffer { request_for_session.digest.emplace(); readIntBinary(request_for_session.digest->version, buffer); - if (request_for_session.digest->version != KeeperStorage::DigestVersion::NO_DIGEST) + if (request_for_session.digest->version != KeeperStorage::DigestVersion::NO_DIGEST || !buffer.eof()) readIntBinary(request_for_session.digest->value, buffer); } diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp index 77ad52c867d..c955d3fdbbe 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.cpp +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -1,5 +1,4 @@ #include -#include namespace DB { From d500a5637db05105d2db64308c1c1ce13e1726c8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 21 Apr 2023 09:41:10 +0000 Subject: [PATCH 0105/2223] Cache large requests --- src/Coordination/CoordinationSettings.h | 3 +- src/Coordination/KeeperServer.cpp | 20 ++-- src/Coordination/KeeperStateMachine.cpp | 124 ++++++++++++++++-------- src/Coordination/KeeperStateMachine.h | 9 +- 4 files changed, 105 insertions(+), 51 deletions(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 90f66ccc09b..24567177ff1 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -47,7 +47,8 @@ struct Settings; M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) \ M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0) \ M(UInt64, max_log_file_size, 50 * 1024 * 1024, "Max size of the Raft log file. If possible, each created log file will preallocate this amount of bytes on disk. Set to 0 to disable the limit", 0) \ - M(UInt64, log_file_overallocate_size, 50 * 1024 * 1024, "If max_log_file_size is not set to 0, this value will be added to it for preallocating bytes on disk. If a log record is larger than this value, it could lead to uncaught out-of-space issues so a larger value is preferred", 0) + M(UInt64, log_file_overallocate_size, 50 * 1024 * 1024, "If max_log_file_size is not set to 0, this value will be added to it for preallocating bytes on disk. If a log record is larger than this value, it could lead to uncaught out-of-space issues so a larger value is preferred", 0) \ + M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index ea97a2bc01d..6af8f967c0c 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -621,21 +621,21 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ auto & entry_buf = entry->get_buf(); auto request_for_session = state_machine->parseRequest(entry_buf); - request_for_session.zxid = next_zxid; - if (!state_machine->preprocess(request_for_session)) + request_for_session->zxid = next_zxid; + if (!state_machine->preprocess(*request_for_session)) return nuraft::cb_func::ReturnCode::ReturnNull; - request_for_session.digest = state_machine->getNodesDigest(); + request_for_session->digest = state_machine->getNodesDigest(); static constexpr size_t write_buffer_size - = sizeof(request_for_session.zxid) + sizeof(request_for_session.digest->version) + sizeof(request_for_session.digest->value); + = sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value); auto * buffer_start = reinterpret_cast(entry_buf.data_begin() + entry_buf.size() - write_buffer_size); WriteBuffer write_buf(buffer_start, write_buffer_size); - writeIntBinary(request_for_session.zxid, write_buf); - writeIntBinary(request_for_session.digest->version, write_buf); - if (request_for_session.digest->version != KeeperStorage::NO_DIGEST) - writeIntBinary(request_for_session.digest->value, write_buf); + writeIntBinary(request_for_session->zxid, write_buf); + writeIntBinary(request_for_session->digest->version, write_buf); + if (request_for_session->digest->version != KeeperStorage::NO_DIGEST) + writeIntBinary(request_for_session->digest->value, write_buf); break; } @@ -648,8 +648,8 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ assert(entry->get_val_type() == nuraft::app_log); auto & entry_buf = entry->get_buf(); - auto request_for_session = state_machine->parseRequest(entry_buf); - state_machine->rollbackRequest(request_for_session, true); + auto request_for_session = state_machine->parseRequest(entry_buf, true); + state_machine->rollbackRequest(*request_for_session, true); break; } default: diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5a077ff85a3..c5db8107652 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -1,16 +1,16 @@ #include -#include -#include #include #include #include #include #include #include +#include +#include #include +#include #include #include -#include #include #include "Coordination/KeeperStorage.h" @@ -60,6 +60,7 @@ KeeperStateMachine::KeeperStateMachine( coordination_settings->dead_session_check_period_ms.totalMilliseconds()) , responses_queue(responses_queue_) , snapshots_queue(snapshots_queue_) + , min_request_size_to_cache(coordination_settings_->min_request_size_for_cache) , last_committed_idx(0) , log(&Poco::Logger::get("KeeperStateMachine")) , superdigest(superdigest_) @@ -150,18 +151,18 @@ void assertDigest( nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data) { auto request_for_session = parseRequest(data); - if (!request_for_session.zxid) - request_for_session.zxid = log_idx; + if (!request_for_session->zxid) + request_for_session->zxid = log_idx; - preprocess(request_for_session); + preprocess(*request_for_session); return nullptr; } -KeeperStorage::RequestForSession KeeperStateMachine::parseRequest(nuraft::buffer & data) +std::shared_ptr KeeperStateMachine::parseRequest(nuraft::buffer & data, bool final) { ReadBufferFromNuraftBuffer buffer(data); - KeeperStorage::RequestForSession request_for_session; - readIntBinary(request_for_session.session_id, buffer); + auto request_for_session = std::make_shared(); + readIntBinary(request_for_session->session_id, buffer); int32_t length; Coordination::read(length, buffer); @@ -169,29 +170,69 @@ KeeperStorage::RequestForSession KeeperStateMachine::parseRequest(nuraft::buffer int32_t xid; Coordination::read(xid, buffer); + static constexpr std::array non_cacheable_xids{ + Coordination::WATCH_XID, + Coordination::PING_XID, + Coordination::AUTH_XID, + Coordination::CLOSE_XID, + }; + + const bool should_cache = request_for_session->session_id != -1 && data.size() > min_request_size_to_cache + && std::all_of(non_cacheable_xids.begin(), + non_cacheable_xids.end(), + [&](const auto non_cacheable_xid) { return xid != non_cacheable_xid; }); + + if (should_cache) + { + std::lock_guard lock(request_cache_mutex); + if (auto xid_to_request_it = parsed_request_cache.find(request_for_session->session_id); + xid_to_request_it != parsed_request_cache.end()) + { + auto & xid_to_request = xid_to_request_it->second; + if (auto request_it = xid_to_request.find(xid); request_it != xid_to_request.end()) + { + if (final) + { + auto request = std::move(request_it->second); + xid_to_request.erase(request_it); + return request; + } + else + return request_it->second; + } + } + } + + Coordination::OpNum opnum; Coordination::read(opnum, buffer); - request_for_session.request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); - request_for_session.request->xid = xid; - request_for_session.request->readImpl(buffer); + request_for_session->request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); + request_for_session->request->xid = xid; + request_for_session->request->readImpl(buffer); if (!buffer.eof()) - readIntBinary(request_for_session.time, buffer); + readIntBinary(request_for_session->time, buffer); else /// backward compatibility - request_for_session.time + request_for_session->time = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); if (!buffer.eof()) - readIntBinary(request_for_session.zxid, buffer); + readIntBinary(request_for_session->zxid, buffer); if (!buffer.eof()) { - request_for_session.digest.emplace(); - readIntBinary(request_for_session.digest->version, buffer); - if (request_for_session.digest->version != KeeperStorage::DigestVersion::NO_DIGEST || !buffer.eof()) - readIntBinary(request_for_session.digest->value, buffer); + request_for_session->digest.emplace(); + readIntBinary(request_for_session->digest->version, buffer); + if (request_for_session->digest->version != KeeperStorage::DigestVersion::NO_DIGEST || !buffer.eof()) + readIntBinary(request_for_session->digest->value, buffer); + } + + if (should_cache && !final) + { + std::lock_guard lock(request_cache_mutex); + parsed_request_cache[request_for_session->session_id].emplace(xid, request_for_session); } return request_for_session; @@ -231,15 +272,15 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) { - auto request_for_session = parseRequest(data); - if (!request_for_session.zxid) - request_for_session.zxid = log_idx; + auto request_for_session = parseRequest(data, true); + if (!request_for_session->zxid) + request_for_session->zxid = log_idx; /// Special processing of session_id request - if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) + if (request_for_session->request->getOpNum() == Coordination::OpNum::SessionID) { const Coordination::ZooKeeperSessionIDRequest & session_id_request - = dynamic_cast(*request_for_session.request); + = dynamic_cast(*request_for_session->request); int64_t session_id; std::shared_ptr response = std::make_shared(); response->internal_id = session_id_request.internal_id; @@ -261,25 +302,34 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n } else { + if (request_for_session->request->getOpNum() == Coordination::OpNum::Close) + { + std::lock_guard lock(request_cache_mutex); + parsed_request_cache.erase(request_for_session->session_id); + } + std::lock_guard lock(storage_and_responses_lock); - KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest( - request_for_session.request, request_for_session.session_id, request_for_session.zxid); + KeeperStorage::ResponsesForSessions responses_for_sessions + = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); for (auto & response_for_session : responses_for_sessions) if (!responses_queue.push(response_for_session)) { ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); - LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response_for_session.session_id); + LOG_WARNING( + log, + "Failed to push response with session id {} to the queue, probably because of shutdown", + response_for_session.session_id); } - if (keeper_context->digest_enabled && request_for_session.digest) - assertDigest(*request_for_session.digest, storage->getNodesDigest(true), *request_for_session.request, true); + if (keeper_context->digest_enabled && request_for_session->digest) + assertDigest(*request_for_session->digest, storage->getNodesDigest(true), *request_for_session->request, true); } ProfileEvents::increment(ProfileEvents::KeeperCommits); last_committed_idx = log_idx; if (commit_callback) - commit_callback(request_for_session); + commit_callback(*request_for_session); return nullptr; } @@ -325,14 +375,14 @@ void KeeperStateMachine::commit_config(const uint64_t /* log_idx */, nuraft::ptr void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data) { - auto request_for_session = parseRequest(data); + auto request_for_session = parseRequest(data, true); // If we received a log from an older node, use the log_idx as the zxid // log_idx will always be larger or equal to the zxid so we can safely do this // (log_idx is increased for all logs, while zxid is only increased for requests) - if (!request_for_session.zxid) - request_for_session.zxid = log_idx; + if (!request_for_session->zxid) + request_for_session->zxid = log_idx; - rollbackRequest(request_for_session, false); + rollbackRequest(*request_for_session, false); } void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing) @@ -523,11 +573,7 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi /// Pure local request, just process it with storage std::lock_guard lock(storage_and_responses_lock); auto responses = storage->processRequest( - request_for_session.request, - request_for_session.session_id, - std::nullopt, - true /*check_acl*/, - true /*is_local*/); + request_for_session.request, request_for_session.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/); for (const auto & response : responses) if (!responses_queue.push(response)) LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 5af5bc05b0f..398c767d010 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -36,7 +36,7 @@ public: /// Read state from the latest snapshot void init(); - static KeeperStorage::RequestForSession parseRequest(nuraft::buffer & data); + std::shared_ptr parseRequest(nuraft::buffer & data, bool final = false); bool preprocess(const KeeperStorage::RequestForSession & request_for_session); @@ -138,6 +138,13 @@ private: /// for request. mutable std::mutex storage_and_responses_lock; + std::unordered_map>> parsed_request_cache; + uint64_t min_request_size_to_cache{0}; + /// we only need to protect the access to the map itself + /// requests can be modified from anywhere without lock because a single request + /// can be processed only in 1 thread at any point + std::mutex request_cache_mutex; + /// Last committed Raft log number. std::atomic last_committed_idx; From 86686fbbc317f60e472663044c5acbf92364e318 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Apr 2023 14:11:18 +0200 Subject: [PATCH 0106/2223] Fix conflicts --- src/Storages/StorageURL.cpp | 560 ++++++++++++++---------------------- src/Storages/StorageURL.h | 127 +------- 2 files changed, 223 insertions(+), 464 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 355daaffcd7..be133794f7f 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -13,11 +14,10 @@ #include #include -#include -#include #include #include #include +#include #include #include @@ -31,14 +31,13 @@ #include #include #include +#include #include #include #include #include #include -#include -#include namespace DB @@ -52,8 +51,8 @@ namespace ErrorCodes } static constexpr auto bad_arguments_error_message = "Storage URL requires 1-4 arguments: " - "url, name of used format (taken from file extension by default), " - "optional compression method, optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; + "url, name of used format (taken from file extension by default), " + "optional compression method, optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; static const std::unordered_set required_configuration_keys = { "url", @@ -84,6 +83,10 @@ static bool urlWithGlobs(const String & uri) return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; } +static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) +{ + return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), {context->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0}); +} IStorageURLBase::IStorageURLBase( const String & uri_, @@ -123,105 +126,131 @@ IStorageURLBase::IStorageURLBase( setInMemoryMetadata(storage_metadata); } -HTTPHeaderEntries getHeaders(const HTTPHeaderEntries & headers_) + +namespace { - HTTPHeaderEntries headers(headers_.begin(), headers_.end()); - - // Propagate OpenTelemetry trace context, if any, downstream. - const auto ¤t_trace_context = OpenTelemetry::CurrentContext(); - if (current_trace_context.isTraceEnabled()) + HTTPHeaderEntries getHeaders(const HTTPHeaderEntries & headers_) { - headers.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); + HTTPHeaderEntries headers(headers_.begin(), headers_.end()); - if (!current_trace_context.tracestate.empty()) + // Propagate OpenTelemetry trace context, if any, downstream. + const auto ¤t_trace_context = OpenTelemetry::CurrentContext(); + if (current_trace_context.isTraceEnabled()) { - headers.emplace_back("tracestate", current_trace_context.tracestate); + headers.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); + + if (!current_trace_context.tracestate.empty()) + { + headers.emplace_back("tracestate", current_trace_context.tracestate); + } } + + return headers; } - return headers; -} -std::vector getPathsList(const String & uri, ContextPtr context) -{ - context->getRemoteHostFilter().checkURL(Poco::URI(uri)); - - Poco::Net::HTTPBasicCredentials credentials; - - std::vector urls_to_check; - if (urlWithGlobs(uri)) + class StorageURLSource : public ISource { - size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; - auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); - for (const auto & description : uri_descriptions) + using URIParams = std::vector>; + + public: + struct URIInfo { - auto options = parseRemoteDescription(description, 0, description.size(), '|', max_addresses); - urls_to_check.insert(urls_to_check.end(), options.begin(), options.end()); - } - } - else - { - urls_to_check = {uri}; - } + using FailoverOptions = std::vector; + std::vector uri_list_to_read; + std::atomic next_uri_to_read = 0; + }; + using URIInfoPtr = std::shared_ptr; - return urls_to_check; -} - -static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri) -{ - const auto & user_info = request_uri.getUserInfo(); - if (!user_info.empty()) - { - std::size_t n = user_info.find(':'); - if (n != std::string::npos) + static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri) { - credentials.setUsername(user_info.substr(0, n)); - credentials.setPassword(user_info.substr(n + 1)); + const auto & user_info = request_uri.getUserInfo(); + if (!user_info.empty()) + { + std::size_t n = user_info.find(':'); + if (n != std::string::npos) + { + credentials.setUsername(user_info.substr(0, n)); + credentials.setPassword(user_info.substr(n + 1)); + } + } } - } -} -void StorageURLSource::setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri) -{ - const auto & user_info = request_uri.getUserInfo(); - if (!user_info.empty()) - { - std::size_t n = user_info.find(':'); - if (n != std::string::npos) + StorageURLSource( + URIInfoPtr uri_info_, + const std::string & http_method, + std::function callback, + const String & format, + const std::optional & format_settings, + String name_, + const Block & sample_block, + ContextPtr context, + const ColumnsDescription & columns, + UInt64 max_block_size, + const ConnectionTimeouts & timeouts, + CompressionMethod compression_method, + size_t download_threads, + const HTTPHeaderEntries & headers_ = {}, + const URIParams & params = {}, + bool glob_url = false) + : ISource(sample_block), name(std::move(name_)), uri_info(uri_info_) { - credentials.setUsername(user_info.substr(0, n)); - credentials.setPassword(user_info.substr(n + 1)); + auto headers = getHeaders(headers_); + + /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. + initialize = [=, this](const URIInfo::FailoverOptions & uri_options) + { + if (uri_options.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); + + auto first_option = uri_options.begin(); + auto buf_factory = getFirstAvailableURLReadBuffer( + first_option, + uri_options.end(), + context, + params, + http_method, + callback, + timeouts, + credentials, + headers, + glob_url, + uri_options.size() == 1); + + try + { + total_size += buf_factory->getFileSize(); + } + catch (...) + { + // we simply continue without total_size + } + + // TODO: Pass max_parsing_threads and max_download_threads adjusted for num_streams. + auto input_format = FormatFactory::instance().getInputRandomAccess( + format, + std::move(buf_factory), + sample_block, + context, + max_block_size, + /* is_remote_fs */ true, + compression_method, + format_settings, + download_threads); + + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); + + builder.addSimpleTransform( + [&](const Block & cur_header) + { return std::make_shared(cur_header, columns, *input_format, context); }); + + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + reader = std::make_unique(*pipeline); + }; } - } -} -StorageURLSource::StorageURLSource( - URIInfoPtr uri_info_, - const std::string & http_method, - std::function callback, - const String & format, - const std::optional & format_settings, - String name_, - const Block & sample_block, - ContextPtr context, - const ColumnsDescription & columns, - UInt64 max_block_size, - const ConnectionTimeouts & timeouts, - CompressionMethod compression_method, - size_t download_threads, - const HTTPHeaderEntries & headers_, - const URIParams & params, - bool glob_url) - : ISource(sample_block), name(std::move(name_)), uri_info(uri_info_) -{ - auto headers = getHeaders(headers_); - - - /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. - initialize = [=, this](const URIInfo::FailoverOptions & uri_options) - { - if (uri_options.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); + String getName() const override { return name; } Chunk generate() override { @@ -240,43 +269,21 @@ StorageURLSource::StorageURLSource( if (current_uri_pos >= uri_info->uri_list_to_read.size()) return {}; - auto first_option = uri_options.begin(); - read_buf = getFirstAvailableURLReadBuffer( - first_option, - uri_options.end(), - context, - params, - http_method, - callback, - timeouts, - compression_method, - credentials, - headers, - glob_url, - uri_options.size() == 1, - download_threads); - - - auto input_format - = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size, format_settings); - QueryPipelineBuilder builder; - builder.init(Pipe(input_format)); - - builder.addSimpleTransform( - [&](const Block & cur_header) - { return std::make_shared(cur_header, columns, *input_format, context); }); - - pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - reader = std::make_unique(*pipeline); - }; -} + auto current_uri = uri_info->uri_list_to_read[current_uri_pos]; initialize(current_uri); } Chunk chunk; if (reader->pull(chunk)) + { + UInt64 num_rows = chunk.getNumRows(); + if (num_rows && total_size) + updateRowsProgressApprox( + *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + return chunk; + } pipeline->reset(); reader.reset(); @@ -284,145 +291,35 @@ StorageURLSource::StorageURLSource( return {}; } - -std::unique_ptr StorageURLSource::getFirstAvailableURLReadBuffer( - std::vector::const_iterator & option, - const std::vector::const_iterator & end, - ContextPtr context, - const URIParams & params, - const String & http_method, - std::function callback, - const ConnectionTimeouts & timeouts, - CompressionMethod compression_method, - Poco::Net::HTTPBasicCredentials & credentials, - const HTTPHeaderEntries & headers, - bool glob_url, - bool delay_initialization, - size_t download_threads) -{ - String first_exception_message; - ReadSettings read_settings = context->getReadSettings(); - - size_t options = std::distance(option, end); - for (; option != end; ++option) - { - bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); - auto request_uri = Poco::URI(*option); - - for (const auto & [param, value] : params) - request_uri.addQueryParameter(param, value); - - setCredentials(credentials, request_uri); - - const auto settings = context->getSettings(); - int zstd_window_log_max = static_cast(settings.zstd_window_log_max); - try + static SeekableReadBufferFactoryPtr getFirstAvailableURLReadBuffer( + std::vector::const_iterator & option, + const std::vector::const_iterator & end, + ContextPtr context, + const URIParams & params, + const String & http_method, + std::function callback, + const ConnectionTimeouts & timeouts, + Poco::Net::HTTPBasicCredentials & credentials, + const HTTPHeaderEntries & headers, + bool glob_url, + bool delay_initialization) { - if (download_threads > 1) + String first_exception_message; + ReadSettings read_settings = context->getReadSettings(); + + size_t options = std::distance(option, end); + for (; option != end; ++option) { - try - { - ReadWriteBufferFromHTTP buffer( - request_uri, - Poco::Net::HTTPRequest::HTTP_HEAD, - callback, - timeouts, - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, - ReadWriteBufferFromHTTP::Range{0, std::nullopt}, - &context->getRemoteHostFilter(), - true, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error); + bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); + auto request_uri = Poco::URI(*option); - Poco::Net::HTTPResponse res; + for (const auto & [param, value] : params) + request_uri.addQueryParameter(param, value); - for (size_t i = 0; i < settings.http_max_tries; ++i) - { - try - { - buffer.callWithRedirects(res, Poco::Net::HTTPRequest::HTTP_HEAD, true); - break; - } - catch (const Poco::Exception & e) - { - LOG_TRACE( - &Poco::Logger::get("StorageURLSource"), - "HTTP HEAD request to `{}` failed at try {}/{}. " - "Error: {}.", - request_uri.toString(), - i + 1, - settings.http_max_tries, - e.displayText()); - if (!ReadWriteBufferFromHTTP::isRetriableError(res.getStatus())) - { - throw; - } - } - } + setCredentials(credentials, request_uri); - // to check if Range header is supported, we need to send a request with it set - const bool supports_ranges = (res.has("Accept-Ranges") && res.get("Accept-Ranges") == "bytes") - || (res.has("Content-Range") && res.get("Content-Range").starts_with("bytes")); - - if (supports_ranges) - LOG_TRACE(&Poco::Logger::get("StorageURLSource"), "HTTP Range is supported"); - else - LOG_TRACE(&Poco::Logger::get("StorageURLSource"), "HTTP Range is not supported"); - - - if (supports_ranges && res.getStatus() == Poco::Net::HTTPResponse::HTTP_PARTIAL_CONTENT - && res.hasContentLength()) - { - LOG_TRACE( - &Poco::Logger::get("StorageURLSource"), - "Using ParallelReadBuffer with {} workers with chunks of {} bytes", - download_threads, - settings.max_download_buffer_size); - - auto read_buffer_factory = std::make_unique( - res.getContentLength(), - settings.max_download_buffer_size, - request_uri, - http_method, - callback, - timeouts, - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, - &context->getRemoteHostFilter(), - delay_initialization, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error); - - return wrapReadBufferWithCompressionMethod( - std::make_unique( - std::move(read_buffer_factory), - threadPoolCallbackRunner(IOThreadPool::get(), "URLParallelRead"), - download_threads), - compression_method, - zstd_window_log_max); - } - } - catch (const Poco::Exception & e) - { - LOG_TRACE( - &Poco::Logger::get("StorageURLSource"), - "Failed to setup ParallelReadBuffer because of an exception:\n{}.\nFalling back to the single-threaded " - "buffer", - e.displayText()); - } - } - - LOG_TRACE(&Poco::Logger::get("StorageURLSource"), "Using single-threaded read buffer"); - - return wrapReadBufferWithCompressionMethod( - std::make_unique( + const auto settings = context->getSettings(); + auto res = std::make_unique( request_uri, http_method, callback, @@ -432,99 +329,52 @@ std::unique_ptr StorageURLSource::getFirstAvailableURLReadBuffer( settings.max_read_buffer_size, read_settings, headers, - ReadWriteBufferFromHTTP::Range{}, &context->getRemoteHostFilter(), delay_initialization, /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error), - compression_method, - zstd_window_log_max); + /* skip_url_not_found_error */ skip_url_not_found_error); + + if (options > 1) + { + // Send a HEAD request to check availability. + try + { + res->getFileInfo(); + } + catch (...) + { + if (first_exception_message.empty()) + first_exception_message = getCurrentExceptionMessage(false); + + tryLogCurrentException(__PRETTY_FUNCTION__); + + continue; + } + } + + return res; + } + + throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message); } - catch (...) - { - if (first_exception_message.empty()) - first_exception_message = getCurrentExceptionMessage(false); - if (options == 1) - throw; + private: + using InitializeFunc = std::function; + InitializeFunc initialize; - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } + String name; + URIInfoPtr uri_info; - throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message); -} - -class StorageURLSource::DisclosedGlobIterator::Impl -{ -public: - Impl(ContextPtr context_, const String & uri) - { - uris = getPathsList(uri, context_); - uris_iter = uris.begin(); - } - - String next() - { - std::lock_guard lock(mutex); - if (uris_iter != uris.end()) - { - auto answer = *uris_iter; - ++uris_iter; - return answer; - } - return {}; - } -private: - std::mutex mutex; - Strings uris; - Strings::iterator uris_iter; -}; - - -StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(ContextPtr context_, const String & uri) - : pimpl(std::make_shared(context_, uri)) {} - - std::unique_ptr read_buf; std::unique_ptr pipeline; std::unique_ptr reader; -String StorageURLSource::DisclosedGlobIterator::next() -{ - return pimpl->next(); -} + Poco::Net::HTTPBasicCredentials credentials; -void StorageURLSource::onCancel() -{ - std::lock_guard lock(reader_mutex); - if (reader) - reader->cancel(); -} - -Chunk StorageURLSource::generate() -{ - while (true) - { - if (!reader) - { - auto current_uri_pos = uri_info->next_uri_to_read.fetch_add(1); - if (current_uri_pos >= uri_info->uri_list_to_read.size()) - return {}; - - auto current_uri = uri_info->uri_list_to_read[current_uri_pos]; - - std::lock_guard lock(reader_mutex); - initialize(current_uri); - } - - Chunk chunk; - std::lock_guard lock(reader_mutex); - if (reader->pull(chunk)) - return chunk; - - pipeline->reset(); - reader.reset(); - } + size_t total_size = 0; + UInt64 total_rows_approx_max = 0; + size_t total_rows_count_times = 0; + UInt64 total_rows_approx_accumulated = 0; + }; } StorageURLSink::StorageURLSink( @@ -535,6 +385,7 @@ StorageURLSink::StorageURLSink( ContextPtr context, const ConnectionTimeouts & timeouts, const CompressionMethod compression_method, + const HTTPHeaderEntries & headers, const String & http_method) : SinkToStorage(sample_block) { @@ -542,12 +393,13 @@ StorageURLSink::StorageURLSink( std::string content_encoding = toContentEncodingName(compression_method); write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(Poco::URI(uri), http_method, content_type, content_encoding, timeouts), + std::make_unique(Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts), compression_method, 3); writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, format_settings); } + void StorageURLSink::consume(Chunk chunk) { std::lock_guard lock(cancel_mutex); @@ -606,6 +458,7 @@ public: ContextPtr context_, const ConnectionTimeouts & timeouts_, const CompressionMethod compression_method_, + const HTTPHeaderEntries & headers_, const String & http_method_) : PartitionedSink(partition_by, context_, sample_block_) , uri(uri_) @@ -615,6 +468,7 @@ public: , context(context_) , timeouts(timeouts_) , compression_method(compression_method_) + , headers(headers_) , http_method(http_method_) { } @@ -624,7 +478,7 @@ public: auto partition_path = PartitionedSink::replaceWildcards(uri, partition_id); context->getRemoteHostFilter().checkURL(Poco::URI(partition_path)); return std::make_shared( - partition_path, format, format_settings, sample_block, context, timeouts, compression_method, http_method); + partition_path, format, format_settings, sample_block, context, timeouts, compression_method, headers, http_method); } private: @@ -636,6 +490,7 @@ private: const ConnectionTimeouts timeouts; const CompressionMethod compression_method; + const HTTPHeaderEntries headers; const String http_method; }; @@ -704,22 +559,23 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( if (it == urls_to_check.cend()) return nullptr; - auto buf = StorageURLSource::getFirstAvailableURLReadBuffer( + auto buf_factory = StorageURLSource::getFirstAvailableURLReadBuffer( it, urls_to_check.cend(), context, {}, Poco::Net::HTTPRequest::HTTP_GET, {}, - ConnectionTimeouts::getHTTPTimeouts(context), - compression_method, + getHTTPTimeouts(context), credentials, headers, false, - false, - context->getSettingsRef().max_download_threads);\ + false); ++it; - return buf; + return wrapReadBufferWithCompressionMethod( + buf_factory->getReader(), + compression_method, + static_cast(context->getSettingsRef().zstd_window_log_max)); }; ColumnsDescription columns; @@ -795,7 +651,7 @@ Pipe IStorageURLBase::read( local_context, columns_description, max_block_size, - ConnectionTimeouts::getHTTPTimeouts(local_context), + getHTTPTimeouts(local_context), compression_method, download_threads, headers, @@ -819,7 +675,7 @@ Pipe IStorageURLBase::read( local_context, columns_description, max_block_size, - ConnectionTimeouts::getHTTPTimeouts(local_context), + getHTTPTimeouts(local_context), compression_method, max_download_threads, headers, @@ -854,6 +710,7 @@ Pipe StorageURLWithFailover::read( auto uri_info = std::make_shared(); uri_info->uri_list_to_read.emplace_back(uri_options); + auto pipe = Pipe(std::make_shared( uri_info, getReadMethod(), @@ -865,7 +722,7 @@ Pipe StorageURLWithFailover::read( local_context, columns_description, max_block_size, - ConnectionTimeouts::getHTTPTimeouts(local_context), + getHTTPTimeouts(local_context), compression_method, local_context->getSettingsRef().max_download_threads, headers, @@ -894,8 +751,9 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad format_settings, metadata_snapshot->getSampleBlock(), context, - ConnectionTimeouts::getHTTPTimeouts(context), + getHTTPTimeouts(context), compression_method, + headers, http_method); } else @@ -906,8 +764,9 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad format_settings, metadata_snapshot->getSampleBlock(), context, - ConnectionTimeouts::getHTTPTimeouts(context), + getHTTPTimeouts(context), compression_method, + headers, http_method); } } @@ -975,13 +834,12 @@ std::optional IStorageURLBase::getLastModificationTime( Poco::URI(url), Poco::Net::HTTPRequest::HTTP_GET, {}, - ConnectionTimeouts::getHTTPTimeouts(context), + getHTTPTimeouts(context), credentials, settings.max_http_get_redirects, settings.max_read_buffer_size, context->getReadSettings(), headers, - ReadWriteBufferFromHTTP::Range{}, &context->getRemoteHostFilter(), true, false, @@ -1166,7 +1024,7 @@ StorageURL::Configuration StorageURL::getConfiguration(ASTs & args, ContextPtr l { StorageURL::Configuration configuration; - if (auto named_collection = tryGetNamedCollectionWithOverrides(args)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(args, local_context)) { StorageURL::processNamedCollectionResult(configuration, *named_collection); collectHeaders(args, configuration.headers, local_context); @@ -1234,4 +1092,4 @@ void registerStorageURL(StorageFactory & factory) .source_access_type = AccessType::URL, }); } -} +} \ No newline at end of file diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index c35d3aebdc7..9e3e3aa46c7 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -7,10 +7,9 @@ #include #include #include -#include #include -#include #include +#include namespace DB @@ -137,6 +136,7 @@ public: ContextPtr context, const ConnectionTimeouts & timeouts, CompressionMethod compression_method, + const HTTPHeaderEntries & headers = {}, const String & method = Poco::Net::HTTPRequest::HTTP_POST); std::string getName() const override { return "StorageURLSink"; } @@ -182,16 +182,12 @@ public: static FormatSettings getFormatSettingsFromArgs(const StorageFactory::Arguments & args); - struct Configuration + struct Configuration : public StatelessTableEngineConfiguration { std::string url; std::string http_method; - - std::string format = "auto"; - std::string compression_method = "auto"; - std::string structure = "auto"; - HTTPHeaderEntries headers; + std::string addresses_expr; }; static Configuration getConfiguration(ASTs & args, ContextPtr context); @@ -201,19 +197,20 @@ public: static void processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection); }; + /// StorageURLWithFailover is allowed only for URL table function, not as a separate storage. class StorageURLWithFailover final : public StorageURL { public: StorageURLWithFailover( - const std::vector & uri_options_, - const StorageID & table_id_, - const String & format_name_, - const std::optional & format_settings_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - ContextPtr context_, - const String & compression_method_); + const std::vector & uri_options_, + const StorageID & table_id_, + const String & format_name_, + const std::optional & format_settings_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + ContextPtr context_, + const String & compression_method_); Pipe read( const Names & column_names, @@ -224,103 +221,7 @@ public: size_t max_block_size, size_t num_streams) override; - struct Configuration - { - String url; - String compression_method = "auto"; - std::vector> headers; - }; - private: std::vector uri_options; }; - -class PullingPipelineExecutor; - -class StorageURLSource : public ISource -{ - using URIParams = std::vector>; - -public: - - class DisclosedGlobIterator - { - public: - DisclosedGlobIterator(ContextPtr context_, const String & uri_); - String next(); - private: - class Impl; - /// shared_ptr to have copy constructor - std::shared_ptr pimpl; - }; - - struct URIInfo - { - using FailoverOptions = std::vector; - std::vector uri_list_to_read; - std::atomic next_uri_to_read = 0; - }; - - using IteratorWrapper = std::function; - using URIInfoPtr = std::shared_ptr; - - void onCancel() override; - - static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri); - - StorageURLSource( - URIInfoPtr uri_info_, - const std::string & http_method, - std::function callback, - const String & format, - const std::optional & format_settings, - String name_, - const Block & sample_block, - ContextPtr context, - const ColumnsDescription & columns, - UInt64 max_block_size, - const ConnectionTimeouts & timeouts, - CompressionMethod compression_method, - size_t download_threads, - const HTTPHeaderEntries & headers_ = {}, - const URIParams & params = {}, - bool glob_url = false); - - String getName() const override { return name; } - - Chunk generate() override; - - static std::unique_ptr getFirstAvailableURLReadBuffer( - std::vector::const_iterator & option, - const std::vector::const_iterator & end, - ContextPtr context, - const URIParams & params, - const String & http_method, - std::function callback, - const ConnectionTimeouts & timeouts, - CompressionMethod compression_method, - Poco::Net::HTTPBasicCredentials & credentials, - const HTTPHeaderEntries & headers, - bool glob_url, - bool delay_initialization, - size_t download_threads); - -private: - using InitializeFunc = std::function; - InitializeFunc initialize; - - String name; - URIInfoPtr uri_info; - - std::unique_ptr read_buf; - std::unique_ptr pipeline; - std::unique_ptr reader; - /// onCancell and generate can be called concurrently and both of them - /// have R/W access to reader pointer. - std::mutex reader_mutex; - - Poco::Net::HTTPBasicCredentials credentials; - -}; - -} +} \ No newline at end of file From 944f54aadf7eeace9041aa00758dc6dee19eec95 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Apr 2023 17:24:37 +0000 Subject: [PATCH 0107/2223] Finish urlCluster, refactor code, reduce code duplication --- src/Storages/HDFS/StorageHDFSCluster.cpp | 91 +-- src/Storages/HDFS/StorageHDFSCluster.h | 9 - src/Storages/IStorageCluster.cpp | 141 +++++ src/Storages/IStorageCluster.h | 22 +- src/Storages/StorageClusterBase.cpp | 0 src/Storages/StorageClusterBase.h | 50 ++ src/Storages/StorageS3Cluster.cpp | 126 +--- src/Storages/StorageS3Cluster.h | 22 +- src/Storages/StorageURL.cpp | 547 +++++++++--------- src/Storages/StorageURL.h | 89 ++- src/Storages/StorageURLCluster.cpp | 88 +-- src/Storages/StorageURLCluster.h | 9 - src/TableFunctions/TableFunctionS3Cluster.cpp | 7 +- src/TableFunctions/TableFunctionS3Cluster.h | 3 +- src/TableFunctions/TableFunctionURL.h | 9 +- .../TableFunctionURLCluster.cpp | 23 +- src/TableFunctions/TableFunctionURLCluster.h | 1 - .../0_stateless/02721_url_cluster.reference | 88 +++ .../queries/0_stateless/02721_url_cluster.sql | 29 + 19 files changed, 734 insertions(+), 620 deletions(-) create mode 100644 src/Storages/IStorageCluster.cpp create mode 100644 src/Storages/StorageClusterBase.cpp create mode 100644 src/Storages/StorageClusterBase.h create mode 100644 tests/queries/0_stateless/02721_url_cluster.reference create mode 100644 tests/queries/0_stateless/02721_url_cluster.sql diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 45f1df92e38..abe24545237 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -5,30 +5,20 @@ #include -#include #include #include -#include -#include #include -#include #include -#include -#include -#include #include #include #include -#include #include #include #include #include -#include -#include #include @@ -46,12 +36,10 @@ StorageHDFSCluster::StorageHDFSCluster( const ConstraintsDescription & constraints_, const String & compression_method_, bool structure_argument_was_provided_) - : IStorageCluster(table_id_) - , cluster_name(cluster_name_) + : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageHDFSCluster (" + table_id_.table_name + ")"), 3, structure_argument_was_provided_) , uri(uri_) , format_name(format_name_) , compression_method(compression_method_) - , structure_argument_was_provided(structure_argument_was_provided_) { checkHDFSURL(uri_); context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); @@ -70,82 +58,6 @@ StorageHDFSCluster::StorageHDFSCluster( setInMemoryMetadata(storage_metadata); } -/// The code executes on initiator -Pipe StorageHDFSCluster::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t /*max_block_size*/, - size_t /*num_streams*/) -{ - auto cluster = getCluster(context); - auto extension = getTaskIteratorExtension(query_info.query, context); - - /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) - Block header; - - if (context->getSettingsRef().allow_experimental_analyzer) - header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_info.query, context, SelectQueryOptions(processed_stage).analyze()); - else - header = InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - - const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; - - Pipes pipes; - - const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; - - auto query_to_send = query_info.original_query->clone(); - if (!structure_argument_was_provided) - addColumnsStructureToQueryWithClusterEngine( - query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 3, getName()); - - const auto & current_settings = context->getSettingsRef(); - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - for (const auto & shard_info : cluster->getShardsInfo()) - { - auto try_results = shard_info.pool->getMany(timeouts, ¤t_settings, PoolMode::GET_MANY); - for (auto & try_result : try_results) - { - auto remote_query_executor = std::make_shared( - std::vector{try_result}, - queryToString(query_to_send), - header, - context, - /*throttler=*/nullptr, - scalars, - Tables(), - processed_stage, - extension); - - pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false, false)); - } - } - - storage_snapshot->check(column_names); - return Pipe::unitePipes(std::move(pipes)); -} - -QueryProcessingStage::Enum StorageHDFSCluster::getQueryProcessingStage( - ContextPtr context, QueryProcessingStage::Enum to_stage, const StorageSnapshotPtr &, SelectQueryInfo &) const -{ - /// Initiator executes query on remote node. - if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) - if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) - return QueryProcessingStage::Enum::WithMergeableState; - - /// Follower just reads the data. - return QueryProcessingStage::Enum::FetchColumns; -} - - -ClusterPtr StorageHDFSCluster::getCluster(ContextPtr context) const -{ - return context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); -} - RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(ASTPtr, ContextPtr context) const { auto iterator = std::make_shared(context, uri); @@ -160,7 +72,6 @@ NamesAndTypesList StorageHDFSCluster::getVirtuals() const {"_file", std::make_shared(std::make_shared())}}; } - } #endif diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h index 4d6548a6b78..9a30d85a37d 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -33,23 +33,14 @@ public: std::string getName() const override { return "HDFSCluster"; } - Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, - ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, size_t /*num_streams*/) override; - - QueryProcessingStage::Enum - getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; - NamesAndTypesList getVirtuals() const override; - ClusterPtr getCluster(ContextPtr context) const override; RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; private: - String cluster_name; String uri; String format_name; String compression_method; - bool structure_argument_was_provided; }; diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp new file mode 100644 index 00000000000..faf0e107c89 --- /dev/null +++ b/src/Storages/IStorageCluster.cpp @@ -0,0 +1,141 @@ +#include "Storages/IStorageCluster.h" + +#include "Common/Exception.h" +#include "Core/QueryProcessingStage.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +IStorageCluster::IStorageCluster( + String cluster_name_, + const StorageID & table_id_, + Poco::Logger * log_, + size_t max_function_arguments_, + bool structure_argument_was_provided_) + : IStorage(table_id_) + , log(log_) + , cluster_name(cluster_name_) + , max_function_arguments(max_function_arguments_) + , structure_argument_was_provided(structure_argument_was_provided_) +{ +} + + +/// The code executes on initiator +Pipe IStorageCluster::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t /*max_block_size*/, + size_t /*num_streams*/) +{ + updateBeforeRead(context); + + auto cluster = getCluster(context); + auto extension = getTaskIteratorExtension(query_info.query, context); + + /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) + + Block sample_block; + ASTPtr query_to_send = query_info.query; + + if (context->getSettingsRef().allow_experimental_analyzer) + { + sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(query_info.query, context, SelectQueryOptions(processed_stage)); + } + else + { + auto interpreter = InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()); + sample_block = interpreter.getSampleBlock(); + query_to_send = interpreter.getQueryInfo().query->clone(); + } + + const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; + + Pipes pipes; + + const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; + + if (!structure_argument_was_provided) + addColumnsStructureToQueryWithClusterEngine( + query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), max_function_arguments, getName()); + + RestoreQualifiedNamesVisitor::Data data; + data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query_info.query->as(), 0)); + data.remote_table.database = context->getCurrentDatabase(); + data.remote_table.table = getName(); + RestoreQualifiedNamesVisitor(data).visit(query_to_send); + AddDefaultDatabaseVisitor visitor(context, context->getCurrentDatabase(), + /* only_replace_current_database_function_= */false, + /* only_replace_in_join_= */true); + visitor.visit(query_to_send); + + const auto & current_settings = context->getSettingsRef(); + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); + for (const auto & shard_info : cluster->getShardsInfo()) + { + auto try_results = shard_info.pool->getMany(timeouts, ¤t_settings, PoolMode::GET_MANY); + for (auto & try_result : try_results) + { + auto remote_query_executor = std::make_shared( + std::vector{try_result}, + queryToString(query_to_send), + sample_block, + context, + /*throttler=*/nullptr, + scalars, + Tables(), + processed_stage, + extension); + + remote_query_executor->setLogger(log); + pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false, false)); + } + } + + storage_snapshot->check(column_names); + return Pipe::unitePipes(std::move(pipes)); +} + +QueryProcessingStage::Enum IStorageCluster::getQueryProcessingStage( + ContextPtr context, QueryProcessingStage::Enum to_stage, const StorageSnapshotPtr &, SelectQueryInfo &) const +{ + /// Initiator executes query on remote node. + if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) + if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) + return QueryProcessingStage::Enum::WithMergeableState; + + /// Follower just reads the data. + return QueryProcessingStage::Enum::FetchColumns; +} + + +ClusterPtr IStorageCluster::getCluster(ContextPtr context) const +{ + return context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); +} + +} diff --git a/src/Storages/IStorageCluster.h b/src/Storages/IStorageCluster.h index 35d297428ba..e3ee6363974 100644 --- a/src/Storages/IStorageCluster.h +++ b/src/Storages/IStorageCluster.h @@ -15,14 +15,32 @@ namespace DB class IStorageCluster : public IStorage { public: + IStorageCluster( + String cluster_name_, + const StorageID & table_id_, + Poco::Logger * log_, + size_t max_function_arguments_, + bool structure_argument_was_provided_); - explicit IStorageCluster(const StorageID & table_id_) : IStorage(table_id_) {} + Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, + ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, size_t /*num_streams*/) override; - virtual ClusterPtr getCluster(ContextPtr context) const = 0; + ClusterPtr getCluster(ContextPtr context) const; /// Query is needed for pruning by virtual columns (_file, _path) virtual RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const = 0; + QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; + bool isRemote() const override { return true; } + +protected: + virtual void updateBeforeRead(const ContextPtr &) {} + +private: + Poco::Logger * log; + String cluster_name; + size_t max_function_arguments; + bool structure_argument_was_provided; }; diff --git a/src/Storages/StorageClusterBase.cpp b/src/Storages/StorageClusterBase.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Storages/StorageClusterBase.h b/src/Storages/StorageClusterBase.h new file mode 100644 index 00000000000..da048b635d8 --- /dev/null +++ b/src/Storages/StorageClusterBase.h @@ -0,0 +1,50 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include +#include + +#include "Client/Connection.h" +#include +#include +#include +#include + +namespace DB +{ + +class Context; + +class StorageClusterBase : public IStorageCluster +{ +public: + StorageClusterBase( + String cluster_name, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + ContextPtr context_, + bool structure_argument_was_provided_); + + Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, + ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, size_t /*num_streams*/) override; + + QueryProcessingStage::Enum + getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; + + RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; + ClusterPtr getCluster(ContextPtr context) const override; + +private: + Poco::Logger * log; + String cluster_name; + bool structure_argument_was_provided; +}; + + +} + +#endif diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 220cc1dc1f6..d0b0941ac9d 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -5,35 +5,17 @@ #if USE_AWS_S3 #include "Common/Exception.h" -#include "Client/Connection.h" -#include "Core/QueryProcessingStage.h" #include #include -#include -#include -#include -#include -#include #include #include -#include -#include #include -#include -#include #include #include -#include -#include -#include #include #include #include #include -#include -#include - -#include #include #include @@ -42,19 +24,15 @@ namespace DB { StorageS3Cluster::StorageS3Cluster( - const Configuration & configuration_, + const String & cluster_name_, + const StorageS3::Configuration & configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, ContextPtr context_, bool structure_argument_was_provided_) - : IStorageCluster(table_id_) - , log(&Poco::Logger::get("StorageS3Cluster (" + table_id_.table_name + ")")) + : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageS3Cluster (" + table_id_.table_name + ")"), 5, structure_argument_was_provided_) , s3_configuration{configuration_} - , cluster_name(configuration_.cluster_name) - , format_name(configuration_.format) - , compression_method(configuration_.compression_method) - , structure_argument_was_provided(structure_argument_was_provided_) { context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri); StorageInMemoryMetadata storage_metadata; @@ -62,8 +40,6 @@ StorageS3Cluster::StorageS3Cluster( if (columns_.empty()) { - /// `distributed_processing` is set to false, because this code is executed on the initiator, so there is no callback set - /// for asking for the next tasks. /// `format_settings` is set to std::nullopt, because StorageS3Cluster is used only as table function auto columns = StorageS3::getTableStructureFromDataImpl(s3_configuration, /*format_settings=*/std::nullopt, context_); storage_metadata.setColumns(columns); @@ -89,102 +65,6 @@ void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context) s3_configuration.update(local_context); } -/// The code executes on initiator -Pipe StorageS3Cluster::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t /*max_block_size*/, - size_t /*num_streams*/) -{ - updateConfigurationIfChanged(context); - - auto cluster = getCluster(context); - auto extension = getTaskIteratorExtension(query_info.query, context); - - /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) - - Block sample_block; - ASTPtr query_to_send = query_info.query; - - if (context->getSettingsRef().allow_experimental_analyzer) - { - sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(query_info.query, context, SelectQueryOptions(processed_stage)); - } - else - { - auto interpreter = InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()); - sample_block = interpreter.getSampleBlock(); - query_to_send = interpreter.getQueryInfo().query->clone(); - } - - const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; - - Pipes pipes; - - const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; - - if (!structure_argument_was_provided) - addColumnsStructureToQueryWithClusterEngine( - query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 5, getName()); - - RestoreQualifiedNamesVisitor::Data data; - data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query_info.query->as(), 0)); - data.remote_table.database = context->getCurrentDatabase(); - data.remote_table.table = getName(); - RestoreQualifiedNamesVisitor(data).visit(query_to_send); - AddDefaultDatabaseVisitor visitor(context, context->getCurrentDatabase(), - /* only_replace_current_database_function_= */false, - /* only_replace_in_join_= */true); - visitor.visit(query_to_send); - - const auto & current_settings = context->getSettingsRef(); - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - for (const auto & shard_info : cluster->getShardsInfo()) - { - auto try_results = shard_info.pool->getMany(timeouts, ¤t_settings, PoolMode::GET_MANY); - for (auto & try_result : try_results) - { - auto remote_query_executor = std::make_shared( - std::vector{try_result}, - queryToString(query_to_send), - sample_block, - context, - /*throttler=*/nullptr, - scalars, - Tables(), - processed_stage, - extension); - - remote_query_executor->setLogger(log); - pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false, false)); - } - } - - storage_snapshot->check(column_names); - return Pipe::unitePipes(std::move(pipes)); -} - -QueryProcessingStage::Enum StorageS3Cluster::getQueryProcessingStage( - ContextPtr context, QueryProcessingStage::Enum to_stage, const StorageSnapshotPtr &, SelectQueryInfo &) const -{ - /// Initiator executes query on remote node. - if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) - if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) - return QueryProcessingStage::Enum::WithMergeableState; - - /// Follower just reads the data. - return QueryProcessingStage::Enum::FetchColumns; -} - - -ClusterPtr StorageS3Cluster::getCluster(ContextPtr context) const -{ - return context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); -} - RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr query, ContextPtr context) const { auto iterator = std::make_shared( diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index 2896ab57f49..96bb4cad679 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -21,13 +21,9 @@ class Context; class StorageS3Cluster : public IStorageCluster { public: - struct Configuration : public StorageS3::Configuration - { - std::string cluster_name; - }; - StorageS3Cluster( - const Configuration & configuration_, + const String & cluster_name_, + const StorageS3::Configuration & configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -36,29 +32,19 @@ public: std::string getName() const override { return "S3Cluster"; } - Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, - ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, size_t /*num_streams*/) override; - - QueryProcessingStage::Enum - getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; - NamesAndTypesList getVirtuals() const override; RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; - ClusterPtr getCluster(ContextPtr context) const override; protected: void updateConfigurationIfChanged(ContextPtr local_context); private: - Poco::Logger * log; + void updateBeforeRead(const ContextPtr & context) override { updateConfigurationIfChanged(context); } + StorageS3::Configuration s3_configuration; - String cluster_name; - String format_name; - String compression_method; NamesAndTypesList virtual_columns; Block virtual_block; - bool structure_argument_was_provided; }; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 2274f4c9440..43281432dcb 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -100,7 +100,8 @@ IStorageURLBase::IStorageURLBase( const String & compression_method_, const HTTPHeaderEntries & headers_, const String & http_method_, - ASTPtr partition_by_) + ASTPtr partition_by_, + bool distributed_processing_) : IStorage(table_id_) , uri(uri_) , compression_method(chooseCompressionMethod(Poco::URI(uri_).getPath(), compression_method_)) @@ -109,6 +110,7 @@ IStorageURLBase::IStorageURLBase( , headers(headers_) , http_method(http_method_) , partition_by(partition_by_) + , distributed_processing(distributed_processing_) { FormatFactory::instance().checkFormatName(format_name); StorageInMemoryMetadata storage_metadata; @@ -134,7 +136,7 @@ namespace HTTPHeaderEntries headers(headers_.begin(), headers_.end()); // Propagate OpenTelemetry trace context, if any, downstream. - const auto ¤t_trace_context = OpenTelemetry::CurrentContext(); + const auto & current_trace_context = OpenTelemetry::CurrentContext(); if (current_trace_context.isTraceEnabled()) { headers.emplace_back("traceparent", current_trace_context.composeTraceparentHeader()); @@ -148,235 +150,244 @@ namespace return headers; } - - class StorageURLSource : public ISource + StorageURLSource::FailoverOptions getFailoverOptions(const String & uri, size_t max_addresses) { - using URIParams = std::vector>; + return parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); + } +} - public: - struct URIInfo - { - using FailoverOptions = std::vector; - std::vector uri_list_to_read; - std::atomic next_uri_to_read = 0; - }; - using URIInfoPtr = std::shared_ptr; +class StorageURLSource::DisclosedGlobIterator::Impl +{ +public: + Impl(ContextPtr context, const String & uri) + { + context->getRemoteHostFilter().checkURL(Poco::URI(uri)); + uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); + } - static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri) - { - const auto & user_info = request_uri.getUserInfo(); - if (!user_info.empty()) - { - std::size_t n = user_info.find(':'); - if (n != std::string::npos) - { - credentials.setUsername(user_info.substr(0, n)); - credentials.setPassword(user_info.substr(n + 1)); - } - } - } - - StorageURLSource( - URIInfoPtr uri_info_, - const std::string & http_method, - std::function callback, - const String & format, - const std::optional & format_settings, - String name_, - const Block & sample_block, - ContextPtr context, - const ColumnsDescription & columns, - UInt64 max_block_size, - const ConnectionTimeouts & timeouts, - CompressionMethod compression_method, - size_t download_threads, - const HTTPHeaderEntries & headers_ = {}, - const URIParams & params = {}, - bool glob_url = false) - : ISource(sample_block), name(std::move(name_)), uri_info(uri_info_) - { - auto headers = getHeaders(headers_); - - /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. - initialize = [=, this](const URIInfo::FailoverOptions & uri_options) - { - if (uri_options.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); - - auto first_option = uri_options.begin(); - auto buf_factory = getFirstAvailableURLReadBuffer( - first_option, - uri_options.end(), - context, - params, - http_method, - callback, - timeouts, - credentials, - headers, - glob_url, - uri_options.size() == 1); - - try - { - total_size += buf_factory->getFileSize(); - } - catch (...) - { - // we simply continue without total_size - } - - // TODO: Pass max_parsing_threads and max_download_threads adjusted for num_streams. - auto input_format = FormatFactory::instance().getInputRandomAccess( - format, - std::move(buf_factory), - sample_block, - context, - max_block_size, - /* is_remote_fs */ true, - compression_method, - format_settings, - download_threads); - - QueryPipelineBuilder builder; - builder.init(Pipe(input_format)); - - builder.addSimpleTransform( - [&](const Block & cur_header) - { return std::make_shared(cur_header, columns, *input_format, context); }); - - pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - reader = std::make_unique(*pipeline); - }; - } - - String getName() const override { return name; } - - Chunk generate() override - { - while (true) - { - if (isCancelled()) - { - if (reader) - reader->cancel(); - break; - } - - if (!reader) - { - auto current_uri_pos = uri_info->next_uri_to_read.fetch_add(1); - if (current_uri_pos >= uri_info->uri_list_to_read.size()) - return {}; - - auto current_uri = uri_info->uri_list_to_read[current_uri_pos]; - - initialize(current_uri); - } - - Chunk chunk; - if (reader->pull(chunk)) - { - UInt64 num_rows = chunk.getNumRows(); - if (num_rows && total_size) - updateRowsProgressApprox( - *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); - - return chunk; - } - - pipeline->reset(); - reader.reset(); - } + String next() + { + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= uris.size()) return {}; - } - static SeekableReadBufferFactoryPtr getFirstAvailableURLReadBuffer( - std::vector::const_iterator & option, - const std::vector::const_iterator & end, - ContextPtr context, - const URIParams & params, - const String & http_method, - std::function callback, - const ConnectionTimeouts & timeouts, - Poco::Net::HTTPBasicCredentials & credentials, - const HTTPHeaderEntries & headers, - bool glob_url, - bool delay_initialization) + return uris[current_index]; + } + + size_t size() + { + return uris.size(); + } + +private: + Strings uris; + std::atomic_size_t index = 0; +}; + +StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(ContextPtr context_, const String & uri) + : pimpl(std::make_shared(context_, uri)) {} + +String StorageURLSource::DisclosedGlobIterator::next() +{ + return pimpl->next(); +} + +size_t StorageURLSource::DisclosedGlobIterator::size() +{ + return pimpl->size(); +} + +void StorageURLSource::setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri) +{ + const auto & user_info = request_uri.getUserInfo(); + if (!user_info.empty()) + { + std::size_t n = user_info.find(':'); + if (n != std::string::npos) { - String first_exception_message; - ReadSettings read_settings = context->getReadSettings(); + credentials.setUsername(user_info.substr(0, n)); + credentials.setPassword(user_info.substr(n + 1)); + } + } +} - size_t options = std::distance(option, end); - for (; option != end; ++option) - { - bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); - auto request_uri = Poco::URI(*option); +StorageURLSource::StorageURLSource( + std::shared_ptr uri_iterator_, + const std::string & http_method, + std::function callback, + const String & format, + const std::optional & format_settings, + String name_, + const Block & sample_block, + ContextPtr context, + const ColumnsDescription & columns, + UInt64 max_block_size, + const ConnectionTimeouts & timeouts, + CompressionMethod compression_method, + size_t download_threads, + const HTTPHeaderEntries & headers_, + const URIParams & params, + bool glob_url) + : ISource(sample_block), name(std::move(name_)), uri_iterator(uri_iterator_) +{ + auto headers = getHeaders(headers_); - for (const auto & [param, value] : params) - request_uri.addQueryParameter(param, value); + /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. + initialize = [=, this](const FailoverOptions & uri_options) + { + if (uri_options.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); - setCredentials(credentials, request_uri); + auto first_option = uri_options.begin(); + auto buf_factory = getFirstAvailableURLReadBuffer( + first_option, + uri_options.end(), + context, + params, + http_method, + callback, + timeouts, + credentials, + headers, + glob_url, + uri_options.size() == 1); - const auto settings = context->getSettings(); - auto res = std::make_unique( - request_uri, - http_method, - callback, - timeouts, - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, - &context->getRemoteHostFilter(), - delay_initialization, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error); - - if (options > 1) - { - // Send a HEAD request to check availability. - try - { - res->getFileInfo(); - } - catch (...) - { - if (first_exception_message.empty()) - first_exception_message = getCurrentExceptionMessage(false); - - tryLogCurrentException(__PRETTY_FUNCTION__); - - continue; - } - } - - return res; - } - - throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message); + try + { + total_size += buf_factory->getFileSize(); + } + catch (...) + { + // we simply continue without total_size } - private: - using InitializeFunc = std::function; - InitializeFunc initialize; + // TODO: Pass max_parsing_threads and max_download_threads adjusted for num_streams. + auto input_format = FormatFactory::instance().getInputRandomAccess( + format, + std::move(buf_factory), + sample_block, + context, + max_block_size, + /* is_remote_fs */ true, + compression_method, + format_settings, + download_threads); - String name; - URIInfoPtr uri_info; + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); - std::unique_ptr pipeline; - std::unique_ptr reader; + builder.addSimpleTransform([&](const Block & cur_header) + { return std::make_shared(cur_header, columns, *input_format, context); }); - Poco::Net::HTTPBasicCredentials credentials; - - size_t total_size = 0; - UInt64 total_rows_approx_max = 0; - size_t total_rows_count_times = 0; - UInt64 total_rows_approx_accumulated = 0; + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + reader = std::make_unique(*pipeline); }; } +Chunk StorageURLSource::generate() +{ + while (true) + { + if (isCancelled()) + { + if (reader) + reader->cancel(); + break; + } + + if (!reader) + { + auto current_uri = (*uri_iterator)(); + if (current_uri.empty()) + return {}; + + initialize(current_uri); + } + + Chunk chunk; + if (reader->pull(chunk)) + { + UInt64 num_rows = chunk.getNumRows(); + if (num_rows && total_size) + updateRowsProgressApprox( + *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + + return chunk; + } + + pipeline->reset(); + reader.reset(); + } + return {}; +} + +SeekableReadBufferFactoryPtr StorageURLSource::getFirstAvailableURLReadBuffer( + std::vector::const_iterator & option, + const std::vector::const_iterator & end, + ContextPtr context, + const URIParams & params, + const String & http_method, + std::function callback, + const ConnectionTimeouts & timeouts, + Poco::Net::HTTPBasicCredentials & credentials, + const HTTPHeaderEntries & headers, + bool glob_url, + bool delay_initialization) +{ + String first_exception_message; + ReadSettings read_settings = context->getReadSettings(); + + size_t options = std::distance(option, end); + for (; option != end; ++option) + { + bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); + auto request_uri = Poco::URI(*option); + + for (const auto & [param, value] : params) + request_uri.addQueryParameter(param, value); + + setCredentials(credentials, request_uri); + + const auto settings = context->getSettings(); + auto res = std::make_unique( + request_uri, + http_method, + callback, + timeouts, + credentials, + settings.max_http_get_redirects, + settings.max_read_buffer_size, + read_settings, + headers, + &context->getRemoteHostFilter(), + delay_initialization, + /* use_external_buffer */ false, + /* skip_url_not_found_error */ skip_url_not_found_error); + + if (options > 1) + { + // Send a HEAD request to check availability. + try + { + res->getFileInfo(); + } + catch (...) + { + if (first_exception_message.empty()) + first_exception_message = getCurrentExceptionMessage(false); + + tryLogCurrentException(__PRETTY_FUNCTION__); + + continue; + } + } + + return res; + } + + throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message); +} + StorageURLSink::StorageURLSink( const String & uri, const String & format, @@ -621,51 +632,55 @@ Pipe IStorageURLBase::read( size_t max_download_threads = local_context->getSettingsRef().max_download_threads; - if (urlWithGlobs(uri)) + std::shared_ptr iterator_wrapper{nullptr}; + bool is_url_with_globs = urlWithGlobs(uri); + size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; + if (distributed_processing) { - size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; - auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); - - if (num_streams > uri_descriptions.size()) - num_streams = uri_descriptions.size(); - - /// For each uri (which acts like shard) check if it has failover options - auto uri_info = std::make_shared(); - for (const auto & description : uri_descriptions) - uri_info->uri_list_to_read.emplace_back(parseRemoteDescription(description, 0, description.size(), '|', max_addresses)); - - Pipes pipes; - pipes.reserve(num_streams); - - size_t download_threads = num_streams >= max_download_threads ? 1 : (max_download_threads / num_streams); - for (size_t i = 0; i < num_streams; ++i) + iterator_wrapper = std::make_shared( + [callback = local_context->getReadTaskCallback(), max_addresses]() + { + String next_uri = callback(); + if (next_uri.empty()) + return StorageURLSource::FailoverOptions{}; + return getFailoverOptions(next_uri, max_addresses); + }); + } + else if (is_url_with_globs) + { + /// Iterate through disclosed globs and make a source for each file + auto glob_iterator = std::make_shared(local_context, uri); + iterator_wrapper = std::make_shared([glob_iterator, max_addresses]() { - pipes.emplace_back(std::make_shared( - uri_info, - getReadMethod(), - getReadPOSTDataCallback(column_names, columns_description, query_info, local_context, processed_stage, max_block_size), - format_name, - format_settings, - getName(), - block_for_format, - local_context, - columns_description, - max_block_size, - getHTTPTimeouts(local_context), - compression_method, - download_threads, - headers, - params, - /* glob_url */ true)); - } - return Pipe::unitePipes(std::move(pipes)); + String next_uri = glob_iterator->next(); + if (next_uri.empty()) + return StorageURLSource::FailoverOptions{}; + return getFailoverOptions(next_uri, max_addresses); + }); + + if (num_streams > glob_iterator->size()) + num_streams = glob_iterator->size(); } else { - auto uri_info = std::make_shared(); - uri_info->uri_list_to_read.emplace_back(std::vector{uri}); - return Pipe(std::make_shared( - uri_info, + iterator_wrapper = std::make_shared([&, done = false]() mutable + { + if (done) + return StorageURLSource::FailoverOptions{}; + done = true; + return getFailoverOptions(uri, max_addresses); + }); + num_streams = 1; + } + + Pipes pipes; + pipes.reserve(num_streams); + + size_t download_threads = num_streams >= max_download_threads ? 1 : (max_download_threads / num_streams); + for (size_t i = 0; i < num_streams; ++i) + { + pipes.emplace_back(std::make_shared( + iterator_wrapper, getReadMethod(), getReadPOSTDataCallback(column_names, columns_description, query_info, local_context, processed_stage, max_block_size), format_name, @@ -677,10 +692,13 @@ Pipe IStorageURLBase::read( max_block_size, getHTTPTimeouts(local_context), compression_method, - max_download_threads, + download_threads, headers, - params)); + params, + is_url_with_globs)); } + + return Pipe::unitePipes(std::move(pipes)); } @@ -708,11 +726,16 @@ Pipe StorageURLWithFailover::read( auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size); - auto uri_info = std::make_shared(); - uri_info->uri_list_to_read.emplace_back(uri_options); + auto iterator_wrapper = std::make_shared([&, done = false]() mutable + { + if (done) + return StorageURLSource::FailoverOptions{}; + done = true; + return uri_options; + }); auto pipe = Pipe(std::make_shared( - uri_info, + iterator_wrapper, getReadMethod(), getReadPOSTDataCallback(column_names, columns_description, query_info, local_context, processed_stage, max_block_size), format_name, @@ -865,7 +888,8 @@ StorageURL::StorageURL( const String & compression_method_, const HTTPHeaderEntries & headers_, const String & http_method_, - ASTPtr partition_by_) + ASTPtr partition_by_, + bool distributed_processing_) : IStorageURLBase( uri_, context_, @@ -878,7 +902,8 @@ StorageURL::StorageURL( compression_method_, headers_, http_method_, - partition_by_) + partition_by_, + distributed_processing_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); } @@ -1092,4 +1117,4 @@ void registerStorageURL(StorageFactory & factory) .source_access_type = AccessType::URL, }); } -} \ No newline at end of file +} diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 9e3e3aa46c7..ad2952c2942 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -20,11 +21,12 @@ using OutputFormatPtr = std::shared_ptr; struct ConnectionTimeouts; class NamedCollection; +class PullingPipelineExecutor; /** * This class represents table engine for external urls. * It sends HTTP GET to server when select is called and - * HTTP POST when insert is called. In POST request the data is send + * HTTP POST when insert is called. In POST raequest the data is send * using Chunked transfer encoding, so server have to support it. */ class IStorageURLBase : public IStorage @@ -66,7 +68,8 @@ protected: const String & compression_method_, const HTTPHeaderEntries & headers_ = {}, const String & method_ = "", - ASTPtr partition_by = nullptr); + ASTPtr partition_by = nullptr, + bool distributed_processing_ = false); String uri; CompressionMethod compression_method; @@ -79,6 +82,7 @@ protected: HTTPHeaderEntries headers; String http_method; /// For insert can choose Put instead of default Post. ASTPtr partition_by; + bool distributed_processing; virtual std::string getReadMethod() const; @@ -125,6 +129,82 @@ private: const ContextPtr & context); }; + +class StorageURLSource : public ISource +{ + using URIParams = std::vector>; + +public: + class DisclosedGlobIterator + { + public: + DisclosedGlobIterator(ContextPtr context_, const String & uri_); + String next(); + size_t size(); + private: + class Impl; + /// shared_ptr to have copy constructor + std::shared_ptr pimpl; + }; + + using FailoverOptions = std::vector; + using IteratorWrapper = std::function; + + StorageURLSource( + std::shared_ptr uri_iterator_, + const std::string & http_method, + std::function callback, + const String & format, + const std::optional & format_settings, + String name_, + const Block & sample_block, + ContextPtr context, + const ColumnsDescription & columns, + UInt64 max_block_size, + const ConnectionTimeouts & timeouts, + CompressionMethod compression_method, + size_t download_threads, + const HTTPHeaderEntries & headers_ = {}, + const URIParams & params = {}, + bool glob_url = false); + + String getName() const override { return name; } + + Chunk generate() override; + + static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri); + + static SeekableReadBufferFactoryPtr getFirstAvailableURLReadBuffer( + std::vector::const_iterator & option, + const std::vector::const_iterator & end, + ContextPtr context, + const URIParams & params, + const String & http_method, + std::function callback, + const ConnectionTimeouts & timeouts, + Poco::Net::HTTPBasicCredentials & credentials, + const HTTPHeaderEntries & headers, + bool glob_url, + bool delay_initialization); + +private: + using InitializeFunc = std::function; + InitializeFunc initialize; + + String name; + std::shared_ptr uri_iterator; + + std::unique_ptr pipeline; + std::unique_ptr reader; + + Poco::Net::HTTPBasicCredentials credentials; + + size_t total_size = 0; + UInt64 total_rows_approx_max = 0; + size_t total_rows_count_times = 0; + UInt64 total_rows_approx_accumulated = 0; +}; + class StorageURLSink : public SinkToStorage { public: @@ -168,7 +248,8 @@ public: const String & compression_method_, const HTTPHeaderEntries & headers_ = {}, const String & method_ = "", - ASTPtr partition_by_ = nullptr); + ASTPtr partition_by_ = nullptr, + bool distributed_processing_ = false); String getName() const override { @@ -224,4 +305,4 @@ public: private: std::vector uri_options; }; -} \ No newline at end of file +} diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 116a08ce3a3..113d234a863 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -1,30 +1,24 @@ -#include "config.h" #include "Interpreters/Context_fwd.h" #include -#include #include #include -#include #include -#include #include -#include -#include +#include +#include #include #include #include -#include #include #include #include #include #include -#include #include @@ -43,12 +37,10 @@ StorageURLCluster::StorageURLCluster( const String & compression_method_, const StorageURL::Configuration &configuration_, bool structure_argument_was_provided_) - : IStorageCluster(table_id_) - , cluster_name(cluster_name_) + : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageURLCluster (" + table_id_.table_name + ")"), 3, structure_argument_was_provided_) , uri(uri_) , format_name(format_name_) , compression_method(compression_method_) - , structure_argument_was_provided(structure_argument_was_provided_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); @@ -71,82 +63,10 @@ StorageURLCluster::StorageURLCluster( setInMemoryMetadata(storage_metadata); } -/// The code executes on initiator -Pipe StorageURLCluster::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t /*max_block_size*/, - size_t /*num_streams*/) -{ - auto cluster = getCluster(context); - auto extension = getTaskIteratorExtension(query_info.query, context); - - /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) - Block header = - InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - - const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; - - Pipes pipes; - - const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; - - auto query_to_send = query_info.original_query->clone(); - if (!structure_argument_was_provided) - addColumnsStructureToQueryWithClusterEngine( - query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 3, getName()); - - const auto & current_settings = context->getSettingsRef(); - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - for (const auto & shard_info : cluster->getShardsInfo()) - { - auto try_results = shard_info.pool->getMany(timeouts, ¤t_settings, PoolMode::GET_MANY); - for (auto & try_result : try_results) - { - auto remote_query_executor = std::make_shared( - shard_info.pool, - std::vector{try_result}, - queryToString(query_to_send), - header, - context, - /*throttler=*/nullptr, - scalars, - Tables(), - processed_stage, - extension); - - pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false)); - } - } - - storage_snapshot->check(column_names); - return Pipe::unitePipes(std::move(pipes)); -} - -QueryProcessingStage::Enum StorageURLCluster::getQueryProcessingStage( - ContextPtr context, QueryProcessingStage::Enum to_stage, const StorageSnapshotPtr &, SelectQueryInfo &) const -{ - /// Initiator executes query on remote node. - if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) - if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) - return QueryProcessingStage::Enum::WithMergeableState; - - /// Follower just reads the data. - return QueryProcessingStage::Enum::FetchColumns; -} - -ClusterPtr StorageURLCluster::getCluster(ContextPtr context) const -{ - return context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); -} - RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(ASTPtr, ContextPtr context) const { auto iterator = std::make_shared(context, uri); - auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); + auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; } diff --git a/src/Storages/StorageURLCluster.h b/src/Storages/StorageURLCluster.h index b90af8df5b8..17436c47240 100644 --- a/src/Storages/StorageURLCluster.h +++ b/src/Storages/StorageURLCluster.h @@ -32,23 +32,14 @@ public: std::string getName() const override { return "URLCluster"; } - Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, - ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, size_t /*num_streams*/) override; - - QueryProcessingStage::Enum - getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; - NamesAndTypesList getVirtuals() const override; - ClusterPtr getCluster(ContextPtr context) const override; RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; private: - String cluster_name; String uri; String format_name; String compression_method; - bool structure_argument_was_provided; }; diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index ede0755efb0..805608564ae 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -62,10 +62,10 @@ void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, Context throw Exception::createDeprecated(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); /// This arguments are always the first - configuration.cluster_name = checkAndGetLiteralArgument(args[0], "cluster_name"); + cluster_name = checkAndGetLiteralArgument(args[0], "cluster_name"); - if (!context->tryGetCluster(configuration.cluster_name)) - throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", configuration.cluster_name); + if (!context->tryGetCluster(cluster_name)) + throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", cluster_name); /// Just cut the first arg (cluster_name) and try to parse s3 table function arguments as is ASTs clipped_args; @@ -121,6 +121,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl( else { storage = std::make_shared( + cluster_name, configuration, StorageID(getDatabaseName(), table_name), columns, diff --git a/src/TableFunctions/TableFunctionS3Cluster.h b/src/TableFunctions/TableFunctionS3Cluster.h index 17df8999470..d86f2164ce3 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.h +++ b/src/TableFunctions/TableFunctionS3Cluster.h @@ -50,7 +50,8 @@ protected: ColumnsDescription getActualTableStructure(ContextPtr) const override; void parseArguments(const ASTPtr &, ContextPtr) override; - mutable StorageS3Cluster::Configuration configuration; + String cluster_name; + mutable StorageS3::Configuration configuration; ColumnsDescription structure_hint; }; diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index f38897250a5..32455eefaea 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -12,7 +12,7 @@ class Context; /* url(source, format[, structure, compression]) - creates a temporary storage from url. */ -class TableFunctionURL final: public ITableFunctionFileLike +class TableFunctionURL : public ITableFunctionFileLike { public: static constexpr auto name = "url"; @@ -23,13 +23,14 @@ public: ColumnsDescription getActualTableStructure(ContextPtr context) const override; -private: - std::vector skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override; - +protected: void parseArguments(const ASTPtr & ast, ContextPtr context) override; StorageURL::Configuration configuration; +private: + std::vector skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override; + StoragePtr getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const String & compression_method_) const override; diff --git a/src/TableFunctions/TableFunctionURLCluster.cpp b/src/TableFunctions/TableFunctionURLCluster.cpp index 00aadab5e01..adaa3492b9d 100644 --- a/src/TableFunctions/TableFunctionURLCluster.cpp +++ b/src/TableFunctions/TableFunctionURLCluster.cpp @@ -31,20 +31,19 @@ void TableFunctionURLCluster::parseArguments(const ASTPtr & ast_function, Contex ASTs & args_func = ast_copy->children; if (args_func.size() != 1) - throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function {} must have arguments", getName()); ASTs & args = args_func.at(0)->children; - const auto message = fmt::format( - "The signature of table function {} shall be the following:\n" \ - " - cluster, uri\n",\ - " - cluster, uri, format\n",\ - " - cluster, uri, format, structure\n",\ - " - cluster, uri, format, structure, compression_method", - getName()); - if (args.size() < 2 || args.size() > 5) - throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "The signature of table function {} shall be the following:\n" + " - cluster, uri\n" + " - cluster, uri, format\n" + " - cluster, uri, format, structure\n" + " - cluster, uri, format, structure, compression_method", + getName()); for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); @@ -98,7 +97,9 @@ StoragePtr TableFunctionURLCluster::getStorage( context, compression_method, configuration.headers, - configuration.http_method); + configuration.http_method, + nullptr, + /*distributed_processing=*/ true); } else { diff --git a/src/TableFunctions/TableFunctionURLCluster.h b/src/TableFunctions/TableFunctionURLCluster.h index 7deda443b05..e7935e9a9be 100644 --- a/src/TableFunctions/TableFunctionURLCluster.h +++ b/src/TableFunctions/TableFunctionURLCluster.h @@ -28,7 +28,6 @@ public: { return name; } - bool hasStaticStructure() const override { return true; } protected: StoragePtr getStorage( diff --git a/tests/queries/0_stateless/02721_url_cluster.reference b/tests/queries/0_stateless/02721_url_cluster.reference new file mode 100644 index 00000000000..4345eb9c47b --- /dev/null +++ b/tests/queries/0_stateless/02721_url_cluster.reference @@ -0,0 +1,88 @@ +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 +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 UInt64 +c2 UInt64 +c3 UInt64 +c1 UInt64 +c2 UInt64 +c3 UInt64 +12 +12 +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +1 2 3 +4 5 6 +7 8 9 +0 0 0 diff --git a/tests/queries/0_stateless/02721_url_cluster.sql b/tests/queries/0_stateless/02721_url_cluster.sql new file mode 100644 index 00000000000..0d5db7949db --- /dev/null +++ b/tests/queries/0_stateless/02721_url_cluster.sql @@ -0,0 +1,29 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: Depends on AWS + + +select * from urlCluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv') ORDER BY c1, c2, c3; +select * from urlCluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV') ORDER BY c1, c2, c3; +select * from urlCluster('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 urlCluster('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; + +desc urlCluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv'); +desc urlCluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV'); +desc urlCluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64'); +desc urlCluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto'); + +select COUNT() from urlCluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv'); +select COUNT(*) from urlCluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv'); + +desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv'); +desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV'); + +select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv') order by c1, c2, c3; +select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV') order by c1, c2, c3; + +drop table if exists test; +create table test (x UInt32, y UInt32, z UInt32) engine=Memory(); +insert into test select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/a.tsv'); +select * from test; +drop table test; + From 6277e10f29055a1cfa9c11c3e6928d88ca0d34bd Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Apr 2023 17:26:42 +0000 Subject: [PATCH 0108/2223] Remove some trash --- src/Storages/StorageClusterBase.cpp | 0 src/Storages/StorageClusterBase.h | 50 ----------------------------- 2 files changed, 50 deletions(-) delete mode 100644 src/Storages/StorageClusterBase.cpp delete mode 100644 src/Storages/StorageClusterBase.h diff --git a/src/Storages/StorageClusterBase.cpp b/src/Storages/StorageClusterBase.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/src/Storages/StorageClusterBase.h b/src/Storages/StorageClusterBase.h deleted file mode 100644 index da048b635d8..00000000000 --- a/src/Storages/StorageClusterBase.h +++ /dev/null @@ -1,50 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include - -#include "Client/Connection.h" -#include -#include -#include -#include - -namespace DB -{ - -class Context; - -class StorageClusterBase : public IStorageCluster -{ -public: - StorageClusterBase( - String cluster_name, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - ContextPtr context_, - bool structure_argument_was_provided_); - - Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, - ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, size_t /*num_streams*/) override; - - QueryProcessingStage::Enum - getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; - - RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; - ClusterPtr getCluster(ContextPtr context) const override; - -private: - Poco::Logger * log; - String cluster_name; - bool structure_argument_was_provided; -}; - - -} - -#endif From 0805b517ee82952586a4b85dc226766dc83155db Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Apr 2023 17:28:14 +0000 Subject: [PATCH 0109/2223] Fix parsing failover options --- src/Storages/StorageURL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 43281432dcb..d0475c243f6 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -152,7 +152,7 @@ namespace StorageURLSource::FailoverOptions getFailoverOptions(const String & uri, size_t max_addresses) { - return parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); + return parseRemoteDescription(uri, 0, uri.size(), '|', max_addresses); } } From 0097230611b0fc9bea09833e60252338734ce544 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Apr 2023 17:35:17 +0000 Subject: [PATCH 0110/2223] Better --- src/Storages/StorageURL.cpp | 11 +++++------ src/Storages/StorageURL.h | 2 +- src/Storages/StorageURLCluster.cpp | 2 +- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index d0475c243f6..f0401c70501 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -159,10 +159,9 @@ namespace class StorageURLSource::DisclosedGlobIterator::Impl { public: - Impl(ContextPtr context, const String & uri) + Impl(const String & uri, size_t max_addresses) { - context->getRemoteHostFilter().checkURL(Poco::URI(uri)); - uris = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements); + uris = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); } String next() @@ -184,8 +183,8 @@ private: std::atomic_size_t index = 0; }; -StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(ContextPtr context_, const String & uri) - : pimpl(std::make_shared(context_, uri)) {} +StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, size_t max_addresses) + : pimpl(std::make_shared(uri, max_addresses)) {} String StorageURLSource::DisclosedGlobIterator::next() { @@ -649,7 +648,7 @@ Pipe IStorageURLBase::read( else if (is_url_with_globs) { /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(local_context, uri); + auto glob_iterator = std::make_shared(uri, max_addresses); iterator_wrapper = std::make_shared([glob_iterator, max_addresses]() { String next_uri = glob_iterator->next(); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index ad2952c2942..30bf18bc98f 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -138,7 +138,7 @@ public: class DisclosedGlobIterator { public: - DisclosedGlobIterator(ContextPtr context_, const String & uri_); + DisclosedGlobIterator(const String & uri_, size_t max_addresses); String next(); size_t size(); private: diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 113d234a863..c3501c9e676 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -65,7 +65,7 @@ StorageURLCluster::StorageURLCluster( RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(ASTPtr, ContextPtr context) const { - auto iterator = std::make_shared(context, uri); + auto iterator = std::make_shared(uri, context->getSettingsRef().glob_expansion_max_elements); auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; } From 447189a6ca177d5faa2f6bfc60bb2eabf85a3335 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Apr 2023 17:54:09 +0000 Subject: [PATCH 0111/2223] Better --- src/Storages/HDFS/StorageHDFSCluster.cpp | 3 ++- src/Storages/HDFS/StorageHDFSCluster.h | 1 + src/Storages/StorageS3Cluster.cpp | 3 ++- src/Storages/StorageS3Cluster.h | 1 + src/Storages/StorageURL.h | 2 +- src/Storages/StorageURLCluster.cpp | 20 ++++++++----------- src/Storages/StorageURLCluster.h | 8 ++------ .../TableFunctionHDFSCluster.cpp | 13 ++++++++---- src/TableFunctions/TableFunctionS3Cluster.cpp | 1 + src/TableFunctions/TableFunctionS3Cluster.h | 2 +- .../TableFunctionURLCluster.cpp | 10 +++++++--- 11 files changed, 35 insertions(+), 29 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index abe24545237..52124720ff6 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -35,8 +35,9 @@ StorageHDFSCluster::StorageHDFSCluster( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & compression_method_, + size_t table_function_max_arguments, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageHDFSCluster (" + table_id_.table_name + ")"), 3, structure_argument_was_provided_) + : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageHDFSCluster (" + table_id_.table_name + ")"), table_function_max_arguments, structure_argument_was_provided_) , uri(uri_) , format_name(format_name_) , compression_method(compression_method_) diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h index 9a30d85a37d..6a73b0ac4c7 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -29,6 +29,7 @@ public: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & compression_method_, + size_t table_function_max_arguments, bool structure_argument_was_provided_); std::string getName() const override { return "HDFSCluster"; } diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index d0b0941ac9d..ecea386f922 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -30,8 +30,9 @@ StorageS3Cluster::StorageS3Cluster( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, ContextPtr context_, + size_t table_function_max_arguments, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageS3Cluster (" + table_id_.table_name + ")"), 5, structure_argument_was_provided_) + : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageS3Cluster (" + table_id_.table_name + ")"), table_function_max_arguments, structure_argument_was_provided_) , s3_configuration{configuration_} { context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri); diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index 96bb4cad679..e4f5e26ae05 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -28,6 +28,7 @@ public: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, ContextPtr context_, + size_t table_function_max_arguments, bool structure_argument_was_provided_); std::string getName() const override { return "S3Cluster"; } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 30bf18bc98f..a7ad3d82dd5 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -26,7 +26,7 @@ class PullingPipelineExecutor; /** * This class represents table engine for external urls. * It sends HTTP GET to server when select is called and - * HTTP POST when insert is called. In POST raequest the data is send + * HTTP POST when insert is called. In POST request the data is send * using Chunked transfer encoding, so server have to support it. */ class IStorageURLBase : public IStorage diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index c3501c9e676..b10e4d94e7a 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -29,28 +29,24 @@ namespace DB StorageURLCluster::StorageURLCluster( ContextPtr context_, String cluster_name_, - const String & uri_, const StorageID & table_id_, - const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - const String & compression_method_, - const StorageURL::Configuration &configuration_, + const StorageURL::Configuration & configuration_, + size_t table_function_max_arguments, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageURLCluster (" + table_id_.table_name + ")"), 3, structure_argument_was_provided_) - , uri(uri_) - , format_name(format_name_) - , compression_method(compression_method_) + : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageURLCluster (" + table_id_.table_name + ")"), table_function_max_arguments, structure_argument_was_provided_) + , uri(configuration_.url) { - context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); + context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); StorageInMemoryMetadata storage_metadata; if (columns_.empty()) { - auto columns = StorageURL::getTableStructureFromData(format_name_, - uri_, - chooseCompressionMethod(Poco::URI(uri_).getPath(), compression_method), + auto columns = StorageURL::getTableStructureFromData(configuration_.format, + uri, + chooseCompressionMethod(Poco::URI(uri).getPath(), configuration_.compression_method), configuration_.headers, std::nullopt, context_); diff --git a/src/Storages/StorageURLCluster.h b/src/Storages/StorageURLCluster.h index 17436c47240..0d5dcc2c930 100644 --- a/src/Storages/StorageURLCluster.h +++ b/src/Storages/StorageURLCluster.h @@ -21,13 +21,11 @@ public: StorageURLCluster( ContextPtr context_, String cluster_name_, - const String & uri_, const StorageID & table_id_, - const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - const String & compression_method_, - const StorageURL::Configuration &configuration_, + const StorageURL::Configuration & configuration_, + size_t table_function_max_arguments, bool structure_argument_was_provided_); std::string getName() const override { return "URLCluster"; } @@ -38,8 +36,6 @@ public: private: String uri; - String format_name; - String compression_method; }; diff --git a/src/TableFunctions/TableFunctionHDFSCluster.cpp b/src/TableFunctions/TableFunctionHDFSCluster.cpp index 7c84a281673..b50509ee8bf 100644 --- a/src/TableFunctions/TableFunctionHDFSCluster.cpp +++ b/src/TableFunctions/TableFunctionHDFSCluster.cpp @@ -106,9 +106,15 @@ StoragePtr TableFunctionHDFSCluster::getStorage( { storage = std::make_shared( context, - cluster_name, filename, StorageID(getDatabaseName(), table_name), - format, columns, ConstraintsDescription{}, - compression_method, structure != "auto"); + cluster_name, + filename, + StorageID(getDatabaseName(), table_name), + format, + columns, + ConstraintsDescription{}, + compression_method, + 5, + structure != "auto"); } return storage; } @@ -118,7 +124,6 @@ void registerTableFunctionHDFSCluster(TableFunctionFactory & factory) factory.registerFunction(); } - } #endif diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 805608564ae..085028c2145 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -127,6 +127,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl( columns, ConstraintsDescription{}, context, + 7, structure_argument_was_provided); } diff --git a/src/TableFunctions/TableFunctionS3Cluster.h b/src/TableFunctions/TableFunctionS3Cluster.h index d86f2164ce3..2dd0e4b6659 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.h +++ b/src/TableFunctions/TableFunctionS3Cluster.h @@ -14,7 +14,7 @@ namespace DB class Context; /** - * s3cluster(cluster_name, source, [access_key_id, secret_access_key,] format, structure) + * s3cluster(cluster_name, source, [access_key_id, secret_access_key,] format, structure, compression_method) * A table function, which allows to process many files from S3 on a specific cluster * On initiator it creates a connection to _all_ nodes in cluster, discloses asterics * in S3 file path and dispatch each file dynamically. diff --git a/src/TableFunctions/TableFunctionURLCluster.cpp b/src/TableFunctions/TableFunctionURLCluster.cpp index adaa3492b9d..03af08c7de9 100644 --- a/src/TableFunctions/TableFunctionURLCluster.cpp +++ b/src/TableFunctions/TableFunctionURLCluster.cpp @@ -105,9 +105,13 @@ StoragePtr TableFunctionURLCluster::getStorage( { storage = std::make_shared( context, - cluster_name, filename, StorageID(getDatabaseName(), table_name), - format, getActualTableStructure(context), ConstraintsDescription{}, - compression_method, configuration, structure != "auto"); + cluster_name, + StorageID(getDatabaseName(), table_name), + getActualTableStructure(context), + ConstraintsDescription{}, + configuration, + 5, + structure != "auto"); } return storage; } From 34d82625b58e157d937d48a9660079d63deab785 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Sun, 23 Apr 2023 02:58:00 +0800 Subject: [PATCH 0112/2223] feat: add dotProduct for array --- src/Functions/array/arrayAUC.cpp | 2 +- src/Functions/array/arrayDotProduct.cpp | 75 ++++++++++++++++++++++++ src/Functions/array/arrayScalarProduct.h | 71 +++++++++++++--------- src/Functions/vectorFunctions.cpp | 16 ++++- 4 files changed, 133 insertions(+), 31 deletions(-) create mode 100644 src/Functions/array/arrayDotProduct.cpp diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 2890ae55886..297394822d9 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -91,7 +91,7 @@ public: return std::make_shared>(); } - template + template static ResultType apply( const T * scores, const U * labels, diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp new file mode 100644 index 00000000000..4b9433f683d --- /dev/null +++ b/src/Functions/array/arrayDotProduct.cpp @@ -0,0 +1,75 @@ +#include +#include +#include +#include "arrayScalarProduct.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +struct NameArrayDotProduct +{ + static constexpr auto name = "arrayDotProduct"; +}; + + +class ArrayDotProductImpl +{ +public: + + static DataTypePtr getReturnType(const DataTypePtr left_type, const DataTypePtr & right_type) + { + const auto & common_type = getLeastSupertype(DataTypes{left_type, right_type}); + switch (common_type->getTypeId()) + { + case TypeIndex::UInt8: + case TypeIndex::UInt16: + case TypeIndex::UInt32: + case TypeIndex::Int8: + case TypeIndex::Int16: + case TypeIndex::Int32: + case TypeIndex::UInt64: + case TypeIndex::Int64: + case TypeIndex::Float64: + return std::make_shared(); + case TypeIndex::Float32: + return std::make_shared(); + default: + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Arguments of function {} has nested type {}. " + "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + std::string(NameArrayDotProduct::name), + common_type->getName()); + } + } + + template + static ResultType apply( + const T * left, + const U * right, + size_t size) + { + ResultType result = 0; + for (size_t i = 0; i < size; ++i) + result += static_cast(left[i]) * static_cast(right[i]); + return result; + } + +}; + +using FunctionArrayDotProduct = FunctionArrayScalarProduct; + +REGISTER_FUNCTION(ArrayDotProduct) +{ + factory.registerFunction(); +} + +/// These functions are used by TupleOrArrayFunction in Function/vectorFunctions.cpp +FunctionPtr createFunctionArrayDotProduct(ContextPtr context_) { return FunctionArrayDotProduct::create(context_); } +} diff --git a/src/Functions/array/arrayScalarProduct.h b/src/Functions/array/arrayScalarProduct.h index 94ce1bc533c..ded6ec8ae29 100644 --- a/src/Functions/array/arrayScalarProduct.h +++ b/src/Functions/array/arrayScalarProduct.h @@ -29,29 +29,28 @@ public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } private: - using ResultColumnType = ColumnVector; - template + template ColumnPtr executeNumber(const ColumnsWithTypeAndName & arguments) const { ColumnPtr res; - if ( (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments)) - || (res = executeNumberNumber(arguments))) + if ( (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments)) + || (res = executeNumberNumber(arguments))) return res; return nullptr; } - template + template ColumnPtr executeNumberNumber(const ColumnsWithTypeAndName & arguments) const { ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst(); @@ -72,7 +71,7 @@ private: if (!col_nested1 || !col_nested2) return nullptr; - auto col_res = ResultColumnType::create(); + auto col_res = ColumnVector::create(); vector( col_nested1->getData(), @@ -83,12 +82,12 @@ private: return col_res; } - template + template static NO_INLINE void vector( const PaddedPODArray & data1, const PaddedPODArray & data2, const ColumnArray::Offsets & offsets, - PaddedPODArray & result) + PaddedPODArray & result) { size_t size = offsets.size(); result.resize(size); @@ -97,7 +96,7 @@ private: for (size_t i = 0; i < size; ++i) { size_t array_size = offsets[i] - current_offset; - result[i] = Method::apply(&data1[current_offset], &data2[current_offset], array_size); + result[i] = Method::template apply(&data1[current_offset], &data2[current_offset], array_size); current_offset = offsets[i]; } } @@ -130,24 +129,40 @@ public: return Method::getReturnType(nested_types[0], nested_types[1]); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /* input_rows_count */) const override + template + ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const { ColumnPtr res; - if (!((res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)) - || (res = executeNumber(arguments)))) + if ( !((res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)) + || (res = executeNumber(arguments)))) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); return res; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override + { + switch (result_type->getTypeId()) + { + case TypeIndex::Float32: + return executeWithResultType(arguments); + break; + case TypeIndex::Float64: + return executeWithResultType(arguments); + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type {}", result_type->getName()); + } + } }; } diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index a7ab09612cf..b52def28755 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -1429,6 +1429,8 @@ private: FunctionPtr array_function; }; +extern FunctionPtr createFunctionArrayDotProduct(ContextPtr context_); + extern FunctionPtr createFunctionArrayL1Norm(ContextPtr context_); extern FunctionPtr createFunctionArrayL2Norm(ContextPtr context_); extern FunctionPtr createFunctionArrayL2SquaredNorm(ContextPtr context_); @@ -1442,6 +1444,14 @@ extern FunctionPtr createFunctionArrayLpDistance(ContextPtr context_); extern FunctionPtr createFunctionArrayLinfDistance(ContextPtr context_); extern FunctionPtr createFunctionArrayCosineDistance(ContextPtr context_); +struct DotProduct +{ + static constexpr auto name = "dotProduct"; + + static constexpr auto CreateTupleFunction = FunctionDotProduct::create; + static constexpr auto CreateArrayFunction = createFunctionArrayDotProduct; +}; + struct L1NormTraits { static constexpr auto name = "L1Norm"; @@ -1530,6 +1540,8 @@ struct CosineDistanceTraits static constexpr auto CreateArrayFunction = createFunctionArrayCosineDistance; }; +using TupleOrArrayFunctionDotProduct = TupleOrArrayFunction; + using TupleOrArrayFunctionL1Norm = TupleOrArrayFunction; using TupleOrArrayFunctionL2Norm = TupleOrArrayFunction; using TupleOrArrayFunctionL2SquaredNorm = TupleOrArrayFunction; @@ -1615,8 +1627,8 @@ If the types of the first interval (or the interval in the tuple) and the second factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerAlias("scalarProduct", FunctionDotProduct::name, FunctionFactory::CaseInsensitive); + factory.registerFunction(); + factory.registerAlias("scalarProduct", TupleOrArrayFunctionDotProduct::name, FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); From 81fa4701aaa00974aeb8173c9847dfbe220c2a27 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Sun, 23 Apr 2023 03:23:14 +0800 Subject: [PATCH 0113/2223] feat: add dotProduct for array --- .../0_stateless/02708_dot_product.reference | 10 ++++++ .../queries/0_stateless/02708_dot_product.sql | 34 +++++++++++++++++++ 2 files changed, 44 insertions(+) create mode 100644 tests/queries/0_stateless/02708_dot_product.reference create mode 100644 tests/queries/0_stateless/02708_dot_product.sql diff --git a/tests/queries/0_stateless/02708_dot_product.reference b/tests/queries/0_stateless/02708_dot_product.reference new file mode 100644 index 00000000000..7106b870fab --- /dev/null +++ b/tests/queries/0_stateless/02708_dot_product.reference @@ -0,0 +1,10 @@ +3881.304 +3881.304 +3881.304 +376.5 +230 +0 +Float64 +Float32 +Float64 +Float64 diff --git a/tests/queries/0_stateless/02708_dot_product.sql b/tests/queries/0_stateless/02708_dot_product.sql new file mode 100644 index 00000000000..46450ae6394 --- /dev/null +++ b/tests/queries/0_stateless/02708_dot_product.sql @@ -0,0 +1,34 @@ +SELECT dotProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); +SELECT scalarProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); +SELECT arrayDotProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); + +SELECT dotProduct([1.3, 2, 3, 4, 5], [222, 12, 5.3, 2, 8]); + +SELECT dotProduct([1, 1, 1, 1, 1], [222, 12, 0, -12, 8]); + +SELECT round(dotProduct([-1, 2, 3.002], [2, 3.4, 4]) - dotProduct((-1, 2, 3.002), (2, 3.4, 4)), 2); + + +DROP TABLE IF EXISTS product_fp64_fp64; +CREATE TABLE product_fp64_fp64 (x Array(Float64), y Array(Float64)) engine = MergeTree() order by x; +INSERT INTO TABLE product_fp64_fp64 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_fp64_fp64; +DROP TABLE product_fp64_fp64; + +DROP TABLE IF EXISTS product_fp32_fp32; +CREATE TABLE product_fp32_fp32 (x Array(Float32), y Array(Float32)) engine = MergeTree() order by x; +INSERT INTO TABLE product_fp32_fp32 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_fp32_fp32; +DROP TABLE product_fp32_fp32; + +DROP TABLE IF EXISTS product_fp32_fp64; +CREATE TABLE product_fp32_fp64 (x Array(Float32), y Array(Float64)) engine = MergeTree() order by x; +INSERT INTO TABLE product_fp32_fp64 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_fp32_fp64; +DROP TABLE product_fp32_fp64; + +DROP TABLE IF EXISTS product_uint8_fp64; +CREATE TABLE product_uint8_fp64 (x Array(UInt8), y Array(Float64)) engine = MergeTree() order by x; +INSERT INTO TABLE product_uint8_fp64 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_uint8_fp64; +DROP TABLE product_uint8_fp64; From 79ae949b566e5023500d037db6327a050dc7a11f Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Sun, 23 Apr 2023 03:35:09 +0800 Subject: [PATCH 0114/2223] fix style --- src/Functions/array/arrayDotProduct.cpp | 7 ++----- src/Functions/array/arrayScalarProduct.h | 2 +- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 4b9433f683d..0d21bae90e3 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -17,12 +17,10 @@ struct NameArrayDotProduct static constexpr auto name = "arrayDotProduct"; }; - class ArrayDotProductImpl { public: - - static DataTypePtr getReturnType(const DataTypePtr left_type, const DataTypePtr & right_type) + static DataTypePtr getReturnType(const DataTypePtr & left_type, const DataTypePtr & right_type) { const auto & common_type = getLeastSupertype(DataTypes{left_type, right_type}); switch (common_type->getTypeId()) @@ -56,11 +54,10 @@ public: size_t size) { ResultType result = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < size; ++i) result += static_cast(left[i]) * static_cast(right[i]); return result; } - }; using FunctionArrayDotProduct = FunctionArrayScalarProduct; diff --git a/src/Functions/array/arrayScalarProduct.h b/src/Functions/array/arrayScalarProduct.h index ded6ec8ae29..0d1bf44a3e7 100644 --- a/src/Functions/array/arrayScalarProduct.h +++ b/src/Functions/array/arrayScalarProduct.h @@ -133,7 +133,7 @@ public: ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const { ColumnPtr res; - if ( !((res = executeNumber(arguments)) + if (!((res = executeNumber(arguments)) || (res = executeNumber(arguments)) || (res = executeNumber(arguments)) || (res = executeNumber(arguments)) From 4145abf547c3f09b849aa8e3429a318a340ef668 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Sun, 23 Apr 2023 04:04:05 +0800 Subject: [PATCH 0115/2223] fix style --- src/Functions/array/arrayScalarProduct.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/array/arrayScalarProduct.h b/src/Functions/array/arrayScalarProduct.h index 0d1bf44a3e7..5c36f2492c6 100644 --- a/src/Functions/array/arrayScalarProduct.h +++ b/src/Functions/array/arrayScalarProduct.h @@ -18,6 +18,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } From 2949ceced1e931367478cf474aa61c23fa6b19a9 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 24 Apr 2023 13:20:04 +0000 Subject: [PATCH 0116/2223] Fix adding structure to cluster table functions, make it better --- src/Storages/HDFS/StorageHDFSCluster.cpp | 12 +- src/Storages/HDFS/StorageHDFSCluster.h | 5 +- src/Storages/IStorageCluster.cpp | 8 +- src/Storages/IStorageCluster.h | 6 +- src/Storages/StorageS3Cluster.cpp | 9 +- src/Storages/StorageS3Cluster.h | 3 +- src/Storages/StorageURLCluster.cpp | 20 ++- src/Storages/StorageURLCluster.h | 11 +- ...lumnsStructureToQueryWithClusterEngine.cpp | 139 +++++++++++++++++- ...ColumnsStructureToQueryWithClusterEngine.h | 6 +- .../TableFunctionHDFSCluster.cpp | 1 - src/TableFunctions/TableFunctionS3Cluster.cpp | 1 - .../TableFunctionURLCluster.cpp | 4 +- ...2457_s3_cluster_schema_inference.reference | 66 +++++++++ .../02457_s3_cluster_schema_inference.sql | 14 ++ .../0_stateless/02721_url_cluster.reference | 33 +++++ .../queries/0_stateless/02721_url_cluster.sql | 6 + 17 files changed, 305 insertions(+), 39 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 52124720ff6..f881783ad1c 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include @@ -28,16 +29,15 @@ namespace DB StorageHDFSCluster::StorageHDFSCluster( ContextPtr context_, - String cluster_name_, + const String & cluster_name_, const String & uri_, const StorageID & table_id_, const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & compression_method_, - size_t table_function_max_arguments, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageHDFSCluster (" + table_id_.table_name + ")"), table_function_max_arguments, structure_argument_was_provided_) + : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageHDFSCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) , uri(uri_) , format_name(format_name_) , compression_method(compression_method_) @@ -59,6 +59,12 @@ StorageHDFSCluster::StorageHDFSCluster( setInMemoryMetadata(storage_metadata); } +void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure) +{ + addColumnsStructureToQueryWithHDFSClusterEngine(query, structure); +} + + RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(ASTPtr, ContextPtr context) const { auto iterator = std::make_shared(context, uri); diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h index 6a73b0ac4c7..a0340b2d03c 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -22,14 +22,13 @@ class StorageHDFSCluster : public IStorageCluster public: StorageHDFSCluster( ContextPtr context_, - String cluster_name_, + const String & cluster_name_, const String & uri_, const StorageID & table_id_, const String & format_name_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & compression_method_, - size_t table_function_max_arguments, bool structure_argument_was_provided_); std::string getName() const override { return "HDFSCluster"; } @@ -39,6 +38,8 @@ public: RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; private: + void addColumnsStructureToQuery(ASTPtr & query, const String & structure) override; + String uri; String format_name; String compression_method; diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index faf0e107c89..f736d672910 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include @@ -28,15 +27,13 @@ namespace DB { IStorageCluster::IStorageCluster( - String cluster_name_, + const String & cluster_name_, const StorageID & table_id_, Poco::Logger * log_, - size_t max_function_arguments_, bool structure_argument_was_provided_) : IStorage(table_id_) , log(log_) , cluster_name(cluster_name_) - , max_function_arguments(max_function_arguments_) , structure_argument_was_provided(structure_argument_was_provided_) { } @@ -80,8 +77,7 @@ Pipe IStorageCluster::read( const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; if (!structure_argument_was_provided) - addColumnsStructureToQueryWithClusterEngine( - query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), max_function_arguments, getName()); + addColumnsStructureToQuery(query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll())); RestoreQualifiedNamesVisitor::Data data; data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query_info.query->as(), 0)); diff --git a/src/Storages/IStorageCluster.h b/src/Storages/IStorageCluster.h index e3ee6363974..c5ee567ff69 100644 --- a/src/Storages/IStorageCluster.h +++ b/src/Storages/IStorageCluster.h @@ -16,10 +16,9 @@ class IStorageCluster : public IStorage { public: IStorageCluster( - String cluster_name_, + const String & cluster_name_, const StorageID & table_id_, Poco::Logger * log_, - size_t max_function_arguments_, bool structure_argument_was_provided_); Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, @@ -36,10 +35,11 @@ public: protected: virtual void updateBeforeRead(const ContextPtr &) {} + virtual void addColumnsStructureToQuery(ASTPtr & query, const String & structure) = 0; + private: Poco::Logger * log; String cluster_name; - size_t max_function_arguments; bool structure_argument_was_provided; }; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index ecea386f922..1a7b936d7ee 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -30,9 +31,8 @@ StorageS3Cluster::StorageS3Cluster( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, ContextPtr context_, - size_t table_function_max_arguments, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageS3Cluster (" + table_id_.table_name + ")"), table_function_max_arguments, structure_argument_was_provided_) + : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageS3Cluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) , s3_configuration{configuration_} { context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri); @@ -61,6 +61,11 @@ StorageS3Cluster::StorageS3Cluster( virtual_block.insert({column.type->createColumn(), column.type, column.name}); } +void StorageS3Cluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure) +{ + addColumnsStructureToQueryWithS3ClusterEngine(query, structure); +} + void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context) { s3_configuration.update(local_context); diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index e4f5e26ae05..e0f10e038da 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -28,7 +28,6 @@ public: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, ContextPtr context_, - size_t table_function_max_arguments, bool structure_argument_was_provided_); std::string getName() const override { return "S3Cluster"; } @@ -43,6 +42,8 @@ protected: private: void updateBeforeRead(const ContextPtr & context) override { updateConfigurationIfChanged(context); } + void addColumnsStructureToQuery(ASTPtr & query, const String & structure) override; + StorageS3::Configuration s3_configuration; NamesAndTypesList virtual_columns; Block virtual_block; diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index b10e4d94e7a..115d29aad6d 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include @@ -28,15 +29,17 @@ namespace DB StorageURLCluster::StorageURLCluster( ContextPtr context_, - String cluster_name_, + const String & cluster_name_, + const String & uri_, + const String & format_, + const String & compression_method_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const StorageURL::Configuration & configuration_, - size_t table_function_max_arguments, bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageURLCluster (" + table_id_.table_name + ")"), table_function_max_arguments, structure_argument_was_provided_) - , uri(configuration_.url) + : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageURLCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) + , uri(uri_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); @@ -44,9 +47,9 @@ StorageURLCluster::StorageURLCluster( if (columns_.empty()) { - auto columns = StorageURL::getTableStructureFromData(configuration_.format, + auto columns = StorageURL::getTableStructureFromData(format_, uri, - chooseCompressionMethod(Poco::URI(uri).getPath(), configuration_.compression_method), + chooseCompressionMethod(Poco::URI(uri).getPath(), compression_method_), configuration_.headers, std::nullopt, context_); @@ -59,6 +62,11 @@ StorageURLCluster::StorageURLCluster( setInMemoryMetadata(storage_metadata); } +void StorageURLCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure) +{ + addColumnsStructureToQueryWithURLClusterEngine(query, structure); +} + RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(ASTPtr, ContextPtr context) const { auto iterator = std::make_shared(uri, context->getSettingsRef().glob_expansion_max_elements); diff --git a/src/Storages/StorageURLCluster.h b/src/Storages/StorageURLCluster.h index 0d5dcc2c930..727a99f17f9 100644 --- a/src/Storages/StorageURLCluster.h +++ b/src/Storages/StorageURLCluster.h @@ -20,12 +20,14 @@ class StorageURLCluster : public IStorageCluster public: StorageURLCluster( ContextPtr context_, - String cluster_name_, + const String & cluster_name_, + const String & uri_, + const String & format_, + const String & compression_method_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const StorageURL::Configuration & configuration_, - size_t table_function_max_arguments, bool structure_argument_was_provided_); std::string getName() const override { return "URLCluster"; } @@ -35,8 +37,11 @@ public: RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; private: - String uri; + void addColumnsStructureToQuery(ASTPtr & query, const String & structure) override; + String uri; + String format_name; + String compression_method; }; diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp index 75c5bebb901..cb8b7dcea59 100644 --- a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp +++ b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp @@ -1,10 +1,14 @@ #include +#include #include #include #include #include #include #include +#include +#include + namespace DB { @@ -29,24 +33,143 @@ static ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & return table_function->arguments->as(); } -void addColumnsStructureToQueryWithClusterEngine(ASTPtr & query, const String & structure, size_t max_arguments, const String & function_name) +static ASTExpressionList * getExpressionListAndCheckArguments(ASTPtr & query, size_t max_arguments, const String & function_name) { ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); if (!expression_list) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function {}, got '{}'", function_name, queryToString(query)); - auto structure_literal = std::make_shared(structure); if (expression_list->children.size() < 2 || expression_list->children.size() > max_arguments) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 2 to {} arguments in {} table functions, got {}", - function_name, max_arguments, expression_list->children.size()); + max_arguments, function_name, expression_list->children.size()); - if (expression_list->children.size() == 2 || expression_list->children.size() == max_arguments - 1) + return expression_list; +} + +static void addColumnsStructureToQueryWithHDFSOrURLClusterEngine(ASTPtr & query, const String & structure, const String & function_name) +{ + ASTExpressionList * expression_list = getExpressionListAndCheckArguments(query, 5, function_name); + auto structure_literal = std::make_shared(structure); + ASTs & args = expression_list->children; + + /// XCuster(cluster_name, source) + if (args.size() == 2) { - auto format_literal = std::make_shared("auto"); - expression_list->children.push_back(format_literal); + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// XCuster(cluster_name, source, format) + else if (args.size() == 3) + { + args.push_back(structure_literal); + } + /// XCuster(cluster_name, source, format, 'auto') + else if (args.size() == 4) + { + args.back() = structure_literal; + } + /// XCuster(cluster_name, source, format, 'auto', compression) + else if (args.size() == 5) + { + args[args.size() - 2] = structure_literal; } - - expression_list->children.push_back(structure_literal); } + +void addColumnsStructureToQueryWithHDFSClusterEngine(ASTPtr & query, const String & structure) +{ + addColumnsStructureToQueryWithHDFSOrURLClusterEngine(query, structure, "hdfsCluster"); +} + +void addColumnsStructureToQueryWithURLClusterEngine(ASTPtr & query, const String & structure) +{ + addColumnsStructureToQueryWithHDFSOrURLClusterEngine(query, structure, "urlCluster"); +} + +void addColumnsStructureToQueryWithS3ClusterEngine(ASTPtr & query, const String & structure) +{ + ASTExpressionList * expression_list = getExpressionListAndCheckArguments(query, 7, "s3Cluster"); + auto structure_literal = std::make_shared(structure); + ASTs & args = expression_list->children; + + /// s3Cluster(cluster_name, s3_url) + if (args.size() == 2) + { + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// s3Cluster(cluster_name, s3_url, format) or s3Cluster(cluster_name, s3_url, NOSIGN) + /// We can distinguish them by looking at the 3-rd argument: check if it's NOSIGN or not. + else if (args.size() == 3) + { + auto third_arg = checkAndGetLiteralArgument(args[2], "format/NOSIGN"); + /// If there is NOSIGN, add format name before structure. + if (boost::iequals(third_arg, "NOSIGN")) + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// s3Cluster(cluster_name, source, format, structure) or + /// s3Cluster(cluster_name, source, access_key_id, access_key_id) or + /// s3Cluster(cluster_name, source, NOSIGN, format) + /// We can distinguish them by looking at the 3-nrd argument: check if it's NOSIGN, format name or neither. + else if (args.size() == 4) + { + auto third_arg = checkAndGetLiteralArgument(args[2], "format/NOSIGN"); + if (boost::iequals(third_arg, "NOSIGN")) + { + args.push_back(structure_literal); + } + else if (third_arg == "auto" || FormatFactory::instance().getAllFormats().contains(third_arg)) + { + args.back() = structure_literal; + } + else + { + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + } + /// s3Cluster(cluster_name, source, format, structure, compression_method) or + /// s3Cluster(cluster_name, source, access_key_id, access_key_id, format) or + /// s3Cluster(cluster_name, source, NOSIGN, format, structure) + /// We can distinguish them by looking at the 3-rd argument: check if it's NOSIGN, format name or neither. + else if (args.size() == 5) + { + auto third_arg = checkAndGetLiteralArgument(args[2], "format/NOSIGN"); + if (boost::iequals(third_arg, "NOSIGN")) + { + args.back() = structure_literal; + } + else if (third_arg == "auto" || FormatFactory::instance().getAllFormats().contains(third_arg)) + { + args[args.size() - 2] = structure_literal; + } + else + { + args.push_back(structure_literal); + } + } + /// s3Cluster(cluster_name, source, access_key_id, access_key_id, format, structure) or + /// s3Cluster(cluster_name, source, NOSIGN, format, structure, compression_method) + /// We can distinguish them by looking at the 3-rd argument: check if it's a NOSIGN keyword name or not. + else if (args.size() == 6) + { + auto third_arg = checkAndGetLiteralArgument(args[2], "format/NOSIGN"); + if (boost::iequals(third_arg, "NOSIGN")) + { + args[args.size() - 2] = structure_literal; + } + else + { + args.back() = structure_literal; + } + } + /// s3Cluster(cluster_name, source, access_key_id, access_key_id, format, structure, compression) + else if (args.size() == 7) + { + args[args.size() - 2] = structure_literal; + } +} + + } diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.h b/src/Storages/addColumnsStructureToQueryWithClusterEngine.h index f39f3a31630..44d85e00e24 100644 --- a/src/Storages/addColumnsStructureToQueryWithClusterEngine.h +++ b/src/Storages/addColumnsStructureToQueryWithClusterEngine.h @@ -5,7 +5,9 @@ namespace DB { -/// Add structure argument for queries with s3Cluster/hdfsCluster table function. -void addColumnsStructureToQueryWithClusterEngine(ASTPtr & query, const String & structure, size_t max_arguments, const String & function_name); +/// Add structure argument for queries with urlCluster/hdfsCluster/s3CLuster table function. +void addColumnsStructureToQueryWithHDFSClusterEngine(ASTPtr & query, const String & structure); +void addColumnsStructureToQueryWithURLClusterEngine(ASTPtr & query, const String & structure); +void addColumnsStructureToQueryWithS3ClusterEngine(ASTPtr & query, const String & structure); } diff --git a/src/TableFunctions/TableFunctionHDFSCluster.cpp b/src/TableFunctions/TableFunctionHDFSCluster.cpp index b50509ee8bf..772965d0b68 100644 --- a/src/TableFunctions/TableFunctionHDFSCluster.cpp +++ b/src/TableFunctions/TableFunctionHDFSCluster.cpp @@ -113,7 +113,6 @@ StoragePtr TableFunctionHDFSCluster::getStorage( columns, ConstraintsDescription{}, compression_method, - 5, structure != "auto"); } return storage; diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 085028c2145..805608564ae 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -127,7 +127,6 @@ StoragePtr TableFunctionS3Cluster::executeImpl( columns, ConstraintsDescription{}, context, - 7, structure_argument_was_provided); } diff --git a/src/TableFunctions/TableFunctionURLCluster.cpp b/src/TableFunctions/TableFunctionURLCluster.cpp index 03af08c7de9..fc37ecc7f1d 100644 --- a/src/TableFunctions/TableFunctionURLCluster.cpp +++ b/src/TableFunctions/TableFunctionURLCluster.cpp @@ -106,11 +106,13 @@ StoragePtr TableFunctionURLCluster::getStorage( storage = std::make_shared( context, cluster_name, + filename, + format, + compression_method, StorageID(getDatabaseName(), table_name), getActualTableStructure(context), ConstraintsDescription{}, configuration, - 5, structure != "auto"); } return storage; diff --git a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference index b918bf2b155..c7ab9b4f989 100644 --- a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference +++ b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference @@ -10,6 +10,72 @@ c3 Nullable(Int64) c1 Nullable(Int64) c2 Nullable(Int64) c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 0 0 0 0 0 0 1 2 3 diff --git a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql index 03e8785b24b..d8cd3b0d34f 100644 --- a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql +++ b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql @@ -5,9 +5,23 @@ desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localh desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV'); desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest'); desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'auto'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto', 'auto'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'auto'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV', 'auto'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV', 'auto', 'auto'); + select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv') order by c1, c2, c3; select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV') order by c1, c2, c3; select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest') order by c1, c2, c3; select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'auto') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto', 'auto') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'auto') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV', 'auto') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV', 'auto', 'auto') order by c1, c2, c3; + diff --git a/tests/queries/0_stateless/02721_url_cluster.reference b/tests/queries/0_stateless/02721_url_cluster.reference index 4345eb9c47b..36a5e136469 100644 --- a/tests/queries/0_stateless/02721_url_cluster.reference +++ b/tests/queries/0_stateless/02721_url_cluster.reference @@ -66,6 +66,39 @@ c3 Nullable(Int64) c1 Nullable(Int64) c2 Nullable(Int64) c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 0 0 0 0 0 0 1 2 3 diff --git a/tests/queries/0_stateless/02721_url_cluster.sql b/tests/queries/0_stateless/02721_url_cluster.sql index 0d5db7949db..5d089c16f5d 100644 --- a/tests/queries/0_stateless/02721_url_cluster.sql +++ b/tests/queries/0_stateless/02721_url_cluster.sql @@ -17,9 +17,15 @@ select COUNT(*) from urlCluster('test_cluster_two_shards_localhost', 'http://loc desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv'); desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV'); +desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'auto'); +desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto'); +desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto', 'auto'); select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv') order by c1, c2, c3; select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV') order by c1, c2, c3; +select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'auto') order by c1, c2, c3; +select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto') order by c1, c2, c3; +select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto', 'auto') order by c1, c2, c3; drop table if exists test; create table test (x UInt32, y UInt32, z UInt32) engine=Memory(); From 69e39aba80b4fadea79bfdf0f83a23848520948e Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 24 Apr 2023 22:13:13 +0800 Subject: [PATCH 0117/2223] fix test --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 4ff8c2d3af1..98cae995b47 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -97,6 +97,7 @@ arrayCumSum arrayCumSumNonNegative arrayDifference arrayDistinct +arrayDotProduct arrayElement arrayEnumerate arrayEnumerateDense From 029c92344b2db5b5129b7adb39eb0b5f09addb42 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 16:57:43 +0200 Subject: [PATCH 0118/2223] Fix possible terminate called for uncaught exception in Connection --- src/Client/Connection.cpp | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index d39148d3016..3dd78afb79b 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -232,12 +232,27 @@ void Connection::disconnect() maybe_compressed_out = nullptr; in = nullptr; last_input_packet_type.reset(); - out = nullptr; // can write to socket + std::exception_ptr finalize_exception; + try + { + // finalize() can write to socket and throw an exception. + out->finalize(); + } + catch (...) + { + /// Don't throw an exception here, it will leave Connection in invalid state. + finalize_exception = std::current_exception(); + } + out = nullptr; + if (socket) socket->close(); socket = nullptr; connected = false; nonce.reset(); + + if (finalize_exception) + std::rethrow_exception(finalize_exception); } From bc0c431eb77d7844429b70da58169dc49d5de4a7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 17:03:48 +0200 Subject: [PATCH 0119/2223] Fix possible terminate called for uncaught exception in InterserverIOHTTPHandler::handleRequest --- src/Server/InterserverIOHTTPHandler.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index 6b0cd543053..f7128e7e5a3 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -93,10 +93,13 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe auto write_response = [&](const std::string & message) { - if (response.sent()) - return; - auto & out = *used_output.out; + if (response.sent()) + { + out.finalize(); + return; + } + try { writeString(message, out); @@ -127,7 +130,10 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe catch (Exception & e) { if (e.code() == ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES) + { + used_output.out->finalize(); return; + } response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); From c503f6532c19ed98a119328c89a9d6e282767332 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 24 Apr 2023 15:11:36 +0000 Subject: [PATCH 0120/2223] Add more finalize() to avoid terminate --- src/Storages/HDFS/StorageHDFS.cpp | 1 + src/Storages/StorageFile.cpp | 1 + src/Storages/StorageS3.cpp | 1 + src/Storages/StorageURL.cpp | 1 + 4 files changed, 4 insertions(+) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index c915213f4ac..f4dd26435b3 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -490,6 +490,7 @@ private: { /// Stop ParallelFormattingOutputFormat correctly. writer.reset(); + write_buf->finalize(); throw; } } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7e5a93c13c1..dafb51509ea 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -928,6 +928,7 @@ private: { /// Stop ParallelFormattingOutputFormat correctly. writer.reset(); + write_buf->finalize(); throw; } } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index bd3e8fe886d..b0fde7c8e02 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -823,6 +823,7 @@ private: { /// Stop ParallelFormattingOutputFormat correctly. writer.reset(); + write_buf->finalize(); throw; } } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index d2df3881c71..00b4c174834 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -442,6 +442,7 @@ void StorageURLSink::finalize() { /// Stop ParallelFormattingOutputFormat correctly. writer.reset(); + write_buf->finalize(); throw; } } From 57801b7a02b5574c1d385deb38d17846780389ac Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 19:06:45 +0200 Subject: [PATCH 0121/2223] Fix --- src/Client/Connection.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 3dd78afb79b..e328d0c4e43 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -236,7 +236,8 @@ void Connection::disconnect() try { // finalize() can write to socket and throw an exception. - out->finalize(); + if (out) + out->finalize(); } catch (...) { From c693c1bd17b62d9bc1bcb969867721415b4cb0a8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 19:07:30 +0200 Subject: [PATCH 0122/2223] Fix style --- src/Server/InterserverIOHTTPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index f7128e7e5a3..ea71d954cc0 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -99,7 +99,7 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe out.finalize(); return; } - + try { writeString(message, out); From 8ff864cd8bf178635fd3a9a9dfc3604f8fc62264 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 19:12:50 +0200 Subject: [PATCH 0123/2223] Fix --- src/Storages/StorageURL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f0401c70501..965a36d12f9 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -662,7 +662,7 @@ Pipe IStorageURLBase::read( } else { - iterator_wrapper = std::make_shared([&, done = false]() mutable + iterator_wrapper = std::make_shared([uri, max_addresses, done = false]() mutable { if (done) return StorageURLSource::FailoverOptions{}; From 75a3b6c32290f18b9b01ae3afbd45e10b3b86090 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Apr 2023 21:08:53 +0200 Subject: [PATCH 0124/2223] Fix build --- src/Storages/StorageURL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 965a36d12f9..8b4538b2c10 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -662,7 +662,7 @@ Pipe IStorageURLBase::read( } else { - iterator_wrapper = std::make_shared([uri, max_addresses, done = false]() mutable + iterator_wrapper = std::make_shared([&, max_addresses, done = false]() mutable { if (done) return StorageURLSource::FailoverOptions{}; From 65d23af611f28fbe0f2baf1dac6cd1aa317676b5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 24 Apr 2023 19:35:49 +0000 Subject: [PATCH 0125/2223] Refactor PreparedSets [1] --- src/Interpreters/PreparedSets.cpp | 101 +++++++++++-------- src/Interpreters/PreparedSets.h | 157 ++++++++++++++++++++++++++---- 2 files changed, 200 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 7b0efddae87..11af41cae8f 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -5,6 +5,8 @@ #include #include #include +#include "Common/logger_useful.h" +#include "Processors/QueryPlan/CreatingSetsStep.h" namespace DB { @@ -66,30 +68,30 @@ String PreparedSetKey::toString() const return buf.str(); } -SubqueryForSet & PreparedSets::createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, - SizeLimits set_size_limit, bool transform_null_in) -{ - SubqueryForSet & subquery = subqueries[subquery_id]; +// SubqueryForSet & PreparedSets::createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, +// SizeLimits set_size_limit, bool transform_null_in) +// { +// SubqueryForSet & subquery = subqueries[subquery_id]; - /// If you already created a Set with the same subquery / table for another ast - /// In that case several PreparedSetKey would share same subquery and set - /// Not sure if it's really possible case (maybe for distributed query when set was filled by external table?) - if (subquery.set.isValid()) - sets[key] = subquery.set; - else - { - subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); - sets[key] = FutureSet(subquery.promise_to_fill_set.get_future()); - } +// /// If you already created a Set with the same subquery / table for another ast +// /// In that case several PreparedSetKey would share same subquery and set +// /// Not sure if it's really possible case (maybe for distributed query when set was filled by external table?) +// if (subquery.set.isValid()) +// sets[key] = subquery.set; +// else +// { +// subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); +// sets[key] = FutureSet(subquery.promise_to_fill_set.get_future()); +// } - if (!subquery.set_in_progress) - { - subquery.key = key.toString(); - subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); - } +// if (!subquery.set_in_progress) +// { +// subquery.key = key.toString(); +// subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); +// } - return subquery; -} +// return subquery; +// } /// If the subquery is not associated with any set, create default-constructed SubqueryForSet. /// It's aimed to fill external table passed to SubqueryForSet::createSource. @@ -154,26 +156,6 @@ QueryPlanPtr SubqueryForSet::detachSource() } -FutureSet::FutureSet(SetPtr set) -{ - std::promise promise; - promise.set_value(set); - *this = FutureSet(promise.get_future()); -} - - -bool FutureSet::isReady() const -{ - return future_set.valid() && - future_set.wait_for(std::chrono::seconds(0)) == std::future_status::ready; -} - -bool FutureSet::isCreated() const -{ - return isReady() && get() != nullptr && get()->isCreated(); -} - - std::variant, SharedSet> PreparedSetsCache::findOrPromiseToBuild(const String & key) { std::lock_guard lock(cache_mutex); @@ -194,4 +176,41 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo return promise_to_fill_set; } +std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) +{ + if (set) + return nullptr; + + auto set_cache = context->getPreparedSetsCache(); + if (set_cache) + { + auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); + if (from_cache.index() == 0) + { + subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); + } + else + { + LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); + set = std::get<1>(from_cache).get(); + return nullptr; + } + } + + subquery.set = set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + + auto plan = subquery.detachSource(); + + const Settings & settings = context->getSettingsRef(); + auto creating_set = std::make_unique( + plan->getCurrentDataStream(), + subquery_id, + std::move(subquery), + SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), + context); + creating_set->setStepDescription("Create set for subquery"); + plan->addStep(std::move(creating_set)); + return plan; +} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 4a7d1c3de46..a4898fe5ec6 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -10,6 +10,14 @@ #include #include #include +#include "Core/Block.h" +#include "Interpreters/Context.h" +#include "Interpreters/Set.h" +#include "Processors/Executors/CompletedPipelineExecutor.h" +#include "Processors/QueryPlan/BuildQueryPipelineSettings.h" +#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" +#include "Processors/Sinks/NullSink.h" +#include namespace DB { @@ -25,30 +33,83 @@ class InterpreterSelectWithUnionQuery; /// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there /// must be an explicit step to build them before they can be used. /// FutureSet objects can be stored in PreparedSets and are not intended to be used from multiple threads. -class FutureSet final +// class FutureSet final +// { +// public: +// FutureSet() = default; + +// /// Create FutureSet from an object that will be created in the future. +// explicit FutureSet(const std::shared_future & future_set_) : future_set(future_set_) {} + +// /// Create FutureSet from a ready set. +// explicit FutureSet(SetPtr readySet); + +// /// The set object will be ready in the future, as opposed to 'null' object when FutureSet is default constructed. +// bool isValid() const { return future_set.valid(); } + +// /// The the value of SetPtr is ready, but the set object might not have been filled yet. +// bool isReady() const; + +// /// The set object is ready and filled. +// bool isCreated() const; + +// SetPtr get() const { chassert(isReady()); return future_set.get(); } + +// private: +// std::shared_future future_set; +// }; + +class FutureSet { public: - FutureSet() = default; + virtual ~FutureSet() = default; - /// Create FutureSet from an object that will be created in the future. - explicit FutureSet(const std::shared_future & future_set_) : future_set(future_set_) {} + virtual bool isReady() const = 0; + virtual SetPtr get() const = 0; - /// Create FutureSet from a ready set. - explicit FutureSet(SetPtr readySet); + virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; + virtual std::unique_ptr build(const ContextPtr & context) = 0; +}; - /// The set object will be ready in the future, as opposed to 'null' object when FutureSet is default constructed. - bool isValid() const { return future_set.valid(); } +using FutureSetPtr = std::unique_ptr; - /// The the value of SetPtr is ready, but the set object might not have been filled yet. - bool isReady() const; +class FutureSetFromTuple final : public FutureSet +{ +public: + FutureSetFromTuple(Block block_, const SizeLimits & size_limits_, bool transform_null_in_); - /// The set object is ready and filled. - bool isCreated() const; + bool isReady() const override { return set != nullptr; } + SetPtr get() const override { return set; } - SetPtr get() const { chassert(isReady()); return future_set.get(); } + SetPtr buildOrderedSetInplace(const ContextPtr &) override + { + fill(true); + return set; + } + + std::unique_ptr build(const ContextPtr &) override + { + fill(false); + return nullptr; + } private: - std::shared_future future_set; + Block block; + SizeLimits size_limits; + bool transform_null_in; + + SetPtr set; + + void fill(bool create_ordered_set) + { + if (set) + return; + + set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); + set->insertFromBlock(block.getColumnsWithTypeAndName()); + set->finishInsert(); + } }; /// Information on how to build set for the [GLOBAL] IN section. @@ -66,11 +127,12 @@ public: /// Build this set from the result of the subquery. String key; - SetPtr set_in_progress; + SetPtr set; /// After set_in_progress is finished it will be put into promise_to_fill_set and thus all FutureSet's /// that are referencing this set will be filled. + std::promise promise_to_fill_set; - FutureSet set = FutureSet{promise_to_fill_set.get_future()}; + // FutureSet set = FutureSet{promise_to_fill_set.get_future()}; /// If set, put the result into the table. /// This is a temporary table for transferring to remote servers for distributed query processing. @@ -80,6 +142,67 @@ public: std::unique_ptr source; }; +class FutureSetFromSubquery : public FutureSet +{ +public: + FutureSetFromSubquery(SubqueryForSet subquery_, String subquery_id, SizeLimits set_size_limit_, bool transform_null_in_); + + bool isReady() const override { return set != nullptr; } + SetPtr get() const override { return set; } + + SetPtr buildOrderedSetInplace(const ContextPtr & context) override + { + auto plan = buildPlan(context, true); + + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + + return set; + } + + std::unique_ptr build(const ContextPtr & context) override + { + return buildPlan(context, false); + } + +private: + SetPtr set; + SubqueryForSet subquery; + String subquery_id; + SizeLimits size_limits; + bool transform_null_in; + + std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); +}; + +// class FutureSetFromFuture : public FutureSet +// { +// public: +// FutureSetFromFuture(std::shared_future future_set_); + +// bool isReady() const override { return future_set.wait_for(std::chrono::seconds(0)) == std::future_status::ready; } +// SetPtr get() const override { return future_set.get(); } + +// SetPtr buildOrderedSetInplace(const ContextPtr &) override +// { +// fill(true); +// return set; +// } + +// std::unique_ptr build(const ContextPtr &) override +// { +// fill(false); +// return nullptr; +// } + +// private: +// std::shared_future future_set; +// } + struct PreparedSetKey { /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired @@ -132,7 +255,7 @@ private: std::unordered_map sets; /// This is the information required for building sets - SubqueriesForSets subqueries; + // SubqueriesForSets subqueries; }; using PreparedSetsPtr = std::shared_ptr; From c58ee893222035a96a2a37ffb02f839825570c84 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 25 Apr 2023 10:54:40 +0000 Subject: [PATCH 0126/2223] Fix test for analyzer --- tests/queries/0_stateless/02721_url_cluster.sql | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02721_url_cluster.sql b/tests/queries/0_stateless/02721_url_cluster.sql index 5d089c16f5d..ea387b06dc3 100644 --- a/tests/queries/0_stateless/02721_url_cluster.sql +++ b/tests/queries/0_stateless/02721_url_cluster.sql @@ -1,7 +1,6 @@ -- Tags: no-fasttest -- Tag no-fasttest: Depends on AWS - select * from urlCluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv') ORDER BY c1, c2, c3; select * from urlCluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV') ORDER BY c1, c2, c3; select * from urlCluster('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; @@ -29,7 +28,7 @@ select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'htt drop table if exists test; create table test (x UInt32, y UInt32, z UInt32) engine=Memory(); -insert into test select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/a.tsv'); +insert into test select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/a.tsv', 'TSV'); select * from test; drop table test; From 307aa127d470b4f8e8099f59445cab4cb4ba1418 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 25 Apr 2023 13:27:20 +0200 Subject: [PATCH 0127/2223] Updated to calculate and send max_parsing_threads --- src/Storages/StorageFile.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 9972549c292..fc36b4a9f62 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -51,6 +51,7 @@ #include #include #include +#include namespace ProfileEvents @@ -606,8 +607,11 @@ public: if (!read_buf) read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context); + const Settings & settings = context->getSettingsRef(); + chassert(storage->paths.size()); + const auto max_parsing_threads = std::max(settings.max_threads/storage->paths.size(),1UL); auto format - = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, storage->paths.size()); + = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, max_parsing_threads); QueryPipelineBuilder builder; builder.init(Pipe(format)); From 80a2f30a0cc651db608735391d5173452f7b41ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 Apr 2023 18:14:08 +0000 Subject: [PATCH 0128/2223] Refactor PreparedSets [2] --- src/Columns/ColumnSet.h | 4 +- src/Interpreters/ActionsVisitor.cpp | 56 +++--- src/Interpreters/ActionsVisitor.h | 7 +- src/Interpreters/ExpressionAnalyzer.cpp | 188 ++++++++++----------- src/Interpreters/ExpressionAnalyzer.h | 4 +- src/Interpreters/GlobalSubqueriesVisitor.h | 24 +-- src/Interpreters/PreparedSets.cpp | 110 +++++++++--- src/Interpreters/PreparedSets.h | 75 +++++--- src/Planner/PlannerContext.h | 20 +-- 9 files changed, 283 insertions(+), 205 deletions(-) diff --git a/src/Columns/ColumnSet.h b/src/Columns/ColumnSet.h index 3f5cf4ad280..ccd9aa19896 100644 --- a/src/Columns/ColumnSet.h +++ b/src/Columns/ColumnSet.h @@ -21,7 +21,7 @@ class ColumnSet final : public COWHelper private: friend class COWHelper; - ColumnSet(size_t s_, FutureSet data_) : data(std::move(data_)) { s = s_; } + ColumnSet(size_t s_, FutureSetPtr data_) : data(std::move(data_)) { s = s_; } ColumnSet(const ColumnSet &) = default; public: @@ -35,7 +35,7 @@ public: Field operator[](size_t) const override { return {}; } private: - FutureSet data; + FutureSetPtr data; }; } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 3bb3ea67e29..00feecb44f5 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -422,9 +422,8 @@ Block createBlockForSet( } -SetPtr makeExplicitSet( - const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, - ContextPtr context, const SizeLimits & size_limits, PreparedSets & prepared_sets) +FutureSetPtr makeExplicitSet( + const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets) { const IAST & args = *node->arguments; @@ -448,7 +447,7 @@ SetPtr makeExplicitSet( element_type = low_cardinality_type->getDictionaryType(); auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types); - if (auto set = prepared_sets.get(set_key)) + if (auto set = prepared_sets.getFuture(set_key)) return set; /// Already prepared. Block block; @@ -458,14 +457,7 @@ SetPtr makeExplicitSet( else block = createBlockForSet(left_arg_type, right_arg, set_element_types, context); - SetPtr set - = std::make_shared(size_limits, create_ordered_set, context->getSettingsRef().transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(block.getColumnsWithTypeAndName()); - set->finishInsert(); - - prepared_sets.set(set_key, set); - return set; + return prepared_sets.addFromTuple(set_key, block); } class ScopeStack::Index @@ -952,7 +944,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & return; } - FutureSet prepared_set; + FutureSetPtr prepared_set; if (checkFunctionIsInOrGlobalInOperator(node)) { /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). @@ -961,7 +953,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (!data.no_makeset && !(data.is_create_parameterized_view && !analyzeReceiveQueryParams(ast).empty())) prepared_set = makeSet(node, data, data.no_subqueries); - if (prepared_set.isValid()) + if (prepared_set) { /// Transform tuple or subquery into a set. } @@ -1174,14 +1166,14 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & num_arguments += columns.size() - 1; arg += columns.size() - 1; } - else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set.isValid()) + else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set) { ColumnWithTypeAndName column; column.type = std::make_shared(); /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). - const bool is_constant_set = prepared_set.isCreated(); + const bool is_constant_set = typeid_cast(prepared_set.get()) == nullptr; if (is_constant_set) column.name = data.getUniqueName("__set"); else @@ -1373,7 +1365,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, data.addColumn(std::move(column)); } -FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries) +FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries) { if (!data.prepared_sets) return {}; @@ -1394,11 +1386,8 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no return {}; auto set_key = PreparedSetKey::forSubquery(*right_in_operand); - { - auto set = data.prepared_sets->getFuture(set_key); - if (set.isValid()) - return set; - } + if (auto set = data.prepared_sets->getFuture(set_key)) + return set; /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). @@ -1409,20 +1398,16 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no if (table) { - StorageSet * storage_set = dynamic_cast(table.get()); - if (storage_set) - { - SetPtr set = storage_set->getSet(); - data.prepared_sets->set(set_key, set); - return FutureSet(set); - } + if (StorageSet * storage_set = dynamic_cast(table.get())) + return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. - String set_id = right_in_operand->getColumnName(); - bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; - SubqueryForSet & subquery_for_set = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); + // String set_id = right_in_operand->getColumnName(); + //bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; + SubqueryForSet subquery_for_set; // = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); + subquery_for_set.key = right_in_operand->getColumnName(); /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, @@ -1432,13 +1417,13 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no * In case that we have HAVING with IN subquery, we have to force creating set for it. * Also it doesn't make sense if it is GLOBAL IN or ordinary IN. */ - if (data.create_source_for_in && !subquery_for_set.hasSource()) + if (data.create_source_for_in) { auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); subquery_for_set.createSource(*interpreter); } - return subquery_for_set.set; + return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); } else { @@ -1446,8 +1431,7 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no const auto & index = data.actions_stack.getLastActionsIndex(); if (data.prepared_sets && index.contains(left_in_operand->getColumnName())) /// An explicit enumeration of values in parentheses. - return FutureSet( - makeExplicitSet(&node, last_actions, false, data.getContext(), data.set_size_limit, *data.prepared_sets)); + return makeExplicitSet(&node, last_actions, data.getContext(), *data.prepared_sets); else return {}; } diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 260fd5ab2c0..71d57620196 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -25,9 +25,8 @@ class IFunctionOverloadResolver; using FunctionOverloadResolverPtr = std::shared_ptr; /// The case of an explicit enumeration of values. -SetPtr makeExplicitSet( - const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, - ContextPtr context, const SizeLimits & limits, PreparedSets & prepared_sets); +FutureSetPtr makeExplicitSet( + const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets); /** For ActionsVisitor * A stack of ExpressionActions corresponding to nested lambda expressions. @@ -219,7 +218,7 @@ private: static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data); static void visit(ASTExpressionList & expression_list, const ASTPtr & ast, Data & data); - static FutureSet makeSet(const ASTFunction & node, Data & data, bool no_subqueries); + static FutureSetPtr makeSet(const ASTFunction & node, Data & data, bool no_subqueries); static ASTs doUntuple(const ASTFunction * function, ActionsMatcher::Data & data); static std::optional getNameAndTypeFromAST(const ASTPtr & ast, Data & data); }; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index cc54e7620f6..7dece81734c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -450,76 +450,76 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global, b } -void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options) -{ - if (!prepared_sets) - return; +// void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options) +// { +// if (!prepared_sets) +// return; - auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); +// auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); - if (prepared_sets->getFuture(set_key).isValid()) - return; /// Already prepared. +// if (prepared_sets->getFuture(set_key).isValid()) +// return; /// Already prepared. - if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name)) - { - prepared_sets->set(set_key, set_ptr_from_storage_set); - return; - } +// if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name)) +// { +// prepared_sets->set(set_key, set_ptr_from_storage_set); +// return; +// } - auto build_set = [&] () -> SetPtr - { - LOG_TRACE(getLogger(), "Building set, key: {}", set_key.toString()); +// auto build_set = [&] () -> SetPtr +// { +// LOG_TRACE(getLogger(), "Building set, key: {}", set_key.toString()); - auto interpreter_subquery = interpretSubquery(subquery_or_table_name, getContext(), {}, query_options); - auto io = interpreter_subquery->execute(); - PullingAsyncPipelineExecutor executor(io.pipeline); +// auto interpreter_subquery = interpretSubquery(subquery_or_table_name, getContext(), {}, query_options); +// auto io = interpreter_subquery->execute(); +// PullingAsyncPipelineExecutor executor(io.pipeline); - SetPtr set = std::make_shared(settings.size_limits_for_set_used_with_index, true, getContext()->getSettingsRef().transform_null_in); - set->setHeader(executor.getHeader().getColumnsWithTypeAndName()); +// SetPtr set = std::make_shared(settings.size_limits_for_set_used_with_index, true, getContext()->getSettingsRef().transform_null_in); +// set->setHeader(executor.getHeader().getColumnsWithTypeAndName()); - Block block; - while (executor.pull(block)) - { - if (block.rows() == 0) - continue; +// Block block; +// while (executor.pull(block)) +// { +// if (block.rows() == 0) +// continue; - /// If the limits have been exceeded, give up and let the default subquery processing actions take place. - if (!set->insertFromBlock(block.getColumnsWithTypeAndName())) - return nullptr; - } +// /// If the limits have been exceeded, give up and let the default subquery processing actions take place. +// if (!set->insertFromBlock(block.getColumnsWithTypeAndName())) +// return nullptr; +// } - set->finishInsert(); +// set->finishInsert(); - return set; - }; +// return set; +// }; - SetPtr set; +// SetPtr set; - auto set_cache = getContext()->getPreparedSetsCache(); - if (set_cache) - { - auto from_cache = set_cache->findOrPromiseToBuild(set_key.toString()); - if (from_cache.index() == 0) - { - set = build_set(); - std::get<0>(from_cache).set_value(set); - } - else - { - LOG_TRACE(getLogger(), "Waiting for set, key: {}", set_key.toString()); - set = std::get<1>(from_cache).get(); - } - } - else - { - set = build_set(); - } +// auto set_cache = getContext()->getPreparedSetsCache(); +// if (set_cache) +// { +// auto from_cache = set_cache->findOrPromiseToBuild(set_key.toString()); +// if (from_cache.index() == 0) +// { +// set = build_set(); +// std::get<0>(from_cache).set_value(set); +// } +// else +// { +// LOG_TRACE(getLogger(), "Waiting for set, key: {}", set_key.toString()); +// set = std::get<1>(from_cache).get(); +// } +// } +// else +// { +// set = build_set(); +// } - if (!set) - return; +// if (!set) +// return; - prepared_sets->set(set_key, std::move(set)); -} +// prepared_sets->set(set_key, std::move(set)); +// } SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name) { @@ -536,50 +536,50 @@ SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_o /// Performance optimization for IN() if storage supports it. -void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) -{ - if (!node || !storage() || !storage()->supportsIndexForIn()) - return; +// void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) +// { +// if (!node || !storage() || !storage()->supportsIndexForIn()) +// return; - for (auto & child : node->children) - { - /// Don't descend into subqueries. - if (child->as()) - continue; +// for (auto & child : node->children) +// { +// /// Don't descend into subqueries. +// if (child->as()) +// continue; - /// Don't descend into lambda functions - const auto * func = child->as(); - if (func && func->name == "lambda") - continue; +// /// Don't descend into lambda functions +// const auto * func = child->as(); +// if (func && func->name == "lambda") +// continue; - makeSetsForIndex(child); - } +// makeSetsForIndex(child); +// } - const auto * func = node->as(); - if (func && functionIsInOrGlobalInOperator(func->name)) - { - const IAST & args = *func->arguments; - const ASTPtr & left_in_operand = args.children.at(0); +// const auto * func = node->as(); +// if (func && functionIsInOrGlobalInOperator(func->name)) +// { +// const IAST & args = *func->arguments; +// const ASTPtr & left_in_operand = args.children.at(0); - if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot)) - { - const ASTPtr & arg = args.children.at(1); - if (arg->as() || arg->as()) - { - if (settings.use_index_for_in_with_subqueries) - tryMakeSetForIndexFromSubquery(arg, query_options); - } - else - { - auto temp_actions = std::make_shared(columns_after_join); - getRootActions(left_in_operand, true, temp_actions); +// if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot)) +// { +// const ASTPtr & arg = args.children.at(1); +// if (arg->as() || arg->as()) +// { +// if (settings.use_index_for_in_with_subqueries) +// tryMakeSetForIndexFromSubquery(arg, query_options); +// } +// else +// { +// auto temp_actions = std::make_shared(columns_after_join); +// getRootActions(left_in_operand, true, temp_actions); - if (prepared_sets && temp_actions->tryFindInOutputs(left_in_operand->getColumnName())) - makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, *prepared_sets); - } - } - } -} +// if (prepared_sets && temp_actions->tryFindInOutputs(left_in_operand->getColumnName())) +// makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, *prepared_sets); +// } +// } +// } +// } void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 1b6e8e24091..e4926b80625 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -144,7 +144,7 @@ public: /** Create Set from a subquery or a table expression in the query. The created set is suitable for using the index. * The set will not be created if its size hits the limit. */ - void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {}); + // void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {}); /** Checks if subquery is not a plain StorageSet. * Because while making set we will read data from StorageSet which is not allowed. @@ -364,7 +364,7 @@ public: ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const; /// Create Set-s that we make from IN section to use index on them. - void makeSetsForIndex(const ASTPtr & node); + // void makeSetsForIndex(const ASTPtr & node); private: StorageMetadataPtr metadata_snapshot; diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index b105cae31c6..9fb2c02bd58 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -167,19 +167,21 @@ public: { /// Do not materialize external tables if it's explain statement. } - else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) - { - auto external_table = external_storage_holder->getTable(); - auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); - auto io = interpreter->execute(); - io.pipeline.complete(std::move(table_out)); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); - } + // else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) + // { + // auto external_table = external_storage_holder->getTable(); + // auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); + // auto io = interpreter->execute(); + // io.pipeline.complete(std::move(table_out)); + // CompletedPipelineExecutor executor(io.pipeline); + // executor.execute(); + // } else { - auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); - subquery_for_set.createSource(*interpreter, external_storage); + // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); + // subquery_for_set.createSource(*interpreter, external_storage); + auto key = subquery_or_table_name->getColumnName(); + prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 11af41cae8f..300ef2aadb6 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -95,11 +95,51 @@ String PreparedSetKey::toString() const /// If the subquery is not associated with any set, create default-constructed SubqueryForSet. /// It's aimed to fill external table passed to SubqueryForSet::createSource. -SubqueryForSet & PreparedSets::getSubquery(const String & subquery_id) { return subqueries[subquery_id]; } +void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) +{ + auto it = subqueries.find(subquery_id); + if (it == subqueries.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); -void PreparedSets::set(const PreparedSetKey & key, SetPtr set_) { sets[key] = FutureSet(set_); } + it->second->addStorage(std::move(storage)); +} -FutureSet PreparedSets::getFuture(const PreparedSetKey & key) const +FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set_) +{ + auto from_storage = std::make_shared(std::move(set_)); + auto [it, inserted] = sets.emplace(key, std::move(from_storage)); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + + return it->second; +} + +FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block) +{ + auto from_tuple = std::make_shared(std::move(block)); + auto [it, inserted] = sets.emplace(key, std::move(from_tuple)); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + + return it->second; +} + +FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery) +{ + auto id = subquery.key; + auto from_subquery = std::make_shared(std::move(subquery)); + auto [it, inserted] = sets.emplace(key, from_subquery); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + + subqueries.emplace(id, std::move(from_subquery)); + return it->second; +} + +FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const { auto it = sets.find(key); if (it == sets.end()) @@ -107,24 +147,24 @@ FutureSet PreparedSets::getFuture(const PreparedSetKey & key) const return it->second; } -SetPtr PreparedSets::get(const PreparedSetKey & key) const -{ - auto it = sets.find(key); - if (it == sets.end() || !it->second.isReady()) - return nullptr; - return it->second.get(); -} +// SetPtr PreparedSets::get(const PreparedSetKey & key) const +// { +// auto it = sets.find(key); +// if (it == sets.end() || !it->second.isReady()) +// return nullptr; +// return it->second.get(); +// } -std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const -{ - std::vector res; - for (const auto & it : this->sets) - { - if (it.first.ast_hash == ast_hash) - res.push_back(it.second); - } - return res; -} +// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const +// { +// std::vector res; +// for (const auto & it : this->sets) +// { +// if (it.first.ast_hash == ast_hash) +// res.push_back(it.second); +// } +// return res; +// } PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() { @@ -197,14 +237,18 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c } } - subquery.set = set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + + subquery.set = set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); auto plan = subquery.detachSource(); + auto description = subquery.key; - const Settings & settings = context->getSettingsRef(); auto creating_set = std::make_unique( plan->getCurrentDataStream(), - subquery_id, + description, std::move(subquery), SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), context); @@ -213,4 +257,24 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c return plan; } + +static SizeLimits getSizeLimitsForUnorderedSet(const Settings & settings) +{ + return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); +} + +static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) +{ + if (settings.use_index_for_in_with_subqueries_max_values && + settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) + return getSizeLimitsForUnorderedSet(settings); + + return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); +} + +SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) +{ + return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); +} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index a4898fe5ec6..8ebabc32b0a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -69,43 +69,46 @@ public: virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; virtual std::unique_ptr build(const ContextPtr & context) = 0; + + static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); }; -using FutureSetPtr = std::unique_ptr; +using FutureSetPtr = std::shared_ptr; class FutureSetFromTuple final : public FutureSet { public: - FutureSetFromTuple(Block block_, const SizeLimits & size_limits_, bool transform_null_in_); + FutureSetFromTuple(Block block_); bool isReady() const override { return set != nullptr; } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr &) override + SetPtr buildOrderedSetInplace(const ContextPtr & context) override { - fill(true); + fill(context, true); return set; } - std::unique_ptr build(const ContextPtr &) override + std::unique_ptr build(const ContextPtr & context) override { - fill(false); + fill(context, false); return nullptr; } private: Block block; - SizeLimits size_limits; - bool transform_null_in; SetPtr set; - void fill(bool create_ordered_set) + void fill(const ContextPtr & context, bool create_ordered_set) { if (set) return; - set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + + set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); set->insertFromBlock(block.getColumnsWithTypeAndName()); set->finishInsert(); @@ -145,13 +148,16 @@ public: class FutureSetFromSubquery : public FutureSet { public: - FutureSetFromSubquery(SubqueryForSet subquery_, String subquery_id, SizeLimits set_size_limit_, bool transform_null_in_); + FutureSetFromSubquery(SubqueryForSet subquery_); bool isReady() const override { return set != nullptr; } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override { + if (!context->getSettingsRef().use_index_for_in_with_subqueries) + return nullptr; + auto plan = buildPlan(context, true); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); @@ -169,16 +175,34 @@ public: return buildPlan(context, false); } + void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } + private: SetPtr set; SubqueryForSet subquery; - String subquery_id; - SizeLimits size_limits; - bool transform_null_in; std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; +class FutureSetFromStorage : public FutureSet +{ +public: + FutureSetFromStorage(SetPtr set_); // : set(std::move(set_) {} + + bool isReady() const override { return set != nullptr; } + SetPtr get() const override { return set; } + + SetPtr buildOrderedSetInplace(const ContextPtr &) override + { + return set->hasExplicitSetElements() ? set : nullptr; + } + + std::unique_ptr build(const ContextPtr &) override { return nullptr; } + +private: + SetPtr set; +}; + // class FutureSetFromFuture : public FutureSet // { // public: @@ -230,15 +254,20 @@ struct PreparedSetKey class PreparedSets { public: - using SubqueriesForSets = std::unordered_map; + using SubqueriesForSets = std::unordered_map>; - SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, - SizeLimits set_size_limit, bool transform_null_in); - SubqueryForSet & getSubquery(const String & subquery_id); + // SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, + // SizeLimits set_size_limit, bool transform_null_in); - void set(const PreparedSetKey & key, SetPtr set_); - FutureSet getFuture(const PreparedSetKey & key) const; - SetPtr get(const PreparedSetKey & key) const; + FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); + FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block); + FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery); + + void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); + + FutureSetPtr getFuture(const PreparedSetKey & key) const; + //SubqueryForSet & getSubquery(const String & subquery_id); + // SetPtr get(const PreparedSetKey & key) const; /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. @@ -252,10 +281,10 @@ public: bool empty() const; private: - std::unordered_map sets; + std::unordered_map sets; /// This is the information required for building sets - // SubqueriesForSets subqueries; + SubqueriesForSets subqueries; }; using PreparedSetsPtr = std::shared_ptr; diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index ccc4ab43638..9ecfdb6117a 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -57,18 +57,18 @@ class PlannerSet { public: /// Construct planner set that is ready for execution - explicit PlannerSet(FutureSet set_) + explicit PlannerSet(FutureSetPtr set_) : set(std::move(set_)) {} /// Construct planner set with set and subquery node explicit PlannerSet(QueryTreeNodePtr subquery_node_) - : set(promise_to_build_set.get_future()) - , subquery_node(std::move(subquery_node_)) + //: set(promise_to_build_set.get_future()) + : subquery_node(std::move(subquery_node_)) {} /// Get a reference to a set that might be not built yet - const FutureSet & getSet() const + const FutureSetPtr & getSet() const { return set; } @@ -80,14 +80,14 @@ public: } /// This promise will be fulfilled when set is built and all FutureSet objects will become ready - std::promise extractPromiseToBuildSet() - { - return std::move(promise_to_build_set); - } + // std::promise extractPromiseToBuildSet() + // { + // return std::move(promise_to_build_set); + // } private: - std::promise promise_to_build_set; - FutureSet set; + //std::promise promise_to_build_set; + FutureSetPtr set; QueryTreeNodePtr subquery_node; }; From 8bef8fc1de5acf9910f83b978c8b91768da7f670 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 00:54:28 -0400 Subject: [PATCH 0129/2223] Parsing Keeper commands via ClickHouse Parser --- programs/keeper-client/CMakeLists.txt | 2 +- programs/keeper-client/Commands.cpp | 184 ++++++++++++++++++++++++ programs/keeper-client/Commands.h | 131 +++++++++++++++++ programs/keeper-client/KeeperClient.cpp | 122 +++++----------- programs/keeper-client/KeeperClient.h | 39 ++--- programs/keeper-client/Parser.cpp | 94 ++++++++++++ programs/keeper-client/Parser.h | 36 +++++ src/Parsers/TokenIterator.cpp | 4 +- src/Parsers/TokenIterator.h | 2 +- src/Parsers/parseQuery.cpp | 5 +- src/Parsers/parseQuery.h | 3 +- 11 files changed, 514 insertions(+), 108 deletions(-) create mode 100644 programs/keeper-client/Commands.cpp create mode 100644 programs/keeper-client/Commands.h create mode 100644 programs/keeper-client/Parser.cpp create mode 100644 programs/keeper-client/Parser.h diff --git a/programs/keeper-client/CMakeLists.txt b/programs/keeper-client/CMakeLists.txt index 06055d6d820..f54892fe559 100644 --- a/programs/keeper-client/CMakeLists.txt +++ b/programs/keeper-client/CMakeLists.txt @@ -1,4 +1,4 @@ -set (CLICKHOUSE_KEEPER_CLIENT_SOURCES KeeperClient.cpp) +set (CLICKHOUSE_KEEPER_CLIENT_SOURCES KeeperClient.cpp Parser.cpp Commands.cpp) set (CLICKHOUSE_KEEPER_CLIENT_LINK PRIVATE diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp new file mode 100644 index 00000000000..a21550e969d --- /dev/null +++ b/programs/keeper-client/Commands.cpp @@ -0,0 +1,184 @@ + +#include "Commands.h" +#include "KeeperClient.h" + + +namespace DB +{ + +bool LSCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return true; + + node->args.push_back(std::move(arg)); + return true; +} + +void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + String path; + if (!query->args.empty()) + path = client->getAbsolutePath(query->args[0].safeGet()); + else + path = client->cwd; + + const auto children = client->zookeeper->getChildren(path); + for (const auto & child : children) + std::cout << child << " "; + std::cout << "\n"; +} + +bool CDCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return true; + + node->args.push_back(std::move(arg)); + return true; +} + +void CDCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + if (!query->args.empty()) + return; + + auto new_path = client->getAbsolutePath(query->args[0].safeGet()); + if (!client->zookeeper->exists(new_path)) + std::cerr << "Path " << new_path << " does not exists\n"; + else + client->cwd = new_path; +} + +bool SetCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + if (!parseKeeperArg(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + ASTPtr version; + if (ParserNumber{}.parse(pos, version, expected)) + node->args.push_back(version->as().value); + + return true; +} + +void SetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + if (query->args.size() == 2) + client->zookeeper->set(client->getAbsolutePath(query->args[0].safeGet()), query->args[1].safeGet()); + else + client->zookeeper->set( + client->getAbsolutePath(query->args[0].safeGet()), + query->args[1].safeGet(), + static_cast(query->args[2].safeGet())); +} + +bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + if (!parseKeeperArg(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + return true; +} + +void CreateCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + client->zookeeper->create( + client->getAbsolutePath(query->args[0].safeGet()), + query->args[1].safeGet(), + zkutil::CreateMode::Persistent); +} + +bool GetCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + return true; +} + +void GetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + std::cout << client->zookeeper->get(client->getAbsolutePath(query->args[0].safeGet())) << "\n"; +} + +bool RMCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + return true; +} + +void RMCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + client->zookeeper->remove(client->getAbsolutePath(query->args[0].safeGet())); +} + +bool RMRCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + return true; +} + +void RMRCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + String path = client->getAbsolutePath(query->args[0].safeGet()); + client->askConfirmation("You are going to recursively delete path " + path, + [client, path]{ client->zookeeper->removeRecursive(path); }); +} + +bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptr & /* node */, Expected & /* expected */) const +{ + return true; +} + +void HelpCommand::execute(const ASTKeeperQuery * /* query */, KeeperClient * /* client */) const +{ + for (const auto & pair : KeeperClient::commands) + std::cout << pair.second->getHelpMessage() << '\n'; +} + +bool FourLetterWordCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + expected.add(pos, "four-letter-word command"); + if (pos->type != TokenType::BareWord) + return false; + + String cmd(pos->begin, pos->end); + if (cmd.size() != 4) + return false; + + ++pos; + node->args.push_back(std::move(cmd)); + return true; +} + +void FourLetterWordCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + std::cout << client->executeFourLetterCommand(query->args[0].safeGet()) << "\n"; +} + +} diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h new file mode 100644 index 00000000000..e4debd53e42 --- /dev/null +++ b/programs/keeper-client/Commands.h @@ -0,0 +1,131 @@ +#pragma once + +#include "Parser.h" + +namespace DB +{ + +class KeeperClient; + +class IKeeperClientCommand +{ +public: + static const String name; + + virtual bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const = 0; + + virtual void execute(const ASTKeeperQuery * query, KeeperClient * client) const = 0; + + virtual String getHelpMessage() const = 0; + + virtual String getName() const = 0; + + virtual ~IKeeperClientCommand() = default; +}; + +using Command = std::shared_ptr; + + +class LSCommand : public IKeeperClientCommand +{ + String getName() const override { return "ls"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "ls [path] -- Lists the nodes for the given path (default: cwd)"; } +}; + +class CDCommand : public IKeeperClientCommand +{ + String getName() const override { return "cd"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "cd [path] -- Change the working path (default `.`)"; } +}; + +class SetCommand : public IKeeperClientCommand +{ + String getName() const override { return "set"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "set [version] -- Updates the node's value. Only update if version matches (default: -1)"; + } +}; + +class CreateCommand : public IKeeperClientCommand +{ + String getName() const override { return "create"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "create -- Creates new node"; } +}; + +class GetCommand : public IKeeperClientCommand +{ + String getName() const override { return "get"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "get -- Returns the node's value"; } +}; + +class RMCommand : public IKeeperClientCommand +{ + String getName() const override { return "rm"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "remove -- Remove the node"; } +}; + +class RMRCommand : public IKeeperClientCommand +{ + String getName() const override { return "rmr"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "rmr -- Recursively deletes path. Confirmation required"; } +}; + +class HelpCommand : public IKeeperClientCommand +{ + String getName() const override { return "help"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "help -- Prints this message"; } +}; + +class FourLetterWordCommand : public IKeeperClientCommand +{ + String getName() const override { return "flwc"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "flwc -- Executes four-letter-word command"; } +}; + +} diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 52a31a388cc..92aa822231d 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -1,9 +1,11 @@ #include "KeeperClient.h" +#include "Commands.h" #include #include #include #include #include +#include #include @@ -13,13 +15,6 @@ namespace fs = std::filesystem; namespace DB { -static const NameSet four_letter_word_commands -{ - "ruok", "mntr", "srvr", "stat", "srst", "conf", - "cons", "crst", "envi", "dirs", "isro", "wchs", - "wchc", "wchp", "dump", "csnp", "lgif", "rqld", -}; - namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -54,7 +49,7 @@ void KeeperClient::askConfirmation(const String & prompt, std::function confirmation_callback = callback; } -String KeeperClient::getAbsolutePath(const String & relative) +String KeeperClient::getAbsolutePath(const String & relative) const { String result; if (relative.starts_with('/')) @@ -68,16 +63,20 @@ String KeeperClient::getAbsolutePath(const String & relative) return result; } -void KeeperClient::loadCommands(std::vector> && new_commands) +void KeeperClient::loadCommands(std::vector && new_commands) { - for (const auto & [name, args_count, callback] : new_commands) + std::vector suggestions; + for (const auto & command : new_commands) { - commands.insert({{name, args_count}, callback}); - suggest.addWords({name}); + String name = command->getName(); + commands.insert({name, command}); + suggestions.push_back(std::move(name)); } for (const auto & command : four_letter_word_commands) - suggest.addWords({command}); + suggestions.push_back(command); + + suggest.addWords(std::move(suggestions)); } void KeeperClient::defineOptions(Poco::Util::OptionSet & options) @@ -132,61 +131,15 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) void KeeperClient::initialize(Poco::Util::Application & /* self */) { loadCommands({ - {"set", 2, [](KeeperClient * client, const std::vector & args) - { - client->zookeeper->set(client->getAbsolutePath(args[1]), args[2]); - }}, - - {"create", 2, [](KeeperClient * client, const std::vector & args) - { - client->zookeeper->create(client->getAbsolutePath(args[1]), args[2], zkutil::CreateMode::Persistent); - }}, - - {"get", 1, [](KeeperClient * client, const std::vector & args) - { - std::cout << client->zookeeper->get(client->getAbsolutePath(args[1])) << "\n"; - }}, - - {"ls", 0, [](KeeperClient * client, const std::vector & /* args */) - { - auto children = client->zookeeper->getChildren(client->cwd); - for (auto & child : children) - std::cout << child << " "; - std::cout << "\n"; - }}, - - {"ls", 1, [](KeeperClient * client, const std::vector & args) - { - auto children = client->zookeeper->getChildren(client->getAbsolutePath(args[1])); - for (auto & child : children) - std::cout << child << " "; - std::cout << "\n"; - }}, - - {"cd", 0, [](KeeperClient * /* client */, const std::vector & /* args */) - { - }}, - - {"cd", 1, [](KeeperClient * client, const std::vector & args) - { - auto new_path = client->getAbsolutePath(args[1]); - if (!client->zookeeper->exists(new_path)) - std::cerr << "Path " << new_path << " does not exists\n"; - else - client->cwd = new_path; - }}, - - {"rm", 1, [](KeeperClient * client, const std::vector & args) - { - client->zookeeper->remove(client->getAbsolutePath(args[1])); - }}, - - {"rmr", 1, [](KeeperClient * client, const std::vector & args) - { - String path = client->getAbsolutePath(args[1]); - client->askConfirmation("You are going to recursively delete path " + path, - [client, path]{ client->zookeeper->removeRecursive(path); }); - }}, + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), }); String home_path; @@ -234,32 +187,31 @@ bool KeeperClient::processQueryText(const String & text) if (exit_strings.find(text) != exit_strings.end()) return false; - std::vector tokens; - boost::algorithm::split(tokens, text, boost::is_any_of(" ")); - try { if (need_confirmation) { need_confirmation = false; - if (tokens.size() == 1 && (tokens[0] == "y" || tokens[0] == "Y")) + if (text.size() == 1 && (text == "y" || text == "Y")) confirmation_callback(); + return true; } - else if (tokens.size() == 1 && tokens[0].size() == 4 && four_letter_word_commands.find(tokens[0]) != four_letter_word_commands.end()) - std::cout << executeFourLetterCommand(tokens[0]) << "\n"; - else + + KeeperParser parser; + String message; + const char * begin = text.data(); + ASTPtr res = tryParseQuery(parser, begin, begin + text.size(), message, true, "", false, 0, 0, false); + + if (!res) { - auto callback = commands.find({tokens[0], tokens.size() - 1}); - if (callback == commands.end()) - { - if (tokens[0].size() == 4 && tokens.size() == 1) /// Treat it like unrecognized four-letter command - std::cout << executeFourLetterCommand(tokens[0]) << "\n"; - else - std::cerr << "No command found with name " << tokens[0] << " and args count " << tokens.size() - 1 << "\n"; - } - else - callback->second(this, tokens); + std::cerr << message << "\n"; + return true; } + + auto * query = res->as(); + + auto command = KeeperClient::commands.find(query->command); + command->second->execute(query, this); } catch (Coordination::Exception & err) { diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 50a8b35f6c5..0297491bd28 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -1,10 +1,12 @@ #pragma once - +#include "Parser.h" +#include "Commands.h" #include #include #include #include +#include #include #include #include @@ -13,13 +15,16 @@ namespace DB { -class KeeperClient; +static const NameSet four_letter_word_commands + { + "ruok", "mntr", "srvr", "stat", "srst", "conf", + "cons", "crst", "envi", "dirs", "isro", "wchs", + "wchc", "wchp", "dump", "csnp", "lgif", "rqld", + }; class KeeperClient: public Poco::Util::Application { public: - using Callback = std::function &)>; - KeeperClient() = default; void initialize(Poco::Util::Application & self) override; @@ -28,29 +33,31 @@ public: void defineOptions(Poco::Util::OptionSet & options) override; -protected: - void runInteractive(); - void loadCommands(std::vector> && new_commands); - bool processQueryText(const String & text); - void executeQuery(const String & query); + String getAbsolutePath(const String & relative) const; + + void askConfirmation(const String & prompt, std::function && callback); String executeFourLetterCommand(const String & command); - String getAbsolutePath(const String & relative); - void askConfirmation(const String & prompt, std::function && callback); + zkutil::ZooKeeperPtr zookeeper; + std::filesystem::path cwd = "/"; + std::function confirmation_callback; - std::map, Callback> commands; + inline static std::map commands; + +protected: + void runInteractive(); + bool processQueryText(const String & text); + void executeQuery(const String & query); + + void loadCommands(std::vector && new_commands); String history_file; LineReader::Suggest suggest; - zkutil::ZooKeeperPtr zookeeper; zkutil::ZooKeeperArgs zk_args; - std::filesystem::path cwd = "/"; - bool need_confirmation = false; - std::function confirmation_callback; }; } diff --git a/programs/keeper-client/Parser.cpp b/programs/keeper-client/Parser.cpp new file mode 100644 index 00000000000..0f3fc39704e --- /dev/null +++ b/programs/keeper-client/Parser.cpp @@ -0,0 +1,94 @@ +#include "Parser.h" +#include "KeeperClient.h" + + +namespace DB +{ + +bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result) +{ + expected.add(pos, getTokenName(TokenType::BareWord)); + + if (pos->type == TokenType::BareWord) + { + result = String(pos->begin, pos->end); + ++pos; + ParserToken{TokenType::Whitespace}.ignore(pos); + return true; + } + + bool status = parseIdentifierOrStringLiteral(pos, expected, result); + ParserToken{TokenType::Whitespace}.ignore(pos); + return status; +} + +bool parseKeeperPath(IParser::Pos & pos, Expected & expected, String & path) +{ + expected.add(pos, "path"); + + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) + return parseIdentifierOrStringLiteral(pos, expected, path); + + String result; + while (pos->type == TokenType::BareWord || pos->type == TokenType::Slash || pos->type == TokenType::Dot) + { + result.append(pos->begin, pos->end); + ++pos; + } + ParserToken{TokenType::Whitespace}.ignore(pos); + + if (result.empty()) + return false; + + path = result; + return true; +} + +bool KeeperParser::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + auto query = std::make_shared(); + + for (const auto & pair : KeeperClient::commands) + expected.add(pos, pair.first.data()); + + for (const auto & flwc : four_letter_word_commands) + expected.add(pos, flwc.data()); + + if (pos->type != TokenType::BareWord) + return false; + + String command_name(pos->begin, pos->end); + Command command; + + auto iter = KeeperClient::commands.find(command_name); + if (iter == KeeperClient::commands.end()) + { + if (command_name.size() == 4) + { + /// Treat it like four-letter command + /// Since keeper server can potentially have different version we don't want to match this command with embedded list + command = std::make_shared(); + command_name = command->getName(); + /// We also don't move the position, so the command will be parsed as an argument + } + else + return false; + } + else + { + command = iter->second; + ++pos; + ParserToken{TokenType::Whitespace}.ignore(pos); + } + + query->command = command_name; + if (!command->parse(pos, query, expected)) + return false; + + ParserToken{TokenType::Whitespace}.ignore(pos); + + node = query; + return true; +} + +} diff --git a/programs/keeper-client/Parser.h b/programs/keeper-client/Parser.h new file mode 100644 index 00000000000..57ee6ce4a18 --- /dev/null +++ b/programs/keeper-client/Parser.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result); + +bool parseKeeperPath(IParser::Pos & pos, Expected & expected, String & path); + + +class ASTKeeperQuery : public IAST +{ +public: + String getID(char) const override { return "KeeperQuery"; } + ASTPtr clone() const override { return std::make_shared(*this); } + + String command; + std::vector args; +}; + +class KeeperParser : public IParserBase +{ +protected: + const char * getName() const override { return "Keeper client query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index 6633ddb9563..6b798f6f576 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -4,7 +4,7 @@ namespace DB { -Tokens::Tokens(const char * begin, const char * end, size_t max_query_size) +Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool skipp_insignificant) { Lexer lexer(begin, end, max_query_size); @@ -13,7 +13,7 @@ Tokens::Tokens(const char * begin, const char * end, size_t max_query_size) { Token token = lexer.nextToken(); stop = token.isEnd() || token.type == TokenType::ErrorMaxQuerySizeExceeded; - if (token.isSignificant()) + if (token.isSignificant() || (!skipp_insignificant && !data.empty() && data.back().isSignificant())) data.emplace_back(std::move(token)); } while (!stop); } diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index c9ac61dfef9..31cb644d879 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -24,7 +24,7 @@ private: std::size_t last_accessed_index = 0; public: - Tokens(const char * begin, const char * end, size_t max_query_size = 0); + Tokens(const char * begin, const char * end, size_t max_query_size = 0, bool skipp_insignificant = true); ALWAYS_INLINE inline const Token & operator[](size_t index) { diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 8d794409f78..9f688f204a2 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -233,10 +233,11 @@ ASTPtr tryParseQuery( const std::string & query_description, bool allow_multi_statements, size_t max_query_size, - size_t max_parser_depth) + size_t max_parser_depth, + bool skipp_insignificant) { const char * query_begin = _out_query_end; - Tokens tokens(query_begin, all_queries_end, max_query_size); + Tokens tokens(query_begin, all_queries_end, max_query_size, skipp_insignificant); /// NOTE: consider use UInt32 for max_parser_depth setting. IParser::Pos token_iterator(tokens, static_cast(max_parser_depth)); diff --git a/src/Parsers/parseQuery.h b/src/Parsers/parseQuery.h index cc077bbdab2..30f43261103 100644 --- a/src/Parsers/parseQuery.h +++ b/src/Parsers/parseQuery.h @@ -18,7 +18,8 @@ ASTPtr tryParseQuery( bool allow_multi_statements, /// If false, check for non-space characters after semicolon and set error message if any. size_t max_query_size, /// If (end - pos) > max_query_size and query is longer than max_query_size then throws "Max query size exceeded". /// Disabled if zero. Is used in order to check query size if buffer can contains data for INSERT query. - size_t max_parser_depth); + size_t max_parser_depth, + bool skipp_insignificant = true); /// If true, lexer will skip all insignificant tokens (e.g. whitespaces) /// Parse query or throw an exception with error message. From a20f8e74a0cbf0b72ee657b3c6c302d16c61952e Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 01:05:51 -0400 Subject: [PATCH 0130/2223] Fix `cd` command. Style consistency --- programs/keeper-client/Commands.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index a21550e969d..7b7c4670b7d 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -42,7 +42,7 @@ bool CDCommand::parse(IParser::Pos & pos, std::shared_ptr & node void CDCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const { - if (!query->args.empty()) + if (query->args.empty()) return; auto new_path = client->getAbsolutePath(query->args[0].safeGet()); @@ -158,7 +158,7 @@ bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptrgetHelpMessage() << '\n'; + std::cout << pair.second->getHelpMessage() << "\n"; } bool FourLetterWordCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const From 2d2483d695f39fd8488e3667d77faaaa4177bd92 Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Mon, 24 Apr 2023 21:50:40 +0300 Subject: [PATCH 0131/2223] Rename DatabaseFileSystem to DatabaseFilesystem --- src/Databases/DatabaseFactory.cpp | 16 +++++++------- ...eFileSystem.cpp => DatabaseFilesystem.cpp} | 22 +++++++++---------- ...abaseFileSystem.h => DatabaseFilesystem.h} | 8 +++---- src/Databases/DatabasesOverlay.cpp | 4 ++-- ...cal_implicit_file_table_function.reference | 4 ++-- ...ouse_local_implicit_file_table_function.sh | 8 +++---- 6 files changed, 31 insertions(+), 31 deletions(-) rename src/Databases/{DatabaseFileSystem.cpp => DatabaseFilesystem.cpp} (82%) rename src/Databases/{DatabaseFileSystem.h => DatabaseFilesystem.h} (83%) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 3356689d892..9950ab5bf45 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -3,11 +3,11 @@ #include #include #include +#include #include #include #include #include -#include #include #include #include @@ -15,10 +15,10 @@ #include #include #include -#include -#include #include +#include #include +#include #include "config.h" @@ -133,13 +133,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String static const std::unordered_set database_engines{"Ordinary", "Atomic", "Memory", "Dictionary", "Lazy", "Replicated", "MySQL", "MaterializeMySQL", "MaterializedMySQL", - "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "FileSystem"}; + "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem"}; if (!database_engines.contains(engine_name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine name `{}` does not exist", engine_name); static const std::unordered_set engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL", - "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "FileSystem"}; + "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem"}; static const std::unordered_set engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"}; bool engine_may_have_arguments = engines_with_arguments.contains(engine_name); @@ -433,7 +433,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared(context, engine_define, create.attach, database_path); } #endif - else if (engine_name == "FileSystem") + else if (engine_name == "Filesystem") { const ASTFunction * engine = engine_define->engine; @@ -443,13 +443,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine->arguments && !engine->arguments->children.empty()) { if (engine->arguments->children.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "FileSystem database requires at most 1 argument: file_system_path"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filesystem database requires at most 1 argument: filesystem_path"); const auto & arguments = engine->arguments->children; init_path = safeGetLiteralValue(arguments[0], engine_name); } - return std::make_shared(database_name, init_path, context); + return std::make_shared(database_name, init_path, context); } throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", engine_name); diff --git a/src/Databases/DatabaseFileSystem.cpp b/src/Databases/DatabaseFilesystem.cpp similarity index 82% rename from src/Databases/DatabaseFileSystem.cpp rename to src/Databases/DatabaseFilesystem.cpp index 8b92ad8080a..177b4717716 100644 --- a/src/Databases/DatabaseFileSystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -17,25 +17,25 @@ namespace DB { -DatabaseFileSystem::DatabaseFileSystem(const String & name_, const String & path_, ContextPtr context_) +DatabaseFilesystem::DatabaseFilesystem(const String & name_, const String & path_, ContextPtr context_) : IDatabase(name_), WithContext(context_->getGlobalContext()), path(path_), log(&Poco::Logger::get("DatabaseFileSystem(" + name_ + ")")) { if (path.empty()) path = Poco::Path::current(); } -std::string DatabaseFileSystem::getTablePath(const std::string& table_name) const +std::string DatabaseFilesystem::getTablePath(const std::string& table_name) const { return Poco::Path(path, table_name).toString(); } -void DatabaseFileSystem::addTable(const std::string& table_name, StoragePtr table_storage) const +void DatabaseFilesystem::addTable(const std::string& table_name, StoragePtr table_storage) const { std::lock_guard lock(mutex); loaded_tables.emplace(table_name, table_storage); } -bool DatabaseFileSystem::isTableExist(const String & name, ContextPtr) const +bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr) const { { std::lock_guard lock(mutex); @@ -47,7 +47,7 @@ bool DatabaseFileSystem::isTableExist(const String & name, ContextPtr) const return table_file.exists() && table_file.isFile(); } -StoragePtr DatabaseFileSystem::tryGetTable(const String & name, ContextPtr context_) const +StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const { // Check if the table exists in the loaded tables map { @@ -85,12 +85,12 @@ StoragePtr DatabaseFileSystem::tryGetTable(const String & name, ContextPtr conte } } -ASTPtr DatabaseFileSystem::getCreateDatabaseQuery() const +ASTPtr DatabaseFilesystem::getCreateDatabaseQuery() const { auto settings = getContext()->getSettingsRef(); ParserCreateQuery parser; - String query = "CREATE DATABASE " + backQuoteIfNeed(getDatabaseName()) + " ENGINE = FileSystem(" + backQuoteIfNeed(path) + ")"; + const String query = fmt::format("CREATE DATABASE {} ENGINE = Filesystem({})", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(path)); ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth); if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) @@ -102,7 +102,7 @@ ASTPtr DatabaseFileSystem::getCreateDatabaseQuery() const return ast; } -void DatabaseFileSystem::shutdown() +void DatabaseFilesystem::shutdown() { Tables tables_snapshot; { @@ -123,7 +123,7 @@ void DatabaseFileSystem::shutdown() /** * Returns an empty vector because the database is read-only and no tables can be backed up. */ -std::vector> DatabaseFileSystem::getTablesForBackup(const FilterByNameFunction&, const ContextPtr&) const +std::vector> DatabaseFilesystem::getTablesForBackup(const FilterByNameFunction&, const ContextPtr&) const { return {}; } @@ -133,7 +133,7 @@ std::vector> DatabaseFileSystem::getTablesForBacku * Returns an empty iterator because the database does not have its own tables * But only caches them for quick access. */ -DatabaseTablesIteratorPtr DatabaseFileSystem::getTablesIterator(ContextPtr, const FilterByNameFunction&) const +DatabaseTablesIteratorPtr DatabaseFilesystem::getTablesIterator(ContextPtr, const FilterByNameFunction&) const { return std::make_unique(Tables{}, getDatabaseName()); } diff --git a/src/Databases/DatabaseFileSystem.h b/src/Databases/DatabaseFilesystem.h similarity index 83% rename from src/Databases/DatabaseFileSystem.h rename to src/Databases/DatabaseFilesystem.h index 474a7e78335..d5fdd528aa5 100644 --- a/src/Databases/DatabaseFileSystem.h +++ b/src/Databases/DatabaseFilesystem.h @@ -12,18 +12,18 @@ namespace DB class Context; /** - * DatabaseFileSystem allows to interact with files stored on the file system + * DatabaseFilesystem allows to interact with files stored on the file system * Uses TableFunctionFile to implicitly load file when a user requests the table, and provides read-only access to the data in the file * Tables are cached inside the database for quick access * * Used in clickhouse-local to access local files */ -class DatabaseFileSystem : public IDatabase, protected WithContext +class DatabaseFilesystem : public IDatabase, protected WithContext { public: - DatabaseFileSystem(const String & name, const String & path, ContextPtr context); + DatabaseFilesystem(const String & name, const String & path, ContextPtr context); - String getEngineName() const override { return "FileSystem"; } + String getEngineName() const override { return "Filesystem"; } bool isTableExist(const String & name, ContextPtr context) const override; diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index da26f9282a0..3563fa715a6 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -4,8 +4,8 @@ #include #include +#include #include -#include #include @@ -258,7 +258,7 @@ DatabaseTablesIteratorPtr DatabasesOverlay::getTablesIterator(ContextPtr context DatabasePtr CreateClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context_) { auto databaseCombiner = std::make_shared(name_, context_); - databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); + databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); databaseCombiner->registerNextDatabase(std::make_shared(name_, context_)); return databaseCombiner; } diff --git a/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.reference b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.reference index 0fcd843e737..ccc02ad4f34 100644 --- a/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.reference +++ b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.reference @@ -3,7 +3,7 @@ explicit: 4 implicit: 4 -Test 2: check FileSystem database +Test 2: check Filesystem database 4 -Test 3: check show database with FileSystem +Test 3: check show database with Filesystem test02707 diff --git a/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh index 24de0ad579c..7c9095b3d8b 100755 --- a/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh +++ b/tests/queries/0_stateless/02707_clickhouse_local_implicit_file_table_function.sh @@ -24,19 +24,19 @@ echo "implicit:" $CLICKHOUSE_LOCAL -q "SELECT COUNT(*) FROM \"${dir}/tmp.csv\"" ################# -echo "Test 2: check FileSystem database" +echo "Test 2: check Filesystem database" $CLICKHOUSE_LOCAL --multiline --multiquery -q """ DROP DATABASE IF EXISTS test; -CREATE DATABASE test ENGINE = FileSystem('${dir}'); +CREATE DATABASE test ENGINE = Filesystem('${dir}'); SELECT COUNT(*) FROM test.\`tmp.csv\`; DROP DATABASE test; """ ################# -echo "Test 3: check show database with FileSystem" +echo "Test 3: check show database with Filesystem" $CLICKHOUSE_LOCAL --multiline --multiquery -q """ DROP DATABASE IF EXISTS test02707; -CREATE DATABASE test02707 ENGINE = FileSystem('${dir}'); +CREATE DATABASE test02707 ENGINE = Filesystem('${dir}'); SHOW DATABASES; DROP DATABASE test02707; """ | grep "test02707" From 79ca39d920fbc52e92f6bbc9496bde2cc1afec42 Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Mon, 24 Apr 2023 22:26:16 +0300 Subject: [PATCH 0132/2223] Fixed exception messages --- src/Databases/DatabasesOverlay.cpp | 42 +++++++++++++++++++++++++----- 1 file changed, 36 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 3563fa715a6..c3af6d9305e 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -65,7 +65,12 @@ void DatabasesOverlay::createTable(ContextPtr context_, const String & table_nam continue; } } - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no databases for CREATE TABLE {} query in Database{}", table_name, getEngineName()); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "There is no databases for CREATE TABLE `{}` query in database `{}` (engine {})", + table_name, + getDatabaseName(), + getEngineName()); } void DatabasesOverlay::dropTable(ContextPtr context_, const String & table_name, bool sync) @@ -82,7 +87,12 @@ void DatabasesOverlay::dropTable(ContextPtr context_, const String & table_name, continue; } } - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no databases for DROP TABLE {} query in Database{}", table_name, getEngineName()); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "There is no databases for DROP TABLE `{}` query in database `{}` (engine {})", + table_name, + getDatabaseName(), + getEngineName()); } void DatabasesOverlay::attachTable( @@ -100,7 +110,12 @@ void DatabasesOverlay::attachTable( continue; } } - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no databases for ATTACH TABLE query in Database{}", getEngineName()); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "There is no databases for ATTACH TABLE `{}` query in database `{}` (engine {})", + table_name, + getDatabaseName(), + getEngineName()); } StoragePtr DatabasesOverlay::detachTable(ContextPtr context_, const String & table_name) @@ -119,7 +134,12 @@ StoragePtr DatabasesOverlay::detachTable(ContextPtr context_, const String & tab continue; } } - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no databases for DETACH TABLE {} query in Database{}", table_name, getEngineName()); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "There is no databases for DETACH TABLE `{}` query in database `{}` (engine {})", + table_name, + getDatabaseName(), + getEngineName()); } ASTPtr DatabasesOverlay::getCreateTableQueryImpl(const String & name, ContextPtr context_, bool throw_on_error) const @@ -132,7 +152,12 @@ ASTPtr DatabasesOverlay::getCreateTableQueryImpl(const String & name, ContextPtr break; } if (!result && throw_on_error) - throw Exception(ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY, "There is no metadata of table {} in Database{}", name, getEngineName()); + throw Exception( + ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY, + "There is no metadata of table `{}` in database `{}` (engine {})", + name, + getDatabaseName(), + getEngineName()); return result; } @@ -201,7 +226,12 @@ void DatabasesOverlay::alterTable(ContextPtr local_context, const StorageID & ta continue; } } - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no databases for alterTable in Database{}", getEngineName()); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "There is no databases for ALTER TABLE `{}` query in database `{}` (engine {})", + table_id.table_name, + getDatabaseName(), + getEngineName()); } std::vector> From c9f8dd8bfd3d4123a0a7111f19d8863b19729d9a Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Mon, 24 Apr 2023 22:53:32 +0300 Subject: [PATCH 0133/2223] Replaced Poco::File with std::filesystem --- src/Databases/DatabaseFactory.cpp | 2 +- src/Databases/DatabaseFilesystem.cpp | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 9950ab5bf45..8a50c31efc8 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -437,7 +437,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String { const ASTFunction * engine = engine_define->engine; - // If init_path is empty, then the current path from Poco will be used + /// If init_path is empty, then the current path will be used std::string init_path; if (engine->arguments && !engine->arguments->children.empty()) diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 177b4717716..1decb273ae1 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -8,11 +8,12 @@ #include #include #include -#include -#include #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -21,12 +22,12 @@ DatabaseFilesystem::DatabaseFilesystem(const String & name_, const String & path : IDatabase(name_), WithContext(context_->getGlobalContext()), path(path_), log(&Poco::Logger::get("DatabaseFileSystem(" + name_ + ")")) { if (path.empty()) - path = Poco::Path::current(); + path = fs::current_path(); } std::string DatabaseFilesystem::getTablePath(const std::string& table_name) const { - return Poco::Path(path, table_name).toString(); + return fs::path(path) / table_name; } void DatabaseFilesystem::addTable(const std::string& table_name, StoragePtr table_storage) const @@ -43,8 +44,8 @@ bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr) const return true; } - Poco::File table_file(getTablePath(name)); - return table_file.exists() && table_file.isFile(); + fs::path table_file_path(getTablePath(name)); + return fs::exists(table_file_path) && fs::is_regular_file(table_file_path); } StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const @@ -62,8 +63,7 @@ StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr conte try { // If the table doesn't exist in the tables map, check if the corresponding file exists - Poco::File table_file(table_path); - if (!table_file.exists()) + if (!fs::exists(table_path) || !fs::is_regular_file(table_path)) return nullptr; // If the file exists, create a new table using TableFunctionFile and return it. From 26812f36fb73ca8a3f1c16a0db54dd4327f7dc6c Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Wed, 26 Apr 2023 01:13:29 +0300 Subject: [PATCH 0134/2223] Added read-only database setting; Fixed error messages for filesystem database; added tests --- src/Databases/DatabaseFilesystem.cpp | 46 ++++++++------- src/Databases/DatabaseFilesystem.h | 8 +++ src/Databases/DatabasesOverlay.cpp | 13 ++--- src/Databases/IDatabase.h | 4 +- src/Interpreters/DatabaseCatalog.cpp | 14 ++++- .../02722_database_filesystem.reference | 12 ++++ .../0_stateless/02722_database_filesystem.sh | 58 +++++++++++++++++++ 7 files changed, 124 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/02722_database_filesystem.reference create mode 100755 tests/queries/0_stateless/02722_database_filesystem.sh diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 1decb273ae1..106885e7c3e 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -21,8 +21,7 @@ namespace DB DatabaseFilesystem::DatabaseFilesystem(const String & name_, const String & path_, ContextPtr context_) : IDatabase(name_), WithContext(context_->getGlobalContext()), path(path_), log(&Poco::Logger::get("DatabaseFileSystem(" + name_ + ")")) { - if (path.empty()) - path = fs::current_path(); + path = fs::path(path).lexically_normal().string(); } std::string DatabaseFilesystem::getTablePath(const std::string& table_name) const @@ -48,7 +47,7 @@ bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr) const return fs::exists(table_file_path) && fs::is_regular_file(table_file_path); } -StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const +StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_) const { // Check if the table exists in the loaded tables map { @@ -60,24 +59,31 @@ StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr conte auto table_path = getTablePath(name); + // If the file exists, create a new table using TableFunctionFile and return it. + auto args = makeASTFunction("file", std::make_shared(table_path)); + + auto table_function = TableFunctionFactory::instance().get(args, context_); + if (!table_function) + return nullptr; + + auto table_storage = table_function->execute(args, context_, name); + if (table_storage) + addTable(name, table_storage); + + return table_storage; +} + +StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_) const +{ + if (auto storage = getTableImpl(name, context_)) + return storage; + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name)); +} + +StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const { try { - // If the table doesn't exist in the tables map, check if the corresponding file exists - if (!fs::exists(table_path) || !fs::is_regular_file(table_path)) - return nullptr; - - // If the file exists, create a new table using TableFunctionFile and return it. - auto args = makeASTFunction("file", std::make_shared(table_path)); - - auto table_function = TableFunctionFactory::instance().get(args, context_); - if (!table_function) - return nullptr; - - auto table_storage = table_function->execute(args, context_, name); - if (table_storage) - addTable(name, table_storage); - - return table_storage; + return getTable(name, context_); } catch (...) { @@ -90,7 +96,7 @@ ASTPtr DatabaseFilesystem::getCreateDatabaseQuery() const auto settings = getContext()->getSettingsRef(); ParserCreateQuery parser; - const String query = fmt::format("CREATE DATABASE {} ENGINE = Filesystem({})", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(path)); + const String query = fmt::format("CREATE DATABASE {} ENGINE = Filesystem('{}')", backQuoteIfNeed(getDatabaseName()), path); ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth); if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) diff --git a/src/Databases/DatabaseFilesystem.h b/src/Databases/DatabaseFilesystem.h index d5fdd528aa5..697511ac5b3 100644 --- a/src/Databases/DatabaseFilesystem.h +++ b/src/Databases/DatabaseFilesystem.h @@ -27,10 +27,14 @@ public: bool isTableExist(const String & name, ContextPtr context) const override; + StoragePtr getTable(const String & name, ContextPtr context) const override; + StoragePtr tryGetTable(const String & name, ContextPtr context) const override; bool empty() const override { return true; } + bool isReadOnly() const override { return true; } + ASTPtr getCreateDatabaseQuery() const override; void shutdown() override; @@ -39,9 +43,13 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; protected: + StoragePtr getTableImpl(const String & name, ContextPtr context) const; + std::string getTablePath(const std::string & table_name) const; + void addTable(const std::string & table_name, StoragePtr table_storage) const; + private: String path; mutable Tables loaded_tables TSA_GUARDED_BY(mutex); diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index c3af6d9305e..5a6a4fe5cc6 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -55,15 +55,11 @@ void DatabasesOverlay::createTable(ContextPtr context_, const String & table_nam { for (auto & db : databases) { - try + if (!db->isReadOnly()) { db->createTable(context_, table_name, table, query); return; } - catch (...) - { - continue; - } } throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -218,8 +214,11 @@ void DatabasesOverlay::alterTable(ContextPtr local_context, const StorageID & ta { try { - db->alterTable(local_context, table_id, metadata); - return; + if (!db->isReadOnly()) + { + db->alterTable(local_context, table_id, metadata); + return; + } } catch (...) { diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 53a2f372814..6508e2ce060 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -170,7 +170,7 @@ public: /// Get the table for work. Return nullptr if there is no table. virtual StoragePtr tryGetTable(const String & name, ContextPtr context) const = 0; - StoragePtr getTable(const String & name, ContextPtr context) const; + virtual StoragePtr getTable(const String & name, ContextPtr context) const; virtual UUID tryGetTableUUID(const String & /*table_name*/) const { return UUIDHelpers::Nil; } @@ -183,6 +183,8 @@ public: /// Is the database empty. virtual bool empty() const = 0; + virtual bool isReadOnly() const { return false; } + /// Add the table to the database. Record its presence in the metadata. virtual void createTable( ContextPtr /*context*/, diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 8d3fa91a7fe..f9e74fadcbd 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -338,9 +338,17 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( database = it->second; } - auto table = database->tryGetTable(table_id.table_name, context_); - if (!table && exception) - exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs())); + StoragePtr table = nullptr; + try + { + table = database->getTable(table_id.table_name, context_); + } + catch (const Exception & e) + { + if (exception) + exception->emplace(*e.clone()); + } + if (!table) database = nullptr; diff --git a/tests/queries/0_stateless/02722_database_filesystem.reference b/tests/queries/0_stateless/02722_database_filesystem.reference new file mode 100644 index 00000000000..a583f1e2e3c --- /dev/null +++ b/tests/queries/0_stateless/02722_database_filesystem.reference @@ -0,0 +1,12 @@ +Test 1: create filesystem database and check implicit calls +0 +test1 +4 +4 +4 +Test 2: check DatabaseFilesystem access rights on server +OK +OK +OK +OK +OK diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh new file mode 100755 index 00000000000..0adeface438 --- /dev/null +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -0,0 +1,58 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# see 01658_read_file_to_stringcolumn.sh +CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +# Prepare data +mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/tmp/ +echo '"id","str","int","text"' > ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv +echo '1,"abc",123,"abacaba"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv +echo '2,"def",456,"bacabaa"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv +echo '3,"story",78912,"acabaab"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv +echo '4,"history",21321321,"cabaaba"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv + +tmp_dir=${CLICKHOUSE_TEST_UNIQUE_NAME} +[[ -d $tmp_dir ]] && rm -rd $tmp_dir +mkdir $tmp_dir +cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${tmp_dir}/tmp.csv +cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp/tmp.csv + +################# +echo "Test 1: create filesystem database and check implicit calls" +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test1; +CREATE DATABASE test1 ENGINE = Filesystem; +""" +echo $? +${CLICKHOUSE_CLIENT} --query "SHOW DATABASES" | grep "test1" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp/tmp.csv\`;" +${CLICKHOUSE_LOCAL} -q "SELECT COUNT(*) FROM \"${tmp_dir}/tmp.csv\"" + +################# +echo "Test 2: check DatabaseFilesystem access rights on server" +# Allows list files only inside user_files +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp.csv\`;" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`/tmp/tmp.csv\`;" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK" + +${CLICKHOUSE_CLIENT} --multiline --multiquery --query """ +USE test1; +SELECT COUNT(*) FROM \"../${tmp_dir}/tmp.csv\"; +""" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../../../../../../tmp.csv\`;" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK" +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test2; +CREATE DATABASE test2 ENGINE = Filesystem('/tmp'); +SELECT COUNT(*) FROM test2.\`tmp.csv\`; +""" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK" + +# Clean +${CLICKHOUSE_CLIENT} --query "DROP DATABASE test1;" +${CLICKHOUSE_CLIENT} --query "DROP DATABASE test2;" +rm -rd $tmp_dir +rm -rd $CLICKHOUSE_USER_FILES_PATH From 4606e660683992b630f9db952beda9b261f82d76 Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Wed, 26 Apr 2023 11:06:01 +0300 Subject: [PATCH 0135/2223] Fix style --- src/Databases/DatabaseFilesystem.cpp | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 106885e7c3e..16aed185669 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -18,18 +18,23 @@ namespace fs = std::filesystem; namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_TABLE; +} + DatabaseFilesystem::DatabaseFilesystem(const String & name_, const String & path_, ContextPtr context_) : IDatabase(name_), WithContext(context_->getGlobalContext()), path(path_), log(&Poco::Logger::get("DatabaseFileSystem(" + name_ + ")")) { path = fs::path(path).lexically_normal().string(); } -std::string DatabaseFilesystem::getTablePath(const std::string& table_name) const +std::string DatabaseFilesystem::getTablePath(const std::string & table_name) const { return fs::path(path) / table_name; } -void DatabaseFilesystem::addTable(const std::string& table_name, StoragePtr table_storage) const +void DatabaseFilesystem::addTable(const std::string & table_name, StoragePtr table_storage) const { std::lock_guard lock(mutex); loaded_tables.emplace(table_name, table_storage); @@ -80,7 +85,8 @@ StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_ throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name)); } -StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const { +StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const +{ try { return getTable(name, context_); @@ -127,9 +133,9 @@ void DatabaseFilesystem::shutdown() } /** - * Returns an empty vector because the database is read-only and no tables can be backed up. + * Returns an empty vector because the database is read-only and no tables can be backed up */ -std::vector> DatabaseFilesystem::getTablesForBackup(const FilterByNameFunction&, const ContextPtr&) const +std::vector> DatabaseFilesystem::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const { return {}; } @@ -137,9 +143,9 @@ std::vector> DatabaseFilesystem::getTablesForBacku /** * * Returns an empty iterator because the database does not have its own tables - * But only caches them for quick access. + * But only caches them for quick access */ -DatabaseTablesIteratorPtr DatabaseFilesystem::getTablesIterator(ContextPtr, const FilterByNameFunction&) const +DatabaseTablesIteratorPtr DatabaseFilesystem::getTablesIterator(ContextPtr, const FilterByNameFunction &) const { return std::make_unique(Tables{}, getDatabaseName()); } From ca1501aeb4e9c7a1db131f4c24255bd24bd99059 Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Wed, 26 Apr 2023 13:05:56 +0300 Subject: [PATCH 0136/2223] retrigger checks From bacba6e34735be07a37283224326172aa1e9a71b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 26 Apr 2023 12:18:12 +0200 Subject: [PATCH 0137/2223] Fix typo --- src/Formats/ReadSchemaUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index f80d9ee41d2..61683b226ee 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -203,7 +203,7 @@ ColumnsDescription readSchemaFromFormat( ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference. You must specify the structure manually", format_name); - /// Some formats like CSVWithNames can contain empty column names. We don't support empty column names and futher processing can fail with an exception. Let's just remove columns with empty names from the structure. + /// Some formats like CSVWithNames can contain empty column names. We don't support empty column names and further processing can fail with an exception. Let's just remove columns with empty names from the structure. names_and_types.erase( std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }), names_and_types.end()); From 1f90e9bde8ab740ae5fda958ca93f9c4abab6008 Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Wed, 26 Apr 2023 14:37:41 +0300 Subject: [PATCH 0138/2223] retrigger checks From 2426c445b0f17a0c98be86463efda8bd552d18de Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 20:10:32 -0400 Subject: [PATCH 0139/2223] Improve `set` command --- programs/keeper-client/Commands.cpp | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 7b7c4670b7d..05928a0d20b 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -24,8 +24,7 @@ void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con else path = client->cwd; - const auto children = client->zookeeper->getChildren(path); - for (const auto & child : children) + for (const auto & child : client->zookeeper->getChildren(path)) std::cout << child << " "; std::cout << "\n"; } @@ -92,6 +91,19 @@ bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr & return false; node->args.push_back(std::move(arg)); + int mode = zkutil::CreateMode::Persistent; + + if (ParserKeyword{"PERSISTENT"}.ignore(pos, expected)) + mode = zkutil::CreateMode::Persistent; + else if (ParserKeyword{"EPHEMERAL"}.ignore(pos, expected)) + mode = zkutil::CreateMode::Ephemeral; + else if (ParserKeyword{"EPHEMERAL SEQUENTIAL"}.ignore(pos, expected)) + mode = zkutil::CreateMode::EphemeralSequential; + else if (ParserKeyword{"PERSISTENT SEQUENTIAL"}.ignore(pos, expected)) + mode = zkutil::CreateMode::PersistentSequential; + + node->args.push_back(mode); + return true; } @@ -100,7 +112,7 @@ void CreateCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) client->zookeeper->create( client->getAbsolutePath(query->args[0].safeGet()), query->args[1].safeGet(), - zkutil::CreateMode::Persistent); + static_cast(query->args[2].safeGet())); } bool GetCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const From 9204e2e3de85563ecc8d24ac356e608f95880c6b Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 20:11:45 -0400 Subject: [PATCH 0140/2223] Better completion --- programs/keeper-client/KeeperClient.cpp | 56 +++++++++++++++++++++---- programs/keeper-client/KeeperClient.h | 8 +++- src/Client/LineReader.cpp | 21 ++++++---- src/Client/LineReader.h | 5 +++ 4 files changed, 75 insertions(+), 15 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 92aa822231d..0ea4c4dde28 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -10,7 +10,6 @@ namespace po = boost::program_options; -namespace fs = std::filesystem; namespace DB { @@ -42,6 +41,48 @@ String KeeperClient::executeFourLetterCommand(const String & command) return result; } +std::vector KeeperClient::getCompletions(const String & prefix) const +{ + Tokens tokens(prefix.data(), prefix.data() + prefix.size(), 0, false); + IParser::Pos pos(tokens, 0); + + if (pos->type != TokenType::BareWord) + return registered_commands_and_four_letter_words; + + ++pos; + if (pos->isEnd()) + return registered_commands_and_four_letter_words; + + ParserToken{TokenType::Whitespace}.ignore(pos); + + std::vector result; + String string_path; + Expected expected; + if (!parseKeeperPath(pos, expected, string_path)) + string_path = cwd; + + if (!pos->isEnd()) + return result; + + fs::path path = string_path; + String parent_path; + if (string_path.ends_with("/")) + parent_path = getAbsolutePath(string_path); + else + parent_path = getAbsolutePath(path.parent_path()); + + try + { + for (const auto & child : zookeeper->getChildren(parent_path)) + result.push_back(child); + } + catch (Coordination::Exception &) {} + + std::sort(result.begin(), result.end()); + + return result; +} + void KeeperClient::askConfirmation(const String & prompt, std::function && callback) { std::cout << prompt << " Continue?\n"; @@ -49,7 +90,7 @@ void KeeperClient::askConfirmation(const String & prompt, std::function confirmation_callback = callback; } -String KeeperClient::getAbsolutePath(const String & relative) const +fs::path KeeperClient::getAbsolutePath(const String & relative) const { String result; if (relative.starts_with('/')) @@ -65,18 +106,17 @@ String KeeperClient::getAbsolutePath(const String & relative) const void KeeperClient::loadCommands(std::vector && new_commands) { - std::vector suggestions; for (const auto & command : new_commands) { String name = command->getName(); commands.insert({name, command}); - suggestions.push_back(std::move(name)); + registered_commands_and_four_letter_words.push_back(std::move(name)); } for (const auto & command : four_letter_word_commands) - suggestions.push_back(command); + registered_commands_and_four_letter_words.push_back(command); - suggest.addWords(std::move(suggestions)); + std::sort(registered_commands_and_four_letter_words.begin(), registered_commands_and_four_letter_words.end()); } void KeeperClient::defineOptions(Poco::Util::OptionSet & options) @@ -130,6 +170,9 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) void KeeperClient::initialize(Poco::Util::Application & /* self */) { + suggest.setCompletionsCallback( + [&](const String & prefix, size_t /* prefix_length */) { return getCompletions(prefix); }); + loadCommands({ std::make_shared(), std::make_shared(), @@ -248,7 +291,6 @@ void KeeperClient::runInteractive() int KeeperClient::main(const std::vector & /* args */) { - auto host = config().getString("host", "localhost"); auto port = config().getString("port", "2181"); zk_args.hosts = {host + ":" + port}; diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 0297491bd28..e7fa5cdc5fe 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -12,6 +12,8 @@ #include +namespace fs = std::filesystem; + namespace DB { @@ -33,7 +35,7 @@ public: void defineOptions(Poco::Util::OptionSet & options) override; - String getAbsolutePath(const String & relative) const; + fs::path getAbsolutePath(const String & relative) const; void askConfirmation(const String & prompt, std::function && callback); @@ -52,12 +54,16 @@ protected: void loadCommands(std::vector && new_commands); + std::vector getCompletions(const String & prefix) const; + String history_file; LineReader::Suggest suggest; zkutil::ZooKeeperArgs zk_args; bool need_confirmation = false; + + std::vector registered_commands_and_four_letter_words; }; } diff --git a/src/Client/LineReader.cpp b/src/Client/LineReader.cpp index 04b387c9f7d..f6cd7bffef7 100644 --- a/src/Client/LineReader.cpp +++ b/src/Client/LineReader.cpp @@ -81,17 +81,24 @@ replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String & std::lock_guard lock(mutex); + Words to_search; /// Only perform case sensitive completion when the prefix string contains any uppercase characters if (std::none_of(prefix.begin(), prefix.end(), [](char32_t x) { return iswupper(static_cast(x)); })) - range = std::equal_range( - words_no_case.begin(), words_no_case.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) - { - return strncasecmp(s.data(), prefix_searched.data(), prefix_length) < 0; - }); + to_search = words_no_case; else - range = std::equal_range(words.begin(), words.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) + to_search = words; + + if (custom_completions_callback) + { + auto new_words = custom_completions_callback(prefix, prefix_length); + assert(std::is_sorted(new_words.begin(), new_words.end())); + addNewWords(to_search, new_words, std::less{}); + } + + range = std::equal_range( + to_search.begin(), to_search.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) { - return strncmp(s.data(), prefix_searched.data(), prefix_length) < 0; + return strncasecmp(s.data(), prefix_searched.data(), prefix_length) < 0; }); return replxx::Replxx::completions_t(range.first, range.second); diff --git a/src/Client/LineReader.h b/src/Client/LineReader.h index 321cf41b77e..7d053df1458 100644 --- a/src/Client/LineReader.h +++ b/src/Client/LineReader.h @@ -18,15 +18,20 @@ public: struct Suggest { using Words = std::vector; + using Callback = std::function; /// Get vector for the matched range of words if any. replxx::Replxx::completions_t getCompletions(const String & prefix, size_t prefix_length); void addWords(Words && new_words); + void setCompletionsCallback(Callback && callback) { custom_completions_callback = callback; } + private: Words words TSA_GUARDED_BY(mutex); Words words_no_case TSA_GUARDED_BY(mutex); + Callback custom_completions_callback = nullptr; + std::mutex mutex; }; From d235fdd5722c724fc7824ae0f7336c053b3d7084 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Apr 2023 20:39:33 -0400 Subject: [PATCH 0141/2223] Produce help message --- programs/keeper-client/KeeperClient.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 0ea4c4dde28..f38da1b72aa 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -6,11 +6,9 @@ #include #include #include -#include +#include -namespace po = boost::program_options; - namespace DB { @@ -291,6 +289,15 @@ void KeeperClient::runInteractive() int KeeperClient::main(const std::vector & /* args */) { + if (config().hasOption("help")) + { + Poco::Util::HelpFormatter help_formatter(KeeperClient::options()); + auto header_str = fmt::format("{} [OPTION]\n", commandName()); + help_formatter.setHeader(header_str); + help_formatter.format(std::cout); + return 0; + } + auto host = config().getString("host", "localhost"); auto port = config().getString("port", "2181"); zk_args.hosts = {host + ":" + port}; From de03b905709bb6da3196b1b76fcbe60c14652a6e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 27 Apr 2023 17:07:00 +0000 Subject: [PATCH 0142/2223] impl --- src/Core/Settings.h | 2 + ...chronousReadIndirectBufferFromRemoteFS.cpp | 26 +++++---- ...ynchronousReadIndirectBufferFromRemoteFS.h | 9 ++-- .../IO/CachedOnDiskReadBufferFromFile.cpp | 19 ++++++- src/Disks/IO/ThreadPoolRemoteFSReader.cpp | 2 +- src/Interpreters/Cache/FileCache.cpp | 38 ++++++++++--- src/Interpreters/Cache/FileCache.h | 4 +- src/Interpreters/Cache/FileCacheSettings.cpp | 2 + src/Interpreters/Cache/FileCacheSettings.h | 3 ++ src/Interpreters/Cache/FileCache_fwd.h | 2 +- src/Interpreters/Cache/FileSegment.cpp | 2 + .../FilesystemReadPrefetchesLog.cpp | 8 +-- .../FilesystemReadPrefetchesLog.h | 2 +- src/Interpreters/executeQuery.cpp | 2 + .../tests/gtest_lru_file_cache.cpp | 53 ++++++++++--------- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 3 ++ src/Storages/MergeTree/MergeTreeReadPool.cpp | 42 ++++++++++++--- src/Storages/MergeTree/MergeTreeReadPool.h | 5 +- src/Storages/MergeTree/MergeTreeSource.cpp | 2 + 19 files changed, 160 insertions(+), 66 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7f1fe838b80..465e27b0985 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -645,6 +645,8 @@ class IColumn; M(UInt64, merge_tree_min_rows_for_concurrent_read_for_remote_filesystem, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \ M(UInt64, merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \ M(UInt64, remote_read_min_bytes_for_seek, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes required for remote read (url, s3) to do seek, instead of read with ignore.", 0) \ + M(UInt64, merge_tree_min_bytes_per_task_for_remote_reading, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes to read per task.", 0) \ + M(Bool, merge_tree_use_const_size_tasks_for_remote_reading, true, "Whether to use constant size tasks for reading from a remote table.", 0) \ \ M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background. Makes sense only for inserts via HTTP protocol. If wait_for_async_insert is false, INSERT query is processed almost instantly, otherwise client will wait until data will be flushed to table", 0) \ diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 9448bbaf798..1dbe22a431f 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -42,21 +42,25 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; } +static size_t chooseBufferSize(const ReadSettings & settings, size_t file_size) +{ + /// Buffers used for prefetch or pre-download better to have enough size, but not bigger than the whole file. + return std::min(std::max(settings.prefetch_buffer_size, DBMS_DEFAULT_BUFFER_SIZE), file_size); +} AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS( - IAsynchronousReader & reader_, - const ReadSettings & settings_, - std::shared_ptr impl_, - size_t min_bytes_for_seek_) - : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0) + IAsynchronousReader & reader_, + const ReadSettings & settings_, + std::shared_ptr impl_, + size_t min_bytes_for_seek_) + : ReadBufferFromFileBase(chooseBufferSize(settings_, impl_->getFileSize()), nullptr, 0) , read_settings(settings_) , reader(reader_) , base_priority(settings_.priority) , impl(impl_) - , prefetch_buffer(settings_.prefetch_buffer_size) + , prefetch_buffer(chooseBufferSize(settings_, impl->getFileSize())) , min_bytes_for_seek(min_bytes_for_seek_) - , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr - ? CurrentThread::getQueryId() : "") + , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") , current_reader_id(getRandomASCIIString(8)) #ifndef NDEBUG , log(&Poco::Logger::get("AsynchronousBufferFromRemoteFS")) @@ -135,11 +139,11 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch(int64_t priority) if (!hasPendingDataToRead()) return; - last_prefetch_info.submit_time = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + last_prefetch_info.submit_time = std::chrono::system_clock::now(); last_prefetch_info.priority = priority; /// Prefetch even in case hasPendingData() == true. - chassert(prefetch_buffer.size() == read_settings.prefetch_buffer_size || prefetch_buffer.size() == read_settings.remote_fs_buffer_size); + chassert(prefetch_buffer.size() == chooseBufferSize(read_settings, impl->getFileSize()) || prefetch_buffer.size() == read_settings.remote_fs_buffer_size); prefetch_future = asyncReadInto(prefetch_buffer.data(), prefetch_buffer.size(), priority); ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); } @@ -224,7 +228,7 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() { ProfileEventTimeIncrement watch(ProfileEvents::SynchronousRemoteReadWaitMicroseconds); - chassert(memory.size() == read_settings.prefetch_buffer_size || memory.size() == read_settings.remote_fs_buffer_size); + chassert(memory.size() == chooseBufferSize(read_settings, impl->getFileSize()) || memory.size() == read_settings.remote_fs_buffer_size); std::tie(size, offset) = impl->readInto(memory.data(), memory.size(), file_offset_of_buffer_end, bytes_to_ignore); bytes_to_ignore = 0; diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index 8cb0e2826b4..e72bbe15e5b 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -1,11 +1,12 @@ #pragma once -#include "config.h" -#include +#include +#include #include +#include #include #include -#include +#include "config.h" namespace Poco { class Logger; } @@ -97,7 +98,7 @@ private: struct LastPrefetchInfo { - UInt64 submit_time = 0; + std::chrono::system_clock::time_point submit_time; size_t priority = 0; }; LastPrefetchInfo last_prefetch_info; diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 68efd3f5d78..66a4d51abaa 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -48,7 +48,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( bool allow_seeks_after_first_read_, bool use_external_buffer_, std::optional read_until_position_) - : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0, file_size_) + : ReadBufferFromFileBase(use_external_buffer_ ? 0 : settings_.remote_fs_buffer_size, nullptr, 0, file_size_) #ifndef NDEBUG , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile(" + source_file_path_ + ")")) #else @@ -120,7 +120,7 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) else { CreateFileSegmentSettings create_settings(is_persistent ? FileSegmentKind::Persistent : FileSegmentKind::Regular); - file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings)); + file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, file_size.value(), create_settings)); } /** @@ -150,6 +150,8 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm ReadSettings local_read_settings{settings}; /// Do not allow to use asynchronous version of LocalFSReadMethod. local_read_settings.local_fs_method = LocalFSReadMethod::pread; + if (use_external_buffer) + local_read_settings.local_fs_buffer_size = 0; // The buffer will unnecessarily allocate a Memory of size local_fs_buffer_size, which will then // most likely be unused because we're swap()ping our own internal_buffer into @@ -538,6 +540,9 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) ProfileEvents::FileSegmentPredownloadMicroseconds, predownload_watch.elapsedMicroseconds()); }); + OpenTelemetry::SpanHolder span{ + fmt::format("CachedOnDiskReadBufferFromFile::predownload(key={}, size={})", file_segment->key().toString(), bytes_to_predownload)}; + if (bytes_to_predownload) { /// Consider this case. Some user needed segment [a, b] and downloaded it partially. @@ -806,6 +811,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (current_file_segment_it == file_segments_holder->file_segments.end()) return false; + const size_t original_buffer_size = internal_buffer.size(); + bool implementation_buffer_can_be_reused = false; SCOPE_EXIT({ try @@ -831,6 +838,9 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } } + if (use_external_buffer && initialized) + internal_buffer.resize(original_buffer_size); + chassert(!file_segment->isDownloader()); } catch (...) @@ -857,6 +867,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert(!internal_buffer.empty()); + /// We allocate buffers not less than 1M so that s3 requests will not be too small. But the same buffers (members of AsynchronousReadIndirectBufferFromRemoteFS) + /// are used for reading from files. Some of these readings are fairly small and their performance degrade when we use big buffers (up to ~20% for queries like Q23 from ClickBench). + if (use_external_buffer && read_type == ReadType::CACHED && settings.local_fs_buffer_size < internal_buffer.size()) + internal_buffer.resize(settings.local_fs_buffer_size); + // Pass a valid external buffer for implementation_buffer to read into. // We then take it back with another swap() after reading is done. // (If we get an exception in between, we'll be left with an invalid internal_buffer. That's ok, as long as diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp index 4d0f39357ab..b8ec98f6044 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -86,7 +86,7 @@ std::future ThreadPoolRemoteFSReader::submit(Reques auto * remote_fs_fd = assert_cast(request.descriptor.get()); - auto watch = std::make_unique(CLOCK_MONOTONIC); + auto watch = std::make_unique(CLOCK_REALTIME); Result result = remote_fs_fd->readInto(request.buf, request.size, request.offset, request.ignore); watch->stop(); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 39399c9ce09..cc39255eb97 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -14,6 +14,20 @@ #include +namespace +{ + +size_t roundDownToMultiple(size_t num, size_t multiple) +{ + return (num / multiple) * multiple; +} + +size_t roundUpToMultiple(size_t num, size_t multiple) +{ + return roundDownToMultiple(num + multiple - 1, multiple); +} +} + namespace fs = std::filesystem; namespace DB @@ -37,6 +51,7 @@ FileCache::FileCache(const FileCacheSettings & settings) , main_priority(std::make_unique()) , stash_priority(std::make_unique()) , max_stash_element_size(settings.max_elements) + , boundary_alignment(settings.boundary_alignment) { } @@ -194,8 +209,7 @@ FileCache::FileSegmentCell * FileCache::getCell( return &cell_it->second; } -FileSegments FileCache::getImpl( - const Key & key, const FileSegment::Range & range, std::lock_guard & cache_lock) +FileSegments FileCache::getImpl(const Key & key, const FileSegment::Range & range, std::lock_guard & cache_lock) { /// Given range = [left, right] and non-overlapping ordered set of file segments, /// find list [segment1, ..., segmentN] of segments which intersect with given range. @@ -409,8 +423,13 @@ void FileCache::fillHolesWithEmptyFileSegments( } } -FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings) +FileSegmentsHolder +FileCache::getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, const CreateFileSegmentSettings & settings) { + const auto aligned_offset = roundDownToMultiple(offset, boundary_alignment); + const auto aligned_end = std::min(roundUpToMultiple(offset + size, boundary_alignment), file_size); + const auto aligned_size = aligned_end - aligned_offset; + std::lock_guard cache_lock(mutex); assertInitialized(cache_lock); @@ -419,18 +438,25 @@ FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t si assertCacheCorrectness(key, cache_lock); #endif - FileSegment::Range range(offset, offset + size - 1); + FileSegment::Range range(aligned_offset, aligned_offset + aligned_size - 1); /// Get all segments which intersect with the given range. auto file_segments = getImpl(key, range, cache_lock); if (file_segments.empty()) { - file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::EMPTY, settings, cache_lock); + file_segments = splitRangeIntoCells(key, range.left, range.size(), FileSegment::State::EMPTY, settings, cache_lock); } else { - fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */false, settings, cache_lock); + fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */ false, settings, cache_lock); } + + while (!file_segments.empty() && file_segments.front()->range().right < offset) + file_segments.pop_front(); + + while (!file_segments.empty() && file_segments.back()->range().left >= offset + size) + file_segments.pop_back(); + assert(!file_segments.empty()); return FileSegmentsHolder(std::move(file_segments)); } diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 83435b67562..fa365f70200 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -58,7 +58,8 @@ public: * As long as pointers to returned file segments are held * it is guaranteed that these file segments are not removed from cache. */ - FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings); + FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, const CreateFileSegmentSettings & settings); + FileSegmentsHolder set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings); /** @@ -214,6 +215,7 @@ private: FileCacheRecords stash_records; std::unique_ptr stash_priority; size_t max_stash_element_size; + size_t boundary_alignment; void loadCacheInfoIntoMemory(std::lock_guard & cache_lock); diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 9d5282047aa..afee69a1f94 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -48,6 +48,8 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & bypass_cache_threashold = REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD; do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", false); + + boundary_alignment = config.getUInt64(config_prefix + ".boundary_alignment", DBMS_DEFAULT_BUFFER_SIZE); } } diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index 689c3ef70fb..8c765631066 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -26,6 +27,8 @@ struct FileCacheSettings bool enable_bypass_cache_with_threashold = false; size_t bypass_cache_threashold = REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD; + size_t boundary_alignment = DBMS_DEFAULT_BUFFER_SIZE; + void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); }; diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 72dc1144fb9..e9da0d21f7e 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -4,7 +4,7 @@ namespace DB { -static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 100 * 1024 * 1024; +static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 8 * 1024 * 1024; static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS = 1024 * 1024; static constexpr int REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD = 0; static constexpr size_t REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024;; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 6ae25c681d4..6a99ea322d3 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -426,6 +426,8 @@ void FileSegment::write(const char * from, size_t size, size_t offset) FileSegment::State FileSegment::wait() { + OpenTelemetry::SpanHolder span{fmt::format("FileSegment::wait({})", key().toString())}; + std::unique_lock segment_lock(mutex); if (is_detached) diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.cpp b/src/Interpreters/FilesystemReadPrefetchesLog.cpp index 9dd0fce84b0..664299017bc 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.cpp +++ b/src/Interpreters/FilesystemReadPrefetchesLog.cpp @@ -1,9 +1,9 @@ -#include #include #include #include #include #include +#include namespace DB @@ -39,12 +39,12 @@ void FilesystemReadPrefetchesLogElement::appendToBlock(MutableColumns & columns) columns[i++]->insert(path); columns[i++]->insert(offset); columns[i++]->insert(size); - columns[i++]->insert(prefetch_submit_time); + columns[i++]->insert(std::chrono::duration_cast(prefetch_submit_time.time_since_epoch()).count()); columns[i++]->insert(priority); if (execution_watch) { - columns[i++]->insert(execution_watch->getStart()); - columns[i++]->insert(execution_watch->getEnd()); + columns[i++]->insert(execution_watch->getStart() / 1000); + columns[i++]->insert(execution_watch->getEnd() / 1000); columns[i++]->insert(execution_watch->elapsedMicroseconds()); } else diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.h b/src/Interpreters/FilesystemReadPrefetchesLog.h index a7672c49d91..685909d41b7 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.h +++ b/src/Interpreters/FilesystemReadPrefetchesLog.h @@ -23,7 +23,7 @@ struct FilesystemReadPrefetchesLogElement String path; UInt64 offset; Int64 size; /// -1 means unknown - Decimal64 prefetch_submit_time{}; + std::chrono::system_clock::time_point prefetch_submit_time; std::optional execution_watch; size_t priority; FilesystemPrefetchState state; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 00a5d0ed1d8..96bcc89c1d2 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -321,6 +321,8 @@ static std::tuple executeQueryImpl( /// This does not have impact on the final span logs, because these internal queries are issued by external queries, /// we still have enough span logs for the execution of external queries. std::shared_ptr query_span = internal ? nullptr : std::make_shared("query"); + if (query_span) + LOG_DEBUG(&Poco::Logger::get("executeQuery"), "Query span trace_id for opentelemetry log: {}", query_span->trace_id); auto query_start_time = std::chrono::system_clock::now(); diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 0754c394f66..3ba13b218ec 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -138,6 +138,7 @@ TEST_F(FileCacheTest, get) settings.base_path = cache_base_path; settings.max_size = 30; settings.max_elements = 5; + settings.boundary_alignment = 1; { auto cache = DB::FileCache(settings); @@ -145,7 +146,7 @@ TEST_F(FileCacheTest, get) auto key = cache.hash("key1"); { - auto holder = cache.getOrSet(key, 0, 10, {}); /// Add range [0, 9] + auto holder = cache.getOrSet(key, 0, 10, 10, {}); /// Add range [0, 9] auto segments = fromHolder(holder); /// Range was not present in cache. It should be added in cache as one while file segment. ASSERT_EQ(segments.size(), 1); @@ -174,7 +175,7 @@ TEST_F(FileCacheTest, get) { /// Want range [5, 14], but [0, 9] already in cache, so only [10, 14] will be put in cache. - auto holder = cache.getOrSet(key, 5, 10, {}); + auto holder = cache.getOrSet(key, 5, 10, 15, {}); auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 2); @@ -194,14 +195,14 @@ TEST_F(FileCacheTest, get) ASSERT_EQ(cache.getUsedCacheSize(), 15); { - auto holder = cache.getOrSet(key, 9, 1, {}); /// Get [9, 9] + auto holder = cache.getOrSet(key, 9, 1, 10, {}); /// Get [9, 9] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 1); assertRange(7, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); } { - auto holder = cache.getOrSet(key, 9, 2, {}); /// Get [9, 10] + auto holder = cache.getOrSet(key, 9, 2, 11, {}); /// Get [9, 10] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 2); assertRange(8, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); @@ -209,15 +210,15 @@ TEST_F(FileCacheTest, get) } { - auto holder = cache.getOrSet(key, 10, 1, {}); /// Get [10, 10] + auto holder = cache.getOrSet(key, 10, 1, 11, {}); /// Get [10, 10] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 1); assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); } - complete(cache_base_path, cache.getOrSet(key, 17, 4, {})); /// Get [17, 20] - complete(cache_base_path, cache.getOrSet(key, 24, 3, {})); /// Get [24, 26] - /// completeWithState(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27] + complete(cache_base_path, cache.getOrSet(key, 17, 4, 21, {})); /// Get [17, 20] + complete(cache_base_path, cache.getOrSet(key, 24, 3, 27, {})); /// Get [24, 26] + /// completeWithState(cache.getOrSet(key, 27, 1, 28, false)); /// Get [27, 27] /// Current cache: [__________][_____] [____] [___][] /// ^ ^^ ^ ^ ^ ^ ^^^ @@ -227,7 +228,7 @@ TEST_F(FileCacheTest, get) ASSERT_EQ(cache.getUsedCacheSize(), 22); { - auto holder = cache.getOrSet(key, 0, 26, {}); /// Get [0, 25] + auto holder = cache.getOrSet(key, 0, 26, 26, {}); /// Get [0, 25] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 6); @@ -261,14 +262,14 @@ TEST_F(FileCacheTest, get) /// as max elements size is reached, next attempt to put something in cache should fail. /// This will also check that [27, 27] was indeed evicted. - auto holder1 = cache.getOrSet(key, 27, 1, {}); + auto holder1 = cache.getOrSet(key, 27, 1, 28, {}); auto segments_1 = fromHolder(holder1); /// Get [27, 27] ASSERT_EQ(segments_1.size(), 1); assertRange(17, segments_1[0], DB::FileSegment::Range(27, 27), DB::FileSegment::State::EMPTY); } { - auto holder = cache.getOrSet(key, 12, 10, {}); /// Get [12, 21] + auto holder = cache.getOrSet(key, 12, 10, 22, {}); /// Get [12, 21] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 4); @@ -292,7 +293,7 @@ TEST_F(FileCacheTest, get) ASSERT_EQ(cache.getFileSegmentsNum(), 5); { - auto holder = cache.getOrSet(key, 23, 5, {}); /// Get [23, 28] + auto holder = cache.getOrSet(key, 23, 5, 28, {}); /// Get [23, 28] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 3); @@ -313,12 +314,12 @@ TEST_F(FileCacheTest, get) /// 17 21 2324 26 28 { - auto holder5 = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4] + auto holder5 = cache.getOrSet(key, 2, 3, 5, {}); /// Get [2, 4] auto s5 = fromHolder(holder5); ASSERT_EQ(s5.size(), 1); assertRange(25, s5[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::EMPTY); - auto holder1 = cache.getOrSet(key, 30, 2, {}); /// Get [30, 31] + auto holder1 = cache.getOrSet(key, 30, 2, 32, {}); /// Get [30, 31] auto s1 = fromHolder(holder1); ASSERT_EQ(s1.size(), 1); assertRange(26, s1[0], DB::FileSegment::Range(30, 31), DB::FileSegment::State::EMPTY); @@ -334,20 +335,20 @@ TEST_F(FileCacheTest, get) /// ^ ^ ^ ^ ^ ^ ^ ^ /// 2 4 23 24 26 27 30 31 - auto holder2 = cache.getOrSet(key, 23, 1, {}); /// Get [23, 23] + auto holder2 = cache.getOrSet(key, 23, 1, 24, {}); /// Get [23, 23] auto s2 = fromHolder(holder2); ASSERT_EQ(s2.size(), 1); - auto holder3 = cache.getOrSet(key, 24, 3, {}); /// Get [24, 26] + auto holder3 = cache.getOrSet(key, 24, 3, 27, {}); /// Get [24, 26] auto s3 = fromHolder(holder3); ASSERT_EQ(s3.size(), 1); - auto holder4 = cache.getOrSet(key, 27, 1, {}); /// Get [27, 27] + auto holder4 = cache.getOrSet(key, 27, 1, 28, {}); /// Get [27, 27] auto s4 = fromHolder(holder4); ASSERT_EQ(s4.size(), 1); /// All cache is now unreleasable because pointers are still hold - auto holder6 = cache.getOrSet(key, 0, 40, {}); + auto holder6 = cache.getOrSet(key, 0, 40, 40, {}); auto f = fromHolder(holder6); ASSERT_EQ(f.size(), 9); @@ -368,7 +369,7 @@ TEST_F(FileCacheTest, get) } { - auto holder = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4] + auto holder = cache.getOrSet(key, 2, 3, 5, {}); /// Get [2, 4] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 1); assertRange(31, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); @@ -379,7 +380,7 @@ TEST_F(FileCacheTest, get) /// 2 4 23 24 26 27 30 31 { - auto holder = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29] + auto holder = cache.getOrSet(key, 25, 5, 30, {}); /// Get [25, 29] auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 3); @@ -403,7 +404,7 @@ TEST_F(FileCacheTest, get) chassert(&DB::CurrentThread::get() == &thread_status_1); DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); - auto holder_2 = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29] once again. + auto holder_2 = cache.getOrSet(key, 25, 5, 30, {}); /// Get [25, 29] once again. auto segments_2 = fromHolder(holder_2); ASSERT_EQ(segments.size(), 3); @@ -446,7 +447,7 @@ TEST_F(FileCacheTest, get) /// and notify_all() is also called from destructor of holder. std::optional holder; - holder.emplace(cache.getOrSet(key, 3, 23, {})); /// Get [3, 25] + holder.emplace(cache.getOrSet(key, 3, 23, 26, {})); /// Get [3, 25] auto segments = fromHolder(*holder); ASSERT_EQ(segments.size(), 3); @@ -472,7 +473,7 @@ TEST_F(FileCacheTest, get) chassert(&DB::CurrentThread::get() == &thread_status_1); DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); - auto holder_2 = cache.getOrSet(key, 3, 23, {}); /// Get [3, 25] once again + auto holder_2 = cache.getOrSet(key, 3, 23, 26, {}); /// Get [3, 25] once again auto segments_2 = fromHolder(*holder); ASSERT_EQ(segments_2.size(), 3); @@ -521,7 +522,7 @@ TEST_F(FileCacheTest, get) cache2.initialize(); auto key = cache2.hash("key1"); - auto holder1 = cache2.getOrSet(key, 2, 28, {}); /// Get [2, 29] + auto holder1 = cache2.getOrSet(key, 2, 28, 30, {}); /// Get [2, 29] auto segments1 = fromHolder(holder1); ASSERT_EQ(segments1.size(), 5); @@ -543,7 +544,7 @@ TEST_F(FileCacheTest, get) cache2.initialize(); auto key = cache2.hash("key1"); - auto holder1 = cache2.getOrSet(key, 0, 25, {}); /// Get [0, 24] + auto holder1 = cache2.getOrSet(key, 0, 25, 25, {}); /// Get [0, 24] auto segments1 = fromHolder(holder1); ASSERT_EQ(segments1.size(), 3); @@ -668,7 +669,7 @@ TEST_F(FileCacheTest, temporaryData) auto tmp_data_scope = std::make_shared(nullptr, &file_cache, 0); - auto some_data_holder = file_cache.getOrSet(file_cache.hash("some_data"), 0, 5_KiB, CreateFileSegmentSettings{}); + auto some_data_holder = file_cache.getOrSet(file_cache.hash("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}); { auto segments = fromHolder(some_data_holder); diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 7428fd5c056..e51120f9305 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -366,6 +366,9 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf part_info->column_name_set = {required_column_names.begin(), required_column_names.end()}; part_info->task_columns = task_columns; + if (settings.prefetch_buffer_size < DBMS_DEFAULT_BUFFER_SIZE) + throw Exception(ErrorCodes::LOGICAL_ERROR, "remove me"); + /// adjustBufferSize(), which is done in MergeTreeReaderStream and MergeTreeReaderCompact, /// lowers buffer size if file size (or required read range) is less. So we know that the /// settings.prefetch_buffer_size will be lowered there, therefore we account it here as well. diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 29be06b4e6a..931a1be6b30 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -21,6 +21,14 @@ namespace ErrorCodes namespace DB { +size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & columns_to_read) +{ + ColumnSize columns_size{}; + for (const auto & col_name : columns_to_read) + columns_size.add(part.getColumnSize(col_name)); + return columns_size.data_compressed; +} + MergeTreeReadPool::MergeTreeReadPool( size_t threads_, size_t sum_marks_, @@ -44,13 +52,36 @@ MergeTreeReadPool::MergeTreeReadPool( , parts_ranges(std::move(parts_)) , predict_block_size_bytes(context_->getSettingsRef().preferred_block_size_bytes > 0) , do_not_steal_tasks(do_not_steal_tasks_) + , merge_tree_use_const_size_tasks_for_remote_reading(context_->getSettingsRef().merge_tree_use_const_size_tasks_for_remote_reading) , backoff_settings{context_->getSettingsRef()} , backoff_state{threads_} { + const auto & settings = context_->getSettingsRef(); + + size_t total_compressed_bytes = 0; + size_t total_marks = 0; + for (const auto & part : parts_ranges) + { + total_compressed_bytes += getApproxSizeOfPart( + *part.data_part, prewhere_info ? prewhere_info->prewhere_actions->getRequiredColumnsNames() : column_names_); + total_marks += part.getMarksCount(); + } + + if (total_marks) + { + const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; + const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); + const auto heuristic_min_marks = std::min(total_marks / threads_, min_bytes_per_task / avg_mark_bytes); + if (heuristic_min_marks > min_marks_for_concurrent_read) + { + min_marks_for_concurrent_read = heuristic_min_marks; + } + } + /// parts don't contain duplicate MergeTreeDataPart's. const auto per_part_sum_marks = fillPerPartInfo( parts_ranges, storage_snapshot, is_part_on_remote_disk, - do_not_steal_tasks, predict_block_size_bytes, + predict_block_size_bytes, column_names, virtual_column_names, prewhere_info, actions_settings, reader_settings, per_part_params); @@ -61,7 +92,6 @@ std::vector MergeTreeReadPool::fillPerPartInfo( const RangesInDataParts & parts, const StorageSnapshotPtr & storage_snapshot, std::vector & is_part_on_remote_disk, - bool & do_not_steal_tasks, bool & predict_block_size_bytes, const Names & column_names, const Names & virtual_column_names, @@ -83,7 +113,6 @@ std::vector MergeTreeReadPool::fillPerPartInfo( bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk(); is_part_on_remote_disk[i] = part_on_remote_disk; - do_not_steal_tasks |= part_on_remote_disk; /// Read marks for every data part. size_t sum_marks = 0; @@ -157,14 +186,13 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t thread) auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); size_t need_marks; - if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks + if (is_part_on_remote_disk[part_idx] && !merge_tree_use_const_size_tasks_for_remote_reading) need_marks = marks_in_part; else /// Get whole part to read if it is small enough. need_marks = std::min(marks_in_part, min_marks_for_concurrent_read); /// Do not leave too little rows in part for next time. - if (marks_in_part > need_marks && - marks_in_part - need_marks < min_marks_for_concurrent_read) + if (marks_in_part > need_marks && marks_in_part - need_marks < min_marks_for_concurrent_read / 2) need_marks = marks_in_part; MarkRanges ranges_to_get_from_part; @@ -294,6 +322,8 @@ void MergeTreeReadPool::fillPerThreadInfo( parts_queue.push(std::move(info.second)); } + LOG_DEBUG(log, "min_marks_for_concurrent_read={}", min_marks_for_concurrent_read); + const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1; for (size_t i = 0; i < threads && !parts_queue.empty(); ++i) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index b3356ec3351..514150566a6 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -92,7 +92,6 @@ public: const RangesInDataParts & parts, const StorageSnapshotPtr & storage_snapshot, std::vector & is_part_on_remote_disk, - bool & do_not_steal_tasks, bool & predict_block_size_bytes, const Names & column_names, const Names & virtual_column_names, @@ -117,6 +116,7 @@ private: RangesInDataParts parts_ranges; bool predict_block_size_bytes; bool do_not_steal_tasks; + bool merge_tree_use_const_size_tasks_for_remote_reading = false; std::vector per_part_params; std::vector is_part_on_remote_disk; @@ -188,7 +188,7 @@ public: , parts_ranges(std::move(parts_)) { MergeTreeReadPool::fillPerPartInfo( - parts_ranges, storage_snapshot, is_part_on_remote_disk, do_not_steal_tasks, + parts_ranges, storage_snapshot, is_part_on_remote_disk, predict_block_size_bytes, column_names, virtual_column_names, prewhere_info, actions_settings, reader_settings, per_part_params); @@ -225,7 +225,6 @@ private: const Names virtual_column_names; RangesInDataParts parts_ranges; - bool do_not_steal_tasks = false; bool predict_block_size_bytes = false; std::vector is_part_on_remote_disk; std::vector per_part_params; diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index 328336ff71a..6cf6cd48534 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -207,6 +207,7 @@ std::optional MergeTreeSource::tryGenerate() try { + OpenTelemetry::SpanHolder span{"MergeTreeSource::tryGenerate()"}; holder->setResult(algorithm->read()); } catch (...) @@ -221,6 +222,7 @@ std::optional MergeTreeSource::tryGenerate() } #endif + OpenTelemetry::SpanHolder span{"MergeTreeSource::tryGenerate()"}; return processReadResult(algorithm->read()); } From e20f92ce0f6ef4d06813932025cdea10a361631c Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Thu, 27 Apr 2023 21:26:36 +0300 Subject: [PATCH 0143/2223] Fixed exceptions handling; Fixed style; --- programs/local/LocalServer.cpp | 10 +++++- src/Databases/DatabaseFactory.cpp | 2 +- src/Databases/DatabaseFilesystem.cpp | 48 +++++++++++++++++++++++---- src/Databases/DatabasesOverlay.cpp | 49 ++++++---------------------- src/Databases/DatabasesOverlay.h | 2 -- 5 files changed, 62 insertions(+), 49 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 566d11791ca..4939997b323 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -149,6 +150,13 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str return system_database; } +static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context_) +{ + auto databaseCombiner = std::make_shared(name_, context_); + databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); + databaseCombiner->registerNextDatabase(std::make_shared(name_, context_)); + return databaseCombiner; +} /// If path is specified and not empty, will try to setup server environment and load existing metadata void LocalServer::tryInitPath() @@ -648,7 +656,7 @@ void LocalServer::processConfig() * if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons. */ std::string default_database = config().getString("default_database", "_local"); - DatabaseCatalog::instance().attachDatabase(default_database, CreateClickHouseLocalDatabaseOverlay(default_database, global_context)); + DatabaseCatalog::instance().attachDatabase(default_database, createClickHouseLocalDatabaseOverlay(default_database, global_context)); global_context->setCurrentDatabase(default_database); applyCmdOptions(global_context); diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 8a50c31efc8..1be0d5dd7b2 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -443,7 +443,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine->arguments && !engine->arguments->children.empty()) { if (engine->arguments->children.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filesystem database requires at most 1 argument: filesystem_path"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filesystem database requires exactly 1 argument: filesystem_path"); const auto & arguments = engine->arguments->children; init_path = safeGetLiteralValue(arguments[0], engine_name); diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 16aed185669..8275bdf6151 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -20,24 +20,42 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int UNKNOWN_TABLE; + extern const int DATABASE_ACCESS_DENIED; + extern const int BAD_ARGUMENTS; } DatabaseFilesystem::DatabaseFilesystem(const String & name_, const String & path_, ContextPtr context_) : IDatabase(name_), WithContext(context_->getGlobalContext()), path(path_), log(&Poco::Logger::get("DatabaseFileSystem(" + name_ + ")")) { - path = fs::path(path).lexically_normal().string(); + fs::path user_files_path; + if (context_->getApplicationType() != Context::ApplicationType::LOCAL) + user_files_path = fs::canonical(fs::path(getContext()->getUserFilesPath())); + + if (fs::path(path).is_relative()) + path = user_files_path / path; + + path = fs::absolute(path).lexically_normal().string(); } std::string DatabaseFilesystem::getTablePath(const std::string & table_name) const { - return fs::path(path) / table_name; + fs::path table_path = fs::path(path) / table_name; + return table_path.lexically_normal().string(); } void DatabaseFilesystem::addTable(const std::string & table_name, StoragePtr table_storage) const { std::lock_guard lock(mutex); - loaded_tables.emplace(table_name, table_storage); + auto [_, inserted] = loaded_tables.emplace(table_name, table_storage); + if (!inserted) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Table with name `{}` already exists in database `{}` (engine {})", + table_name, + getDatabaseName(), + getEngineName()); } bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr) const @@ -62,8 +80,20 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont return it->second; } + // If run in Local mode, no need for path checking. + bool need_check_path = context_->getApplicationType() != Context::ApplicationType::LOCAL; + std::string user_files_path = fs::canonical(fs::path(context_->getUserFilesPath())).string(); + auto table_path = getTablePath(name); + // Check access for file before checking its existence + if (need_check_path && table_path.find(user_files_path) != 0) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "File is not inside {}", user_files_path); + + // If the table doesn't exist in the tables map, check if the corresponding file exists + if (!fs::exists(table_path) || !fs::is_regular_file(table_path)) + return nullptr; + // If the file exists, create a new table using TableFunctionFile and return it. auto args = makeASTFunction("file", std::make_shared(table_path)); @@ -89,11 +119,17 @@ StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr conte { try { - return getTable(name, context_); + return getTableImpl(name, context_); } - catch (...) + catch (const Exception & e) { - return nullptr; + // Ignore exceptions thrown by TableFunctionFile and which indicate that there is no table + if (e.code() == ErrorCodes::BAD_ARGUMENTS) + return nullptr; + if (e.code() == ErrorCodes::DATABASE_ACCESS_DENIED) + return nullptr; + + throw; } } diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 5a6a4fe5cc6..b44a9798072 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -1,11 +1,9 @@ #include +#include #include #include -#include - -#include -#include +#include #include @@ -73,15 +71,11 @@ void DatabasesOverlay::dropTable(ContextPtr context_, const String & table_name, { for (auto & db : databases) { - try + if (db->isTableExist(table_name, context_)) { db->dropTable(context_, table_name, sync); return; } - catch (...) - { - continue; - } } throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -119,16 +113,8 @@ StoragePtr DatabasesOverlay::detachTable(ContextPtr context_, const String & tab StoragePtr result = nullptr; for (auto & db : databases) { - try - { - result = db->detachTable(context_, table_name); - if (result) - return result; - } - catch (...) - { - continue; - } + if (db->isTableExist(table_name, context_)) + return db->detachTable(context_, table_name); } throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -212,17 +198,10 @@ void DatabasesOverlay::alterTable(ContextPtr local_context, const StorageID & ta { for (auto & db : databases) { - try + if (!db->isReadOnly() && db->isTableExist(table_id.table_name, local_context)) { - if (!db->isReadOnly()) - { - db->alterTable(local_context, table_id, metadata); - return; - } - } - catch (...) - { - continue; + db->alterTable(local_context, table_id, metadata); + return; } } throw Exception( @@ -239,8 +218,8 @@ DatabasesOverlay::getTablesForBackup(const FilterByNameFunction & filter, const std::vector> result; for (const auto & db : databases) { - auto dbBackup = db->getTablesForBackup(filter, local_context); - result.insert(result.end(), std::make_move_iterator(dbBackup.begin()), std::make_move_iterator(dbBackup.end())); + auto db_backup = db->getTablesForBackup(filter, local_context); + result.insert(result.end(), std::make_move_iterator(db_backup.begin()), std::make_move_iterator(db_backup.end())); } return result; } @@ -284,12 +263,4 @@ DatabaseTablesIteratorPtr DatabasesOverlay::getTablesIterator(ContextPtr context return std::make_unique(std::move(tables), getDatabaseName()); } -DatabasePtr CreateClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context_) -{ - auto databaseCombiner = std::make_shared(name_, context_); - databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); - databaseCombiner->registerNextDatabase(std::make_shared(name_, context_)); - return databaseCombiner; -} - } diff --git a/src/Databases/DatabasesOverlay.h b/src/Databases/DatabasesOverlay.h index 77f0085161b..0f31bbd6a47 100644 --- a/src/Databases/DatabasesOverlay.h +++ b/src/Databases/DatabasesOverlay.h @@ -63,6 +63,4 @@ protected: Poco::Logger * log; }; -DatabasePtr CreateClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context_); - } From 7dde282027891aa0afbd0889b4be896981354ca1 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Apr 2023 18:50:35 -0400 Subject: [PATCH 0144/2223] Fix no case completion --- src/Client/LineReader.cpp | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Client/LineReader.cpp b/src/Client/LineReader.cpp index f6cd7bffef7..82dbe03e5d3 100644 --- a/src/Client/LineReader.cpp +++ b/src/Client/LineReader.cpp @@ -82,9 +82,13 @@ replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String & std::lock_guard lock(mutex); Words to_search; + bool no_case = false; /// Only perform case sensitive completion when the prefix string contains any uppercase characters if (std::none_of(prefix.begin(), prefix.end(), [](char32_t x) { return iswupper(static_cast(x)); })) + { to_search = words_no_case; + no_case = true; + } else to_search = words; @@ -95,11 +99,18 @@ replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String & addNewWords(to_search, new_words, std::less{}); } - range = std::equal_range( - to_search.begin(), to_search.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) - { - return strncasecmp(s.data(), prefix_searched.data(), prefix_length) < 0; - }); + if (no_case) + range = std::equal_range( + to_search.begin(), to_search.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) + { + return strncasecmp(s.data(), prefix_searched.data(), prefix_length) < 0; + }); + else + range = std::equal_range( + to_search.begin(), to_search.end(), last_word, [prefix_length](std::string_view s, std::string_view prefix_searched) + { + return strncmp(s.data(), prefix_searched.data(), prefix_length) < 0; + }); return replxx::Replxx::completions_t(range.first, range.second); } From 39ae0c1c4e4701f07846425a22d6e6c3f587b194 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Apr 2023 22:51:52 -0400 Subject: [PATCH 0145/2223] Fix tests --- tests/integration/test_keeper_client/test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 64ef62b6243..4fa8ce82aa6 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -30,7 +30,10 @@ def test_base_commands(started_cluster): [ started_cluster.server_bin_path, "keeper-client", - f"{cluster.get_instance_ip('zoo1')}:{cluster.zookeeper_port}", + "--host", + cluster.get_instance_ip('zoo1'), + "--port", + cluster.zookeeper_port, "-q", "create test_create_zk_node1 testvalue1;create test_create_zk_node_2 testvalue2;get test_create_zk_node1;", ], @@ -47,7 +50,10 @@ def test_four_letter_word_commands(started_cluster): [ started_cluster.server_bin_path, "keeper-client", - f"{cluster.get_instance_ip('zoo1')}:{cluster.zookeeper_port}", + "--host", + cluster.get_instance_ip('zoo1'), + "--port", + cluster.zookeeper_port, "-q", "ruok", ], From 711d8d4442cbe8f074d39c4cb2477a761037140e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Apr 2023 03:19:21 +0000 Subject: [PATCH 0146/2223] Automatic style fix --- tests/integration/test_keeper_client/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 4fa8ce82aa6..f7aec7852b1 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -31,7 +31,7 @@ def test_base_commands(started_cluster): started_cluster.server_bin_path, "keeper-client", "--host", - cluster.get_instance_ip('zoo1'), + cluster.get_instance_ip("zoo1"), "--port", cluster.zookeeper_port, "-q", @@ -51,7 +51,7 @@ def test_four_letter_word_commands(started_cluster): started_cluster.server_bin_path, "keeper-client", "--host", - cluster.get_instance_ip('zoo1'), + cluster.get_instance_ip("zoo1"), "--port", cluster.zookeeper_port, "-q", From c862eca0f735433e7eb17199d21a8c8dd2d1dc07 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 28 Apr 2023 14:08:46 +0000 Subject: [PATCH 0147/2223] fix build --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- src/Interpreters/Cache/FileCache.cpp | 7 ++----- src/Interpreters/Cache/FileSegment.cpp | 15 ++++++++------- 3 files changed, 11 insertions(+), 13 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index ddc5b385b07..b186b9b1d28 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -536,7 +536,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) }); OpenTelemetry::SpanHolder span{ - fmt::format("CachedOnDiskReadBufferFromFile::predownload(key={}, size={})", file_segment->key().toString(), bytes_to_predownload)}; + fmt::format("CachedOnDiskReadBufferFromFile::predownload(key={}, size={})", file_segment.key().toString(), bytes_to_predownload)}; if (bytes_to_predownload) { diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 03cea3ee2ab..626ac7f5d49 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -405,11 +405,8 @@ FileSegmentsHolderPtr FileCache::set( return std::make_unique(std::move(file_segments)); } -FileSegmentsHolderPtr FileCache::getOrSet( - const Key & key, - size_t offset, - size_t size, - const CreateFileSegmentSettings & settings) +FileSegmentsHolderPtr +FileCache::getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, const CreateFileSegmentSettings & settings) { assertInitialized(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 91356f699e3..2b1801a46f0 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -1,13 +1,14 @@ #include "FileSegment.h" -#include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include +#include #include From 30bc74c859ae4a33abe0d5bb7e42369865ea3b4c Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Apr 2023 14:09:24 -0400 Subject: [PATCH 0148/2223] Fix tests --- tests/integration/test_keeper_client/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 4fa8ce82aa6..eee30741bdd 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -31,9 +31,9 @@ def test_base_commands(started_cluster): started_cluster.server_bin_path, "keeper-client", "--host", - cluster.get_instance_ip('zoo1'), + str(cluster.get_instance_ip('zoo1')), "--port", - cluster.zookeeper_port, + str(cluster.zookeeper_port), "-q", "create test_create_zk_node1 testvalue1;create test_create_zk_node_2 testvalue2;get test_create_zk_node1;", ], @@ -51,9 +51,9 @@ def test_four_letter_word_commands(started_cluster): started_cluster.server_bin_path, "keeper-client", "--host", - cluster.get_instance_ip('zoo1'), + str(cluster.get_instance_ip('zoo1')), "--port", - cluster.zookeeper_port, + str(cluster.zookeeper_port), "-q", "ruok", ], From e0abb251e5344b431de768e91dbf4f791283467d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 28 Apr 2023 21:52:38 +0000 Subject: [PATCH 0149/2223] fix test --- tests/queries/0_stateless/02344_describe_cache.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 7561b32bae1..a803ca1fab1 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1,2 @@ -134217728 1048576 104857600 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 +134217728 1048576 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 134217728 1048576 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 From 6a0e5fad09a54a86fb612630147885917195699a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 29 Apr 2023 10:02:54 +0200 Subject: [PATCH 0150/2223] Removed extra line --- src/Formats/FormatFactory.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index f6882f04aff..5d31d4d182e 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -336,8 +336,7 @@ InputFormatPtr FormatFactory::getInputImpl( auto parser_creator = [input_getter, sample, row_input_format_params, format_settings] (ReadBuffer & input) -> InputFormatPtr { return input_getter(input, sample, row_input_format_params, format_settings); }; - - + ParallelParsingInputFormat::Params params{ *buf, sample, parser_creator, creators.file_segmentation_engine, name, max_parsing_threads, settings.min_chunk_bytes_for_parallel_parsing, max_block_size, context->getApplicationType() == Context::ApplicationType::SERVER}; From 0726db7d9a5920424133ca086044f985e9aade8f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 29 Apr 2023 10:34:59 +0200 Subject: [PATCH 0151/2223] Renamed parameter to max_parsing_threads --- src/Formats/FormatFactory.cpp | 2 +- src/Interpreters/Context.cpp | 4 ++-- src/Interpreters/Context.h | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 5d31d4d182e..628b81a79f8 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -336,7 +336,7 @@ InputFormatPtr FormatFactory::getInputImpl( auto parser_creator = [input_getter, sample, row_input_format_params, format_settings] (ReadBuffer & input) -> InputFormatPtr { return input_getter(input, sample, row_input_format_params, format_settings); }; - + ParallelParsingInputFormat::Params params{ *buf, sample, parser_creator, creators.file_segmentation_engine, name, max_parsing_threads, settings.min_chunk_bytes_for_parallel_parsing, max_block_size, context->getApplicationType() == Context::ApplicationType::SERVER}; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1421113c8e8..cdc11fbee53 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3540,9 +3540,9 @@ void Context::checkPartitionCanBeDropped(const String & database, const String & } -InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings, const std::optional number_of_streams) const +InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings, const std::optional max_parsing_threads) const { - return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings, number_of_streams); + return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings, max_parsing_threads); } OutputFormatPtr Context::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f60730ffc9a..81804372571 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -733,7 +733,7 @@ public: /// Eg: max_threads = 16, number_of_streams = 8, then each stream will use 2 threads /// This is done to improve performance InputFormatPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, - const std::optional & format_settings = std::nullopt, const std::optional number_of_streams = std::nullopt) const; + const std::optional & format_settings = std::nullopt, const std::optional max_parsing_threads = std::nullopt) const; OutputFormatPtr getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const; OutputFormatPtr getOutputFormatParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const; From 57d852a60e804da746ce5e4cde2d56222afe677e Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Sun, 30 Apr 2023 14:46:11 +0300 Subject: [PATCH 0152/2223] Fixed table existence checking --- src/Databases/DatabaseFactory.cpp | 2 +- src/Databases/DatabaseFilesystem.cpp | 60 ++++++++++++++----- src/Databases/DatabaseFilesystem.h | 2 + .../02722_database_filesystem.reference | 5 +- .../0_stateless/02722_database_filesystem.sh | 24 ++++++-- 5 files changed, 70 insertions(+), 23 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 1be0d5dd7b2..8a50c31efc8 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -443,7 +443,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine->arguments && !engine->arguments->children.empty()) { if (engine->arguments->children.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filesystem database requires exactly 1 argument: filesystem_path"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filesystem database requires at most 1 argument: filesystem_path"); const auto & arguments = engine->arguments->children; init_path = safeGetLiteralValue(arguments[0], engine_name); diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 8275bdf6151..7f22b8a16a0 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -24,19 +25,27 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int DATABASE_ACCESS_DENIED; extern const int BAD_ARGUMENTS; + extern const int FILE_DOESNT_EXIST; } DatabaseFilesystem::DatabaseFilesystem(const String & name_, const String & path_, ContextPtr context_) : IDatabase(name_), WithContext(context_->getGlobalContext()), path(path_), log(&Poco::Logger::get("DatabaseFileSystem(" + name_ + ")")) { fs::path user_files_path; - if (context_->getApplicationType() != Context::ApplicationType::LOCAL) + const auto & application_type = context_->getApplicationType(); + + if (application_type != Context::ApplicationType::LOCAL) user_files_path = fs::canonical(fs::path(getContext()->getUserFilesPath())); if (fs::path(path).is_relative()) path = user_files_path / path; + else if (application_type != Context::ApplicationType::LOCAL && !pathStartsWith(fs::path(path), user_files_path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path must be inside user-files path ({})", user_files_path.string()); path = fs::absolute(path).lexically_normal().string(); + + if (!fs::exists(path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path does not exist ({})", path); } std::string DatabaseFilesystem::getTablePath(const std::string & table_name) const @@ -58,7 +67,32 @@ void DatabaseFilesystem::addTable(const std::string & table_name, StoragePtr tab getEngineName()); } -bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr) const +bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, ContextPtr context_, bool throw_on_error) const { + // If run in Local mode, no need for path checking. + bool need_check_path = context_->getApplicationType() != Context::ApplicationType::LOCAL; + std::string user_files_path = fs::canonical(fs::path(context_->getUserFilesPath())).string(); + + // Check access for file before checking its existence + if (need_check_path && !fileOrSymlinkPathStartsWith(table_path, user_files_path)) + { + if (throw_on_error) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "File is not inside {}", user_files_path); + else + return false; + } + + // Check if the corresponding file exists + if (!fs::exists(table_path) || !fs::is_regular_file(table_path)) { + if (throw_on_error) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File does not exist ({})", table_path); + else + return false; + } + + return true; +} + +bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr context_) const { { std::lock_guard lock(mutex); @@ -67,7 +101,8 @@ bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr) const } fs::path table_file_path(getTablePath(name)); - return fs::exists(table_file_path) && fs::is_regular_file(table_file_path); + + return checkTableFilePath(table_file_path, context_, false); } StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_) const @@ -80,19 +115,9 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont return it->second; } - // If run in Local mode, no need for path checking. - bool need_check_path = context_->getApplicationType() != Context::ApplicationType::LOCAL; - std::string user_files_path = fs::canonical(fs::path(context_->getUserFilesPath())).string(); - auto table_path = getTablePath(name); - // Check access for file before checking its existence - if (need_check_path && table_path.find(user_files_path) != 0) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "File is not inside {}", user_files_path); - - // If the table doesn't exist in the tables map, check if the corresponding file exists - if (!fs::exists(table_path) || !fs::is_regular_file(table_path)) - return nullptr; + checkTableFilePath(table_path, context_, true); // If the file exists, create a new table using TableFunctionFile and return it. auto args = makeASTFunction("file", std::make_shared(table_path)); @@ -101,6 +126,7 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont if (!table_function) return nullptr; + // TableFunctionFile throws exceptions, if table cannot be created auto table_storage = table_function->execute(args, context_, name); if (table_storage) addTable(name, table_storage); @@ -110,6 +136,7 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_) const { + // rethrow all exceptions from TableFunctionFile to show correct error to user if (auto storage = getTableImpl(name, context_)) return storage; throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name)); @@ -123,11 +150,14 @@ StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr conte } catch (const Exception & e) { - // Ignore exceptions thrown by TableFunctionFile and which indicate that there is no table + // Ignore exceptions thrown by TableFunctionFile, which indicate that there is no table + // see tests/02722_database_filesystem.sh for more details if (e.code() == ErrorCodes::BAD_ARGUMENTS) return nullptr; if (e.code() == ErrorCodes::DATABASE_ACCESS_DENIED) return nullptr; + if (e.code() == ErrorCodes::FILE_DOESNT_EXIST) + return nullptr; throw; } diff --git a/src/Databases/DatabaseFilesystem.h b/src/Databases/DatabaseFilesystem.h index 697511ac5b3..3d2ad695cc6 100644 --- a/src/Databases/DatabaseFilesystem.h +++ b/src/Databases/DatabaseFilesystem.h @@ -49,6 +49,8 @@ protected: void addTable(const std::string & table_name, StoragePtr table_storage) const; + bool checkTableFilePath(const std::string & table_path, ContextPtr context_, bool throw_on_error) const; + private: String path; diff --git a/tests/queries/0_stateless/02722_database_filesystem.reference b/tests/queries/0_stateless/02722_database_filesystem.reference index a583f1e2e3c..c65dda7933a 100644 --- a/tests/queries/0_stateless/02722_database_filesystem.reference +++ b/tests/queries/0_stateless/02722_database_filesystem.reference @@ -4,7 +4,10 @@ test1 4 4 4 -Test 2: check DatabaseFilesystem access rights on server +Test 2: check DatabaseFilesystem access rights and errors handling on server +OK +OK +OK OK OK OK diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index 0adeface438..80f97af693e 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -21,6 +21,7 @@ tmp_dir=${CLICKHOUSE_TEST_UNIQUE_NAME} mkdir $tmp_dir cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${tmp_dir}/tmp.csv cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp/tmp.csv +cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp.myext ################# echo "Test 1: create filesystem database and check implicit calls" @@ -35,24 +36,35 @@ ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp/tmp.csv\`;" ${CLICKHOUSE_LOCAL} -q "SELECT COUNT(*) FROM \"${tmp_dir}/tmp.csv\"" ################# -echo "Test 2: check DatabaseFilesystem access rights on server" -# Allows list files only inside user_files +echo "Test 2: check DatabaseFilesystem access rights and errors handling on server" +# DATABASE_ACCESS_DENIED: Allows list files only inside user_files ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp.csv\`;" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK" ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`/tmp/tmp.csv\`;" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK" - ${CLICKHOUSE_CLIENT} --multiline --multiquery --query """ USE test1; SELECT COUNT(*) FROM \"../${tmp_dir}/tmp.csv\"; """ 2>&1| grep -F "Code: 291" > /dev/null && echo "OK" ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../../../../../../tmp.csv\`;" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK" + +# BAD_ARGUMENTS: path should be inside user_files ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test2; CREATE DATABASE test2 ENGINE = Filesystem('/tmp'); -SELECT COUNT(*) FROM test2.\`tmp.csv\`; -""" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK" +""" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" + +# BAD_ARGUMENTS: .../user_files/relative_unknown_dir does not exists +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test2; +CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); +""" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" + +# FILE_DOESNT_EXIST: unknown file +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 107" > /dev/null && echo "OK" + +# BAD_ARGUMENTS: Cannot determine the file format by it's extension +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp.myext\`;" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" # Clean ${CLICKHOUSE_CLIENT} --query "DROP DATABASE test1;" -${CLICKHOUSE_CLIENT} --query "DROP DATABASE test2;" rm -rd $tmp_dir rm -rd $CLICKHOUSE_USER_FILES_PATH From 6831eb20013aadeec451ac8fb94d894abbfccef9 Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Sun, 30 Apr 2023 14:51:04 +0300 Subject: [PATCH 0153/2223] fix style --- src/Databases/DatabaseFilesystem.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 7f22b8a16a0..8de609f0ca2 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -11,6 +10,7 @@ #include #include #include +#include #include @@ -67,7 +67,8 @@ void DatabaseFilesystem::addTable(const std::string & table_name, StoragePtr tab getEngineName()); } -bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, ContextPtr context_, bool throw_on_error) const { +bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, ContextPtr context_, bool throw_on_error) const +{ // If run in Local mode, no need for path checking. bool need_check_path = context_->getApplicationType() != Context::ApplicationType::LOCAL; std::string user_files_path = fs::canonical(fs::path(context_->getUserFilesPath())).string(); @@ -82,7 +83,8 @@ bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, Cont } // Check if the corresponding file exists - if (!fs::exists(table_path) || !fs::is_regular_file(table_path)) { + if (!fs::exists(table_path) || !fs::is_regular_file(table_path)) + { if (throw_on_error) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File does not exist ({})", table_path); else From a5d47ea4892b9f8dcc372a101ab5c9e59540fcc5 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 30 Apr 2023 19:01:06 +0200 Subject: [PATCH 0154/2223] Fixed build issues --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index fc36b4a9f62..d4c5ee25502 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -608,7 +608,7 @@ public: read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context); const Settings & settings = context->getSettingsRef(); - chassert(storage->paths.size()); + chassert(!storage->paths.empty()); const auto max_parsing_threads = std::max(settings.max_threads/storage->paths.size(),1UL); auto format = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, max_parsing_threads); From 1846b76982828ed3223b25e2e5d6f5c8cee937eb Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Sun, 30 Apr 2023 23:13:42 +0300 Subject: [PATCH 0155/2223] Added DatabaseS3 with test --- programs/local/LocalServer.cpp | 9 + src/Databases/DatabaseFactory.cpp | 29 ++- src/Databases/DatabaseS3.cpp | 199 ++++++++++++++++++ src/Databases/DatabaseS3.h | 63 ++++++ .../0_stateless/02724_database_s3.reference | 18 ++ .../queries/0_stateless/02724_database_s3.sh | 51 +++++ 6 files changed, 367 insertions(+), 2 deletions(-) create mode 100644 src/Databases/DatabaseS3.cpp create mode 100644 src/Databases/DatabaseS3.h create mode 100644 tests/queries/0_stateless/02724_database_s3.reference create mode 100755 tests/queries/0_stateless/02724_database_s3.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 4939997b323..215a92e1944 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -51,6 +51,8 @@ #include #include +#include "config.h" + #if defined(FUZZING_MODE) #include #endif @@ -59,6 +61,10 @@ # include #endif +#if USE_AWS_S3 +#include +#endif + namespace fs = std::filesystem; @@ -155,6 +161,9 @@ static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, Co auto databaseCombiner = std::make_shared(name_, context_); databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); databaseCombiner->registerNextDatabase(std::make_shared(name_, context_)); +#if USE_AWS_S3 + databaseCombiner->registerNextDatabase(std::make_shared(name_, "", "", context_)); +#endif return databaseCombiner; } diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 8a50c31efc8..b21435527a5 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -49,6 +49,10 @@ #include #endif +#if USE_AWS_S3 +#include +#endif + namespace fs = std::filesystem; namespace DB @@ -133,13 +137,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String static const std::unordered_set database_engines{"Ordinary", "Atomic", "Memory", "Dictionary", "Lazy", "Replicated", "MySQL", "MaterializeMySQL", "MaterializedMySQL", - "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem"}; + "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3"}; if (!database_engines.contains(engine_name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine name `{}` does not exist", engine_name); static const std::unordered_set engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL", - "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem"}; + "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3"}; static const std::unordered_set engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"}; bool engine_may_have_arguments = engines_with_arguments.contains(engine_name); @@ -451,6 +455,27 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared(database_name, init_path, context); } +#if USE_AWS_S3 + else if (engine_name == "S3") + { + const ASTFunction * engine = engine_define->engine; + + std::string key_id; + std::string secret_key; + + if (engine->arguments && !engine->arguments->children.empty()) + { + if (engine->arguments->children.size() != 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 database requires 0 or 2 argument: [access_key_id, secret_access_key]"); + + const auto & arguments = engine->arguments->children; + key_id = safeGetLiteralValue(arguments[0], engine_name); + secret_key = safeGetLiteralValue(arguments[1], engine_name); + } + + return std::make_shared(database_name, key_id, secret_key, context); + } +#endif throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", engine_name); } diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp new file mode 100644 index 00000000000..d4412ba7973 --- /dev/null +++ b/src/Databases/DatabaseS3.cpp @@ -0,0 +1,199 @@ +#include "config.h" + +#if USE_AWS_S3 + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int UNKNOWN_TABLE; + extern const int BAD_ARGUMENTS; + extern const int FILE_DOESNT_EXIST; + extern const int UNACCEPTABLE_URL; + extern const int S3_ERROR; +} + +DatabaseS3::DatabaseS3(const String & name_, const String & key_id, const String & secret_key, ContextPtr context_) + : IDatabase(name_) + , WithContext(context_->getGlobalContext()) + , access_key_id(key_id) + , secret_access_key(secret_key) + , log(&Poco::Logger::get("DatabaseS3(" + name_ + ")")) +{ +} + +void DatabaseS3::addTable(const std::string & table_name, StoragePtr table_storage) const +{ + std::lock_guard lock(mutex); + auto [_, inserted] = loaded_tables.emplace(table_name, table_storage); + if (!inserted) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Table with name `{}` already exists in database `{}` (engine {})", + table_name, + getDatabaseName(), + getEngineName()); +} + +bool DatabaseS3::checkUrl(const std::string & url, ContextPtr context_, bool throw_on_error) const +{ + try + { + S3::URI uri(url); + context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri.uri); + } + catch (...) + { + if (throw_on_error) + throw; + return false; + } + return true; +} + +bool DatabaseS3::isTableExist(const String & name, ContextPtr context_) const +{ + std::lock_guard lock(mutex); + if (loaded_tables.find(name) != loaded_tables.end()) + return true; + + return checkUrl(name, context_, false); +} + +StoragePtr DatabaseS3::getTableImpl(const String & url, ContextPtr context_) const +{ + // Check if the table exists in the loaded tables map + { + std::lock_guard lock(mutex); + auto it = loaded_tables.find(url); + if (it != loaded_tables.end()) + return it->second; + } + + checkUrl(url, context_, true); + + // call TableFunctionS3 + auto args = makeASTFunction( + "s3", + std::make_shared(url), + std::make_shared(access_key_id), + std::make_shared(secret_access_key)); + + auto table_function = TableFunctionFactory::instance().get(args, context_); + if (!table_function) + return nullptr; + + // TableFunctionS3 throws exceptions, if table cannot be created + auto table_storage = table_function->execute(args, context_, url); + if (table_storage) + addTable(url, table_storage); + + return table_storage; +} + +StoragePtr DatabaseS3::getTable(const String & name, ContextPtr context_) const +{ + // rethrow all exceptions from TableFunctionS3 to show correct error to user + if (auto storage = getTableImpl(name, context_)) + return storage; + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name)); +} + +StoragePtr DatabaseS3::tryGetTable(const String & name, ContextPtr context_) const +{ + try + { + return getTableImpl(name, context_); + } + catch (const Exception & e) + { + // Ignore exceptions thrown by TableFunctionS3, which indicate that there is no table + if (e.code() == ErrorCodes::BAD_ARGUMENTS) + return nullptr; + if (e.code() == ErrorCodes::S3_ERROR) + return nullptr; + if (e.code() == ErrorCodes::FILE_DOESNT_EXIST) + return nullptr; + if (e.code() == ErrorCodes::UNACCEPTABLE_URL) + return nullptr; + throw; + } + catch (const Poco::URISyntaxException &) + { + return nullptr; + } +} + +ASTPtr DatabaseS3::getCreateDatabaseQuery() const +{ + auto settings = getContext()->getSettingsRef(); + ParserCreateQuery parser; + + const String query = fmt::format("CREATE DATABASE {} ENGINE = S3('{}', '{}')", + backQuoteIfNeed(getDatabaseName()), + access_key_id, + secret_access_key); + ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth); + + if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) + { + auto & ast_create_query = ast->as(); + ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); + } + + return ast; +} + +void DatabaseS3::shutdown() +{ + Tables tables_snapshot; + { + std::lock_guard lock(mutex); + tables_snapshot = loaded_tables; + } + + for (const auto & kv : tables_snapshot) + { + auto table_id = kv.second->getStorageID(); + kv.second->flushAndShutdown(); + } + + std::lock_guard lock(mutex); + loaded_tables.clear(); +} + +/** + * Returns an empty vector because the database is read-only and no tables can be backed up + */ +std::vector> DatabaseS3::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const +{ + return {}; +} + +/** + * + * Returns an empty iterator because the database does not have its own tables + * But only caches them for quick access + */ +DatabaseTablesIteratorPtr DatabaseS3::getTablesIterator(ContextPtr, const FilterByNameFunction &) const +{ + return std::make_unique(Tables{}, getDatabaseName()); +} + +} // DB + +#endif diff --git a/src/Databases/DatabaseS3.h b/src/Databases/DatabaseS3.h new file mode 100644 index 00000000000..d5269e57f5a --- /dev/null +++ b/src/Databases/DatabaseS3.h @@ -0,0 +1,63 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; + +/** + * DatabaseS3 provides access to data stored in S3 + * Uses TableFunctionS3 to implicitly load file when a user requests the table, and provides read-only access to the data in the file + * Tables are cached inside the database for quick access + */ +class DatabaseS3 : public IDatabase, protected WithContext +{ +public: + DatabaseS3(const String & name, const String & key_id, const String & secret_key, ContextPtr context); + + String getEngineName() const override { return "S3"; } + + bool isTableExist(const String & name, ContextPtr context) const override; + + StoragePtr getTable(const String & name, ContextPtr context) const override; + + StoragePtr tryGetTable(const String & name, ContextPtr context) const override; + + bool empty() const override { return true; } + + bool isReadOnly() const override { return true; } + + ASTPtr getCreateDatabaseQuery() const override; + + void shutdown() override; + + std::vector> getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; + +protected: + StoragePtr getTableImpl(const String & url, ContextPtr context) const; + + void addTable(const std::string & table_name, StoragePtr table_storage) const; + + bool checkUrl(const std::string & url, ContextPtr context_, bool throw_on_error) const; + +private: + const String access_key_id; + const String secret_access_key; + mutable Tables loaded_tables TSA_GUARDED_BY(mutex); + Poco::Logger * log; +}; + +} // DB + +#endif diff --git a/tests/queries/0_stateless/02724_database_s3.reference b/tests/queries/0_stateless/02724_database_s3.reference new file mode 100644 index 00000000000..8a985913ff9 --- /dev/null +++ b/tests/queries/0_stateless/02724_database_s3.reference @@ -0,0 +1,18 @@ +Test 1: select from s3 +1 2 3 +4 5 6 +7 8 9 +0 0 0 +test1 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +Test 2: check exceptions +OK +OK +OK diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh new file mode 100755 index 00000000000..4f9df402040 --- /dev/null +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -0,0 +1,51 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: Depends on AWS + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +################# +echo "Test 1: select from s3" +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test1; +CREATE DATABASE test1 ENGINE = S3; +USE test1; +SELECT * FROM \"http://localhost:11111/test/a.tsv\" +""" +${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test1 +${CLICKHOUSE_CLIENT} -q "DROP DATABASE test1;" + +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test2; +CREATE DATABASE test2 ENGINE = S3('test', 'testtest'); +USE test2; +SELECT * FROM \"http://localhost:11111/test/b.tsv\" +""" +${CLICKHOUSE_CLIENT} -q "DROP DATABASE test2;" + +${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"http://localhost:11111/test/c.tsv\"" + +################# +echo "Test 2: check exceptions" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"http://localhost:11111/test/c.myext\"" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK" + +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test3; +CREATE DATABASE test3 ENGINE = S3; +USE test3; +SELECT * FROM \"http://localhost:11111/test/a.myext\" +""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK" + +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +USE test3; +SELECT * FROM \"abacaba\" +""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK" + +# Cleanup +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test1; +DROP DATABASE IF EXISTS test2; +DROP DATABASE IF EXISTS test3; +""" From 3d1affbddb3de6c464f05459c1e9e5f34b6ff957 Mon Sep 17 00:00:00 2001 From: Aleksei Golub Date: Mon, 1 May 2023 12:17:10 +0300 Subject: [PATCH 0156/2223] retrigger checks From c1c69553741af4789170590f8a669d17f2dffbeb Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 26 Apr 2023 14:06:22 +0200 Subject: [PATCH 0157/2223] Deprecate delete-on-destroy.txt, do not create it any more --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 17 +---------------- src/Storages/MergeTree/IMergeTreeDataPart.h | 7 +++++-- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++---- 3 files changed, 9 insertions(+), 22 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 148cbf93948..d7f2f3ca7c7 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -953,24 +953,9 @@ void IMergeTreeDataPart::writeVersionMetadata(const VersionMetadata & version_, } } -void IMergeTreeDataPart::writeDeleteOnDestroyMarker() -{ - static constexpr auto marker_path = "delete-on-destroy.txt"; - - try - { - getDataPartStorage().createFile(marker_path); - } - catch (Poco::Exception & e) - { - LOG_ERROR(storage.log, "{} (while creating DeleteOnDestroy marker: {})", - e.what(), (fs::path(getDataPartStorage().getFullPath()) / marker_path).string()); - } -} - void IMergeTreeDataPart::removeDeleteOnDestroyMarker() { - getDataPartStorage().removeFileIfExists("delete-on-destroy.txt"); + getDataPartStorage().removeFileIfExists(DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED); } void IMergeTreeDataPart::removeVersionMetadata() diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a36634d2cf9..f7bcaa263d6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -381,7 +381,8 @@ public: /// default will be stored in this file. static inline constexpr auto DEFAULT_COMPRESSION_CODEC_FILE_NAME = "default_compression_codec.txt"; - static inline constexpr auto DELETE_ON_DESTROY_MARKER_FILE_NAME = "delete-on-destroy.txt"; + /// "delete-on-destroy.txt" is deprecated. It is no longer being created, only is removed. + static inline constexpr auto DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED = "delete-on-destroy.txt"; static inline constexpr auto UUID_FILE_NAME = "uuid.txt"; @@ -456,8 +457,10 @@ public: void writeChecksums(const MergeTreeDataPartChecksums & checksums_, const WriteSettings & settings); - void writeDeleteOnDestroyMarker(); + /// "delete-on-destroy.txt" is deprecated. It is no longer being created, only is removed. + /// TODO: remove this method after some time. void removeDeleteOnDestroyMarker(); + /// It may look like a stupid joke. but these two methods are absolutely unrelated. /// This one is about removing file with metadata about part version (for transactions) void removeVersionMetadata(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 83f5c0d359c..2def6fb08d3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1214,7 +1214,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( .build(); String part_path = fs::path(relative_data_path) / part_name; - String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; + String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED; if (part_disk_ptr->exists(marker_path)) { @@ -4410,7 +4410,6 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) /// All other locks are taken in StorageReplicatedMergeTree lockSharedData(*part_copy); - asMutableDeletingPart(original_active_part)->writeDeleteOnDestroyMarker(); return; } } @@ -7174,7 +7173,7 @@ std::pair MergeTreeData::cloneAn for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) { if (!files_to_copy_instead_of_hardlinks.contains(it->name()) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) { hardlinked_files->hardlinks_from_source_part.insert(it->name()); @@ -7189,7 +7188,7 @@ std::pair MergeTreeData::cloneAn { auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); if (!files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) { hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); From 967951cab7a29b14dcfb43b453494886b58fc3e4 Mon Sep 17 00:00:00 2001 From: Ziy1-Tan Date: Mon, 1 May 2023 21:24:44 +0800 Subject: [PATCH 0158/2223] Improve error message for function url --- src/Common/parseRemoteDescription.cpp | 43 +++++++++++++++++---------- src/Common/parseRemoteDescription.h | 3 +- 2 files changed, 30 insertions(+), 16 deletions(-) diff --git a/src/Common/parseRemoteDescription.cpp b/src/Common/parseRemoteDescription.cpp index aa7122ffb4c..7a50d5f4e45 100644 --- a/src/Common/parseRemoteDescription.cpp +++ b/src/Common/parseRemoteDescription.cpp @@ -64,7 +64,7 @@ static bool parseNumber(const String & description, size_t l, size_t r, size_t & * abc{1..9}de{f,g,h} - is a direct product, 27 shards. * abc{1..9}de{0|1} - is a direct product, 9 shards, in each 2 replicas. */ -std::vector parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses) +std::vector parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses, const String& func_name) { std::vector res; std::vector cur; @@ -97,28 +97,41 @@ std::vector parseRemoteDescription(const String & description, size_t l, if (cnt == 0) break; } if (cnt != 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': incorrect brace sequence in first argument"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}': incorrect brace sequence in first argument", func_name); /// The presence of a dot - numeric interval if (last_dot != -1) { size_t left, right; if (description[last_dot - 1] != '.') - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': incorrect argument in braces (only one dot): {}", - description.substr(i, m - i + 1)); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Table function '{}': incorrect argument in braces (only one dot): {}", + func_name, + description.substr(i, m - i + 1)); if (!parseNumber(description, i + 1, last_dot - 1, left)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': " - "incorrect argument in braces (Incorrect left number): {}", - description.substr(i, m - i + 1)); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Table function '{}': " + "incorrect argument in braces (Incorrect left number): {}", + func_name, + description.substr(i, m - i + 1)); if (!parseNumber(description, last_dot + 1, m, right)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': " - "incorrect argument in braces (Incorrect right number): {}", - description.substr(i, m - i + 1)); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Table function '{}': " + "incorrect argument in braces (Incorrect right number): {}", + func_name, + description.substr(i, m - i + 1)); if (left > right) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': " - "incorrect argument in braces (left number is greater then right): {}", - description.substr(i, m - i + 1)); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Table function '{}': " + "incorrect argument in braces (left number is greater then right): {}", + func_name, + description.substr(i, m - i + 1)); if (right - left + 1 > max_addresses) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': first argument generates too many result addresses"); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Table function '{}': first argument generates too many result addresses", func_name); bool add_leading_zeroes = false; size_t len = last_dot - 1 - (i + 1); /// If the left and right borders have equal numbers, then you must add leading zeros. @@ -161,7 +174,7 @@ std::vector parseRemoteDescription(const String & description, size_t l, res.insert(res.end(), cur.begin(), cur.end()); if (res.size() > max_addresses) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': first argument generates too many result addresses"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}': first argument generates too many result addresses", func_name); return res; } diff --git a/src/Common/parseRemoteDescription.h b/src/Common/parseRemoteDescription.h index eef8fbfefe4..e3e4a3f523c 100644 --- a/src/Common/parseRemoteDescription.h +++ b/src/Common/parseRemoteDescription.h @@ -15,7 +15,8 @@ namespace DB * abc{1..9}de{f,g,h} - is a direct product, 27 shards. * abc{1..9}de{0|1} - is a direct product, 9 shards, in each 2 replicas. */ -std::vector parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses); +std::vector parseRemoteDescription( + const String & description, size_t l, size_t r, char separator, size_t max_addresses, const String & func_name = "remote"); /// Parse remote description for external database (MySQL or PostgreSQL). std::vector> parseRemoteDescriptionForExternalDatabase(const String & description, size_t max_addresses, UInt16 default_port); From 2c159061edce46adf9dfd83a82129cb68ed1a9f3 Mon Sep 17 00:00:00 2001 From: Ziy1-Tan Date: Mon, 1 May 2023 21:40:14 +0800 Subject: [PATCH 0159/2223] Support `_path` and `_file` virtual columns for table function `url`. --- docs/en/sql-reference/table-functions/url.md | 9 ++ docs/ru/sql-reference/table-functions/url.md | 9 ++ docs/zh/sql-reference/table-functions/url.md | 8 ++ src/Storages/StorageURL.cpp | 86 ++++++++++++++++--- src/Storages/StorageURL.h | 2 + ...02725_url_support_virtual_column.reference | 5 ++ .../02725_url_support_virtual_column.sql | 9 ++ 7 files changed, 115 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/02725_url_support_virtual_column.reference create mode 100644 tests/queries/0_stateless/02725_url_support_virtual_column.sql diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 014dc3ae853..9b4a02e2393 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -46,3 +46,12 @@ SELECT * FROM test_table; Patterns in curly brackets `{ }` are used to generate a set of shards or to specify failover addresses. Supported pattern types and examples see in the description of the [remote](remote.md#globs-in-addresses) function. Character `|` inside patterns is used to specify failover addresses. They are iterated in the same order as listed in the pattern. The number of generated addresses is limited by [glob_expansion_max_elements](../../operations/settings/settings.md#glob_expansion_max_elements) setting. + +## Virtual Columns + +- `_path` — Path to the `URL`. +- `_file` — Resource name of the `URL`. + +**See Also** + +- [Virtual columns](/docs/en/engines/table-engines/index.md#table_engines-virtual_columns) diff --git a/docs/ru/sql-reference/table-functions/url.md b/docs/ru/sql-reference/table-functions/url.md index e5d9faeec00..ec9548229c8 100644 --- a/docs/ru/sql-reference/table-functions/url.md +++ b/docs/ru/sql-reference/table-functions/url.md @@ -46,3 +46,12 @@ SELECT * FROM test_table; Шаблоны в фигурных скобках `{ }` используются, чтобы сгенерировать список шардов или указать альтернативные адреса на случай отказа. Поддерживаемые типы шаблонов и примеры смотрите в описании функции [remote](remote.md#globs-in-addresses). Символ `|` внутри шаблонов используется, чтобы задать адреса, если предыдущие оказались недоступны. Эти адреса перебираются в том же порядке, в котором они указаны в шаблоне. Количество адресов, которые могут быть сгенерированы, ограничено настройкой [glob_expansion_max_elements](../../operations/settings/settings.md#glob_expansion_max_elements). + +## Виртуальные столбцы + +- `_path` — Путь до `URL`. +- `_file` — Имя ресурса `URL`. + +**Смотрите также** + +- [Виртуальные столбцы](index.md#table_engines-virtual_columns) diff --git a/docs/zh/sql-reference/table-functions/url.md b/docs/zh/sql-reference/table-functions/url.md index d3b7665d21b..c8ca9b775b2 100644 --- a/docs/zh/sql-reference/table-functions/url.md +++ b/docs/zh/sql-reference/table-functions/url.md @@ -41,3 +41,11 @@ CREATE TABLE test_table (column1 String, column2 UInt32) ENGINE=Memory; INSERT INTO FUNCTION url('http://127.0.0.1:8123/?query=INSERT+INTO+test_table+FORMAT+CSV', 'CSV', 'column1 String, column2 UInt32') VALUES ('http interface', 42); SELECT * FROM test_table; ``` +## 虚拟列 {#virtual-columns} + +- `_path` — `URL`路径。 +- `_file` — 资源名称。 + +**另请参阅** + +- [虚拟列](https://clickhouse.com/docs/en/operations/table_engines/#table_engines-virtual_columns) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index d2df3881c71..1847eccce12 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -38,6 +38,7 @@ #include #include #include +#include "DataTypes/DataTypeString.h" namespace DB @@ -159,6 +160,9 @@ namespace using FailoverOptions = std::vector; std::vector uri_list_to_read; std::atomic next_uri_to_read = 0; + + bool need_path_column = false; + bool need_file_column = false; }; using URIInfoPtr = std::shared_ptr; @@ -176,6 +180,27 @@ namespace } } + static Block getBlockForSource(const Block & block_for_format, const URIInfoPtr & uri_info) + { + auto res = block_for_format; + if (uri_info->need_path_column) + { + res.insert( + {DataTypeLowCardinality{std::make_shared()}.createColumn(), + std::make_shared(std::make_shared()), + "_path"}); + } + + if (uri_info->need_file_column) + { + res.insert( + {DataTypeLowCardinality{std::make_shared()}.createColumn(), + std::make_shared(std::make_shared()), + "_file"}); + } + return res; + } + StorageURLSource( URIInfoPtr uri_info_, const std::string & http_method, @@ -193,7 +218,7 @@ namespace const HTTPHeaderEntries & headers_ = {}, const URIParams & params = {}, bool glob_url = false) - : ISource(sample_block), name(std::move(name_)), uri_info(uri_info_) + : ISource(getBlockForSource(sample_block, uri_info_)), name(std::move(name_)), uri_info(uri_info_) { auto headers = getHeaders(headers_); @@ -204,7 +229,7 @@ namespace throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); auto first_option = uri_options.begin(); - auto buf_factory = getFirstAvailableURLReadBuffer( + auto [actual_uri, buf_factory] = getFirstAvailableURIAndReadBuffer( first_option, uri_options.end(), context, @@ -217,6 +242,8 @@ namespace glob_url, uri_options.size() == 1); + curr_uri = actual_uri; + try { total_size += buf_factory->getFileSize(); @@ -269,15 +296,32 @@ namespace if (current_uri_pos >= uri_info->uri_list_to_read.size()) return {}; - auto current_uri = uri_info->uri_list_to_read[current_uri_pos]; + auto current_uri_options = uri_info->uri_list_to_read[current_uri_pos]; - initialize(current_uri); + initialize(current_uri_options); } Chunk chunk; if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); + + const String & path{curr_uri.getPath()}; + if (uri_info->need_path_column) + { + auto column = DataTypeLowCardinality{std::make_shared()}.createColumnConst(num_rows, path); + chunk.addColumn(column->convertToFullColumnIfConst()); + } + + if (uri_info->need_file_column) + { + const size_t last_slash_pos = path.find_last_of('/'); + auto file_name = path.substr(last_slash_pos + 1); + auto column + = DataTypeLowCardinality{std::make_shared()}.createColumnConst(num_rows, std::move(file_name)); + chunk.addColumn(column->convertToFullColumnIfConst()); + } + if (num_rows && total_size) updateRowsProgressApprox( *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); @@ -291,7 +335,7 @@ namespace return {}; } - static SeekableReadBufferFactoryPtr getFirstAvailableURLReadBuffer( + static std::tuple getFirstAvailableURIAndReadBuffer( std::vector::const_iterator & option, const std::vector::const_iterator & end, ContextPtr context, @@ -352,7 +396,7 @@ namespace } } - return res; + return std::make_tuple(request_uri, std::move(res)); } throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message); @@ -364,6 +408,7 @@ namespace String name; URIInfoPtr uri_info; + Poco::URI curr_uri; std::unique_ptr pipeline; std::unique_ptr reader; @@ -538,10 +583,10 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( if (urlWithGlobs(uri)) { size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; - auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); + auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses, "url"); for (const auto & description : uri_descriptions) { - auto options = parseRemoteDescription(description, 0, description.size(), '|', max_addresses); + auto options = parseRemoteDescription(description, 0, description.size(), '|', max_addresses,"url"); urls_to_check.insert(urls_to_check.end(), options.begin(), options.end()); } } @@ -559,7 +604,7 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( if (it == urls_to_check.cend()) return nullptr; - auto buf_factory = StorageURLSource::getFirstAvailableURLReadBuffer( + auto [_, buf_factory] = StorageURLSource::getFirstAvailableURIAndReadBuffer( it, urls_to_check.cend(), context, @@ -621,18 +666,27 @@ Pipe IStorageURLBase::read( size_t max_download_threads = local_context->getSettingsRef().max_download_threads; + auto uri_info = std::make_shared(); + for (const auto & column : column_names) + { + if (column == "_path") + uri_info->need_path_column = true; + if (column == "_file") + uri_info->need_file_column = true; + } + if (urlWithGlobs(uri)) { size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; - auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); + auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses, "url"); if (num_streams > uri_descriptions.size()) num_streams = uri_descriptions.size(); /// For each uri (which acts like shard) check if it has failover options - auto uri_info = std::make_shared(); + uri_info->uri_list_to_read.reserve(uri_descriptions.size()); for (const auto & description : uri_descriptions) - uri_info->uri_list_to_read.emplace_back(parseRemoteDescription(description, 0, description.size(), '|', max_addresses)); + uri_info->uri_list_to_read.emplace_back(parseRemoteDescription(description, 0, description.size(), '|', max_addresses, "url")); Pipes pipes; pipes.reserve(num_streams); @@ -662,7 +716,6 @@ Pipe IStorageURLBase::read( } else { - auto uri_info = std::make_shared(); uri_info->uri_list_to_read.emplace_back(std::vector{uri}); return Pipe(std::make_shared( uri_info, @@ -771,6 +824,13 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad } } +NamesAndTypesList IStorageURLBase::getVirtuals() const +{ + return NamesAndTypesList{ + {"_path", std::make_shared(std::make_shared())}, + {"_file", std::make_shared(std::make_shared())}}; +} + SchemaCache & IStorageURLBase::getSchemaCache(const ContextPtr & context) { static SchemaCache schema_cache(context->getConfigRef().getUInt("schema_inference_cache_max_elements_for_url", DEFAULT_SCHEMA_CACHE_ELEMENTS)); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 65ee78e1e73..48498836e8d 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -43,6 +43,8 @@ public: bool supportsPartitionBy() const override { return true; } + NamesAndTypesList getVirtuals() const override; + static ColumnsDescription getTableStructureFromData( const String & format, const String & uri, diff --git a/tests/queries/0_stateless/02725_url_support_virtual_column.reference b/tests/queries/0_stateless/02725_url_support_virtual_column.reference new file mode 100644 index 00000000000..9d4051cc242 --- /dev/null +++ b/tests/queries/0_stateless/02725_url_support_virtual_column.reference @@ -0,0 +1,5 @@ +/ + + 1 +/ 1 +/other/pagecounts-raw/2007/2007-12/pagecounts-20071209-180000.gz pagecounts-20071209-180000.gz 856769 diff --git a/tests/queries/0_stateless/02725_url_support_virtual_column.sql b/tests/queries/0_stateless/02725_url_support_virtual_column.sql new file mode 100644 index 00000000000..417835a1e53 --- /dev/null +++ b/tests/queries/0_stateless/02725_url_support_virtual_column.sql @@ -0,0 +1,9 @@ +-- Tags: no-parallel + +select _path from url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); +select _file from url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); +select _file, count() from url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String') group by _file; +select _path, _file, s from url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); +select _path, _file, s from url('http://127.0.0.1:8123/?query=select+1&user=default&password=wrong', LineAsString, 's String'); -- { serverError RECEIVED_ERROR_FROM_REMOTE_IO_SERVER } + +SELECT _path, _file, count() FROM url('https://dumps.wikimedia.org/other/pagecounts-raw/2007/2007-12/pagecounts-20071209-180000.gz', LineAsString) group by _path, _file; From 64d232f1aa584b3eba5abf9fe02bfa9b0535701c Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Mon, 1 May 2023 18:00:26 +0000 Subject: [PATCH 0160/2223] Fix memory leak --- src/Interpreters/DatabaseCatalog.cpp | 2 +- tests/queries/0_stateless/02724_database_s3.reference | 6 +++--- tests/queries/0_stateless/02724_database_s3.sh | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f9e74fadcbd..129323cd6b3 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -346,7 +346,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( catch (const Exception & e) { if (exception) - exception->emplace(*e.clone()); + exception->emplace(e); } if (!table) diff --git a/tests/queries/0_stateless/02724_database_s3.reference b/tests/queries/0_stateless/02724_database_s3.reference index 8a985913ff9..b3800a27305 100644 --- a/tests/queries/0_stateless/02724_database_s3.reference +++ b/tests/queries/0_stateless/02724_database_s3.reference @@ -8,9 +8,9 @@ test1 13 14 15 16 17 18 0 0 0 -20 21 22 -23 24 25 -26 27 28 +10 11 12 +13 14 15 +16 17 18 0 0 0 Test 2: check exceptions OK diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index 4f9df402040..9b539407884 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -25,7 +25,7 @@ SELECT * FROM \"http://localhost:11111/test/b.tsv\" """ ${CLICKHOUSE_CLIENT} -q "DROP DATABASE test2;" -${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"http://localhost:11111/test/c.tsv\"" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"http://localhost:11111/test/b.tsv\"" ################# echo "Test 2: check exceptions" From 979b68a488c88bad53a44e4c9a8c525572091de5 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 1 May 2023 18:24:30 +0000 Subject: [PATCH 0161/2223] fix --- src/Common/OpenTelemetryTraceContext.cpp | 2 +- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 7 +------ src/Storages/MergeTree/MergeTreeReadPool.cpp | 3 ++- 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 0d89c581318..b68795814fb 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -120,7 +120,7 @@ SpanHolder::SpanHolder(std::string_view _operation_name, SpanKind _kind) this->start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - /// Add new initialization here + this->addAttribute("clickhouse.thread_id", getThreadId()); } catch (...) { diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index b186b9b1d28..de3edf29086 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -152,11 +152,6 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm if (use_external_buffer) local_read_settings.local_fs_buffer_size = 0; - // The buffer will unnecessarily allocate a Memory of size local_fs_buffer_size, which will then - // most likely be unused because we're swap()ping our own internal_buffer into - // implementation_buffer before each read. But we can't just set local_fs_buffer_size = 0 here - // because some buffer implementations actually use that memory (e.g. for prefetching). - auto buf = createReadBufferFromFileBase(path, local_read_settings); if (getFileSizeFromReadBuffer(*buf) == 0) @@ -827,7 +822,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } } - if (use_external_buffer && initialized) + if (use_external_buffer && !internal_buffer.empty()) internal_buffer.resize(original_buffer_size); chassert(!file_segment.isDownloader()); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 931a1be6b30..b3e3cf1361e 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -71,7 +71,8 @@ MergeTreeReadPool::MergeTreeReadPool( { const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); - const auto heuristic_min_marks = std::min(total_marks / threads_, min_bytes_per_task / avg_mark_bytes); + /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. + const auto heuristic_min_marks = std::min(total_marks / threads_ / 8, min_bytes_per_task / avg_mark_bytes); if (heuristic_min_marks > min_marks_for_concurrent_read) { min_marks_for_concurrent_read = heuristic_min_marks; From 95522ad7a6486bdbe5861c4f65c3a0ffe9610372 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Mon, 1 May 2023 21:46:17 +0000 Subject: [PATCH 0162/2223] Added DatabaseHDFS --- programs/local/LocalServer.cpp | 7 + src/Databases/DatabaseFactory.cpp | 31 ++- src/Databases/DatabaseHDFS.cpp | 228 ++++++++++++++++++ src/Databases/DatabaseHDFS.h | 65 +++++ .../0_stateless/02725_database_hdfs.reference | 16 ++ .../0_stateless/02725_database_hdfs.sh | 66 +++++ 6 files changed, 411 insertions(+), 2 deletions(-) create mode 100644 src/Databases/DatabaseHDFS.cpp create mode 100644 src/Databases/DatabaseHDFS.h create mode 100644 tests/queries/0_stateless/02725_database_hdfs.reference create mode 100755 tests/queries/0_stateless/02725_database_hdfs.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 215a92e1944..0cf94892171 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -65,6 +65,10 @@ #include #endif +#if USE_HDFS +#include +#endif + namespace fs = std::filesystem; @@ -163,6 +167,9 @@ static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, Co databaseCombiner->registerNextDatabase(std::make_shared(name_, context_)); #if USE_AWS_S3 databaseCombiner->registerNextDatabase(std::make_shared(name_, "", "", context_)); +#endif +#if USE_HDFS + databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); #endif return databaseCombiner; } diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index b21435527a5..5c4256c8a9f 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -53,6 +53,10 @@ #include #endif +#if USE_HDFS +#include +#endif + namespace fs = std::filesystem; namespace DB @@ -137,13 +141,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String static const std::unordered_set database_engines{"Ordinary", "Atomic", "Memory", "Dictionary", "Lazy", "Replicated", "MySQL", "MaterializeMySQL", "MaterializedMySQL", - "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3"}; + "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"}; if (!database_engines.contains(engine_name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine name `{}` does not exist", engine_name); static const std::unordered_set engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL", - "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3"}; + "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"}; static const std::unordered_set engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"}; bool engine_may_have_arguments = engines_with_arguments.contains(engine_name); @@ -437,6 +441,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared(context, engine_define, create.attach, database_path); } #endif + else if (engine_name == "Filesystem") { const ASTFunction * engine = engine_define->engine; @@ -455,6 +460,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared(database_name, init_path, context); } + #if USE_AWS_S3 else if (engine_name == "S3") { @@ -477,6 +483,27 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String } #endif +#if USE_HDFS + else if (engine_name == "HDFS") + { + const ASTFunction * engine = engine_define->engine; + + /// If source_url is empty, then table name must contain full url + std::string source_url; + + if (engine->arguments && !engine->arguments->children.empty()) + { + if (engine->arguments->children.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS database requires at most 1 argument: source_url"); + + const auto & arguments = engine->arguments->children; + source_url = safeGetLiteralValue(arguments[0], engine_name); + } + + return std::make_shared(database_name, source_url, context); + } +#endif + throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", engine_name); } diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp new file mode 100644 index 00000000000..39c3f955bf5 --- /dev/null +++ b/src/Databases/DatabaseHDFS.cpp @@ -0,0 +1,228 @@ +#include "config.h" + +#if USE_HDFS + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include + +namespace fs = std::filesystem; + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int UNKNOWN_TABLE; + extern const int BAD_ARGUMENTS; + extern const int FILE_DOESNT_EXIST; + extern const int UNACCEPTABLE_URL; + extern const int ACCESS_DENIED; + extern const int DATABASE_ACCESS_DENIED; + extern const int HDFS_ERROR; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; +} + +static constexpr std::string_view HDFS_HOST_REGEXP = "^hdfs://[^/]*"; + + +DatabaseHDFS::DatabaseHDFS(const String & name_, const String & source_url, ContextPtr context_) + : IDatabase(name_) + , WithContext(context_->getGlobalContext()) + , source(source_url) + , log(&Poco::Logger::get("DatabaseHDFS(" + name_ + ")")) +{ + if (!source.empty()) + { + if (!re2::RE2::FullMatch(source, std::string(HDFS_HOST_REGEXP))) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs host: {}. It should have structure 'hdfs://:'", source); + context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(source)); + } +} + +void DatabaseHDFS::addTable(const std::string & table_name, StoragePtr table_storage) const +{ + std::lock_guard lock(mutex); + auto [_, inserted] = loaded_tables.emplace(table_name, table_storage); + if (!inserted) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Table with name `{}` already exists in database `{}` (engine {})", + table_name, + getDatabaseName(), + getEngineName()); +} + +std::string DatabaseHDFS::getTablePath(const std::string & table_name) const +{ + if (table_name.starts_with("hdfs://")) + return table_name; + if (source.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}. It should have structure 'hdfs://:/path'", table_name); + return (fs::path(source) / table_name).string(); +} + +bool DatabaseHDFS::checkUrl(const std::string & url, ContextPtr context_, bool throw_on_error) const +{ + try + { + checkHDFSURL(url); + context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(url)); + } + catch (...) + { + if (throw_on_error) + throw; + return false; + } + + return true; +} + +bool DatabaseHDFS::isTableExist(const String & name, ContextPtr context_) const +{ + std::lock_guard lock(mutex); + if (loaded_tables.find(name) != loaded_tables.end()) + return true; + + return checkUrl(name, context_, false); +} + +StoragePtr DatabaseHDFS::getTableImpl(const String & name, ContextPtr context_) const +{ + // Check if the table exists in the loaded tables map + { + std::lock_guard lock(mutex); + auto it = loaded_tables.find(name); + if (it != loaded_tables.end()) + return it->second; + } + + auto url = getTablePath(name); + + checkUrl(url, context_, true); + + // call TableFunctionHDFS + auto args = makeASTFunction("hdfs", std::make_shared(url)); + + auto table_function = TableFunctionFactory::instance().get(args, context_); + if (!table_function) + return nullptr; + + // TableFunctionHDFS throws exceptions, if table cannot be created + auto table_storage = table_function->execute(args, context_, name); + if (table_storage) + addTable(name, table_storage); + + return table_storage; +} + +StoragePtr DatabaseHDFS::getTable(const String & name, ContextPtr context_) const +{ + // rethrow all exceptions from TableFunctionHDFS to show correct error to user + if (auto storage = getTableImpl(name, context_)) + return storage; + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name)); +} + +StoragePtr DatabaseHDFS::tryGetTable(const String & name, ContextPtr context_) const +{ + try + { + return getTableImpl(name, context_); + } + catch (const Exception & e) + { + // Ignore exceptions thrown by TableFunctionHDFS, which indicate that there is no table + if (e.code() == ErrorCodes::BAD_ARGUMENTS) + return nullptr; + if (e.code() == ErrorCodes::ACCESS_DENIED) + return nullptr; + if (e.code() == ErrorCodes::DATABASE_ACCESS_DENIED) + return nullptr; + if (e.code() == ErrorCodes::FILE_DOESNT_EXIST) + return nullptr; + if (e.code() == ErrorCodes::UNACCEPTABLE_URL) + return nullptr; + if (e.code() == ErrorCodes::HDFS_ERROR) + return nullptr; + if (e.code() == ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE) + return nullptr; + throw; + } + catch (const Poco::URISyntaxException &) + { + return nullptr; + } +} + +ASTPtr DatabaseHDFS::getCreateDatabaseQuery() const +{ + auto settings = getContext()->getSettingsRef(); + ParserCreateQuery parser; + + const String query = fmt::format("CREATE DATABASE {} ENGINE = HDFS('{}')", backQuoteIfNeed(getDatabaseName()), source); + ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth); + + if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) + { + auto & ast_create_query = ast->as(); + ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); + } + + return ast; +} + +void DatabaseHDFS::shutdown() +{ + Tables tables_snapshot; + { + std::lock_guard lock(mutex); + tables_snapshot = loaded_tables; + } + + for (const auto & kv : tables_snapshot) + { + auto table_id = kv.second->getStorageID(); + kv.second->flushAndShutdown(); + } + + std::lock_guard lock(mutex); + loaded_tables.clear(); +} + +/** + * Returns an empty vector because the database is read-only and no tables can be backed up + */ +std::vector> DatabaseHDFS::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const +{ + return {}; +} + +/** + * + * Returns an empty iterator because the database does not have its own tables + * But only caches them for quick access + */ +DatabaseTablesIteratorPtr DatabaseHDFS::getTablesIterator(ContextPtr, const FilterByNameFunction &) const +{ + return std::make_unique(Tables{}, getDatabaseName()); +} + +} // DB + +#endif diff --git a/src/Databases/DatabaseHDFS.h b/src/Databases/DatabaseHDFS.h new file mode 100644 index 00000000000..4e2b8578fcd --- /dev/null +++ b/src/Databases/DatabaseHDFS.h @@ -0,0 +1,65 @@ +#pragma once + +#include "config.h" + +#if USE_HDFS + +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; + +/** + * DatabaseHDFS allows to interact with files stored on the file system + * Uses TableFunctionHDFS to implicitly load file when a user requests the table, and provides read-only access to the data in the file + * Tables are cached inside the database for quick access + */ +class DatabaseHDFS : public IDatabase, protected WithContext +{ +public: + DatabaseHDFS(const String & name, const String & source_url, ContextPtr context); + + String getEngineName() const override { return "S3"; } + + bool isTableExist(const String & name, ContextPtr context) const override; + + StoragePtr getTable(const String & name, ContextPtr context) const override; + + StoragePtr tryGetTable(const String & name, ContextPtr context) const override; + + bool empty() const override { return true; } + + bool isReadOnly() const override { return true; } + + ASTPtr getCreateDatabaseQuery() const override; + + void shutdown() override; + + std::vector> getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; + +protected: + StoragePtr getTableImpl(const String & url, ContextPtr context) const; + + void addTable(const std::string & table_name, StoragePtr table_storage) const; + + bool checkUrl(const std::string & name, ContextPtr context_, bool throw_on_error) const; + + std::string getTablePath(const std::string & table_name) const; + +private: + const String source; + + mutable Tables loaded_tables TSA_GUARDED_BY(mutex); + Poco::Logger * log; +}; + +} // DB + +#endif diff --git a/tests/queries/0_stateless/02725_database_hdfs.reference b/tests/queries/0_stateless/02725_database_hdfs.reference new file mode 100644 index 00000000000..2a2e6c20aaa --- /dev/null +++ b/tests/queries/0_stateless/02725_database_hdfs.reference @@ -0,0 +1,16 @@ +Test 1: select from hdfs database +1 2 3 +test1 +1 2 3 +test2 +4 5 6 +Test 2: check exceptions +OK0 +OK1 +OK2 +OK3 +OK4 +OK5 +OK6 +OK7 +OK8 diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh new file mode 100755 index 00000000000..ea16dd4024c --- /dev/null +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -0,0 +1,66 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, use-hdfs + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Prepare data +${CLICKHOUSE_CLIENT} -q "insert into table function hdfs('hdfs://localhost:12222/test_02725_1.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') select 1, 2, 3 settings hdfs_truncate_on_insert=1;" +${CLICKHOUSE_CLIENT} -q "insert into table function hdfs('hdfs://localhost:12222/test_02725_2.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') select 4, 5, 6 settings hdfs_truncate_on_insert=1;" + +################# +echo "Test 1: select from hdfs database" + +# Database without specific host +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test1; +CREATE DATABASE test1 ENGINE = HDFS; +USE test1; +SELECT * FROM \"hdfs://localhost:12222/test_02725_1.tsv\" +""" +${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test1 + +# Database with host +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test2; +CREATE DATABASE test2 ENGINE = HDFS('hdfs://localhost:12222'); +USE test2; +SELECT * FROM \"test_02725_1.tsv\" +""" +${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test2 + +# Check implicit call in clickhouse-local +${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"hdfs://localhost:12222/test_02725_2.tsv\"" + +################# +echo "Test 2: check exceptions" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"hdfs://localhost:12222/file.myext\"" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK0" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"hdfs://localhost:12222/test_02725_3.tsv\"" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK1" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"hdfs://localhost:12222\"" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK2" + +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test3; +CREATE DATABASE test3 ENGINE = HDFS('abacaba'); +""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK3" + +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test4; +CREATE DATABASE test4 ENGINE = HDFS; +USE test4; +SELECT * FROM \"abacaba/file.tsv\" +""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK4" + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK5" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK6" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1| grep -F "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK7" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK8" + + +# Cleanup +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test1; +DROP DATABASE IF EXISTS test2; +DROP DATABASE IF EXISTS test3; +DROP DATABASE IF EXISTS test4; +""" \ No newline at end of file From c93ceedbef7422e5cb7a3635afa9e2525e3c78b6 Mon Sep 17 00:00:00 2001 From: Ziy1-Tan Date: Tue, 2 May 2023 10:38:37 +0800 Subject: [PATCH 0163/2223] Fix style Signed-off-by: Ziy1-Tan --- src/Common/parseRemoteDescription.cpp | 3 ++- src/Storages/StorageURL.cpp | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/parseRemoteDescription.cpp b/src/Common/parseRemoteDescription.cpp index 7a50d5f4e45..0bcd62d30c7 100644 --- a/src/Common/parseRemoteDescription.cpp +++ b/src/Common/parseRemoteDescription.cpp @@ -64,7 +64,8 @@ static bool parseNumber(const String & description, size_t l, size_t r, size_t & * abc{1..9}de{f,g,h} - is a direct product, 27 shards. * abc{1..9}de{0|1} - is a direct product, 9 shards, in each 2 replicas. */ -std::vector parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses, const String& func_name) +std::vector +parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses, const String & func_name) { std::vector res; std::vector cur; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 1847eccce12..8a4ea3631e5 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -586,7 +586,7 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses, "url"); for (const auto & description : uri_descriptions) { - auto options = parseRemoteDescription(description, 0, description.size(), '|', max_addresses,"url"); + auto options = parseRemoteDescription(description, 0, description.size(), '|', max_addresses, "url"); urls_to_check.insert(urls_to_check.end(), options.begin(), options.end()); } } From 10f9039e00320fc7b38ac6524a5940872859707e Mon Sep 17 00:00:00 2001 From: Ziy1-Tan Date: Tue, 2 May 2023 15:35:58 +0800 Subject: [PATCH 0164/2223] Fix tests --- .../0_stateless/00646_url_engine.python | 22 ++++++++++++++++++- ...02725_url_support_virtual_column.reference | 1 - .../02725_url_support_virtual_column.sql | 1 - 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00646_url_engine.python b/tests/queries/0_stateless/00646_url_engine.python index 0a26f8039c2..a5ab02f4b0e 100644 --- a/tests/queries/0_stateless/00646_url_engine.python +++ b/tests/queries/0_stateless/00646_url_engine.python @@ -7,6 +7,7 @@ import tempfile import threading import os import traceback +from urllib.parse import urljoin import urllib.request import subprocess from io import StringIO @@ -163,6 +164,7 @@ def test_select( requests=[], answers=[], test_data="", + res_path="", ): with open(CSV_DATA, "w") as f: # clear file f.write("") @@ -183,7 +185,7 @@ def test_select( tbl = table_name if not tbl: tbl = "url('{addr}', 'CSV', '{schema}')".format( - addr=HTTP_SERVER_URL_STR, schema=schema + addr=urljoin(HTTP_SERVER_URL_STR, res_path), schema=schema ) check_answers(requests[i].format(tbl=tbl), answers[i]) @@ -252,6 +254,16 @@ def main(): "select double, count(*) from {tbl} group by double order by double": "7.7\t2\n9.9\t10", } + pathname = CSV_DATA + filename = os.path.basename(CSV_DATA) + select_virtual_requests = { + "select _path from {tbl}": '\n'.join([pathname]for _ in range(2)), + "select _file from {tbl}": '\n'.join([filename]for _ in range(2)), + "select _file, from {tbl} order by _path": '\n'.join([filename]for _ in range(2)), + "select _path, _file from {tbl}": '\n'.join(f'{pathname}\t{filename}' for _ in range(2)), + "select _path, count(*) from {tbl} group by _path": f'{pathname}\t2', + } + t, httpd = start_server() t.start() # test table with url engine @@ -267,6 +279,14 @@ def main(): answers=list(select_only_requests.values()), test_data=test_data, ) + # test table function url for virtual column + test_select( + requests=list(select_virtual_requests.keys()), + answers=list(select_virtual_requests.values()), + test_data=test_data, + res_path=CSV_DATA, + ) + # test insert into table with url engine test_insert( table_name="test_table_insert", diff --git a/tests/queries/0_stateless/02725_url_support_virtual_column.reference b/tests/queries/0_stateless/02725_url_support_virtual_column.reference index 9d4051cc242..d20c91344e5 100644 --- a/tests/queries/0_stateless/02725_url_support_virtual_column.reference +++ b/tests/queries/0_stateless/02725_url_support_virtual_column.reference @@ -2,4 +2,3 @@ 1 / 1 -/other/pagecounts-raw/2007/2007-12/pagecounts-20071209-180000.gz pagecounts-20071209-180000.gz 856769 diff --git a/tests/queries/0_stateless/02725_url_support_virtual_column.sql b/tests/queries/0_stateless/02725_url_support_virtual_column.sql index 417835a1e53..02a4dfc449b 100644 --- a/tests/queries/0_stateless/02725_url_support_virtual_column.sql +++ b/tests/queries/0_stateless/02725_url_support_virtual_column.sql @@ -6,4 +6,3 @@ select _file, count() from url('http://127.0.0.1:8123/?query=select+1&user=defau select _path, _file, s from url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); select _path, _file, s from url('http://127.0.0.1:8123/?query=select+1&user=default&password=wrong', LineAsString, 's String'); -- { serverError RECEIVED_ERROR_FROM_REMOTE_IO_SERVER } -SELECT _path, _file, count() FROM url('https://dumps.wikimedia.org/other/pagecounts-raw/2007/2007-12/pagecounts-20071209-180000.gz', LineAsString) group by _path, _file; From bb44a38efe10831164a1a3c286b28d9038a41c47 Mon Sep 17 00:00:00 2001 From: Ziy1-Tan Date: Tue, 2 May 2023 15:54:35 +0800 Subject: [PATCH 0165/2223] Fix style Signed-off-by: Ziy1-Tan --- tests/queries/0_stateless/00646_url_engine.python | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00646_url_engine.python b/tests/queries/0_stateless/00646_url_engine.python index a5ab02f4b0e..7e2171eaec5 100644 --- a/tests/queries/0_stateless/00646_url_engine.python +++ b/tests/queries/0_stateless/00646_url_engine.python @@ -257,11 +257,15 @@ def main(): pathname = CSV_DATA filename = os.path.basename(CSV_DATA) select_virtual_requests = { - "select _path from {tbl}": '\n'.join([pathname]for _ in range(2)), - "select _file from {tbl}": '\n'.join([filename]for _ in range(2)), - "select _file, from {tbl} order by _path": '\n'.join([filename]for _ in range(2)), - "select _path, _file from {tbl}": '\n'.join(f'{pathname}\t{filename}' for _ in range(2)), - "select _path, count(*) from {tbl} group by _path": f'{pathname}\t2', + "select _path from {tbl}": "\n".join([pathname] for _ in range(2)), + "select _file from {tbl}": "\n".join([filename] for _ in range(2)), + "select _file, from {tbl} order by _path": "\n".join( + [filename] for _ in range(2) + ), + "select _path, _file from {tbl}": "\n".join( + f"{pathname}\t{filename}" for _ in range(2) + ), + "select _path, count(*) from {tbl} group by _path": f"{pathname}\t2", } t, httpd = start_server() From 1bb0d1519e69ec78fa2e8895e001e0efa2ee84f7 Mon Sep 17 00:00:00 2001 From: Ziy1-Tan Date: Tue, 2 May 2023 16:54:14 +0800 Subject: [PATCH 0166/2223] Fix style Signed-off-by: Ziy1-Tan --- src/Storages/StorageURL.cpp | 2 +- tests/queries/0_stateless/00646_url_engine.python | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 8a4ea3631e5..0d36319d894 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -38,7 +38,7 @@ #include #include #include -#include "DataTypes/DataTypeString.h" +#include namespace DB diff --git a/tests/queries/0_stateless/00646_url_engine.python b/tests/queries/0_stateless/00646_url_engine.python index 7e2171eaec5..dc0fdd1a71d 100644 --- a/tests/queries/0_stateless/00646_url_engine.python +++ b/tests/queries/0_stateless/00646_url_engine.python @@ -257,10 +257,10 @@ def main(): pathname = CSV_DATA filename = os.path.basename(CSV_DATA) select_virtual_requests = { - "select _path from {tbl}": "\n".join([pathname] for _ in range(2)), - "select _file from {tbl}": "\n".join([filename] for _ in range(2)), + "select _path from {tbl}": "\n".join(pathname for _ in range(2)), + "select _file from {tbl}": "\n".join(filename for _ in range(2)), "select _file, from {tbl} order by _path": "\n".join( - [filename] for _ in range(2) + filename for _ in range(2) ), "select _path, _file from {tbl}": "\n".join( f"{pathname}\t{filename}" for _ in range(2) From 8205398f31459ea8c8bbaa4dac2bab106d4d188b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 2 May 2023 16:31:39 +0200 Subject: [PATCH 0167/2223] Fixed comment --- src/Interpreters/Context.h | 3 --- src/Storages/StorageFile.cpp | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 81804372571..9f99a3295dd 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -729,9 +729,6 @@ public: BackupsWorker & getBackupsWorker() const; /// I/O formats. - /// If number_of_streams is set then the number of threads are divided among the streams - /// Eg: max_threads = 16, number_of_streams = 8, then each stream will use 2 threads - /// This is done to improve performance InputFormatPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings = std::nullopt, const std::optional max_parsing_threads = std::nullopt) const; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index d4c5ee25502..008fae60966 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -609,7 +609,7 @@ public: const Settings & settings = context->getSettingsRef(); chassert(!storage->paths.empty()); - const auto max_parsing_threads = std::max(settings.max_threads/storage->paths.size(),1UL); + const auto max_parsing_threads = std::max(settings.max_threads / storage->paths.size(), 1UL); auto format = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, max_parsing_threads); From 82bb1e8bf2a3183179938629cc8f6aab3d876e87 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Tue, 2 May 2023 18:51:35 +0000 Subject: [PATCH 0168/2223] Fix build and try fix tests --- src/Databases/DatabaseHDFS.h | 4 ++-- tests/queries/0_stateless/02724_database_s3.sh | 2 +- tests/queries/0_stateless/02725_database_hdfs.sh | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseHDFS.h b/src/Databases/DatabaseHDFS.h index 4e2b8578fcd..9a506c5c8ac 100644 --- a/src/Databases/DatabaseHDFS.h +++ b/src/Databases/DatabaseHDFS.h @@ -45,11 +45,11 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; protected: - StoragePtr getTableImpl(const String & url, ContextPtr context) const; + StoragePtr getTableImpl(const String & name, ContextPtr context) const; void addTable(const std::string & table_name, StoragePtr table_storage) const; - bool checkUrl(const std::string & name, ContextPtr context_, bool throw_on_error) const; + bool checkUrl(const std::string & url, ContextPtr context_, bool throw_on_error) const; std::string getTablePath(const std::string & table_name) const; diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index 9b539407884..af858d140d7 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel # Tag no-fasttest: Depends on AWS CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index ea16dd4024c..8d4e982504a 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, use-hdfs +# Tags: no-fasttest, use-hdfs, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 58cb6c7837872ae4eb46eed84d5aa0d75607d661 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Tue, 2 May 2023 19:57:36 +0000 Subject: [PATCH 0169/2223] S3, HDFS only for explicit creation --- programs/local/LocalServer.cpp | 14 -------------- .../0_stateless/02724_database_s3.reference | 5 ----- tests/queries/0_stateless/02724_database_s3.sh | 4 ---- .../0_stateless/02725_database_hdfs.reference | 4 ---- .../queries/0_stateless/02725_database_hdfs.sh | 18 ++++++------------ 5 files changed, 6 insertions(+), 39 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 0cf94892171..b413483686a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -61,14 +61,6 @@ # include #endif -#if USE_AWS_S3 -#include -#endif - -#if USE_HDFS -#include -#endif - namespace fs = std::filesystem; @@ -165,12 +157,6 @@ static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, Co auto databaseCombiner = std::make_shared(name_, context_); databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); databaseCombiner->registerNextDatabase(std::make_shared(name_, context_)); -#if USE_AWS_S3 - databaseCombiner->registerNextDatabase(std::make_shared(name_, "", "", context_)); -#endif -#if USE_HDFS - databaseCombiner->registerNextDatabase(std::make_shared(name_, "", context_)); -#endif return databaseCombiner; } diff --git a/tests/queries/0_stateless/02724_database_s3.reference b/tests/queries/0_stateless/02724_database_s3.reference index b3800a27305..72ba0e240b1 100644 --- a/tests/queries/0_stateless/02724_database_s3.reference +++ b/tests/queries/0_stateless/02724_database_s3.reference @@ -8,11 +8,6 @@ test1 13 14 15 16 17 18 0 0 0 -10 11 12 -13 14 15 -16 17 18 -0 0 0 Test 2: check exceptions OK OK -OK diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index af858d140d7..2758580a355 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -25,12 +25,8 @@ SELECT * FROM \"http://localhost:11111/test/b.tsv\" """ ${CLICKHOUSE_CLIENT} -q "DROP DATABASE test2;" -${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"http://localhost:11111/test/b.tsv\"" - ################# echo "Test 2: check exceptions" -${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"http://localhost:11111/test/c.myext\"" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK" - ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = S3; diff --git a/tests/queries/0_stateless/02725_database_hdfs.reference b/tests/queries/0_stateless/02725_database_hdfs.reference index 2a2e6c20aaa..ef8adae2bbc 100644 --- a/tests/queries/0_stateless/02725_database_hdfs.reference +++ b/tests/queries/0_stateless/02725_database_hdfs.reference @@ -3,7 +3,6 @@ Test 1: select from hdfs database test1 1 2 3 test2 -4 5 6 Test 2: check exceptions OK0 OK1 @@ -11,6 +10,3 @@ OK2 OK3 OK4 OK5 -OK6 -OK7 -OK8 diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index 8d4e982504a..a78f3e6bbdc 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -30,31 +30,25 @@ SELECT * FROM \"test_02725_1.tsv\" """ ${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test2 -# Check implicit call in clickhouse-local -${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"hdfs://localhost:12222/test_02725_2.tsv\"" - ################# echo "Test 2: check exceptions" -${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"hdfs://localhost:12222/file.myext\"" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK0" -${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"hdfs://localhost:12222/test_02725_3.tsv\"" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK1" -${CLICKHOUSE_LOCAL} --query "SELECT * FROM \"hdfs://localhost:12222\"" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK2" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = HDFS('abacaba'); -""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK3" +""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK0" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test4; CREATE DATABASE test4 ENGINE = HDFS; USE test4; SELECT * FROM \"abacaba/file.tsv\" -""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK4" +""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK1" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK5" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK6" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1| grep -F "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK7" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK8" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK2" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK3" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1| grep -F "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK4" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK5" # Cleanup From d624066ed5a85b8398f4cf22415e24ad863f0818 Mon Sep 17 00:00:00 2001 From: Ziy1-Tan Date: Wed, 3 May 2023 13:50:48 +0800 Subject: [PATCH 0170/2223] Fix Style From 30d216f863583e7216b5cc5f9144f33068092d44 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 3 May 2023 13:49:46 +0000 Subject: [PATCH 0171/2223] fix --- src/Storages/MergeTree/MergeTreeReadPool.cpp | 43 +++++++++++--------- 1 file changed, 23 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index b3e3cf1361e..34e270fdd24 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -1,10 +1,10 @@ -#include -#include #include +#include +#include +#include #include #include #include -#include namespace ProfileEvents @@ -56,26 +56,29 @@ MergeTreeReadPool::MergeTreeReadPool( , backoff_settings{context_->getSettingsRef()} , backoff_state{threads_} { - const auto & settings = context_->getSettingsRef(); - - size_t total_compressed_bytes = 0; - size_t total_marks = 0; - for (const auto & part : parts_ranges) + if (std::ranges::count(is_part_on_remote_disk, true)) { - total_compressed_bytes += getApproxSizeOfPart( - *part.data_part, prewhere_info ? prewhere_info->prewhere_actions->getRequiredColumnsNames() : column_names_); - total_marks += part.getMarksCount(); - } + const auto & settings = context_->getSettingsRef(); - if (total_marks) - { - const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; - const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); - /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. - const auto heuristic_min_marks = std::min(total_marks / threads_ / 8, min_bytes_per_task / avg_mark_bytes); - if (heuristic_min_marks > min_marks_for_concurrent_read) + size_t total_compressed_bytes = 0; + size_t total_marks = 0; + for (const auto & part : parts_ranges) { - min_marks_for_concurrent_read = heuristic_min_marks; + total_compressed_bytes += getApproxSizeOfPart( + *part.data_part, prewhere_info ? prewhere_info->prewhere_actions->getRequiredColumnsNames() : column_names_); + total_marks += part.getMarksCount(); + } + + if (total_marks) + { + const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; + const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); + /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. + const auto heuristic_min_marks = std::min(total_marks / threads_, min_bytes_per_task / avg_mark_bytes); + if (heuristic_min_marks > min_marks_for_concurrent_read) + { + min_marks_for_concurrent_read = heuristic_min_marks; + } } } From 913b63edc93c80f8bfaedfe2332859fc5dab83d3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 3 May 2023 21:28:33 +0200 Subject: [PATCH 0172/2223] Fix another zero copy bug --- src/Storages/MergeTree/MergeTreeData.cpp | 4 +-- src/Storages/MergeTree/ZeroCopyLock.cpp | 2 +- src/Storages/MergeTree/ZeroCopyLock.h | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 39 ++++++++++++++++++--- 4 files changed, 40 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fd98db7962e..e4181a5f9de 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7202,8 +7202,8 @@ std::pair MergeTreeData::cloneAn copy_instead_of_hardlink, files_to_copy_instead_of_hardlinks); - LOG_DEBUG(log, "Clone {} part {} to {}{}", - src_flushed_tmp_part ? "flushed" : "", + LOG_DEBUG(log, "Clone{} part {} to {}{}", + src_flushed_tmp_part ? " flushed" : "", src_part_storage->getFullPath(), std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), with_copy); diff --git a/src/Storages/MergeTree/ZeroCopyLock.cpp b/src/Storages/MergeTree/ZeroCopyLock.cpp index 53dfe0c769f..cca005dd7c0 100644 --- a/src/Storages/MergeTree/ZeroCopyLock.cpp +++ b/src/Storages/MergeTree/ZeroCopyLock.cpp @@ -3,7 +3,7 @@ namespace DB { ZeroCopyLock::ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path, const std::string & lock_message) - : lock(zkutil::createSimpleZooKeeperLock(zookeeper, lock_path, "part_exclusive_lock", lock_message)) + : lock(zkutil::createSimpleZooKeeperLock(zookeeper, lock_path, ZERO_COPY_LOCK_NAME, lock_message)) { } } diff --git a/src/Storages/MergeTree/ZeroCopyLock.h b/src/Storages/MergeTree/ZeroCopyLock.h index 4400ea55b8f..2803952af18 100644 --- a/src/Storages/MergeTree/ZeroCopyLock.h +++ b/src/Storages/MergeTree/ZeroCopyLock.h @@ -12,6 +12,8 @@ namespace DB /// because due to bad abstraction we use it in MergeTreeData. struct ZeroCopyLock { + static inline const std::string_view ZERO_COPY_LOCK_NAME = "part_exclusive_lock"; + ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path, const std::string & lock_message); bool isLocked() const { return lock->isLocked(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 20839a61c92..94abc1422fd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8249,7 +8249,7 @@ void StorageReplicatedMergeTree::lockSharedData( { String zookeeper_node = fs::path(zc_zookeeper_path) / id / replica_name; - LOG_TRACE(log, "Trying to create zookeeper persistent lock {}", zookeeper_node); + LOG_TRACE(log, "Trying to create zookeeper persistent lock {} with hardlinks [{}]", zookeeper_node, fmt::join(hardlinks, ", ")); createZeroCopyLockNode( zookeeper, zookeeper_node, zkutil::CreateMode::Persistent, @@ -8362,7 +8362,7 @@ namespace /// But sometimes we need an opposite. When we deleting all_0_0_0_1 it can be non replicated to other replicas, so we are the only owner of this part. /// In this case when we will drop all_0_0_0_1 we will drop blobs for all_0_0_0. But it will lead to dataloss. For such case we need to check that other replicas /// still need parent part. -std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const MergeTreePartInfo & part_info, MergeTreeDataFormatVersion format_version, Poco::Logger * log) +std::pair> getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const MergeTreePartInfo & part_info, MergeTreeDataFormatVersion format_version, Poco::Logger * log) { NameSet files_not_to_remove; @@ -8404,15 +8404,40 @@ std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionP /// Get hardlinked files String files_not_to_remove_str; Coordination::Error code; - zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, nullptr, &code); + zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, &code); if (code != Coordination::Error::ZOK) + { LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); + return {true, std::nullopt}; + } if (!files_not_to_remove_str.empty()) { boost::split(files_not_to_remove, files_not_to_remove_str, boost::is_any_of("\n ")); LOG_TRACE(log, "Found files not to remove from parent part {}: [{}]", part_candidate_info_str, fmt::join(files_not_to_remove, ", ")); } + else + { + std::vector children; + code = zookeeper_ptr->tryGetChildren(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, children); + if (code != Coordination::Error::ZOK) + { + LOG_TRACE(log, "Cannot get parent locks in ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); + return {true, std::nullopt}; + } + + if (children.size() > 1 || children.size() == 1 && children[0] != ZeroCopyLock::ZERO_COPY_LOCK_NAME) + { + LOG_TRACE(log, "No files not to remove found for part {} from parent {}", part_info_str, part_candidate_info_str); + } + else + { + /// The case when part is actually removed, but some stale replica trying to execute merge/mutation. + /// We shouldn't use the part to check hardlinked blobs, it just doesn't exist. + LOG_TRACE(log, "Part {} is not parent (only merge/mutation locks exist), refusing to use as parent", part_candidate_info_str); + continue; + } + } return {true, files_not_to_remove}; } @@ -8448,6 +8473,12 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( auto [has_parent, parent_not_to_remove] = getParentLockedBlobs( zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_info, data_format_version, logger); + if (has_parent && parent_not_to_remove == std::nullopt) + { + LOG_TRACE(logger, "Failed to get mutation parent on {} for part {}, refusing to remove blobs", zookeeper_part_replica_node, part_name); + return {false, {}}; + } + files_not_to_remove.insert(parent_not_to_remove.begin(), parent_not_to_remove.end()); String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / part_id; @@ -8527,7 +8558,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( if (error_code == Coordination::Error::ZOK) { - LOG_TRACE(logger, "Removed last parent zookeeper lock {} for part {} (part is finally unlocked)", zookeeper_part_uniq_node, part_name); + LOG_TRACE(logger, "Removed last parent zookeeper lock {} for part {} (part is finally unlocked)", zookeeper_part_node, part_name); } else if (error_code == Coordination::Error::ZNOTEMPTY) { From 610e63bfd2b21e06c312c625e54279c31d06853c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 3 May 2023 23:54:24 +0200 Subject: [PATCH 0173/2223] Fix build --- src/Storages/MergeTree/ZeroCopyLock.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 19 +++++++++++-------- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/ZeroCopyLock.h b/src/Storages/MergeTree/ZeroCopyLock.h index 2803952af18..d4c829a3652 100644 --- a/src/Storages/MergeTree/ZeroCopyLock.h +++ b/src/Storages/MergeTree/ZeroCopyLock.h @@ -12,7 +12,7 @@ namespace DB /// because due to bad abstraction we use it in MergeTreeData. struct ZeroCopyLock { - static inline const std::string_view ZERO_COPY_LOCK_NAME = "part_exclusive_lock"; + static inline const auto ZERO_COPY_LOCK_NAME = "part_exclusive_lock"; ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path, const std::string & lock_message); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 94abc1422fd..280150f27ad 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8404,7 +8404,7 @@ std::pair> getParentLockedBlobs(const ZooKeeperWith /// Get hardlinked files String files_not_to_remove_str; Coordination::Error code; - zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, &code); + zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, nullptr, &code); if (code != Coordination::Error::ZOK) { LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); @@ -8426,7 +8426,7 @@ std::pair> getParentLockedBlobs(const ZooKeeperWith return {true, std::nullopt}; } - if (children.size() > 1 || children.size() == 1 && children[0] != ZeroCopyLock::ZERO_COPY_LOCK_NAME) + if (children.size() > 1 || (children.size() == 1 && children[0] != ZeroCopyLock::ZERO_COPY_LOCK_NAME)) { LOG_TRACE(log, "No files not to remove found for part {} from parent {}", part_info_str, part_candidate_info_str); } @@ -8471,20 +8471,23 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( if (!files_not_to_remove_str.empty()) boost::split(files_not_to_remove, files_not_to_remove_str, boost::is_any_of("\n ")); + String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / part_id; + + /// Delete our replica node for part from zookeeper (we are not interested in it anymore) + String zookeeper_part_replica_node = fs::path(zookeeper_part_uniq_node) / replica_name_; + auto [has_parent, parent_not_to_remove] = getParentLockedBlobs( zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_info, data_format_version, logger); - if (has_parent && parent_not_to_remove == std::nullopt) + + // parent_not_to_remove == std::nullopt means that we were unable to retrieve parts set + if (has_parent || parent_not_to_remove == std::nullopt) { LOG_TRACE(logger, "Failed to get mutation parent on {} for part {}, refusing to remove blobs", zookeeper_part_replica_node, part_name); return {false, {}}; } - files_not_to_remove.insert(parent_not_to_remove.begin(), parent_not_to_remove.end()); + files_not_to_remove.insert(parent_not_to_remove->begin(), parent_not_to_remove->end()); - String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / part_id; - - /// Delete our replica node for part from zookeeper (we are not interested in it anymore) - String zookeeper_part_replica_node = fs::path(zookeeper_part_uniq_node) / replica_name_; LOG_TRACE(logger, "Remove zookeeper lock {} for part {}", zookeeper_part_replica_node, part_name); From 0f9271a34979fe44a31e4a950a0823b4f3ff3338 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 4 May 2023 12:10:06 +0000 Subject: [PATCH 0174/2223] Passing sort description apart of fill description --- src/Interpreters/FillingRow.cpp | 10 ++++--- src/Interpreters/FillingRow.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 13 +++++----- src/Planner/Planner.cpp | 6 +++-- src/Processors/QueryPlan/FillingStep.cpp | 14 +++++++--- src/Processors/QueryPlan/FillingStep.h | 7 ++++- .../Transforms/FillingTransform.cpp | 26 +++++++++++++++---- src/Processors/Transforms/FillingTransform.h | 13 +++++++--- 8 files changed, 65 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 05795842902..98197533aa1 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -67,12 +67,14 @@ bool FillingRow::next(const FillingRow & to_row) /// we need to generate rows up to 'fill_to' value. for (size_t i = row_size - 1; i > pos; --i) { - if (getFillDescription(i).fill_to.isNull() || row[i].isNull()) + auto & fill_column_desc = getFillDescription(i); + + if (fill_column_desc.fill_to.isNull() || row[i].isNull()) continue; - auto next_value = row[i]; - getFillDescription(i).step_func(next_value); - if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) + Field next_value = row[i]; + fill_column_desc.step_func(next_value); + if (less(next_value, fill_column_desc.fill_to, getDirection(i))) { row[i] = next_value; initFromDefaults(i + 1); diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index c56bd875151..9f414d92dd3 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -32,6 +32,7 @@ public: int getDirection(size_t index) const { return sort_description[index].direction; } FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } + const FillColumnDescription & getFillDescription(size_t index) const { return sort_description[index].fill_description; } String dump() const; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index afd34cb044e..e36a1287503 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2995,20 +2995,21 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query, context); - SortDescription fill_descr; - for (auto & desc : order_descr) + SortDescription sort_description = getSortDescription(query, context); + SortDescription fill_description; + for (auto & desc : sort_description) { if (desc.with_fill) - fill_descr.push_back(desc); + fill_description.push_back(desc); } - if (fill_descr.empty()) + if (fill_description.empty()) return; InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); - auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_descr), interpolate_descr); + auto filling_step = std::make_unique( + query_plan.getCurrentDataStream(), std::move(sort_description), std::move(fill_description), interpolate_descr); query_plan.addStep(std::move(filling_step)); } } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d036c895fbb..e3a4d01a46a 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -622,7 +622,8 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, interpolate_description = std::make_shared(std::move(interpolate_actions_dag), empty_aliases); } - auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_description), interpolate_description); + auto filling_step = std::make_unique( + query_plan.getCurrentDataStream(), std::move(sort_description), std::move(fill_description), interpolate_description); query_plan.addStep(std::move(filling_step)); } @@ -1432,7 +1433,8 @@ void Planner::buildPlanForQueryNode() addLimitByStep(query_plan, limit_by_analysis_result, query_node); } - addWithFillStepIfNeeded(query_plan, query_analysis_result, planner_context, query_node); + if (query_node.hasOrderBy()) + addWithFillStepIfNeeded(query_plan, query_analysis_result, planner_context, query_node); bool apply_offset = query_processing_info.getToStage() != QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 20d7d6d0f8f..37c5307fcce 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -27,9 +27,15 @@ static ITransformingStep::Traits getTraits() }; } -FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescriptionPtr interpolate_description_) +FillingStep::FillingStep( + const DataStream & input_stream_, + SortDescription sort_description_, + SortDescription fill_description_, + InterpolateDescriptionPtr interpolate_description_) : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_), getTraits()) - , sort_description(std::move(sort_description_)), interpolate_description(interpolate_description_) + , sort_description(std::move(sort_description_)) + , fill_description(std::move(fill_description_)) + , interpolate_description(interpolate_description_) { if (!input_stream_.has_single_port) throw Exception(ErrorCodes::LOGICAL_ERROR, "FillingStep expects single input"); @@ -40,9 +46,9 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type == QueryPipelineBuilder::StreamType::Totals) - return std::make_shared(header, sort_description); + return std::make_shared(header, fill_description); - return std::make_shared(header, sort_description, std::move(interpolate_description)); + return std::make_shared(header, sort_description, fill_description, std::move(interpolate_description)); }); } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index 4e1b5b3654d..0febbbb8eb0 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -10,7 +10,11 @@ namespace DB class FillingStep : public ITransformingStep { public: - FillingStep(const DataStream & input_stream_, SortDescription sort_description_, InterpolateDescriptionPtr interpolate_description_); + FillingStep( + const DataStream & input_stream_, + SortDescription sort_description_, + SortDescription fill_description_, + InterpolateDescriptionPtr interpolate_description_); String getName() const override { return "Filling"; } @@ -25,6 +29,7 @@ private: void updateOutputStream() override; SortDescription sort_description; + SortDescription fill_description; InterpolateDescriptionPtr interpolate_description; }; diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 4c5b74aad7c..4fc2b953a4f 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -187,9 +187,10 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & } FillingTransform::FillingTransform( - const Block & header_, const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_) + const Block & header_, const SortDescription & sort_description_, const SortDescription& fill_description_, InterpolateDescriptionPtr interpolate_description_) : ISimpleTransform(header_, transformHeader(header_, sort_description_), true) , sort_description(sort_description_) + , fill_description(fill_description_) , interpolate_description(interpolate_description_) , filling_row(sort_description_) , next_row(sort_description_) @@ -198,14 +199,14 @@ FillingTransform::FillingTransform( interpolate_actions = std::make_shared(interpolate_description->actions); std::vector is_fill_column(header_.columns()); - for (size_t i = 0, size = sort_description.size(); i < size; ++i) + for (size_t i = 0, size = fill_description.size(); i < size; ++i) { - if (interpolate_description && interpolate_description->result_columns_set.contains(sort_description[i].column_name)) + if (interpolate_description && interpolate_description->result_columns_set.contains(fill_description[i].column_name)) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Column '{}' is participating in ORDER BY ... WITH FILL expression and can't be INTERPOLATE output", - sort_description[i].column_name); + fill_description[i].column_name); - size_t block_position = header_.getPositionByName(sort_description[i].column_name); + size_t block_position = header_.getPositionByName(fill_description[i].column_name); is_fill_column[block_position] = true; fill_column_positions.push_back(block_position); @@ -232,6 +233,21 @@ FillingTransform::FillingTransform( if (!unique_positions.insert(pos).second) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Multiple WITH FILL for identical expressions is not supported in ORDER BY"); + /// build sorting prefix for first fill column + for (const auto & desc : sort_description) + { + if (desc.column_name == fill_description[0].column_name) + break; + + size_t pos = header_.getPositionByName(desc.column_name); + sort_prefix_positions.push_back(pos); + + sort_prefix.push_back(desc); + } + logDebug("sort prefix", dumpSortDescription(sort_prefix)); + + /// TODO: check conflict in positions between interpolate and sorting prefix columns + size_t idx = 0; for (const ColumnWithTypeAndName & column : header_.getColumnsWithTypeAndName()) { diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 7aa5e4c1e8a..021784c263d 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -16,7 +16,11 @@ namespace DB class FillingTransform : public ISimpleTransform { public: - FillingTransform(const Block & header_, const SortDescription & sort_description_, InterpolateDescriptionPtr interpolate_description_); + FillingTransform( + const Block & header_, + const SortDescription & sort_description_, + const SortDescription & fill_description_, + InterpolateDescriptionPtr interpolate_description_); String getName() const override { return "FillingTransform"; } @@ -25,7 +29,7 @@ public: static Block transformHeader(Block header, const SortDescription & sort_description); protected: - void transform(Chunk & Chunk) override; + void transform(Chunk & chunk) override; private: void saveLastRow(const MutableColumns & result_columns); @@ -46,7 +50,9 @@ private: const Columns & input_columns, MutableColumns & result_columns); - const SortDescription sort_description; /// Contains only columns with WITH FILL. + const SortDescription sort_description; + const SortDescription fill_description; /// Contains only columns with WITH FILL. + SortDescription sort_prefix; const InterpolateDescriptionPtr interpolate_description; /// Contains INTERPOLATE columns FillingRow filling_row; /// Current row, which is used to fill gaps. @@ -56,6 +62,7 @@ private: Positions fill_column_positions; Positions interpolate_column_positions; Positions other_column_positions; + Positions sort_prefix_positions; std::vector> input_positions; /// positions in result columns required for actions ExpressionActionsPtr interpolate_actions; Columns last_row; From 3aaea2b91d34a98651e62f9cedd263d9333530d5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 4 May 2023 13:07:28 +0000 Subject: [PATCH 0175/2223] join thread even exception happened --- src/Interpreters/TraceCollector.cpp | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 49588d490f5..bedc27f7d15 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -56,9 +56,17 @@ TraceCollector::~TraceCollector() */ void TraceCollector::stop() { - WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); - writeChar(true, out); - out.next(); + try + { + WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); + writeChar(true, out); + out.next(); + } + catch (...) + { + tryLogCurrentException("TraceCollector"); + } + thread.join(); } From b9afc70583d7fc600131d8583a16f40a8d3d8b48 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 4 May 2023 09:36:27 -0400 Subject: [PATCH 0176/2223] WIP --- .../en/sql-reference/functions/geo/polygon.md | 344 ++++++++++++++++++ 1 file changed, 344 insertions(+) create mode 100644 docs/en/sql-reference/functions/geo/polygon.md diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md new file mode 100644 index 00000000000..a394e40af6a --- /dev/null +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -0,0 +1,344 @@ +--- +slug: /en/sql-reference/functions/geo/polygons +sidebar_label: Polygons +title: "Functions for Working with Polygons" +--- + +## readWKTMultiPolygon + +Calculates + +### Example + +``` sql +SELECT + toTypeName(readWKTMultiPolygon('MULTIPOLYGON(((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))')) AS type, + readWKTMultiPolygon('MULTIPOLYGON(((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))') AS output FORMAT Markdown + +``` +| type | output | +|:-|:-| +| MultiPolygon | [[[(2,0),(10,0),(10,10),(0,10),(2,0)],[(4,4),(5,4),(5,5),(4,5),(4,4)]],[[(-10,-10),(-10,-9),(-9,10),(-10,-10)]]] | + + +### Input parameters + +String starting with `MULTIPOLYGON` + +### Returned value + +MultiPolygon + +## readWKTPolygon + +Calculates + +### Example + +``` sql +SELECT + toTypeName(readWKTPolygon('POLYGON((2 0,10 0,10 10,0 10,2 0))')) AS type, + readWKTPolygon('POLYGON((2 0,10 0,10 10,0 10,2 0))') AS output +FORMAT Markdown +``` +| type | output | +|:-|:-| +| Polygon | [[(2,0),(10,0),(10,10),(0,10),(2,0)]] | + +### Input parameters + +String starting with `POLYGON` + +### Returned value + +Polygon + +## polygonsWithinSpherical + +Calculates + +### Example + +``` sql +SELECT +``` +```response +``` + +### Input parameters + +### Returned value + +UInt8, 0 for false, 1 for true + +## polygonsDistanceSpherical + +Calculates + +### Example + +``` sql +SELECT polygonsDistanceSpherical([[[(0, 0), (0, 0.1), (0.1, 0.1), (0.1, 0)]]], [[[(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)]]]) +``` +```response +0.24372872211133834 +``` + +### Input parameters + +Two polygons + +### Returned value + +Float64 + +## polygonsDistanceCartesian + +Calculates distance between two polygons + +### Example + +``` sql +SELECT polygonsDistanceCartesian([[[(0, 0), (0, 0.1), (0.1, 0.1), (0.1, 0)]]], [[[(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)]]]) +``` +```response +14.000714267493642 +``` + +### Input parameters + +Two polygons + +### Returned value + +Float64 + +## polygonsEqualsCartesian + +Returns true if two polygons are equal + +### Example + +``` sql +SELECT polygonsEqualsCartesian([[[(1., 1.), (1., 4.), (4., 4.), (4., 1.)]]], [[[(1., 1.), (1., 4.), (4., 4.), (4., 1.), (1., 1.)]]]) +``` +```response +1 +``` + +### Input parameters + +Two polygons + +### Returned value + +UInt8, 0 for false, 1 for true + +## polygonsSymDifferenceSpherical + +Calculates + +### Example + +``` sql +SELECT +``` +```response +``` + +### Input parameters + +### Returned value + +## polygonsSymDifferenceCartesian + +Calculates + +### Example + +``` sql +SELECT +``` +```response +``` + +### Input parameters + +### Returned value + +## polygonsIntersectionSpherical + +Calculates + +### Example + +``` sql +SELECT +``` +```response +``` + +### Input parameters + +### Returned value + +## polygonsWithinCartesian + +Returns true if the second polygon is within the first polygon. + +### Example + +``` sql +SELECT polygonsWithinCartesian([[[(2., 2.), (2., 3.), (3., 3.), (3., 2.)]]], [[[(1., 1.), (1., 4.), (4., 4.), (4., 1.), (1., 1.)]]]) +``` +```response +1 +``` + +### Input parameters + +Two polygons + +### Returned value + +UInt8, 0 for false, 1 for true + +## polygonConvexHullCartesian + +Calculates + +### Example + +``` sql +SELECT +``` +```response +``` + +### Input parameters + +### Returned value + +## polygonAreaSpherical + +Calculates + +### Example + +``` sql +SELECT +``` +```response +``` + +### Input parameters + +### Returned value + +## polygonsUnionSpherical + +Calculates + +### Example + +``` sql +SELECT +``` +```response +``` + +### Input parameters + +### Returned value + +## polygonPerimeterSpherical + +Calculates + +### Example + +``` sql +SELECT +``` +```response +``` + +### Input parameters + +### Returned value + +## polygonsIntersectionCartesian + +Calculates + +### Example + +``` sql +SELECT +``` +```response +``` + +### Input parameters + + +### Returned value + +## polygonAreaCartesian + +Calculates the area of a polygon + +### Example + +``` sql +SELECT polygonAreaCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.)]]]) +``` +```response +25 +``` + +### Input parameters + +One polygon + +### Returned value + +Float64 + +## polygonPerimeterCartesian + +Calculates + +### Example + +``` sql +SELECT polygonPerimeterCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.)]]]) +``` +```response +15 +``` + +### Input parameters + +One polygon + +### Returned value + +Float64 + +## polygonsUnionCartesian + +Calculates + +### Example + +``` sql +SELECT +``` +```response +``` + +### Input parameters + +### Returned value + From 6670cc2154ee0780e5d810a9fc11685339b48e30 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 4 May 2023 09:43:01 -0400 Subject: [PATCH 0177/2223] add description --- docs/en/sql-reference/functions/geo/polygon.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index a394e40af6a..7c5a2ff41fa 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -6,7 +6,7 @@ title: "Functions for Working with Polygons" ## readWKTMultiPolygon -Calculates +Converts a WKT (Well Known Text) MultiPolygon into a MultiPolygon type. ### Example @@ -31,7 +31,7 @@ MultiPolygon ## readWKTPolygon -Calculates +Converts a WKT (Well Known Text) MultiPolygon into a Polygon type. ### Example From 963d6be120da9c00c583cec2a051f9386de47b0e Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Thu, 4 May 2023 16:44:08 +0000 Subject: [PATCH 0178/2223] Added configurations for DatabaseS3 --- src/Databases/DatabaseFactory.cpp | 13 +- src/Databases/DatabaseS3.cpp | 159 +++++++++++++++--- src/Databases/DatabaseS3.h | 20 ++- tests/config/config.d/named_collection.xml | 5 + .../0_stateless/02724_database_s3.reference | 12 ++ .../queries/0_stateless/02724_database_s3.sh | 28 ++- 6 files changed, 202 insertions(+), 35 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 5c4256c8a9f..41ca1de6a0e 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -466,20 +466,15 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String { const ASTFunction * engine = engine_define->engine; - std::string key_id; - std::string secret_key; + DatabaseS3::Configuration config; if (engine->arguments && !engine->arguments->children.empty()) { - if (engine->arguments->children.size() != 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 database requires 0 or 2 argument: [access_key_id, secret_access_key]"); - - const auto & arguments = engine->arguments->children; - key_id = safeGetLiteralValue(arguments[0], engine_name); - secret_key = safeGetLiteralValue(arguments[1], engine_name); + ASTs & engine_args = engine->arguments->children; + config = DatabaseS3::parseArguments(engine_args, context); } - return std::make_shared(database_name, key_id, secret_key, context); + return std::make_shared(database_name, config, context); } #endif diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index d4412ba7973..f4aafc5d03a 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -4,19 +4,36 @@ #include -#include #include +#include +#include #include #include #include -#include #include +#include +#include #include +#include #include +#include +#include + +#include "DatabaseS3.h" + +namespace fs = std::filesystem; + namespace DB { +static const std::unordered_set optional_configuration_keys = { + "url", + "access_key_id", + "secret_access_key", + "no_sign_request" +}; + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -25,13 +42,14 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; extern const int UNACCEPTABLE_URL; extern const int S3_ERROR; + + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -DatabaseS3::DatabaseS3(const String & name_, const String & key_id, const String & secret_key, ContextPtr context_) +DatabaseS3::DatabaseS3(const String & name_, const Configuration& config_, ContextPtr context_) : IDatabase(name_) , WithContext(context_->getGlobalContext()) - , access_key_id(key_id) - , secret_access_key(secret_key) + , config(config_) , log(&Poco::Logger::get("DatabaseS3(" + name_ + ")")) { } @@ -49,6 +67,20 @@ void DatabaseS3::addTable(const std::string & table_name, StoragePtr table_stora getEngineName()); } +std::string DatabaseS3::getFullUrl(const std::string & name) const +{ + try + { + S3::URI uri(name); + } + catch (...) + { + return (fs::path(config.url_prefix) / name).string(); + } + + return name; +} + bool DatabaseS3::checkUrl(const std::string & url, ContextPtr context_, bool throw_on_error) const { try @@ -71,36 +103,49 @@ bool DatabaseS3::isTableExist(const String & name, ContextPtr context_) const if (loaded_tables.find(name) != loaded_tables.end()) return true; - return checkUrl(name, context_, false); + return checkUrl(getFullUrl(name), context_, false); } -StoragePtr DatabaseS3::getTableImpl(const String & url, ContextPtr context_) const +StoragePtr DatabaseS3::getTableImpl(const String & name, ContextPtr context_) const { // Check if the table exists in the loaded tables map { std::lock_guard lock(mutex); - auto it = loaded_tables.find(url); + auto it = loaded_tables.find(name); if (it != loaded_tables.end()) return it->second; } + auto url = getFullUrl(name); + checkUrl(url, context_, true); // call TableFunctionS3 - auto args = makeASTFunction( - "s3", - std::make_shared(url), - std::make_shared(access_key_id), - std::make_shared(secret_access_key)); + auto function = std::make_shared(); - auto table_function = TableFunctionFactory::instance().get(args, context_); + function->name = "s3"; + function->arguments = std::make_shared(); + function->children.push_back(function->arguments); + + function->arguments->children.push_back(std::make_shared(url)); + if (config.no_sign_request) + { + function->arguments->children.push_back(std::make_shared("NOSIGN")); + } + else if (config.access_key_id.has_value() && config.secret_access_key.has_value()) + { + function->arguments->children.push_back(std::make_shared(config.access_key_id.value())); + function->arguments->children.push_back(std::make_shared(config.secret_access_key.value())); + } + + auto table_function = TableFunctionFactory::instance().get(function, context_); if (!table_function) return nullptr; // TableFunctionS3 throws exceptions, if table cannot be created - auto table_storage = table_function->execute(args, context_, url); + auto table_storage = table_function->execute(function, context_, name); if (table_storage) - addTable(url, table_storage); + addTable(name, table_storage); return table_storage; } @@ -143,10 +188,14 @@ ASTPtr DatabaseS3::getCreateDatabaseQuery() const auto settings = getContext()->getSettingsRef(); ParserCreateQuery parser; - const String query = fmt::format("CREATE DATABASE {} ENGINE = S3('{}', '{}')", - backQuoteIfNeed(getDatabaseName()), - access_key_id, - secret_access_key); + std::string creation_args; + creation_args += fmt::format("'{}'", config.url_prefix); + if (config.no_sign_request) + creation_args += ", 'NOSIGN'"; + else if (config.access_key_id.has_value() && config.secret_access_key.has_value()) + creation_args += fmt::format(", '{}', '{}'", config.access_key_id.value(), config.secret_access_key.value()); + + const String query = fmt::format("CREATE DATABASE {} ENGINE = S3({})", backQuoteIfNeed(getDatabaseName()), creation_args); ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth); if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) @@ -176,6 +225,76 @@ void DatabaseS3::shutdown() loaded_tables.clear(); } +DatabaseS3::Configuration DatabaseS3::parseArguments(ASTs engine_args, ContextPtr context_) +{ + Configuration result; + + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context_)) + { + auto & collection = *named_collection; + + validateNamedCollection(collection, {}, optional_configuration_keys); + + result.url_prefix = collection.getOrDefault("url", ""); + result.no_sign_request = collection.getOrDefault("no_sign_request", false); + + auto key_id = collection.getOrDefault("access_key_id", ""); + auto secret_key = collection.getOrDefault("secret_access_key", ""); + + if (!key_id.empty()) + result.access_key_id = key_id; + + if (!secret_key.empty()) + result.secret_access_key = secret_key; + } + else + { + auto supported_signature = + " - S3()\n" + " - S3('url')\n" + " - S3('url', 'NOSIGN')\n" + " - S3('url', 'access_key_id', 'secret_access_key')\n"; + const auto error_message = + fmt::format("Engine DatabaseS3 must have the following arguments signature\n{}", supported_signature); + + for (auto & arg : engine_args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context_); + + if (engine_args.size() > 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message.c_str()); + + if (engine_args.empty()) + return result; + + result.url_prefix = checkAndGetLiteralArgument(engine_args[0], "url"); + + // url, NOSIGN + if (engine_args.size() == 2) + { + auto second_arg = checkAndGetLiteralArgument(engine_args[1], "NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + result.no_sign_request = true; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, error_message.c_str()); + } + + // url, access_key_id, secret_access_key + if (engine_args.size() == 3) + { + auto key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + auto secret_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + if (key_id.empty() || secret_key.empty() || boost::iequals(key_id, "NOSIGN")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, error_message.c_str()); + + result.access_key_id = key_id; + result.secret_access_key = secret_key; + } + } + + return result; +} + /** * Returns an empty vector because the database is read-only and no tables can be backed up */ diff --git a/src/Databases/DatabaseS3.h b/src/Databases/DatabaseS3.h index d5269e57f5a..65f80dca2ba 100644 --- a/src/Databases/DatabaseS3.h +++ b/src/Databases/DatabaseS3.h @@ -23,7 +23,17 @@ class Context; class DatabaseS3 : public IDatabase, protected WithContext { public: - DatabaseS3(const String & name, const String & key_id, const String & secret_key, ContextPtr context); + struct Configuration + { + std::string url_prefix; + + bool no_sign_request = false; + + std::optional access_key_id; + std::optional secret_access_key; + }; + + DatabaseS3(const String & name, const Configuration& config, ContextPtr context); String getEngineName() const override { return "S3"; } @@ -44,6 +54,8 @@ public: std::vector> getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; + static Configuration parseArguments(ASTs engine_args, ContextPtr context); + protected: StoragePtr getTableImpl(const String & url, ContextPtr context) const; @@ -51,9 +63,11 @@ protected: bool checkUrl(const std::string & url, ContextPtr context_, bool throw_on_error) const; + std::string getFullUrl(const std::string & name) const; + private: - const String access_key_id; - const String secret_access_key; + const Configuration config; + mutable Tables loaded_tables TSA_GUARDED_BY(mutex); Poco::Logger * log; }; diff --git a/tests/config/config.d/named_collection.xml b/tests/config/config.d/named_collection.xml index 2e49c0c596f..5b716a7b8da 100644 --- a/tests/config/config.d/named_collection.xml +++ b/tests/config/config.d/named_collection.xml @@ -32,5 +32,10 @@ testtest auto + + http://localhost:11111/test/ + test + testtest + diff --git a/tests/queries/0_stateless/02724_database_s3.reference b/tests/queries/0_stateless/02724_database_s3.reference index 72ba0e240b1..811e38b7f2b 100644 --- a/tests/queries/0_stateless/02724_database_s3.reference +++ b/tests/queries/0_stateless/02724_database_s3.reference @@ -8,6 +8,18 @@ test1 13 14 15 16 17 18 0 0 0 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +0 0 0 +10 11 12 +13 14 15 +16 17 18 +0 0 0 Test 2: check exceptions OK OK diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index 2758580a355..ac1b97beecf 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -15,15 +15,35 @@ USE test1; SELECT * FROM \"http://localhost:11111/test/a.tsv\" """ ${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test1 -${CLICKHOUSE_CLIENT} -q "DROP DATABASE test1;" +# check credentials with absolute path ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test2; -CREATE DATABASE test2 ENGINE = S3('test', 'testtest'); +CREATE DATABASE test2 ENGINE = S3('', 'test', 'testtest'); USE test2; SELECT * FROM \"http://localhost:11111/test/b.tsv\" """ -${CLICKHOUSE_CLIENT} -q "DROP DATABASE test2;" + +# check credentials with relative path +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test4; +CREATE DATABASE test4 ENGINE = S3('http://localhost:11111/test', 'test', 'testtest'); +USE test4; +SELECT * FROM \"b.tsv\" +""" + +# check that database url_prefix is ignored if pass full url as table name +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +USE test4; +SELECT * FROM \"http://localhost:11111/test/a.tsv\" +""" + +# Check named collection loading +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ +DROP DATABASE IF EXISTS test5; +CREATE DATABASE test5 ENGINE = S3(s3_conn_db); +SELECT * FROM test5.\`b.tsv\` +""" ################# echo "Test 2: check exceptions" @@ -44,4 +64,6 @@ ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test1; DROP DATABASE IF EXISTS test2; DROP DATABASE IF EXISTS test3; +DROP DATABASE IF EXISTS test4; +DROP DATABASE IF EXISTS test5; """ From f083372c0cd5618b07873e886078ac86aecf54cd Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Thu, 4 May 2023 16:46:51 +0000 Subject: [PATCH 0179/2223] remove extra include --- src/Databases/DatabaseS3.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index f4aafc5d03a..5529f582572 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -20,8 +20,6 @@ #include #include -#include "DatabaseS3.h" - namespace fs = std::filesystem; namespace DB From 9815b6584975c4feb678e5d55dd748c4291be55b Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 4 May 2023 12:47:39 -0400 Subject: [PATCH 0180/2223] add description --- docs/en/sql-reference/functions/geo/polygon.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index 7c5a2ff41fa..771fa29667c 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -55,14 +55,15 @@ Polygon ## polygonsWithinSpherical -Calculates +Returns true or false ### Example ``` sql -SELECT +select polygonsWithinSpherical([[[(4.3613577, 50.8651821), (4.349556, 50.8535879), (4.3602419, 50.8435626), (4.3830299, 50.8428851), (4.3904543, 50.8564867), (4.3613148, 50.8651279)]]], [[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]]); ``` ```response +0 ``` ### Input parameters From 814a3f04cd421c991e6976fa314f13e96e96069f Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Thu, 4 May 2023 17:12:35 +0000 Subject: [PATCH 0181/2223] fix style --- src/Databases/DatabaseS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index 5529f582572..bc318ecd9bf 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -253,7 +253,7 @@ DatabaseS3::Configuration DatabaseS3::parseArguments(ASTs engine_args, ContextPt " - S3('url', 'NOSIGN')\n" " - S3('url', 'access_key_id', 'secret_access_key')\n"; const auto error_message = - fmt::format("Engine DatabaseS3 must have the following arguments signature\n{}", supported_signature); + fmt::format("Engine DatabaseS3 must have the following arguments signature\n{}", supported_signature); for (auto & arg : engine_args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context_); From 367583b96ea53a5203163a90e49581b2bdf225de Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Thu, 4 May 2023 17:38:41 +0000 Subject: [PATCH 0182/2223] retriger checks From f598a39ea261161cefa363f21112ddb49c7300a5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 4 May 2023 17:54:08 +0000 Subject: [PATCH 0183/2223] Refactor PreparedSets [3] --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 18 ++- src/Analyzer/SetUtils.cpp | 24 ++-- src/Analyzer/SetUtils.h | 3 +- src/Columns/ColumnSet.h | 2 +- src/DataTypes/DataTypeSet.h | 2 +- src/Functions/in.cpp | 21 +++- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/ExpressionActions.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 14 ++- src/Interpreters/InterpreterSelectQuery.cpp | 11 +- src/Interpreters/PreparedSets.cpp | 22 ++-- src/Interpreters/PreparedSets.h | 42 ++++--- src/Planner/CollectSets.cpp | 61 ++++++++-- src/Planner/CollectSets.h | 4 +- src/Planner/Planner.cpp | 90 ++++++++------- src/Planner/PlannerActionsVisitor.cpp | 35 ++++-- src/Planner/PlannerContext.cpp | 72 ++++++------ src/Planner/PlannerContext.h | 107 +++++++++--------- src/Planner/PlannerJoinTree.cpp | 21 ++-- src/Planner/Utils.cpp | 3 +- src/Planner/Utils.h | 1 + src/Processors/QueryPlan/CreatingSetsStep.cpp | 72 +++++++++--- src/Processors/QueryPlan/CreatingSetsStep.h | 19 +++- .../QueryPlan/Optimizations/Optimizations.h | 1 + .../Optimizations/addPlansForSets.cpp | 35 ++++++ .../QueryPlan/Optimizations/optimizeTree.cpp | 1 + src/Processors/QueryPlan/QueryPlan.cpp | 5 + src/Processors/QueryPlan/QueryPlan.h | 5 +- .../Transforms/CreatingSetsTransform.cpp | 24 ++-- src/Storages/KVStorageUtils.cpp | 13 ++- src/Storages/MergeTree/KeyCondition.cpp | 20 +++- .../MergeTreeIndexConditionBloomFilter.cpp | 8 +- .../MergeTree/MergeTreeIndexFullText.cpp | 9 +- .../MergeTree/MergeTreeIndexInverted.cpp | 9 +- src/Storages/MergeTree/RPNBuilder.cpp | 42 ++++--- src/Storages/MergeTree/RPNBuilder.h | 6 +- .../System/StorageSystemZooKeeper.cpp | 11 +- src/Storages/VirtualColumnUtils.cpp | 38 +++---- 38 files changed, 585 insertions(+), 292 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 8afb9078fae..78e8b1760cb 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5117,14 +5117,26 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi const auto & second_argument_constant_literal = second_argument_constant_node->getValue(); const auto & second_argument_constant_type = second_argument_constant_node->getResultType(); - auto set = makeSetForConstantValue(first_argument_constant_type, + const auto & settings = scope.context->getSettingsRef(); + + auto result_block = makeSetForConstantValue(first_argument_constant_type, second_argument_constant_literal, second_argument_constant_type, - scope.context->getSettingsRef()); + settings.transform_null_in); + + SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + + auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, settings.transform_null_in); + + set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); + set->insertFromBlock(result_block.getColumnsWithTypeAndName()); + set->finishInsert(); + + auto future_set = std::make_shared(std::move(set)); /// Create constant set column for constant folding - auto column_set = ColumnSet::create(1, FutureSet(std::move(set))); + auto column_set = ColumnSet::create(1, std::move(future_set)); argument_columns[1].column = ColumnConst::create(std::move(column_set), 1); } diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 0fb075e925b..2fb05449714 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -118,7 +118,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & } -SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings) +Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in) { DataTypes set_element_types = {expression_type}; const auto * lhs_tuple_type = typeid_cast(expression_type.get()); @@ -135,8 +135,8 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field size_t lhs_type_depth = getCompoundTypeDepth(*expression_type); size_t rhs_type_depth = getCompoundTypeDepth(*value_type); - SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - bool tranform_null_in = settings.transform_null_in; + // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + // bool transform_null_in = settings.transform_null_in; Block result_block; @@ -145,7 +145,7 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. Array array{value}; - result_block = createBlockFromCollection(array, set_element_types, tranform_null_in); + result_block = createBlockFromCollection(array, set_element_types, transform_null_in); } else if (lhs_type_depth + 1 == rhs_type_depth) { @@ -154,9 +154,9 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field WhichDataType rhs_which_type(value_type); if (rhs_which_type.isArray()) - result_block = createBlockFromCollection(value.get(), set_element_types, tranform_null_in); + result_block = createBlockFromCollection(value.get(), set_element_types, transform_null_in); else if (rhs_which_type.isTuple()) - result_block = createBlockFromCollection(value.get(), set_element_types, tranform_null_in); + result_block = createBlockFromCollection(value.get(), set_element_types, transform_null_in); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Unsupported type at the right-side of IN. Expected Array or Tuple. Actual {}", @@ -170,13 +170,15 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field value_type->getName()); } - auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, tranform_null_in); + return result_block; - set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(result_block.getColumnsWithTypeAndName()); - set->finishInsert(); + // auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, tranform_null_in); - return set; + // set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); + // set->insertFromBlock(result_block.getColumnsWithTypeAndName()); + // set->finishInsert(); + + // return set; } } diff --git a/src/Analyzer/SetUtils.h b/src/Analyzer/SetUtils.h index 7afc8e5259c..fdeaa4a3c48 100644 --- a/src/Analyzer/SetUtils.h +++ b/src/Analyzer/SetUtils.h @@ -21,10 +21,9 @@ using SetPtr = std::shared_ptr; * @param expression_type - type of first argument of function IN. * @param value - constant value of second argument of function IN. * @param value_type - type of second argument of function IN. - * @param settings - query settings. * * @return SetPtr for constant value. */ -SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings); +Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in); } diff --git a/src/Columns/ColumnSet.h b/src/Columns/ColumnSet.h index ccd9aa19896..935a72e551a 100644 --- a/src/Columns/ColumnSet.h +++ b/src/Columns/ColumnSet.h @@ -29,7 +29,7 @@ public: TypeIndex getDataType() const override { return TypeIndex::Set; } MutableColumnPtr cloneDummy(size_t s_) const override { return ColumnSet::create(s_, data); } - ConstSetPtr getData() const { if (!data.isReady()) return nullptr; return data.get(); } + FutureSetPtr getData() const { return data; } // Used only for debugging, making it DUMPABLE Field operator[](size_t) const override { return {}; } diff --git a/src/DataTypes/DataTypeSet.h b/src/DataTypes/DataTypeSet.h index 7ddfeb9fe30..7ef0d931279 100644 --- a/src/DataTypes/DataTypeSet.h +++ b/src/DataTypes/DataTypeSet.h @@ -20,7 +20,7 @@ public: bool isParametric() const override { return true; } // Used for expressions analysis. - MutableColumnPtr createColumn() const override { return ColumnSet::create(0, FutureSet{}); } + MutableColumnPtr createColumn() const override { return ColumnSet::create(0, nullptr); } // Used only for debugging, making it DUMPABLE Field getDefault() const override { return Tuple(); } diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index 7a41ae2e3ea..eb623951bf3 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -55,9 +55,13 @@ public: /// It is needed to perform type analysis without creation of set. static constexpr auto name = FunctionInName::name; - static FunctionPtr create(ContextPtr) + FunctionIn(SizeLimits size_limits_, bool transform_null_in_) + : size_limits(std::move(size_limits_)), transform_null_in(transform_null_in_) {} + + static FunctionPtr create(ContextPtr context) { - return std::make_shared(); + const auto & settings = context->getSettingsRef(); + return std::make_shared(FutureSet::getSizeLimitsForSet(settings, false), settings.transform_null_in); } String getName() const override @@ -122,10 +126,15 @@ public: tuple = typeid_cast(materialized_tuple.get()); } - auto set = column_set->getData(); - if (!set) + auto future_set = column_set->getData(); + if (!future_set || !future_set->isFilled()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName()); + if (auto * for_tuple = typeid_cast(future_set.get())) + if (!for_tuple->isReady()) + for_tuple->buildForTuple(size_limits, transform_null_in); + + auto set = future_set->get(); auto set_types = set->getDataTypes(); if (tuple && set_types.size() != 1 && set_types.size() == tuple->tupleSize()) @@ -173,6 +182,10 @@ public: return res; } + +private: + SizeLimits size_limits; + bool transform_null_in; }; template diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 00feecb44f5..18e44ab9f2f 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -446,7 +446,7 @@ FutureSetPtr makeExplicitSet( if (const auto * low_cardinality_type = typeid_cast(element_type.get())) element_type = low_cardinality_type->getDictionaryType(); - auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types); + auto set_key = PreparedSetKey::forLiteral(right_arg->getTreeHash(), set_element_types); if (auto set = prepared_sets.getFuture(set_key)) return set; /// Already prepared. @@ -1384,7 +1384,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool { if (no_subqueries) return {}; - auto set_key = PreparedSetKey::forSubquery(*right_in_operand); + auto set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); if (auto set = data.prepared_sets->getFuture(set_key)) return set; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 36725f36804..989fa12cba0 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -941,7 +941,7 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con if (const auto * column_set = checkAndGetColumn(action.node->column.get())) { auto set = column_set->getData(); - if (set && set->isCreated() && set->getTotalRowCount() == 0) + if (set && set->isReady() && set->get()->getTotalRowCount() == 0) return true; } } diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 9fb2c02bd58..3a846bb4bc3 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -181,7 +181,19 @@ public: // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); - prepared_sets->addStorageToSubquery(key, std::move(external_storage)); + auto set_key = PreparedSetKey::forSubquery(subquery_or_table_name->getTreeHash()); + + if (!prepared_sets->getFuture(set_key)) + { + SubqueryForSet subquery_for_set; + subquery_for_set.key = std::move(key); + subquery_for_set.table = std::move(external_storage); + subquery_for_set.createSource(*interpreter); + + prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); + } + else + prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index afd34cb044e..2487381e2fb 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -905,8 +905,8 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (storage && !options.only_analyze) { - query_analyzer->makeSetsForIndex(select_query.where()); - query_analyzer->makeSetsForIndex(select_query.prewhere()); + // query_analyzer->makeSetsForIndex(select_query.where()); + // query_analyzer->makeSetsForIndex(select_query.prewhere()); query_info.prepared_sets = query_analyzer->getPreparedSets(); from_stage = storage->getQueryProcessingStage(context, options.to_stage, storage_snapshot, query_info); @@ -3088,7 +3088,12 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - addCreatingSetsStep(query_plan, prepared_sets, context); + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + prepared_sets->detachSubqueries(context), + context); + + query_plan.addStep(std::move(step)); } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 300ef2aadb6..733eb1c24bb 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -11,7 +11,7 @@ namespace DB { -PreparedSetKey PreparedSetKey::forLiteral(const IAST & ast, DataTypes types_) +PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) { /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, /// just converts LowCardinality to ordinary types. @@ -19,15 +19,15 @@ PreparedSetKey PreparedSetKey::forLiteral(const IAST & ast, DataTypes types_) type = recursiveRemoveLowCardinality(type); PreparedSetKey key; - key.ast_hash = ast.getTreeHash(); + key.ast_hash = hash; key.types = std::move(types_); return key; } -PreparedSetKey PreparedSetKey::forSubquery(const IAST & ast) +PreparedSetKey PreparedSetKey::forSubquery(Hash hash) { PreparedSetKey key; - key.ast_hash = ast.getTreeHash(); + key.ast_hash = hash; return key; } @@ -155,9 +155,9 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return it->second.get(); // } -// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const +// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const // { -// std::vector res; +// std::vector res; // for (const auto & it : this->sets) // { // if (it.first.ast_hash == ast_hash) @@ -166,7 +166,7 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return res; // } -PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() +PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries(const ContextPtr &) { auto res = std::move(subqueries); subqueries = SubqueriesForSets(); @@ -221,6 +221,8 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; + std::cerr << StackTrace().toString() << std::endl; + auto set_cache = context->getPreparedSetsCache(); if (set_cache) { @@ -277,4 +279,10 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } +FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} + +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} + +FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 8ebabc32b0a..8661d81a96a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -65,6 +65,7 @@ public: virtual ~FutureSet() = default; virtual bool isReady() const = 0; + virtual bool isFilled() const = 0; virtual SetPtr get() const = 0; virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; @@ -81,34 +82,41 @@ public: FutureSetFromTuple(Block block_); bool isReady() const override { return set != nullptr; } + bool isFilled() const override { return true; } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override { - fill(context, true); + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, true); + fill(size_limits, settings.transform_null_in, true); return set; } std::unique_ptr build(const ContextPtr & context) override { - fill(context, false); + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, false); + fill(size_limits, settings.transform_null_in, false); return nullptr; } + void buildForTuple(SizeLimits size_limits, bool transform_null_in) + { + fill(size_limits, transform_null_in, false); + } + private: Block block; SetPtr set; - void fill(const ContextPtr & context, bool create_ordered_set) + void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) { if (set) return; - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); - - set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + set = std::make_shared(size_limits, create_ordered_set, transform_null_in); set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); set->insertFromBlock(block.getColumnsWithTypeAndName()); set->finishInsert(); @@ -151,6 +159,7 @@ public: FutureSetFromSubquery(SubqueryForSet subquery_); bool isReady() const override { return set != nullptr; } + bool isFilled() const override { return isReady(); } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override @@ -190,6 +199,7 @@ public: FutureSetFromStorage(SetPtr set_); // : set(std::move(set_) {} bool isReady() const override { return set != nullptr; } + bool isFilled() const override { return isReady(); } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr &) override @@ -229,23 +239,25 @@ private: struct PreparedSetKey { + using Hash = std::pair; + /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired /// data types of set elements (two different Sets can be required for two tuples with the same contents /// if left hand sides of the IN operators have different types). - static PreparedSetKey forLiteral(const IAST & ast, DataTypes types_); + static PreparedSetKey forLiteral(Hash hash, DataTypes types_); /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting /// set is fully determined by the subquery. - static PreparedSetKey forSubquery(const IAST & ast); + static PreparedSetKey forSubquery(Hash hash); - IAST::Hash ast_hash; + Hash ast_hash; DataTypes types; /// Empty for subqueries. bool operator==(const PreparedSetKey & other) const; String toString() const; - struct Hash + struct Hashing { UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } }; @@ -272,16 +284,18 @@ public: /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. /// SetPtr would still be available for consumers of PreparedSets. - SubqueriesForSets detachSubqueries(); + SubqueriesForSets detachSubqueries(const ContextPtr &); /// Returns all sets that match the given ast hash not checking types /// Used in KeyCondition and MergeTreeIndexConditionBloomFilter to make non exact match for types in PreparedSetKey - std::vector getByTreeHash(IAST::Hash ast_hash) const; + //std::vector getByTreeHash(IAST::Hash ast_hash) const; + + const std::unordered_map & getSets() const { return sets; } bool empty() const; private: - std::unordered_map sets; + std::unordered_map sets; /// This is the information required for building sets SubqueriesForSets subqueries; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 02069aad292..6c970e0e91b 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -11,6 +11,8 @@ #include #include #include +#include +#include namespace DB { @@ -26,8 +28,9 @@ namespace class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor { public: - explicit CollectSetsVisitor(PlannerContext & planner_context_) + explicit CollectSetsVisitor(PlannerContext & planner_context_, const SelectQueryOptions & select_query_options_) : planner_context(planner_context_) + , select_query_options(select_query_options_) {} void visitImpl(const QueryTreeNodePtr & node) @@ -42,10 +45,12 @@ public: const auto & settings = planner_context.getQueryContext()->getSettingsRef(); - String set_key = planner_context.createSetKey(in_second_argument); + // String set_key = planner_context.createSetKey(in_second_argument); - if (planner_context.hasSet(set_key)) - return; + // if (planner_context.hasSet(set_key)) + // return; + + auto & sets = planner_context.getPreparedSets(); /// Tables and table functions are replaced with subquery at Analysis stage, except special Set table. auto * second_argument_table = in_second_argument->as(); @@ -54,7 +59,9 @@ public: if (storage_set) { /// Handle storage_set as ready set. - planner_context.registerSet(set_key, PlannerSet(FutureSet(storage_set->getSet()))); + auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + sets.addFromStorage(set_key, storage_set->getSet()); + //planner_context.registerSet(set_key, PlannerSet(FutureSet(storage_set->getSet()))); } else if (const auto * constant_node = in_second_argument->as()) { @@ -62,14 +69,47 @@ public: in_first_argument->getResultType(), constant_node->getValue(), constant_node->getResultType(), - settings); + settings.transform_null_in); - planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); + DataTypes set_element_types = {in_first_argument->getResultType()}; + const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + + auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + + sets.addFromTuple(set_key, std::move(set)); + + //planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); } else if (in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION) { - planner_context.registerSet(set_key, PlannerSet(in_second_argument)); + auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + + auto subquery_options = select_query_options.subquery(); + Planner subquery_planner( + in_second_argument, + subquery_options, + planner_context.getGlobalPlannerContext()); + subquery_planner.buildQueryPlanIfNeeded(); + + // const auto & settings = planner_context.getQueryContext()->getSettingsRef(); + // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + // bool tranform_null_in = settings.transform_null_in; + // auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); + + SubqueryForSet subquery_for_set; + subquery_for_set.key = planner_context.createSetKey(in_second_argument); + subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + + sets.addFromSubquery(set_key, std::move(subquery_for_set)); + + //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } else { @@ -87,13 +127,14 @@ public: private: PlannerContext & planner_context; + const SelectQueryOptions & select_query_options; }; } -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options) { - CollectSetsVisitor visitor(planner_context); + CollectSetsVisitor visitor(planner_context, select_query_options); visitor.visit(node); } diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index 94f792e877b..57e662a392e 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -7,9 +7,11 @@ namespace DB { +struct SelectQueryOptions; + /** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations. * Collected sets are registered in planner context. */ -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d036c895fbb..38d0aa29d24 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -878,50 +878,50 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana query_plan.addStep(std::move(offsets_step)); } -void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, - const SelectQueryOptions & select_query_options, - const PlannerContextPtr & planner_context, - const std::vector & result_actions_to_execute) -{ - PreparedSets::SubqueriesForSets subqueries_for_sets; +// void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, +// const SelectQueryOptions & select_query_options, +// const PlannerContextPtr & planner_context, +// const std::vector & result_actions_to_execute) +// { +// PreparedSets::SubqueriesForSets subqueries_for_sets; - for (const auto & actions_to_execute : result_actions_to_execute) - { - for (const auto & node : actions_to_execute->getNodes()) - { - const auto & set_key = node.result_name; - auto * planner_set = planner_context->getSetOrNull(set_key); - if (!planner_set) - continue; +// for (const auto & actions_to_execute : result_actions_to_execute) +// { +// for (const auto & node : actions_to_execute->getNodes()) +// { +// const auto & set_key = node.result_name; +// auto * planner_set = planner_context->getSetOrNull(set_key); +// if (!planner_set) +// continue; - if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) - continue; +// if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) +// continue; - auto subquery_options = select_query_options.subquery(); - Planner subquery_planner( - planner_set->getSubqueryNode(), - subquery_options, - planner_context->getGlobalPlannerContext()); - subquery_planner.buildQueryPlanIfNeeded(); +// auto subquery_options = select_query_options.subquery(); +// Planner subquery_planner( +// planner_set->getSubqueryNode(), +// subquery_options, +// planner_context->getGlobalPlannerContext()); +// subquery_planner.buildQueryPlanIfNeeded(); - const auto & settings = planner_context->getQueryContext()->getSettingsRef(); - SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - bool tranform_null_in = settings.transform_null_in; - auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); +// const auto & settings = planner_context->getQueryContext()->getSettingsRef(); +// SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; +// bool tranform_null_in = settings.transform_null_in; +// auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); - SubqueryForSet subquery_for_set; - subquery_for_set.key = set_key; - subquery_for_set.set_in_progress = set; - subquery_for_set.set = planner_set->getSet(); - subquery_for_set.promise_to_fill_set = planner_set->extractPromiseToBuildSet(); - subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); +// SubqueryForSet subquery_for_set; +// subquery_for_set.key = set_key; +// subquery_for_set.set_in_progress = set; +// subquery_for_set.set = planner_set->getSet(); +// subquery_for_set.promise_to_fill_set = planner_set->extractPromiseToBuildSet(); +// subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); - subqueries_for_sets.emplace(set_key, std::move(subquery_for_set)); - } - } +// subqueries_for_sets.emplace(set_key, std::move(subquery_for_set)); +// } +// } - addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); -} +// addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); +// } /// Support for `additional_result_filter` setting void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, @@ -951,7 +951,7 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, auto storage = std::make_shared(StorageID{"dummy", "dummy"}, fake_column_descriptions); auto fake_table_expression = std::make_shared(std::move(storage), query_context); - auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, std::move(fake_name_set)); + auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, select_query_options, std::move(fake_name_set)); if (!filter_info.actions || !query_plan.isInitialized()) return; @@ -1179,7 +1179,7 @@ void Planner::buildPlanForQueryNode() } checkStoragesSupportTransactions(planner_context); - collectSets(query_tree, *planner_context); + collectSets(query_tree, *planner_context, select_query_options); collectTableExpressionData(query_tree, planner_context); const auto & settings = query_context->getSettingsRef(); @@ -1467,7 +1467,17 @@ void Planner::buildPlanForQueryNode() } if (!select_query_options.only_analyze) - addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), + planner_context->getQueryContext()); + + query_plan.addStep(std::move(step)); + + //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); + //addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + } } SelectQueryInfo Planner::buildSelectQueryInfo() const diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index c64d82299ca..453b02a2f8f 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -623,33 +624,51 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::makeSetForInFunction(const QueryTreeNodePtr & node) { const auto & function_node = node->as(); + auto in_first_argument = function_node.getArguments().getNodes().at(0); auto in_second_argument = function_node.getArguments().getNodes().at(1); - auto set_key = planner_context->createSetKey(in_second_argument); - const auto & planner_set = planner_context->getSetOrThrow(set_key); + //auto set_key = planner_context->createSetKey(in_second_argument); + + DataTypes set_element_types = {in_first_argument->getResultType()}; + const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + + auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + + + auto set = planner_context->getPreparedSets().getFuture(set_key); + if (!set) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No set is registered for key {}", + set_key.toString()); ColumnWithTypeAndName column; - column.name = set_key; + column.name = planner_context->createSetKey(in_second_argument); column.type = std::make_shared(); - bool set_is_created = planner_set.getSet().isCreated(); - auto column_set = ColumnSet::create(1, planner_set.getSet()); + bool set_is_created = set->isFilled(); + auto column_set = ColumnSet::create(1, std::move(set)); if (set_is_created) column.column = ColumnConst::create(std::move(column_set), 1); else column.column = std::move(column_set); - actions_stack[0].addConstantIfNecessary(set_key, column); + actions_stack[0].addConstantIfNecessary(column.name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(set_key, column); + actions_stack_node.addInputConstantColumnIfNecessary(column.name, column); } - return {set_key, 0}; + return {column.name, 0}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node) diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 346cc6d2080..e8c7bb7ef48 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -126,49 +126,49 @@ PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); } -void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) -{ - if (!planner_set.getSet().isValid()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Set must be initialized"); +// void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) +// { +// if (!planner_set.getSet().isValid()) +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Set must be initialized"); - const auto & subquery_node = planner_set.getSubqueryNode(); - if (subquery_node) - { - auto node_type = subquery_node->getNodeType(); +// const auto & subquery_node = planner_set.getSubqueryNode(); +// if (subquery_node) +// { +// auto node_type = subquery_node->getNodeType(); - if (node_type != QueryTreeNodeType::QUERY && - node_type != QueryTreeNodeType::UNION) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Invalid node for set table expression. Expected query or union. Actual {}", - subquery_node->formatASTForErrorMessage()); - } +// if (node_type != QueryTreeNodeType::QUERY && +// node_type != QueryTreeNodeType::UNION) +// throw Exception(ErrorCodes::LOGICAL_ERROR, +// "Invalid node for set table expression. Expected query or union. Actual {}", +// subquery_node->formatASTForErrorMessage()); +// } - set_key_to_set.emplace(key, std::move(planner_set)); -} +// set_key_to_set.emplace(key, std::move(planner_set)); +// } -bool PlannerContext::hasSet(const SetKey & key) const -{ - return set_key_to_set.contains(key); -} +// bool PlannerContext::hasSet(const SetKey & key) const +// { +// return set_key_to_set.contains(key); +// } -const PlannerSet & PlannerContext::getSetOrThrow(const SetKey & key) const -{ - auto it = set_key_to_set.find(key); - if (it == set_key_to_set.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "No set is registered for key {}", - key); +// const PlannerSet & PlannerContext::getSetOrThrow(const SetKey & key) const +// { +// auto it = set_key_to_set.find(key); +// if (it == set_key_to_set.end()) +// throw Exception(ErrorCodes::LOGICAL_ERROR, +// "No set is registered for key {}", +// key); - return it->second; -} +// return it->second; +// } -PlannerSet * PlannerContext::getSetOrNull(const SetKey & key) -{ - auto it = set_key_to_set.find(key); - if (it == set_key_to_set.end()) - return nullptr; +// PlannerSet * PlannerContext::getSetOrNull(const SetKey & key) +// { +// auto it = set_key_to_set.find(key); +// if (it == set_key_to_set.end()) +// return nullptr; - return &it->second; -} +// return &it->second; +// } } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 9ecfdb6117a..df3fad01824 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -55,43 +55,47 @@ using GlobalPlannerContextPtr = std::shared_ptr; */ class PlannerSet { -public: - /// Construct planner set that is ready for execution - explicit PlannerSet(FutureSetPtr set_) - : set(std::move(set_)) - {} - /// Construct planner set with set and subquery node - explicit PlannerSet(QueryTreeNodePtr subquery_node_) - //: set(promise_to_build_set.get_future()) - : subquery_node(std::move(subquery_node_)) - {} - - /// Get a reference to a set that might be not built yet - const FutureSetPtr & getSet() const - { - return set; - } - - /// Get subquery node - const QueryTreeNodePtr & getSubqueryNode() const - { - return subquery_node; - } - - /// This promise will be fulfilled when set is built and all FutureSet objects will become ready - // std::promise extractPromiseToBuildSet() - // { - // return std::move(promise_to_build_set); - // } - -private: - //std::promise promise_to_build_set; - FutureSetPtr set; - - QueryTreeNodePtr subquery_node; }; +// { +// public: +// /// Construct planner set that is ready for execution +// explicit PlannerSet(FutureSetPtr set_) +// : set(std::move(set_)) +// {} + +// /// Construct planner set with set and subquery node +// explicit PlannerSet(QueryTreeNodePtr subquery_node_) +// //: set(promise_to_build_set.get_future()) +// : subquery_node(std::move(subquery_node_)) +// {} + +// /// Get a reference to a set that might be not built yet +// const FutureSetPtr & getSet() const +// { +// return set; +// } + +// /// Get subquery node +// const QueryTreeNodePtr & getSubqueryNode() const +// { +// return subquery_node; +// } + +// /// This promise will be fulfilled when set is built and all FutureSet objects will become ready +// // std::promise extractPromiseToBuildSet() +// // { +// // return std::move(promise_to_build_set); +// // } + +// private: +// //std::promise promise_to_build_set; +// FutureSetPtr set; + +// QueryTreeNodePtr subquery_node; +// }; + class PlannerContext { public: @@ -179,28 +183,30 @@ public: using SetKey = std::string; - using SetKeyToSet = std::unordered_map; + // using SetKeyToSet = std::unordered_map; - /// Create set key for set source node + // /// Create set key for set source node static SetKey createSetKey(const QueryTreeNodePtr & set_source_node); - /// Register set for set key - void registerSet(const SetKey & key, PlannerSet planner_set); + // /// Register set for set key + // void registerSet(const SetKey & key, PlannerSet planner_set); - /// Returns true if set is registered for key, false otherwise - bool hasSet(const SetKey & key) const; + // /// Returns true if set is registered for key, false otherwise + // bool hasSet(const SetKey & key) const; - /// Get set for key, if no set is registered logical exception is thrown - const PlannerSet & getSetOrThrow(const SetKey & key) const; + // /// Get set for key, if no set is registered logical exception is thrown + // const PlannerSet & getSetOrThrow(const SetKey & key) const; - /// Get set for key, if no set is registered null is returned - PlannerSet * getSetOrNull(const SetKey & key); + // /// Get set for key, if no set is registered null is returned + // PlannerSet * getSetOrNull(const SetKey & key); - /// Get registered sets - const SetKeyToSet & getRegisteredSets() const - { - return set_key_to_set; - } + // /// Get registered sets + // const SetKeyToSet & getRegisteredSets() const + // { + // return set_key_to_set; + // } + + PreparedSets & getPreparedSets() { return prepared_sets; } private: /// Query context @@ -216,8 +222,7 @@ private: std::unordered_map table_expression_node_to_data; /// Set key to set - SetKeyToSet set_key_to_set; - + PreparedSets prepared_sets; }; using PlannerContextPtr = std::shared_ptr; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 93a9945b1ca..77f53a6b7ac 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -388,7 +388,8 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) + PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options) { auto storage_id = storage->getStorageID(); const auto & query_context = planner_context->getQueryContext(); @@ -397,12 +398,13 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, if (!row_policy_filter) return {}; - return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context); + return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context, select_query_options); } FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) + PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -428,14 +430,15 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, *storage, query_context); - return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context); + return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context, select_query_options); } /// Apply filters from additional_table_filters setting FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, const String & table_expression_alias, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) + PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -469,7 +472,7 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, return {}; table_expression_query_info.additional_filter_ast = additional_filter_ast; - return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context); + return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context, select_query_options); } JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, @@ -679,14 +682,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } }; - auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context); + auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, select_query_options); add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) { if (settings.parallel_replicas_count > 1) { - auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context); + auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context, select_query_options); add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); } else @@ -701,7 +704,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } const auto & table_expression_alias = table_expression->getAlias(); - auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context); + auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context, select_query_options); add_filter(additional_filters_info, "additional filter"); from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 5c61b2fc2c7..2b4febf58ea 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -426,6 +426,7 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter) { const auto & query_context = planner_context->getQueryContext(); @@ -443,7 +444,7 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, } collectSourceColumns(filter_query_tree, planner_context); - collectSets(filter_query_tree, *planner_context); + collectSets(filter_query_tree, *planner_context, select_query_options); auto filter_actions_dag = std::make_shared(); diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index d9412800e61..8071e201f88 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -82,6 +82,7 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter = {}); ASTPtr parseAdditionalResultFilter(const Settings & settings); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 9eec3e90494..d77c6627994 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -38,16 +38,16 @@ CreatingSetStep::CreatingSetStep( SizeLimits network_transfer_limits_, ContextPtr context_) : ITransformingStep(input_stream_, Block{}, getTraits()) - , WithContext(context_) , description(std::move(description_)) , subquery_for_set(std::move(subquery_for_set_)) , network_transfer_limits(std::move(network_transfer_limits_)) + , context(std::move(context_)) { } void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, getContext()); + pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context); } void CreatingSetStep::updateOutputStream() @@ -60,7 +60,7 @@ void CreatingSetStep::describeActions(FormatSettings & settings) const String prefix(settings.offset, ' '); settings.out << prefix; - if (subquery_for_set.set_in_progress) + if (subquery_for_set.set) settings.out << "Set: "; settings.out << description << '\n'; @@ -68,7 +68,7 @@ void CreatingSetStep::describeActions(FormatSettings & settings) const void CreatingSetStep::describeActions(JSONBuilder::JSONMap & map) const { - if (subquery_for_set.set_in_progress) + if (subquery_for_set.set) map.add("Set", description); } @@ -130,22 +130,14 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets plans.emplace_back(std::make_unique(std::move(query_plan))); query_plan = QueryPlan(); - for (auto & [description, subquery_for_set] : subqueries_for_sets) + for (auto & [description, future_set] : subqueries_for_sets) { - if (!subquery_for_set.hasSource()) + if (future_set->isReady()) continue; - auto plan = subquery_for_set.detachSource(); - - const Settings & settings = context->getSettingsRef(); - auto creating_set = std::make_unique( - plan->getCurrentDataStream(), - description, - std::move(subquery_for_set), - SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), - context); - creating_set->setStepDescription("Create set for subquery"); - plan->addStep(std::move(creating_set)); + auto plan = future_set->build(context); + if (!plan) + continue; input_streams.emplace_back(plan->getCurrentDataStream()); plans.emplace_back(std::move(plan)); @@ -162,12 +154,56 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets query_plan.unitePlans(std::move(creating_sets), std::move(plans)); } +//void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context) + +std::vector> DelayedCreatingSetsStep::makePlansForSets(DelayedCreatingSetsStep && step) +{ + // DataStreams input_streams; + // input_streams.emplace_back(query_plan.getCurrentDataStream()); + + std::vector> plans; + // plans.emplace_back(std::make_unique(std::move(query_plan))); + // query_plan = QueryPlan(); + + for (auto & [description, future_set] : step.subqueries_for_sets) + { + if (future_set->isReady()) + continue; + + auto plan = future_set->build(step.context); + if (!plan) + continue; + + plan->optimize(QueryPlanOptimizationSettings::fromContext(step.context)); + + //input_streams.emplace_back(plan->getCurrentDataStream()); + plans.emplace_back(std::move(plan)); + } + + return plans; +} + void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context) { if (!prepared_sets || prepared_sets->empty()) return; - addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(), context); + addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(context), context); +} + +DelayedCreatingSetsStep::DelayedCreatingSetsStep( + DataStream input_stream, PreparedSets::SubqueriesForSets subqueries_for_sets_, ContextPtr context_) + : subqueries_for_sets(std::move(subqueries_for_sets_)), context(std::move(context_)) +{ + input_streams = {input_stream}; + output_stream = std::move(input_stream); +} + +QueryPipelineBuilderPtr DelayedCreatingSetsStep::updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings &) +{ + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot build pipeline in DelayedCreatingSets. This step should be optimized out."); } } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index b4777578a30..96ab26077fc 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -9,7 +9,7 @@ namespace DB { /// Creates sets for subqueries and JOIN. See CreatingSetsTransform. -class CreatingSetStep : public ITransformingStep, WithContext +class CreatingSetStep : public ITransformingStep { public: CreatingSetStep( @@ -32,6 +32,7 @@ private: String description; SubqueryForSet subquery_for_set; SizeLimits network_transfer_limits; + ContextPtr context; }; class CreatingSetsStep : public IQueryPlanStep @@ -46,6 +47,22 @@ public: void describePipeline(FormatSettings & settings) const override; }; +class DelayedCreatingSetsStep final : public IQueryPlanStep +{ +public: + DelayedCreatingSetsStep(DataStream input_stream, PreparedSets::SubqueriesForSets subqueries_for_sets_, ContextPtr context_); + + String getName() const override { return "DelayedCreatingSets"; } + + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings &) override; + + static std::vector> makePlansForSets(DelayedCreatingSetsStep && step); + +private: + PreparedSets::SubqueriesForSets subqueries_for_sets; + ContextPtr context; +}; + void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context); void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index de1d43bed1b..2b934ec440b 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -110,6 +110,7 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes); bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); +bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp new file mode 100644 index 00000000000..e9100ae9d02 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp @@ -0,0 +1,35 @@ +#include +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +{ + auto * delayed = typeid_cast(node.step.get()); + if (!delayed) + return false; + + auto plans = DelayedCreatingSetsStep::makePlansForSets(std::move(*delayed)); + node.children.reserve(1 + plans.size()); + + DataStreams input_streams; + input_streams.reserve(1 + plans.size()); + input_streams.push_back(node.children.front()->step->getOutputStream()); + + for (const auto & plan : plans) + { + input_streams.push_back(plan->getCurrentDataStream()); + node.children.push_back(plan->getRootNode()); + nodes.splice(nodes.end(), QueryPlan::detachNodes(std::move(*plan))); + } + + auto creating_sets = std::make_unique(std::move(input_streams)); + creating_sets->setStepDescription("Create sets before main query execution"); + node.step = std::move(creating_sets); + return true; +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 37e3b2f67d8..c74487f7cc1 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -163,6 +163,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizePrewhere(stack, nodes); optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*frame.node, nodes); + addPlansForSets(*frame.node, nodes); stack.pop_back(); } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 3fbe3d89845..cb732e58855 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -541,4 +541,9 @@ void QueryPlan::explainEstimate(MutableColumns & columns) } } +QueryPlan::Nodes QueryPlan::detachNodes(QueryPlan && plan) +{ + return std::move(plan.nodes); +} + } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 19d87b101de..d89bdc534be 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -105,10 +105,11 @@ public: std::vector children = {}; }; - const Node * getRootNode() const { return root; } - using Nodes = std::list; + Node * getRootNode() const { return root; } + static Nodes detachNodes(QueryPlan && plan); + private: QueryPlanResourceHolder resources; Nodes nodes; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index e3ae2d4fd4e..de10be599c8 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -76,7 +76,7 @@ void CreatingSetsTransform::startSubquery() } subquery.promise_to_fill_set.set_value(ready_set); - subquery.set_in_progress.reset(); + subquery.set.reset(); done_with_set = true; set_from_cache = true; } @@ -84,7 +84,7 @@ void CreatingSetsTransform::startSubquery() } } - if (subquery.set_in_progress) + if (subquery.set) LOG_TRACE(log, "Creating set, key: {}", subquery.key); if (subquery.table) LOG_TRACE(log, "Filling temporary table."); @@ -93,7 +93,7 @@ void CreatingSetsTransform::startSubquery() /// TODO: make via port table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); - done_with_set = !subquery.set_in_progress; + done_with_set = !subquery.set; done_with_table = !subquery.table; if ((done_with_set && !set_from_cache) /*&& done_with_join*/ && done_with_table) @@ -116,8 +116,8 @@ void CreatingSetsTransform::finishSubquery() } else if (read_rows != 0) { - if (subquery.set_in_progress) - LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set_in_progress->getTotalRowCount(), read_rows, seconds); + if (subquery.set) + LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set->getTotalRowCount(), read_rows, seconds); if (subquery.table) LOG_DEBUG(log, "Created Table with {} rows in {} sec.", read_rows, seconds); } @@ -131,9 +131,9 @@ void CreatingSetsTransform::init() { is_initialized = true; - if (subquery.set_in_progress) + if (subquery.set) { - subquery.set_in_progress->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); + subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); } watch.restart(); @@ -147,7 +147,7 @@ void CreatingSetsTransform::consume(Chunk chunk) if (!done_with_set) { - if (!subquery.set_in_progress->insertFromBlock(block.getColumnsWithTypeAndName())) + if (!subquery.set->insertFromBlock(block.getColumnsWithTypeAndName())) done_with_set = true; } @@ -170,12 +170,12 @@ void CreatingSetsTransform::consume(Chunk chunk) Chunk CreatingSetsTransform::generate() { - if (subquery.set_in_progress) + if (subquery.set) { - subquery.set_in_progress->finishInsert(); - subquery.promise_to_fill_set.set_value(subquery.set_in_progress); + subquery.set->finishInsert(); + subquery.promise_to_fill_set.set_value(subquery.set); if (promise_to_build) - promise_to_build->set_value(subquery.set_in_progress); + promise_to_build->set_value(subquery.set); } if (table_out.initialized()) diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index d5563108ad7..281236e631e 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -68,11 +68,18 @@ bool traverseASTFilter( PreparedSetKey set_key; if ((value->as() || value->as())) - set_key = PreparedSetKey::forSubquery(*value); + set_key = PreparedSetKey::forSubquery(value->getTreeHash()); else - set_key = PreparedSetKey::forLiteral(*value, {primary_key_type}); + set_key = PreparedSetKey::forLiteral(value->getTreeHash(), {primary_key_type}); - SetPtr set = prepared_sets->get(set_key); + FutureSetPtr future_set = prepared_sets->getFuture(set_key); + if (!future_set) + return false; + + if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); + + auto set = future_set->get(); if (!set) return false; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 5d961425469..7a1c3b10c8a 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1204,14 +1204,32 @@ bool KeyCondition::tryPrepareSetIndex( const auto right_arg = func.getArgumentAt(1); - auto prepared_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); + LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); + + auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); + if (!future_set) + return false; + + LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); + + if (!future_set->isReady()) + { + LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); + future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); + } + + auto prepared_set = future_set->get(); if (!prepared_set) return false; + LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); + /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) return false; + LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); + prepared_set->checkColumnsNumber(left_args_count); for (size_t i = 0; i < indexes_mapping.size(); ++i) prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]); diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 235d90bb974..5e186c25b83 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -310,7 +310,13 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (functionIsInOrGlobalInOperator(function_name)) { - ConstSetPtr prepared_set = rhs_argument.tryGetPreparedSet(); + auto future_set = rhs_argument.tryGetPreparedSet(); + if (future_set && !future_set->isReady()) + future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); + + ConstSetPtr prepared_set; + if (future_set) + prepared_set = future_set->get(); if (prepared_set && prepared_set->hasExplicitSetElements()) { diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 06fddd51cb8..5e1d23df3c7 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -624,7 +624,14 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( if (key_tuple_mapping.empty()) return false; - auto prepared_set = right_argument.tryGetPreparedSet(data_types); + auto future_set = right_argument.tryGetPreparedSet(data_types); + if (future_set && !future_set->isReady()) + future_set->buildOrderedSetInplace(right_argument.getTreeContext().getQueryContext()); + + ConstSetPtr prepared_set; + if (future_set) + prepared_set = future_set->get(); + if (!prepared_set || !prepared_set->hasExplicitSetElements()) return false; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index baa11368c8b..6ffba0ad029 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -655,7 +655,14 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( if (key_tuple_mapping.empty()) return false; - ConstSetPtr prepared_set = rhs.tryGetPreparedSet(); + auto future_set = rhs.tryGetPreparedSet(); + if (future_set && !future_set->isReady()) + future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); + + ConstSetPtr prepared_set; + if (future_set) + prepared_set = future_set->get(); + if (!prepared_set || !prepared_set->hasExplicitSetElements()) return false; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index e49459d3d17..e8843ff1489 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -275,7 +275,7 @@ bool RPNBuilderTreeNode::tryGetConstant(Field & output_value, DataTypePtr & outp namespace { -ConstSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) +FutureSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) { if (!dag_node->column) return {}; @@ -285,28 +285,20 @@ ConstSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) column = &column_const->getDataColumn(); if (const auto * column_set = typeid_cast(column)) - { - auto set = column_set->getData(); - - if (set && set->isCreated()) - return set; - } + return column_set->getData(); return {}; } } -ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const +FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const { const auto & prepared_sets = getTreeContext().getPreparedSets(); if (ast_node && prepared_sets) { - auto prepared_sets_with_same_hash = prepared_sets->getByTreeHash(ast_node->getTreeHash()); - for (auto & set : prepared_sets_with_same_hash) - if (set.isCreated()) - return set.get(); + return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); } else if (dag_node) { @@ -317,16 +309,16 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const return {}; } -ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) const +FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) const { const auto & prepared_sets = getTreeContext().getPreparedSets(); if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->get(PreparedSetKey::forSubquery(*ast_node)); + return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); - return prepared_sets->get(PreparedSetKey::forLiteral(*ast_node, data_types)); + return prepared_sets->getFuture(PreparedSetKey::forLiteral(ast_node->getTreeHash(), data_types)); } else if (dag_node) { @@ -337,7 +329,7 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) return nullptr; } -ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( +FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( const std::vector & indexes_mapping, const DataTypes & data_types) const { @@ -346,19 +338,25 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->get(PreparedSetKey::forSubquery(*ast_node)); + return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check /// that the types it was prepared with are compatible with the types of the primary key. - auto types_match = [&indexes_mapping, &data_types](const SetPtr & candidate_set) + auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) { assert(indexes_mapping.size() == data_types.size()); for (size_t i = 0; i < indexes_mapping.size(); ++i) { - if (!candidate_set->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i])) + if (indexes_mapping[i].tuple_index >= set_types.size()) + return false; + + auto lhs = recursiveRemoveLowCardinality(data_types[i]); + auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); + + if (!lhs->equals(*rhs)) return false; } @@ -366,10 +364,10 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( }; auto tree_hash = ast_node->getTreeHash(); - for (const auto & set : prepared_sets->getByTreeHash(tree_hash)) + for (const auto & [key, future_set] : prepared_sets->getSets()) { - if (set.isCreated() && types_match(set.get())) - return set.get(); + if (key.ast_hash == tree_hash && types_match(key.types)) + return future_set; } } else diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index 626eb288493..6f624d93cd6 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -109,13 +109,13 @@ public: bool tryGetConstant(Field & output_value, DataTypePtr & output_type) const; /// Try get prepared set from node - ConstSetPtr tryGetPreparedSet() const; + FutureSetPtr tryGetPreparedSet() const; /// Try get prepared set from node that match data types - ConstSetPtr tryGetPreparedSet(const DataTypes & data_types) const; + FutureSetPtr tryGetPreparedSet(const DataTypes & data_types) const; /// Try get prepared set from node that match indexes mapping and data types - ConstSetPtr tryGetPreparedSet( + FutureSetPtr tryGetPreparedSet( const std::vector & indexes_mapping, const DataTypes & data_types) const; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 6ca74406b17..0f45f6825f6 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -313,8 +313,15 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP if (!column_set) return; - auto set = column_set->getData(); - if (!set || !set->isCreated()) + auto future_set = column_set->getData(); + if (!future_set) + return; + + if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); + + auto set = future_set->get(); + if (!set) return; if (!set->hasExplicitSetElements()) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 4ff00facfdc..6e7b9ea5849 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -80,24 +80,24 @@ ASTPtr buildWhereExpression(const ASTs & functions) return makeASTFunction("and", functions); } -void buildSets(const ASTPtr & expression, ExpressionAnalyzer & analyzer) -{ - const auto * func = expression->as(); - if (func && functionIsInOrGlobalInOperator(func->name)) - { - const IAST & args = *func->arguments; - const ASTPtr & arg = args.children.at(1); - if (arg->as() || arg->as()) - { - analyzer.tryMakeSetForIndexFromSubquery(arg); - } - } - else - { - for (const auto & child : expression->children) - buildSets(child, analyzer); - } -} +// void buildSets(const ASTPtr & expression, ExpressionAnalyzer & analyzer) +// { +// const auto * func = expression->as(); +// if (func && functionIsInOrGlobalInOperator(func->name)) +// { +// const IAST & args = *func->arguments; +// const ASTPtr & arg = args.children.at(1); +// if (arg->as() || arg->as()) +// { +// analyzer.tryMakeSetForIndexFromSubquery(arg); +// } +// } +// else +// { +// for (const auto & child : expression->children) +// buildSets(child, analyzer); +// } +// } } @@ -199,7 +199,7 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex /// Let's analyze and calculate the prepared expression. auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(expression_ast, syntax_result, context); - buildSets(expression_ast, analyzer); + //buildSets(expression_ast, analyzer); ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes); Block block_with_filter = block; From e7c3d74b7ce6a9703a0b71b0ec9cdffa4739fc9a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 4 May 2023 21:37:15 +0000 Subject: [PATCH 0184/2223] Fix filling_row & next_row construction --- .../Transforms/FillingTransform.cpp | 58 +++++++++++++++++-- src/Processors/Transforms/FillingTransform.h | 16 ++++- 2 files changed, 69 insertions(+), 5 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 4fc2b953a4f..f3e32c3cd89 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -14,7 +14,7 @@ namespace DB { -constexpr bool debug_logging_enabled = false; +constexpr bool debug_logging_enabled = true; template void logDebug(String key, const T & value, const char * separator = " : ") @@ -192,8 +192,8 @@ FillingTransform::FillingTransform( , sort_description(sort_description_) , fill_description(fill_description_) , interpolate_description(interpolate_description_) - , filling_row(sort_description_) - , next_row(sort_description_) + , filling_row(fill_description_) + , next_row(fill_description_) { if (interpolate_description) interpolate_actions = std::make_shared(interpolate_description->actions); @@ -227,6 +227,7 @@ FillingTransform::FillingTransform( "WITH FILL bound values cannot be negative for unsigned type {}", type->getName()); } } + logDebug("fill description", dumpSortDescription(fill_description)); std::set unique_positions; for (auto pos : fill_column_positions) @@ -369,7 +370,7 @@ static void initColumnsByPositions( MutableColumnRawPtrs & output_columns_by_position, const std::vector & positions) { - for (size_t pos : positions) + for (const size_t pos : positions) { input_columns_by_positions.push_back(input_columns[pos]); output_columns_by_position.push_back(output_columns[pos].get()); @@ -380,10 +381,12 @@ void FillingTransform::initColumns( const Columns & input_columns, Columns & input_fill_columns, Columns & input_interpolate_columns, + Columns & ,//input_sort_prefix_columns, Columns & input_other_columns, MutableColumns & output_columns, MutableColumnRawPtrs & output_fill_columns, MutableColumnRawPtrs & output_interpolate_columns, + MutableColumnRawPtrs & , //output_sort_prefix_columns, MutableColumnRawPtrs & output_other_columns) { Columns non_const_columns; @@ -398,6 +401,7 @@ void FillingTransform::initColumns( initColumnsByPositions(non_const_columns, input_fill_columns, output_columns, output_fill_columns, fill_column_positions); initColumnsByPositions( non_const_columns, input_interpolate_columns, output_columns, output_interpolate_columns, interpolate_column_positions); + // initColumnsByPositions(non_const_columns, input_sort_prefix_columns, output_columns, output_sort_prefix_columns, sort_prefix_positions); initColumnsByPositions(non_const_columns, input_other_columns, output_columns, output_other_columns, other_column_positions); } @@ -423,19 +427,23 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut Columns input_fill_columns; Columns input_interpolate_columns; + Columns input_sort_prefix_columns; Columns input_other_columns; MutableColumnRawPtrs res_fill_columns; MutableColumnRawPtrs res_interpolate_columns; + MutableColumnRawPtrs res_sort_prefix_columns; MutableColumnRawPtrs res_other_columns; initColumns( input_columns, input_fill_columns, input_interpolate_columns, + input_sort_prefix_columns, input_other_columns, result_columns, res_fill_columns, res_interpolate_columns, + res_sort_prefix_columns, res_other_columns); if (first) @@ -457,6 +465,43 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut return true; } +// void FillingTransform::transformRange( +// const Columns & input_fill_columns, +// const Columns & input_interpolate_columns, +// const Columns & input_sort_prefix_columns, +// const Columns & input_other_columns, +// const MutableColumns & result_columns, +// const MutableColumnRawPtrs & res_fill_columns, +// const MutableColumnRawPtrs & res_interpolate_columns, +// const MutableColumnRawPtrs & res_sort_prefix_columns, +// const MutableColumnRawPtrs & res_other_columns, +// std::pair range) +// { +// Block interpolate_block; +// if (first) +// { +// for (size_t i = 0, size = filling_row.size(); i < size; ++i) +// { +// auto current_value = (*input_fill_columns[i])[0]; +// const auto & fill_from = filling_row.getFillDescription(i).fill_from; + +// if (!fill_from.isNull() && !equals(current_value, fill_from)) +// { +// filling_row.initFromDefaults(i); +// if (less(fill_from, current_value, filling_row.getDirection(i))) +// { +// interpolate(result_columns, interpolate_block); +// insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); +// } +// break; +// } +// filling_row[i] = current_value; +// } +// first = false; +// } + +// } + void FillingTransform::transform(Chunk & chunk) { logDebug("new chunk rows", chunk.getNumRows()); @@ -469,9 +514,11 @@ void FillingTransform::transform(Chunk & chunk) Columns input_fill_columns; Columns input_interpolate_columns; + Columns input_sort_prefix_columns; Columns input_other_columns; MutableColumnRawPtrs res_fill_columns; MutableColumnRawPtrs res_interpolate_columns; + MutableColumnRawPtrs res_sort_prefix_columns; MutableColumnRawPtrs res_other_columns; MutableColumns result_columns; @@ -501,10 +548,12 @@ void FillingTransform::transform(Chunk & chunk) input_columns, input_fill_columns, input_interpolate_columns, + input_sort_prefix_columns, input_other_columns, result_columns, res_fill_columns, res_interpolate_columns, + res_sort_prefix_columns, res_other_columns); if (first) @@ -566,6 +615,7 @@ void FillingTransform::transform(Chunk & chunk) copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); + // copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); copyRowFromColumns(res_other_columns, input_other_columns, row_ind); } diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 021784c263d..dac92e43231 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -32,18 +32,32 @@ protected: void transform(Chunk & chunk) override; private: + using MutableColumnRawPtrs = std::vector; + void transformRange( + const Columns & input_fill_columns, + const Columns & input_interpolate_columns, + const Columns & input_sort_prefix_columns, + const Columns & input_other_columns, + const MutableColumns & result_columns, + const MutableColumnRawPtrs & res_fill_columns, + const MutableColumnRawPtrs & res_interpolate_columns, + const MutableColumnRawPtrs & res_sort_prefix_columns, + const MutableColumnRawPtrs & res_other_columns, + std::pair range); + void saveLastRow(const MutableColumns & result_columns); void interpolate(const MutableColumns & result_columns, Block & interpolate_block); - using MutableColumnRawPtrs = std::vector; void initColumns( const Columns & input_columns, Columns & input_fill_columns, Columns & input_interpolate_columns, + Columns & input_sort_prefix_columns, Columns & input_other_columns, MutableColumns & output_columns, MutableColumnRawPtrs & output_fill_columns, MutableColumnRawPtrs & output_interpolate_columns, + MutableColumnRawPtrs & output_sort_prefix_columns, MutableColumnRawPtrs & output_other_columns); bool generateSuffixIfNeeded( From 0370b6fbe68ec7c0965434d8934c2e00ba4e2d3e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 4 May 2023 21:58:02 +0000 Subject: [PATCH 0185/2223] Add transformRange() --- .../Transforms/FillingTransform.cpp | 184 +++++++++--------- 1 file changed, 88 insertions(+), 96 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index f3e32c3cd89..99f9d55d130 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -465,42 +465,85 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut return true; } -// void FillingTransform::transformRange( -// const Columns & input_fill_columns, -// const Columns & input_interpolate_columns, -// const Columns & input_sort_prefix_columns, -// const Columns & input_other_columns, -// const MutableColumns & result_columns, -// const MutableColumnRawPtrs & res_fill_columns, -// const MutableColumnRawPtrs & res_interpolate_columns, -// const MutableColumnRawPtrs & res_sort_prefix_columns, -// const MutableColumnRawPtrs & res_other_columns, -// std::pair range) -// { -// Block interpolate_block; -// if (first) -// { -// for (size_t i = 0, size = filling_row.size(); i < size; ++i) -// { -// auto current_value = (*input_fill_columns[i])[0]; -// const auto & fill_from = filling_row.getFillDescription(i).fill_from; +void FillingTransform::transformRange( + const Columns & input_fill_columns, + const Columns & input_interpolate_columns, + const Columns &, //input_sort_prefix_columns, + const Columns & input_other_columns, + const MutableColumns & result_columns, + const MutableColumnRawPtrs & res_fill_columns, + const MutableColumnRawPtrs & res_interpolate_columns, + const MutableColumnRawPtrs &, //res_sort_prefix_columns, + const MutableColumnRawPtrs & res_other_columns, + std::pair range) +{ + const size_t range_begin = range.first; + const size_t range_end = range.second; -// if (!fill_from.isNull() && !equals(current_value, fill_from)) -// { -// filling_row.initFromDefaults(i); -// if (less(fill_from, current_value, filling_row.getDirection(i))) -// { -// interpolate(result_columns, interpolate_block); -// insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); -// } -// break; -// } -// filling_row[i] = current_value; -// } -// first = false; -// } + Block interpolate_block; + if (first) + { + for (size_t i = 0, size = filling_row.size(); i < size; ++i) + { + auto current_value = (*input_fill_columns[i])[0]; + const auto & fill_from = filling_row.getFillDescription(i).fill_from; -// } + if (!fill_from.isNull() && !equals(current_value, fill_from)) + { + filling_row.initFromDefaults(i); + if (less(fill_from, current_value, filling_row.getDirection(i))) + { + interpolate(result_columns, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + } + break; + } + filling_row[i] = current_value; + } + first = false; + } + + for (size_t row_ind = range_begin; row_ind < range_end; ++row_ind) + { + logDebug("row", row_ind); + logDebug("filling_row", filling_row); + logDebug("next_row", next_row); + + bool should_insert_first = next_row < filling_row; + logDebug("should_insert_first", should_insert_first); + + for (size_t i = 0, size = filling_row.size(); i < size; ++i) + { + auto current_value = (*input_fill_columns[i])[row_ind]; + const auto & fill_to = filling_row.getFillDescription(i).fill_to; + + if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i))) + next_row[i] = current_value; + else + next_row[i] = fill_to; + } + logDebug("next_row updated", next_row); + + /// A case, when at previous step row was initialized from defaults 'fill_from' values + /// and probably we need to insert it to block. + if (should_insert_first && filling_row < next_row) + { + interpolate(result_columns, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + } + + while (filling_row.next(next_row)) + { + interpolate(result_columns, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + } + + copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); + copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); + // copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + copyRowFromColumns(res_other_columns, input_other_columns, row_ind); + } +} void FillingTransform::transform(Chunk & chunk) { @@ -556,68 +599,17 @@ void FillingTransform::transform(Chunk & chunk) res_sort_prefix_columns, res_other_columns); - if (first) - { - for (size_t i = 0, size = filling_row.size(); i < size; ++i) - { - auto current_value = (*input_fill_columns[i])[0]; - const auto & fill_from = filling_row.getFillDescription(i).fill_from; - - if (!fill_from.isNull() && !equals(current_value, fill_from)) - { - filling_row.initFromDefaults(i); - if (less(fill_from, current_value, filling_row.getDirection(i))) - { - interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - } - break; - } - filling_row[i] = current_value; - } - first = false; - } - - for (size_t row_ind = 0; row_ind < num_rows; ++row_ind) - { - logDebug("row", row_ind); - logDebug("filling_row", filling_row); - logDebug("next_row", next_row); - - bool should_insert_first = next_row < filling_row; - logDebug("should_insert_first", should_insert_first); - - for (size_t i = 0, size = filling_row.size(); i < size; ++i) - { - auto current_value = (*input_fill_columns[i])[row_ind]; - const auto & fill_to = filling_row.getFillDescription(i).fill_to; - - if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i))) - next_row[i] = current_value; - else - next_row[i] = fill_to; - } - logDebug("next_row updated", next_row); - - /// A case, when at previous step row was initialized from defaults 'fill_from' values - /// and probably we need to insert it to block. - if (should_insert_first && filling_row < next_row) - { - interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - } - - while (filling_row.next(next_row)) - { - interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - } - - copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); - copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); - // copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); - copyRowFromColumns(res_other_columns, input_other_columns, row_ind); - } + transformRange( + input_fill_columns, + input_interpolate_columns, + input_sort_prefix_columns, + input_other_columns, + result_columns, + res_fill_columns, + res_interpolate_columns, + res_sort_prefix_columns, + res_other_columns, + {0, num_rows}); saveLastRow(result_columns); size_t num_output_rows = result_columns[0]->size(); From a218f010e85c0bc995df17f5416bee4bcd8e6148 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 5 May 2023 14:12:14 +0200 Subject: [PATCH 0186/2223] First portion of speedup Do not generate DateLUT for each serialized and deserialized date --- src/Common/LocalDate.h | 8 ++++---- .../Serializations/SerializationDate.cpp | 11 +++++++---- src/DataTypes/Serializations/SerializationDate.h | 5 ++++- src/Functions/FunctionsConversion.h | 12 ++++++------ src/IO/ReadHelpers.h | 16 ++++++++-------- src/IO/WriteHelpers.h | 8 ++++---- 6 files changed, 33 insertions(+), 27 deletions(-) diff --git a/src/Common/LocalDate.h b/src/Common/LocalDate.h index dc36f92bebf..4a383129ae4 100644 --- a/src/Common/LocalDate.h +++ b/src/Common/LocalDate.h @@ -61,17 +61,17 @@ public: init(time); } - LocalDate(DayNum day_num) /// NOLINT + LocalDate(DayNum day_num, const DateLUTImpl & time_zone = DateLUT::instance()) /// NOLINT { - const auto & values = DateLUT::instance().getValues(day_num); + const auto & values = time_zone.getValues(day_num); m_year = values.year; m_month = values.month; m_day = values.day_of_month; } - explicit LocalDate(ExtendedDayNum day_num) + explicit LocalDate(ExtendedDayNum day_num, const DateLUTImpl & time_zone = DateLUT::instance()) { - const auto & values = DateLUT::instance().getValues(day_num); + const auto & values = time_zone.getValues(day_num); m_year = values.year; m_month = values.month; m_day = values.day_of_month; diff --git a/src/DataTypes/Serializations/SerializationDate.cpp b/src/DataTypes/Serializations/SerializationDate.cpp index 678817017e0..bc2057d549e 100644 --- a/src/DataTypes/Serializations/SerializationDate.cpp +++ b/src/DataTypes/Serializations/SerializationDate.cpp @@ -13,7 +13,7 @@ namespace DB void SerializationDate::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeDateText(DayNum(assert_cast(column).getData()[row_num]), ostr); + writeDateText(DayNum(assert_cast(column).getData()[row_num]), ostr, time_zone); } void SerializationDate::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -26,7 +26,7 @@ void SerializationDate::deserializeWholeText(IColumn & column, ReadBuffer & istr void SerializationDate::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { DayNum x; - readDateText(x, istr); + readDateText(x, istr, time_zone); assert_cast(column).getData().push_back(x); } @@ -46,7 +46,7 @@ void SerializationDate::deserializeTextQuoted(IColumn & column, ReadBuffer & ist { DayNum x; assertChar('\'', istr); - readDateText(x, istr); + readDateText(x, istr, time_zone); assertChar('\'', istr); assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } @@ -62,7 +62,7 @@ void SerializationDate::deserializeTextJSON(IColumn & column, ReadBuffer & istr, { DayNum x; assertChar('"', istr); - readDateText(x, istr); + readDateText(x, istr, time_zone); assertChar('"', istr); assert_cast(column).getData().push_back(x); } @@ -80,5 +80,8 @@ void SerializationDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, readCSV(value, istr); assert_cast(column).getData().push_back(value); } +SerializationDate::SerializationDate(const TimezoneMixin & time_zone_) : TimezoneMixin(time_zone_) +{ +} } diff --git a/src/DataTypes/Serializations/SerializationDate.h b/src/DataTypes/Serializations/SerializationDate.h index 099d7444c3d..c4e57470673 100644 --- a/src/DataTypes/Serializations/SerializationDate.h +++ b/src/DataTypes/Serializations/SerializationDate.h @@ -1,13 +1,16 @@ #pragma once #include +#include namespace DB { -class SerializationDate final : public SerializationNumber +class SerializationDate final : public SerializationNumber, public TimezoneMixin { public: + explicit SerializationDate(const TimezoneMixin & time_zone_ = TimezoneMixin()); + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 28002d34acc..645504df829 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -718,9 +718,9 @@ template <> struct FormatImpl { template - static ReturnType execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl *) + static ReturnType execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl * time_zone) { - writeDateText(DayNum(x), wb); + writeDateText(DayNum(x), wb, *time_zone); return ReturnType(true); } }; @@ -729,9 +729,9 @@ template <> struct FormatImpl { template - static ReturnType execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl *) + static ReturnType execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl * time_zone) { - writeDateText(ExtendedDayNum(x), wb); + writeDateText(ExtendedDayNum(x), wb, *time_zone); return ReturnType(true); } }; @@ -830,8 +830,8 @@ struct ConvertImpl(*col_with_type_and_name.type); const DateLUTImpl * time_zone = nullptr; - /// For argument of DateTime type, second argument with time zone could be specified. - if constexpr (std::is_same_v || std::is_same_v) + /// For argument of Date or DateTime type, second argument with time zone could be specified. + if constexpr (std::is_same_v || std::is_same_v || std::is_same_v) { auto non_null_args = createBlockWithNestedColumns(arguments); time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 0b0a0640cb1..0127809c832 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -723,7 +723,7 @@ inline void convertToDayNum(DayNum & date, ExtendedDayNum & from) } template -inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf) +inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { static constexpr bool throw_exception = std::is_same_v; @@ -734,13 +734,13 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf) else if (!readDateTextImpl(local_date, buf)) return false; - ExtendedDayNum ret = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day()); + ExtendedDayNum ret = date_lut.makeDayNum(local_date.year(), local_date.month(), local_date.day()); convertToDayNum(date,ret); return ReturnType(true); } template -inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf) +inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { static constexpr bool throw_exception = std::is_same_v; @@ -752,7 +752,7 @@ inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf) return false; /// When the parameter is out of rule or out of range, Date32 uses 1925-01-01 as the default value (-DateLUT::instance().getDayNumOffsetEpoch(), -16436) and Date uses 1970-01-01. - date = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day(), -static_cast(DateLUT::instance().getDayNumOffsetEpoch())); + date = date_lut.makeDayNum(local_date.year(), local_date.month(), local_date.day(), -static_cast(date_lut.getDayNumOffsetEpoch())); return ReturnType(true); } @@ -762,14 +762,14 @@ inline void readDateText(LocalDate & date, ReadBuffer & buf) readDateTextImpl(date, buf); } -inline void readDateText(DayNum & date, ReadBuffer & buf) +inline void readDateText(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { - readDateTextImpl(date, buf); + readDateTextImpl(date, buf, date_lut); } -inline void readDateText(ExtendedDayNum & date, ReadBuffer & buf) +inline void readDateText(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { - readDateTextImpl(date, buf); + readDateTextImpl(date, buf, date_lut); } inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index e08e451e0a7..9ee11d3cc9f 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -694,15 +694,15 @@ inline void writeDateText(const LocalDate & date, WriteBuffer & buf) } template -inline void writeDateText(DayNum date, WriteBuffer & buf) +inline void writeDateText(DayNum date, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { - writeDateText(LocalDate(date), buf); + writeDateText(LocalDate(date, time_zone), buf); } template -inline void writeDateText(ExtendedDayNum date, WriteBuffer & buf) +inline void writeDateText(ExtendedDayNum date, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { - writeDateText(LocalDate(date), buf); + writeDateText(LocalDate(date, time_zone), buf); } /// In the format YYYY-MM-DD HH:MM:SS From 2b08801ae9bc1ca456247282ebfe060a9df0bce4 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 5 May 2023 15:50:19 +0200 Subject: [PATCH 0187/2223] add timezone param --- src/IO/WriteHelpers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 9ee11d3cc9f..8a7cd72f79a 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -883,7 +883,7 @@ inline void writeText(is_enum auto x, WriteBuffer & buf) { writeText(magic_enum: inline void writeText(std::string_view x, WriteBuffer & buf) { writeString(x.data(), x.size(), buf); } -inline void writeText(const DayNum & x, WriteBuffer & buf) { writeDateText(LocalDate(x), buf); } +inline void writeText(const DayNum & x, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { writeDateText(LocalDate(x, time_zone), buf); } inline void writeText(const LocalDate & x, WriteBuffer & buf) { writeDateText(x, buf); } inline void writeText(const LocalDateTime & x, WriteBuffer & buf) { writeDateTimeText(x, buf); } inline void writeText(const UUID & x, WriteBuffer & buf) { writeUUIDText(x, buf); } From ea2970d344a9fee2e9f3b3d8d830a59f3c890673 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 5 May 2023 18:17:18 +0000 Subject: [PATCH 0188/2223] Fix: correct header for FillingTransform fixes 01921_with_fill_with_totals --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 99f9d55d130..93dca03e4ac 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -188,7 +188,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & FillingTransform::FillingTransform( const Block & header_, const SortDescription & sort_description_, const SortDescription& fill_description_, InterpolateDescriptionPtr interpolate_description_) - : ISimpleTransform(header_, transformHeader(header_, sort_description_), true) + : ISimpleTransform(header_, transformHeader(header_, fill_description_), true) , sort_description(sort_description_) , fill_description(fill_description_) , interpolate_description(interpolate_description_) From 54ef6769f1131e49c87235c6948a39951adeae49 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 May 2023 18:30:08 +0000 Subject: [PATCH 0189/2223] Refactor PreparedSets [4] --- src/Interpreters/PreparedSets.cpp | 8 +- src/Interpreters/PreparedSets.h | 3 +- src/Planner/PlannerActionsVisitor.cpp | 20 +- .../optimizePrimaryKeyCondition.cpp | 2 + .../QueryPlan/ReadFromMergeTree.cpp | 173 ++++++++++++------ src/Processors/QueryPlan/ReadFromMergeTree.h | 11 +- .../QueryPlan/SourceStepWithFilter.h | 2 + .../Transforms/CreatingSetsTransform.cpp | 1 + src/Storages/MergeTree/KeyCondition.cpp | 10 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- 10 files changed, 160 insertions(+), 74 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 733eb1c24bb..5d9a0f27496 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -221,7 +221,7 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; - std::cerr << StackTrace().toString() << std::endl; + // std::cerr << StackTrace().toString() << std::endl; auto set_cache = context->getPreparedSetsCache(); if (set_cache) @@ -248,6 +248,10 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c auto plan = subquery.detachSource(); auto description = subquery.key; + // WriteBufferFromOwnString buf; + // plan->explainPlan(buf, {.header=true}); + // std::cerr << buf.str() << std::endl; + auto creating_set = std::make_unique( plan->getCurrentDataStream(), description, @@ -279,7 +283,7 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } -FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} +FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) { std::cerr << block.dumpStructure() << std::endl; } FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 8661d81a96a..b4d01754ea8 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -16,6 +16,7 @@ #include "Processors/Executors/CompletedPipelineExecutor.h" #include "Processors/QueryPlan/BuildQueryPipelineSettings.h" #include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" +#include "Processors/Sinks/EmptySink.h" #include "Processors/Sinks/NullSink.h" #include @@ -171,7 +172,7 @@ public: auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); + pipeline.complete(std::make_shared(Block())); CompletedPipelineExecutor executor(pipeline); executor.execute(); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 453b02a2f8f..e0844a6d2b1 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -629,14 +629,20 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma //auto set_key = planner_context->createSetKey(in_second_argument); - DataTypes set_element_types = {in_first_argument->getResultType()}; - const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); - if (left_tuple_type && left_tuple_type->getElements().size() != 1) - set_element_types = left_tuple_type->getElements(); + DataTypes set_element_types; - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); + auto in_second_argument_node_type = in_second_argument->getNodeType(); + if (!(in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION)) + { + set_element_types = {in_first_argument->getResultType()}; + const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + } auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index e98386a6ee9..5ef786ff975 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -28,6 +28,8 @@ void optimizePrimaryKeyCondition(const Stack & stack) else break; } + + source_step_with_filter->onAddFilterFinish(); } } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 185ec9bace8..ad63b486c7c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -247,7 +247,7 @@ ReadFromMergeTree::ReadFromMergeTree( { /// build sort description for output stream SortDescription sort_description; - const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(); + const Names & sorting_key_columns = metadata_for_reading->getSortingKeyColumns(); const Block & header = output_stream->header; const int sort_direction = getSortDirection(); for (const auto & column_name : sorting_key_columns) @@ -1118,7 +1118,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge prewhere_info, filter_nodes, storage_snapshot->metadata, - storage_snapshot->getMetadataForQuery(), + metadata_for_reading, query_info, context, requested_num_streams, @@ -1126,7 +1126,90 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); +} + +static ActionsDAGPtr buildFilterDAG( + const ContextPtr & context, + const PrewhereInfoPtr & prewhere_info, + const ActionDAGNodes & added_filter_nodes, + const SelectQueryInfo & query_info) +{ + const auto & settings = context->getSettingsRef(); + ActionsDAG::NodeRawConstPtrs nodes; + + if (prewhere_info) + { + { + const auto & node = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); + nodes.push_back(&node); + } + + if (prewhere_info->row_level_filter) + { + const auto & node = prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name); + nodes.push_back(&node); + } + } + + for (const auto & node : added_filter_nodes.nodes) + nodes.push_back(node); + + std::unordered_map node_name_to_input_node_column; + + if (settings.allow_experimental_analyzer && query_info.planner_context) + { + const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression); + for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) + { + const auto & column = table_expression_data.getColumnOrThrow(column_name); + node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name)); + } + } + + return ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); +} + +static void buildKeyCondition( + std::optional & key_condition, + ActionsDAGPtr filter_actions_dag, + const ContextPtr & context, + const SelectQueryInfo & query_info, + const StorageMetadataPtr & metadata_snapshot) +{ + key_condition.reset(); + + // Build and check if primary key is used when necessary + const auto & primary_key = metadata_snapshot->getPrimaryKey(); + const Names & primary_key_column_names = primary_key.column_names; + + const auto & settings = context->getSettingsRef(); + if (settings.query_plan_optimize_primary_key) + { + NameSet array_join_name_set; + if (query_info.syntax_analyzer_result) + array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); + + key_condition.emplace(filter_actions_dag, + context, + primary_key_column_names, + primary_key.expression, + array_join_name_set); + } + else + { + key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); + } +} + +void ReadFromMergeTree::onAddFilterFinish() +{ + if (!filter_nodes.nodes.empty()) + { + auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); + buildKeyCondition(key_condition, filter_actions_dag, context, query_info, metadata_for_reading); + } } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( @@ -1142,44 +1225,14 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log) + Poco::Logger * log, + std::optional & key_condition) { const auto & settings = context->getSettingsRef(); if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key) { - ActionsDAG::NodeRawConstPtrs nodes; - - if (prewhere_info) - { - { - const auto & node = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); - nodes.push_back(&node); - } - - if (prewhere_info->row_level_filter) - { - const auto & node = prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name); - nodes.push_back(&node); - } - } - - for (const auto & node : added_filter_nodes.nodes) - nodes.push_back(node); - - std::unordered_map node_name_to_input_node_column; - - if (settings.allow_experimental_analyzer && query_info.planner_context) - { - const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression); - for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) - { - const auto & column = table_expression_data.getColumnOrThrow(column_name); - node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name)); - } - } - auto updated_query_info_with_filter_dag = query_info; - updated_query_info_with_filter_dag.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); + updated_query_info_with_filter_dag.filter_actions_dag = buildFilterDAG(context, prewhere_info, added_filter_nodes, query_info); return selectRangesToReadImpl( parts, @@ -1192,7 +1245,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); } return selectRangesToReadImpl( @@ -1206,7 +1260,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( @@ -1220,7 +1275,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log) + Poco::Logger * log, + std::optional & key_condition) { AnalysisResult result; const auto & settings = context->getSettingsRef(); @@ -1246,24 +1302,29 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; - std::optional key_condition; - if (settings.query_plan_optimize_primary_key) - { - NameSet array_join_name_set; - if (query_info.syntax_analyzer_result) - array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); + // if (!key_condition) + // { + // if (settings.query_plan_optimize_primary_key) + // { + // NameSet array_join_name_set; + // if (query_info.syntax_analyzer_result) + // array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); - key_condition.emplace(query_info.filter_actions_dag, - context, - primary_key_column_names, - primary_key.expression, - array_join_name_set); - } - else - { - key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); - } + // key_condition.emplace(query_info.filter_actions_dag, + // context, + // primary_key_column_names, + // primary_key.expression, + // array_join_name_set); + // } + // else + // { + // key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); + // } + // } + + if (!key_condition) + buildKeyCondition(key_condition, query_info.filter_actions_dag, context, query_info, metadata_snapshot); if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue()) { @@ -1395,7 +1456,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, /// update sort info for output stream SortDescription sort_description; - const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(); + const Names & sorting_key_columns = metadata_for_reading->getSortingKeyColumns(); const Block & header = output_stream->header; const int sort_direction = getSortDirection(); for (const auto & column_name : sorting_key_columns) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 5e4ba117967..121970f2ca9 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -145,7 +145,8 @@ public: const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log); + Poco::Logger * log, + std::optional & key_condition); MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; @@ -177,6 +178,8 @@ public: size_t getNumStreams() const { return requested_num_streams; } bool isParallelReadingEnabled() const { return read_task_callback != std::nullopt; } + void onAddFilterFinish() override; + private: static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl( MergeTreeData::DataPartsVector parts, @@ -189,7 +192,8 @@ private: const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log); + Poco::Logger * log, + std::optional & key_condition); int getSortDirection() const { @@ -228,6 +232,9 @@ private: std::shared_ptr max_block_numbers_to_read; + /// Pre-computed value, needed to trigger sets creatin for PK + mutable std::optional key_condition; + Poco::Logger * log; UInt64 selected_parts = 0; UInt64 selected_rows = 0; diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index a363451fff2..34b6e3c6a7b 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -37,6 +37,8 @@ public: filter_dags.push_back(std::move(filter_dag)); } + virtual void onAddFilterFinish() {} + protected: std::vector filter_dags; ActionDAGNodes filter_nodes; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index de10be599c8..6626d4b9795 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -133,6 +133,7 @@ void CreatingSetsTransform::init() if (subquery.set) { + //std::cerr << "=========== " << getInputPort().getHeader().dumpStructure() << std::endl; subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7a1c3b10c8a..efeb9e40dd4 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1204,17 +1204,17 @@ bool KeyCondition::tryPrepareSetIndex( const auto right_arg = func.getArgumentAt(1); - LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); if (!future_set) return false; - LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); if (!future_set->isReady()) { - LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); } @@ -1222,13 +1222,13 @@ bool KeyCondition::tryPrepareSetIndex( if (!prepared_set) return false; - LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) return false; - LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); prepared_set->checkColumnsNumber(left_args_count); for (size_t i = 0; i < indexes_mapping.size(); ++i) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 22df8f298c4..aae8f843a3c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1291,6 +1291,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); + std::optional key_condition; return ReadFromMergeTree::selectRangesToRead( std::move(parts), prewhere_info, @@ -1304,7 +1305,8 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); } QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( From e1151f150f23e0bbcb52ae0a1a3ef01a0ecb97da Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Fri, 5 May 2023 18:37:25 +0000 Subject: [PATCH 0190/2223] Fix clang build errors --- src/Databases/DatabaseS3.cpp | 2 +- src/Databases/DatabaseS3.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index bc318ecd9bf..96616426475 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -247,7 +247,7 @@ DatabaseS3::Configuration DatabaseS3::parseArguments(ASTs engine_args, ContextPt } else { - auto supported_signature = + const std::string supported_signature = " - S3()\n" " - S3('url')\n" " - S3('url', 'NOSIGN')\n" diff --git a/src/Databases/DatabaseS3.h b/src/Databases/DatabaseS3.h index 65f80dca2ba..4e6910566df 100644 --- a/src/Databases/DatabaseS3.h +++ b/src/Databases/DatabaseS3.h @@ -57,7 +57,7 @@ public: static Configuration parseArguments(ASTs engine_args, ContextPtr context); protected: - StoragePtr getTableImpl(const String & url, ContextPtr context) const; + StoragePtr getTableImpl(const String & name, ContextPtr context) const; void addTable(const std::string & table_name, StoragePtr table_storage) const; From 18d1a4356d2ba1e7502d0ba207e6ac8f53fc3e02 Mon Sep 17 00:00:00 2001 From: tpanetti Date: Fri, 5 May 2023 12:19:35 -0700 Subject: [PATCH 0191/2223] Change SHOW COLUMNS query to display MySQL types in MySQL Compatibility mode This updates the SHOW COLUMN SQL query to display MySQL types when this query is issued by a client connected via MySQL Compatibility port --- .../InterpreterShowColumnsQuery.cpp | 78 ++++++- .../InterpreterShowColumnsQuery.h | 1 + ...show_columns_mysql_compatibility.reference | 213 ++++++++++++++++++ .../02726_show_columns_mysql_compatibility.sh | 115 ++++++++++ 4 files changed, 405 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference create mode 100755 tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 4474be21d8b..17ccafdd1ce 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -26,11 +26,17 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() WriteBufferFromOwnString rewritten_query; - rewritten_query << "SELECT name AS field, type AS type, startsWith(type, 'Nullable') AS null, trim(concatWithSeparator(' ', if(is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, if(default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra "; - // TODO Interpret query.extended. It is supposed to show internal/virtual columns. Need to fetch virtual column names, see // IStorage::getVirtuals(). We can't easily do that via SQL. + // If connected via MySQL Compatibility mode, convert ClickHouse types to MySQL + if (getContext()->getClientInfo().interface == DB::ClientInfo::Interface::MYSQL) + { + rewritten_query << getMySQLQuery(); + } + else { + rewritten_query << "SELECT name AS field, type AS type, startsWith(type, 'Nullable') AS null, trim(concatWithSeparator(' ', if(is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, if(default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra "; + } if (query.full) { /// "Full" mode is mostly for MySQL compat @@ -93,6 +99,74 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() } +String InterpreterShowColumnsQuery::getMySQLQuery() +{ + WriteBufferFromOwnString mysql_specific_query; + + mysql_specific_query << "SELECT name AS field, " + << "CASE " + << " WHEN startsWith(type, 'Nullable') THEN " + << " CASE " + << " WHEN substring(type, 10, length(type) - 10) IN ('UInt8', 'Int8') THEN 'tinyint' " + << " WHEN substring(type, 10, length(type) - 10) IN ('UInt16', 'Int16') THEN 'smallint' " + << " WHEN substring(type, 10, length(type) - 10) IN ('UInt32', 'Int32') THEN 'int' " + << " WHEN substring(type, 10, length(type) - 10) IN ('UInt64', 'Int64', 'UInt128', 'Int128', 'UInt256', 'Int256') THEN 'bigint' " + << " WHEN substring(type, 10, length(type) - 10) = 'Float32' THEN 'float' " + << " WHEN substring(type, 10, length(type) - 10) = 'Float64' THEN 'double' " + << " WHEN substring(type, 10, length(type) - 10) LIKE 'Decimal%' THEN 'decimal' " + << " WHEN substring(type, 10, length(type) - 10) = 'Boolean' THEN 'tinyint' " + << " WHEN substring(type, 10, length(type) - 10) = 'String' THEN 'text' " + << " WHEN substring(type, 10, length(type) - 10) LIKE 'FixedString%' THEN 'text' " + << " WHEN substring(type, 10, length(type) - 10) LIKE 'Date%' THEN 'date' " + << " WHEN substring(type, 10, length(type) - 10) LIKE 'DateTime%' THEN 'datetime' " + << " WHEN substring(type, 10, length(type) - 10) = 'JSON' THEN 'json' " + << " WHEN substring(type, 10, length(type) - 10) = 'UUID' THEN 'binary' " + << " WHEN substring(type, 10, length(type) - 10) LIKE 'Enum%' THEN 'enum' " + << " WHEN substring(type, 10, length(type) - 10) LIKE 'LowCardinality%' THEN 'text' " + << " WHEN substring(type, 10, length(type) - 10) LIKE 'Array%' THEN 'json' " + << " WHEN substring(type, 10, length(type) - 10) LIKE 'Map%' THEN 'json' " + << " WHEN substring(type, 10, length(type) - 10) IN ('SimpleAggregateFunction', 'AggregateFunction') THEN 'text' " + << " WHEN substring(type, 10, length(type) - 10) = 'Nested' THEN 'json' " + << " WHEN substring(type, 10, length(type) - 10) LIKE 'Tuple%' THEN 'json' " + << " WHEN substring(type, 10, length(type) - 10) LIKE 'IPv%' THEN 'text' " + << " WHEN substring(type, 10, length(type) - 10) IN ('Expression', 'Set', 'Nothing', 'Interval') THEN 'text' " + << " ELSE substring(type, 10, length(type) - 10) " + << " END " + << " ELSE " + << " CASE " + << " WHEN type IN ('UInt8', 'Int8') THEN 'tinyint' " + << " WHEN type IN ('UInt16', 'Int16') THEN 'smallint' " + << " WHEN type IN ('UInt32', 'Int32') THEN 'int' " + << " WHEN type IN ('UInt64', 'Int64', 'UInt128', 'Int128', 'UInt256', 'Int256') THEN 'bigint' " + << " WHEN type = 'Float32' THEN 'float' " + << " WHEN type = 'Float64' THEN 'double' " + << " WHEN type LIKE 'Decimal%' THEN 'decimal' " + << " WHEN type = 'Boolean' THEN 'tinyint' " + << " WHEN type = 'String' THEN 'text' " + << " WHEN type LIKE 'FixedString%' THEN 'text' " + << " WHEN type LIKE 'Date%' THEN 'date' " + << " WHEN type LIKE 'DateTime%' THEN 'datetime' " + << " WHEN type = 'JSON' THEN 'json' " + << " WHEN type = 'UUID' THEN 'binary' " + << " WHEN type LIKE 'Enum%' THEN 'enum' " + << " WHEN type LIKE 'LowCardinality%' THEN 'text' " + << " WHEN type LIKE 'Array%' THEN 'json' " + << " WHEN type LIKE 'Map%' THEN 'json' " + << " WHEN type IN ('SimpleAggregateFunction', 'AggregateFunction') THEN 'text' " + << " WHEN type = 'Nested' THEN 'json' " + << " WHEN type LIKE 'Tuple%' THEN 'json' " + << " WHEN type LIKE 'IPv%' THEN 'text' " + << " WHEN type IN ('Expression', 'Set', 'Nothing', 'Interval') THEN 'text' " + << " ELSE type " + << " END " + << "END AS type, " + << "startsWith(type, 'Nullable') AS null, " + << "trim(concatWithSeparator(' ', if(is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, " + << "if(default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, " + << "'' AS extra "; + + return mysql_specific_query.str(); +} BlockIO InterpreterShowColumnsQuery::execute() { diff --git a/src/Interpreters/InterpreterShowColumnsQuery.h b/src/Interpreters/InterpreterShowColumnsQuery.h index ee6dcabd97b..b843a163978 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.h +++ b/src/Interpreters/InterpreterShowColumnsQuery.h @@ -26,6 +26,7 @@ private: ASTPtr query_ptr; String getRewrittenQuery(); + String getMySQLQuery(); }; diff --git a/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference new file mode 100644 index 00000000000..c9ad94a34c4 --- /dev/null +++ b/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference @@ -0,0 +1,213 @@ +Drop tables if they exist +Create tab table +Create pseudo-random database name +Create tab duplicate table +Run MySQL test +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_int json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uint64 bigint 0 PRI SOR NULL +uuid_value binary 0 NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_int json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uint64 bigint 0 PRI SOR NULL +uuid_value binary 0 NULL +field type null key default extra collation comment privileges +array_value json 0 NULL NULL +boolean_value tinyint 0 NULL NULL +date32_value date 0 NULL NULL +date_value date 0 NULL NULL +datetime64_value date 0 NULL NULL +datetime_value date 0 NULL NULL +decimal_value decimal 0 NULL NULL +enum_value enum 0 NULL NULL +fixed_string_value text 0 NULL NULL +float32 float 0 NULL NULL +float64 double 0 NULL NULL +int32 int 0 NULL NULL +ipv4_value text 0 NULL NULL +ipv6_value text 0 NULL NULL +json_value text 0 NULL NULL +low_cardinality text 0 NULL NULL +map_value json 0 NULL NULL +nested.nested_int json 0 NULL NULL +nested.nested_string json 0 NULL NULL +nullable_value int 0 NULL NULL +string_value text 0 NULL NULL +tuple_value json 0 NULL NULL +uint64 bigint 0 PRI SOR NULL NULL +uuid_value binary 0 NULL NULL +field type null key default extra +int32 int 0 NULL +nested.nested_int json 0 NULL +uint64 bigint 0 PRI SOR NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uuid_value binary 0 NULL +field type null key default extra +int32 int 0 NULL +nested.nested_int json 0 NULL +uint64 bigint 0 PRI SOR NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uuid_value binary 0 NULL +field type null key default extra +int32 int 0 NULL +nested.nested_int json 0 NULL +uint64 bigint 0 PRI SOR NULL +field type null key default extra +array_value json 0 NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_int json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uint64 bigint 0 PRI SOR NULL +uuid_value binary 0 NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_int json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uint64 bigint 0 PRI SOR NULL +uuid_value binary 0 NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_int json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uint64 bigint 0 PRI SOR NULL +uuid_value binary 0 NULL diff --git a/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh b/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh new file mode 100755 index 00000000000..5324496edd3 --- /dev/null +++ b/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh @@ -0,0 +1,115 @@ +#!/bin/bash + +# This script tests the MySQL compatibility of the SHOW COLUMNS command in ClickHouse +USER="default" +PASSWORD="" +HOST="127.0.0.1" +PORT=9004 + +# First run the clickhouse test to create the ClickHouse Tables + +echo "Drop tables if they exist" +${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS tab" +${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde" +${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde.tab" + +echo "Create tab table " +${CLICKHOUSE_LOCAL} --query " + CREATE TABLE tab + ( + uint64 UInt64, + int32 Nullable(Int32), + float32 Float32, + float64 Float64, + decimal_value Decimal(10, 2), + boolean_value UInt8, -- Use 0 for false, 1 for true + string_value String, + fixed_string_value FixedString(10), + date_value Date, + date32_value Date32, + datetime_value DateTime, + datetime64_value DateTime64(3), + json_value String, -- Store JSON as a string + uuid_value UUID, + enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), + low_cardinality LowCardinality(String), + array_value Array(Int32), + map_value Map(String, Int32), + tuple_value Tuple(Int32, String), + nullable_value Nullable(Int32), + ipv4_value IPv4, + ipv6_value IPv6, + nested Nested + ( + nested_int Int32, + nested_string String + ) + ) ENGINE = MergeTree + ORDER BY uint64; + " + + +echo "Create pseudo-random database name" +${CLICKHOUSE_LOCAL} --query "CREATE DATABASE database_123456789abcde;" + +echo "Create tab duplicate table" +${CLICKHOUSE_LOCAL} --query " + CREATE TABLE database_123456789abcde.tab + ( + uint64 UInt64, + int32 Nullable(Int32), + float32 Float32, + float64 Float64, + decimal_value Decimal(10, 2), + boolean_value UInt8, -- Use 0 for false, 1 for true + string_value String, + fixed_string_value FixedString(10), + date_value Date, + date32_value Date32, + datetime_value DateTime, + datetime64_value DateTime64(3), + json_value String, -- Store JSON as a string + uuid_value UUID, + enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), + low_cardinality LowCardinality(String), + array_value Array(Int32), + map_value Map(String, Int32), + tuple_value Tuple(Int32, String), + nullable_value Nullable(Int32), + ipv4_value IPv4, + ipv6_value IPv6, + nested Nested + ( + nested_int Int32, + nested_string String + ) + ) ENGINE = MergeTree + ORDER BY uint64; + " + +# Write sql to temp file +TEMP_FILE=$(mktemp) + +cat < $TEMP_FILE +SHOW COLUMNS FROM tab; +SHOW EXTENDED COLUMNS FROM tab; +SHOW FULL COLUMNS FROM tab; +SHOW COLUMNS FROM tab LIKE '%int%'; +SHOW COLUMNS FROM tab NOT LIKE '%int%'; +SHOW COLUMNS FROM tab ILIKE '%INT%'; +SHOW COLUMNS FROM tab NOT ILIKE '%INT%'; +SHOW COLUMNS FROM tab WHERE field LIKE '%int%'; +SHOW COLUMNS FROM tab LIMIT 1; +SHOW COLUMNS FROM tab; +SHOW COLUMNS FROM tab FROM database_123456789abcde; +SHOW COLUMNS FROM database_123456789abcde.tab; +DROP DATABASE database_123456789abcde; +DROP TABLE tab; +EOT + +# Now run the MySQL test script on the ClickHouse DB +echo "Run MySQL test" +mysql --user="$USER" --password="$PASSWORD" --host="$HOST" --port="$PORT" < $TEMP_FILE + +# Clean up the temp file +rm $TEMP_FILE From b2c36fc3e5a968cd3223261585cb00f89be2c783 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 May 2023 20:51:38 +0000 Subject: [PATCH 0192/2223] Fixing style. --- src/Interpreters/PreparedSets.cpp | 7 ++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 5d9a0f27496..1d7d90432b0 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -11,6 +11,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) { /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, @@ -283,7 +288,7 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } -FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) { std::cerr << block.dumpStructure() << std::endl; } +FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 121970f2ca9..f13f75bfebc 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -232,7 +232,7 @@ private: std::shared_ptr max_block_numbers_to_read; - /// Pre-computed value, needed to trigger sets creatin for PK + /// Pre-computed value, needed to trigger sets creating for PK mutable std::optional key_condition; Poco::Logger * log; From 06b0503fe8e8f37322056941aad301b119f58c2b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 5 May 2023 21:16:14 +0000 Subject: [PATCH 0193/2223] Simple case works --- .../Transforms/FillingTransform.cpp | 163 +++++++++++++++--- ...2730_with_fill_by_sorting_prefix.reference | 23 +++ .../02730_with_fill_by_sorting_prefix.sql | 13 ++ 3 files changed, 179 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference create mode 100644 tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 93dca03e4ac..7e9085afeb1 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -255,9 +255,10 @@ FillingTransform::FillingTransform( if (interpolate_description) if (const auto & p = interpolate_description->required_columns_map.find(column.name); p != interpolate_description->required_columns_map.end()) - input_positions.emplace_back(idx, p->second); + input_positions.emplace_back(idx, p->second); - if (!is_fill_column[idx] && !(interpolate_description && interpolate_description->result_columns_set.contains(column.name))) + if (!is_fill_column[idx] && !(interpolate_description && interpolate_description->result_columns_set.contains(column.name)) + && sort_prefix_positions.end() == std::find(sort_prefix_positions.begin(), sort_prefix_positions.end(), idx)) other_column_positions.push_back(idx); ++idx; @@ -359,6 +360,8 @@ static void insertFromFillingRow(const MutableColumnRawPtrs & filling_columns, c static void copyRowFromColumns(const MutableColumnRawPtrs & dest, const Columns & source, size_t row_num) { + chassert(dest.size() == source.size()); + for (size_t i = 0, size = source.size(); i < size; ++i) dest[i]->insertFrom(*source[i], row_num); } @@ -381,12 +384,12 @@ void FillingTransform::initColumns( const Columns & input_columns, Columns & input_fill_columns, Columns & input_interpolate_columns, - Columns & ,//input_sort_prefix_columns, + Columns & input_sort_prefix_columns, Columns & input_other_columns, MutableColumns & output_columns, MutableColumnRawPtrs & output_fill_columns, MutableColumnRawPtrs & output_interpolate_columns, - MutableColumnRawPtrs & , //output_sort_prefix_columns, + MutableColumnRawPtrs & output_sort_prefix_columns, MutableColumnRawPtrs & output_other_columns) { Columns non_const_columns; @@ -401,7 +404,7 @@ void FillingTransform::initColumns( initColumnsByPositions(non_const_columns, input_fill_columns, output_columns, output_fill_columns, fill_column_positions); initColumnsByPositions( non_const_columns, input_interpolate_columns, output_columns, output_interpolate_columns, interpolate_column_positions); - // initColumnsByPositions(non_const_columns, input_sort_prefix_columns, output_columns, output_sort_prefix_columns, sort_prefix_positions); + initColumnsByPositions(non_const_columns, input_sort_prefix_columns, output_columns, output_sort_prefix_columns, sort_prefix_positions); initColumnsByPositions(non_const_columns, input_other_columns, output_columns, output_other_columns, other_column_positions); } @@ -449,31 +452,84 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut if (first) filling_row.initFromDefaults(); + /// if any rows was processed and there is sort prefix, get last row sort prefix + Columns last_row_sort_prefix; + if (!last_row.empty()) + { + last_row_sort_prefix.reserve(sort_prefix_positions.size()); + for (const size_t pos : sort_prefix_positions) + last_row_sort_prefix.push_back(last_row[pos]); + } + Block interpolate_block; if (should_insert_first && filling_row < next_row) { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + /// fullfill sort prefix columns with last row values or defaults + if (!last_row_sort_prefix.empty()) + copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); + else + for (auto * sort_prefix_column : res_sort_prefix_columns) + sort_prefix_column->insertDefault(); } while (filling_row.next(next_row)) { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + /// fullfill sort prefix columns with last row values or defaults + if (!last_row_sort_prefix.empty()) + copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); + else + for (auto * sort_prefix_column : res_sort_prefix_columns) + sort_prefix_column->insertDefault(); } return true; } +template +size_t getRangeEnd(size_t begin, size_t end, Predicate pred) +{ + chassert(begin < end); + + const size_t linear_probe_threadhold = 16; + size_t linear_probe_end = begin + linear_probe_threadhold; + if (linear_probe_end > end) + linear_probe_end = end; + + for (size_t pos = begin; pos < linear_probe_end; ++pos) + { + if (!pred(begin, pos)) + return pos; + } + + size_t low = linear_probe_end; + size_t high = end - 1; + while (low <= high) + { + size_t mid = low + (high - low) / 2; + if (pred(begin, mid)) + low = mid + 1; + else + { + high = mid - 1; + end = mid; + } + } + return end; +} + void FillingTransform::transformRange( const Columns & input_fill_columns, const Columns & input_interpolate_columns, - const Columns &, //input_sort_prefix_columns, + const Columns & input_sort_prefix_columns, const Columns & input_other_columns, const MutableColumns & result_columns, const MutableColumnRawPtrs & res_fill_columns, const MutableColumnRawPtrs & res_interpolate_columns, - const MutableColumnRawPtrs &, //res_sort_prefix_columns, + const MutableColumnRawPtrs & res_sort_prefix_columns, const MutableColumnRawPtrs & res_other_columns, std::pair range) { @@ -485,7 +541,7 @@ void FillingTransform::transformRange( { for (size_t i = 0, size = filling_row.size(); i < size; ++i) { - auto current_value = (*input_fill_columns[i])[0]; + auto current_value = (*input_fill_columns[i])[range_begin]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; if (!fill_from.isNull() && !equals(current_value, fill_from)) @@ -495,6 +551,7 @@ void FillingTransform::transformRange( { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, range_begin); } break; } @@ -530,17 +587,19 @@ void FillingTransform::transformRange( { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } while (filling_row.next(next_row)) { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); - // copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); copyRowFromColumns(res_other_columns, input_other_columns, row_ind); } } @@ -599,17 +658,81 @@ void FillingTransform::transform(Chunk & chunk) res_sort_prefix_columns, res_other_columns); - transformRange( - input_fill_columns, - input_interpolate_columns, - input_sort_prefix_columns, - input_other_columns, - result_columns, - res_fill_columns, - res_interpolate_columns, - res_sort_prefix_columns, - res_other_columns, - {0, num_rows}); + if (sort_prefix.empty()) + { + transformRange( + input_fill_columns, + input_interpolate_columns, + input_sort_prefix_columns, + input_other_columns, + result_columns, + res_fill_columns, + res_interpolate_columns, + res_sort_prefix_columns, + res_other_columns, + {0, num_rows}); + + saveLastRow(result_columns); + size_t num_output_rows = result_columns[0]->size(); + chunk.setColumns(std::move(result_columns), num_output_rows); + return; + } + + /// check if last row in prev chunk had the same sorting prefix as the first in new one + /// if not, we need to reinitialize filling row + if (!last_row.empty()) + { + ColumnRawPtrs last_sort_prefix_columns; + last_sort_prefix_columns.reserve(sort_prefix.size()); + for (size_t pos : sort_prefix_positions) + last_sort_prefix_columns.push_back(last_row[pos].get()); + + first = false; + for (size_t i = 0; i < input_sort_prefix_columns.size(); ++i) + { + const int res = input_sort_prefix_columns[i]->compareAt(0, 0, *last_sort_prefix_columns[i], sort_prefix[i].nulls_direction); + if (res != 0) + { + first = true; + break; + } + } + } + + for (size_t row_ind = 0; row_ind < num_rows;) + { + /// find next range + auto current_sort_prefix_end_pos = getRangeEnd( + row_ind, + num_rows, + [&](size_t pos_with_current_sort_prefix, size_t row_pos) + { + for (size_t i = 0; i < input_sort_prefix_columns.size(); ++i) + { + const int res = input_sort_prefix_columns[i]->compareAt( + pos_with_current_sort_prefix, row_pos, *input_sort_prefix_columns[i], sort_prefix[i].nulls_direction); + if (res != 0) + return false; + } + return true; + }); + + transformRange( + input_fill_columns, + input_interpolate_columns, + input_sort_prefix_columns, + input_other_columns, + result_columns, + res_fill_columns, + res_interpolate_columns, + res_sort_prefix_columns, + res_other_columns, + {row_ind, current_sort_prefix_end_pos}); + + logDebug("range end", current_sort_prefix_end_pos); + row_ind = current_sort_prefix_end_pos; + first = true; + } saveLastRow(result_columns); size_t num_output_rows = result_columns[0]->size(); diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference new file mode 100644 index 00000000000..121835a80e8 --- /dev/null +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -0,0 +1,23 @@ +-- { echoOn } +-- corner case with constant sort prefix +SELECT number +FROM numbers(1) +ORDER BY 10 ASC, number DESC WITH FILL FROM 1 +SETTINGS enable_positional_arguments=0; +1 +0 +-- sensor table +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +insert into ts VALUES (1, 10, 1), (1, 12, 2), (3, 5, 1), (3, 7, 3), (5, 1, 1), (5, 3, 1); +-- FillingTransform: 6 rows will be processed in 1 chunks +select * from ts order by sensor_id, timestamp with fill step 1; +1 10 1 +1 11 0 +1 12 2 +3 5 1 +3 6 0 +3 7 3 +5 1 1 +5 2 0 +5 3 1 diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql new file mode 100644 index 00000000000..828572eb620 --- /dev/null +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -0,0 +1,13 @@ +-- { echoOn } +-- corner case with constant sort prefix +SELECT number +FROM numbers(1) +ORDER BY 10 ASC, number DESC WITH FILL FROM 1 +SETTINGS enable_positional_arguments=0; + +-- sensor table +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +insert into ts VALUES (1, 10, 1), (1, 12, 2), (3, 5, 1), (3, 7, 3), (5, 1, 1), (5, 3, 1); +-- FillingTransform: 6 rows will be processed in 1 chunks +select * from ts order by sensor_id, timestamp with fill step 1; From 690ebb9d6bdc3a97eb00f6538ece8503108d9d1c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 6 May 2023 11:33:53 +0000 Subject: [PATCH 0194/2223] Fast fix: correct suffix generation --- src/Processors/Transforms/FillingTransform.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 7e9085afeb1..6bd4557e74b 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -414,6 +414,7 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut logDebug("generateSuffixIfNeeded() next_row", next_row); logDebug("generateSuffixIfNeeded() first", first); + first = last_row.empty(); /// Determines should we insert filling row before start generating next rows. bool should_insert_first = next_row < filling_row || first; @@ -588,6 +589,7 @@ void FillingTransform::transformRange( interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + logDebug("filling_row should_insert_first", filling_row); } while (filling_row.next(next_row)) @@ -595,6 +597,7 @@ void FillingTransform::transformRange( interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + logDebug("filling_row", filling_row); } copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); From 80bf023382089d30112287e665125fa0dfd7face Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 6 May 2023 12:18:20 +0000 Subject: [PATCH 0195/2223] Fix typos --- src/Processors/Transforms/FillingTransform.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 6bd4557e74b..27090a4078c 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -467,7 +467,7 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - /// fullfill sort prefix columns with last row values or defaults + /// fulfill sort prefix columns with last row values or defaults if (!last_row_sort_prefix.empty()) copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); else @@ -479,7 +479,7 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - /// fullfill sort prefix columns with last row values or defaults + /// fulfill sort prefix columns with last row values or defaults if (!last_row_sort_prefix.empty()) copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); else From 5200a31da33a29814a57bac728f7e65596a4d147 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 6 May 2023 20:07:25 +0000 Subject: [PATCH 0196/2223] Check if processing range with new sort prefix differently --- .../Transforms/FillingTransform.cpp | 32 +++++++++++-------- src/Processors/Transforms/FillingTransform.h | 4 +-- 2 files changed, 21 insertions(+), 15 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 27090a4078c..29bc5367774 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -410,20 +410,22 @@ void FillingTransform::initColumns( bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, MutableColumns & result_columns) { + /// true means we'll generate rows for empty result set + const bool no_data_processed = last_row.empty(); + logDebug("generateSuffixIfNeeded() filling_row", filling_row); logDebug("generateSuffixIfNeeded() next_row", next_row); - logDebug("generateSuffixIfNeeded() first", first); + logDebug("generateSuffixIfNeeded() no_data_processed", no_data_processed); - first = last_row.empty(); /// Determines should we insert filling row before start generating next rows. - bool should_insert_first = next_row < filling_row || first; + bool should_insert_first = next_row < filling_row || no_data_processed; for (size_t i = 0, size = filling_row.size(); i < size; ++i) next_row[i] = filling_row.getFillDescription(i).fill_to; logDebug("generateSuffixIfNeeded() next_row updated", next_row); - if (!first && filling_row >= next_row) + if (!no_data_processed && filling_row >= next_row) { logDebug("generateSuffixIfNeeded()", "no need to generate suffix"); return false; @@ -450,7 +452,7 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut res_sort_prefix_columns, res_other_columns); - if (first) + if (no_data_processed) filling_row.initFromDefaults(); /// if any rows was processed and there is sort prefix, get last row sort prefix @@ -532,13 +534,14 @@ void FillingTransform::transformRange( const MutableColumnRawPtrs & res_interpolate_columns, const MutableColumnRawPtrs & res_sort_prefix_columns, const MutableColumnRawPtrs & res_other_columns, - std::pair range) + std::pair range, + const bool new_sorting_prefix) { const size_t range_begin = range.first; const size_t range_end = range.second; Block interpolate_block; - if (first) + if (new_sorting_prefix) { for (size_t i = 0, size = filling_row.size(); i < size; ++i) { @@ -558,7 +561,6 @@ void FillingTransform::transformRange( } filling_row[i] = current_value; } - first = false; } for (size_t row_ind = range_begin; row_ind < range_end; ++row_ind) @@ -663,6 +665,7 @@ void FillingTransform::transform(Chunk & chunk) if (sort_prefix.empty()) { + constexpr bool new_sort_prefix = true; transformRange( input_fill_columns, input_interpolate_columns, @@ -673,7 +676,8 @@ void FillingTransform::transform(Chunk & chunk) res_interpolate_columns, res_sort_prefix_columns, res_other_columns, - {0, num_rows}); + {0, num_rows}, + new_sort_prefix); saveLastRow(result_columns); size_t num_output_rows = result_columns[0]->size(); @@ -683,6 +687,7 @@ void FillingTransform::transform(Chunk & chunk) /// check if last row in prev chunk had the same sorting prefix as the first in new one /// if not, we need to reinitialize filling row + bool new_sort_prefix = last_row.empty(); if (!last_row.empty()) { ColumnRawPtrs last_sort_prefix_columns; @@ -690,13 +695,13 @@ void FillingTransform::transform(Chunk & chunk) for (size_t pos : sort_prefix_positions) last_sort_prefix_columns.push_back(last_row[pos].get()); - first = false; + new_sort_prefix = false; for (size_t i = 0; i < input_sort_prefix_columns.size(); ++i) { const int res = input_sort_prefix_columns[i]->compareAt(0, 0, *last_sort_prefix_columns[i], sort_prefix[i].nulls_direction); if (res != 0) { - first = true; + new_sort_prefix = true; break; } } @@ -730,11 +735,12 @@ void FillingTransform::transform(Chunk & chunk) res_interpolate_columns, res_sort_prefix_columns, res_other_columns, - {row_ind, current_sort_prefix_end_pos}); + {row_ind, current_sort_prefix_end_pos}, + new_sort_prefix); logDebug("range end", current_sort_prefix_end_pos); row_ind = current_sort_prefix_end_pos; - first = true; + new_sort_prefix = true; } saveLastRow(result_columns); diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index dac92e43231..0506bb92efb 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -43,7 +43,8 @@ private: const MutableColumnRawPtrs & res_interpolate_columns, const MutableColumnRawPtrs & res_sort_prefix_columns, const MutableColumnRawPtrs & res_other_columns, - std::pair range); + std::pair range, + bool new_sorting_prefix); void saveLastRow(const MutableColumns & result_columns); void interpolate(const MutableColumns & result_columns, Block & interpolate_block); @@ -80,7 +81,6 @@ private: std::vector> input_positions; /// positions in result columns required for actions ExpressionActionsPtr interpolate_actions; Columns last_row; - bool first = true; /// flag to determine if transform is/will be called for the first time bool all_chunks_processed = false; /// flag to determine if we have already processed all chunks }; From 0dfeb880ceb04e6a8e4043baaa1745b3308ff76d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 7 May 2023 08:08:02 +0000 Subject: [PATCH 0197/2223] Fix 01034_with_fill_and_push_down_predicate.sql --- src/Processors/Transforms/FillingTransform.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 29bc5367774..b9c3341948a 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -665,7 +665,6 @@ void FillingTransform::transform(Chunk & chunk) if (sort_prefix.empty()) { - constexpr bool new_sort_prefix = true; transformRange( input_fill_columns, input_interpolate_columns, @@ -677,7 +676,7 @@ void FillingTransform::transform(Chunk & chunk) res_sort_prefix_columns, res_other_columns, {0, num_rows}, - new_sort_prefix); + last_row.empty()); saveLastRow(result_columns); size_t num_output_rows = result_columns[0]->size(); From 29bbde85a6839f34d517341b207ea43587378031 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 7 May 2023 22:06:25 +0200 Subject: [PATCH 0198/2223] split selectPartsToMerge into smaller functions --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 250 +++++++++++------- .../MergeTree/MergeTreeDataMergerMutator.h | 37 +++ 2 files changed, 189 insertions(+), 98 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 8300abd51bd..e96e683443c 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -138,66 +138,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( const MergeTreeTransactionPtr & txn, String * out_disable_reason) { - MergeTreeData::DataPartsVector data_parts; - if (txn) - { - /// Merge predicate (for simple MergeTree) allows to merge two parts only if both parts are visible for merge transaction. - /// So at the first glance we could just get all active parts. - /// Active parts include uncommitted parts, but it's ok and merge predicate handles it. - /// However, it's possible that some transaction is trying to remove a part in the middle, for example, all_2_2_0. - /// If parts all_1_1_0 and all_3_3_0 are active and visible for merge transaction, then we would try to merge them. - /// But it's wrong, because all_2_2_0 may become active again if transaction will roll back. - /// That's why we must include some outdated parts into `data_part`, more precisely, such parts that removal is not committed. - MergeTreeData::DataPartsVector active_parts; - MergeTreeData::DataPartsVector outdated_parts; + MergeTreeData::DataPartsVector data_parts = getDataPartsToSelectMergeFrom(txn); - { - auto lock = data.lockParts(); - active_parts = data.getDataPartsVectorForInternalUsage({MergeTreeData::DataPartState::Active}, lock); - outdated_parts = data.getDataPartsVectorForInternalUsage({MergeTreeData::DataPartState::Outdated}, lock); - } - - ActiveDataPartSet active_parts_set{data.format_version}; - for (const auto & part : active_parts) - active_parts_set.add(part->name); - - for (const auto & part : outdated_parts) - { - /// We don't need rolled back parts. - /// NOTE When rolling back a transaction we set creation_csn to RolledBackCSN at first - /// and then remove part from working set, so there's no race condition - if (part->version.creation_csn == Tx::RolledBackCSN) - continue; - - /// We don't need parts that are finally removed. - /// NOTE There's a minor race condition: we may get UnknownCSN if a transaction has been just committed concurrently. - /// But it's not a problem if we will add such part to `data_parts`. - if (part->version.removal_csn != Tx::UnknownCSN) - continue; - - active_parts_set.add(part->name); - } - - /// Restore "active" parts set from selected active and outdated parts - auto remove_pred = [&](const MergeTreeData::DataPartPtr & part) -> bool - { - return active_parts_set.getContainingPart(part->info) != part->name; - }; - - std::erase_if(active_parts, remove_pred); - - std::erase_if(outdated_parts, remove_pred); - - std::merge(active_parts.begin(), active_parts.end(), - outdated_parts.begin(), outdated_parts.end(), - std::back_inserter(data_parts), MergeTreeData::LessDataPart()); - } - else - { - /// Simply get all active parts - data_parts = data.getDataPartsVectorForInternalUsage(); - } - const auto data_settings = data.getSettings(); auto metadata_snapshot = data.getInMemoryMetadataPtr(); if (data_parts.empty()) @@ -207,9 +149,118 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( return SelectPartsDecision::CANNOT_SELECT; } - time_t current_time = std::time(nullptr); + MergeSelectingInfo info = getPossibleMergeRanges(data_parts, can_merge_callback, txn, out_disable_reason); - IMergeSelector::PartsRanges parts_ranges; + if (info.parts_selected_precondition == 0) + { + if (out_disable_reason) + *out_disable_reason = "No parts satisfy preconditions for merge"; + return SelectPartsDecision::CANNOT_SELECT; + } + + auto res = selectPartsToMergeFromRanges(future_part, aggressive, max_total_size_to_merge, merge_with_ttl_allowed, + metadata_snapshot, info, out_disable_reason); + + if (res == SelectPartsDecision::SELECTED) + return res; + + String best_partition_id_to_optimize = getBestPartitionToOptimizeEntire(info.partitions_info); + if (!best_partition_id_to_optimize.empty()) + { + return selectAllPartsToMergeWithinPartition( + future_part, + can_merge_callback, + best_partition_id_to_optimize, + /*final=*/true, + metadata_snapshot, + txn, + out_disable_reason, + /*optimize_skip_merged_partitions=*/true); + } + + if (out_disable_reason) + *out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; + return SelectPartsDecision::CANNOT_SELECT; +} + +MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::getDataPartsToSelectMergeFrom(const MergeTreeTransactionPtr & txn) const +{ + if (!txn) + { + /// Simply get all active parts + return data.getDataPartsVectorForInternalUsage(); + } + + /// Merge predicate (for simple MergeTree) allows to merge two parts only if both parts are visible for merge transaction. + /// So at the first glance we could just get all active parts. + /// Active parts include uncommitted parts, but it's ok and merge predicate handles it. + /// However, it's possible that some transaction is trying to remove a part in the middle, for example, all_2_2_0. + /// If parts all_1_1_0 and all_3_3_0 are active and visible for merge transaction, then we would try to merge them. + /// But it's wrong, because all_2_2_0 may become active again if transaction will roll back. + /// That's why we must include some outdated parts into `data_part`, more precisely, such parts that removal is not committed. + MergeTreeData::DataPartsVector active_parts; + MergeTreeData::DataPartsVector outdated_parts; + + { + auto lock = data.lockParts(); + active_parts = data.getDataPartsVectorForInternalUsage({MergeTreeData::DataPartState::Active}, lock); + outdated_parts = data.getDataPartsVectorForInternalUsage({MergeTreeData::DataPartState::Outdated}, lock); + } + + ActiveDataPartSet active_parts_set{data.format_version}; + for (const auto & part : active_parts) + active_parts_set.add(part->name); + + for (const auto & part : outdated_parts) + { + /// We don't need rolled back parts. + /// NOTE When rolling back a transaction we set creation_csn to RolledBackCSN at first + /// and then remove part from working set, so there's no race condition + if (part->version.creation_csn == Tx::RolledBackCSN) + continue; + + /// We don't need parts that are finally removed. + /// NOTE There's a minor race condition: we may get UnknownCSN if a transaction has been just committed concurrently. + /// But it's not a problem if we will add such part to `data_parts`. + if (part->version.removal_csn != Tx::UnknownCSN) + continue; + + active_parts_set.add(part->name); + } + + /// Restore "active" parts set from selected active and outdated parts + auto remove_pred = [&](const MergeTreeData::DataPartPtr & part) -> bool + { + return active_parts_set.getContainingPart(part->info) != part->name; + }; + + std::erase_if(active_parts, remove_pred); + + std::erase_if(outdated_parts, remove_pred); + + MergeTreeData::DataPartsVector data_parts; + std::merge( + active_parts.begin(), + active_parts.end(), + outdated_parts.begin(), + outdated_parts.end(), + std::back_inserter(data_parts), + MergeTreeData::LessDataPart()); + + return data_parts; +} + +MergeTreeDataMergerMutator::MergeSelectingInfo MergeTreeDataMergerMutator::getPossibleMergeRanges( + const MergeTreeData::DataPartsVector & data_parts, + const AllowedMergingPredicate & can_merge_callback, + const MergeTreeTransactionPtr & txn, + String * out_disable_reason) const +{ + MergeSelectingInfo res; + + res.current_time = std::time(nullptr); + + IMergeSelector::PartsRanges & parts_ranges = res.parts_ranges; StoragePolicyPtr storage_policy = data.getStoragePolicy(); /// Volumes with stopped merges are extremely rare situation. @@ -221,14 +272,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( const MergeTreeData::DataPartPtr * prev_part = nullptr; /// collect min_age for each partition while iterating parts - struct PartitionInfo - { - time_t min_age{std::numeric_limits::max()}; - }; + PartitionsInfo & partitions_info = res.partitions_info; - std::unordered_map partitions_info; - - size_t parts_selected_precondition = 0; for (const MergeTreeData::DataPartPtr & part : data_parts) { const String & partition_id = part->info.partition_id; @@ -284,7 +329,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( IMergeSelector::Part part_info; part_info.size = part->getBytesOnDisk(); - part_info.age = current_time - part->modification_time; + part_info.age = res.current_time - part->modification_time; part_info.level = part->info.level; part_info.data = ∂ part_info.ttl_infos = &part->ttl_infos; @@ -294,7 +339,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( auto & partition_info = partitions_info[partition_id]; partition_info.min_age = std::min(partition_info.min_age, part_info.age); - ++parts_selected_precondition; + ++res.parts_selected_precondition; parts_ranges.back().emplace_back(part_info); @@ -311,13 +356,21 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( prev_part = ∂ } - if (parts_selected_precondition == 0) - { - if (out_disable_reason) - *out_disable_reason = "No parts satisfy preconditions for merge"; - return SelectPartsDecision::CANNOT_SELECT; - } + return res; +} +SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( + FutureMergedMutatedPartPtr future_part, + bool aggressive, + size_t max_total_size_to_merge, + bool merge_with_ttl_allowed, + const StorageMetadataPtr & metadata_snapshot, + const MergeSelectingInfo & info, + String * out_disable_reason) +{ + const time_t & current_time = info.current_time; + const IMergeSelector::PartsRanges & parts_ranges = info.parts_ranges; + const auto data_settings = data.getSettings(); IMergeSelector::PartsRange parts_to_merge; if (metadata_snapshot->hasAnyTTL() && merge_with_ttl_allowed && !ttl_merges_blocker.isCancelled()) @@ -382,29 +435,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( if (parts_to_merge.empty()) { - if (data_settings->min_age_to_force_merge_on_partition_only && data_settings->min_age_to_force_merge_seconds) - { - auto best_partition_it = std::max_element( - partitions_info.begin(), - partitions_info.end(), - [](const auto & e1, const auto & e2) { return e1.second.min_age < e2.second.min_age; }); - - assert(best_partition_it != partitions_info.end()); - - if (static_cast(best_partition_it->second.min_age) >= data_settings->min_age_to_force_merge_seconds) - return selectAllPartsToMergeWithinPartition( - future_part, - can_merge_callback, - best_partition_it->first, - /*final=*/true, - metadata_snapshot, - txn, - out_disable_reason, - /*optimize_skip_merged_partitions=*/true); - } - if (out_disable_reason) - *out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; + *out_disable_reason = "Did not find any parts to merge (with usual merge selectors)"; return SelectPartsDecision::CANNOT_SELECT; } } @@ -422,6 +454,28 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( return SelectPartsDecision::SELECTED; } +String MergeTreeDataMergerMutator::getBestPartitionToOptimizeEntire( + const PartitionsInfo & partitions_info) const +{ + const auto data_settings = data.getSettings(); + if (!data_settings->min_age_to_force_merge_on_partition_only) + return {}; + if (!data_settings->min_age_to_force_merge_seconds) + return {}; + + auto best_partition_it = std::max_element( + partitions_info.begin(), + partitions_info.end(), + [](const auto & e1, const auto & e2) { return e1.second.min_age < e2.second.min_age; }); + + assert(best_partition_it != partitions_info.end()); + + if (static_cast(best_partition_it->second.min_age) < data_settings->min_age_to_force_merge_seconds) + return {}; + + return best_partition_it->first; +} + SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( FutureMergedMutatedPartPtr future_part, const AllowedMergingPredicate & can_merge, diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 7c96fcfaeb3..6935f3452b8 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -62,6 +62,43 @@ public: */ UInt64 getMaxSourcePartSizeForMutation() const; + struct PartitionInfo + { + time_t min_age{std::numeric_limits::max()}; + }; + using PartitionsInfo = std::unordered_map; + + /// The first step of selecting parts to merge: returns a list of all active/visible parts + MergeTreeData::DataPartsVector getDataPartsToSelectMergeFrom(const MergeTreeTransactionPtr & txn) const; + + struct MergeSelectingInfo + { + time_t current_time; + PartitionsInfo partitions_info; + IMergeSelector::PartsRanges parts_ranges; + size_t parts_selected_precondition = 0; + }; + + /// The second step of selecting parts to merge: splits parts list into a set of ranges according to can_merge_callback. + /// All parts withing a range can be merged without violating some invariants. + MergeSelectingInfo getPossibleMergeRanges( + const MergeTreeData::DataPartsVector & data_parts, + const AllowedMergingPredicate & can_merge_callback, + const MergeTreeTransactionPtr & txn, + String * out_disable_reason = nullptr) const; + + /// The third step of selecting parts to merge: takes ranges that we can merge, and selects parts that we want to merge + SelectPartsDecision selectPartsToMergeFromRanges( + FutureMergedMutatedPartPtr future_part, + bool aggressive, + size_t max_total_size_to_merge, + bool merge_with_ttl_allowed, + const StorageMetadataPtr & metadata_snapshot, + const MergeSelectingInfo & info, + String * out_disable_reason = nullptr); + + String getBestPartitionToOptimizeEntire(const PartitionsInfo & partitions_info) const; + /** Selects which parts to merge. Uses a lot of heuristics. * * can_merge - a function that determines if it is possible to merge a pair of adjacent parts. From 91489243c92dbe063f10ce21cf27d062e293d4ec Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 8 May 2023 00:22:07 +0200 Subject: [PATCH 0199/2223] provide better partitions hint for merge selecting task --- src/Storages/MergeTree/MergeSelector.h | 8 ++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 102 ++++++++++++++++-- .../MergeTree/MergeTreeDataMergerMutator.h | 23 +++- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 49 +++++++-- .../MergeTree/ReplicatedMergeTreeQueue.h | 33 +++++- src/Storages/MergeTree/TTLMergeSelector.cpp | 9 +- src/Storages/MergeTree/TTLMergeSelector.h | 14 ++- src/Storages/StorageReplicatedMergeTree.cpp | 30 ++++-- 8 files changed, 230 insertions(+), 38 deletions(-) diff --git a/src/Storages/MergeTree/MergeSelector.h b/src/Storages/MergeTree/MergeSelector.h index c55f738f879..2f17e1e9654 100644 --- a/src/Storages/MergeTree/MergeSelector.h +++ b/src/Storages/MergeTree/MergeSelector.h @@ -11,6 +11,8 @@ namespace DB { +class IMergeTreeDataPart; + /** Interface of algorithm to select data parts to merge * (merge is also known as "compaction"). * Following properties depend on it: @@ -26,6 +28,7 @@ namespace DB */ class IMergeSelector { + using DataPartPtr = std::shared_ptr; public: /// Information about data part relevant to merge selecting strategy. struct Part @@ -50,6 +53,11 @@ public: ASTPtr compression_codec_desc; bool shall_participate_in_merges = true; + + const DataPartPtr & getDataPartPtr() const + { + return *static_cast(data); + } }; /// Parts are belong to partitions. Only parts within same partition could be merged. diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index e96e683443c..029ee528276 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -136,9 +136,10 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( const AllowedMergingPredicate & can_merge_callback, bool merge_with_ttl_allowed, const MergeTreeTransactionPtr & txn, - String * out_disable_reason) + String * out_disable_reason, + const PartitionIdsHint * partitions_hint) { - MergeTreeData::DataPartsVector data_parts = getDataPartsToSelectMergeFrom(txn); + MergeTreeData::DataPartsVector data_parts = getDataPartsToSelectMergeFrom(txn, partitions_hint); auto metadata_snapshot = data.getInMemoryMetadataPtr(); @@ -159,7 +160,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( } auto res = selectPartsToMergeFromRanges(future_part, aggressive, max_total_size_to_merge, merge_with_ttl_allowed, - metadata_snapshot, info, out_disable_reason); + metadata_snapshot, info.parts_ranges, info.current_time, out_disable_reason); if (res == SelectPartsDecision::SELECTED) return res; @@ -183,8 +184,84 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( return SelectPartsDecision::CANNOT_SELECT; } +MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPartitionsThatMayBeMerged( + size_t max_total_size_to_merge, + const AllowedMergingPredicate & can_merge_callback, + bool merge_with_ttl_allowed, + const MergeTreeTransactionPtr & txn) const +{ + PartitionIdsHint res; + MergeTreeData::DataPartsVector data_parts = getDataPartsToSelectMergeFrom(txn); + if (data_parts.empty()) + return res; + + auto metadata_snapshot = data.getInMemoryMetadataPtr(); + + MergeSelectingInfo info = getPossibleMergeRanges(data_parts, can_merge_callback, txn); + + if (info.parts_selected_precondition == 0) + return res; + + Strings all_partition_ids; + std::vector ranges_per_partition; + + String curr_partition; + for (auto & range : info.parts_ranges) + { + if (range.empty()) + continue; + const String & partition_id = range.front().getDataPartPtr()->info.partition_id; + if (partition_id != curr_partition) + { + curr_partition = partition_id; + all_partition_ids.push_back(curr_partition); + ranges_per_partition.emplace_back(); + } + ranges_per_partition.back().emplace_back(std::move(range)); + } + + for (size_t i = 0; i < all_partition_ids.size(); ++i) + { + auto future_part = std::make_shared(); + String out_disable_reason; + /// This method should have been const, but something went wrong... it's const with dry_run = true + auto status = const_cast(this)->selectPartsToMergeFromRanges( + future_part, /*aggressive*/ false, max_total_size_to_merge, merge_with_ttl_allowed, + metadata_snapshot, ranges_per_partition[i], info.current_time, &out_disable_reason, + /* dry_run */ true); + if (status == SelectPartsDecision::SELECTED) + res.insert(all_partition_ids[i]); + else + LOG_TEST(log, "Nothing to merge in partition {}: {}", all_partition_ids[i], out_disable_reason); + } + + String best_partition_id_to_optimize = getBestPartitionToOptimizeEntire(info.partitions_info); + if (!best_partition_id_to_optimize.empty()) + res.emplace(std::move(best_partition_id_to_optimize)); + + LOG_TRACE(log, "Checked {} partitions, found {} partitions with parts that may be merged: {}", + all_partition_ids.size(), res.size(), fmt::join(res, ", ")); + return res; +} + +MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::getDataPartsToSelectMergeFrom( + const MergeTreeTransactionPtr & txn, const PartitionIdsHint * partitions_hint) const +{ + auto res = getDataPartsToSelectMergeFrom(txn); + if (!partitions_hint) + return res; + + auto new_end_it = std::remove_if(res.begin(), res.end(), [partitions_hint](const auto & part) + { + return !partitions_hint->contains(part->info.partition_id); + }); + res.erase(new_end_it, res.end()); + return res; +} + MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::getDataPartsToSelectMergeFrom(const MergeTreeTransactionPtr & txn) const { + MergeTreeData::DataPartsVector res; if (!txn) { /// Simply get all active parts @@ -365,11 +442,11 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( size_t max_total_size_to_merge, bool merge_with_ttl_allowed, const StorageMetadataPtr & metadata_snapshot, - const MergeSelectingInfo & info, - String * out_disable_reason) + const IMergeSelector::PartsRanges & parts_ranges, + const time_t & current_time, + String * out_disable_reason, + bool dry_run) { - const time_t & current_time = info.current_time; - const IMergeSelector::PartsRanges & parts_ranges = info.parts_ranges; const auto data_settings = data.getSettings(); IMergeSelector::PartsRange parts_to_merge; @@ -380,7 +457,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( next_delete_ttl_merge_times_by_partition, current_time, data_settings->merge_with_ttl_timeout, - true); + /*only_drop_parts*/ true, + dry_run); /// The size of the completely expired part of TTL drop is not affected by the merge pressure and the size of the storage space parts_to_merge = drop_ttl_selector.select(parts_ranges, data_settings->max_bytes_to_merge_at_max_space_in_pool); @@ -394,7 +472,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( next_delete_ttl_merge_times_by_partition, current_time, data_settings->merge_with_ttl_timeout, - false); + /*only_drop_parts*/ false, + dry_run); parts_to_merge = delete_ttl_selector.select(parts_ranges, max_total_size_to_merge); if (!parts_to_merge.empty()) @@ -407,7 +486,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( next_recompress_ttl_merge_times_by_partition, current_time, data_settings->merge_with_recompression_ttl_timeout, - metadata_snapshot->getRecompressionTTLs()); + metadata_snapshot->getRecompressionTTLs(), + dry_run); parts_to_merge = recompress_ttl_selector.select(parts_ranges, max_total_size_to_merge); if (!parts_to_merge.empty()) @@ -445,7 +525,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( parts.reserve(parts_to_merge.size()); for (IMergeSelector::Part & part_info : parts_to_merge) { - const MergeTreeData::DataPartPtr & part = *static_cast(part_info.data); + const MergeTreeData::DataPartPtr & part = part_info.getDataPartPtr(); parts.push_back(part); } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 6935f3452b8..1c73bb23d33 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -68,9 +68,16 @@ public: }; using PartitionsInfo = std::unordered_map; + using PartitionIdsHint = std::unordered_set; + /// The first step of selecting parts to merge: returns a list of all active/visible parts MergeTreeData::DataPartsVector getDataPartsToSelectMergeFrom(const MergeTreeTransactionPtr & txn) const; + /// Same as above, but filters partitions according to partitions_hint + MergeTreeData::DataPartsVector getDataPartsToSelectMergeFrom( + const MergeTreeTransactionPtr & txn, + const PartitionIdsHint * partitions_hint) const; + struct MergeSelectingInfo { time_t current_time; @@ -94,11 +101,20 @@ public: size_t max_total_size_to_merge, bool merge_with_ttl_allowed, const StorageMetadataPtr & metadata_snapshot, - const MergeSelectingInfo & info, - String * out_disable_reason = nullptr); + const IMergeSelector::PartsRanges & parts_ranges, + const time_t & current_time, + String * out_disable_reason = nullptr, + bool dry_run = false); String getBestPartitionToOptimizeEntire(const PartitionsInfo & partitions_info) const; + /// Useful to quickly get a list of partitions that contain parts that we may want to merge + PartitionIdsHint getPartitionsThatMayBeMerged( + size_t max_total_size_to_merge, + const AllowedMergingPredicate & can_merge_callback, + bool merge_with_ttl_allowed, + const MergeTreeTransactionPtr & txn) const; + /** Selects which parts to merge. Uses a lot of heuristics. * * can_merge - a function that determines if it is possible to merge a pair of adjacent parts. @@ -113,7 +129,8 @@ public: const AllowedMergingPredicate & can_merge, bool merge_with_ttl_allowed, const MergeTreeTransactionPtr & txn, - String * out_disable_reason = nullptr); + String * out_disable_reason = nullptr, + const PartitionIdsHint * partitions_hint = nullptr); /** Select all the parts in the specified partition for merge, if possible. * final - choose to merge even a single part - that is, allow to merge one part "with itself", diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 1762c7aabe9..067d2a87eb3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1758,7 +1758,8 @@ size_t ReplicatedMergeTreeQueue::countUnfinishedMutations() const } -ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, PartitionIdsHint && partition_ids_hint) +ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, + std::optional && partition_ids_hint) { return ReplicatedMergeTreeMergePredicate(*this, zookeeper, std::move(partition_ids_hint)); } @@ -2094,9 +2095,15 @@ ReplicatedMergeTreeQueue::QueueLocks ReplicatedMergeTreeQueue::lockQueue() return QueueLocks(state_mutex, pull_logs_to_queue_mutex, update_mutations_mutex); } -ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( - ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, PartitionIdsHint && partition_ids_hint_) +TrivialMergePredicate::TrivialMergePredicate(ReplicatedMergeTreeQueue & queue_) : queue(queue_) +{ +} + +ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( + ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, std::optional && partition_ids_hint_) + : nested_pred(queue_) + , queue(queue_) , partition_ids_hint(std::move(partition_ids_hint_)) , prev_virtual_parts(queue.format_version) { @@ -2113,10 +2120,10 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( /// Dropped (or cleaned up by TTL) partitions are never removed from ZK, /// so without hint it can do a few thousands requests (if not using MultiRead). Strings partitions; - if (partition_ids_hint.empty()) + if (!partition_ids_hint) partitions = zookeeper->getChildren(fs::path(queue.zookeeper_path) / "block_numbers"); else - std::copy(partition_ids_hint.begin(), partition_ids_hint.end(), std::back_inserter(partitions)); + std::copy(partition_ids_hint->begin(), partition_ids_hint->end(), std::back_inserter(partitions)); std::vector paths; paths.reserve(partitions.size()); @@ -2162,6 +2169,18 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( inprogress_quorum_part.clear(); } +bool TrivialMergePredicate::operator()( + const MergeTreeData::DataPartPtr & left, + const MergeTreeData::DataPartPtr & right, + const MergeTreeTransaction *, + String * out_reason) const +{ + if (left) + return canMergeTwoParts(left, right, out_reason); + else + return canMergeSinglePart(right, out_reason); +} + bool ReplicatedMergeTreeMergePredicate::operator()( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, @@ -2249,7 +2268,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( if (left_max_block + 1 < right_min_block) { - if (!partition_ids_hint.empty() && !partition_ids_hint.contains(left->info.partition_id)) + if (partition_ids_hint && !partition_ids_hint->contains(left->info.partition_id)) { if (out_reason) *out_reason = fmt::format("Uncommitted block were not loaded for unexpected partition {}", left->info.partition_id); @@ -2273,6 +2292,17 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( } } + return nested_pred.canMergeTwoParts(left, right, out_reason); +} + +bool TrivialMergePredicate::canMergeTwoParts( + const MergeTreeData::DataPartPtr & left, + const MergeTreeData::DataPartPtr & right, + String * out_reason) const +{ + Int64 left_max_block = left->info.max_block; + Int64 right_min_block = right->info.min_block; + std::lock_guard lock(queue.state_mutex); for (const MergeTreeData::DataPartPtr & part : {left, right}) @@ -2350,6 +2380,11 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( return false; } + return nested_pred.canMergeSinglePart(part, out_reason); +} + +bool TrivialMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const +{ std::lock_guard lock(queue.state_mutex); /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer @@ -2481,7 +2516,7 @@ bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const std::string & z const String & partition_id = kv.first; Int64 block_num = kv.second; - if (!partition_ids_hint.empty() && !partition_ids_hint.contains(partition_id)) + if (partition_ids_hint && !partition_ids_hint->contains(partition_id)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Partition id {} was not provided as hint, it's a bug", partition_id); auto partition_it = committing_blocks.find(partition_id); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 368f2d4bc1f..2a2e47cc83f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -32,6 +32,7 @@ class ReplicatedMergeTreeQueue { private: friend class CurrentlyExecuting; + friend class TrivialMergePredicate; friend class ReplicatedMergeTreeMergePredicate; friend class MergeFromLogEntryTask; friend class ReplicatedMergeMutateTaskBase; @@ -390,7 +391,8 @@ public: size_t countUnfinishedMutations() const; /// Returns functor which used by MergeTreeMergerMutator to select parts for merge - ReplicatedMergeTreeMergePredicate getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, PartitionIdsHint && partition_ids_hint); + ReplicatedMergeTreeMergePredicate getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, + std::optional && partition_ids_hint); MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version) const; @@ -488,10 +490,33 @@ public: void createLogEntriesToFetchBrokenParts(); }; +/// Lightweight version of ReplicatedMergeTreeMergePredicate that do not make any ZooKeeper requests, +/// but may return false-positive results. Checks only a subset of required conditions. +class TrivialMergePredicate +{ +public: + TrivialMergePredicate(ReplicatedMergeTreeQueue & queue_); + + bool operator()(const MergeTreeData::DataPartPtr & left, + const MergeTreeData::DataPartPtr & right, + const MergeTreeTransaction * txn, + String * out_reason = nullptr) const; + + bool canMergeTwoParts(const MergeTreeData::DataPartPtr & left, + const MergeTreeData::DataPartPtr & right, + String * out_reason = nullptr) const; + + bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const; + +private: + const ReplicatedMergeTreeQueue & queue; +}; + class ReplicatedMergeTreeMergePredicate { public: - ReplicatedMergeTreeMergePredicate(ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, PartitionIdsHint && partition_ids_hint_); + ReplicatedMergeTreeMergePredicate(ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, + std::optional && partition_ids_hint_); /// Depending on the existence of left part checks a merge predicate for two parts or for single part. bool operator()(const MergeTreeData::DataPartPtr & left, @@ -534,9 +559,11 @@ public: String getCoveringVirtualPart(const String & part_name) const; private: + TrivialMergePredicate nested_pred; + const ReplicatedMergeTreeQueue & queue; - PartitionIdsHint partition_ids_hint; + std::optional partition_ids_hint; /// A snapshot of active parts that would appear if the replica executes all log entries in its queue. ActiveDataPartSet prev_virtual_parts; diff --git a/src/Storages/MergeTree/TTLMergeSelector.cpp b/src/Storages/MergeTree/TTLMergeSelector.cpp index d5657aa680d..f4c698d76d7 100644 --- a/src/Storages/MergeTree/TTLMergeSelector.cpp +++ b/src/Storages/MergeTree/TTLMergeSelector.cpp @@ -11,7 +11,7 @@ namespace DB const String & getPartitionIdForPart(const ITTLMergeSelector::Part & part_info) { - const MergeTreeData::DataPartPtr & part = *static_cast(part_info.data); + const MergeTreeData::DataPartPtr & part = part_info.getDataPartPtr(); return part->info.partition_id; } @@ -90,8 +90,11 @@ IMergeSelector::PartsRange ITTLMergeSelector::select( ++best_end; } - const auto & best_partition_id = getPartitionIdForPart(best_partition.front()); - merge_due_times[best_partition_id] = current_time + merge_cooldown_time; + if (!dry_run) + { + const auto & best_partition_id = getPartitionIdForPart(best_partition.front()); + merge_due_times[best_partition_id] = current_time + merge_cooldown_time; + } return PartsRange(best_begin, best_end); } diff --git a/src/Storages/MergeTree/TTLMergeSelector.h b/src/Storages/MergeTree/TTLMergeSelector.h index 88dc1fffee2..8c82e284a45 100644 --- a/src/Storages/MergeTree/TTLMergeSelector.h +++ b/src/Storages/MergeTree/TTLMergeSelector.h @@ -21,10 +21,11 @@ class ITTLMergeSelector : public IMergeSelector public: using PartitionIdToTTLs = std::map; - ITTLMergeSelector(PartitionIdToTTLs & merge_due_times_, time_t current_time_, Int64 merge_cooldown_time_) + ITTLMergeSelector(PartitionIdToTTLs & merge_due_times_, time_t current_time_, Int64 merge_cooldown_time_, bool dry_run_) : current_time(current_time_) , merge_due_times(merge_due_times_) , merge_cooldown_time(merge_cooldown_time_) + , dry_run(dry_run_) { } @@ -46,6 +47,7 @@ protected: private: PartitionIdToTTLs & merge_due_times; Int64 merge_cooldown_time; + bool dry_run; }; @@ -56,8 +58,9 @@ class TTLDeleteMergeSelector : public ITTLMergeSelector public: using PartitionIdToTTLs = std::map; - TTLDeleteMergeSelector(PartitionIdToTTLs & merge_due_times_, time_t current_time_, Int64 merge_cooldown_time_, bool only_drop_parts_) - : ITTLMergeSelector(merge_due_times_, current_time_, merge_cooldown_time_) + TTLDeleteMergeSelector(PartitionIdToTTLs & merge_due_times_, time_t current_time_, Int64 merge_cooldown_time_, + bool only_drop_parts_, bool dry_run_) + : ITTLMergeSelector(merge_due_times_, current_time_, merge_cooldown_time_, dry_run_) , only_drop_parts(only_drop_parts_) {} time_t getTTLForPart(const IMergeSelector::Part & part) const override; @@ -75,8 +78,9 @@ private: class TTLRecompressMergeSelector : public ITTLMergeSelector { public: - TTLRecompressMergeSelector(PartitionIdToTTLs & merge_due_times_, time_t current_time_, Int64 merge_cooldown_time_, const TTLDescriptions & recompression_ttls_) - : ITTLMergeSelector(merge_due_times_, current_time_, merge_cooldown_time_) + TTLRecompressMergeSelector(PartitionIdToTTLs & merge_due_times_, time_t current_time_, Int64 merge_cooldown_time_, + const TTLDescriptions & recompression_ttls_, bool dry_run_) + : ITTLMergeSelector(merge_due_times_, current_time_, merge_cooldown_time_, dry_run_) , recompression_ttls(recompression_ttls_) {} diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b548b1efd93..19f1c793164 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3242,7 +3242,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() auto zookeeper = getZooKeeperAndAssertNotReadonly(); - ReplicatedMergeTreeMergePredicate merge_pred = queue.getMergePredicate(zookeeper, getAllPartitionIds()); + std::optional merge_pred; /// If many merges is already queued, then will queue only small enough merges. /// Otherwise merge queue could be filled with only large merges, @@ -3272,8 +3272,22 @@ void StorageReplicatedMergeTree::mergeSelectingTask() if (storage_settings.get()->assign_part_uuids) future_merged_part->uuid = UUIDHelpers::generateV4(); - if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, merge_with_ttl_allowed, NO_TRANSACTION_PTR, nullptr) == SelectPartsDecision::SELECTED) + bool can_assign_merge = max_source_parts_size_for_merge > 0; + PartitionIdsHint partitions_to_merge_in; + if (can_assign_merge) + { + auto lightweight_merge_pred = TrivialMergePredicate(queue); + partitions_to_merge_in = merger_mutator.getPartitionsThatMayBeMerged( + max_source_parts_size_for_merge, lightweight_merge_pred, merge_with_ttl_allowed, NO_TRANSACTION_PTR); + if (partitions_to_merge_in.empty()) + can_assign_merge = false; + else + merge_pred.emplace(queue.getMergePredicate(zookeeper, partitions_to_merge_in)); + } + + if (can_assign_merge && + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, *merge_pred, + merge_with_ttl_allowed, NO_TRANSACTION_PTR, nullptr, &partitions_to_merge_in) == SelectPartsDecision::SELECTED) { create_result = createLogEntryToMergeParts( zookeeper, @@ -3285,13 +3299,17 @@ void StorageReplicatedMergeTree::mergeSelectingTask() deduplicate_by_columns, cleanup, nullptr, - merge_pred.getVersion(), + merge_pred->getVersion(), future_merged_part->merge_type); } /// If there are many mutations in queue, it may happen, that we cannot enqueue enough merges to merge all new parts else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0 && merges_and_mutations_queued.mutations < storage_settings_ptr->max_replicated_mutations_in_queue) { + /// We don't need the list of committing blocks to choose a part to mutate + if (!merge_pred) + merge_pred.emplace(queue.getMergePredicate(zookeeper, PartitionIdsHint{})); + /// Choose a part to mutate. DataPartsVector data_parts = getDataPartsVectorForInternalUsage(); for (const auto & part : data_parts) @@ -3299,7 +3317,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() if (part->getBytesOnDisk() > max_source_part_size_for_mutation) continue; - std::optional> desired_mutation_version = merge_pred.getDesiredMutationVersion(part); + std::optional> desired_mutation_version = merge_pred->getDesiredMutationVersion(part); if (!desired_mutation_version) continue; @@ -3308,7 +3326,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() future_merged_part->uuid, desired_mutation_version->first, desired_mutation_version->second, - merge_pred.getVersion()); + merge_pred->getVersion()); if (create_result == CreateMergeEntryResult::Ok || create_result == CreateMergeEntryResult::LogUpdated) From 500f3d395117b46889c7f6b0ff2fb78d67e1a0f6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 10:18:55 +0000 Subject: [PATCH 0200/2223] Add SQL functions for Entropy Learned Hashing Courtesy to @Dmitry909, I just wrapped up his work. --- src/Functions/EntropyLearnedHash.cpp | 286 ++++++++++++++++++ ...new_functions_must_be_documented.reference | 2 + .../02734_entropy_learned_hashing.reference | 12 + .../02734_entropy_learned_hashing.sql | 24 ++ 4 files changed, 324 insertions(+) create mode 100644 src/Functions/EntropyLearnedHash.cpp create mode 100644 tests/queries/0_stateless/02734_entropy_learned_hashing.reference create mode 100644 tests/queries/0_stateless/02734_entropy_learned_hashing.sql diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp new file mode 100644 index 00000000000..3bb99e51c48 --- /dev/null +++ b/src/Functions/EntropyLearnedHash.cpp @@ -0,0 +1,286 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +/// Implementation of entropy-learned hashing: https://dl.acm.org/doi/10.1145/3514221.3517894 +/// TODOs for future work: +/// - allow to specify an arbitrary hash funtion (currently always CityHash is used) +/// - allow function chaining a la entropyLearnedHash(trainEntropyLearnedHash()) + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + +namespace EntropyLearnedHashing +{ + +using Key = String; +using PartialKeyPositions = std::vector; + +namespace +{ + +Key getPartialKey(const Key & key, const std::vector & positions) +{ + Key result_key; + result_key.reserve(positions.size()); + for (auto position : positions) + if (position < key.size()) + result_key.push_back(key[position]); + return result_key; +} + +bool allPartialKeysAreUnique(const std::vector & data, const std::vector & positions) +{ + std::unordered_set partial_keys; + partial_keys.reserve(data.size()); + for (const auto & key : data) + if (!partial_keys.insert(EntropyLearnedHashing::getPartialKey(key, positions)).second) + return false; + return true; +} + +// NextByte returns position of byte which adds the most entropy and the new entropy +std::pair nextByte(const std::vector & keys, size_t max_len, std::vector & chosen_bytes) +{ + size_t min_collisions = std::numeric_limits::max(); + size_t best_position = 0; + + std::unordered_map count_table; + for (size_t i = 0; i < max_len; ++i) + { + count_table.clear(); + count_table.reserve(keys.size()); + + chosen_bytes.push_back(i); + size_t collisions = 0; + for (const auto & key : keys) + { + auto partial_key = EntropyLearnedHashing::getPartialKey(key, chosen_bytes); + collisions += count_table[partial_key]++; + } + + if (collisions < min_collisions) + { + min_collisions = collisions; + best_position = i; + } + chosen_bytes.pop_back(); + } + return {best_position, min_collisions}; +} + +// std::pair nextByte(const std::vector & keys, std::vector & chosen_bytes) +// { +// size_t max_len = 0; +// for (const auto & key : keys) +// max_len = std::max(max_len, key.size()); + +// return nextByte(keys, max_len, chosen_bytes); +// } + +class IdManager +{ +public: + static IdManager & instance() + { + static IdManager instance; + return instance; + } + void setPartialKeyPositionsForId(const String & user_name, const String & id, const PartialKeyPositions & partial_key_positions) + { + std::lock_guard lock(mutex); + /// partial_key_positions_by_id[id] = partial_key_positions; + auto & ids_for_user = partial_key_positions_by_id[user_name]; + ids_for_user[id] = partial_key_positions; + } + const PartialKeyPositions & getPartialKeyPositionsForId(const String & user_name, const String & id) const + { + std::lock_guard lock(mutex); + auto it_user = partial_key_positions_by_id.find(user_name); + if (it_user == partial_key_positions_by_id.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Id {} not registered for user in entropy learned hashing", id); + auto it_id = it_user->second.find(id); + if (it_id == it_user->second.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Id {} not registered for user in entropy learned hashing", id); + return it_id->second; + } + +private: + mutable std::mutex mutex; + /// Map: user name --> (Map: dataset id --> byte positions to hash) + std::map> partial_key_positions_by_id TSA_GUARDED_BY(mutex); +}; + +std::pair, std::vector> chooseBytes(const std::vector & train_data) +{ + if (train_data.size() <= 1) + return {}; + + // position contains numbers of chosen bytes + std::vector positions; + + // entropies contains entropies of keys after each new chosen byte + std::vector entropies; + + // max_len is a maximal length of any key in train_data + size_t max_len = 0; + for (const auto & key : train_data) + max_len = std::max(max_len, key.size()); + + // while not all partial keys unique, choose new byte and recalculate the entropy + while (!allPartialKeysAreUnique(train_data, positions)) + { + auto [new_position, new_entropy] = nextByte(train_data, max_len, positions); + if (!entropies.empty() && new_entropy == entropies.back()) + break; + positions.push_back(new_position); + entropies.push_back(new_entropy); + } + return {positions, entropies}; +} + +} + +class FunctionTrainEntropyLearnedHash : public IFunction +{ +public: + static constexpr auto name = "trainEntropyLearnedHash"; + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getUserName()); } + explicit FunctionTrainEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} + + String getName() const override { return name; } + bool isVariadic() const override { return false; } + size_t getNumberOfArguments() const override { return 2; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"data", &isString, nullptr, "String"}, + {"id", &isString, nullptr, "String"} + }; + + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override + { + const IColumn * id_col = arguments[1].column.get(); + const ColumnConst * id_col_const = checkAndGetColumn(id_col); + const String id = id_col_const->getValue(); + + const auto * data_col = arguments[0].column.get(); + if (const ColumnString * col_data_string = checkAndGetColumn(data_col)) + { + const size_t num_rows = col_data_string->size(); + + std::vector training_data; + for (size_t i = 0; i < num_rows; ++i) + { + std::string_view string_ref = col_data_string->getDataAt(i).toView(); + training_data.emplace_back(string_ref.data(), string_ref.size()); + } + + EntropyLearnedHashing::PartialKeyPositions partial_key_positions = EntropyLearnedHashing::chooseBytes(training_data).first; + auto & id_manager = EntropyLearnedHashing::IdManager::instance(); + id_manager.setPartialKeyPositionsForId(user_name, id, partial_key_positions); + + return result_type->createColumnConst(num_rows, 0u)->convertToFullColumnIfConst(); + } + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", + arguments.begin()->column->getName(), getName()); + } +private: + const String user_name; +}; + + +class FunctionEntropyLearnedHash : public IFunction +{ +public: + static constexpr auto name = "entropyLearnedHash"; + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getUserName()); } + explicit FunctionEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} + + String getName() const override { return name; } + bool isVariadic() const override { return false; } + size_t getNumberOfArguments() const override { return 2; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"data", &isString, nullptr, "String"}, + {"id", &isString, nullptr, "String"} + }; + + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + const IColumn * id_col = arguments.back().column.get(); + const ColumnConst * id_col_const = checkAndGetColumn(id_col); + const String id = id_col_const->getValue(); + + const auto & id_manager = EntropyLearnedHashing::IdManager::instance(); + const auto & partial_key_positions = id_manager.getPartialKeyPositionsForId(user_name, id); + + const auto * data_col = arguments[0].column.get(); + if (const auto * col_data_string = checkAndGetColumn(data_col)) + { + const size_t num_rows = col_data_string->size(); + auto col_res = ColumnUInt64::create(num_rows); + + auto & col_res_vec = col_res->getData(); + for (size_t i = 0; i < num_rows; ++i) + { + std::string_view string_ref = col_data_string->getDataAt(i).toView(); + EntropyLearnedHashing::Key key(string_ref.data(), string_ref.size()); + EntropyLearnedHashing::Key partial_key = EntropyLearnedHashing::getPartialKey(key, partial_key_positions); + col_res_vec[i] = CityHash_v1_0_2::CityHash64(partial_key.data(), partial_key.size()); + } + + return col_res; + } + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", + arguments.begin()->column->getName(), getName()); + } +private: + const String user_name; +}; + +REGISTER_FUNCTION(Hashing) +{ + factory.registerFunction(); + factory.registerFunction(); +} + +} + +} diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index eb574c4e4f5..315cf7fc0d4 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -265,6 +265,7 @@ encodeURLComponent encodeURLFormComponent encodeXMLComponent endsWith +entropyLearnedHash equals erf erfc @@ -860,6 +861,7 @@ toYear toYearWeek today tokens +trainEntropyLearnedHash transactionID transactionLatestSnapshot transactionOldestSnapshot diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference b/tests/queries/0_stateless/02734_entropy_learned_hashing.reference new file mode 100644 index 00000000000..2240da7fea4 --- /dev/null +++ b/tests/queries/0_stateless/02734_entropy_learned_hashing.reference @@ -0,0 +1,12 @@ +0 +0 +0 +2603192927274642682 +4947675599669400333 +10783339242466472992 +0 +0 +0 +2603192927274642682 +4947675599669400333 +10783339242466472992 diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql b/tests/queries/0_stateless/02734_entropy_learned_hashing.sql new file mode 100644 index 00000000000..d50883f3cf8 --- /dev/null +++ b/tests/queries/0_stateless/02734_entropy_learned_hashing.sql @@ -0,0 +1,24 @@ +-- Tags: no-parallel +-- Tag no-parallel: Accesses global state + +DROP TABLE IF EXISTS tbl1; +CREATE TABLE tbl1 (x String) ENGINE=Memory; +-- no commonalities between keys +INSERT INTO tbl1 VALUES ('a'), ('b'), ('c'); +SELECT trainEntropyLearnedHash(x, 'id1') FROM tbl1; +SELECT entropyLearnedHash(x, 'id1') FROM tbl1; + +SELECT trainEntropyLearnedHash(x, 1) FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- SELECT trainEntropyLearnedHash(x, NULL) FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT trainEntropyLearnedHash(1, 'id1') FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT entropyLearnedHash(x, 'non-existing id') FROM tbl1; -- { serverError BAD_ARGUMENTS } + +DROP TABLE IF EXISTS tbl2; +CREATE TABLE tbl2 (x String) ENGINE=Memory; +-- with commonalities between keys +INSERT INTO tbl2 VALUES ('aa'), ('ba'), ('ca'); +SELECT trainEntropyLearnedHash(x, 'id1') FROM tbl2; +SELECT entropyLearnedHash(x, 'id1') FROM tbl2; + +DROP TABLE tbl1; +DROP TABLE tbl2; From bb5a25e81c9cec334693bbc7e7b3b0f14c6f107b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:05:44 +0000 Subject: [PATCH 0201/2223] Fix typo --- src/Functions/EntropyLearnedHash.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index 3bb99e51c48..921b7921e3b 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -11,7 +11,7 @@ /// Implementation of entropy-learned hashing: https://dl.acm.org/doi/10.1145/3514221.3517894 /// TODOs for future work: -/// - allow to specify an arbitrary hash funtion (currently always CityHash is used) +/// - allow to specify an arbitrary hash function (currently always CityHash is used) /// - allow function chaining a la entropyLearnedHash(trainEntropyLearnedHash()) From b9e8c52057a584991ce9914eaae00a230431c5c4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:08:22 +0000 Subject: [PATCH 0202/2223] Fix function registration --- src/Functions/EntropyLearnedHash.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index 921b7921e3b..1b9f8d0772b 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -275,7 +275,7 @@ private: const String user_name; }; -REGISTER_FUNCTION(Hashing) +REGISTER_FUNCTION(EntropyLearnedHash) { factory.registerFunction(); factory.registerFunction(); From 267e0c4ef5085a2b18d15b2ad5a8402f7c3912bc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:10:06 +0000 Subject: [PATCH 0203/2223] More typedef usage --- src/Functions/EntropyLearnedHash.cpp | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index 1b9f8d0772b..873ce9a0386 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -33,22 +33,22 @@ using PartialKeyPositions = std::vector; namespace { -Key getPartialKey(const Key & key, const std::vector & positions) +Key getPartialKey(const Key & key, const PartialKeyPositions & partial_key_positions) { Key result_key; - result_key.reserve(positions.size()); - for (auto position : positions) - if (position < key.size()) - result_key.push_back(key[position]); + result_key.reserve(partial_key_positions.size()); + for (auto partial_key_position : partial_key_positions) + if (partial_key_position < key.size()) + result_key.push_back(key[partial_key_position]); return result_key; } -bool allPartialKeysAreUnique(const std::vector & data, const std::vector & positions) +bool allPartialKeysAreUnique(const std::vector & data, const PartialKeyPositions & partial_key_positions) { std::unordered_set partial_keys; partial_keys.reserve(data.size()); for (const auto & key : data) - if (!partial_keys.insert(EntropyLearnedHashing::getPartialKey(key, positions)).second) + if (!partial_keys.insert(EntropyLearnedHashing::getPartialKey(key, partial_key_positions)).second) return false; return true; } @@ -125,13 +125,13 @@ private: std::map> partial_key_positions_by_id TSA_GUARDED_BY(mutex); }; -std::pair, std::vector> chooseBytes(const std::vector & train_data) +std::pair> chooseBytes(const std::vector & train_data) { if (train_data.size() <= 1) return {}; // position contains numbers of chosen bytes - std::vector positions; + PartialKeyPositions partial_key_positions; // entropies contains entropies of keys after each new chosen byte std::vector entropies; @@ -142,15 +142,15 @@ std::pair, std::vector> chooseBytes(const std::vecto max_len = std::max(max_len, key.size()); // while not all partial keys unique, choose new byte and recalculate the entropy - while (!allPartialKeysAreUnique(train_data, positions)) + while (!allPartialKeysAreUnique(train_data, partial_key_positions)) { - auto [new_position, new_entropy] = nextByte(train_data, max_len, positions); + auto [new_position, new_entropy] = nextByte(train_data, max_len, partial_key_positions); if (!entropies.empty() && new_entropy == entropies.back()) break; - positions.push_back(new_position); + partial_key_positions.push_back(new_position); entropies.push_back(new_entropy); } - return {positions, entropies}; + return {partial_key_positions, entropies}; } } From 03e9522de42141af75302fbc483a79fb603fe72d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:13:52 +0000 Subject: [PATCH 0204/2223] Less namespace clutter --- src/Functions/EntropyLearnedHash.cpp | 31 ++++++++++++---------------- 1 file changed, 13 insertions(+), 18 deletions(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index 873ce9a0386..8b9fc592696 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -24,9 +24,6 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -namespace EntropyLearnedHashing -{ - using Key = String; using PartialKeyPositions = std::vector; @@ -43,23 +40,23 @@ Key getPartialKey(const Key & key, const PartialKeyPositions & partial_key_posit return result_key; } -bool allPartialKeysAreUnique(const std::vector & data, const PartialKeyPositions & partial_key_positions) +bool allPartialKeysAreUnique(const std::vector & data, const PartialKeyPositions & partial_key_positions) { - std::unordered_set partial_keys; + std::unordered_set partial_keys; partial_keys.reserve(data.size()); for (const auto & key : data) - if (!partial_keys.insert(EntropyLearnedHashing::getPartialKey(key, partial_key_positions)).second) + if (!partial_keys.insert(getPartialKey(key, partial_key_positions)).second) return false; return true; } // NextByte returns position of byte which adds the most entropy and the new entropy -std::pair nextByte(const std::vector & keys, size_t max_len, std::vector & chosen_bytes) +std::pair nextByte(const std::vector & keys, size_t max_len, std::vector & chosen_bytes) { size_t min_collisions = std::numeric_limits::max(); size_t best_position = 0; - std::unordered_map count_table; + std::unordered_map count_table; for (size_t i = 0; i < max_len; ++i) { count_table.clear(); @@ -69,7 +66,7 @@ std::pair nextByte(const std::vector size_t collisions = 0; for (const auto & key : keys) { - auto partial_key = EntropyLearnedHashing::getPartialKey(key, chosen_bytes); + auto partial_key = getPartialKey(key, chosen_bytes); collisions += count_table[partial_key]++; } @@ -83,7 +80,7 @@ std::pair nextByte(const std::vector return {best_position, min_collisions}; } -// std::pair nextByte(const std::vector & keys, std::vector & chosen_bytes) +// std::pair nextByte(const std::vector & keys, std::vector & chosen_bytes) // { // size_t max_len = 0; // for (const auto & key : keys) @@ -192,15 +189,15 @@ public: { const size_t num_rows = col_data_string->size(); - std::vector training_data; + std::vector training_data; for (size_t i = 0; i < num_rows; ++i) { std::string_view string_ref = col_data_string->getDataAt(i).toView(); training_data.emplace_back(string_ref.data(), string_ref.size()); } - EntropyLearnedHashing::PartialKeyPositions partial_key_positions = EntropyLearnedHashing::chooseBytes(training_data).first; - auto & id_manager = EntropyLearnedHashing::IdManager::instance(); + PartialKeyPositions partial_key_positions = chooseBytes(training_data).first; + auto & id_manager = IdManager::instance(); id_manager.setPartialKeyPositionsForId(user_name, id, partial_key_positions); return result_type->createColumnConst(num_rows, 0u)->convertToFullColumnIfConst(); @@ -247,7 +244,7 @@ public: const ColumnConst * id_col_const = checkAndGetColumn(id_col); const String id = id_col_const->getValue(); - const auto & id_manager = EntropyLearnedHashing::IdManager::instance(); + const auto & id_manager = IdManager::instance(); const auto & partial_key_positions = id_manager.getPartialKeyPositionsForId(user_name, id); const auto * data_col = arguments[0].column.get(); @@ -260,8 +257,8 @@ public: for (size_t i = 0; i < num_rows; ++i) { std::string_view string_ref = col_data_string->getDataAt(i).toView(); - EntropyLearnedHashing::Key key(string_ref.data(), string_ref.size()); - EntropyLearnedHashing::Key partial_key = EntropyLearnedHashing::getPartialKey(key, partial_key_positions); + Key key(string_ref.data(), string_ref.size()); + Key partial_key = getPartialKey(key, partial_key_positions); col_res_vec[i] = CityHash_v1_0_2::CityHash64(partial_key.data(), partial_key.size()); } @@ -282,5 +279,3 @@ REGISTER_FUNCTION(EntropyLearnedHash) } } - -} From fdabce9a6818249e1ea7f75feeaf4c59708efb4f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:15:08 +0000 Subject: [PATCH 0205/2223] Move chooseBytes() up --- src/Functions/EntropyLearnedHash.cpp | 62 ++++++++++++++-------------- 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index 8b9fc592696..6cb4d7350a5 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -24,12 +24,12 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -using Key = String; -using PartialKeyPositions = std::vector; - namespace { +using Key = String; +using PartialKeyPositions = std::vector; + Key getPartialKey(const Key & key, const PartialKeyPositions & partial_key_positions) { Key result_key; @@ -89,6 +89,34 @@ std::pair nextByte(const std::vector & keys, size_t max_len // return nextByte(keys, max_len, chosen_bytes); // } +std::pair> chooseBytes(const std::vector & train_data) +{ + if (train_data.size() <= 1) + return {}; + + // position contains numbers of chosen bytes + PartialKeyPositions partial_key_positions; + + // entropies contains entropies of keys after each new chosen byte + std::vector entropies; + + // max_len is a maximal length of any key in train_data + size_t max_len = 0; + for (const auto & key : train_data) + max_len = std::max(max_len, key.size()); + + // while not all partial keys unique, choose new byte and recalculate the entropy + while (!allPartialKeysAreUnique(train_data, partial_key_positions)) + { + auto [new_position, new_entropy] = nextByte(train_data, max_len, partial_key_positions); + if (!entropies.empty() && new_entropy == entropies.back()) + break; + partial_key_positions.push_back(new_position); + entropies.push_back(new_entropy); + } + return {partial_key_positions, entropies}; +} + class IdManager { public: @@ -122,34 +150,6 @@ private: std::map> partial_key_positions_by_id TSA_GUARDED_BY(mutex); }; -std::pair> chooseBytes(const std::vector & train_data) -{ - if (train_data.size() <= 1) - return {}; - - // position contains numbers of chosen bytes - PartialKeyPositions partial_key_positions; - - // entropies contains entropies of keys after each new chosen byte - std::vector entropies; - - // max_len is a maximal length of any key in train_data - size_t max_len = 0; - for (const auto & key : train_data) - max_len = std::max(max_len, key.size()); - - // while not all partial keys unique, choose new byte and recalculate the entropy - while (!allPartialKeysAreUnique(train_data, partial_key_positions)) - { - auto [new_position, new_entropy] = nextByte(train_data, max_len, partial_key_positions); - if (!entropies.empty() && new_entropy == entropies.back()) - break; - partial_key_positions.push_back(new_position); - entropies.push_back(new_entropy); - } - return {partial_key_positions, entropies}; -} - } class FunctionTrainEntropyLearnedHash : public IFunction From 1b7c207d7a9dd20a50e253e28f47482d09a8a0f4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:16:12 +0000 Subject: [PATCH 0206/2223] Replace ACM link by DOI link --- src/Functions/EntropyLearnedHash.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index 6cb4d7350a5..e926bd3cfcc 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -9,7 +9,7 @@ #include #include -/// Implementation of entropy-learned hashing: https://dl.acm.org/doi/10.1145/3514221.3517894 +/// Implementation of entropy-learned hashing: https://doi.org/10.1145/3514221.3517894 /// TODOs for future work: /// - allow to specify an arbitrary hash function (currently always CityHash is used) /// - allow function chaining a la entropyLearnedHash(trainEntropyLearnedHash()) From d2216a433987b6eba1902e0fd753fb0d54a6e6b5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:19:39 +0000 Subject: [PATCH 0207/2223] Remove leftover --- src/Functions/EntropyLearnedHash.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index e926bd3cfcc..cb3ca0b132f 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -13,6 +13,7 @@ /// TODOs for future work: /// - allow to specify an arbitrary hash function (currently always CityHash is used) /// - allow function chaining a la entropyLearnedHash(trainEntropyLearnedHash()) +/// - support more datatypes for data (besides String) namespace DB @@ -128,7 +129,6 @@ public: void setPartialKeyPositionsForId(const String & user_name, const String & id, const PartialKeyPositions & partial_key_positions) { std::lock_guard lock(mutex); - /// partial_key_positions_by_id[id] = partial_key_positions; auto & ids_for_user = partial_key_positions_by_id[user_name]; ids_for_user[id] = partial_key_positions; } From 8b77b706c40b09f849432f2c809fdc9e61940911 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:29:43 +0000 Subject: [PATCH 0208/2223] Optimize allocations --- src/Functions/EntropyLearnedHash.cpp | 27 ++++++++++++++++++--------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index cb3ca0b132f..2d07198d583 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -31,23 +31,31 @@ namespace using Key = String; using PartialKeyPositions = std::vector; -Key getPartialKey(const Key & key, const PartialKeyPositions & partial_key_positions) +Key getPartialKey(std::string_view key, const PartialKeyPositions & partial_key_positions, Key & result) { - Key result_key; - result_key.reserve(partial_key_positions.size()); + result.clear(); + result.reserve(partial_key_positions.size()); + for (auto partial_key_position : partial_key_positions) if (partial_key_position < key.size()) - result_key.push_back(key[partial_key_position]); - return result_key; + result.push_back(key[partial_key_position]); + + return result; } bool allPartialKeysAreUnique(const std::vector & data, const PartialKeyPositions & partial_key_positions) { std::unordered_set partial_keys; partial_keys.reserve(data.size()); + Key partial_key; + for (const auto & key : data) - if (!partial_keys.insert(getPartialKey(key, partial_key_positions)).second) + { + getPartialKey(key, partial_key_positions, partial_key); + if (!partial_keys.insert(partial_key).second) return false; + } + return true; } @@ -58,6 +66,7 @@ std::pair nextByte(const std::vector & keys, size_t max_len size_t best_position = 0; std::unordered_map count_table; + Key partial_key; for (size_t i = 0; i < max_len; ++i) { count_table.clear(); @@ -67,7 +76,7 @@ std::pair nextByte(const std::vector & keys, size_t max_len size_t collisions = 0; for (const auto & key : keys) { - auto partial_key = getPartialKey(key, chosen_bytes); + getPartialKey(key, chosen_bytes, partial_key); collisions += count_table[partial_key]++; } @@ -254,11 +263,11 @@ public: auto col_res = ColumnUInt64::create(num_rows); auto & col_res_vec = col_res->getData(); + Key partial_key; for (size_t i = 0; i < num_rows; ++i) { std::string_view string_ref = col_data_string->getDataAt(i).toView(); - Key key(string_ref.data(), string_ref.size()); - Key partial_key = getPartialKey(key, partial_key_positions); + getPartialKey(string_ref, partial_key_positions, partial_key); col_res_vec[i] = CityHash_v1_0_2::CityHash64(partial_key.data(), partial_key.size()); } From d2dc5e9fc835d0dd95420d6bc50848b1d6a97ce5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:36:28 +0000 Subject: [PATCH 0209/2223] Improve naming --- src/Functions/EntropyLearnedHash.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index 2d07198d583..c42f0f65192 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -45,14 +45,14 @@ Key getPartialKey(std::string_view key, const PartialKeyPositions & partial_key_ bool allPartialKeysAreUnique(const std::vector & data, const PartialKeyPositions & partial_key_positions) { - std::unordered_set partial_keys; - partial_keys.reserve(data.size()); + std::unordered_set unique_partial_keys; + unique_partial_keys.reserve(data.size()); Key partial_key; for (const auto & key : data) { getPartialKey(key, partial_key_positions, partial_key); - if (!partial_keys.insert(partial_key).second) + if (!unique_partial_keys.insert(partial_key).second) return false; } From 1e3cd4ed6d39961addf82ab75d5fb2f661801ae5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 8 May 2023 12:39:01 +0000 Subject: [PATCH 0210/2223] close pipe before join --- src/Interpreters/TraceCollector.cpp | 47 ++++++++++++++--------------- src/Interpreters/TraceCollector.h | 1 - 2 files changed, 22 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index bedc27f7d15..0fc9aeb170b 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -34,40 +34,37 @@ TraceCollector::~TraceCollector() { try { - if (!thread.joinable()) - LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined"); + if (thread.joinable()) + { + /** Sends TraceCollector stop message + * + * Each sequence of data for TraceCollector thread starts with a boolean flag. + * If this flag is true, TraceCollector must stop reading trace_pipe and exit. + * This function sends flag with a true value to stop TraceCollector gracefully. + */ + WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); + writeChar(true, out); + out.next(); + } else - stop(); + LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined"); + } + catch (...) + { + tryLogCurrentException("TraceCollector"); + } + try + { TraceSender::pipe.close(); } catch (...) { tryLogCurrentException("TraceCollector"); } -} - -/** Sends TraceCollector stop message - * - * Each sequence of data for TraceCollector thread starts with a boolean flag. - * If this flag is true, TraceCollector must stop reading trace_pipe and exit. - * This function sends flag with a true value to stop TraceCollector gracefully. - */ -void TraceCollector::stop() -{ - try - { - WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); - writeChar(true, out); - out.next(); - } - catch (...) - { - tryLogCurrentException("TraceCollector"); - } - - thread.join(); + if (thread.joinable()) + thread.join(); } diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index 40fa854b791..d5bd09f1757 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -24,7 +24,6 @@ private: ThreadFromGlobalPool thread; void run(); - void stop(); }; } From 6a454ed6c3e959d1b7ea6cb9af2e74e638cc8b7d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:37:28 +0000 Subject: [PATCH 0211/2223] Add Entropies typedef --- src/Functions/EntropyLearnedHash.cpp | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index c42f0f65192..f7ea42dc55c 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -30,6 +30,7 @@ namespace using Key = String; using PartialKeyPositions = std::vector; +using Entropies = std::vector; Key getPartialKey(std::string_view key, const PartialKeyPositions & partial_key_positions, Key & result) { @@ -99,23 +100,18 @@ std::pair nextByte(const std::vector & keys, size_t max_len // return nextByte(keys, max_len, chosen_bytes); // } -std::pair> chooseBytes(const std::vector & train_data) +std::pair chooseBytes(const std::vector & train_data) { if (train_data.size() <= 1) return {}; - // position contains numbers of chosen bytes PartialKeyPositions partial_key_positions; + Entropies entropies; - // entropies contains entropies of keys after each new chosen byte - std::vector entropies; - - // max_len is a maximal length of any key in train_data - size_t max_len = 0; + size_t max_len = 0; /// length of the longest key in training data for (const auto & key : train_data) max_len = std::max(max_len, key.size()); - // while not all partial keys unique, choose new byte and recalculate the entropy while (!allPartialKeysAreUnique(train_data, partial_key_positions)) { auto [new_position, new_entropy] = nextByte(train_data, max_len, partial_key_positions); @@ -198,6 +194,7 @@ public: { const size_t num_rows = col_data_string->size(); + /// TODO this does some needless copying ... chooseBytes() should ideally understand the native ColumnString representation std::vector training_data; for (size_t i = 0; i < num_rows; ++i) { From e9d9eda3a2384fe0a9bc58bbe098e9ab990a8ff4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 12:44:25 +0000 Subject: [PATCH 0212/2223] More typedef usage --- src/Functions/EntropyLearnedHash.cpp | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index f7ea42dc55c..6b6effa72d5 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -10,6 +10,8 @@ #include /// Implementation of entropy-learned hashing: https://doi.org/10.1145/3514221.3517894 +/// If you change something in this file, please don't deviate too much from the pseudocode in the paper! + /// TODOs for future work: /// - allow to specify an arbitrary hash function (currently always CityHash is used) /// - allow function chaining a la entropyLearnedHash(trainEntropyLearnedHash()) @@ -61,23 +63,24 @@ bool allPartialKeysAreUnique(const std::vector & data, const PartialKeyPosi } // NextByte returns position of byte which adds the most entropy and the new entropy -std::pair nextByte(const std::vector & keys, size_t max_len, std::vector & chosen_bytes) +std::pair nextByte(const std::vector & keys, size_t max_len, PartialKeyPositions & partial_key_positions) { size_t min_collisions = std::numeric_limits::max(); size_t best_position = 0; std::unordered_map count_table; Key partial_key; + for (size_t i = 0; i < max_len; ++i) { count_table.clear(); count_table.reserve(keys.size()); - chosen_bytes.push_back(i); + partial_key_positions.push_back(i); size_t collisions = 0; for (const auto & key : keys) { - getPartialKey(key, chosen_bytes, partial_key); + getPartialKey(key, partial_key_positions, partial_key); collisions += count_table[partial_key]++; } @@ -86,18 +89,19 @@ std::pair nextByte(const std::vector & keys, size_t max_len min_collisions = collisions; best_position = i; } - chosen_bytes.pop_back(); + partial_key_positions.pop_back(); } + return {best_position, min_collisions}; } -// std::pair nextByte(const std::vector & keys, std::vector & chosen_bytes) +// std::pair nextByte(const std::vector & keys, PartialKeyPositions & partial_key_positions) // { // size_t max_len = 0; // for (const auto & key : keys) // max_len = std::max(max_len, key.size()); -// return nextByte(keys, max_len, chosen_bytes); +// return nextByte(keys, max_len, partial_key_positions); // } std::pair chooseBytes(const std::vector & train_data) From 49ecba63af61313d4419a80e827a1ba22f163838 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 8 May 2023 14:51:04 +0200 Subject: [PATCH 0213/2223] Removed setStageForCluster and added option all_hosts to set stage for cluster --- src/Backups/BackupCoordinationLocal.cpp | 4 -- src/Backups/BackupCoordinationLocal.h | 3 +- src/Backups/BackupCoordinationRemote.cpp | 11 ++---- src/Backups/BackupCoordinationRemote.h | 3 +- src/Backups/BackupCoordinationStageSync.cpp | 43 +++++++++++---------- src/Backups/BackupCoordinationStageSync.h | 3 +- src/Backups/BackupsWorker.cpp | 6 +-- src/Backups/IBackupCoordination.h | 3 +- src/Backups/IRestoreCoordination.h | 3 +- src/Backups/RestoreCoordinationLocal.cpp | 4 -- src/Backups/RestoreCoordinationLocal.h | 3 +- src/Backups/RestoreCoordinationRemote.cpp | 11 ++---- src/Backups/RestoreCoordinationRemote.h | 3 +- 13 files changed, 40 insertions(+), 60 deletions(-) diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 5b7ee37618b..27e0f173cf3 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -19,10 +19,6 @@ void BackupCoordinationLocal::setStage(const String &, const String &) { } -void BackupCoordinationLocal::setStageForCluster(const String &) -{ -} - void BackupCoordinationLocal::setError(const Exception &) { } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index f1ffa8e8517..a7b05fbb83c 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -22,8 +22,7 @@ public: BackupCoordinationLocal(bool plain_backup_); ~BackupCoordinationLocal() override; - void setStage(const String & new_stage, const String & message) override; - void setStageForCluster(const String & new_stage) override; /// Sets stage for cluster + void setStage(const String & new_stage, const String & message = "") override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index af88b15d622..27e7d23ce5f 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -254,17 +254,14 @@ void BackupCoordinationRemote::removeAllNodes() void BackupCoordinationRemote::setStage(const String & new_stage, const String & message) { - stage_sync->set(current_host, new_stage, message); -} - -void BackupCoordinationRemote::setStageForCluster(const String & new_stage) -{ - stage_sync->setStageForCluster(new_stage); + if (is_internal) + stage_sync->set(current_host, new_stage, message); + else + stage_sync->set(current_host, new_stage, /* message */ "", /* all_hosts */ true); } void BackupCoordinationRemote::setError(const Exception & exception) { - stage_sync->setStageForCluster(Stage::ERROR); stage_sync->setError(current_host, exception); } diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index c659cb0d459..5671079fa27 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -33,8 +33,7 @@ public: ~BackupCoordinationRemote() override; - void setStage(const String & new_stage, const String & message) override; - void setStageForCluster(const String & new_stage) override; /// Sets stage for cluster + void setStage(const String & new_stage, const String & message = "") override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index 5cbeec0ec76..3d8c283f084 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -8,11 +8,13 @@ #include #include #include - +#include namespace DB { +namespace Stage = BackupCoordinationStage; + namespace ErrorCodes { extern const int FAILED_TO_SYNC_BACKUP_OR_RESTORE; @@ -42,7 +44,7 @@ void BackupCoordinationStageSync::createRootNodes() }); } -void BackupCoordinationStageSync::set(const String & current_host, const String & new_stage, const String & message) +void BackupCoordinationStageSync::set(const String & current_host, const String & new_stage, const String & message, const bool & all_hosts) { auto holder = with_retries.createRetriesControlHolder("set"); holder.retries_ctl.retryLoop( @@ -50,29 +52,24 @@ void BackupCoordinationStageSync::set(const String & current_host, const String { with_retries.renewZooKeeper(zookeeper); - /// Make an ephemeral node so the initiator can track if the current host is still working. - String alive_node_path = zookeeper_path + "/alive|" + current_host; - auto code = zookeeper->tryCreate(alive_node_path, "", zkutil::CreateMode::Ephemeral); - if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) - throw zkutil::KeeperException(code, alive_node_path); - - zookeeper->createIfNotExists(zookeeper_path + "/started|" + current_host, ""); - zookeeper->createIfNotExists(zookeeper_path + "/current|" + current_host + "|" + new_stage, message); - }); -} - -void BackupCoordinationStageSync::setStageForCluster(const String & new_stage) -{ - auto holder = with_retries.createRetriesControlHolder("setStageForCluster"); - holder.retries_ctl.retryLoop( - [&, &zookeeper = holder.faulty_zookeeper]() + if (all_hosts) { - with_retries.renewZooKeeper(zookeeper); - zookeeper->trySet(zookeeper_path, new_stage); auto code = zookeeper->trySet(zookeeper_path, new_stage); if (code != Coordination::Error::ZOK) throw zkutil::KeeperException(code, zookeeper_path); - }); + } + else + { + /// Make an ephemeral node so the initiator can track if the current host is still working. + String alive_node_path = zookeeper_path + "/alive|" + current_host; + auto code = zookeeper->tryCreate(alive_node_path, "", zkutil::CreateMode::Ephemeral); + if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) + throw zkutil::KeeperException(code, alive_node_path); + + zookeeper->createIfNotExists(zookeeper_path + "/started|" + current_host, ""); + zookeeper->createIfNotExists(zookeeper_path + "/current|" + current_host + "|" + new_stage, message); + } + }); } void BackupCoordinationStageSync::setError(const String & current_host, const Exception & exception) @@ -87,6 +84,10 @@ void BackupCoordinationStageSync::setError(const String & current_host, const Ex writeStringBinary(current_host, buf); writeException(exception, buf, true); zookeeper->createIfNotExists(zookeeper_path + "/error", buf.str()); + + auto code = zookeeper->trySet(zookeeper_path, Stage::ERROR); + if (code != Coordination::Error::ZOK) + throw zkutil::KeeperException(code, zookeeper_path); }); } diff --git a/src/Backups/BackupCoordinationStageSync.h b/src/Backups/BackupCoordinationStageSync.h index 9dde4e3095f..2efaec46b3a 100644 --- a/src/Backups/BackupCoordinationStageSync.h +++ b/src/Backups/BackupCoordinationStageSync.h @@ -15,8 +15,7 @@ public: Poco::Logger * log_); /// Sets the stage of the current host and signal other hosts if there were other hosts waiting for that. - void set(const String & current_host, const String & new_stage, const String & message); - void setStageForCluster(const String & new_stage); + void set(const String & current_host, const String & new_stage, const String & message, const bool & all_hosts = false); void setError(const String & current_host, const Exception & exception); /// Sets the stage of the current host and waits until all hosts come to the same stage. diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index de05cc2b092..720ca994a40 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -368,7 +368,7 @@ void BackupsWorker::doBackup( /// Wait until all the hosts have written their backup entries. backup_coordination->waitForStage(Stage::COMPLETED); - backup_coordination->setStageForCluster(Stage::COMPLETED); + backup_coordination->setStage(Stage::COMPLETED); } else { @@ -386,7 +386,7 @@ void BackupsWorker::doBackup( writeBackupEntries(backup, std::move(backup_entries), backup_id, backup_coordination, backup_settings.internal); /// We have written our backup entries, we need to tell other hosts (they could be waiting for it). - backup_coordination->setStage(Stage::COMPLETED, ""); + backup_coordination->setStage(Stage::COMPLETED); } size_t num_files = 0; @@ -709,7 +709,7 @@ void BackupsWorker::doRestore( /// Wait until all the hosts have written their backup entries. restore_coordination->waitForStage(Stage::COMPLETED); - restore_coordination->setStageForCluster(Stage::COMPLETED); + restore_coordination->setStage(Stage::COMPLETED); } else { diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 6caae1dd741..68a13ab7846 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -21,8 +21,7 @@ public: virtual ~IBackupCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & new_stage, const String & message) = 0; - virtual void setStageForCluster(const String & new_stage) = 0; + virtual void setStage(const String & new_stage, const String & message = "") = 0; virtual void setError(const Exception & exception) = 0; virtual Strings waitForStage(const String & stage_to_wait) = 0; virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index a5c8db84c86..b4df9491c4c 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -18,8 +18,7 @@ public: virtual ~IRestoreCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & new_stage, const String & message) = 0; - virtual void setStageForCluster(const String & new_stage) = 0; /// Sets stage for cluster + virtual void setStage(const String & new_stage, const String & message = "") = 0; virtual void setError(const Exception & exception) = 0; virtual Strings waitForStage(const String & stage_to_wait) = 0; virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index 513204c931c..068c4fe7e52 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -15,10 +15,6 @@ void RestoreCoordinationLocal::setStage(const String &, const String &) { } -void RestoreCoordinationLocal::setStageForCluster(const String &) -{ -} - void RestoreCoordinationLocal::setError(const Exception &) { } diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index 0e4f4f01846..2240a25ef3d 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -19,8 +19,7 @@ public: ~RestoreCoordinationLocal() override; /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & new_stage, const String & message) override; - void setStageForCluster(const String & new_stage) override; + void setStage(const String & new_stage, const String & message = "") override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index a3541614f36..c4ecee4aaa6 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -93,17 +93,14 @@ void RestoreCoordinationRemote::createRootNodes() void RestoreCoordinationRemote::setStage(const String & new_stage, const String & message) { - stage_sync->set(current_host, new_stage, message); -} - -void RestoreCoordinationRemote::setStageForCluster(const String & new_stage) -{ - stage_sync->setStageForCluster(new_stage); + if (is_internal) + stage_sync->set(current_host, new_stage, message); + else + stage_sync->set(current_host, new_stage, /* message */ "", /* all_hosts */ true); } void RestoreCoordinationRemote::setError(const Exception & exception) { - stage_sync->setStageForCluster(Stage::ERROR); stage_sync->setError(current_host, exception); } diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 947d08a66e5..989b1c1b727 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -26,8 +26,7 @@ public: ~RestoreCoordinationRemote() override; /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & new_stage, const String & message) override; - void setStageForCluster(const String & new_stage) override; + void setStage(const String & new_stage, const String & message = "") override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; From 93a2b44702fbd185a1ec691c6e6036cfe83e6cbc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 May 2023 13:11:21 +0000 Subject: [PATCH 0214/2223] Add docs --- .../sql-reference/functions/hash-functions.md | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index fe842732b89..1e597ed9b7c 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -558,6 +558,54 @@ Result: └───────────────────────────┘ ``` +## Entropy-learned hashing + +Entropy-learned hashing is not a standalone hash function like `metroHash64`, `cityHash64`, `sipHash64` etc. Instead, it aims to preprocess +the data to be hashed in a way that a standalone hash function can be computed more efficiently while not compromising the hash quality, +i.e. the randomness of the hashes. For that, entropy-based hashing chooses a subset of the bytes in a training data set of Strings which has +the same randomness (entropy) as the original Strings. For example, if the Strings are in average 100 bytes long, and we pick a subset of 5 +bytes, then a hash function will be 95% less expensive to evaluate. For details of the method, refer to [Entropy-Learned Hashing: Constant +Time Hashing with Controllable Uniformity](https://doi.org/10.1145/3514221.3517894). + +Entropy-learned hashing has two phases: +1. A training phase on a representative but typically small set of Strings to be hashed. Function `trainEntropyLearnedHash(data, id)` + calculates a minimal partial sub-key of `data` and stores it as `id`. +2. An evaluation phase where hashes are computed using the previously calculated partial sub-keys. Function `entropyLearnedHash(data, id)` + hashes `data` using the partial subkey stored as `id`. CityHash64 is used as hash function. + +**Syntax** + +``` sql +trainEntropyLearnedHash(data, id); +entropyLearnedHash(data, id); +``` + +**Example** + +```sql +CREATE TABLE tab (col String) ENGINE=Memory; +INSERT INTO tab VALUES ('aa'), ('ba'), ('ca'); + +SELECT trainEntropyLearnedHash(col, 'id1') AS trained FROM tab; +SELECT entropyLearnedHash(col, 'id1') as hashes FROM tab; +``` + +Result: + +``` response +┌─trained─┐ +│ 0 │ +│ 0 │ +│ 0 │ +└─────────┘ + +┌───────────────hashes─┐ +│ 2603192927274642682 │ +│ 4947675599669400333 │ +│ 10783339242466472992 │ +└──────────────────────┘ +``` + ## metroHash64 Produces a 64-bit [MetroHash](http://www.jandrewrogers.com/2015/05/27/metrohash/) hash value. From fc02e9efc9eb2dcb9b7209e41eafb1e50abced7d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 May 2023 17:23:46 +0000 Subject: [PATCH 0215/2223] update fasttest a bit --- tests/ci/fast_test_check.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 89066ade2cb..fe211d79810 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -214,8 +214,11 @@ def main(): # Refuse other checks to run if fast test failed if state != "success": - if FORCE_TESTS_LABEL in pr_info.labels and state != "error": - print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") + if state == "error": + print("The status is 'error', report failure disregard the labels") + sys.exit(1) + elif FORCE_TESTS_LABEL in pr_info.labels: + print(f"'{FORCE_TESTS_LABEL}' enabled, reporting success") else: sys.exit(1) From 3fedd683ef97e61ebcc17b2f8b38feb297fbc26c Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 8 May 2023 22:28:31 +0200 Subject: [PATCH 0216/2223] speedup vol. II --- .../Serializations/SerializationDate.cpp | 2 +- src/Functions/FunctionsConversion.h | 16 ++++---- src/IO/ReadHelpers.h | 40 +++++++++++++++---- 3 files changed, 42 insertions(+), 16 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationDate.cpp b/src/DataTypes/Serializations/SerializationDate.cpp index bc2057d549e..8b4956f7826 100644 --- a/src/DataTypes/Serializations/SerializationDate.cpp +++ b/src/DataTypes/Serializations/SerializationDate.cpp @@ -77,7 +77,7 @@ void SerializationDate::serializeTextCSV(const IColumn & column, size_t row_num, void SerializationDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { DayNum value; - readCSV(value, istr); + readCSV(value, istr, time_zone); assert_cast(column).getData().push_back(value); } SerializationDate::SerializationDate(const TimezoneMixin & time_zone_) : TimezoneMixin(time_zone_) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 0f2d49f2557..6af5c44eb5e 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -981,18 +981,18 @@ void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTI } template <> -inline void parseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline void parseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { DayNum tmp(0); - readDateText(tmp, rb); + readDateText(tmp, rb, *time_zone); x = tmp; } template <> -inline void parseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline void parseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { ExtendedDayNum tmp(0); - readDateText(tmp, rb); + readDateText(tmp, rb, *time_zone); x = tmp; } @@ -1040,20 +1040,20 @@ bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateL } template <> -inline bool tryParseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline bool tryParseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { DayNum tmp(0); - if (!tryReadDateText(tmp, rb)) + if (!tryReadDateText(tmp, rb, *time_zone)) return false; x = tmp; return true; } template <> -inline bool tryParseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline bool tryParseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { ExtendedDayNum tmp(0); - if (!tryReadDateText(tmp, rb)) + if (!tryReadDateText(tmp, rb, *time_zone)) return false; x = tmp; return true; diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 16c28b89667..f9e21418a41 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -760,14 +760,14 @@ inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf) return readDateTextImpl(date, buf); } -inline bool tryReadDateText(DayNum & date, ReadBuffer & buf) +inline bool tryReadDateText(DayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { - return readDateTextImpl(date, buf); + return readDateTextImpl(date, buf, time_zone); } -inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf) +inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { - return readDateTextImpl(date, buf); + return readDateTextImpl(date, buf, time_zone); } template @@ -1160,7 +1160,7 @@ inline void readText(is_floating_point auto & x, ReadBuffer & buf) { readFloatTe inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf); } inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); } -inline void readText(DayNum & x, ReadBuffer & buf) { readDateText(x, buf); } +inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readDateText(x, buf, time_zone); } inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); } inline void readText(UUID & x, ReadBuffer & buf) { readUUIDText(x, buf); } inline void readText(IPv4 & x, ReadBuffer & buf) { readIPv4Text(x, buf); } @@ -1172,6 +1172,10 @@ template requires is_arithmetic_v inline void readQuoted(T & x, ReadBuffer & buf) { readText(x, buf); } +template +requires is_arithmetic_v +inline void readQuoted(T & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readText(x, buf, time_zone); } + inline void readQuoted(String & x, ReadBuffer & buf) { readQuotedString(x, buf); } inline void readQuoted(LocalDate & x, ReadBuffer & buf) @@ -1214,6 +1218,10 @@ template requires is_arithmetic_v inline void readDoubleQuoted(T & x, ReadBuffer & buf) { readText(x, buf); } +template + requires is_arithmetic_v +inline void readDoubleQuoted(T & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readText(x, buf, time_zone); } + inline void readDoubleQuoted(String & x, ReadBuffer & buf) { readDoubleQuotedString(x, buf); } inline void readDoubleQuoted(LocalDate & x, ReadBuffer & buf) @@ -1230,7 +1238,7 @@ inline void readDoubleQuoted(LocalDateTime & x, ReadBuffer & buf) assertChar('"', buf); } -/// CSV, for numbers, dates: quotes are optional, no special escaping rules. +/// CSV for numbers: quotes are optional, no special escaping rules. template inline void readCSVSimple(T & x, ReadBuffer & buf) { @@ -1248,6 +1256,24 @@ inline void readCSVSimple(T & x, ReadBuffer & buf) assertChar(maybe_quote, buf); } +// standalone overload for dates: to avoid instantiating DateLUTs while parsing other types +template +inline void readCSVSimple(T & x, ReadBuffer & buf, const DateLUTImpl & time_zone) +{ + if (buf.eof()) [[unlikely]] + throwReadAfterEOF(); + + char maybe_quote = *buf.position(); + + if (maybe_quote == '\'' || maybe_quote == '\"') + ++buf.position(); + + readText(x, buf, time_zone); + + if (maybe_quote == '\'' || maybe_quote == '\"') + assertChar(maybe_quote, buf); +} + template requires is_arithmetic_v inline void readCSV(T & x, ReadBuffer & buf) @@ -1257,7 +1283,7 @@ inline void readCSV(T & x, ReadBuffer & buf) inline void readCSV(String & x, ReadBuffer & buf, const FormatSettings::CSV & settings) { readCSVString(x, buf, settings); } inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); } -inline void readCSV(DayNum & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline void readCSV(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readCSVSimple(x, buf, time_zone); } inline void readCSV(LocalDateTime & x, ReadBuffer & buf) { readCSVSimple(x, buf); } inline void readCSV(UUID & x, ReadBuffer & buf) { readCSVSimple(x, buf); } inline void readCSV(IPv4 & x, ReadBuffer & buf) { readCSVSimple(x, buf); } From aa829c4ddcd243e3cc199eebde1db37f22870495 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 9 May 2023 00:21:07 +0200 Subject: [PATCH 0217/2223] better partitions hint in mutations finalization --- .../MergeTree/MergeTreeDataMergerMutator.h | 2 +- .../ReplicatedMergeTreeMutationEntry.h | 3 ++ .../MergeTree/ReplicatedMergeTreeQueue.cpp | 34 +++++++++++++------ .../MergeTree/ReplicatedMergeTreeQueue.h | 3 +- 4 files changed, 30 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 1c73bb23d33..428161ea71e 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -87,7 +87,7 @@ public: }; /// The second step of selecting parts to merge: splits parts list into a set of ranges according to can_merge_callback. - /// All parts withing a range can be merged without violating some invariants. + /// All parts within a range can be merged without violating some invariants. MergeSelectingInfo getPossibleMergeRanges( const MergeTreeData::DataPartsVector & data_parts, const AllowedMergingPredicate & can_merge_callback, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h index 8c359a57279..623a79d898f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h @@ -41,6 +41,9 @@ struct ReplicatedMergeTreeMutationEntry using BlockNumbersType = std::map; BlockNumbersType block_numbers; + /// List of partitions that do not have relevant uncommitted blocks to mutate + mutable std::unordered_set checked_partitions_cache; + /// Mutation commands which will give to MUTATE_PART entries MutationCommands commands; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 067d2a87eb3..f420145c4e2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -808,13 +808,15 @@ QueueRepresentation getQueueRepresentation(const std::listalter_version, lock); if (mutation.parts_to_do.size() != 0) { - LOG_INFO(log, "Seems like we jumped over mutation {} when downloaded part with bigger mutation number.{}", znode, " It's OK, tasks for rest parts will be skipped, but probably a lot of mutations were executed concurrently on different replicas."); + LOG_INFO(log, "Seems like we jumped over mutation {} when downloaded part with bigger mutation number. " + "It's OK, tasks for rest parts will be skipped, but probably a lot of mutations " + "were executed concurrently on different replicas.", znode); mutation.parts_to_do.clear(); } } @@ -1897,14 +1901,15 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep PartitionIdsHint partition_ids_hint; for (const auto & candidate : candidates) for (const auto & partitions : candidate->block_numbers) - partition_ids_hint.insert(partitions.first); + if (!candidate->checked_partitions_cache.contains(partitions.first)) + partition_ids_hint.insert(partitions.first); auto merge_pred = getMergePredicate(zookeeper, std::move(partition_ids_hint)); std::vector finished; for (const auto & candidate : candidates) { - if (merge_pred.isMutationFinished(candidate->znode_name, candidate->block_numbers)) + if (merge_pred.isMutationFinished(candidate->znode_name, candidate->block_numbers, candidate->checked_partitions_cache)) finished.push_back(candidate.get()); } @@ -2507,7 +2512,8 @@ std::optional> ReplicatedMergeTreeMergePredicate::getDesir } -bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const std::string & znode_name, const std::map & block_numbers) const +bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const std::string & znode_name, const std::map & block_numbers, + std::unordered_set & checked_partitions_cache) const { /// Check committing block numbers, maybe some affected inserts /// still not written to disk and committed to ZK. @@ -2516,6 +2522,10 @@ bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const std::string & z const String & partition_id = kv.first; Int64 block_num = kv.second; + /// Maybe we already know that there are no relevant uncommitted blocks + if (checked_partitions_cache.contains(partition_id)) + continue; + if (partition_ids_hint && !partition_ids_hint->contains(partition_id)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Partition id {} was not provided as hint, it's a bug", partition_id); @@ -2530,6 +2540,10 @@ bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const std::string & z return false; } } + + /// There are no committing blocks less than block_num in that partition and there's no way they can appear + /// TODO Why not to get committing blocks when pulling a mutation? We could get rid of finalization task or simplify it + checked_partitions_cache.insert(partition_id); } std::lock_guard lock(queue.state_mutex); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 2a2e47cc83f..360abceadbd 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -547,7 +547,8 @@ public: /// don't glue them together. Alter is rare operation, so it shouldn't affect performance. std::optional> getDesiredMutationVersion(const MergeTreeData::DataPartPtr & part) const; - bool isMutationFinished(const std::string & znode_name, const std::map & block_numbers) const; + bool isMutationFinished(const std::string & znode_name, const std::map & block_numbers, + std::unordered_set & checked_partitions_cache) const; /// The version of "log" node that is used to check that no new merges have appeared. int32_t getVersion() const { return merges_version; } From 70516f2f451fd9bdb82f6e3ee689772fce589a8e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 8 May 2023 23:00:20 +0000 Subject: [PATCH 0218/2223] close pipe in run() if exception happens --- src/Interpreters/TraceCollector.cpp | 128 +++++++++++++++------------- src/Interpreters/TraceCollector.h | 2 + 2 files changed, 72 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 0fc9aeb170b..b8ef7044781 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -29,6 +29,17 @@ TraceCollector::TraceCollector(std::shared_ptr trace_log_) thread = ThreadFromGlobalPool(&TraceCollector::run, this); } +void TraceCollector::tryClosePipe() +{ + try + { + TraceSender::pipe.close(); + } + catch (...) + { + tryLogCurrentException("TraceCollector"); + } +} TraceCollector::~TraceCollector() { @@ -54,14 +65,7 @@ TraceCollector::~TraceCollector() tryLogCurrentException("TraceCollector"); } - try - { - TraceSender::pipe.close(); - } - catch (...) - { - tryLogCurrentException("TraceCollector"); - } + tryClosePipe(); if (thread.joinable()) thread.join(); @@ -74,60 +78,68 @@ void TraceCollector::run() ReadBufferFromFileDescriptor in(TraceSender::pipe.fds_rw[0]); - while (true) + try { - char is_last; - readChar(is_last, in); - if (is_last) - break; - - std::string query_id; - UInt8 query_id_size = 0; - readBinary(query_id_size, in); - query_id.resize(query_id_size); - in.readStrict(query_id.data(), query_id_size); - - UInt8 trace_size = 0; - readIntBinary(trace_size, in); - - Array trace; - trace.reserve(trace_size); - - for (size_t i = 0; i < trace_size; ++i) + while (true) { - uintptr_t addr = 0; - readPODBinary(addr, in); - trace.emplace_back(static_cast(addr)); - } - - TraceType trace_type; - readPODBinary(trace_type, in); - - UInt64 thread_id; - readPODBinary(thread_id, in); - - Int64 size; - readPODBinary(size, in); - - ProfileEvents::Event event; - readPODBinary(event, in); - - ProfileEvents::Count increment; - readPODBinary(increment, in); - - if (trace_log) - { - // time and time_in_microseconds are both being constructed from the same timespec so that the - // times will be equal up to the precision of a second. - struct timespec ts; - clock_gettime(CLOCK_REALTIME, &ts); - - UInt64 time = static_cast(ts.tv_sec * 1000000000LL + ts.tv_nsec); - UInt64 time_in_microseconds = static_cast((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); - TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size, event, increment}; - trace_log->add(element); + char is_last; + readChar(is_last, in); + if (is_last) + break; + + std::string query_id; + UInt8 query_id_size = 0; + readBinary(query_id_size, in); + query_id.resize(query_id_size); + in.readStrict(query_id.data(), query_id_size); + + UInt8 trace_size = 0; + readIntBinary(trace_size, in); + + Array trace; + trace.reserve(trace_size); + + for (size_t i = 0; i < trace_size; ++i) + { + uintptr_t addr = 0; + readPODBinary(addr, in); + trace.emplace_back(static_cast(addr)); + } + + TraceType trace_type; + readPODBinary(trace_type, in); + + UInt64 thread_id; + readPODBinary(thread_id, in); + + Int64 size; + readPODBinary(size, in); + + ProfileEvents::Event event; + readPODBinary(event, in); + + ProfileEvents::Count increment; + readPODBinary(increment, in); + + if (trace_log) + { + // time and time_in_microseconds are both being constructed from the same timespec so that the + // times will be equal up to the precision of a second. + struct timespec ts; + clock_gettime(CLOCK_REALTIME, &ts); + + UInt64 time = static_cast(ts.tv_sec * 1000000000LL + ts.tv_nsec); + UInt64 time_in_microseconds = static_cast((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); + TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size, event, increment}; + trace_log->add(element); + } } } + catch (...) + { + tryClosePipe(); + throw; + } } } diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index d5bd09f1757..382e7511ac6 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -23,6 +23,8 @@ private: std::shared_ptr trace_log; ThreadFromGlobalPool thread; + void tryClosePipe(); + void run(); }; From a7e04b7576d5dc5cc2b7feb4a043eb4361d9eeb9 Mon Sep 17 00:00:00 2001 From: fhbai Date: Tue, 9 May 2023 11:36:15 +0800 Subject: [PATCH 0219/2223] fix return type --- src/Functions/array/arrayDotProduct.cpp | 59 +++++++++++-------- src/Functions/array/arrayScalarProduct.h | 22 +++++-- .../0_stateless/02708_dot_product.reference | 4 ++ .../queries/0_stateless/02708_dot_product.sql | 23 +++++++- 4 files changed, 76 insertions(+), 32 deletions(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 0d21bae90e3..7aa9f1d49c7 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -1,7 +1,12 @@ #include #include #include -#include "arrayScalarProduct.h" +#include +#include +#include +#include +#include +#include namespace DB @@ -20,31 +25,33 @@ struct NameArrayDotProduct class ArrayDotProductImpl { public: - static DataTypePtr getReturnType(const DataTypePtr & left_type, const DataTypePtr & right_type) + static DataTypePtr getReturnType(const DataTypePtr & left, const DataTypePtr & right) { - const auto & common_type = getLeastSupertype(DataTypes{left_type, right_type}); - switch (common_type->getTypeId()) - { - case TypeIndex::UInt8: - case TypeIndex::UInt16: - case TypeIndex::UInt32: - case TypeIndex::Int8: - case TypeIndex::Int16: - case TypeIndex::Int32: - case TypeIndex::UInt64: - case TypeIndex::Int64: - case TypeIndex::Float64: - return std::make_shared(); - case TypeIndex::Float32: - return std::make_shared(); - default: - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Arguments of function {} has nested type {}. " - "Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", - std::string(NameArrayDotProduct::name), - common_type->getName()); - } + using Types = TypeList; + + DataTypePtr result_type; + bool valid = castTypeToEither(Types{}, left.get(), [&](const auto & left_) { + return castTypeToEither(Types{}, right.get(), [&](const auto & right_) { + using LeftDataType = typename std::decay_t::FieldType; + using RightDataType = typename std::decay_t::FieldType; + using ResultType = typename NumberTraits::ResultOfAdditionMultiplication::Type; + if (std::is_same_v && std::is_same_v) + result_type = std::make_shared(); + else + result_type = std::make_shared>(); + return true; + }); + }); + + if (!valid) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Arguments of function {} " + "only support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", + std::string(NameArrayDotProduct::name)); + return result_type; } template @@ -67,6 +74,6 @@ REGISTER_FUNCTION(ArrayDotProduct) factory.registerFunction(); } -/// These functions are used by TupleOrArrayFunction in Function/vectorFunctions.cpp +// These functions are used by TupleOrArrayFunction in Function/vectorFunctions.cpp FunctionPtr createFunctionArrayDotProduct(ContextPtr context_) { return FunctionArrayDotProduct::create(context_); } } diff --git a/src/Functions/array/arrayScalarProduct.h b/src/Functions/array/arrayScalarProduct.h index 5c36f2492c6..374a2d8a194 100644 --- a/src/Functions/array/arrayScalarProduct.h +++ b/src/Functions/array/arrayScalarProduct.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -154,12 +155,23 @@ public: { switch (result_type->getTypeId()) { - case TypeIndex::Float32: - return executeWithResultType(arguments); - break; - case TypeIndex::Float64: - return executeWithResultType(arguments); + #define SUPPORTED_TYPE(type) \ + case TypeIndex::type: \ + return executeWithResultType(arguments); \ break; + + SUPPORTED_TYPE(UInt8) + SUPPORTED_TYPE(UInt16) + SUPPORTED_TYPE(UInt32) + SUPPORTED_TYPE(UInt64) + SUPPORTED_TYPE(Int8) + SUPPORTED_TYPE(Int16) + SUPPORTED_TYPE(Int32) + SUPPORTED_TYPE(Int64) + SUPPORTED_TYPE(Float32) + SUPPORTED_TYPE(Float64) + #undef SUPPORTED_TYPE + default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type {}", result_type->getName()); } diff --git a/tests/queries/0_stateless/02708_dot_product.reference b/tests/queries/0_stateless/02708_dot_product.reference index 7106b870fab..45e53871aa2 100644 --- a/tests/queries/0_stateless/02708_dot_product.reference +++ b/tests/queries/0_stateless/02708_dot_product.reference @@ -4,7 +4,11 @@ 376.5 230 0 +0 Float64 Float32 Float64 Float64 +UInt16 +UInt64 +Int64 diff --git a/tests/queries/0_stateless/02708_dot_product.sql b/tests/queries/0_stateless/02708_dot_product.sql index 46450ae6394..e94cb577bf4 100644 --- a/tests/queries/0_stateless/02708_dot_product.sql +++ b/tests/queries/0_stateless/02708_dot_product.sql @@ -1,13 +1,16 @@ SELECT dotProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); + SELECT scalarProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); + SELECT arrayDotProduct([12, 2.22, 302], [1.32, 231.2, 11.1]); SELECT dotProduct([1.3, 2, 3, 4, 5], [222, 12, 5.3, 2, 8]); SELECT dotProduct([1, 1, 1, 1, 1], [222, 12, 0, -12, 8]); -SELECT round(dotProduct([-1, 2, 3.002], [2, 3.4, 4]) - dotProduct((-1, 2, 3.002), (2, 3.4, 4)), 2); +SELECT round(dotProduct([12345678901234567], [1]) - dotProduct(tuple(12345678901234567), tuple(1)), 2); +SELECT round(dotProduct([-1, 2, 3.002], [2, 3.4, 4]) - dotProduct((-1, 2, 3.002), (2, 3.4, 4)), 2); DROP TABLE IF EXISTS product_fp64_fp64; CREATE TABLE product_fp64_fp64 (x Array(Float64), y Array(Float64)) engine = MergeTree() order by x; @@ -32,3 +35,21 @@ CREATE TABLE product_uint8_fp64 (x Array(UInt8), y Array(Float64)) engine = Merg INSERT INTO TABLE product_uint8_fp64 (x, y) values ([1, 2], [3, 4]); SELECT toTypeName(dotProduct(x, y)) from product_uint8_fp64; DROP TABLE product_uint8_fp64; + +DROP TABLE IF EXISTS product_uint8_uint8; +CREATE TABLE product_uint8_uint8 (x Array(UInt8), y Array(UInt8)) engine = MergeTree() order by x; +INSERT INTO TABLE product_uint8_uint8 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_uint8_uint8; +DROP TABLE product_uint8_uint8; + +DROP TABLE IF EXISTS product_uint64_uint64; +CREATE TABLE product_uint64_uint64 (x Array(UInt64), y Array(UInt64)) engine = MergeTree() order by x; +INSERT INTO TABLE product_uint64_uint64 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_uint64_uint64; +DROP TABLE product_uint64_uint64; + +DROP TABLE IF EXISTS product_int32_uint64; +CREATE TABLE product_int32_uint64 (x Array(Int32), y Array(UInt64)) engine = MergeTree() order by x; +INSERT INTO TABLE product_int32_uint64 (x, y) values ([1, 2], [3, 4]); +SELECT toTypeName(dotProduct(x, y)) from product_int32_uint64; +DROP TABLE product_int32_uint64; From 79398f612f4a94cee996d6256155c5e229ee090b Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Tue, 9 May 2023 11:50:38 +0800 Subject: [PATCH 0220/2223] fix style --- src/Functions/array/arrayDotProduct.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 7aa9f1d49c7..e3c80775f1b 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -32,8 +32,10 @@ public: DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64>; DataTypePtr result_type; - bool valid = castTypeToEither(Types{}, left.get(), [&](const auto & left_) { - return castTypeToEither(Types{}, right.get(), [&](const auto & right_) { + bool valid = castTypeToEither(Types{}, left.get(), [&](const auto & left_) + { + return castTypeToEither(Types{}, right.get(), [&](const auto & right_) + { using LeftDataType = typename std::decay_t::FieldType; using RightDataType = typename std::decay_t::FieldType; using ResultType = typename NumberTraits::ResultOfAdditionMultiplication::Type; From 1751ccc7aca3830d21a06ec4f09bd28bf9254f79 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 9 May 2023 14:18:04 +0200 Subject: [PATCH 0221/2223] fix stateless --- src/Functions/FunctionsConversion.h | 15 ++++++++++++--- src/IO/ReadHelpers.h | 12 ++++++++---- 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 6af5c44eb5e..e0e188f68c2 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -439,7 +439,7 @@ struct ToDate32Transform32Or64Signed static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { - static const Int32 daynum_min_offset = -static_cast(DateLUT::instance().getDayNumOffsetEpoch()); + static const Int32 daynum_min_offset = -static_cast(time_zone.getDayNumOffsetEpoch()); if (from < daynum_min_offset) return daynum_min_offset; return (from < DATE_LUT_MAX_EXTEND_DAY_NUM) @@ -830,8 +830,11 @@ struct ConvertImpl(*col_with_type_and_name.type); const DateLUTImpl * time_zone = nullptr; + + if constexpr (std::is_same_v) + time_zone = &DateLUT::instance(); /// For argument of Date or DateTime type, second argument with time zone could be specified. - if constexpr (std::is_same_v || std::is_same_v || std::is_same_v) + if constexpr (std::is_same_v || std::is_same_v) { auto non_null_args = createBlockWithNestedColumns(arguments); time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); @@ -1193,7 +1196,7 @@ struct ConvertThroughParsing const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr; const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr; - /// For conversion to DateTime type, second argument with time zone could be specified. + /// For conversion to Date or DateTime type, second argument with time zone could be specified. if constexpr (std::is_same_v || to_datetime64) { const auto result_type = removeNullable(res_type); @@ -1206,6 +1209,12 @@ struct ConvertThroughParsing if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort || parsing_mode == ConvertFromStringParsingMode::BestEffortUS) utc_time_zone = &DateLUT::instance("UTC"); } + else if constexpr (std::is_same_v || std::is_same_v) + { + // Timezone is more or less dummy when parsing Date/Date32 from string. + local_time_zone = &DateLUT::instance(); + utc_time_zone = &DateLUT::instance("UTC"); + } const IColumn * col_from = arguments[0].column.get(); const ColumnString * col_from_string = checkAndGetColumn(col_from); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index f9e21418a41..ea565d11914 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -718,7 +718,7 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLU return false; ExtendedDayNum ret = date_lut.makeDayNum(local_date.year(), local_date.month(), local_date.day()); - convertToDayNum(date,ret); + convertToDayNum(date, ret); return ReturnType(true); } @@ -1159,8 +1159,11 @@ inline bool tryReadText(IPv6 & x, ReadBuffer & buf) { return tryReadIPv6Text(x, inline void readText(is_floating_point auto & x, ReadBuffer & buf) { readFloatText(x, buf); } inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf); } + +inline void readText(DayNum & x, ReadBuffer & buf) { readDateText(x, buf); } +inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readDateText(x, buf, time_zone); } + inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); } -inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readDateText(x, buf, time_zone); } inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); } inline void readText(UUID & x, ReadBuffer & buf) { readUUIDText(x, buf); } inline void readText(IPv4 & x, ReadBuffer & buf) { readIPv4Text(x, buf); } @@ -1219,7 +1222,7 @@ requires is_arithmetic_v inline void readDoubleQuoted(T & x, ReadBuffer & buf) { readText(x, buf); } template - requires is_arithmetic_v +requires is_arithmetic_v inline void readDoubleQuoted(T & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readText(x, buf, time_zone); } inline void readDoubleQuoted(String & x, ReadBuffer & buf) { readDoubleQuotedString(x, buf); } @@ -1283,7 +1286,8 @@ inline void readCSV(T & x, ReadBuffer & buf) inline void readCSV(String & x, ReadBuffer & buf, const FormatSettings::CSV & settings) { readCSVString(x, buf, settings); } inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); } -inline void readCSV(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readCSVSimple(x, buf, time_zone); } +inline void readCSV(DayNum & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline void readCSV(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readCSVSimple(x, buf, time_zone); } inline void readCSV(LocalDateTime & x, ReadBuffer & buf) { readCSVSimple(x, buf); } inline void readCSV(UUID & x, ReadBuffer & buf) { readCSVSimple(x, buf); } inline void readCSV(IPv4 & x, ReadBuffer & buf) { readCSVSimple(x, buf); } From ddbad79c5e67518acebbacaad5be0cad3967ac67 Mon Sep 17 00:00:00 2001 From: tpanetti Date: Fri, 5 May 2023 12:19:35 -0700 Subject: [PATCH 0222/2223] Change SHOW COLUMNS query to display MySQL types in MySQL Compatibility mode This updates the SHOW COLUMN SQL query to display MySQL types when this query is issued by a client connected via MySQL Compatibility port --- .../InterpreterShowColumnsQuery.cpp | 76 +++++++ .../InterpreterShowColumnsQuery.h | 1 + ...show_columns_mysql_compatibility.reference | 213 ++++++++++++++++++ .../02726_show_columns_mysql_compatibility.sh | 115 ++++++++++ 4 files changed, 405 insertions(+) create mode 100644 tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference create mode 100755 tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index c86d3c753c4..c545c621abb 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -45,6 +45,14 @@ SELECT // TODO Interpret query.extended. It is supposed to show internal/virtual columns. Need to fetch virtual column names, see // IStorage::getVirtuals(). We can't easily do that via SQL. + // If connected via MySQL Compatibility mode, convert ClickHouse types to MySQL + if (getContext()->getClientInfo().interface == DB::ClientInfo::Interface::MYSQL) + { + rewritten_query += getMySQLQuery(); + } + else { + rewritten_query += "SELECT name AS field, type AS type, startsWith(type, 'Nullable') AS null, trim(concatWithSeparator(' ', if(is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, if(default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra "; + } if (query.full) { /// "Full" mode is mostly for MySQL compat @@ -88,6 +96,74 @@ WHERE return rewritten_query; } +String InterpreterShowColumnsQuery::getMySQLQuery() +{ + String mysql_specific_query; + + mysql_specific_query = R"(SELECT name AS field, + CASE + WHEN startsWith(type, 'Nullable') THEN + CASE + WHEN substring(type, 10, length(type) - 10) IN ('UInt8', 'Int8') THEN 'tinyint' + WHEN substring(type, 10, length(type) - 10) IN ('UInt16', 'Int16') THEN 'smallint' + WHEN substring(type, 10, length(type) - 10) IN ('UInt32', 'Int32') THEN 'int' + WHEN substring(type, 10, length(type) - 10) IN ('UInt64', 'Int64', 'UInt128', 'Int128', 'UInt256', 'Int256') THEN 'bigint' + WHEN substring(type, 10, length(type) - 10) = 'Float32' THEN 'float' + WHEN substring(type, 10, length(type) - 10) = 'Float64' THEN 'double' + WHEN substring(type, 10, length(type) - 10) LIKE 'Decimal%' THEN 'decimal' + WHEN substring(type, 10, length(type) - 10) = 'Boolean' THEN 'tinyint' + WHEN substring(type, 10, length(type) - 10) = 'String' THEN 'text' + WHEN substring(type, 10, length(type) - 10) LIKE 'FixedString%' THEN 'text' + WHEN substring(type, 10, length(type) - 10) LIKE 'Date%' THEN 'date' + WHEN substring(type, 10, length(type) - 10) LIKE 'DateTime%' THEN 'datetime' + WHEN substring(type, 10, length(type) - 10) = 'JSON' THEN 'json' + WHEN substring(type, 10, length(type) - 10) = 'UUID' THEN 'binary' + WHEN substring(type, 10, length(type) - 10) LIKE 'Enum%' THEN 'enum' + WHEN substring(type, 10, length(type) - 10) LIKE 'LowCardinality%' THEN 'text' + WHEN substring(type, 10, length(type) - 10) LIKE 'Array%' THEN 'json' + WHEN substring(type, 10, length(type) - 10) LIKE 'Map%' THEN 'json' + WHEN substring(type, 10, length(type) - 10) IN ('SimpleAggregateFunction', 'AggregateFunction') THEN 'text' + WHEN substring(type, 10, length(type) - 10) = 'Nested' THEN 'json' + WHEN substring(type, 10, length(type) - 10) LIKE 'Tuple%' THEN 'json' + WHEN substring(type, 10, length(type) - 10) LIKE 'IPv%' THEN 'text' + WHEN substring(type, 10, length(type) - 10) IN ('Expression', 'Set', 'Nothing', 'Interval') THEN 'text' + ELSE substring(type, 10, length(type) - 10) + END + ELSE + CASE + WHEN type IN ('UInt8', 'Int8') THEN 'tinyint' + WHEN type IN ('UInt16', 'Int16') THEN 'smallint' + WHEN type IN ('UInt32', 'Int32') THEN 'int' + WHEN type IN ('UInt64', 'Int64', 'UInt128', 'Int128', 'UInt256', 'Int256') THEN 'bigint' + WHEN type = 'Float32' THEN 'float' + WHEN type = 'Float64' THEN 'double' + WHEN type LIKE 'Decimal%' THEN 'decimal' + WHEN type = 'Boolean' THEN 'tinyint' + WHEN type = 'String' THEN 'text' + WHEN type LIKE 'FixedString%' THEN 'text' + WHEN type LIKE 'Date%' THEN 'date' + WHEN type LIKE 'DateTime%' THEN 'datetime' + WHEN type = 'JSON' THEN 'json' + WHEN type = 'UUID' THEN 'binary' + WHEN type LIKE 'Enum%' THEN 'enum' + WHEN type LIKE 'LowCardinality%' THEN 'text' + WHEN type LIKE 'Array%' THEN 'json' + WHEN type LIKE 'Map%' THEN 'json' + WHEN type IN ('SimpleAggregateFunction', 'AggregateFunction') THEN 'text' + WHEN type = 'Nested' THEN 'json' + WHEN type LIKE 'Tuple%' THEN 'json' + WHEN type LIKE 'IPv%' THEN 'text' + WHEN type IN ('Expression', 'Set', 'Nothing', 'Interval') THEN 'text' + ELSE type + END + END AS type, + startsWith(type, 'Nullable') AS null, + trim(concatWithSeparator(' ', if(is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, + if(default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, + '' AS extra )"; + + return mysql_specific_query.str(); +} BlockIO InterpreterShowColumnsQuery::execute() { diff --git a/src/Interpreters/InterpreterShowColumnsQuery.h b/src/Interpreters/InterpreterShowColumnsQuery.h index ee6dcabd97b..b843a163978 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.h +++ b/src/Interpreters/InterpreterShowColumnsQuery.h @@ -26,6 +26,7 @@ private: ASTPtr query_ptr; String getRewrittenQuery(); + String getMySQLQuery(); }; diff --git a/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference new file mode 100644 index 00000000000..c9ad94a34c4 --- /dev/null +++ b/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference @@ -0,0 +1,213 @@ +Drop tables if they exist +Create tab table +Create pseudo-random database name +Create tab duplicate table +Run MySQL test +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_int json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uint64 bigint 0 PRI SOR NULL +uuid_value binary 0 NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_int json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uint64 bigint 0 PRI SOR NULL +uuid_value binary 0 NULL +field type null key default extra collation comment privileges +array_value json 0 NULL NULL +boolean_value tinyint 0 NULL NULL +date32_value date 0 NULL NULL +date_value date 0 NULL NULL +datetime64_value date 0 NULL NULL +datetime_value date 0 NULL NULL +decimal_value decimal 0 NULL NULL +enum_value enum 0 NULL NULL +fixed_string_value text 0 NULL NULL +float32 float 0 NULL NULL +float64 double 0 NULL NULL +int32 int 0 NULL NULL +ipv4_value text 0 NULL NULL +ipv6_value text 0 NULL NULL +json_value text 0 NULL NULL +low_cardinality text 0 NULL NULL +map_value json 0 NULL NULL +nested.nested_int json 0 NULL NULL +nested.nested_string json 0 NULL NULL +nullable_value int 0 NULL NULL +string_value text 0 NULL NULL +tuple_value json 0 NULL NULL +uint64 bigint 0 PRI SOR NULL NULL +uuid_value binary 0 NULL NULL +field type null key default extra +int32 int 0 NULL +nested.nested_int json 0 NULL +uint64 bigint 0 PRI SOR NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uuid_value binary 0 NULL +field type null key default extra +int32 int 0 NULL +nested.nested_int json 0 NULL +uint64 bigint 0 PRI SOR NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uuid_value binary 0 NULL +field type null key default extra +int32 int 0 NULL +nested.nested_int json 0 NULL +uint64 bigint 0 PRI SOR NULL +field type null key default extra +array_value json 0 NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_int json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uint64 bigint 0 PRI SOR NULL +uuid_value binary 0 NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_int json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uint64 bigint 0 PRI SOR NULL +uuid_value binary 0 NULL +field type null key default extra +array_value json 0 NULL +boolean_value tinyint 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value date 0 NULL +datetime_value date 0 NULL +decimal_value decimal 0 NULL +enum_value enum 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value text 0 NULL +low_cardinality text 0 NULL +map_value json 0 NULL +nested.nested_int json 0 NULL +nested.nested_string json 0 NULL +nullable_value int 0 NULL +string_value text 0 NULL +tuple_value json 0 NULL +uint64 bigint 0 PRI SOR NULL +uuid_value binary 0 NULL diff --git a/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh b/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh new file mode 100755 index 00000000000..5324496edd3 --- /dev/null +++ b/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh @@ -0,0 +1,115 @@ +#!/bin/bash + +# This script tests the MySQL compatibility of the SHOW COLUMNS command in ClickHouse +USER="default" +PASSWORD="" +HOST="127.0.0.1" +PORT=9004 + +# First run the clickhouse test to create the ClickHouse Tables + +echo "Drop tables if they exist" +${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS tab" +${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde" +${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde.tab" + +echo "Create tab table " +${CLICKHOUSE_LOCAL} --query " + CREATE TABLE tab + ( + uint64 UInt64, + int32 Nullable(Int32), + float32 Float32, + float64 Float64, + decimal_value Decimal(10, 2), + boolean_value UInt8, -- Use 0 for false, 1 for true + string_value String, + fixed_string_value FixedString(10), + date_value Date, + date32_value Date32, + datetime_value DateTime, + datetime64_value DateTime64(3), + json_value String, -- Store JSON as a string + uuid_value UUID, + enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), + low_cardinality LowCardinality(String), + array_value Array(Int32), + map_value Map(String, Int32), + tuple_value Tuple(Int32, String), + nullable_value Nullable(Int32), + ipv4_value IPv4, + ipv6_value IPv6, + nested Nested + ( + nested_int Int32, + nested_string String + ) + ) ENGINE = MergeTree + ORDER BY uint64; + " + + +echo "Create pseudo-random database name" +${CLICKHOUSE_LOCAL} --query "CREATE DATABASE database_123456789abcde;" + +echo "Create tab duplicate table" +${CLICKHOUSE_LOCAL} --query " + CREATE TABLE database_123456789abcde.tab + ( + uint64 UInt64, + int32 Nullable(Int32), + float32 Float32, + float64 Float64, + decimal_value Decimal(10, 2), + boolean_value UInt8, -- Use 0 for false, 1 for true + string_value String, + fixed_string_value FixedString(10), + date_value Date, + date32_value Date32, + datetime_value DateTime, + datetime64_value DateTime64(3), + json_value String, -- Store JSON as a string + uuid_value UUID, + enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), + low_cardinality LowCardinality(String), + array_value Array(Int32), + map_value Map(String, Int32), + tuple_value Tuple(Int32, String), + nullable_value Nullable(Int32), + ipv4_value IPv4, + ipv6_value IPv6, + nested Nested + ( + nested_int Int32, + nested_string String + ) + ) ENGINE = MergeTree + ORDER BY uint64; + " + +# Write sql to temp file +TEMP_FILE=$(mktemp) + +cat < $TEMP_FILE +SHOW COLUMNS FROM tab; +SHOW EXTENDED COLUMNS FROM tab; +SHOW FULL COLUMNS FROM tab; +SHOW COLUMNS FROM tab LIKE '%int%'; +SHOW COLUMNS FROM tab NOT LIKE '%int%'; +SHOW COLUMNS FROM tab ILIKE '%INT%'; +SHOW COLUMNS FROM tab NOT ILIKE '%INT%'; +SHOW COLUMNS FROM tab WHERE field LIKE '%int%'; +SHOW COLUMNS FROM tab LIMIT 1; +SHOW COLUMNS FROM tab; +SHOW COLUMNS FROM tab FROM database_123456789abcde; +SHOW COLUMNS FROM database_123456789abcde.tab; +DROP DATABASE database_123456789abcde; +DROP TABLE tab; +EOT + +# Now run the MySQL test script on the ClickHouse DB +echo "Run MySQL test" +mysql --user="$USER" --password="$PASSWORD" --host="$HOST" --port="$PORT" < $TEMP_FILE + +# Clean up the temp file +rm $TEMP_FILE From 815cc8de9bb843d6b0d54b5507f08533f52b8190 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 9 May 2023 19:43:39 +0000 Subject: [PATCH 0223/2223] removed unnecessary check for thread's joinability --- src/Interpreters/TraceCollector.cpp | 25 +++++++++++-------------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index b8ef7044781..cb00e37df69 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -45,20 +45,15 @@ TraceCollector::~TraceCollector() { try { - if (thread.joinable()) - { - /** Sends TraceCollector stop message - * - * Each sequence of data for TraceCollector thread starts with a boolean flag. - * If this flag is true, TraceCollector must stop reading trace_pipe and exit. - * This function sends flag with a true value to stop TraceCollector gracefully. - */ - WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); - writeChar(true, out); - out.next(); - } - else - LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined"); + /** Sends TraceCollector stop message + * + * Each sequence of data for TraceCollector thread starts with a boolean flag. + * If this flag is true, TraceCollector must stop reading trace_pipe and exit. + * This function sends flag with a true value to stop TraceCollector gracefully. + */ + WriteBufferFromFileDescriptor out(TraceSender::pipe.fds_rw[1]); + writeChar(true, out); + out.next(); } catch (...) { @@ -69,6 +64,8 @@ TraceCollector::~TraceCollector() if (thread.joinable()) thread.join(); + else + LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined"); } From 297188ce583a94f9942f7fd141a85dbdcfdcd587 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 9 May 2023 22:37:25 +0200 Subject: [PATCH 0224/2223] fix Date32 --- src/DataTypes/Serializations/SerializationDate32.cpp | 11 +++++++---- src/DataTypes/Serializations/SerializationDate32.h | 5 ++++- src/Functions/FunctionsConversion.h | 7 +++++-- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationDate32.cpp b/src/DataTypes/Serializations/SerializationDate32.cpp index ef92202f89d..8dcaee8d266 100644 --- a/src/DataTypes/Serializations/SerializationDate32.cpp +++ b/src/DataTypes/Serializations/SerializationDate32.cpp @@ -11,7 +11,7 @@ namespace DB void SerializationDate32::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeDateText(ExtendedDayNum(assert_cast(column).getData()[row_num]), ostr); + writeDateText(ExtendedDayNum(assert_cast(column).getData()[row_num]), ostr, time_zone); } void SerializationDate32::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -24,7 +24,7 @@ void SerializationDate32::deserializeWholeText(IColumn & column, ReadBuffer & is void SerializationDate32::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { ExtendedDayNum x; - readDateText(x, istr); + readDateText(x, istr, time_zone); assert_cast(column).getData().push_back(x); } @@ -44,7 +44,7 @@ void SerializationDate32::deserializeTextQuoted(IColumn & column, ReadBuffer & i { ExtendedDayNum x; assertChar('\'', istr); - readDateText(x, istr); + readDateText(x, istr, time_zone); assertChar('\'', istr); assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } @@ -60,7 +60,7 @@ void SerializationDate32::deserializeTextJSON(IColumn & column, ReadBuffer & ist { ExtendedDayNum x; assertChar('"', istr); - readDateText(x, istr); + readDateText(x, istr, time_zone); assertChar('"', istr); assert_cast(column).getData().push_back(x); } @@ -78,4 +78,7 @@ void SerializationDate32::deserializeTextCSV(IColumn & column, ReadBuffer & istr readCSV(value, istr); assert_cast(column).getData().push_back(value.getExtenedDayNum()); } +SerializationDate32::SerializationDate32(const TimezoneMixin & time_zone_) : TimezoneMixin(time_zone_) +{ +} } diff --git a/src/DataTypes/Serializations/SerializationDate32.h b/src/DataTypes/Serializations/SerializationDate32.h index 484b4f4a958..e8e8f1a74d6 100644 --- a/src/DataTypes/Serializations/SerializationDate32.h +++ b/src/DataTypes/Serializations/SerializationDate32.h @@ -1,12 +1,15 @@ #pragma once #include +#include namespace DB { -class SerializationDate32 final : public SerializationNumber +class SerializationDate32 final : public SerializationNumber, public TimezoneMixin { public: + explicit SerializationDate32(const TimezoneMixin & time_zone_ = TimezoneMixin()); + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e0e188f68c2..2f751e72222 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -731,7 +731,10 @@ struct FormatImpl template static ReturnType execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl * time_zone) { + std::cerr << "BEFORE: " << std::endl; + std::cerr << time_zone->getTimeZone() << std::endl; writeDateText(ExtendedDayNum(x), wb, *time_zone); + std::cerr << "AFTER" << std::endl; return ReturnType(true); } }; @@ -831,7 +834,7 @@ struct ConvertImpl) + if constexpr (std::is_same_v || std::is_same_v) time_zone = &DateLUT::instance(); /// For argument of Date or DateTime type, second argument with time zone could be specified. if constexpr (std::is_same_v || std::is_same_v) @@ -1765,7 +1768,7 @@ public: || std::is_same_v // toDate(value[, timezone : String]) || std::is_same_v // TODO: shall we allow timestamp argument for toDate? DateTime knows nothing about timezones and this argument is ignored below. - // toDate(value[, timezone : String]) + // toDate32(value[, timezone : String]) || std::is_same_v // toDateTime(value[, timezone: String]) || std::is_same_v From 8d0644e79301a9a0ccf67b66d44c43e4766d8aa7 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 9 May 2023 23:02:03 +0200 Subject: [PATCH 0225/2223] cleanup --- src/Functions/FunctionsConversion.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 2f751e72222..b10d9f4a31a 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -731,10 +731,7 @@ struct FormatImpl template static ReturnType execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl * time_zone) { - std::cerr << "BEFORE: " << std::endl; - std::cerr << time_zone->getTimeZone() << std::endl; writeDateText(ExtendedDayNum(x), wb, *time_zone); - std::cerr << "AFTER" << std::endl; return ReturnType(true); } }; From 6f53ee289db7939188ce1d9bcbe97b4203205d58 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 9 May 2023 23:26:56 +0200 Subject: [PATCH 0226/2223] fix another issue, add a test --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 16 +++++++++++-- ...02439_merge_selecting_partitions.reference | 1 + .../02439_merge_selecting_partitions.sql | 24 +++++++++++++++++++ 3 files changed, 39 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02439_merge_selecting_partitions.reference create mode 100644 tests/queries/0_stateless/02439_merge_selecting_partitions.sql diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index f420145c4e2..d2de7951677 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -2135,11 +2135,23 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( for (const String & partition : partitions) paths.push_back(fs::path(queue.zookeeper_path) / "block_numbers" / partition); - auto locks_children = zookeeper->getChildren(paths); + auto locks_children = zookeeper->tryGetChildren(paths); for (size_t i = 0; i < partitions.size(); ++i) { - Strings partition_block_numbers = locks_children[i].names; + auto & response = locks_children[i]; + if (response.error != Coordination::Error::ZOK && !partition_ids_hint) + throw Coordination::Exception(response.error, paths[i]); + + if (response.error != Coordination::Error::ZOK) + { + /// Probably a wrong hint was provided (it's ok if a user passed non-existing partition to OPTIMIZE) + LOG_WARNING(queue.log, "Partition id '{}' was provided as a hint, but there's not such partition in ZooKeeper", partitions[i]); + partition_ids_hint->erase(partitions[i]); + continue; + } + + Strings partition_block_numbers = response.names; for (const String & entry : partition_block_numbers) { if (!startsWith(entry, "block-")) diff --git a/tests/queries/0_stateless/02439_merge_selecting_partitions.reference b/tests/queries/0_stateless/02439_merge_selecting_partitions.reference new file mode 100644 index 00000000000..7dae4cb5eff --- /dev/null +++ b/tests/queries/0_stateless/02439_merge_selecting_partitions.reference @@ -0,0 +1 @@ +/test/02439/default/block_numbers/123 diff --git a/tests/queries/0_stateless/02439_merge_selecting_partitions.sql b/tests/queries/0_stateless/02439_merge_selecting_partitions.sql new file mode 100644 index 00000000000..28bb0014cca --- /dev/null +++ b/tests/queries/0_stateless/02439_merge_selecting_partitions.sql @@ -0,0 +1,24 @@ + +create table rmt (n int, m int) engine=ReplicatedMergeTree('/test/02439/{database}', '1') partition by n order by n; +insert into rmt select number, number from numbers(50); +insert into rmt values (1, 2); +insert into rmt values (1, 3); +insert into rmt values (1, 4); +insert into rmt values (1, 5); +insert into rmt values (1, 6); +insert into rmt values (1, 7); +insert into rmt values (1, 8); +insert into rmt values (1, 9); +-- there's nothing to merge in all partitions but '1' + +optimize table rmt partition tuple(123); + +set optimize_throw_if_noop=1; +optimize table rmt partition tuple(123); -- { serverError CANNOT_ASSIGN_OPTIMIZE } + +select sleepEachRow(3) as higher_probability_of_reproducing_the_issue format Null; +system flush logs; + +-- it should not list unneeded partitions where we cannot merge anything +select distinct path from system.zookeeper_log where path like '/test/02439/' || currentDatabase() || '/block_numbers/%' + and op_num in ('List', 'SimpleList', 'FilteredList') and path not like '%/block_numbers/1' From 93161f70d15e4a3587ddde494532a874c6b432d0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 10 May 2023 10:45:43 +0000 Subject: [PATCH 0227/2223] Correct suffix handling for ranges --- .../Transforms/FillingTransform.cpp | 84 +++++++++++-------- src/Processors/Transforms/FillingTransform.h | 9 +- ...2730_with_fill_by_sorting_prefix.reference | 51 +++++++++++ .../02730_with_fill_by_sorting_prefix.sql | 19 +++++ 4 files changed, 126 insertions(+), 37 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index b9c3341948a..5fa89e89a4b 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -246,6 +246,7 @@ FillingTransform::FillingTransform( sort_prefix.push_back(desc); } logDebug("sort prefix", dumpSortDescription(sort_prefix)); + last_range_sort_prefix.reserve(sort_prefix.size()); /// TODO: check conflict in positions between interpolate and sorting prefix columns @@ -334,6 +335,8 @@ using MutableColumnRawPtrs = std::vector; static void insertFromFillingRow(const MutableColumnRawPtrs & filling_columns, const MutableColumnRawPtrs & interpolate_columns, const MutableColumnRawPtrs & other_columns, const FillingRow & filling_row, const Block & interpolate_block) { + logDebug("insertFromFillingRow", filling_row); + for (size_t i = 0, size = filling_columns.size(); i < size; ++i) { if (filling_row[i].isNull()) @@ -410,27 +413,6 @@ void FillingTransform::initColumns( bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, MutableColumns & result_columns) { - /// true means we'll generate rows for empty result set - const bool no_data_processed = last_row.empty(); - - logDebug("generateSuffixIfNeeded() filling_row", filling_row); - logDebug("generateSuffixIfNeeded() next_row", next_row); - logDebug("generateSuffixIfNeeded() no_data_processed", no_data_processed); - - /// Determines should we insert filling row before start generating next rows. - bool should_insert_first = next_row < filling_row || no_data_processed; - - for (size_t i = 0, size = filling_row.size(); i < size; ++i) - next_row[i] = filling_row.getFillDescription(i).fill_to; - - logDebug("generateSuffixIfNeeded() next_row updated", next_row); - - if (!no_data_processed && filling_row >= next_row) - { - logDebug("generateSuffixIfNeeded()", "no need to generate suffix"); - return false; - } - Columns input_fill_columns; Columns input_interpolate_columns; Columns input_sort_prefix_columns; @@ -452,16 +434,32 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut res_sort_prefix_columns, res_other_columns); - if (no_data_processed) - filling_row.initFromDefaults(); + return generateSuffixIfNeeded(result_columns, res_fill_columns, res_interpolate_columns, res_sort_prefix_columns, res_other_columns); +} - /// if any rows was processed and there is sort prefix, get last row sort prefix - Columns last_row_sort_prefix; - if (!last_row.empty()) +bool FillingTransform::generateSuffixIfNeeded( + const MutableColumns & result_columns, + MutableColumnRawPtrs res_fill_columns, + MutableColumnRawPtrs res_interpolate_columns, + MutableColumnRawPtrs res_sort_prefix_columns, + MutableColumnRawPtrs res_other_columns) +{ + logDebug("generateSuffixIfNeeded() filling_row", filling_row); + logDebug("generateSuffixIfNeeded() next_row", next_row); + + /// Determines should we insert filling row before start generating next rows. + bool should_insert_first = next_row < filling_row; + logDebug("should_insert_first", should_insert_first); + + for (size_t i = 0, size = filling_row.size(); i < size; ++i) + next_row[i] = filling_row.getFillDescription(i).fill_to; + + logDebug("generateSuffixIfNeeded() next_row updated", next_row); + + if (filling_row >= next_row) { - last_row_sort_prefix.reserve(sort_prefix_positions.size()); - for (const size_t pos : sort_prefix_positions) - last_row_sort_prefix.push_back(last_row[pos]); + logDebug("generateSuffixIfNeeded()", "no need to generate suffix"); + return false; } Block interpolate_block; @@ -470,8 +468,8 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); /// fulfill sort prefix columns with last row values or defaults - if (!last_row_sort_prefix.empty()) - copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); + if (!last_range_sort_prefix.empty()) + copyRowFromColumns(res_sort_prefix_columns, last_range_sort_prefix, 0); else for (auto * sort_prefix_column : res_sort_prefix_columns) sort_prefix_column->insertDefault(); @@ -482,8 +480,8 @@ bool FillingTransform::generateSuffixIfNeeded(const Columns & input_columns, Mut interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); /// fulfill sort prefix columns with last row values or defaults - if (!last_row_sort_prefix.empty()) - copyRowFromColumns(res_sort_prefix_columns, last_row_sort_prefix, 0); + if (!last_range_sort_prefix.empty()) + copyRowFromColumns(res_sort_prefix_columns, last_range_sort_prefix, 0); else for (auto * sort_prefix_column : res_sort_prefix_columns) sort_prefix_column->insertDefault(); @@ -591,7 +589,6 @@ void FillingTransform::transformRange( interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); - logDebug("filling_row should_insert_first", filling_row); } while (filling_row.next(next_row)) @@ -599,7 +596,6 @@ void FillingTransform::transformRange( interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); - logDebug("filling_row", filling_row); } copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); @@ -607,6 +603,15 @@ void FillingTransform::transformRange( copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); copyRowFromColumns(res_other_columns, input_other_columns, row_ind); } + + /// save sort prefix of last row in the range, it's used to generate suffix + last_range_sort_prefix.clear(); + for (const auto & sort_prefix_column : input_sort_prefix_columns) + { + auto column = sort_prefix_column->cloneEmpty(); + column->insertFrom(*sort_prefix_column, range_end - 1); + last_range_sort_prefix.push_back(std::move(column)); + } } void FillingTransform::transform(Chunk & chunk) @@ -638,6 +643,11 @@ void FillingTransform::transform(Chunk & chunk) /// if all chunks are processed, then we may need to generate suffix for the following cases: /// (1) when all data are processed and WITH FILL .. TO is provided /// (2) for empty result set when WITH FILL FROM .. TO is provided (see PR #30888) + + /// if no data was processed, then need to initialize filling_row + if (last_row.empty()) + filling_row.initFromDefaults(); + if (generateSuffixIfNeeded(input.getHeader().getColumns(), result_columns)) { size_t num_output_rows = result_columns[0]->size(); @@ -724,6 +734,10 @@ void FillingTransform::transform(Chunk & chunk) return true; }); + /// generate suffix for the previous range + if (!last_range_sort_prefix.empty()) + generateSuffixIfNeeded(result_columns, res_fill_columns, res_interpolate_columns, res_sort_prefix_columns, res_other_columns); + transformRange( input_fill_columns, input_interpolate_columns, diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 0506bb92efb..def4e87c7ed 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -62,8 +62,12 @@ private: MutableColumnRawPtrs & output_other_columns); bool generateSuffixIfNeeded( - const Columns & input_columns, - MutableColumns & result_columns); + const MutableColumns & result_columns, + MutableColumnRawPtrs res_fill_columns, + MutableColumnRawPtrs res_interpolate_columns, + MutableColumnRawPtrs res_sort_prefix_columns, + MutableColumnRawPtrs res_other_columns); + bool generateSuffixIfNeeded(const Columns & input_columns, MutableColumns & result_columns); const SortDescription sort_description; const SortDescription fill_description; /// Contains only columns with WITH FILL. @@ -81,6 +85,7 @@ private: std::vector> input_positions; /// positions in result columns required for actions ExpressionActionsPtr interpolate_actions; Columns last_row; + Columns last_range_sort_prefix; bool all_chunks_processed = false; /// flag to determine if we have already processed all chunks }; diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 121835a80e8..54a4faf46f2 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -21,3 +21,54 @@ select * from ts order by sensor_id, timestamp with fill step 1; 5 1 1 5 2 0 5 3 1 +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +system stop merges ts; +-- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); +insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; +1 10 1 +1 11 0 +1 12 1 +3 5 1 +3 6 0 +3 7 1 +5 1 1 +5 2 0 +5 3 1 +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +system stop merges ts; +-- FillingTransform: 6 rows will be processed in 3 chunks with 2 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1); +insert into ts VALUES (3, 5, 1), (3, 7, 1); +insert into ts VALUES (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; +1 10 1 +1 11 0 +1 12 1 +3 5 1 +3 6 0 +3 7 1 +5 1 1 +5 2 0 +5 3 1 +select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); +1 6 9999 +1 7 9999 +1 8 9999 +1 9 9999 +1 10 1 +1 12 1 +3 5 1 +3 6 9999 +3 7 1 +3 8 9999 +3 9 9999 +5 1 1 +5 3 1 +5 6 9999 +5 7 9999 +5 8 9999 +5 9 9999 diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index 828572eb620..1b2288da323 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -11,3 +11,22 @@ create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=Merge insert into ts VALUES (1, 10, 1), (1, 12, 2), (3, 5, 1), (3, 7, 3), (5, 1, 1), (5, 3, 1); -- FillingTransform: 6 rows will be processed in 1 chunks select * from ts order by sensor_id, timestamp with fill step 1; + +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +system stop merges ts; +-- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); +insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; + +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +system stop merges ts; +-- FillingTransform: 6 rows will be processed in 3 chunks with 2 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1); +insert into ts VALUES (3, 5, 1), (3, 7, 1); +insert into ts VALUES (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; + +select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); From a0ef2e272faf5c4f2ab1dd74274ab624b74e7f16 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 10 May 2023 12:41:33 +0000 Subject: [PATCH 0228/2223] Fix suffix generation for empty result set --- src/Interpreters/FillingRow.cpp | 9 +++++++++ src/Interpreters/FillingRow.h | 1 + src/Processors/Transforms/FillingTransform.cpp | 2 +- 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 98197533aa1..87cc91681c4 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -50,6 +50,15 @@ bool FillingRow::operator>=(const FillingRow & other) const return !(*this < other); } +bool FillingRow::isNull() const +{ + for (const auto & field : row) + if (!field.isNull()) + return false; + + return true; +} + bool FillingRow::next(const FillingRow & to_row) { const size_t row_size = size(); diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 9f414d92dd3..b1af851d792 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -29,6 +29,7 @@ public: bool operator<(const FillingRow & other) const; bool operator==(const FillingRow & other) const; bool operator>=(const FillingRow & other) const; + bool isNull() const; int getDirection(size_t index) const { return sort_description[index].direction; } FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 5fa89e89a4b..45967b646a6 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -448,7 +448,7 @@ bool FillingTransform::generateSuffixIfNeeded( logDebug("generateSuffixIfNeeded() next_row", next_row); /// Determines should we insert filling row before start generating next rows. - bool should_insert_first = next_row < filling_row; + bool should_insert_first = next_row < filling_row || next_row.isNull(); logDebug("should_insert_first", should_insert_first); for (size_t i = 0, size = filling_row.size(); i < size; ++i) From 1cc2499c70db4908331a02d1bbf71d89a27f8875 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 10 May 2023 16:10:43 +0000 Subject: [PATCH 0229/2223] fix build --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index fb25770ed8d..d956dac9fe0 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -49,7 +49,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( bool allow_seeks_after_first_read_, bool use_external_buffer_, std::optional read_until_position_, - std::shared_ptr cache_log_)) + std::shared_ptr cache_log_) : ReadBufferFromFileBase(use_external_buffer_ ? 0 : settings_.remote_fs_buffer_size, nullptr, 0, file_size_) #ifndef NDEBUG , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile(" + source_file_path_ + ")")) From 5143b37d0612fe1cc00f3fc91a7ca13423e851fa Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 10 May 2023 17:03:07 +0000 Subject: [PATCH 0230/2223] Fix clang-tidy build --- src/Planner/Planner.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e3a4d01a46a..e19a92c273c 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -623,7 +623,7 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, } auto filling_step = std::make_unique( - query_plan.getCurrentDataStream(), std::move(sort_description), std::move(fill_description), interpolate_description); + query_plan.getCurrentDataStream(), sort_description, std::move(fill_description), interpolate_description); query_plan.addStep(std::move(filling_step)); } From ac7c54a4d3636c8616799fa8272f6da1ce6683fc Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 10 May 2023 18:39:38 +0000 Subject: [PATCH 0231/2223] Refactor CapnProto format to improve input/output performance --- src/Core/Settings.h | 2 +- src/Core/SettingsEnums.cpp | 8 +- src/Core/SettingsEnums.h | 2 +- src/Formats/CapnProtoSchema.cpp | 298 ++++ .../{CapnProtoUtils.h => CapnProtoSchema.h} | 13 +- src/Formats/CapnProtoSerializer.cpp | 1218 +++++++++++++++++ src/Formats/CapnProtoSerializer.h | 25 + src/Formats/CapnProtoUtils.cpp | 734 ---------- src/Formats/FormatSettings.h | 6 +- .../Formats/Impl/CapnProtoRowInputFormat.cpp | 253 +--- .../Formats/Impl/CapnProtoRowInputFormat.h | 9 +- .../Formats/Impl/CapnProtoRowOutputFormat.cpp | 266 +--- .../Formats/Impl/CapnProtoRowOutputFormat.h | 17 +- .../Formats/Impl/ProtobufListInputFormat.cpp | 9 +- .../Formats/Impl/ProtobufRowInputFormat.cpp | 9 +- .../queries/0_stateless/02030_capnp_format.sh | 4 +- ...p_case_insensitive_names_matcing.reference | 1 + ...35_capnp_case_insensitive_names_matcing.sh | 10 + ...ing_and_writing_structure_fields.reference | 3 + ...36_reading_and_writing_structure_fields.sh | 24 + ...2735_case_insensitive_names_matching.capnp | 13 + .../02736_nested_structures.capnp | 21 + 22 files changed, 1686 insertions(+), 1259 deletions(-) create mode 100644 src/Formats/CapnProtoSchema.cpp rename src/Formats/{CapnProtoUtils.h => CapnProtoSchema.h} (59%) create mode 100644 src/Formats/CapnProtoSerializer.cpp create mode 100644 src/Formats/CapnProtoSerializer.h delete mode 100644 src/Formats/CapnProtoUtils.cpp create mode 100644 tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.reference create mode 100755 tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.sh create mode 100644 tests/queries/0_stateless/02736_reading_and_writing_structure_fields.reference create mode 100755 tests/queries/0_stateless/02736_reading_and_writing_structure_fields.sh create mode 100644 tests/queries/0_stateless/format_schemas/02735_case_insensitive_names_matching.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02736_nested_structures.capnp diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1bea2c26392..269fa832f45 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -958,7 +958,7 @@ class IColumn; M(Bool, output_format_orc_string_as_string, false, "Use ORC String type instead of Binary for String columns", 0) \ M(ORCCompression, output_format_orc_compression_method, "lz4", "Compression method for ORC output format. Supported codecs: lz4, snappy, zlib, zstd, none (uncompressed)", 0) \ \ - M(EnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::EnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \ + M(CapnProtoEnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::CapnProtoEnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \ \ M(String, input_format_mysql_dump_table_name, "", "Name of the table in MySQL dump from which to read data", 0) \ M(Bool, input_format_mysql_dump_map_column_names, true, "Match columns from table in MySQL dump and columns from ClickHouse table by names", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index e0f16ea00db..a291a23c140 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -144,10 +144,10 @@ IMPLEMENT_SETTING_ENUM(TransactionsWaitCSNMode, ErrorCodes::BAD_ARGUMENTS, {"wait", TransactionsWaitCSNMode::WAIT}, {"wait_unknown", TransactionsWaitCSNMode::WAIT_UNKNOWN}}) -IMPLEMENT_SETTING_ENUM(EnumComparingMode, ErrorCodes::BAD_ARGUMENTS, - {{"by_names", FormatSettings::EnumComparingMode::BY_NAMES}, - {"by_values", FormatSettings::EnumComparingMode::BY_VALUES}, - {"by_names_case_insensitive", FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE}}) +IMPLEMENT_SETTING_ENUM(CapnProtoEnumComparingMode, ErrorCodes::BAD_ARGUMENTS, + {{"by_names", FormatSettings::CapnProtoEnumComparingMode::BY_NAMES}, + {"by_values", FormatSettings::CapnProtoEnumComparingMode::BY_VALUES}, + {"by_names_case_insensitive", FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE}}) IMPLEMENT_SETTING_ENUM(EscapingRule, ErrorCodes::BAD_ARGUMENTS, {{"None", FormatSettings::EscapingRule::None}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 3ae7bfaa673..1c5be910ef7 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -188,7 +188,7 @@ enum class TransactionsWaitCSNMode DECLARE_SETTING_ENUM(TransactionsWaitCSNMode) -DECLARE_SETTING_ENUM_WITH_RENAME(EnumComparingMode, FormatSettings::EnumComparingMode) +DECLARE_SETTING_ENUM_WITH_RENAME(CapnProtoEnumComparingMode, FormatSettings::CapnProtoEnumComparingMode) DECLARE_SETTING_ENUM_WITH_RENAME(EscapingRule, FormatSettings::EscapingRule) diff --git a/src/Formats/CapnProtoSchema.cpp b/src/Formats/CapnProtoSchema.cpp new file mode 100644 index 00000000000..22518d5061a --- /dev/null +++ b/src/Formats/CapnProtoSchema.cpp @@ -0,0 +1,298 @@ +#include + +#if USE_CAPNP + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_CAPN_PROTO_SCHEMA; + extern const int BAD_TYPE_OF_FIELD; + extern const int FILE_DOESNT_EXIST; + extern const int UNKNOWN_EXCEPTION; + extern const int CAPN_PROTO_BAD_TYPE; + extern const int BAD_ARGUMENTS; +} + +capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info) +{ + capnp::ParsedSchema schema; + try + { + int fd; + KJ_SYSCALL(fd = open(schema_info.schemaDirectory().data(), O_RDONLY)); // NOLINT(bugprone-suspicious-semicolon) + auto schema_dir = kj::newDiskDirectory(kj::OsFileHandle(fd)); + schema = impl.parseFromDirectory(*schema_dir, kj::Path::parse(schema_info.schemaPath()), {}); + } + catch (const kj::Exception & e) + { + /// That's not good to determine the type of error by its description, but + /// this is the only way to do it here, because kj doesn't specify the type of error. + auto description = std::string_view(e.getDescription().cStr()); + if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); + + if (description.find("Parse error") != String::npos) + throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "Cannot parse CapnProto schema {}:{}", schema_info.schemaPath(), e.getLine()); + + throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, + "Unknown exception while parsing CapnProto schema: {}, schema dir and file: {}, {}", + description, schema_info.schemaDirectory(), schema_info.schemaPath()); + } + + auto message_maybe = schema.findNested(schema_info.messageName()); + auto * message_schema = kj::_::readMaybe(message_maybe); + if (!message_schema) + throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, + "CapnProto schema doesn't contain message with name {}", schema_info.messageName()); + return message_schema->asStruct(); +} + +bool checkIfStructContainsUnnamedUnion(const capnp::StructSchema & struct_schema) +{ + return struct_schema.getFields().size() != struct_schema.getNonUnionFields().size(); +} + +bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema) +{ + return struct_schema.getFields().size() == struct_schema.getUnionFields().size(); +} + +/// Get full name of type for better exception messages. +String getCapnProtoFullTypeName(const capnp::Type & type) +{ + static const std::map capnp_simple_type_names = + { + {capnp::schema::Type::Which::BOOL, "Bool"}, + {capnp::schema::Type::Which::VOID, "Void"}, + {capnp::schema::Type::Which::INT8, "Int8"}, + {capnp::schema::Type::Which::INT16, "Int16"}, + {capnp::schema::Type::Which::INT32, "Int32"}, + {capnp::schema::Type::Which::INT64, "Int64"}, + {capnp::schema::Type::Which::UINT8, "UInt8"}, + {capnp::schema::Type::Which::UINT16, "UInt16"}, + {capnp::schema::Type::Which::UINT32, "UInt32"}, + {capnp::schema::Type::Which::UINT64, "UInt64"}, + {capnp::schema::Type::Which::FLOAT32, "Float32"}, + {capnp::schema::Type::Which::FLOAT64, "Float64"}, + {capnp::schema::Type::Which::TEXT, "Text"}, + {capnp::schema::Type::Which::DATA, "Data"}, + {capnp::schema::Type::Which::INTERFACE, "Interface"}, + {capnp::schema::Type::Which::ANY_POINTER, "AnyPointer"}, + }; + + switch (type.which()) + { + case capnp::schema::Type::Which::STRUCT: + { + auto struct_schema = type.asStruct(); + + auto non_union_fields = struct_schema.getNonUnionFields(); + std::vector non_union_field_names; + for (auto nested_field : non_union_fields) + non_union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); + + auto union_fields = struct_schema.getUnionFields(); + std::vector union_field_names; + for (auto nested_field : union_fields) + union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); + + String union_name = "Union(" + boost::algorithm::join(union_field_names, ", ") + ")"; + /// Check if the struct is a named union. + if (non_union_field_names.empty()) + return union_name; + + String type_name = "Struct(" + boost::algorithm::join(non_union_field_names, ", "); + /// Check if the struct contains unnamed union. + if (!union_field_names.empty()) + type_name += ", " + union_name; + type_name += ")"; + return type_name; + } + case capnp::schema::Type::Which::LIST: + return "List(" + getCapnProtoFullTypeName(type.asList().getElementType()) + ")"; + case capnp::schema::Type::Which::ENUM: + { + auto enum_schema = type.asEnum(); + String enum_name = "Enum("; + auto enumerants = enum_schema.getEnumerants(); + for (unsigned i = 0; i != enumerants.size(); ++i) + { + enum_name += String(enumerants[i].getProto().getName()) + " = " + std::to_string(enumerants[i].getOrdinal()); + if (i + 1 != enumerants.size()) + enum_name += ", "; + } + enum_name += ")"; + return enum_name; + } + default: + auto it = capnp_simple_type_names.find(type.which()); + if (it == capnp_simple_type_names.end()) + throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Unknown CapnProto type"); + return it->second; + } +} + +namespace +{ + + template + static DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) + { + std::vector> values; + for (auto enumerant : enumerants) + values.emplace_back(enumerant.getProto().getName(), ValueType(enumerant.getOrdinal())); + return std::make_shared>(std::move(values)); + } + + static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) + { + auto enumerants = enum_schema.getEnumerants(); + if (enumerants.size() < 128) + return getEnumDataTypeFromEnumerants(enumerants); + if (enumerants.size() < 32768) + return getEnumDataTypeFromEnumerants(enumerants); + + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "ClickHouse supports only 8 and 16-bit Enums"); + } + + static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) + { + switch (capnp_type.which()) + { + case capnp::schema::Type::INT8: + return std::make_shared(); + case capnp::schema::Type::INT16: + return std::make_shared(); + case capnp::schema::Type::INT32: + return std::make_shared(); + case capnp::schema::Type::INT64: + return std::make_shared(); + case capnp::schema::Type::BOOL: [[fallthrough]]; + case capnp::schema::Type::UINT8: + return std::make_shared(); + case capnp::schema::Type::UINT16: + return std::make_shared(); + case capnp::schema::Type::UINT32: + return std::make_shared(); + case capnp::schema::Type::UINT64: + return std::make_shared(); + case capnp::schema::Type::FLOAT32: + return std::make_shared(); + case capnp::schema::Type::FLOAT64: + return std::make_shared(); + case capnp::schema::Type::DATA: [[fallthrough]]; + case capnp::schema::Type::TEXT: + return std::make_shared(); + case capnp::schema::Type::ENUM: + return getEnumDataTypeFromEnumSchema(capnp_type.asEnum()); + case capnp::schema::Type::LIST: + { + auto list_schema = capnp_type.asList(); + auto nested_type = getDataTypeFromCapnProtoType(list_schema.getElementType(), skip_unsupported_fields); + if (!nested_type) + return nullptr; + return std::make_shared(nested_type); + } + case capnp::schema::Type::STRUCT: + { + auto struct_schema = capnp_type.asStruct(); + + + if (struct_schema.getFields().size() == 0) + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Empty messages are not supported"); + } + + /// Check if it can be Nullable. + if (checkIfStructIsNamedUnion(struct_schema)) + { + auto fields = struct_schema.getUnionFields(); + if (fields.size() != 2 || (!fields[0].getType().isVoid() && !fields[1].getType().isVoid())) + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unions are not supported"); + } + auto value_type = fields[0].getType().isVoid() ? fields[1].getType() : fields[0].getType(); + if (value_type.isStruct() || value_type.isList()) + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Tuples and Lists cannot be inside Nullable"); + } + + auto nested_type = getDataTypeFromCapnProtoType(value_type, skip_unsupported_fields); + if (!nested_type) + return nullptr; + return std::make_shared(nested_type); + } + + if (checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); + + /// Treat Struct as Tuple. + DataTypes nested_types; + Names nested_names; + for (auto field : struct_schema.getNonUnionFields()) + { + auto nested_type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); + if (!nested_type) + continue; + nested_names.push_back(field.getProto().getName()); + nested_types.push_back(nested_type); + } + if (nested_types.empty()) + return nullptr; + return std::make_shared(std::move(nested_types), std::move(nested_names)); + } + default: + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unsupported CapnProtoType: {}", getCapnProtoFullTypeName(capnp_type)); + } + } +} + +} + +NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields) +{ + if (checkIfStructContainsUnnamedUnion(schema)) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); + + NamesAndTypesList names_and_types; + for (auto field : schema.getNonUnionFields()) + { + auto name = field.getProto().getName(); + auto type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); + if (type) + names_and_types.emplace_back(name, type); + } + if (names_and_types.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot convert CapnProto schema to ClickHouse table schema, all fields have unsupported types"); + + return names_and_types; +} + +} + +#endif diff --git a/src/Formats/CapnProtoUtils.h b/src/Formats/CapnProtoSchema.h similarity index 59% rename from src/Formats/CapnProtoUtils.h rename to src/Formats/CapnProtoSchema.h index 2d8cdb418d7..225f6f56207 100644 --- a/src/Formats/CapnProtoUtils.h +++ b/src/Formats/CapnProtoSchema.h @@ -30,17 +30,14 @@ public: capnp::StructSchema getMessageSchema(const FormatSchemaInfo & schema_info); }; -std::pair splitCapnProtoFieldName(const String & name); +bool checkIfStructContainsUnnamedUnion(const capnp::StructSchema & struct_schema); +bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema); -bool compareEnumNames(const String & first, const String & second, FormatSettings::EnumComparingMode mode); - -std::pair getStructBuilderAndFieldByColumnName(capnp::DynamicStruct::Builder struct_builder, const String & name); - -capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Reader & struct_reader, const String & name); - -void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode); +/// Get full name of type for better exception messages. +String getCapnProtoFullTypeName(const capnp::Type & type); NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields); + } #endif diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp new file mode 100644 index 00000000000..e0c8ae2a79a --- /dev/null +++ b/src/Formats/CapnProtoSerializer.cpp @@ -0,0 +1,1218 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int THERE_IS_NO_COLUMN; + extern const int BAD_TYPE_OF_FIELD; + extern const int CAPN_PROTO_BAD_CAST; + extern const int INCORRECT_DATA; + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + std::pair splitFieldName(const String & name) + { + const auto * begin = name.data(); + const auto * end = name.data() + name.size(); + const auto * it = find_first_symbols<'_', '.'>(begin, end); + String first = String(begin, it); + String second = it == end ? "" : String(it + 1, end); + return {first, second}; + } + + std::optional findFieldByName(const capnp::StructSchema & struct_schema, const String & name) + { + const auto & fields = struct_schema.getFields(); + for (auto field : fields) + { + auto field_name = String(field.getProto().getName()); + if (boost::to_lower_copy(name) == boost::to_lower_copy(field_name)) + return field; + } + return std::nullopt; + } + + [[noreturn]] void throwCannotConvert(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type) + { + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto type {}", + name, + type->getName(), + getCapnProtoFullTypeName(capnp_type)); + } + + struct FieldBuilder + { + virtual ~FieldBuilder() = default; + }; + + struct ListBuilder : public FieldBuilder + { + explicit ListBuilder(capnp::DynamicValue::Builder builder) : impl(builder.as()) + { + } + + capnp::DynamicList::Builder impl; + std::vector> nested_builders; + }; + + struct StructBuilder : public FieldBuilder + { + explicit StructBuilder(capnp::DynamicValue::Builder builder, size_t fields_size) : impl(builder.as()), field_builders(fields_size) + { + } + + explicit StructBuilder(capnp::DynamicStruct::Builder struct_builder, size_t fields_size) : impl(std::move(struct_builder)), field_builders(fields_size) + { + } + + capnp::DynamicStruct::Builder impl; + std::vector> field_builders; + }; + + std::unique_ptr initStructFieldBuilderIfNeeded(const ColumnPtr & column, size_t row_num, capnp::DynamicStruct::Builder & struct_builder, const capnp::StructSchema::Field & field, const capnp::Type & capnp_type, size_t nested_fields_size) + { + switch (capnp_type.which()) + { + case capnp::schema::Type::LIST: + { + const auto * array_column = assert_cast(column.get()); + size_t size = array_column->getOffsets()[row_num] - array_column->getOffsets()[row_num - 1]; + return std::make_unique(struct_builder.init(field, static_cast(size))); + } + case capnp::schema::Type::STRUCT: + { + return std::make_unique(struct_builder.init(field), nested_fields_size); + } + default: + return nullptr; + } + } + + class ICapnProtoSerializer + { + public: + virtual std::optional writeRow(const ColumnPtr & column, FieldBuilder * builder, size_t row_num) = 0; + virtual void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) = 0; + + virtual ~ICapnProtoSerializer() = default; + }; + + template + class CapnProtoIntegerSerializer : public ICapnProtoSerializer + { + public: + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::INT) + return capnp::DynamicValue::Reader(column->getInt(row_num)); + if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::UINT) + return capnp::DynamicValue::Reader(column->getUInt(row_num)); + return capnp::DynamicValue::Reader(column->getBool(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + NumericType value; + if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::INT) + value = static_cast(reader.as()); + else if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::UINT) + value = static_cast(reader.as()); + else if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::BOOL) + value = static_cast(reader.as()); + + if constexpr (is_bool_data_type) + assert_cast(column).insertValue(static_cast(value)); + else + assert_cast &>(column).insertValue(value); + } + }; + + template + static std::unique_ptr createIntegerSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + switch (capnp_type.which()) + { + case capnp::schema::Type::INT8: [[fallthrough]]; + case capnp::schema::Type::INT16: [[fallthrough]]; + case capnp::schema::Type::INT32: [[fallthrough]]; + case capnp::schema::Type::INT64: + return std::make_unique>(); + case capnp::schema::Type::UINT8: [[fallthrough]]; + case capnp::schema::Type::UINT16: [[fallthrough]]; + case capnp::schema::Type::UINT32: [[fallthrough]]; + case capnp::schema::Type::UINT64: + return std::make_unique>(); + case capnp::schema::Type::BOOL: + return std::make_unique>(); + default: + throwCannotConvert(data_type, column_name, capnp_type); + } + } + + template + class CapnProtoBigIntegerSerializer : public ICapnProtoSerializer + { + public: + CapnProtoBigIntegerSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(NumericType)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + + private: + DataTypePtr data_type; + }; + + template + class CapnProtoFloatSerializer : public ICapnProtoSerializer + { + public: + CapnProtoFloatSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isFloat32() && !capnp_type.isFloat64()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getFloat64(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast &>(column).insertValue(reader.as()); + } + }; + + template + class CapnProtoEnumSerializer : public ICapnProtoSerializer + { + public: + CapnProtoEnumSerializer( + const DataTypePtr & data_type_, + const String & column_name, + const capnp::Type & capnp_type, + const FormatSettings::CapnProtoEnumComparingMode enum_comparing_mode_) : data_type(data_type_), enum_comparing_mode(enum_comparing_mode_) + { + if (!capnp_type.isEnum()) + throwCannotConvert(data_type, column_name, capnp_type); + + bool to_lower = enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE; + const auto * enum_type = assert_cast *>(data_type.get()); + const auto & enum_values = dynamic_cast &>(*enum_type); + + enum_schema = capnp_type.asEnum(); + auto enumerants = enum_schema.getEnumerants(); + constexpr auto max_value = std::is_same_v ? INT8_MAX : INT16_MAX; + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + { + /// In CapnProto Enum fields are numbered sequentially starting from zero. + if (enumerants.size() > max_value) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Enum from CapnProto schema contains values that are out of range for Clickhouse enum type {}", + data_type->getName()); + + auto values = enum_values.getSetOfAllValues(); + std::unordered_set capn_enum_values; + for (auto enumerant : enumerants) + capn_enum_values.insert(EnumType(enumerant.getOrdinal())); + if (values != capn_enum_values) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "The set of values in Enum from CapnProto schema is different from the set of values in ClickHouse Enum"); + } + else + { + auto names = enum_values.getSetOfAllNames(to_lower); + std::unordered_set capn_enum_names; + + for (auto enumerant : enumerants) + { + String name = enumerant.getProto().getName(); + capn_enum_names.insert(to_lower ? boost::algorithm::to_lower_copy(name) : name); + } + + if (names != capn_enum_names) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"); + } + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + const auto * enum_data_type = assert_cast *>(data_type.get()); + EnumType enum_value = assert_cast &>(*column).getElement(row_num); + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + return capnp::DynamicValue::Reader(capnp::DynamicEnum(enum_schema, enum_value)); + + auto enum_name = enum_data_type->getNameForValue(enum_value); + for (const auto enumerant : enum_schema.getEnumerants()) + { + if (compareEnumNames(String(enum_name), enumerant.getProto().getName(), enum_comparing_mode)) + return capnp::DynamicValue::Reader(capnp::DynamicEnum(enumerant)); + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert CLickHouse Enum value to CapnProto Enum"); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto enum_value = reader.as(); + auto enumerant = *kj::_::readMaybe(enum_value.getEnumerant()); + auto enum_type = assert_cast *>(data_type.get()); + DataTypePtr nested_type = std::make_shared>(); + switch (enum_comparing_mode) + { + case FormatSettings::CapnProtoEnumComparingMode::BY_VALUES: + { + assert_cast &>(column).insertValue(static_cast(enumerant.getOrdinal())); + return; + } + case FormatSettings::CapnProtoEnumComparingMode::BY_NAMES: + { + auto value = enum_type->getValue(String(enumerant.getProto().getName())); + assert_cast &>(column).insertValue(value); + return; + } + case FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE: + { + /// Find the same enum name case insensitive. + String enum_name = enumerant.getProto().getName(); + for (auto & name : enum_type->getAllRegisteredNames()) + { + if (compareEnumNames(name, enum_name, enum_comparing_mode)) + { + assert_cast &>(column).insertValue(enum_type->getValue(name)); + break; + } + } + return; + } + } + } + + private: + bool compareEnumNames(const String & first, const String & second, const FormatSettings::CapnProtoEnumComparingMode mode) + { + if (mode == FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE) + return boost::algorithm::to_lower_copy(first) == boost::algorithm::to_lower_copy(second); + return first == second; + } + + DataTypePtr data_type; + capnp::EnumSchema enum_schema; + const FormatSettings::CapnProtoEnumComparingMode enum_comparing_mode; + }; + + class CapnProtoDateSerializer : public ICapnProtoSerializer + { + public: + CapnProtoDateSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isUInt16()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getUInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + class CapnProtoDate32Serializer : public ICapnProtoSerializer + { + public: + CapnProtoDate32Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isInt32()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + class CapnProtoDateTimeSerializer : public ICapnProtoSerializer + { + public: + CapnProtoDateTimeSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isUInt32()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + class CapnProtoDateTime64Serializer : public ICapnProtoSerializer + { + public: + CapnProtoDateTime64Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isInt64()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + template + class CapnProtoDecimalSerializer : public ICapnProtoSerializer + { + public: + CapnProtoDecimalSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + auto which = WhichDataType(data_type); + if ((!capnp_type.isInt32() && which.isDecimal32()) || (!capnp_type.isInt64() && which.isDecimal64())) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast &>(column).insertValue(reader.as()); + } + }; + + template + class CapnProtoBigDecimalSerializer : public ICapnProtoSerializer + { + public: + CapnProtoBigDecimalSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(DecimalType)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + + private: + DataTypePtr data_type; + }; + + template + class CapnProtoStringSerializer : public ICapnProtoSerializer + { + public: + CapnProtoStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type_) : capnp_type(capnp_type_) + { + if (!capnp_type.isData() && !capnp_type.isText()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + + if constexpr (is_binary) + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + + /// For type TEXT data must be null-terminated, but in String column we always have 0 byte at the end of each value. + return capnp::DynamicValue::Reader(capnp::Text::Reader(data.data, data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + if constexpr (is_binary) + { + auto value = reader.as(); + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + else + { + auto value = reader.as(); + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + } + + private: + capnp::Type capnp_type; + }; + + template + class CapnProtoFixedStringSerializer : public ICapnProtoSerializer + { + public: + CapnProtoFixedStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type_) : capnp_type(capnp_type_) + { + if (!capnp_type.isData() && !capnp_type.isText()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + if constexpr (is_binary) + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + + if (data.data[data.size - 1] == 0) + return capnp::DynamicValue::Reader(capnp::Text::Reader(reinterpret_cast(data.data), data.size)); + + /// In TEXT type data should be null-terminated, but ClickHouse FixedString data could not be. + /// To make data null-terminated we should copy it to temporary String object and use it in capnp::Text::Reader. + /// Note that capnp::Text::Reader works only with pointer to the data and it's size, so we should + /// guarantee that new String object life time is longer than capnp::Text::Reader life time. + tmp_string = data.toString(); + return capnp::DynamicValue::Reader(capnp::Text::Reader(reinterpret_cast(tmp_string.data()), tmp_string.size())); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto & fixed_string_column = assert_cast(column); + if constexpr (is_binary) + { + auto value = reader.as(); + if (value.size() > fixed_string_column.getN()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); + + fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); + } + else + { + auto value = reader.as(); + if (value.size() > fixed_string_column.getN()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); + + fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); + } + } + + private: + String tmp_string; + capnp::Type capnp_type; + }; + + class CapnProtoIPv4Serializer : public ICapnProtoSerializer + { + public: + CapnProtoIPv4Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isUInt32()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(assert_cast(*column).getElement(row_num).toUnderType()); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(IPv4(reader.as())); + } + }; + + class CapnProtoIPv6Serializer : public ICapnProtoSerializer + { + public: + CapnProtoIPv6Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(IPv6)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of IPv6 value: {}", value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + }; + + class CapnProtoUUIDSerializer : public ICapnProtoSerializer + { + public: + CapnProtoUUIDSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(UUID)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of UUID value: {}", value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + }; + + std::unique_ptr createSerializer(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings); + + class CapnProtoLowCardinalitySerializer : public ICapnProtoSerializer + { + public: + CapnProtoLowCardinalitySerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + nested_serializer = createSerializer(assert_cast(*data_type).getDictionaryType(), column_name, capnp_type, settings); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + const auto & low_cardinality_column = assert_cast(*column); + size_t index = low_cardinality_column.getIndexAt(row_num); + const auto & dict_column = low_cardinality_column.getDictionary().getNestedColumn(); + return nested_serializer->writeRow(dict_column, field_builder, index); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto & low_cardinality_column = assert_cast(column); + auto tmp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); + nested_serializer->readRow(*tmp_column, reader); + low_cardinality_column.insertFromFullColumn(*tmp_column, 0); + } + + private: + std::unique_ptr nested_serializer; + }; + + class CapnProtoNullableSerializer : public ICapnProtoSerializer + { + public: + CapnProtoNullableSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + if (!capnp_type.isStruct()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type, got CapnProto type {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + /// Check that struct is a named union of type VOID and one arbitrary type. + auto struct_schema = capnp_type.asStruct(); + if (!checkIfStructIsNamedUnion(struct_schema)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type." + "Given CapnProto struct is not a named union: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + auto union_fields = struct_schema.getUnionFields(); + if (union_fields.size() != 2) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type." + "Given CapnProto union have more than 2 fields: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + auto first = union_fields[0]; + auto second = union_fields[1]; + auto nested_type = assert_cast(data_type.get())->getNestedType(); + if (first.getType().isVoid()) + { + null_field = first; + nested_field = second; + nested_capnp_type = second.getType(); + if (nested_capnp_type.isStruct()) + nested_fields_size = nested_capnp_type.asStruct().getFields().size(); + nested_serializer = createSerializer(nested_type, column_name, nested_capnp_type, settings); + } + else if (second.getType().isVoid()) + { + null_field = second; + nested_field = first; + nested_capnp_type = first.getType(); + if (nested_capnp_type.isStruct()) + nested_fields_size = nested_capnp_type.asStruct().getFields().size(); + nested_serializer = createSerializer(nested_type, column_name, nested_capnp_type, settings); + } + else + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type." + "Given CapnProto union doesn't have field with type Void: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + assert(field_builder); + auto & struct_builder = assert_cast(*field_builder); + const auto & nullable_column = assert_cast(*column); + if (nullable_column.isNullAt(row_num)) + { + struct_builder.impl.set(null_field, capnp::Void()); + } + else + { + struct_builder.impl.clear(nested_field); + const auto & nested_column = nullable_column.getNestedColumnPtr(); + auto nested_field_builder = initStructFieldBuilderIfNeeded(nested_column, row_num, struct_builder.impl, nested_field, nested_capnp_type, nested_fields_size); + auto value = nested_serializer->writeRow(nested_column, nested_field_builder.get(), row_num); + if (value) + struct_builder.impl.set(nested_field, *value); + } + + return std::nullopt; + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto struct_reader = reader.as(); + auto & nullable_column = assert_cast(column); + auto field = *kj::_::readMaybe(struct_reader.which()); + if (field.getType().isVoid()) + nullable_column.insertDefault(); + else + { + auto & nested_column = nullable_column.getNestedColumn(); + auto nested_reader = struct_reader.get(field); + nested_serializer->readRow(nested_column, nested_reader); + nullable_column.getNullMapData().push_back(0); + } + } + + private: + std::unique_ptr nested_serializer; + capnp::StructSchema::Field null_field; + capnp::StructSchema::Field nested_field; + size_t nested_fields_size = 0; + capnp::Type nested_capnp_type; + }; + + class CapnProtoArraySerializer : public ICapnProtoSerializer + { + public: + CapnProtoArraySerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + if (!capnp_type.isList()) + throwCannotConvert(data_type, column_name, capnp_type); + + auto nested_type = assert_cast(data_type.get())->getNestedType(); + element_type = capnp_type.asList().getElementType(); + if (element_type.isStruct()) + element_struct_fields = element_type.asStruct().getFields().size(); + nested_serializer = createSerializer(nested_type, column_name, capnp_type.asList().getElementType(), settings); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + assert(field_builder); + auto & list_builder = assert_cast(*field_builder); + const auto * array_column = assert_cast(column.get()); + const auto & nested_column = array_column->getDataPtr(); + const auto & offsets = array_column->getOffsets(); + auto offset = offsets[row_num - 1]; + size_t size = offsets[row_num] - offset; + bool need_nested_builders = list_builder.nested_builders.empty(); + for (unsigned i = 0; i != static_cast(size); ++i) + { + if (need_nested_builders) + { + /// For nested lists we need to initialize nested list builder. + if (element_type.isList()) + { + const auto & nested_offset = checkAndGetColumn(*nested_column)->getOffsets(); + size_t nested_array_size = nested_offset[offset + i] - nested_offset[offset + i - 1]; + list_builder.nested_builders.emplace_back(std::make_unique(list_builder.impl.init(i, static_cast(nested_array_size)))); + } + else if (element_type.isStruct()) + { + list_builder.nested_builders.emplace_back(std::make_unique(list_builder.impl[i], element_struct_fields)); + } + else + { + list_builder.nested_builders.emplace_back(); + } + } + + auto value = nested_serializer->writeRow(nested_column, list_builder.nested_builders[i].get(), offset + i); + if (value) + list_builder.impl.set(i, *value); + } + + return std::nullopt; + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto list_reader = reader.as(); + auto & column_array = assert_cast(column); + auto & offsets = column_array.getOffsets(); + offsets.push_back(offsets.back() + list_reader.size()); + + auto & nested_column = column_array.getData(); + for (const auto & nested_reader : list_reader) + nested_serializer->readRow(nested_column, nested_reader); + } + + private: + std::unique_ptr nested_serializer; + capnp::Type element_type; + size_t element_struct_fields; + }; + + class CapnProtoMapSerializer : public ICapnProtoSerializer + { + public: + CapnProtoMapSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + /// We output/input Map type as follow CapnProto schema + /// + /// struct Map { + /// struct Entry { + /// key @0: Key; + /// value @1: Value; + /// } + /// entries @0 :List(Entry); + /// } + + if (!capnp_type.isStruct()) + throwCannotConvert(data_type, column_name, capnp_type); + + auto struct_schema = capnp_type.asStruct(); + + if (checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto Struct with unnamed union {}", + column_name, + data_type->getName(), + getCapnProtoFullTypeName(capnp_type)); + + if (struct_schema.getFields().size() != 1) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": Map type can be represented as a Struct with one list field, got struct: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + const auto & field_type = struct_schema.getFields()[0].getType(); + if (!field_type.isList()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": Map type can be represented as a Struct with one list field, got field: {}", + column_name, + getCapnProtoFullTypeName(field_type)); + + auto list_element_type = field_type.asList().getElementType(); + if (!list_element_type.isStruct()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": Field of struct that represents Map should be a list of structs, got list of {}", + column_name, + getCapnProtoFullTypeName(list_element_type)); + + auto key_value_struct = list_element_type.asStruct(); + if (checkIfStructContainsUnnamedUnion(key_value_struct)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": struct that represents Map entries is unnamed union: {}", + column_name, + getCapnProtoFullTypeName(list_element_type)); + + if (key_value_struct.getFields().size() != 2) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": struct that represents Map entries should contain only 2 fields, got struct {}", + column_name, + getCapnProtoFullTypeName(list_element_type)); + + const auto & map_type = assert_cast(*data_type); + DataTypes types = {map_type.getKeyType(), map_type.getValueType()}; + Names names = {"key", "value"}; + auto entries_type = std::make_shared(std::make_shared(types, names)); + entries_field = struct_schema.getFields()[0]; + entries_capnp_type = entries_field.getType(); + nested_serializer = createSerializer(entries_type, column_name, field_type, settings); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + assert(field_builder); + auto & struct_builder = assert_cast(*field_builder); + const auto & entries_column = assert_cast(column.get())->getNestedColumnPtr(); + auto entries_builder = initStructFieldBuilderIfNeeded(entries_column, row_num, struct_builder.impl, entries_field, entries_capnp_type, 0); + nested_serializer->writeRow(entries_column, entries_builder.get(), row_num); + return std::nullopt; + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto struct_reader = reader.as(); + auto & entries_column = assert_cast(column).getNestedColumn(); + nested_serializer->readRow(entries_column, struct_reader.get(entries_field)); + } + + private: + std::unique_ptr nested_serializer; + capnp::StructSchema::Field entries_field; + capnp::Type entries_capnp_type; + }; + + class CapnProtoStructureSerializer : public ICapnProtoSerializer + { + public: + CapnProtoStructureSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + { + if (checkIfStructIsNamedUnion(schema) || checkIfStructContainsUnnamedUnion(schema)) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Root CapnProto Struct cannot be named union/struct with unnamed union"); + + initialize(data_types, names, schema, settings); + } + + CapnProtoStructureSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + if (!capnp_type.isStruct()) + throwCannotConvert(data_type, column_name, capnp_type); + + auto struct_schema = capnp_type.asStruct(); + + if (checkIfStructIsNamedUnion(struct_schema) || checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto named union/struct with unnamed union {}", + column_name, + data_type->getName(), + getCapnProtoFullTypeName(capnp_type)); + + const auto * tuple_data_type = assert_cast(data_type.get()); + auto nested_types = tuple_data_type->getElements(); + Names nested_names; + bool have_explicit_names = tuple_data_type->haveExplicitNames(); + auto structure_fields = struct_schema.getFields(); + if (!have_explicit_names) + { + if (nested_types.size() != structure_fields.size()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto type {}: Tuple and Struct have different sizes {} != {}", + column_name, + data_type->getName(), + getCapnProtoFullTypeName(capnp_type), + nested_types.size(), + structure_fields.size()); + nested_names.reserve(structure_fields.size()); + for (auto field : structure_fields) + nested_names.push_back(field.getProto().getName()); + } + else + { + nested_names = tuple_data_type->getElementNames(); + } + + try + { + initialize(nested_types, nested_names, struct_schema, settings); + } + catch (Exception & e) + { + e.addMessage("(while converting column {})", column_name); + throw e; + } + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * builder, size_t row_num) override + { + assert(builder); + auto & struct_builder = assert_cast(*builder); + if (auto tuple_column = typeid_cast(column.get())) + writeRow(tuple_column->getColumnsCopy(), struct_builder, row_num); + else + writeRow(Columns{column}, struct_builder, row_num); + return std::nullopt; + } + + void writeRow(const Columns & columns, StructBuilder & struct_builder, size_t row_num) + { + for (size_t i = 0; i != columns.size(); ++i) + { + const auto & field = fields[i]; + size_t field_index = field.getIndex(); + if (likely(!struct_builder.field_builders[field_index])) + struct_builder.field_builders[field_index] = initStructFieldBuilderIfNeeded( + columns[i], row_num, struct_builder.impl, field, fields_types[i], nested_field_sizes[i]); + + auto value = field_serializers[i]->writeRow(columns[i], struct_builder.field_builders[field_index].get(), row_num); + if (value) + struct_builder.impl.set(field, *value); + } + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto struct_reader = reader.as(); + if (auto * tuple_column = typeid_cast(&column)) + { + for (size_t i = 0; i != tuple_column->tupleSize(); ++i) + field_serializers[i]->readRow(tuple_column->getColumn(i), struct_reader.get(fields[i])); + } + else + field_serializers[0]->readRow(column, struct_reader.get(fields[0])); + } + + void readRow(MutableColumns & columns, const capnp::DynamicStruct::Reader & reader) + { + for (size_t i = 0; i != columns.size(); ++i) + field_serializers[i]->readRow(*columns[i], reader.get(fields[i])); + } + + private: + void initialize(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + { + field_serializers.reserve(data_types.size()); + fields.reserve(data_types.size()); + fields_types.reserve(data_types.size()); + nested_field_sizes.reserve(data_types.size()); + for (size_t i = 0; i != data_types.size(); ++i) + { + auto [field_name, _] = splitFieldName(names[i]); + auto field = findFieldByName(schema, field_name); + if (!field) + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto schema doesn't contain field with name {}", field_name); + + fields.push_back(*field); + auto capnp_type = field->getType(); + fields_types.push_back(capnp_type); + nested_field_sizes.push_back(capnp_type.isStruct() ? capnp_type.asStruct().getFields().size() : 0); + field_serializers.push_back(createSerializer(data_types[i], names[i], capnp_type, settings)); + } + } + + std::vector> field_serializers; + std::vector fields; + std::vector nested_field_sizes; + std::vector fields_types; + }; + + std::unique_ptr createSerializer(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + auto [field_name, nested_name] = splitFieldName(name); + if (!nested_name.empty() && !capnp_type.isList()) + { + if (!capnp_type.isStruct()) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); + + return std::make_unique(DataTypes{type}, Names{nested_name}, capnp_type.asStruct(), settings); + } + + switch (type->getTypeId()) + { + case TypeIndex::Int8: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt8: + if (isBool(type)) + return createIntegerSerializer(type, name, capnp_type); + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int16: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt16: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int32: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt32: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int64: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt64: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int128: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::UInt128: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Int256: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::UInt256: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Float32: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Float64: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Date: + return std::make_unique(type, name, capnp_type); + case TypeIndex::Date32: + return std::make_unique(type, name, capnp_type); + case TypeIndex::DateTime: + return std::make_unique(type, name, capnp_type); + case TypeIndex::DateTime64: + return std::make_unique(type, name, capnp_type); + case TypeIndex::Decimal32: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Decimal64: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Decimal128: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Decimal256: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::IPv4: + return std::make_unique(type, name, capnp_type); + case TypeIndex::IPv6: + return std::make_unique(type, name, capnp_type); + case TypeIndex::UUID: + return std::make_unique(type, name, capnp_type); + case TypeIndex::Enum8: + return std::make_unique>(type, name, capnp_type, settings.enum_comparing_mode); + case TypeIndex::Enum16: + return std::make_unique>(type, name, capnp_type, settings.enum_comparing_mode); + case TypeIndex::String: + if (capnp_type.isData()) + return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); + case TypeIndex::FixedString: + if (capnp_type.isData()) + return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); + case TypeIndex::LowCardinality: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Nullable: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Array: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Map: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Tuple: + return std::make_unique(type, name, capnp_type, settings); + default: + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Type {} is not supported in CapnProto format", type->getName()); + } + } +} + +class CapnProtoSerializer::Impl +{ +public: + Impl(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + : struct_serializer(std::make_unique(data_types, names, schema, settings)) + , fields_size(schema.getFields().size()) + { + } + + void writeRow(const Columns & columns, capnp::DynamicStruct::Builder builder, size_t row_num) + { + StructBuilder struct_builder(std::move(builder), fields_size); + struct_serializer->writeRow(columns, struct_builder, row_num); + } + + void readRow(MutableColumns & columns, capnp::DynamicStruct::Reader & reader) + { + struct_serializer->readRow(columns, reader); + } + +private: + std::unique_ptr struct_serializer; + size_t fields_size; +}; + +CapnProtoSerializer::CapnProtoSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + : serializer_impl(std::make_unique(data_types, names, schema, settings)) +{ +} + +void CapnProtoSerializer::writeRow(const Columns & columns, capnp::DynamicStruct::Builder builder, size_t row_num) +{ + serializer_impl->writeRow(columns, std::move(builder), row_num); +} + +void CapnProtoSerializer::readRow(MutableColumns & columns, capnp::DynamicStruct::Reader & reader) +{ + serializer_impl->readRow(columns, reader); +} + +CapnProtoSerializer::~CapnProtoSerializer() = default; + +} diff --git a/src/Formats/CapnProtoSerializer.h b/src/Formats/CapnProtoSerializer.h new file mode 100644 index 00000000000..efae797875b --- /dev/null +++ b/src/Formats/CapnProtoSerializer.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class CapnProtoSerializer +{ +public: + CapnProtoSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings); + + void writeRow(const Columns & columns, capnp::DynamicStruct::Builder builder, size_t row_num); + + void readRow(MutableColumns & columns, capnp::DynamicStruct::Reader & reader); + + ~CapnProtoSerializer(); + +private: + class Impl; + std::unique_ptr serializer_impl; +}; + +} diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp deleted file mode 100644 index d6c032408bb..00000000000 --- a/src/Formats/CapnProtoUtils.cpp +++ /dev/null @@ -1,734 +0,0 @@ -#include - -#if USE_CAPNP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_CAPN_PROTO_SCHEMA; - extern const int THERE_IS_NO_COLUMN; - extern const int BAD_TYPE_OF_FIELD; - extern const int CAPN_PROTO_BAD_CAST; - extern const int FILE_DOESNT_EXIST; - extern const int UNKNOWN_EXCEPTION; - extern const int INCORRECT_DATA; - extern const int CAPN_PROTO_BAD_TYPE; - extern const int BAD_ARGUMENTS; -} - -std::pair splitCapnProtoFieldName(const String & name) -{ - const auto * begin = name.data(); - const auto * end = name.data() + name.size(); - const auto * it = find_first_symbols<'_', '.'>(begin, end); - String first = String(begin, it); - String second = it == end ? "" : String(it + 1, end); - return {first, second}; -} - -capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info) -{ - capnp::ParsedSchema schema; - try - { - int fd; - KJ_SYSCALL(fd = open(schema_info.schemaDirectory().data(), O_RDONLY)); // NOLINT(bugprone-suspicious-semicolon) - auto schema_dir = kj::newDiskDirectory(kj::OsFileHandle(fd)); - schema = impl.parseFromDirectory(*schema_dir, kj::Path::parse(schema_info.schemaPath()), {}); - } - catch (const kj::Exception & e) - { - /// That's not good to determine the type of error by its description, but - /// this is the only way to do it here, because kj doesn't specify the type of error. - auto description = std::string_view(e.getDescription().cStr()); - if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); - - if (description.find("Parse error") != String::npos) - throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "Cannot parse CapnProto schema {}:{}", schema_info.schemaPath(), e.getLine()); - - throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, - "Unknown exception while parsing CapnProto schema: {}, schema dir and file: {}, {}", - description, schema_info.schemaDirectory(), schema_info.schemaPath()); - } - - auto message_maybe = schema.findNested(schema_info.messageName()); - auto * message_schema = kj::_::readMaybe(message_maybe); - if (!message_schema) - throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, - "CapnProto schema doesn't contain message with name {}", schema_info.messageName()); - return message_schema->asStruct(); -} - -bool compareEnumNames(const String & first, const String & second, FormatSettings::EnumComparingMode mode) -{ - if (mode == FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE) - return boost::algorithm::to_lower_copy(first) == boost::algorithm::to_lower_copy(second); - return first == second; -} - -static const std::map capnp_simple_type_names = -{ - {capnp::schema::Type::Which::BOOL, "Bool"}, - {capnp::schema::Type::Which::VOID, "Void"}, - {capnp::schema::Type::Which::INT8, "Int8"}, - {capnp::schema::Type::Which::INT16, "Int16"}, - {capnp::schema::Type::Which::INT32, "Int32"}, - {capnp::schema::Type::Which::INT64, "Int64"}, - {capnp::schema::Type::Which::UINT8, "UInt8"}, - {capnp::schema::Type::Which::UINT16, "UInt16"}, - {capnp::schema::Type::Which::UINT32, "UInt32"}, - {capnp::schema::Type::Which::UINT64, "UInt64"}, - {capnp::schema::Type::Which::FLOAT32, "Float32"}, - {capnp::schema::Type::Which::FLOAT64, "Float64"}, - {capnp::schema::Type::Which::TEXT, "Text"}, - {capnp::schema::Type::Which::DATA, "Data"}, - {capnp::schema::Type::Which::INTERFACE, "Interface"}, - {capnp::schema::Type::Which::ANY_POINTER, "AnyPointer"}, -}; - -static bool checkIfStructContainsUnnamedUnion(const capnp::StructSchema & struct_schema) -{ - return struct_schema.getFields().size() != struct_schema.getNonUnionFields().size(); -} - -static bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema) -{ - return struct_schema.getFields().size() == struct_schema.getUnionFields().size(); -} - -/// Get full name of type for better exception messages. -static String getCapnProtoFullTypeName(const capnp::Type & type) -{ - switch (type.which()) - { - case capnp::schema::Type::Which::STRUCT: - { - auto struct_schema = type.asStruct(); - - auto non_union_fields = struct_schema.getNonUnionFields(); - std::vector non_union_field_names; - for (auto nested_field : non_union_fields) - non_union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); - - auto union_fields = struct_schema.getUnionFields(); - std::vector union_field_names; - for (auto nested_field : union_fields) - union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); - - String union_name = "Union(" + boost::algorithm::join(union_field_names, ", ") + ")"; - /// Check if the struct is a named union. - if (non_union_field_names.empty()) - return union_name; - - String type_name = "Struct(" + boost::algorithm::join(non_union_field_names, ", "); - /// Check if the struct contains unnamed union. - if (!union_field_names.empty()) - type_name += ", " + union_name; - type_name += ")"; - return type_name; - } - case capnp::schema::Type::Which::LIST: - return "List(" + getCapnProtoFullTypeName(type.asList().getElementType()) + ")"; - case capnp::schema::Type::Which::ENUM: - { - auto enum_schema = type.asEnum(); - String enum_name = "Enum("; - auto enumerants = enum_schema.getEnumerants(); - for (unsigned i = 0; i != enumerants.size(); ++i) - { - enum_name += String(enumerants[i].getProto().getName()) + " = " + std::to_string(enumerants[i].getOrdinal()); - if (i + 1 != enumerants.size()) - enum_name += ", "; - } - enum_name += ")"; - return enum_name; - } - default: - auto it = capnp_simple_type_names.find(type.which()); - if (it == capnp_simple_type_names.end()) - throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Unknown CapnProto type"); - return it->second; - } -} - -template -static bool checkEnums(const capnp::Type & capnp_type, const DataTypePtr column_type, FormatSettings::EnumComparingMode mode, UInt64 max_value, String & error_message) -{ - if (!capnp_type.isEnum()) - return false; - - auto enum_schema = capnp_type.asEnum(); - bool to_lower = mode == FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE; - const auto * enum_type = assert_cast *>(column_type.get()); - const auto & enum_values = dynamic_cast &>(*enum_type); - - auto enumerants = enum_schema.getEnumerants(); - if (mode == FormatSettings::EnumComparingMode::BY_VALUES) - { - /// In CapnProto Enum fields are numbered sequentially starting from zero. - if (enumerants.size() > max_value) - { - error_message += "Enum from CapnProto schema contains values that is out of range for Clickhouse Enum"; - return false; - } - - auto values = enum_values.getSetOfAllValues(); - std::unordered_set capn_enum_values; - for (auto enumerant : enumerants) - capn_enum_values.insert(Type(enumerant.getOrdinal())); - auto result = values == capn_enum_values; - if (!result) - error_message += "The set of values in Enum from CapnProto schema is different from the set of values in ClickHouse Enum"; - return result; - } - - auto names = enum_values.getSetOfAllNames(to_lower); - std::unordered_set capn_enum_names; - - for (auto enumerant : enumerants) - { - String name = enumerant.getProto().getName(); - capn_enum_names.insert(to_lower ? boost::algorithm::to_lower_copy(name) : name); - } - - auto result = names == capn_enum_names; - if (!result) - error_message += "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"; - return result; -} - -static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name); - -static bool checkNullableType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name) -{ - if (!capnp_type.isStruct()) - return false; - - /// Check that struct is a named union of type VOID and one arbitrary type. - auto struct_schema = capnp_type.asStruct(); - if (!checkIfStructIsNamedUnion(struct_schema)) - return false; - - auto union_fields = struct_schema.getUnionFields(); - if (union_fields.size() != 2) - return false; - - auto first = union_fields[0]; - auto second = union_fields[1]; - - auto nested_type = assert_cast(data_type.get())->getNestedType(); - if (first.getType().isVoid()) - return checkCapnProtoType(second.getType(), nested_type, mode, error_message, column_name); - if (second.getType().isVoid()) - return checkCapnProtoType(first.getType(), nested_type, mode, error_message, column_name); - return false; -} - -static bool checkTupleType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message) -{ - if (!capnp_type.isStruct()) - return false; - auto struct_schema = capnp_type.asStruct(); - - if (checkIfStructIsNamedUnion(struct_schema)) - return false; - - if (checkIfStructContainsUnnamedUnion(struct_schema)) - { - error_message += "CapnProto struct contains unnamed union"; - return false; - } - - const auto * tuple_data_type = assert_cast(data_type.get()); - auto nested_types = tuple_data_type->getElements(); - if (nested_types.size() != struct_schema.getFields().size()) - { - error_message += "Tuple and Struct types have different sizes"; - return false; - } - - bool have_explicit_names = tuple_data_type->haveExplicitNames(); - const auto & nested_names = tuple_data_type->getElementNames(); - for (uint32_t i = 0; i != nested_names.size(); ++i) - { - if (have_explicit_names) - { - KJ_IF_MAYBE (field, struct_schema.findFieldByName(nested_names[i])) - { - if (!checkCapnProtoType(field->getType(), nested_types[tuple_data_type->getPositionByName(nested_names[i])], mode, error_message, nested_names[i])) - return false; - } - else - { - error_message += "CapnProto struct doesn't contain a field with name " + nested_names[i]; - return false; - } - } - else if (!checkCapnProtoType(struct_schema.getFields()[i].getType(), nested_types[tuple_data_type->getPositionByName(nested_names[i])], mode, error_message, nested_names[i])) - return false; - } - - return true; -} - -static bool checkArrayType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name) -{ - if (!capnp_type.isList()) - return false; - auto list_schema = capnp_type.asList(); - auto nested_type = assert_cast(data_type.get())->getNestedType(); - - auto [field_name, nested_name] = splitCapnProtoFieldName(column_name); - if (!nested_name.empty() && list_schema.getElementType().isStruct()) - { - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(field, struct_schema.findFieldByName(nested_name)) - return checkCapnProtoType(field->getType(), nested_type, mode, error_message, nested_name); - - error_message += "Element type of List {} doesn't contain field with name " + nested_name; - return false; - } - - return checkCapnProtoType(list_schema.getElementType(), nested_type, mode, error_message, column_name); -} - -static bool checkMapType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message) -{ - /// We output/input Map type as follow CapnProto schema - /// - /// struct Map { - /// struct Entry { - /// key @0: Key; - /// value @1: Value; - /// } - /// entries @0 :List(Entry); - /// } - - if (!capnp_type.isStruct()) - return false; - auto struct_schema = capnp_type.asStruct(); - - if (checkIfStructContainsUnnamedUnion(struct_schema)) - { - error_message += "CapnProto struct contains unnamed union"; - return false; - } - - if (struct_schema.getFields().size() != 1) - { - error_message += "CapnProto struct that represents Map type can contain only one field"; - return false; - } - - const auto & field_type = struct_schema.getFields()[0].getType(); - if (!field_type.isList()) - { - error_message += "Field of CapnProto struct that represents Map is not a list"; - return false; - } - - auto list_element_type = field_type.asList().getElementType(); - if (!list_element_type.isStruct()) - { - error_message += "Field of CapnProto struct that represents Map is not a list of structs"; - return false; - } - - auto key_value_struct = list_element_type.asStruct(); - if (checkIfStructContainsUnnamedUnion(key_value_struct)) - { - error_message += "CapnProto struct contains unnamed union"; - return false; - } - - if (key_value_struct.getFields().size() != 2) - { - error_message += "Key-value structure for Map struct should have exactly 2 fields"; - return false; - } - - const auto & map_type = assert_cast(*data_type); - DataTypes types = {map_type.getKeyType(), map_type.getValueType()}; - Names names = {"key", "value"}; - - for (size_t i = 0; i != types.size(); ++i) - { - KJ_IF_MAYBE(field, key_value_struct.findFieldByName(names[i])) - { - if (!checkCapnProtoType(field->getType(), types[i], mode, error_message, names[i])) - return false; - } - else - { - error_message += R"(Key-value structure for Map struct should have exactly 2 fields with names "key" and "value")"; - return false; - } - } - - return true; -} - -static bool isCapnInteger(const capnp::Type & capnp_type) -{ - return capnp_type.isInt8() || capnp_type.isUInt8() || capnp_type.isInt16() || capnp_type.isUInt16() || capnp_type.isInt32() - || capnp_type.isUInt32() || capnp_type.isInt64() || capnp_type.isUInt64(); -} - -static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name) -{ - switch (data_type->getTypeId()) - { - case TypeIndex::UInt8: - return capnp_type.isBool() || isCapnInteger(capnp_type); - case TypeIndex::Int8: [[fallthrough]]; - case TypeIndex::Int16: [[fallthrough]]; - case TypeIndex::UInt16: [[fallthrough]]; - case TypeIndex::Int32: [[fallthrough]]; - case TypeIndex::UInt32: [[fallthrough]]; - case TypeIndex::Int64: [[fallthrough]]; - case TypeIndex::UInt64: - /// Allow integer conversions durin input/output. - return isCapnInteger(capnp_type); - case TypeIndex::Date: - return capnp_type.isUInt16(); - case TypeIndex::DateTime: [[fallthrough]]; - case TypeIndex::IPv4: - return capnp_type.isUInt32(); - case TypeIndex::Date32: [[fallthrough]]; - case TypeIndex::Decimal32: - return capnp_type.isInt32() || capnp_type.isUInt32(); - case TypeIndex::DateTime64: [[fallthrough]]; - case TypeIndex::Decimal64: - return capnp_type.isInt64() || capnp_type.isUInt64(); - case TypeIndex::Float32:[[fallthrough]]; - case TypeIndex::Float64: - /// Allow converting between Float32 and isFloat64 - return capnp_type.isFloat32() || capnp_type.isFloat64(); - case TypeIndex::Enum8: - return checkEnums(capnp_type, data_type, mode, INT8_MAX, error_message); - case TypeIndex::Enum16: - return checkEnums(capnp_type, data_type, mode, INT16_MAX, error_message); - case TypeIndex::Int128: [[fallthrough]]; - case TypeIndex::UInt128: [[fallthrough]]; - case TypeIndex::Int256: [[fallthrough]]; - case TypeIndex::UInt256: [[fallthrough]]; - case TypeIndex::Decimal128: [[fallthrough]]; - case TypeIndex::Decimal256: - return capnp_type.isData(); - case TypeIndex::Tuple: - return checkTupleType(capnp_type, data_type, mode, error_message); - case TypeIndex::Nullable: - { - auto result = checkNullableType(capnp_type, data_type, mode, error_message, column_name); - if (!result) - error_message += "Nullable can be represented only as a named union of type Void and nested type"; - return result; - } - case TypeIndex::Array: - return checkArrayType(capnp_type, data_type, mode, error_message, column_name); - case TypeIndex::LowCardinality: - return checkCapnProtoType(capnp_type, assert_cast(data_type.get())->getDictionaryType(), mode, error_message, column_name); - case TypeIndex::FixedString: [[fallthrough]]; - case TypeIndex::IPv6: [[fallthrough]]; - case TypeIndex::String: - return capnp_type.isText() || capnp_type.isData(); - case TypeIndex::Map: - return checkMapType(capnp_type, data_type, mode, error_message); - default: - return false; - } -} - -capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Reader & struct_reader, const String & name) -{ - auto [field_name, nested_name] = splitCapnProtoFieldName(name); - KJ_IF_MAYBE(field, struct_reader.getSchema().findFieldByName(field_name)) - { - capnp::DynamicValue::Reader field_reader; - try - { - field_reader = struct_reader.get(*field); - } - catch (const kj::Exception & e) - { - throw Exception(ErrorCodes::INCORRECT_DATA, - "Cannot extract field value from struct by provided schema, error: " - "{} Perhaps the data was generated by another schema", String(e.getDescription().cStr())); - } - - if (nested_name.empty()) - return field_reader; - - /// Support reading Nested as List of Structs. - if (field_reader.getType() == capnp::DynamicValue::LIST) - { - auto list_schema = field->getType().asList(); - if (!list_schema.getElementType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Element type of List {} is not a struct", field_name); - - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(nested_field, struct_schema.findFieldByName(nested_name)) - return field_reader; - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Element type of List {} doesn't contain field with name \"{}\"", field_name, nested_name); - } - - if (field_reader.getType() != capnp::DynamicValue::STRUCT) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); - - return getReaderByColumnName(field_reader.as(), nested_name); - } - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto struct doesn't contain field with name {}", field_name); -} - -std::pair getStructBuilderAndFieldByColumnName(capnp::DynamicStruct::Builder struct_builder, const String & name) -{ - auto [field_name, nested_name] = splitCapnProtoFieldName(name); - KJ_IF_MAYBE(field, struct_builder.getSchema().findFieldByName(field_name)) - { - if (nested_name.empty()) - return {struct_builder, *field}; - - auto field_builder = struct_builder.get(*field); - - /// Support reading Nested as List of Structs. - if (field_builder.getType() == capnp::DynamicValue::LIST) - { - auto list_schema = field->getType().asList(); - if (!list_schema.getElementType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Element type of List {} is not a struct", field_name); - - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(nested_field, struct_schema.findFieldByName(nested_name)) - return {struct_builder, *field}; - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Element type of List {} doesn't contain field with name \"{}\"", field_name, nested_name); - } - - if (field_builder.getType() != capnp::DynamicValue::STRUCT) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); - - return getStructBuilderAndFieldByColumnName(field_builder.as(), nested_name); - } - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto struct doesn't contain field with name {}", field_name); -} - -static std::pair getFieldByName(const capnp::StructSchema & schema, const String & name) -{ - auto [field_name, nested_name] = splitCapnProtoFieldName(name); - KJ_IF_MAYBE(field, schema.findFieldByName(field_name)) - { - if (nested_name.empty()) - return {*field, name}; - - /// Support reading Nested as List of Structs. - if (field->getType().isList()) - { - auto list_schema = field->getType().asList(); - if (!list_schema.getElementType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Element type of List {} is not a struct", field_name); - - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(nested_field, struct_schema.findFieldByName(nested_name)) - return {*field, name}; - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Element type of List {} doesn't contain field with name \"{}\"", field_name, nested_name); - } - - if (!field->getType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); - - return getFieldByName(field->getType().asStruct(), nested_name); - } - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto schema doesn't contain field with name {}", field_name); -} - -void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode) -{ - /// Firstly check that struct doesn't contain unnamed union, because we don't support it. - if (checkIfStructContainsUnnamedUnion(schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Schema contains unnamed union that is not supported"); - auto names_and_types = header.getNamesAndTypesList(); - String additional_error_message; - for (auto & [name, type] : names_and_types) - { - auto [field, field_name] = getFieldByName(schema, name); - if (!checkCapnProtoType(field.getType(), type, mode, additional_error_message, field_name)) - { - auto e = Exception( - ErrorCodes::CAPN_PROTO_BAD_CAST, - "Cannot convert ClickHouse type {} to CapnProto type {}", - type->getName(), - getCapnProtoFullTypeName(field.getType())); - if (!additional_error_message.empty()) - e.addMessage(additional_error_message); - throw std::move(e); - } - } -} - -template -static DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) -{ - std::vector> values; - for (auto enumerant : enumerants) - values.emplace_back(enumerant.getProto().getName(), ValueType(enumerant.getOrdinal())); - return std::make_shared>(std::move(values)); -} - -static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) -{ - auto enumerants = enum_schema.getEnumerants(); - if (enumerants.size() < 128) - return getEnumDataTypeFromEnumerants(enumerants); - if (enumerants.size() < 32768) - return getEnumDataTypeFromEnumerants(enumerants); - - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "ClickHouse supports only 8 and 16-bit Enums"); -} - -static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) -{ - switch (capnp_type.which()) - { - case capnp::schema::Type::INT8: - return std::make_shared(); - case capnp::schema::Type::INT16: - return std::make_shared(); - case capnp::schema::Type::INT32: - return std::make_shared(); - case capnp::schema::Type::INT64: - return std::make_shared(); - case capnp::schema::Type::BOOL: [[fallthrough]]; - case capnp::schema::Type::UINT8: - return std::make_shared(); - case capnp::schema::Type::UINT16: - return std::make_shared(); - case capnp::schema::Type::UINT32: - return std::make_shared(); - case capnp::schema::Type::UINT64: - return std::make_shared(); - case capnp::schema::Type::FLOAT32: - return std::make_shared(); - case capnp::schema::Type::FLOAT64: - return std::make_shared(); - case capnp::schema::Type::DATA: [[fallthrough]]; - case capnp::schema::Type::TEXT: - return std::make_shared(); - case capnp::schema::Type::ENUM: - return getEnumDataTypeFromEnumSchema(capnp_type.asEnum()); - case capnp::schema::Type::LIST: - { - auto list_schema = capnp_type.asList(); - auto nested_type = getDataTypeFromCapnProtoType(list_schema.getElementType(), skip_unsupported_fields); - if (!nested_type) - return nullptr; - return std::make_shared(nested_type); - } - case capnp::schema::Type::STRUCT: - { - auto struct_schema = capnp_type.asStruct(); - - - if (struct_schema.getFields().size() == 0) - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Empty messages are not supported"); - } - - /// Check if it can be Nullable. - if (checkIfStructIsNamedUnion(struct_schema)) - { - auto fields = struct_schema.getUnionFields(); - if (fields.size() != 2 || (!fields[0].getType().isVoid() && !fields[1].getType().isVoid())) - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unions are not supported"); - } - auto value_type = fields[0].getType().isVoid() ? fields[1].getType() : fields[0].getType(); - if (value_type.isStruct() || value_type.isList()) - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Tuples and Lists cannot be inside Nullable"); - } - - auto nested_type = getDataTypeFromCapnProtoType(value_type, skip_unsupported_fields); - if (!nested_type) - return nullptr; - return std::make_shared(nested_type); - } - - if (checkIfStructContainsUnnamedUnion(struct_schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); - - /// Treat Struct as Tuple. - DataTypes nested_types; - Names nested_names; - for (auto field : struct_schema.getNonUnionFields()) - { - auto nested_type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); - if (!nested_type) - continue; - nested_names.push_back(field.getProto().getName()); - nested_types.push_back(nested_type); - } - if (nested_types.empty()) - return nullptr; - return std::make_shared(std::move(nested_types), std::move(nested_names)); - } - default: - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unsupported CapnProtoType: {}", getCapnProtoFullTypeName(capnp_type)); - } - } -} - -NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields) -{ - if (checkIfStructContainsUnnamedUnion(schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); - - NamesAndTypesList names_and_types; - for (auto field : schema.getNonUnionFields()) - { - auto name = field.getProto().getName(); - auto type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); - if (type) - names_and_types.emplace_back(name, type); - } - if (names_and_types.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Cannot convert CapnProto schema to ClickHouse table schema, all fields have unsupported types"); - - return names_and_types; -} - -} - -#endif diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 3a2e818d540..384c6a725dc 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -325,16 +325,16 @@ struct FormatSettings /// For capnProto format we should determine how to /// compare ClickHouse Enum and Enum from schema. - enum class EnumComparingMode + enum class CapnProtoEnumComparingMode { BY_NAMES, // Names in enums should be the same, values can be different. BY_NAMES_CASE_INSENSITIVE, // Case-insensitive name comparison. BY_VALUES, // Values should be the same, names can be different. }; - struct + struct CapnProto { - EnumComparingMode enum_comparing_mode = EnumComparingMode::BY_VALUES; + CapnProtoEnumComparingMode enum_comparing_mode = CapnProtoEnumComparingMode::BY_VALUES; bool skip_fields_with_unsupported_types_in_schema_inference = false; } capn_proto; diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index 2f84e9bde3c..e686ae86997 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -9,23 +9,6 @@ #include #include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include - namespace DB { @@ -35,16 +18,14 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } -CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_) - : IRowInputFormat(std::move(header), in_, std::move(params_)) +CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings) + : IRowInputFormat(std::move(header_), in_, std::move(params_)) , parser(std::make_shared()) - , format_settings(format_settings_) - , column_types(getPort().getHeader().getDataTypes()) - , column_names(getPort().getHeader().getNames()) { // Parse the schema and fetch the root object - root = parser->getMessageSchema(info); - checkCapnProtoSchemaStructure(root, getPort().getHeader(), format_settings.capn_proto.enum_comparing_mode); + schema = parser->getMessageSchema(info); + const auto & header = getPort().getHeader(); + serializer = std::make_unique(header.getDataTypes(), header.getNames(), schema, format_settings.capn_proto); } kj::Array CapnProtoRowInputFormat::readMessage() @@ -82,213 +63,6 @@ kj::Array CapnProtoRowInputFormat::readMessage() return msg; } -static void insertInteger(IColumn & column, const DataTypePtr & column_type, UInt64 value) -{ - switch (column_type->getTypeId()) - { - case TypeIndex::Int8: - assert_cast(column).insertValue(value); - break; - case TypeIndex::UInt8: - assert_cast(column).insertValue(value); - break; - case TypeIndex::Int16: - assert_cast(column).insertValue(value); - break; - case TypeIndex::Date: [[fallthrough]]; - case TypeIndex::UInt16: - assert_cast(column).insertValue(value); - break; - case TypeIndex::Int32: - assert_cast(column).insertValue(static_cast(value)); - break; - case TypeIndex::DateTime: [[fallthrough]]; - case TypeIndex::UInt32: - assert_cast(column).insertValue(static_cast(value)); - break; - case TypeIndex::IPv4: - assert_cast(column).insertValue(IPv4(static_cast(value))); - break; - case TypeIndex::Int64: - assert_cast(column).insertValue(value); - break; - case TypeIndex::UInt64: - assert_cast(column).insertValue(value); - break; - case TypeIndex::DateTime64: - assert_cast &>(column).insertValue(value); - break; - case TypeIndex::Decimal32: - assert_cast &>(column).insertValue(static_cast(value)); - break; - case TypeIndex::Decimal64: - assert_cast &>(column).insertValue(value); - break; - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type {} cannot be parsed from integer", column_type->getName()); - } -} - -static void insertFloat(IColumn & column, const DataTypePtr & column_type, Float64 value) -{ - switch (column_type->getTypeId()) - { - case TypeIndex::Float32: - assert_cast(column).insertValue(static_cast(value)); - break; - case TypeIndex::Float64: - assert_cast(column).insertValue(value); - break; - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type is not a float."); - } -} - -template -static void insertData(IColumn & column, const DataTypePtr & column_type, Value value) -{ - if (column_type->haveMaximumSizeOfValue() && value.size() != column_type->getSizeOfValueInMemory()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", column_type->getName(), value.size()); - - column.insertData(reinterpret_cast(value.begin()), value.size()); -} - -template -static void insertEnum(IColumn & column, const DataTypePtr & column_type, const capnp::DynamicEnum & enum_value, FormatSettings::EnumComparingMode enum_comparing_mode) -{ - auto enumerant = *kj::_::readMaybe(enum_value.getEnumerant()); - auto enum_type = assert_cast *>(column_type.get()); - DataTypePtr nested_type = std::make_shared>(); - switch (enum_comparing_mode) - { - case FormatSettings::EnumComparingMode::BY_VALUES: - insertInteger(column, nested_type, Int64(enumerant.getOrdinal())); - return; - case FormatSettings::EnumComparingMode::BY_NAMES: - insertInteger(column, nested_type, Int64(enum_type->getValue(String(enumerant.getProto().getName())))); - return; - case FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE: - { - /// Find the same enum name case insensitive. - String enum_name = enumerant.getProto().getName(); - for (auto & name : enum_type->getAllRegisteredNames()) - { - if (compareEnumNames(name, enum_name, enum_comparing_mode)) - { - insertInteger(column, nested_type, Int64(enum_type->getValue(name))); - break; - } - } - } - } -} - -static void insertValue(IColumn & column, const DataTypePtr & column_type, const String & column_name, const capnp::DynamicValue::Reader & value, FormatSettings::EnumComparingMode enum_comparing_mode) -{ - if (column_type->lowCardinality()) - { - auto & lc_column = assert_cast(column); - auto tmp_column = lc_column.getDictionary().getNestedColumn()->cloneEmpty(); - auto dict_type = assert_cast(column_type.get())->getDictionaryType(); - insertValue(*tmp_column, dict_type, column_name, value, enum_comparing_mode); - lc_column.insertFromFullColumn(*tmp_column, 0); - return; - } - - switch (value.getType()) - { - case capnp::DynamicValue::Type::INT: - insertInteger(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::UINT: - insertInteger(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::FLOAT: - insertFloat(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::BOOL: - insertInteger(column, column_type, UInt64(value.as())); - break; - case capnp::DynamicValue::Type::DATA: - insertData(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::TEXT: - insertData(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::ENUM: - if (column_type->getTypeId() == TypeIndex::Enum8) - insertEnum(column, column_type, value.as(), enum_comparing_mode); - else - insertEnum(column, column_type, value.as(), enum_comparing_mode); - break; - case capnp::DynamicValue::LIST: - { - auto list_value = value.as(); - auto & column_array = assert_cast(column); - auto & offsets = column_array.getOffsets(); - offsets.push_back(offsets.back() + list_value.size()); - - auto & nested_column = column_array.getData(); - auto nested_type = assert_cast(column_type.get())->getNestedType(); - for (const auto & nested_value : list_value) - insertValue(nested_column, nested_type, column_name, nested_value, enum_comparing_mode); - break; - } - case capnp::DynamicValue::Type::STRUCT: - { - auto struct_value = value.as(); - if (column_type->isNullable()) - { - auto & nullable_column = assert_cast(column); - auto field = *kj::_::readMaybe(struct_value.which()); - if (field.getType().isVoid()) - nullable_column.insertDefault(); - else - { - auto & nested_column = nullable_column.getNestedColumn(); - auto nested_type = assert_cast(column_type.get())->getNestedType(); - auto nested_value = struct_value.get(field); - insertValue(nested_column, nested_type, column_name, nested_value, enum_comparing_mode); - nullable_column.getNullMapData().push_back(0); - } - } - else if (isTuple(column_type)) - { - auto & tuple_column = assert_cast(column); - const auto * tuple_type = assert_cast(column_type.get()); - bool have_explicit_names = tuple_type->haveExplicitNames(); - auto struct_schema = struct_value.getSchema(); - for (uint32_t i = 0; i != tuple_column.tupleSize(); ++i) - insertValue( - tuple_column.getColumn(i), - tuple_type->getElements()[i], - tuple_type->getElementNames()[i], - struct_value.get(have_explicit_names ? struct_schema.getFieldByName(tuple_type->getElementNames()[i]) : struct_schema.getFields()[i]), - enum_comparing_mode); - } - else if (isMap(column_type)) - { - const auto & map_type = assert_cast(*column_type); - DataTypes key_value_types = {map_type.getKeyType(), map_type.getValueType()}; - Names key_value_names = {"key", "value"}; - auto entries_type = std::make_shared(std::make_shared(key_value_types, key_value_names)); - auto & entries_column = assert_cast(column).getNestedColumn(); - auto entries_field = struct_value.getSchema().getFields()[0]; - insertValue(entries_column, entries_type, column_name, struct_value.get(entries_field), enum_comparing_mode); - } - else - { - /// It can be nested column from Nested type. - auto [field_name, nested_name] = splitCapnProtoFieldName(column_name); - insertValue(column, column_type, nested_name, struct_value.get(nested_name), enum_comparing_mode); - } - break; - } - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CapnProto value type."); - } -} - bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { if (in->eof()) @@ -298,12 +72,8 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension { auto array = readMessage(); capnp::FlatArrayMessageReader msg(array); - auto root_reader = msg.getRoot(root); - for (size_t i = 0; i != columns.size(); ++i) - { - auto value = getReaderByColumnName(root_reader, column_names[i]); - insertValue(*columns[i], column_types[i], column_names[i], value, format_settings.capn_proto.enum_comparing_mode); - } + auto root_reader = msg.getRoot(schema); + serializer->readRow(columns, root_reader); } catch (const kj::Exception & e) { @@ -343,7 +113,14 @@ void registerInputFormatCapnProto(FormatFactory & factory) factory.markFormatSupportsSubsetOfColumns("CapnProto"); factory.registerFileExtension("capnp", "CapnProto"); factory.registerAdditionalInfoForSchemaCacheGetter( - "CapnProto", [](const FormatSettings & settings) { return fmt::format("format_schema={}", settings.schema.format_schema); }); + "CapnProto", + [](const FormatSettings & settings) + { + return fmt::format( + "format_schema={}, skip_fields_with_unsupported_types_in_schema_inference={}", + settings.schema.format_schema, + settings.capn_proto.skip_fields_with_unsupported_types_in_schema_inference); + }); } void registerCapnProtoSchemaReader(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h index cf23f22b643..06e94da123f 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h @@ -4,7 +4,8 @@ #if USE_CAPNP #include -#include +#include +#include #include #include @@ -33,10 +34,8 @@ private: kj::Array readMessage(); std::shared_ptr parser; - capnp::StructSchema root; - const FormatSettings format_settings; - DataTypes column_types; - Names column_names; + capnp::StructSchema schema; + std::unique_ptr serializer; }; class CapnProtoSchemaReader : public IExternalSchemaReader diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index 0225680b396..7dd18be27f4 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -1,28 +1,13 @@ #include #if USE_CAPNP -#include +#include #include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - namespace DB { @@ -45,252 +30,25 @@ CapnProtoRowOutputFormat::CapnProtoRowOutputFormat( WriteBuffer & out_, const Block & header_, const FormatSchemaInfo & info, - const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_), column_names(header_.getNames()), column_types(header_.getDataTypes()), output_stream(std::make_unique(out_)), format_settings(format_settings_) + const FormatSettings & format_settings) + : IRowOutputFormat(header_, out_) + , column_names(header_.getNames()) + , column_types(header_.getDataTypes()) + , output_stream(std::make_unique(out_)) { schema = schema_parser.getMessageSchema(info); - checkCapnProtoSchemaStructure(schema, getPort(PortKind::Main).getHeader(), format_settings.capn_proto.enum_comparing_mode); -} - -template -static capnp::DynamicEnum getDynamicEnum( - const ColumnPtr & column, - const DataTypePtr & data_type, - size_t row_num, - const capnp::EnumSchema & enum_schema, - FormatSettings::EnumComparingMode mode) -{ - const auto * enum_data_type = assert_cast *>(data_type.get()); - EnumValue enum_value = column->getInt(row_num); - if (mode == FormatSettings::EnumComparingMode::BY_VALUES) - return capnp::DynamicEnum(enum_schema, enum_value); - - auto enum_name = enum_data_type->getNameForValue(enum_value); - for (const auto enumerant : enum_schema.getEnumerants()) - { - if (compareEnumNames(String(enum_name), enumerant.getProto().getName(), mode)) - return capnp::DynamicEnum(enumerant); - } - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert CLickHouse Enum value to CapnProto Enum"); -} - -static capnp::DynamicValue::Builder initStructFieldBuilder(const ColumnPtr & column, size_t row_num, capnp::DynamicStruct::Builder & struct_builder, capnp::StructSchema::Field field) -{ - if (const auto * array_column = checkAndGetColumn(*column)) - { - size_t size = array_column->getOffsets()[row_num] - array_column->getOffsets()[row_num - 1]; - return struct_builder.init(field, static_cast(size)); - } - - if (field.getType().isStruct()) - return struct_builder.init(field); - - return struct_builder.get(field); -} - -static std::optional convertToDynamicValue( - const ColumnPtr & column, - const DataTypePtr & data_type, - size_t row_num, - const String & column_name, - capnp::DynamicValue::Builder builder, - FormatSettings::EnumComparingMode enum_comparing_mode, - std::vector> & temporary_text_data_storage) -{ - /// Here we don't do any types validation, because we did it in CapnProtoRowOutputFormat constructor. - - if (data_type->lowCardinality()) - { - const auto * lc_column = assert_cast(column.get()); - const auto & dict_type = assert_cast(data_type.get())->getDictionaryType(); - size_t index = lc_column->getIndexAt(row_num); - return convertToDynamicValue(lc_column->getDictionary().getNestedColumn(), dict_type, index, column_name, builder, enum_comparing_mode, temporary_text_data_storage); - } - - switch (builder.getType()) - { - case capnp::DynamicValue::Type::INT: - return capnp::DynamicValue::Reader(column->getInt(row_num)); - case capnp::DynamicValue::Type::UINT: - { - /// IPv4 column doesn't support getUInt method. - if (isIPv4(data_type)) - return capnp::DynamicValue::Reader(assert_cast(column.get())->getElement(row_num)); - return capnp::DynamicValue::Reader(column->getUInt(row_num)); - } - case capnp::DynamicValue::Type::BOOL: - return capnp::DynamicValue::Reader(column->getBool(row_num)); - case capnp::DynamicValue::Type::FLOAT: - return capnp::DynamicValue::Reader(column->getFloat64(row_num)); - case capnp::DynamicValue::Type::ENUM: - { - auto enum_schema = builder.as().getSchema(); - if (data_type->getTypeId() == TypeIndex::Enum8) - return capnp::DynamicValue::Reader( - getDynamicEnum(column, data_type, row_num, enum_schema, enum_comparing_mode)); - return capnp::DynamicValue::Reader( - getDynamicEnum(column, data_type, row_num, enum_schema, enum_comparing_mode)); - } - case capnp::DynamicValue::Type::DATA: - { - auto data = column->getDataAt(row_num); - return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); - } - case capnp::DynamicValue::Type::TEXT: - { - /// In TEXT type data should be null-terminated, but ClickHouse String data could not be. - /// To make data null-terminated we should copy it to temporary String object, but - /// capnp::Text::Reader works only with pointer to the data and it's size, so we should - /// guarantee that new String object life time is longer than capnp::Text::Reader life time. - /// To do this we store new String object in a temporary storage, passed in this function - /// by reference. We use unique_ptr instead of just String to avoid pointers - /// invalidation on vector reallocation. - temporary_text_data_storage.push_back(std::make_unique(column->getDataAt(row_num))); - auto & data = temporary_text_data_storage.back(); - return capnp::DynamicValue::Reader(capnp::Text::Reader(data->data(), data->size())); - } - case capnp::DynamicValue::Type::STRUCT: - { - auto struct_builder = builder.as(); - auto nested_struct_schema = struct_builder.getSchema(); - /// Struct can represent Tuple, Nullable (named union with two fields) or single column when it contains one nested column. - if (data_type->isNullable()) - { - const auto * nullable_type = assert_cast(data_type.get()); - const auto * nullable_column = assert_cast(column.get()); - auto fields = nested_struct_schema.getUnionFields(); - if (nullable_column->isNullAt(row_num)) - { - auto null_field = fields[0].getType().isVoid() ? fields[0] : fields[1]; - struct_builder.set(null_field, capnp::Void()); - } - else - { - auto value_field = fields[0].getType().isVoid() ? fields[1] : fields[0]; - struct_builder.clear(value_field); - const auto & nested_column = nullable_column->getNestedColumnPtr(); - auto value_builder = initStructFieldBuilder(nested_column, row_num, struct_builder, value_field); - auto value = convertToDynamicValue(nested_column, nullable_type->getNestedType(), row_num, column_name, value_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(value_field, *value); - } - } - else if (isTuple(data_type)) - { - const auto * tuple_data_type = assert_cast(data_type.get()); - const auto & nested_types = tuple_data_type->getElements(); - const auto & nested_names = tuple_data_type->getElementNames(); - const auto & nested_columns = assert_cast(column.get())->getColumns(); - bool have_explicit_names = tuple_data_type->haveExplicitNames(); - for (uint32_t i = 0; i != nested_names.size(); ++i) - { - capnp::StructSchema::Field nested_field = have_explicit_names ? nested_struct_schema.getFieldByName(nested_names[i]) : nested_struct_schema.getFields()[i]; - auto field_builder = initStructFieldBuilder(nested_columns[i], row_num, struct_builder, nested_field); - auto value = convertToDynamicValue(nested_columns[i], nested_types[i], row_num, nested_names[i], field_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(nested_field, *value); - } - } - else if (isMap(data_type)) - { - /// We output Map type as follow CapnProto schema - /// - /// struct Map { - /// struct Entry { - /// key @0: Key; - /// value @1: Value; - /// } - /// entries @0 :List(Entry); - /// } - /// - /// And we don't need to check that struct have this form here because we checked it before. - const auto & map_type = assert_cast(*data_type); - DataTypes key_value_types = {map_type.getKeyType(), map_type.getValueType()}; - Names key_value_names = {"key", "value"}; - auto entries_type = std::make_shared(std::make_shared(key_value_types, key_value_names)); - - /// Nested column in Map is actually Array(Tuple), so we can output it according to "entries" field schema. - const auto & entries_column = assert_cast(column.get())->getNestedColumnPtr(); - - auto entries_field = nested_struct_schema.getFields()[0]; - auto field_builder = initStructFieldBuilder(entries_column, row_num, struct_builder, entries_field); - auto entries_value = convertToDynamicValue(entries_column, entries_type, row_num, column_name, field_builder, enum_comparing_mode, temporary_text_data_storage); - if (entries_value) - struct_builder.set(entries_field, *entries_value); - } - else - { - /// It can be nested column from Nested type. - auto [field_name, nested_name] = splitCapnProtoFieldName(column_name); - auto nested_field = nested_struct_schema.getFieldByName(nested_name); - auto field_builder = initStructFieldBuilder(column, row_num, struct_builder, nested_field); - auto value = convertToDynamicValue(column, data_type, row_num, nested_name, field_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(nested_field, *value); - } - return std::nullopt; - } - case capnp::DynamicValue::Type::LIST: - { - auto list_builder = builder.as(); - const auto * array_column = assert_cast(column.get()); - const auto & nested_column = array_column->getDataPtr(); - const auto & nested_type = assert_cast(data_type.get())->getNestedType(); - const auto & offsets = array_column->getOffsets(); - auto offset = offsets[row_num - 1]; - size_t size = offsets[row_num] - offset; - - const auto * nested_array_column = checkAndGetColumn(*nested_column); - for (unsigned i = 0; i != static_cast(size); ++i) - { - capnp::DynamicValue::Builder value_builder; - /// For nested arrays we need to initialize nested list builder. - if (nested_array_column) - { - const auto & nested_offset = nested_array_column->getOffsets(); - size_t nested_array_size = nested_offset[offset + i] - nested_offset[offset + i - 1]; - value_builder = list_builder.init(i, static_cast(nested_array_size)); - } - else - value_builder = list_builder[i]; - - auto value = convertToDynamicValue(nested_column, nested_type, offset + i, column_name, value_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - list_builder.set(i, *value); - } - return std::nullopt; - } - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CapnProto type."); - } + const auto & header = getPort(PortKind::Main).getHeader(); + serializer = std::make_unique(header.getDataTypes(), header.getNames(), schema, format_settings.capn_proto); + capnp::MallocMessageBuilder message; } void CapnProtoRowOutputFormat::write(const Columns & columns, size_t row_num) { capnp::MallocMessageBuilder message; - /// Temporary storage for data that will be outputted in fields with CapnProto type TEXT. - /// See comment in convertToDynamicValue() for more details. - std::vector> temporary_text_data_storage; capnp::DynamicStruct::Builder root = message.initRoot(schema); - - /// Some columns can share same field builder. For example when we have - /// column with Nested type that was flattened into several columns. - std::unordered_map field_builders; - for (size_t i = 0; i != columns.size(); ++i) - { - auto [struct_builder, field] = getStructBuilderAndFieldByColumnName(root, column_names[i]); - if (!field_builders.contains(field.getIndex())) - { - auto field_builder = initStructFieldBuilder(columns[i], row_num, struct_builder, field); - field_builders[field.getIndex()] = field_builder; - } - auto value = convertToDynamicValue(columns[i], column_types[i], row_num, column_names[i], field_builders[field.getIndex()], format_settings.capn_proto.enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(field, *value); - } - + serializer->writeRow(columns, std::move(root), row_num); capnp::writeMessage(*output_stream, message); + } void registerOutputFormatCapnProto(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h index 5cc7099d4c7..dd9dcc6b340 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h @@ -3,15 +3,17 @@ #include "config.h" #if USE_CAPNP -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include namespace DB { + class CapnProtoOutputStream : public kj::OutputStream { public: @@ -43,8 +45,9 @@ private: DataTypes column_types; capnp::StructSchema schema; std::unique_ptr output_stream; - const FormatSettings format_settings; CapnProtoSchemaParser schema_parser; + std::unique_ptr serializer; + }; } diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp index 9777f2361a2..6098923a195 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp @@ -88,7 +88,14 @@ void registerInputFormatProtobufList(FormatFactory & factory) }); factory.markFormatSupportsSubsetOfColumns("ProtobufList"); factory.registerAdditionalInfoForSchemaCacheGetter( - "ProtobufList", [](const FormatSettings & settings) { return fmt::format("format_schema={}", settings.schema.format_schema); }); + "ProtobufList", + [](const FormatSettings & settings) + { + return fmt::format( + "format_schema={}, skip_fields_with_unsupported_types_in_schema_inference={}", + settings.schema.format_schema, + settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference); + }); } void registerProtobufListSchemaReader(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index ee60501dba5..126f3673571 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -128,7 +128,14 @@ void registerProtobufSchemaReader(FormatFactory & factory) for (const auto & name : {"Protobuf", "ProtobufSingle"}) factory.registerAdditionalInfoForSchemaCacheGetter( - name, [](const FormatSettings & settings) { return fmt::format("format_schema={}", settings.schema.format_schema); }); + name, + [](const FormatSettings & settings) + { + return fmt::format( + "format_schema={}, skip_fields_with_unsupported_types_in_schema_inference={}", + settings.schema.format_schema, + settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference); + }); } } diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index c15d6fe442e..625104fb590 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -96,8 +96,8 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a_b U $CLICKHOUSE_CLIENT --query="SELECT number AS a_b, number + 1 AS a_c_d, number + 2 AS a_c_e_f FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_tuples:Message'" > $CAPN_PROTO_FILE $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(f UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" -$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(bb UInt64, c Tuple(d UInt64, e Tuple(f UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(ff UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(bb UInt64, c Tuple(d UInt64, e Tuple(f UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "THERE_IS_NO_COLUMN" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(ff UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "THERE_IS_NO_COLUMN" && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'string String') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'; diff --git a/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.reference b/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.reference new file mode 100644 index 00000000000..f34c857e2f6 --- /dev/null +++ b/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.reference @@ -0,0 +1 @@ +42 (42,42) diff --git a/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.sh b/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.sh new file mode 100755 index 00000000000..c3835948437 --- /dev/null +++ b/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-replicated-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SCHEMADIR=$CURDIR/format_schemas +$CLICKHOUSE_LOCAL -q "select 42 as Field1, (42, 42)::Tuple(Field1 UInt32, Field2 UInt32) as Nested format CapnProto settings format_schema='$SCHEMADIR/02735_case_insensitive_names_matching:Message'" | $CLICKHOUSE_LOCAL --input-format CapnProto --structure "Field1 UInt32, Nested Tuple(Field1 UInt32, Field2 UInt32)" -q "select * from table" --format_schema="$SCHEMADIR/02735_case_insensitive_names_matching:Message" + diff --git a/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.reference b/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.reference new file mode 100644 index 00000000000..b6e6d485929 --- /dev/null +++ b/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.reference @@ -0,0 +1,3 @@ +(42,(42,42),[(42,42),(24,24)]) [(42,(42,42),[(42,42),(24,24)]),(24,(24,24),[(24,24),(42,42)])] +42 42 42 +[42,24] [42,24] [42,24] [[42,24],[24,42]] [[42,24],[24,42]] diff --git a/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.sh b/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.sh new file mode 100755 index 00000000000..c669be2ed33 --- /dev/null +++ b/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-replicated-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SCHEMADIR=$CURDIR/format_schemas +DATA_FILE=02736_$CLICKHOUSE_TEST_UNIQUE_NAME.bin + +$CLICKHOUSE_LOCAL -q "select tuple(42, tuple(42, 42), [tuple(42, 42), tuple(24, 24)]) as nested, [tuple(42, tuple(42, 42), [tuple(42, 42), tuple(24, 24)]), tuple(24, tuple(24, 24), [tuple(24, 24), tuple(42, 42)])] as nestedList format CapnProto settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" > $DATA_FILE + +$CLICKHOUSE_LOCAL -q "select * from file($DATA_FILE, CapnProto) settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" + +$CLICKHOUSE_LOCAL -q "select 42 as nested_field1, 42 as nested_nested_field1, 42 as nested_nested_field2 format CapnProto settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" > $DATA_FILE + +$CLICKHOUSE_LOCAL -q "select * from file($DATA_FILE, CapnProto, 'nested_field1 UInt32, nested_nested_field1 UInt32, nested_nested_field2 UInt32') settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" + +$CLICKHOUSE_LOCAL -q "select [42, 24] as nestedList_field1, [42, 24] as nestedList_nested_field1, [42, 24] as nestedList_nested_field2, [[42, 24], [24, 42]] as nestedList_nestedList_field1, [[42, 24], [24, 42]] as nestedList_nestedList_field2 format CapnProto settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" > $DATA_FILE + +$CLICKHOUSE_LOCAL -q "select * from file($DATA_FILE, CapnProto, 'nestedList_field1 Array(UInt32), nestedList_nested_field1 Array(UInt32), nestedList_nested_field2 Array(UInt32), nestedList_nestedList_field1 Array(Array(UInt32)), nestedList_nestedList_field2 Array(Array(UInt32))') settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" + +rm $DATA_FILE + diff --git a/tests/queries/0_stateless/format_schemas/02735_case_insensitive_names_matching.capnp b/tests/queries/0_stateless/format_schemas/02735_case_insensitive_names_matching.capnp new file mode 100644 index 00000000000..6b12aab081a --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02735_case_insensitive_names_matching.capnp @@ -0,0 +1,13 @@ +@0x9ef128e10a8010b8; + +struct Nested +{ + field1 @0 : UInt32; + field2 @1 : UInt32; +} + +struct Message +{ + field1 @0 : UInt32; + nested @1 : Nested; +} diff --git a/tests/queries/0_stateless/format_schemas/02736_nested_structures.capnp b/tests/queries/0_stateless/format_schemas/02736_nested_structures.capnp new file mode 100644 index 00000000000..a03eb27f383 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02736_nested_structures.capnp @@ -0,0 +1,21 @@ +@0x9ef128e10a8010b8; + +struct Nested2 +{ + field1 @0 : UInt32; + field2 @1 : UInt32; +} + +struct Nested +{ + field1 @0 : UInt32; + nested @1 : Nested2; + nestedList @2 : List(Nested2); +} + +struct Message +{ + nested @0 : Nested; + nestedList @1 : List(Nested); +} + From 167516b6b088dea2e6b44a6e81caf2c088f2481e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 May 2023 21:07:56 +0200 Subject: [PATCH 0232/2223] Fix style --- src/Formats/CapnProtoSerializer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index e0c8ae2a79a..c31623286d0 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -26,7 +26,7 @@ namespace DB namespace ErrorCodes { extern const int THERE_IS_NO_COLUMN; - extern const int BAD_TYPE_OF_FIELD; + extern const int LOGICAL_ERROR; extern const int CAPN_PROTO_BAD_CAST; extern const int INCORRECT_DATA; extern const int ILLEGAL_COLUMN; @@ -293,7 +293,7 @@ namespace return capnp::DynamicValue::Reader(capnp::DynamicEnum(enumerant)); } - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert CLickHouse Enum value to CapnProto Enum"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert ClickHouse Enum value to CapnProto Enum"); } void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override From 344885dd27fbc652a5e93040ead764cf88232bbf Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 May 2023 21:08:12 +0200 Subject: [PATCH 0233/2223] Fix style --- src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index e686ae86997..c056ee2b4a4 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int INCORRECT_DATA; } From e3cb1e40e4830d9b5499c2c410e1d12add4cc90f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 May 2023 21:08:31 +0200 Subject: [PATCH 0234/2223] Fix style --- src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index 7dd18be27f4..66a7160dd89 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -11,12 +11,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - CapnProtoOutputStream::CapnProtoOutputStream(WriteBuffer & out_) : out(out_) { } From 72f4aed64b8e8367ef4ab5ffae8ef67c58a09f15 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 10 May 2023 21:08:29 +0000 Subject: [PATCH 0235/2223] Add setting use_with_fill_by_sorting_prefix --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 8 ++++++- src/Planner/Planner.cpp | 8 ++++++- src/Processors/QueryPlan/FillingStep.cpp | 7 +++++-- src/Processors/QueryPlan/FillingStep.h | 4 +++- .../Transforms/FillingTransform.cpp | 21 ++++++++++++------- src/Processors/Transforms/FillingTransform.h | 4 +++- 7 files changed, 39 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1bea2c26392..0d7ffbadf8c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -716,6 +716,7 @@ class IColumn; \ M(Bool, parallelize_output_from_storages, true, "Parallelize output for reading step from storage. It allows parallelizing query processing right after reading from storage if possible", 0) \ \ + M(Bool, use_with_fill_by_sorting_prefix, false, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with the same values in sorting prefix will be filled separately", 0) \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e36a1287503..f6da77d66ed 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3008,8 +3008,14 @@ void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan) InterpolateDescriptionPtr interpolate_descr = getInterpolateDescription(query, source_header, result_header, syntax_analyzer_result->aliases, context); + + const Settings & settings = context->getSettingsRef(); auto filling_step = std::make_unique( - query_plan.getCurrentDataStream(), std::move(sort_description), std::move(fill_description), interpolate_descr); + query_plan.getCurrentDataStream(), + std::move(sort_description), + std::move(fill_description), + interpolate_descr, + settings.use_with_fill_by_sorting_prefix); query_plan.addStep(std::move(filling_step)); } } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e19a92c273c..ba338de4072 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -622,8 +622,14 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, interpolate_description = std::make_shared(std::move(interpolate_actions_dag), empty_aliases); } + const auto & query_context = planner_context->getQueryContext(); + const Settings & settings = query_context->getSettingsRef(); auto filling_step = std::make_unique( - query_plan.getCurrentDataStream(), sort_description, std::move(fill_description), interpolate_description); + query_plan.getCurrentDataStream(), + sort_description, + std::move(fill_description), + interpolate_description, + settings.use_with_fill_by_sorting_prefix); query_plan.addStep(std::move(filling_step)); } diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 37c5307fcce..65c9cf11661 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -31,11 +31,13 @@ FillingStep::FillingStep( const DataStream & input_stream_, SortDescription sort_description_, SortDescription fill_description_, - InterpolateDescriptionPtr interpolate_description_) + InterpolateDescriptionPtr interpolate_description_, + bool use_with_fill_by_sorting_prefix_) : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_), getTraits()) , sort_description(std::move(sort_description_)) , fill_description(std::move(fill_description_)) , interpolate_description(interpolate_description_) + , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (!input_stream_.has_single_port) throw Exception(ErrorCodes::LOGICAL_ERROR, "FillingStep expects single input"); @@ -48,7 +50,8 @@ void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build if (stream_type == QueryPipelineBuilder::StreamType::Totals) return std::make_shared(header, fill_description); - return std::make_shared(header, sort_description, fill_description, std::move(interpolate_description)); + return std::make_shared( + header, sort_description, fill_description, std::move(interpolate_description), use_with_fill_by_sorting_prefix); }); } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index 0febbbb8eb0..3e0e1e79060 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -14,7 +14,8 @@ public: const DataStream & input_stream_, SortDescription sort_description_, SortDescription fill_description_, - InterpolateDescriptionPtr interpolate_description_); + InterpolateDescriptionPtr interpolate_description_, + bool use_with_fill_by_sorting_prefix); String getName() const override { return "Filling"; } @@ -31,6 +32,7 @@ private: SortDescription sort_description; SortDescription fill_description; InterpolateDescriptionPtr interpolate_description; + const bool use_with_fill_by_sorting_prefix; }; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 45967b646a6..5b3200ac89e 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -187,13 +187,18 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & } FillingTransform::FillingTransform( - const Block & header_, const SortDescription & sort_description_, const SortDescription& fill_description_, InterpolateDescriptionPtr interpolate_description_) - : ISimpleTransform(header_, transformHeader(header_, fill_description_), true) - , sort_description(sort_description_) - , fill_description(fill_description_) - , interpolate_description(interpolate_description_) - , filling_row(fill_description_) - , next_row(fill_description_) + const Block & header_, + const SortDescription & sort_description_, + const SortDescription & fill_description_, + InterpolateDescriptionPtr interpolate_description_, + const bool use_with_fill_by_sorting_prefix_) + : ISimpleTransform(header_, transformHeader(header_, fill_description_), true) + , sort_description(sort_description_) + , fill_description(fill_description_) + , interpolate_description(interpolate_description_) + , filling_row(fill_description_) + , next_row(fill_description_) + , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) { if (interpolate_description) interpolate_actions = std::make_shared(interpolate_description->actions); @@ -673,7 +678,7 @@ void FillingTransform::transform(Chunk & chunk) res_sort_prefix_columns, res_other_columns); - if (sort_prefix.empty()) + if (sort_prefix.empty() || !use_with_fill_by_sorting_prefix) { transformRange( input_fill_columns, diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index def4e87c7ed..2a4da05ae29 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -20,7 +20,8 @@ public: const Block & header_, const SortDescription & sort_description_, const SortDescription & fill_description_, - InterpolateDescriptionPtr interpolate_description_); + InterpolateDescriptionPtr interpolate_description_, + bool use_with_fill_by_sorting_prefix_); String getName() const override { return "FillingTransform"; } @@ -87,6 +88,7 @@ private: Columns last_row; Columns last_range_sort_prefix; bool all_chunks_processed = false; /// flag to determine if we have already processed all chunks + const bool use_with_fill_by_sorting_prefix; }; class FillingNoopTransform : public ISimpleTransform From 07630ef43fd40f46dfba9adca487c3b69ca2ad3c Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 11 May 2023 01:10:34 +0200 Subject: [PATCH 0236/2223] upd --- src/Client/ClientBase.cpp | 4 +--- src/Client/Connection.cpp | 8 ++++---- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8b5db85fc02..fad9494ba4b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1082,9 +1082,7 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { - Settings settings; - settings.session_timezone = tz; - global_context->applySettingsChanges(settings.changes()); + global_context->setSetting("session_timezone", tz); } diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 062f05105aa..86585d805d9 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -355,10 +355,10 @@ void Connection::receiveHello() nonce.emplace(read_nonce); } } - else if (packet_type == Protocol::Server::TimezoneUpdate) - { - // skip this packet at hello, will receive and process it later - } +// else if (packet_type == Protocol::Server::TimezoneUpdate) +// { +// // skip this packet at hello, will receive and process it later +// } else if (packet_type == Protocol::Server::Exception) receiveException()->rethrow(); else From 58bdcc29315a712e1255c13d31669f4545de9edb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 10 May 2023 23:55:13 +0000 Subject: [PATCH 0237/2223] allow to cast IPv6 to IPv4 for address in proper mapping block --- src/Functions/FunctionsConversion.h | 30 ++++++++++++++++++++++++++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9c4085f9745..9cdd09780e3 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -57,6 +57,7 @@ #include #include #include +#include #include @@ -217,13 +218,13 @@ struct ConvertImpl } else if constexpr ( (std::is_same_v != std::is_same_v) - && !(is_any_of || is_any_of) + && !(is_any_of || is_any_of) ) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion from {} to {} is not supported", TypeName, TypeName); } - else if constexpr (std::is_same_v != std::is_same_v) + else if constexpr (std::is_same_v != std::is_same_v && !std::is_same_v) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion between numeric types and IPv6 is not supported. " @@ -304,7 +305,30 @@ struct ConvertImpl } else { - if constexpr (std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v && std::is_same_v) + { + const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; + const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); + if (!matchIPv6Subnet(src, ip4_cidr, 96)) + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", named_from.column->getName()); + + uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); + if constexpr (std::endian::native == std::endian::little) + { + dst[0] = src[15]; + dst[1] = src[14]; + dst[2] = src[13]; + dst[3] = src[12]; + } + else + { + dst[3] = src[15]; + dst[2] = src[14]; + dst[1] = src[13]; + dst[0] = src[12]; + } + } + else if constexpr (std::is_same_v && std::is_same_v) vec_to[i] = static_cast(static_cast(vec_from[i])); else vec_to[i] = static_cast(vec_from[i]); From 604bd24995b411cbf405e79a48b555beda604b6e Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 May 2023 11:58:08 +0000 Subject: [PATCH 0238/2223] Refactor, remove no more needed arguments --- .../functions/other-functions.md | 40 +- src/Functions/generateRandomStructure.cpp | 480 +++++++----------- .../02586_generate_random_structure.reference | 11 +- .../02586_generate_random_structure.sql | 26 +- 4 files changed, 189 insertions(+), 368 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 43330b75b8f..e235a3db393 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2617,18 +2617,13 @@ Generates random table structure in a format `column1_name column1_type, column2 **Syntax** ``` sql -generateRandomStructure([number_of_columns, seed, allow_big_numbers, allow_enums, allow_decimals, allow_ip, allow_only_string_map_keys]) +generateRandomStructure([number_of_columns, seed]) ``` **Arguments** - `number_of_columns` — The desired number of columns in the result table structure. If set to 0 or `Null`, the number of columns will be random from 1 to 128. Default value: `Null`. - `seed` - Random seed to produce stable results. If seed is not specified or set to `Null`, it is randomly generated. -- `allow_big_numbers` - Indicates if big number types (`Int128/UInt128/Int256/UInt256/Decimal128/Decinal256`) can be generated. Default value: true. -- `allow_enums` - Indicates if enum types (`Enum8/Enum16`) can be generated. Default - true. -- `allow_decimals` - Indicates if decimal types (`Decimal(P, S)`) can be generated. Default - true. -- `allow_ip` - Indicates if ip types (`IPv4/IPv6`) can be generated. Default - true. -- `allow_only_string_map_keys` - Indicates if Map key type can be only `String/FixedString`. Default - false. All arguments must be constant. @@ -2671,41 +2666,16 @@ Result: Query: ``` sql -SELECT generateRandomStructure(Null, 11) +SELECT generateRandomStructure(NULL, 33) ``` Result: ``` text -┌─generateRandomStructure(0, 11)──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ c1 Date32, c2 String, c3 IPv6, c4 DateTime, c5 UInt16, c6 Tuple(e1 UInt32, e2 Date, e3 Date, e4 IPv6, e5 Nested(e1 DateTime, e2 FixedString(110), e3 Int256, e4 Array(Decimal64(4)), e5 Decimal128(18), e6 Enum16('v0' = 0, 'v1' = 1, 'v2' = 2, 'v3' = 3, 'v4' = 4)), e6 DateTime64(4)), c7 DateTime, c8 DateTime64(6), c9 Bool │ -└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +┌─generateRandomStructure(NULL, 33)─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ c1 DateTime, c2 Enum8('c2V0' = 0, 'c2V1' = 1, 'c2V2' = 2, 'c2V3' = 3), c3 LowCardinality(Nullable(FixedString(30))), c4 Int16, c5 Enum8('c5V0' = 0, 'c5V1' = 1, 'c5V2' = 2, 'c5V3' = 3), c6 Nullable(UInt8), c7 String, c8 Nested(e1 IPv4, e2 UInt8, e3 UInt16, e4 UInt16, e5 Int32, e6 Map(Date, Decimal256(70))) │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` -``` sql -SELECT generateRandomStructure(6, Null, false, false) -``` - -Result: - -``` text -┌─generateRandomStructure(6, NULL, false, false)───────────────────────────────────────────────────────┐ -│ c1 Float32, c2 Tuple(DateTime), c3 UInt8, c4 UInt16, c5 Int64, c6 Array(Map(FixedString(108), Date)) │ -└──────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -``` sql -SELECT generateRandomStructure(6, Null, false, false, false, false, true) -``` - -Result: - -``` text -┌─generateRandomStructure(6, NULL, false, false, false, false, true)─────────────────────────────────────────────────┐ -│ c1 String, c2 UInt32, c3 Int32, c4 Int32, c5 Tuple(LowCardinality(Nullable(FixedString(101))), UInt8), c6 DateTime │ -└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - - This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index e6766e731b2..9fe321365e5 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -1,5 +1,3 @@ -#include "config.h" - #include #include #include @@ -24,130 +22,69 @@ namespace ErrorCodes class FunctionGenerateRandomStructure : public IFunction { private: - enum class Type + static constexpr std::array simple_types { - Int8, - UInt8, - Bool, - Int16, - UInt16, - Int32, - UInt32, - Int64, - UInt64, - Float32, - Float64, - DateTime64, - Decimal32, - Decimal64, - Date, - Date32, - DateTime, - String, - FixedString, - IPv4, - IPv6, - Int128, - UInt128, - Int256, - UInt256, - Decimal128, - Decimal256, - Enum8, - Enum16, - Nullable, - LowCardinality, - Array, - Tuple, - Map, - Nested, + TypeIndex::Int8, + TypeIndex::UInt8, + TypeIndex::Int16, + TypeIndex::UInt16, + TypeIndex::Int32, + TypeIndex::UInt32, + TypeIndex::Int64, + TypeIndex::UInt64, + TypeIndex::Int128, + TypeIndex::UInt128, + TypeIndex::Int256, + TypeIndex::UInt256, + TypeIndex::Float32, + TypeIndex::Float64, + TypeIndex::Decimal32, + TypeIndex::Decimal64, + TypeIndex::Decimal128, + TypeIndex::Decimal256, + TypeIndex::Date, + TypeIndex::Date32, + TypeIndex::DateTime, + TypeIndex::DateTime64, + TypeIndex::String, + TypeIndex::FixedString, + TypeIndex::Enum8, + TypeIndex::Enum16, + TypeIndex::IPv4, + TypeIndex::IPv6, + TypeIndex::UUID, }; - static constexpr std::array simple_types + static constexpr std::array complex_types { - Type::Int8, - Type::UInt8, - Type::Bool, - Type::Int16, - Type::UInt16, - Type::Int32, - Type::UInt32, - Type::Int64, - Type::UInt64, - Type::Float32, - Type::Float64, - Type::Date, - Type::Date32, - Type::DateTime, - Type::String, - Type::FixedString, + TypeIndex::Nullable, + TypeIndex::LowCardinality, + TypeIndex::Array, + TypeIndex::Tuple, + TypeIndex::Map, }; - static constexpr std::array big_integer_types + static constexpr std::array map_key_types { - Type::Int128, - Type::UInt128, - Type::Int256, - Type::UInt256, - }; - - static constexpr std::array decimal_types - { - Type::DateTime64, - Type::Decimal32, - Type::Decimal64, - }; - - static constexpr std::array big_decimal_types - { - Type::Decimal128, - Type::Decimal256, - }; - - static constexpr std::array enum_types - { - Type::Enum8, - Type::Enum16, - }; - - static constexpr std::array ip_types - { - Type::IPv4, - Type::IPv6, - }; - - static constexpr std::array complex_types - { - Type::Nullable, - Type::LowCardinality, - Type::Array, - Type::Tuple, - Type::Map, - Type::Nested, - }; - - static constexpr std::array map_key_types - { - Type::Int8, - Type::UInt8, - Type::Bool, - Type::Int16, - Type::UInt16, - Type::Int32, - Type::UInt32, - Type::Int64, - Type::UInt64, - Type::Date, - Type::Date32, - Type::DateTime, - Type::String, - Type::FixedString, - }; - - static constexpr std::array map_key_string_types - { - Type::String, - Type::FixedString + TypeIndex::Int8, + TypeIndex::UInt8, + TypeIndex::Int16, + TypeIndex::UInt16, + TypeIndex::Int32, + TypeIndex::UInt32, + TypeIndex::Int64, + TypeIndex::UInt64, + TypeIndex::Int128, + TypeIndex::UInt128, + TypeIndex::Int256, + TypeIndex::UInt256, + TypeIndex::Date, + TypeIndex::Date32, + TypeIndex::DateTime, + TypeIndex::String, + TypeIndex::FixedString, + TypeIndex::IPv4, + TypeIndex::UUID, }; static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; @@ -175,18 +112,19 @@ public: bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1, 2, 3, 4, 5, 6}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } bool useDefaultImplementationForConstants() const override { return false; } bool useDefaultImplementationForNulls() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() > 7) + if (arguments.size() > 2) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 7", + "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 2", getName(), arguments.size()); + for (size_t i = 0; i != 2; ++i) { if (arguments.size() == i) @@ -203,22 +141,6 @@ public: } } - for (size_t i = 2; i != 7; ++i) - { - if (arguments.size() <= i) - break; - - if (!isUInt8(arguments[i])) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the {} argument of function {}, expected UInt8", - i + 1, - arguments[i]->getName(), - getName()); - } - } - return std::make_shared(); } @@ -241,40 +163,27 @@ public: if (arguments.size() > 1 && !arguments[1].column->onlyNull()) seed = arguments[1].column->getUInt(0); - bool allow_big_numbers = true; - if (arguments.size() > 2) - allow_big_numbers = arguments[2].column->getBool(0); - - bool allow_enums = true; - if (arguments.size() > 3) - allow_enums = arguments[3].column->getBool(0); - - bool allow_decimals = true; - if (arguments.size() > 4) - allow_decimals = arguments[4].column->getBool(0); - - bool allow_ip = true; - if (arguments.size() > 5) - allow_ip = arguments[5].column->getBool(0); - - bool only_string_map_key = false; - if (arguments.size() > 6) - only_string_map_key = arguments[6].column->getBool(0); - pcg64 rng(seed); if (number_of_columns == 0) number_of_columns = generateNumberOfColumns(rng); auto col_res = ColumnString::create(); - String generated_structure; + auto & string_column = assert_cast(*col_res); + auto & chars = string_column.getChars(); + WriteBufferFromVector buf(chars); for (size_t i = 0; i != number_of_columns; ++i) { if (i != 0) - generated_structure += ", "; - auto type = generateRandomType(rng, allow_big_numbers, allow_enums, allow_decimals, allow_ip, only_string_map_key); - generated_structure += "c" + std::to_string(i + 1) + " " + type; + writeCString(", ", buf); + String column_name = "c" + std::to_string(i + 1); + writeString(column_name, buf); + writeChar(' ', buf); + writeRandomType(column_name, rng, buf); } - col_res->insert(generated_structure); + + buf.finalize(); + chars.push_back(0); + string_column.getOffsets().push_back(chars.size()); return ColumnConst::create(std::move(col_res), input_rows_count); } @@ -285,181 +194,159 @@ private: return rng() % MAX_NUMBER_OF_COLUMNS + 1; } - /// Helper struct to call generateRandomTypeImpl with lots of bool template arguments without writing big if/else over all bool variables. - template - struct Dispatcher + template + void writeRandomType(const String & column_name, pcg64 & rng, WriteBuffer & buf, size_t depth = 0) const { - static auto call(const FunctionGenerateRandomStructure * f, pcg64 & rng) - { - return f->generateRandomTypeImpl(rng); - } - - template - static auto call(const FunctionGenerateRandomStructure * f, pcg64 & rng, bool b, Args1... ar1) - { - if (b) - return Dispatcher::call(f, rng, ar1...); - else - return Dispatcher::call(f, rng, ar1...); - } - - friend FunctionGenerateRandomStructure; - }; - - String generateRandomType(pcg64 & rng, bool allow_big_numbers, bool allow_enums, bool allow_decimals, bool allow_ip, bool allow_only_string_map_keys) const - { - return Dispatcher<>::call(this, rng, allow_big_numbers, allow_enums, allow_decimals, allow_ip, allow_only_string_map_keys, true); - } - - template - String generateRandomTypeImpl(pcg64 & rng, size_t depth = 0) const - { - constexpr auto all_types = getAllTypes(); + constexpr auto all_types = getAllTypes(); auto type = all_types[rng() % all_types.size()]; switch (type) { - case Type::FixedString: - return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - case Type::DateTime64: - return "DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")"; - case Type::Decimal32: - return "Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")"; - case Type::Decimal64: - return "Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")"; - case Type::Decimal128: - return "Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")"; - case Type::Decimal256: - return "Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")"; - case Type::Enum8: - return "Enum8(" + generateEnumValues(rng) + ")"; - case Type::Enum16: - return "Enum16(" + generateEnumValues(rng) + ")"; - case Type::LowCardinality: - return "LowCardinality(" + generateLowCardinalityNestedType(rng) + ")"; - case Type::Nullable: + case TypeIndex::UInt8: + if (rng() % 2) + writeCString("UInt8", buf); + else + writeCString("Bool", buf); + return; + case TypeIndex::FixedString: + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); + return; + case TypeIndex::DateTime64: + writeString("DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")", buf); + return; + case TypeIndex::Decimal32: + writeString("Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")", buf); + return; + case TypeIndex::Decimal64: + writeString("Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")", buf); + return; + case TypeIndex::Decimal128: + writeString("Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")", buf); + return; + case TypeIndex::Decimal256: + writeString("Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")", buf); + return; + case TypeIndex::Enum8: + writeCString("Enum8(", buf); + writeEnumValues(column_name, rng, buf); + writeChar(')', buf); + return; + case TypeIndex::Enum16: + writeCString("Enum16(", buf); + writeEnumValues(column_name, rng, buf); + writeChar(')', buf); + return; + case TypeIndex::LowCardinality: + writeCString("LowCardinality(", buf); + writeLowCardinalityNestedType(rng, buf); + writeChar(')', buf); + return; + case TypeIndex::Nullable: { - auto nested_type = generateRandomTypeImpl(rng, depth + 1); - return "Nullable(" + nested_type + ")"; + writeCString("Nullable(", buf); + writeRandomType(column_name, rng, buf, depth + 1); + writeChar(')', buf); + return; } - case Type::Array: + case TypeIndex::Array: { - auto nested_type = generateRandomTypeImpl(rng, depth + 1); - return "Array(" + nested_type + ")"; + writeCString("Array(", buf); + writeRandomType(column_name, rng, buf, depth + 1); + writeChar(')', buf); + return; } - case Type::Map: + case TypeIndex::Map: { - auto key_type = generateMapKeyType(rng); - auto value_type = generateRandomTypeImpl(rng, depth + 1); - return "Map(" + key_type + ", " + value_type + ")"; + writeCString("Map(", buf); + writeMapKeyType(rng, buf); + writeCString(", ", buf); + writeRandomType(column_name, rng, buf, depth + 1); + writeChar(')', buf); + return; } - case Type::Tuple: + case TypeIndex::Tuple: { size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; - bool named_tuple = rng() % 2; - String tuple_type = "Tuple("; + bool generate_nested = rng() % 2; + bool generate_named_tuple = rng() % 2; + if (generate_nested) + writeCString("Nested(", buf); + else + writeCString("Tuple(", buf); + for (size_t i = 0; i != elements; ++i) { if (i != 0) - tuple_type += ", "; - if (named_tuple) - tuple_type += "e" + std::to_string(i + 1) + " "; - tuple_type += generateRandomTypeImpl(rng, depth + 1); + writeCString(", ", buf); + + String element_name = "e" + std::to_string(i + 1); + if (generate_named_tuple || generate_nested) + { + writeString(element_name, buf); + writeChar(' ', buf); + } + writeRandomType(element_name, rng, buf, depth + 1); } - return tuple_type + ")"; - } - case Type::Nested: - { - size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; - String nested_type = "Nested("; - for (size_t i = 0; i != elements; ++i) - { - if (i != 0) - nested_type += ", "; - auto element_type = generateRandomTypeImpl(rng, depth + 1); - nested_type += "e" + std::to_string(i + 1) + " " + element_type; - } - return nested_type + ")"; + writeChar(')', buf); + return; } default: - return String(magic_enum::enum_name(type)); + writeString(magic_enum::enum_name(type), buf); + return; } } - template - String generateMapKeyType(pcg64 & rng) const + void writeMapKeyType(pcg64 & rng, WriteBuffer & buf) const { - Type type; - if constexpr (allow_only_string_map_keys) - type = map_key_string_types[rng() % map_key_string_types.size()]; + TypeIndex type = map_key_types[rng() % map_key_types.size()]; + if (type == TypeIndex::FixedString) + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); else - type = map_key_types[rng() % map_key_types.size()]; - - if (type == Type::FixedString) - return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - return String(magic_enum::enum_name(type)); + writeString(magic_enum::enum_name(type), buf); } - String generateLowCardinalityNestedType(pcg64 & rng) const + void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf) const { - /// Support only String and FixedString. + /// Support only String and FixedString (maybe Nullable). String nested_type; + bool make_nullable = rng() % 2; + if (make_nullable) + writeCString("Nullable(", buf); + if (rng() % 2) - nested_type = "String"; + writeCString("String", buf); else - nested_type = "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - return rng() % 2 ? nested_type : "Nullable(" + nested_type + ")"; + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); + + if (make_nullable) + writeChar(')', buf); } - String generateEnumValues(pcg64 & rng) const + void writeEnumValues(const String & column_name, pcg64 & rng, WriteBuffer & buf) const { - /// Don't generate big enums, because it will lead to really big strings + /// Don't generate big enums, because it will lead to really big result /// and slowness of this function, and it can lead to `Max query size exceeded` /// while using this function with generateRandom. ssize_t num_values = rng() % 16 + 1; - String result; for (ssize_t i = 0; i != num_values; ++i) { if (i != 0) - result += ", "; - result += "'v" + std::to_string(i) + "' = " + std::to_string(i); + writeCString(", ", buf); + writeString("'" + column_name + "V" + std::to_string(i) + "' = " + std::to_string(i), buf); } - return result; } - template + template static constexpr auto getAllTypes() { - constexpr size_t big_integer_types_size = big_integer_types.size() * allow_big_numbers; - constexpr size_t enum_types_size = enum_types.size() * allow_enums; - constexpr size_t decimal_types_size = decimal_types.size() * allow_decimals; - constexpr size_t big_decimal_types_size = big_decimal_types.size() * allow_big_numbers * allow_decimals; - constexpr size_t ip_types_size = ip_types.size() * allow_ip; constexpr size_t complex_types_size = complex_types.size() * allow_complex_types; - - constexpr size_t result_size = simple_types.size() + big_integer_types_size + enum_types_size + decimal_types_size - + big_decimal_types_size + ip_types_size + complex_types_size; - std::array result; + constexpr size_t result_size = simple_types.size() + complex_types_size; + std::array result; size_t index = 0; for (size_t i = 0; i != simple_types.size(); ++i, ++index) result[index] = simple_types[i]; - for (size_t i = 0; i != big_integer_types_size; ++i, ++index) - result[index] = big_integer_types[i]; - - for (size_t i = 0; i != enum_types_size; ++i, ++index) - result[index] = enum_types[i]; - - for (size_t i = 0; i != decimal_types_size; ++i, ++index) - result[index] = decimal_types[i]; - - for (size_t i = 0; i != big_decimal_types_size; ++i, ++index) - result[index] = big_decimal_types[i]; - - for (size_t i = 0; i != ip_types_size; ++i, ++index) - result[index] = ip_types[i]; - for (size_t i = 0; i != complex_types_size; ++i, ++index) result[index] = complex_types[i]; @@ -474,26 +361,15 @@ REGISTER_FUNCTION(GenerateRandomStructure) { R"( Generates a random table structure. -This function takes 4 optional constant arguments: -1) the number of column in the result structure (random by default) -2) random seed (random by default) -3) flag that indicates if big number types can be used (true by default) -4) flag that indicates if enum types can be used (true by default) -5) flag that indicates if decimal types can be used (true by default) -6) flag that indicates if ip types (IPv4, IPv6) can be used (true by default) -7) flag that indicates if map keys should be only String or FixedString (false by default) +This function takes 2 optional constant arguments: +the number of columns in the result structure (random by default) and random seed (random by default) The maximum number of columns is 128. The function returns a value of type String. )", Documentation::Examples{ {"random", "SELECT generateRandomStructure()"}, - {"with specified number of arguments", "SELECT generateRandomStructure(10)"}, + {"with specified number of columns", "SELECT generateRandomStructure(10)"}, {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, - {"without big number types", "SELECT generateRandomStructure(10, NULL, false)"}, - {"without enum types", "SELECT generateRandomStructure(10, NULL, true, false)"}, - {"without decimal types", "SELECT generateRandomStructure(10, NULL, true, true, false)"}, - {"without ip types", "SELECT generateRandomStructure(10, NULL, true, true, true, false)"}, - {"with only string mak key types", "SELECT generateRandomStructure(10, NULL, true, true, true, true, true)"}, }, Documentation::Categories{"Random"} }, diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index 76d89828071..65bdc530f10 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -1,11 +1,4 @@ -c1 Date, c2 Bool, c3 Int16, c4 Map(Int64, Array(Bool)), c5 Decimal256(30) -c1 String, c2 Float64, c3 Enum8(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4), c4 UInt64, c5 Date -c1 Nested(e1 Int64, e2 Int16, e3 Map(Int16, LowCardinality(Nullable(String))), e4 UInt8, e5 Nested(e1 Array(Nullable(DateTime)), e2 Nullable(Bool), e3 UInt8, e4 UInt64, e5 Decimal64(6), e6 DateTime), e6 LowCardinality(Nullable(String))), c2 Date, c3 Int32, c4 IPv4, c5 Decimal32(8) -c1 Date, c2 UInt16, c3 UInt256, c4 Nullable(IPv4), c5 Nullable(Decimal64(17)) -c1 Array(Int64), c2 Map(String, LowCardinality(String)), c3 Date, c4 Map(Int64, UInt128), c5 UInt8 -c1 Date, c2 UInt16, c3 UInt256, c4 Nullable(Decimal128(37)), c5 DateTime64(8) -c1 Date, c2 Bool, c3 Int16, c4 Map(FixedString(120), Bool), c5 Decimal256(30) -c1 String, c2 Float64, c3 Enum8(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4), c4 UInt64, c5 Date +c1 String, c2 UInt256, c3 String, c4 Decimal128(7), c5 UInt128 String Const(String) -1977-07-28 true 5389 +` 90465455320735604871982424534384518837533904778028808627865442405232847164685 5& -3034771008825448884614719061068.2821046 75820566154622566322847299106656624693 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index 061fbc24219..a28c159cff5 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -1,29 +1,11 @@ select generateRandomStructure(5, 42); -select generateRandomStructure(5, 42, false); -select generateRandomStructure(5, 42, false, false); -select generateRandomStructure(5, 42, true, false); -select generateRandomStructure(5, 42, true, true, false); -select generateRandomStructure(5, 42, true, true, true, false); -select generateRandomStructure(5, 42, true, true, true, true, true); -select generateRandomStructure(5, 42, false, true, true); select toTypeName(generateRandomStructure(5, 42)); select toColumnTypeName(generateRandomStructure(5, 42)); -SELECT * FROM generateRandom(generateRandomStructure(3, 42), 42) LIMIT 1; +SELECT * FROM generateRandom(generateRandomStructure(5, 42), 42) LIMIT 1; -select generateRandomStructure(5, 42, false, false, false, false, true, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select generateRandomStructure(5, 42, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} select generateRandomStructure('5'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(5, 42, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(5, 42, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(5, 42, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(5, 42, false, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(5, 42, false, false, false, false, 'false'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -select generateRandomStructure(materialize(5), 42, false, false); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, materialize(42), false, false); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, materialize(false), false); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} -select generateRandomStructure(5, 42, false, false, false, false, materialize(false)); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN} +select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN} From 9d1ee044b080237df41ff40f3376d433e6b274a4 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 May 2023 12:02:00 +0000 Subject: [PATCH 0239/2223] Update example --- docs/en/sql-reference/table-functions/generate.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/table-functions/generate.md b/docs/en/sql-reference/table-functions/generate.md index 7a17c61eeca..b2777418e4f 100644 --- a/docs/en/sql-reference/table-functions/generate.md +++ b/docs/en/sql-reference/table-functions/generate.md @@ -56,15 +56,15 @@ SELECT * FROM random; In combination with [generateRandomStructure](../../sql-reference/functions/other-functions.md#generateRandomStructure): ```sql -SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 3; +SELECT * FROM generateRandom(generateRandomStructure(4, 101), 101) LIMIT 3; ``` ```text -┌─────────────────────────c1─┬─────c2─┬───────────────────c3─┬───────────────────────────────────────c4─┐ -│ 2085-07-05 23:48:43.345759 │ -20656 │ 1632406185424686785 │ -210464718903845545171230673454802.15238 │ -│ 1971-07-17 16:32:36.390777 │ -27071 │ -1553021742787219162 │ 1095158319964381336405161704296125.08074 │ -│ 2024-02-19 13:14:32.902513 │ 24913 │ 7727442383333447640 │ 1090748832613398997057187200834127.07109 │ -└────────────────────────────┴────────┴──────────────────────┴──────────────────────────────────────────┘ +┌──────────────────c1─┬──────────────────c2─┬─c3─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─c4──────────────────────────────────────┐ +│ 1996-04-15 06:40:05 │ 33954608387.2844801 │ ['232.78.216.176','9.244.59.211','211.21.80.152','44.49.94.109','165.77.195.182','68.167.134.239','212.13.24.185','1.197.255.35','192.55.131.232'] │ 45d9:2b52:ab6:1c59:185b:515:c5b6:b781 │ +│ 2063-01-13 01:22:27 │ 36155064970.9514454 │ ['176.140.188.101'] │ c65a:2626:41df:8dee:ec99:f68d:c6dd:6b30 │ +│ 2090-02-28 14:50:56 │ 3864327452.3901373 │ ['155.114.30.32'] │ 57e9:5229:93ab:fbf3:aae7:e0e4:d1eb:86b │ +└─────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────┘ ``` ## Related content From 288988b59912173883daa757fff394fc5f40b497 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 May 2023 12:08:50 +0000 Subject: [PATCH 0240/2223] Fix build --- src/Formats/CapnProtoSerializer.cpp | 6 ++++++ src/Formats/CapnProtoSerializer.h | 4 ++++ 2 files changed, 10 insertions(+) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index c31623286d0..00ccfc7717d 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -1,3 +1,7 @@ +#include "config.h" + +#if USE_CAPNP + #include #include #include @@ -1216,3 +1220,5 @@ void CapnProtoSerializer::readRow(MutableColumns & columns, capnp::DynamicStruct CapnProtoSerializer::~CapnProtoSerializer() = default; } + +#endif diff --git a/src/Formats/CapnProtoSerializer.h b/src/Formats/CapnProtoSerializer.h index efae797875b..692f5e5301f 100644 --- a/src/Formats/CapnProtoSerializer.h +++ b/src/Formats/CapnProtoSerializer.h @@ -1,5 +1,7 @@ #pragma once +#if USE_CAPNP + #include #include @@ -23,3 +25,5 @@ private: }; } + +#endif From 5ed1b12e19939e399dccc1776901c4e4ddede29a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 May 2023 12:12:43 +0000 Subject: [PATCH 0241/2223] Fix build --- src/Functions/generateRandomStructure.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp index 9fe321365e5..eccccdf563f 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/generateRandomStructure.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes class FunctionGenerateRandomStructure : public IFunction { private: - static constexpr std::array simple_types + static constexpr std::array simple_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -64,7 +64,7 @@ private: TypeIndex::Map, }; - static constexpr std::array map_key_types + static constexpr std::array map_key_types { TypeIndex::Int8, TypeIndex::UInt8, From 20007504a7d475274e07e46a0f4c27e577cad43f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 11 May 2023 14:16:48 +0200 Subject: [PATCH 0242/2223] Handle exception in finalize inside WriteBufferFromPocoSocket destructor --- src/IO/WriteBufferFromPocoSocket.cpp | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 039110dfb62..79534b0f030 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -106,7 +106,22 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { - finalize(); +#ifndef NDEBUG + if (!finalized) + { + LOG_ERROR(log, "WriteBufferFromPocoSocket is not finalized in destructor. It's a bug"); + std::terminate(); + } +#else + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +#endif } } From ef1100bb900563232e8263402e126234827fb2b0 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 11 May 2023 14:44:36 +0200 Subject: [PATCH 0243/2223] Added include to fix build issue --- src/Storages/StorageFile.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 095e9323a7c..f6ce93b0688 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,6 +52,7 @@ #include #include #include +#include namespace ProfileEvents From 108e256578574b26f8adeb3916b15238f0557ee9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 11 May 2023 16:17:52 +0000 Subject: [PATCH 0244/2223] allow to cast IPv4 to IPv6 --- src/Functions/FunctionsConversion.h | 32 ++++++++++++++++++++++++----- 1 file changed, 27 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9cdd09780e3..5bf59f33cb5 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -218,13 +218,13 @@ struct ConvertImpl } else if constexpr ( (std::is_same_v != std::is_same_v) - && !(is_any_of || is_any_of) + && !(is_any_of || is_any_of) ) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion from {} to {} is not supported", TypeName, TypeName); } - else if constexpr (std::is_same_v != std::is_same_v && !std::is_same_v) + else if constexpr (std::is_same_v != std::is_same_v && !(std::is_same_v || std::is_same_v)) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion between numeric types and IPv6 is not supported. " @@ -322,10 +322,32 @@ struct ConvertImpl } else { - dst[3] = src[15]; - dst[2] = src[14]; - dst[1] = src[13]; dst[0] = src[12]; + dst[1] = src[13]; + dst[2] = src[14]; + dst[3] = src[15]; + } + } + else if constexpr (std::is_same_v && std::is_same_v) + { + const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); + uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); + std::memset(dst, '\0', IPV6_BINARY_LENGTH); + dst[10] = dst[11] = 0xff; + + if constexpr (std::endian::native == std::endian::little) + { + dst[12] = src[3]; + dst[13] = src[2]; + dst[14] = src[1]; + dst[15] = src[0]; + } + else + { + dst[12] = src[0]; + dst[13] = src[1]; + dst[14] = src[2]; + dst[15] = src[3]; } } else if constexpr (std::is_same_v && std::is_same_v) From 37c1b1aa58a3d67c6643b0630331c65a55a87df4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 11 May 2023 18:49:05 +0000 Subject: [PATCH 0245/2223] Some fixups --- .../sql-reference/functions/hash-functions.md | 2 +- src/Functions/EntropyLearnedHash.cpp | 26 ++++++------------- .../02734_entropy_learned_hashing.sql | 3 --- 3 files changed, 9 insertions(+), 22 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 1e597ed9b7c..52bb4089b10 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -569,7 +569,7 @@ Time Hashing with Controllable Uniformity](https://doi.org/10.1145/3514221.35178 Entropy-learned hashing has two phases: 1. A training phase on a representative but typically small set of Strings to be hashed. Function `trainEntropyLearnedHash(data, id)` - calculates a minimal partial sub-key of `data` and stores it as `id`. + calculates a minimal partial sub-key of `data` and stores it as `id`. The training step outputs dummy `0` values. 2. An evaluation phase where hashes are computed using the previously calculated partial sub-keys. Function `entropyLearnedHash(data, id)` hashes `data` using the partial subkey stored as `id`. CityHash64 is used as hash function. diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index 6b6effa72d5..ed4ea8ba701 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -34,7 +34,7 @@ using Key = String; using PartialKeyPositions = std::vector; using Entropies = std::vector; -Key getPartialKey(std::string_view key, const PartialKeyPositions & partial_key_positions, Key & result) +void getPartialKey(std::string_view key, const PartialKeyPositions & partial_key_positions, Key & result) { result.clear(); result.reserve(partial_key_positions.size()); @@ -42,17 +42,15 @@ Key getPartialKey(std::string_view key, const PartialKeyPositions & partial_key_ for (auto partial_key_position : partial_key_positions) if (partial_key_position < key.size()) result.push_back(key[partial_key_position]); - - return result; } -bool allPartialKeysAreUnique(const std::vector & data, const PartialKeyPositions & partial_key_positions) +bool allPartialKeysAreUnique(const std::vector & keys, const PartialKeyPositions & partial_key_positions) { std::unordered_set unique_partial_keys; - unique_partial_keys.reserve(data.size()); + unique_partial_keys.reserve(keys.size()); Key partial_key; - for (const auto & key : data) + for (const auto & key : keys) { getPartialKey(key, partial_key_positions, partial_key); if (!unique_partial_keys.insert(partial_key).second) @@ -69,12 +67,13 @@ std::pair nextByte(const std::vector & keys, size_t max_len size_t best_position = 0; std::unordered_map count_table; + count_table.reserve(keys.size()); + Key partial_key; for (size_t i = 0; i < max_len; ++i) { count_table.clear(); - count_table.reserve(keys.size()); partial_key_positions.push_back(i); size_t collisions = 0; @@ -95,15 +94,6 @@ std::pair nextByte(const std::vector & keys, size_t max_len return {best_position, min_collisions}; } -// std::pair nextByte(const std::vector & keys, PartialKeyPositions & partial_key_positions) -// { -// size_t max_len = 0; -// for (const auto & key : keys) -// max_len = std::max(max_len, key.size()); - -// return nextByte(keys, max_len, partial_key_positions); -// } - std::pair chooseBytes(const std::vector & train_data) { if (train_data.size() <= 1) @@ -202,8 +192,8 @@ public: std::vector training_data; for (size_t i = 0; i < num_rows; ++i) { - std::string_view string_ref = col_data_string->getDataAt(i).toView(); - training_data.emplace_back(string_ref.data(), string_ref.size()); + std::string_view string_view = col_data_string->getDataAt(i).toView(); + training_data.emplace_back(string_view.data(), string_view.size()); } PartialKeyPositions partial_key_positions = chooseBytes(training_data).first; diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql b/tests/queries/0_stateless/02734_entropy_learned_hashing.sql index d50883f3cf8..2fee866710f 100644 --- a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql +++ b/tests/queries/0_stateless/02734_entropy_learned_hashing.sql @@ -1,6 +1,3 @@ --- Tags: no-parallel --- Tag no-parallel: Accesses global state - DROP TABLE IF EXISTS tbl1; CREATE TABLE tbl1 (x String) ENGINE=Memory; -- no commonalities between keys From 4a168444fa7156c401730b76ae5953c16446b2e0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 11 May 2023 19:03:17 +0000 Subject: [PATCH 0246/2223] Store keys as std::string_view --- src/Functions/EntropyLearnedHash.cpp | 23 +++++++++++------------ 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index ed4ea8ba701..af5b2c3d232 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -30,11 +30,10 @@ namespace ErrorCodes namespace { -using Key = String; using PartialKeyPositions = std::vector; using Entropies = std::vector; -void getPartialKey(std::string_view key, const PartialKeyPositions & partial_key_positions, Key & result) +void getPartialKey(std::string_view key, const PartialKeyPositions & partial_key_positions, String & result) { result.clear(); result.reserve(partial_key_positions.size()); @@ -44,11 +43,11 @@ void getPartialKey(std::string_view key, const PartialKeyPositions & partial_key result.push_back(key[partial_key_position]); } -bool allPartialKeysAreUnique(const std::vector & keys, const PartialKeyPositions & partial_key_positions) +bool allPartialKeysAreUnique(const std::vector & keys, const PartialKeyPositions & partial_key_positions) { - std::unordered_set unique_partial_keys; + std::unordered_set unique_partial_keys; unique_partial_keys.reserve(keys.size()); - Key partial_key; + String partial_key; for (const auto & key : keys) { @@ -61,15 +60,15 @@ bool allPartialKeysAreUnique(const std::vector & keys, const PartialKeyPosi } // NextByte returns position of byte which adds the most entropy and the new entropy -std::pair nextByte(const std::vector & keys, size_t max_len, PartialKeyPositions & partial_key_positions) +std::pair nextByte(const std::vector & keys, size_t max_len, PartialKeyPositions & partial_key_positions) { size_t min_collisions = std::numeric_limits::max(); size_t best_position = 0; - std::unordered_map count_table; + std::unordered_map count_table; count_table.reserve(keys.size()); - Key partial_key; + String partial_key; for (size_t i = 0; i < max_len; ++i) { @@ -94,7 +93,7 @@ std::pair nextByte(const std::vector & keys, size_t max_len return {best_position, min_collisions}; } -std::pair chooseBytes(const std::vector & train_data) +std::pair chooseBytes(const std::vector & train_data) { if (train_data.size() <= 1) return {}; @@ -189,11 +188,11 @@ public: const size_t num_rows = col_data_string->size(); /// TODO this does some needless copying ... chooseBytes() should ideally understand the native ColumnString representation - std::vector training_data; + std::vector training_data; for (size_t i = 0; i < num_rows; ++i) { std::string_view string_view = col_data_string->getDataAt(i).toView(); - training_data.emplace_back(string_view.data(), string_view.size()); + training_data.emplace_back(string_view); } PartialKeyPositions partial_key_positions = chooseBytes(training_data).first; @@ -254,7 +253,7 @@ public: auto col_res = ColumnUInt64::create(num_rows); auto & col_res_vec = col_res->getData(); - Key partial_key; + String partial_key; for (size_t i = 0; i < num_rows; ++i) { std::string_view string_ref = col_data_string->getDataAt(i).toView(); From bbfb74ab706d003f2206e229efcc2f03c9bd1c65 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 11 May 2023 19:06:04 +0000 Subject: [PATCH 0247/2223] Update comment --- src/Functions/EntropyLearnedHash.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index af5b2c3d232..9c517a51f77 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -187,7 +187,6 @@ public: { const size_t num_rows = col_data_string->size(); - /// TODO this does some needless copying ... chooseBytes() should ideally understand the native ColumnString representation std::vector training_data; for (size_t i = 0; i < num_rows; ++i) { From 60b69601e9b1e3563eb43bf4ea1deee582e088fa Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 12 May 2023 00:27:11 +0200 Subject: [PATCH 0248/2223] update docs --- .../server-configuration-parameters/settings.md | 4 ++++ docs/en/operations/settings/settings.md | 16 +++++++++++----- .../functions/date-time-functions.md | 14 +++++++++++--- .../server-configuration-parameters/settings.md | 4 ++++ docs/ru/operations/settings/settings.md | 12 ++++++++---- .../functions/date-time-functions.md | 13 +++++++++++-- 6 files changed, 49 insertions(+), 14 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index e3ca04f5b9b..36ddf6faad0 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1612,6 +1612,10 @@ The time zone is necessary for conversions between String and DateTime formats w Asia/Istanbul ``` +**See also** + +- [session_timezone](../settings/settings.md#session_timezone) + ## tcp_port {#server_configuration_parameters-tcp_port} Port for communicating with clients over the TCP protocol. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cc5f292f677..2a929acd5f2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4036,23 +4036,25 @@ Use this setting only for backward compatibility if your use cases depend on old ## session_timezone {#session_timezone} -If specified, sets an implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. -Setting this to `''` (empty string) effectively resets implicit timezone to server timezone. +If specified, sets an implicit timezone (instead of [server default](../server-configuration-parameters/settimgs.md#server_configuration_parameters-timezone). +All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. +A value of `''` (empty string) configures the session timezone to the server default timezone. + Examples: -```clickhouse +```sql SELECT timeZone(), serverTimezone() FORMAT TSV Europe/Berlin Europe/Berlin ``` -```clickhouse +```sql SELECT timeZone(), serverTimezone() SETTINGS session_timezone = 'Asia/Novosibirsk' FORMAT TSV Asia/Novosibirsk Europe/Berlin ``` -```clickhouse +```sql SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver' FORMAT TSV 1999-12-13 07:23:23.123 @@ -4064,6 +4066,10 @@ Possible values: Default value: `''`. +**See also** + +- [timezone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone) + ## final {#final} Automatically applies [FINAL](../../sql-reference/statements/select/from.md#final-modifier) modifier to all tables in a query, to tables where [FINAL](../../sql-reference/statements/select/from.md#final-modifier) is applicable, including joined tables and tables in sub-queries, and diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 596a2c509cd..9207a135c67 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -139,7 +139,7 @@ makeDateTime32(year, month, day, hour, minute, second[, fraction[, precision[, t ## timeZone -Returns the default timezone of the server for current session. This can be modified using `SET session_timezone = 'New/Value'` +Returns the default timezone of the current session, i.e. the value of setting [session_timezone](../../operations/settings/settings.md#session_timezone). If the function is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard, otherwise it produces a constant value. **Syntax** @@ -156,9 +156,13 @@ Alias: `timezone`. Type: [String](../../sql-reference/data-types/string.md). +**See also** + +- [serverTimeZone](#serverTimeZone) + ## serverTimeZone -Returns the actual timezone in which the server runs in. +Returns the default timezone of the server, i.e. the value of setting [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. **Syntax** @@ -175,6 +179,10 @@ Alias: `ServerTimezone`, `servertimezone`. Type: [String](../../sql-reference/data-types/string.md). +**See also** + +- [timeZone](#timeZone) + ## toTimeZone Converts a date or date with time to the specified time zone. Does not change the internal value (number of unix seconds) of the data, only the value's time zone attribute and the value's string representation changes. @@ -408,7 +416,7 @@ Result: ``` :::note -The return type of `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` functions described below is determined by the configuration parameter [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) which is `0` by default. +Thes return type of `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` functions described below is determined by the configuration parameter [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) which is `0` by default. Behavior for * `enable_extended_results_for_datetime_functions = 0`: Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime`. Functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` return `DateTime`. Though these functions can take values of the extended types `Date32` and `DateTime64` as an argument, passing them a time outside the normal range (year 1970 to 2149 for `Date` / 2106 for `DateTime`) will produce wrong results. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 787153d4d19..33db6df0fdd 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1355,6 +1355,10 @@ Parameters: Europe/Moscow ``` +**См. также** + +- [session_timezone](../settings/settings.md#session_timezone) + ## tcp_port {#server_configuration_parameters-tcp_port} Порт для взаимодействия с клиентами по протоколу TCP. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 585a3995afe..56bfbf8a57f 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4077,23 +4077,23 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca ## session_timezone {#session_timezone} -Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. +Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо [часового пояса сервера](../server-configuration-parameters/settimgs.md#server_configuration_parameters-timezone). То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. При значении настройки `''` (пустая строка), будет совпадать с часовым поясом сервера. Примеры: -```clickhouse +```sql SELECT timeZone(), serverTimezone() FORMAT TSV Europe/Berlin Europe/Berlin ``` -```clickhouse +```sql SELECT timeZone(), serverTimezone() SETTINGS session_timezone = 'Asia/Novosibirsk' FORMAT TSV Asia/Novosibirsk Europe/Berlin ``` -```clickhouse +```sql SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver' FORMAT TSV 1999-12-13 07:23:23.123 @@ -4104,3 +4104,7 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric - Любая зона из `system.time_zones`, например `Europe/Berlin`, `UTC` или `Zulu` Значение по умолчанию: `''`. + +**Смотрите также** + +- [timezone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone) \ No newline at end of file diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 3e378c08308..2d9f96c3199 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -26,7 +26,8 @@ SELECT ## timeZone {#timezone} -Возвращает часовой пояс сервера, считающийся умолчанием для текущей сессии. Можно изменить значение с помощью `SET session_timezone = 'New/Timezone''` +Возвращает часовой пояс сервера, считающийся умолчанием для текущей сессии: значение параметра [session_timezone](../../operations/settings/settings.md#session_timezone), если установлено. + Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. **Синтаксис** @@ -43,9 +44,13 @@ timeZone() Тип: [String](../../sql-reference/data-types/string.md). +**Смотрите также** + +- [serverTimeZone](#servertimezone) + ## serverTimeZone {#servertimezone} -Возвращает (истинный) часовой пояс сервера, в котором тот работает. +Возвращает часовой пояс сервера по умолчанию, в т.ч. установленный [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. **Синтаксис** @@ -62,6 +67,10 @@ serverTimeZone() Тип: [String](../../sql-reference/data-types/string.md). +**Смотрите также** + +- [timeZone](#timezone) + ## toTimeZone {#totimezone} Переводит дату или дату с временем в указанный часовой пояс. Часовой пояс - это атрибут типов `Date` и `DateTime`. Внутреннее значение (количество секунд) поля таблицы или результирующего столбца не изменяется, изменяется тип поля и, соответственно, его текстовое отображение. From c3af36915f049794a9c44c55ebb6a6bc950eadc8 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 12 May 2023 01:29:34 +0200 Subject: [PATCH 0249/2223] fixed docs 2 --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- src/DataTypes/Serializations/SerializationDate.cpp | 3 ++- src/DataTypes/Serializations/SerializationDate.h | 7 +++++-- src/DataTypes/Serializations/SerializationDate32.cpp | 3 ++- src/DataTypes/Serializations/SerializationDate32.h | 7 +++++-- src/IO/ReadHelpers.h | 3 +-- 7 files changed, 17 insertions(+), 10 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2a929acd5f2..e796ea83a6f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4036,7 +4036,7 @@ Use this setting only for backward compatibility if your use cases depend on old ## session_timezone {#session_timezone} -If specified, sets an implicit timezone (instead of [server default](../server-configuration-parameters/settimgs.md#server_configuration_parameters-timezone). +If specified, sets an implicit timezone (instead of [server default](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. A value of `''` (empty string) configures the session timezone to the server default timezone. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 56bfbf8a57f..98486847fd9 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4077,7 +4077,7 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca ## session_timezone {#session_timezone} -Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо [часового пояса сервера](../server-configuration-parameters/settimgs.md#server_configuration_parameters-timezone). То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. +Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо [часового пояса сервера](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. При значении настройки `''` (пустая строка), будет совпадать с часовым поясом сервера. Примеры: diff --git a/src/DataTypes/Serializations/SerializationDate.cpp b/src/DataTypes/Serializations/SerializationDate.cpp index 8b4956f7826..1ed48fdd31d 100644 --- a/src/DataTypes/Serializations/SerializationDate.cpp +++ b/src/DataTypes/Serializations/SerializationDate.cpp @@ -80,7 +80,8 @@ void SerializationDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, readCSV(value, istr, time_zone); assert_cast(column).getData().push_back(value); } -SerializationDate::SerializationDate(const TimezoneMixin & time_zone_) : TimezoneMixin(time_zone_) + +SerializationDate::SerializationDate(const DateLUTImpl & time_zone_) : time_zone(time_zone_) { } diff --git a/src/DataTypes/Serializations/SerializationDate.h b/src/DataTypes/Serializations/SerializationDate.h index c4e57470673..4d6a6fa36ec 100644 --- a/src/DataTypes/Serializations/SerializationDate.h +++ b/src/DataTypes/Serializations/SerializationDate.h @@ -6,10 +6,10 @@ namespace DB { -class SerializationDate final : public SerializationNumber, public TimezoneMixin +class SerializationDate final : public SerializationNumber { public: - explicit SerializationDate(const TimezoneMixin & time_zone_ = TimezoneMixin()); + explicit SerializationDate(const DateLUTImpl & time_zone_ = DateLUT::instance()); void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; @@ -21,6 +21,9 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + +protected: + const DateLUTImpl & time_zone; }; } diff --git a/src/DataTypes/Serializations/SerializationDate32.cpp b/src/DataTypes/Serializations/SerializationDate32.cpp index 8dcaee8d266..851710de839 100644 --- a/src/DataTypes/Serializations/SerializationDate32.cpp +++ b/src/DataTypes/Serializations/SerializationDate32.cpp @@ -78,7 +78,8 @@ void SerializationDate32::deserializeTextCSV(IColumn & column, ReadBuffer & istr readCSV(value, istr); assert_cast(column).getData().push_back(value.getExtenedDayNum()); } -SerializationDate32::SerializationDate32(const TimezoneMixin & time_zone_) : TimezoneMixin(time_zone_) + +SerializationDate32::SerializationDate32(const DateLUTImpl & time_zone_) : time_zone(time_zone_) { } } diff --git a/src/DataTypes/Serializations/SerializationDate32.h b/src/DataTypes/Serializations/SerializationDate32.h index e8e8f1a74d6..6b6e5442240 100644 --- a/src/DataTypes/Serializations/SerializationDate32.h +++ b/src/DataTypes/Serializations/SerializationDate32.h @@ -5,10 +5,10 @@ namespace DB { -class SerializationDate32 final : public SerializationNumber, public TimezoneMixin +class SerializationDate32 final : public SerializationNumber { public: - explicit SerializationDate32(const TimezoneMixin & time_zone_ = TimezoneMixin()); + explicit SerializationDate32(const DateLUTImpl & time_zone_ = DateLUT::instance()); void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; @@ -20,5 +20,8 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + +protected: + const DateLUTImpl & time_zone; }; } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index ea565d11914..3bd9275322e 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1160,8 +1160,7 @@ inline void readText(is_floating_point auto & x, ReadBuffer & buf) { readFloatTe inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf); } -inline void readText(DayNum & x, ReadBuffer & buf) { readDateText(x, buf); } -inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readDateText(x, buf, time_zone); } +inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readDateText(x, buf, time_zone); } inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); } inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); } From 1f06065de7bb05eda3da98c94c7d464fab511a4c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 12 May 2023 08:21:53 +0000 Subject: [PATCH 0250/2223] Try to fix test --- tests/queries/0_stateless/02734_entropy_learned_hashing.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql b/tests/queries/0_stateless/02734_entropy_learned_hashing.sql index 2fee866710f..c0c07b26938 100644 --- a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql +++ b/tests/queries/0_stateless/02734_entropy_learned_hashing.sql @@ -14,8 +14,8 @@ DROP TABLE IF EXISTS tbl2; CREATE TABLE tbl2 (x String) ENGINE=Memory; -- with commonalities between keys INSERT INTO tbl2 VALUES ('aa'), ('ba'), ('ca'); -SELECT trainEntropyLearnedHash(x, 'id1') FROM tbl2; -SELECT entropyLearnedHash(x, 'id1') FROM tbl2; +SELECT trainEntropyLearnedHash(x, 'id2') FROM tbl2; +SELECT entropyLearnedHash(x, 'id2') FROM tbl2; DROP TABLE tbl1; DROP TABLE tbl2; From fa3d564cc6bf2df00276195adf983eae94221a9b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 12 May 2023 11:56:41 +0000 Subject: [PATCH 0251/2223] (wip) What happens if I remove everything in msan_suppressions? Cf. https://github.com/ClickHouse/ClickHouse/pull/49678#discussion_r1192252923 --- tests/msan_suppressions.txt | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/tests/msan_suppressions.txt b/tests/msan_suppressions.txt index cf468b0be96..e69de29bb2d 100644 --- a/tests/msan_suppressions.txt +++ b/tests/msan_suppressions.txt @@ -1,12 +0,0 @@ -# https://github.com/google/oss-fuzz/issues/1099 -fun:__gxx_personality_* - -# We apply std::tolower to uninitialized padding, but don't use the result, so -# it is OK. Reproduce with "select ngramDistanceCaseInsensitive(materialize(''), '')" -fun:tolower - -# Suppress some failures in contrib so that we can enable MSan in CI. -# Ideally, we should report these upstream. - -# Hyperscan -fun:roseRunProgram From 0f7c0859b036ee1d9ede3da0fa0f325f7314364a Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 12 May 2023 09:22:29 -0400 Subject: [PATCH 0252/2223] add allow_experimental_parallel_reading_from_replicas docs --- docs/en/operations/settings/settings.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c1d6416837d..c560b8e2ad8 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1366,6 +1366,12 @@ Possible values: Default value: `default`. +## allow_experimental_parallel_reading_from_replicas + +If true, ClickHouse will send a SELECT query to all replicas of a table (up to `max_parallel_replicas`) . It will work for any kind of MergeTree table. + +Default value: `false`. + ## compile_expressions {#compile-expressions} Enables or disables compilation of frequently used simple functions and operators to native code with LLVM at runtime. From 224646698e2748baaf732f1d9b59e61450d9d56b Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 12 May 2023 09:30:06 -0400 Subject: [PATCH 0253/2223] update settings header --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 215e5b38056..99e48a5b411 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -156,7 +156,7 @@ class IColumn; M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ \ M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \ - M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ + M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table (up to `max_parallel_replicas`). It will work for any kind of MergeTree table.", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ \ From 70a8fd2c50ecaad74ab41d371bac76a9f1da74b2 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 May 2023 13:58:45 +0000 Subject: [PATCH 0254/2223] Fix schema inference with named collection, refactor Cluster table functions --- src/Storages/HDFS/StorageHDFSCluster.cpp | 16 +- src/Storages/HDFS/StorageHDFSCluster.h | 4 +- src/Storages/IStorageCluster.cpp | 6 +- src/Storages/IStorageCluster.h | 19 ++- src/Storages/StorageS3Cluster.cpp | 23 ++- src/Storages/StorageS3Cluster.h | 4 +- src/Storages/StorageURLCluster.cpp | 19 ++- src/Storages/StorageURLCluster.h | 4 +- ...tTableFunctionArgumentsFromSelectQuery.cpp | 29 ++++ ...actTableFunctionArgumentsFromSelectQuery.h | 11 ++ src/TableFunctions/ITableFunctionCluster.cpp | 3 + src/TableFunctions/ITableFunctionCluster.h | 73 +++++++++ src/TableFunctions/ITableFunctionDataLake.h | 35 ++-- src/TableFunctions/ITableFunctionFileLike.cpp | 65 +++++--- src/TableFunctions/ITableFunctionFileLike.h | 18 ++- src/TableFunctions/TableFunctionHDFS.h | 14 +- .../TableFunctionHDFSCluster.cpp | 73 +-------- src/TableFunctions/TableFunctionHDFSCluster.h | 23 +-- src/TableFunctions/TableFunctionS3.cpp | 153 +++++++++++++++--- src/TableFunctions/TableFunctionS3.h | 25 +-- src/TableFunctions/TableFunctionS3Cluster.cpp | 77 +-------- src/TableFunctions/TableFunctionS3Cluster.h | 33 ++-- src/TableFunctions/TableFunctionURL.cpp | 58 +++++-- src/TableFunctions/TableFunctionURL.h | 18 ++- .../TableFunctionURLCluster.cpp | 76 +-------- src/TableFunctions/TableFunctionURLCluster.h | 24 +-- .../0_stateless/02721_url_cluster.reference | 15 ++ .../queries/0_stateless/02721_url_cluster.sql | 6 + ...l_cluster_with_named_collections.reference | 16 ++ ...7_s3_url_cluster_with_named_collections.sh | 14 ++ 30 files changed, 572 insertions(+), 382 deletions(-) create mode 100644 src/Storages/extractTableFunctionArgumentsFromSelectQuery.cpp create mode 100644 src/Storages/extractTableFunctionArgumentsFromSelectQuery.h create mode 100644 src/TableFunctions/ITableFunctionCluster.cpp create mode 100644 src/TableFunctions/ITableFunctionCluster.h create mode 100644 tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.reference create mode 100755 tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.sh diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index f881783ad1c..90756d8be6d 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -15,12 +15,14 @@ #include #include +#include +#include #include #include -#include -#include +#include +#include #include @@ -59,13 +61,17 @@ StorageHDFSCluster::StorageHDFSCluster( setInMemoryMetadata(storage_metadata); } -void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure) +void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) { - addColumnsStructureToQueryWithHDFSClusterEngine(query, structure); + ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); + if (!expression_list) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function hdfsCluster, got '{}'", queryToString(query)); + + TableFunctionHDFSCluster::addColumnsStructureToArguments(expression_list->children, structure, context); } -RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(ASTPtr, ContextPtr context) const +RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(ASTPtr, const ContextPtr & context) const { auto iterator = std::make_shared(context, uri); auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h index a0340b2d03c..350051ab089 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -35,10 +35,10 @@ public: NamesAndTypesList getVirtuals() const override; - RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; + RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; private: - void addColumnsStructureToQuery(ASTPtr & query, const String & structure) override; + void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override; String uri; String format_name; diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index 840c5572837..9eeed18e788 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -77,7 +77,7 @@ Pipe IStorageCluster::read( const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; if (!structure_argument_was_provided) - addColumnsStructureToQuery(query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll())); + addColumnsStructureToQuery(query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), context); RestoreQualifiedNamesVisitor::Data data; data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query_info.query->as(), 0)); @@ -89,7 +89,7 @@ Pipe IStorageCluster::read( /* only_replace_in_join_= */true); visitor.visit(query_to_send); - auto new_context = updateSettingsForTableFunctionCluster(context, context->getSettingsRef()); + auto new_context = updateSettings(context, context->getSettingsRef()); const auto & current_settings = new_context->getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); for (const auto & shard_info : cluster->getShardsInfo()) @@ -129,7 +129,7 @@ QueryProcessingStage::Enum IStorageCluster::getQueryProcessingStage( return QueryProcessingStage::Enum::FetchColumns; } -ContextPtr IStorageCluster::updateSettingsForTableFunctionCluster(ContextPtr context, const Settings & settings) +ContextPtr IStorageCluster::updateSettings(ContextPtr context, const Settings & settings) { Settings new_settings = settings; diff --git a/src/Storages/IStorageCluster.h b/src/Storages/IStorageCluster.h index c37a288290d..b15ed37202a 100644 --- a/src/Storages/IStorageCluster.h +++ b/src/Storages/IStorageCluster.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -21,12 +22,18 @@ public: Poco::Logger * log_, bool structure_argument_was_provided_); - Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, - ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, size_t /*num_streams*/) override; + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t /*max_block_size*/, + size_t /*num_streams*/) override; ClusterPtr getCluster(ContextPtr context) const; /// Query is needed for pruning by virtual columns (_file, _path) - virtual RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const = 0; + virtual RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const = 0; QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; @@ -35,14 +42,14 @@ public: protected: virtual void updateBeforeRead(const ContextPtr &) {} - virtual void addColumnsStructureToQuery(ASTPtr & query, const String & structure) = 0; + virtual void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) = 0; private: + ContextPtr updateSettings(ContextPtr context, const Settings & settings); + Poco::Logger * log; String cluster_name; bool structure_argument_was_provided; - - ContextPtr updateSettingsForTableFunctionCluster(ContextPtr context, const Settings & settings); }; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 1a7b936d7ee..daf0af4f6e3 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -4,19 +4,22 @@ #if USE_AWS_S3 -#include "Common/Exception.h" #include #include -#include #include -#include +#include #include +#include #include #include +#include #include -#include #include -#include +#include +#include +#include +#include +#include #include #include @@ -61,9 +64,13 @@ StorageS3Cluster::StorageS3Cluster( virtual_block.insert({column.type->createColumn(), column.type, column.name}); } -void StorageS3Cluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure) +void StorageS3Cluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) { - addColumnsStructureToQueryWithS3ClusterEngine(query, structure); + ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); + if (!expression_list) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function s3Cluster, got '{}'", queryToString(query)); + + TableFunctionS3Cluster::addColumnsStructureToArguments(expression_list->children, structure, context); } void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context) @@ -71,7 +78,7 @@ void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context) s3_configuration.update(local_context); } -RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr query, ContextPtr context) const +RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const { auto iterator = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_block, context); diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index e0f10e038da..5c2229875e5 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -34,7 +34,7 @@ public: NamesAndTypesList getVirtuals() const override; - RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; + RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; protected: void updateConfigurationIfChanged(ContextPtr local_context); @@ -42,7 +42,7 @@ protected: private: void updateBeforeRead(const ContextPtr & context) override { updateConfigurationIfChanged(context); } - void addColumnsStructureToQuery(ASTPtr & query, const String & structure) override; + void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override; StorageS3::Configuration s3_configuration; NamesAndTypesList virtual_columns; diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 115d29aad6d..409291ec801 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -13,13 +13,14 @@ #include #include -#include +#include #include -#include #include -#include -#include +#include +#include + +#include #include @@ -62,12 +63,16 @@ StorageURLCluster::StorageURLCluster( setInMemoryMetadata(storage_metadata); } -void StorageURLCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure) +void StorageURLCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) { - addColumnsStructureToQueryWithURLClusterEngine(query, structure); + ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); + if (!expression_list) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function urlCluster, got '{}'", queryToString(query)); + + TableFunctionURLCluster::addColumnsStructureToArguments(expression_list->children, structure, context); } -RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(ASTPtr, ContextPtr context) const +RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(ASTPtr, const ContextPtr & context) const { auto iterator = std::make_shared(uri, context->getSettingsRef().glob_expansion_max_elements); auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); diff --git a/src/Storages/StorageURLCluster.h b/src/Storages/StorageURLCluster.h index 727a99f17f9..67771416771 100644 --- a/src/Storages/StorageURLCluster.h +++ b/src/Storages/StorageURLCluster.h @@ -34,10 +34,10 @@ public: NamesAndTypesList getVirtuals() const override; - RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, ContextPtr context) const override; + RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; private: - void addColumnsStructureToQuery(ASTPtr & query, const String & structure) override; + void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override; String uri; String format_name; diff --git a/src/Storages/extractTableFunctionArgumentsFromSelectQuery.cpp b/src/Storages/extractTableFunctionArgumentsFromSelectQuery.cpp new file mode 100644 index 00000000000..382964d9fe1 --- /dev/null +++ b/src/Storages/extractTableFunctionArgumentsFromSelectQuery.cpp @@ -0,0 +1,29 @@ +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query) +{ + auto * select_query = query->as(); + if (!select_query || !select_query->tables()) + return nullptr; + + auto * tables = select_query->tables()->as(); + auto * table_expression = tables->children[0]->as()->table_expression->as(); + if (!table_expression->table_function) + return nullptr; + + auto * table_function = table_expression->table_function->as(); + return table_function->arguments->as(); +} + +} diff --git a/src/Storages/extractTableFunctionArgumentsFromSelectQuery.h b/src/Storages/extractTableFunctionArgumentsFromSelectQuery.h new file mode 100644 index 00000000000..8bf5d95a42c --- /dev/null +++ b/src/Storages/extractTableFunctionArgumentsFromSelectQuery.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + +namespace DB +{ + +ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query); + +} diff --git a/src/TableFunctions/ITableFunctionCluster.cpp b/src/TableFunctions/ITableFunctionCluster.cpp new file mode 100644 index 00000000000..bde08294a81 --- /dev/null +++ b/src/TableFunctions/ITableFunctionCluster.cpp @@ -0,0 +1,3 @@ +// +// Created by Павел Круглов on 12/05/2023. +// diff --git a/src/TableFunctions/ITableFunctionCluster.h b/src/TableFunctions/ITableFunctionCluster.h new file mode 100644 index 00000000000..f3aabc77706 --- /dev/null +++ b/src/TableFunctions/ITableFunctionCluster.h @@ -0,0 +1,73 @@ +#pragma once + +#include "config.h" + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int BAD_GET; + extern const int LOGICAL_ERROR; +} + +class Context; + +/// Base class for *Cluster table functions that require cluster_name for the first argument. +template +class ITableFunctionCluster : public Base +{ +public: + String getName() const override = 0; + String getSignature() const override = 0; + + static void addColumnsStructureToArguments(ASTs & args, const String & desired_structure, const ContextPtr & context) + { + if (args.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected empty list of arguments for {}Cluster table function", Base::name); + + ASTPtr cluster_name_arg = args.front(); + args.erase(args.begin()); + Base::addColumnsStructureToArguments(args, desired_structure, context); + args.insert(args.begin(), cluster_name_arg); + } + +protected: + void parseArguments(const ASTPtr & ast, ContextPtr context) override + { + /// Clone ast function, because we can modify its arguments like removing cluster_name + Base::parseArguments(ast->clone(), context); + } + + void parseArgumentsImpl(ASTs & args, const ContextPtr & context) override + { + if (args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); + + /// Evaluate only first argument, everything else will be done Base class + args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context); + + /// Cluster name is always the first + cluster_name = checkAndGetLiteralArgument(args[0], "cluster_name"); + + if (!context->tryGetCluster(cluster_name)) + throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", cluster_name); + + /// Just cut the first arg (cluster_name) and try to parse other table function arguments as is + args.erase(args.begin()); + + Base::parseArgumentsImpl(args, context); + } + + String cluster_name; +}; + +} diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 1a20632430b..34331e86bc1 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -19,7 +19,7 @@ namespace ErrorCodes } template -class ITableFunctionDataLake : public ITableFunction +class ITableFunctionDataLake : public TableFunction { public: static constexpr auto name = Name::name; @@ -33,11 +33,11 @@ protected: ColumnsDescription /*cached_columns*/) const override { ColumnsDescription columns; - if (configuration.structure != "auto") - columns = parseColumnsListFromString(configuration.structure, context); + if (TableFunction::configuration.structure != "auto") + columns = parseColumnsListFromString(TableFunction::configuration.structure, context); StoragePtr storage = std::make_shared( - configuration, context, StorageID(getDatabaseName(), table_name), + TableFunction::configuration, context, StorageID(TableFunction::getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, std::nullopt); storage->startup(); @@ -48,34 +48,21 @@ protected: ColumnsDescription getActualTableStructure(ContextPtr context) const override { - if (configuration.structure == "auto") + if (TableFunction::configuration.structure == "auto") { - context->checkAccess(getSourceAccessType()); - return Storage::getTableStructureFromData(configuration, std::nullopt, context); + context->checkAccess(TableFunction::getSourceAccessType()); + return Storage::getTableStructureFromData(TableFunction::configuration, std::nullopt, context); } - return parseColumnsListFromString(configuration.structure, context); + return parseColumnsListFromString(TableFunction::configuration.structure, context); } void parseArguments(const ASTPtr & ast_function, ContextPtr context) override { - ASTs & args_func = ast_function->children; - - const auto message = fmt::format( - "The signature of table function '{}' could be the following:\n{}", getName(), TableFunction::signature); - - if (args_func.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments", getName()); - - auto & args = args_func.at(0)->children; - - TableFunction::parseArgumentsImpl(message, args, context, configuration, false); - - if (configuration.format == "auto") - configuration.format = "Parquet"; + /// Set default format to Parquet if it's not specified in arguments. + TableFunction::configuration.format = "Parquet"; + TableFunction::parseArguments(ast_function, context); } - - mutable typename Storage::Configuration configuration; }; } diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index bbaf2b68418..4b6a8f06741 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -47,9 +47,13 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function '{}' must have arguments.", getName()); ASTs & args = args_func.at(0)->children; + parseArgumentsImpl(args, context); +} - if (args.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires at least 1 argument", getName()); +void ITableFunctionFileLike::parseArgumentsImpl(ASTs & args, const ContextPtr & context) +{ + if (args.empty() || args.size() > 4) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); @@ -62,26 +66,51 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context if (format == "auto") format = getFormatFromFirstArgument(); - if (args.size() <= 2) - return; + if (args.size() > 2) + { + structure = checkAndGetLiteralArgument(args[2], "structure"); + if (structure.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Table structure is empty for table function '{}'. If you want to use automatic schema inference, use 'auto'", + getName()); + } - if (args.size() != 3 && args.size() != 4) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Table function '{}' requires 1, 2, 3 or 4 arguments: " - "filename, format (default auto), structure (default auto) and compression method (default auto)", - getName()); - - structure = checkAndGetLiteralArgument(args[2], "structure"); - - if (structure.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Table structure is empty for table function '{}'. If you want to use automatic schema inference, use 'auto'", - ast_function->formatForErrorMessage()); - - if (args.size() == 4) + if (args.size() > 3) compression_method = checkAndGetLiteralArgument(args[3], "compression_method"); } +void ITableFunctionFileLike::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr &) +{ + if (args.empty() || args.size() > getMaxNumberOfArguments()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function, got {}", getMaxNumberOfArguments(), args.size()); + + auto structure_literal = std::make_shared(structure); + + /// f(filename) + if (args.size() == 1) + { + /// Add format=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// f(filename, format) + else if (args.size() == 2) + { + args.push_back(structure_literal); + } + /// f(filename, format, 'auto') + else if (args.size() == 3) + { + args.back() = structure_literal; + } + /// f(filename, format, 'auto', compression) + else if (args.size() == 4) + { + args[args.size() - 2] = structure_literal; + } +} + StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { ColumnsDescription columns; diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 589fce67638..8300cc27591 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -9,19 +9,35 @@ class ColumnsDescription; class Context; /* - * function(source, format, structure[, compression_method]) - creates a temporary storage from formatted source + * function(source, [format, structure, compression_method]) - creates a temporary storage from formatted source */ class ITableFunctionFileLike : public ITableFunction { public: + static constexpr auto signature = " - filename\n" + " - filename, format\n" + " - filename, format, structure\n" + " - filename, format, structure, compression_method\n"; + virtual String getSignature() const + { + return signature; + } + bool needStructureHint() const override { return structure == "auto"; } void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } bool supportsReadingSubsetOfColumns() override; + static size_t getMaxNumberOfArguments() { return 4; } + + static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr &); + protected: + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); + virtual void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context); virtual String getFormatFromFirstArgument(); diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h index c4c111de6e5..eec0a05fe8d 100644 --- a/src/TableFunctions/TableFunctionHDFS.h +++ b/src/TableFunctions/TableFunctionHDFS.h @@ -12,18 +12,28 @@ namespace DB class Context; -/* hdfs(URI, format[, structure, compression]) - creates a temporary storage from hdfs files +/* hdfs(URI, [format, structure, compression]) - creates a temporary storage from hdfs files * */ class TableFunctionHDFS : public ITableFunctionFileLike { public: static constexpr auto name = "hdfs"; - std::string getName() const override + static constexpr auto signature = " - uri\n" + " - uri, format\n" + " - uri, format, structure\n" + " - uri, format, structure, compression_method\n"; + + String getName() const override { return name; } + String getSignature() const override + { + return signature; + } + ColumnsDescription getActualTableStructure(ContextPtr context) const override; std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override diff --git a/src/TableFunctions/TableFunctionHDFSCluster.cpp b/src/TableFunctions/TableFunctionHDFSCluster.cpp index 772965d0b68..6fb7ed0fce5 100644 --- a/src/TableFunctions/TableFunctionHDFSCluster.cpp +++ b/src/TableFunctions/TableFunctionHDFSCluster.cpp @@ -2,86 +2,19 @@ #if USE_HDFS -#include - -#include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include -#include -#include +#include +#include +#include #include "registerTableFunctions.h" #include -#include namespace DB { -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_GET; -} - - -void TableFunctionHDFSCluster::parseArguments(const ASTPtr & ast_function, ContextPtr context) -{ - auto ast_copy = ast_function->clone(); - /// Parse args - ASTs & args_func = ast_copy->children; - - if (args_func.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); - - ASTs & args = args_func.at(0)->children; - - if (args.size() < 2 || args.size() > 5) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "The signature of table function {} shall be the following:\n" - " - cluster, uri\n" - " - cluster, uri, format\n" - " - cluster, uri, format, structure\n" - " - cluster, uri, format, structure, compression_method", - getName()); - - for (auto & arg : args) - arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - - /// This argument is always the first - cluster_name = checkAndGetLiteralArgument(args[0], "cluster_name"); - - if (!context->tryGetCluster(cluster_name)) - throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", cluster_name); - - /// Just cut the first arg (cluster_name) and try to parse other table function arguments as is - args.erase(args.begin()); - - ITableFunctionFileLike::parseArguments(ast_copy, context); -} - - -ColumnsDescription TableFunctionHDFSCluster::getActualTableStructure(ContextPtr context) const -{ - if (structure == "auto") - { - context->checkAccess(getSourceAccessType()); - return StorageHDFS::getTableStructureFromData(format, filename, compression_method, context); - } - - return parseColumnsListFromString(structure, context); -} - - StoragePtr TableFunctionHDFSCluster::getStorage( const String & /*source*/, const String & /*format_*/, const ColumnsDescription & columns, ContextPtr context, const std::string & table_name, const String & /*compression_method_*/) const diff --git a/src/TableFunctions/TableFunctionHDFSCluster.h b/src/TableFunctions/TableFunctionHDFSCluster.h index fff2c8ad116..0253217feb7 100644 --- a/src/TableFunctions/TableFunctionHDFSCluster.h +++ b/src/TableFunctions/TableFunctionHDFSCluster.h @@ -5,6 +5,8 @@ #if USE_HDFS #include +#include +#include namespace DB @@ -20,28 +22,31 @@ class Context; * On worker node it asks initiator about next task to process, processes it. * This is repeated until the tasks are finished. */ -class TableFunctionHDFSCluster : public ITableFunctionFileLike +class TableFunctionHDFSCluster : public ITableFunctionCluster { public: static constexpr auto name = "hdfsCluster"; - std::string getName() const override + 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"; + + String getName() const override { return name; } + String getSignature() const override + { + return signature; + } + protected: StoragePtr getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const String & compression_method_) const override; const char * getStorageTypeName() const override { return "HDFSCluster"; } - - AccessType getSourceAccessType() const override { return AccessType::HDFS; } - - ColumnsDescription getActualTableStructure(ContextPtr) const override; - void parseArguments(const ASTPtr &, ContextPtr) override; - - String cluster_name; }; } diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 2369a4f384f..cd175ad2ea0 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -10,13 +10,14 @@ #include #include #include +#include +#include #include #include #include #include #include #include "registerTableFunctions.h" -#include #include @@ -31,19 +32,18 @@ namespace ErrorCodes /// This is needed to avoid copy-pase. Because s3Cluster arguments only differ in additional argument (first) - cluster name -void TableFunctionS3::parseArgumentsImpl( - const String & error_message, ASTs & args, ContextPtr context, StorageS3::Configuration & s3_configuration, bool get_format_from_file) +void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context) { if (auto named_collection = tryGetNamedCollectionWithOverrides(args, context)) { - StorageS3::processNamedCollectionResult(s3_configuration, *named_collection); + StorageS3::processNamedCollectionResult(configuration, *named_collection); } else { if (args.empty() || args.size() > 6) - throw Exception::createDeprecated(error_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); - auto * header_it = StorageURL::collectHeaders(args, s3_configuration.headers_from_ast, context); + auto * header_it = StorageURL::collectHeaders(args, configuration.headers_from_ast, context); if (header_it != args.end()) args.erase(header_it); @@ -130,46 +130,159 @@ void TableFunctionS3::parseArgumentsImpl( } /// This argument is always the first - s3_configuration.url = S3::URI(checkAndGetLiteralArgument(args[0], "url")); + configuration.url = S3::URI(checkAndGetLiteralArgument(args[0], "url")); if (args_to_idx.contains("format")) - s3_configuration.format = checkAndGetLiteralArgument(args[args_to_idx["format"]], "format"); + configuration.format = checkAndGetLiteralArgument(args[args_to_idx["format"]], "format"); if (args_to_idx.contains("structure")) - s3_configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); + configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); if (args_to_idx.contains("compression_method")) - s3_configuration.compression_method = checkAndGetLiteralArgument(args[args_to_idx["compression_method"]], "compression_method"); + configuration.compression_method = checkAndGetLiteralArgument(args[args_to_idx["compression_method"]], "compression_method"); if (args_to_idx.contains("access_key_id")) - s3_configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(args[args_to_idx["access_key_id"]], "access_key_id"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(args[args_to_idx["access_key_id"]], "access_key_id"); if (args_to_idx.contains("secret_access_key")) - s3_configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); - s3_configuration.auth_settings.no_sign_request = no_sign_request; + configuration.auth_settings.no_sign_request = no_sign_request; } - s3_configuration.keys = {s3_configuration.url.key}; + configuration.keys = {configuration.url.key}; - /// For DataLake table functions, we should specify default format. - if (s3_configuration.format == "auto" && get_format_from_file) - s3_configuration.format = FormatFactory::instance().getFormatFromFileName(s3_configuration.url.uri.getPath(), true); + if (configuration.format == "auto") + configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.url.uri.getPath(), true); } void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr context) { + /// Clone ast function, because we can modify its arguments like removing headers. + auto ast_copy = ast_function->clone(); + /// Parse args ASTs & args_func = ast_function->children; - const auto message = fmt::format("The signature of table function '{}' could be the following:\n{}", getName(), signature); - if (args_func.size() != 1) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); auto & args = args_func.at(0)->children; - parseArgumentsImpl(message, args, context, configuration); + parseArgumentsImpl(args, context); +} + +void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context) +{ + if (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); + } + else + { + /// If arguments contain headers, just remove it and add to the end of arguments later + /// (header argument can be at any position). + HTTPHeaderEntries tmp_headers; + auto * headers_it = StorageURL::collectHeaders(args, tmp_headers, context); + ASTPtr headers_ast; + if (headers_it != args.end()) + { + headers_ast = *headers_it; + args.erase(headers_it); + } + + if (args.empty() || args.size() > getMaxNumberOfArguments()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function, got {}", getMaxNumberOfArguments(), args.size()); + + auto structure_literal = std::make_shared(structure); + + /// s3(s3_url) + if (args.size() == 1) + { + /// Add format=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// 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 (args.size() == 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(structure_literal); + } + /// s3(source, format, structure) or + /// s3(source, access_key_id, access_key_id) or + /// s3(source, NOSIGN, format) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither. + else if (args.size() == 3) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + args.push_back(structure_literal); + } + else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + { + args.back() = structure_literal; + } + else + { + /// Add format=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + } + /// s3(source, format, structure, compression_method) or + /// s3(source, access_key_id, access_key_id, format) 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 (args.size() == 4) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + args.back() = structure_literal; + } + else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + { + args[args.size() - 2] = structure_literal; + } + else + { + args.push_back(structure_literal); + } + } + /// s3(source, access_key_id, access_key_id, format, structure) 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 (args.size() == 5) + { + auto sedond_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(sedond_arg, "NOSIGN")) + { + args[args.size() - 2] = structure_literal; + } + else + { + args.back() = structure_literal; + } + } + /// s3(source, access_key_id, access_key_id, format, structure, compression) + else if (args.size() == 6) + { + args[args.size() - 2] = structure_literal; + } + + if (headers_ast) + args.push_back(headers_ast); + } } ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) const diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index 14c87cbf022..c983bec9bf4 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -13,7 +13,7 @@ namespace DB class Context; -/* s3(source, [access_key_id, secret_access_key,] format, structure[, compression]) - creates a temporary storage for a file in S3. +/* s3(source, [access_key_id, secret_access_key,] [format, structure, compression]) - creates a temporary storage for a file in S3. */ class TableFunctionS3 : public ITableFunction { @@ -26,11 +26,21 @@ public: " - url, format, structure, compression_method\n" " - url, access_key_id, secret_access_key, format\n" " - url, access_key_id, secret_access_key, format, structure\n" - " - url, access_key_id, secret_access_key, format, structure, compression_method"; - std::string getName() const override + " - url, access_key_id, secret_access_key, format, structure, compression_method\n" + "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; + + static size_t getMaxNumberOfArguments() { return 6; } + + 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"; } @@ -44,12 +54,9 @@ public: return {"_path", "_file"}; } - static void parseArgumentsImpl( - const String & error_message, - ASTs & args, - ContextPtr context, - StorageS3::Configuration & configuration, - bool get_format_from_file = true); + virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); + + static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context); protected: diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 2b148ddda9e..1d93132c411 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -2,92 +2,19 @@ #if USE_AWS_S3 -#include -#include - -#include -#include -#include -#include -#include -#include -#include #include +#include #include -#include -#include -#include -#include +#include #include "registerTableFunctions.h" #include -#include namespace DB { -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_GET; -} - - -void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, ContextPtr context) -{ - /// Parse args - ASTs & args_func = ast_function->children; - - if (args_func.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); - - ASTs & args = args_func.at(0)->children; - - constexpr auto fmt_string = "The signature of table function {} could be the following:\n" - " - 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"; - auto message = PreformattedMessage{fmt::format(fmt_string, getName()), fmt_string}; - if (args.size() < 2 || args.size() > 7) - throw Exception::createDeprecated(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - /// Evaluate only first argument, everything else will be done TableFunctionS3 - args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context); - - /// Cluster name is always the first - cluster_name = checkAndGetLiteralArgument(args[0], "cluster_name"); - - if (!context->tryGetCluster(cluster_name)) - throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", cluster_name); - - /// Just cut the first arg (cluster_name) and try to parse s3 table function arguments as is - ASTs clipped_args; - clipped_args.reserve(args.size() - 1); - std::copy(args.begin() + 1, args.end(), std::back_inserter(clipped_args)); - - /// StorageS3ClusterConfiguration inherints from StorageS3::Configuration, so it is safe to upcast it. - TableFunctionS3::parseArgumentsImpl(message.text, clipped_args, context, static_cast(configuration)); -} - - -ColumnsDescription TableFunctionS3Cluster::getActualTableStructure(ContextPtr context) const -{ - context->checkAccess(getSourceAccessType()); - - configuration.update(context); - if (configuration.structure == "auto") - return StorageS3::getTableStructureFromData(configuration, std::nullopt, context); - - return parseColumnsListFromString(configuration.structure, context); -} - StoragePtr TableFunctionS3Cluster::executeImpl( const ASTPtr & /*function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const diff --git a/src/TableFunctions/TableFunctionS3Cluster.h b/src/TableFunctions/TableFunctionS3Cluster.h index 9ca6636b6b6..459ff144f02 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.h +++ b/src/TableFunctions/TableFunctionS3Cluster.h @@ -6,6 +6,7 @@ #include #include +#include #include @@ -22,20 +23,29 @@ class Context; * On worker node it asks initiator about next task to process, processes it. * This is repeated until the tasks are finished. */ -class TableFunctionS3Cluster : public ITableFunction +class TableFunctionS3Cluster : public ITableFunctionCluster { public: static constexpr auto name = "s3Cluster"; - std::string getName() const override + 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" + "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; + + String getName() const override { return name; } - bool hasStaticStructure() const override { return configuration.structure != "auto"; } - - bool needStructureHint() const override { return configuration.structure == "auto"; } - - void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } + String getSignature() const override + { + return signature; + } protected: StoragePtr executeImpl( @@ -45,15 +55,6 @@ protected: ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "S3Cluster"; } - - AccessType getSourceAccessType() const override { return AccessType::S3; } - - ColumnsDescription getActualTableStructure(ContextPtr) const override; - void parseArguments(const ASTPtr &, ContextPtr) override; - - String cluster_name; - mutable StorageS3::Configuration configuration; - ColumnsDescription structure_hint; }; } diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index d8c3e72efe1..b138d80a9ec 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -18,10 +18,6 @@ namespace DB { -static const String bad_arguments_error_message = "Table function URL can have the following arguments: " - "url, name of used format (taken from file extension by default), " - "optional table structure, optional compression method, " - "optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; namespace ErrorCodes { @@ -48,15 +44,13 @@ std::vector TableFunctionURL::skipAnalysisForArguments(const QueryTreeNo void TableFunctionURL::parseArguments(const ASTPtr & ast, ContextPtr context) { - const auto & ast_function = assert_cast(ast.get()); + /// Clone ast function, because we can modify it's arguments like removing headers. + ITableFunctionFileLike::parseArguments(ast->clone(), context); +} - const auto & args = ast_function->children; - if (args.empty()) - throw Exception::createDeprecated(bad_arguments_error_message, ErrorCodes::BAD_ARGUMENTS); - - auto & url_function_args = assert_cast(args[0].get())->children; - - if (auto named_collection = tryGetNamedCollectionWithOverrides(url_function_args, context)) +void TableFunctionURL::parseArgumentsImpl(ASTs & args, const ContextPtr & context) +{ + if (auto named_collection = tryGetNamedCollectionWithOverrides(args, context)) { StorageURL::processNamedCollectionResult(configuration, *named_collection); @@ -68,16 +62,46 @@ void TableFunctionURL::parseArguments(const ASTPtr & ast, ContextPtr context) if (format == "auto") format = FormatFactory::instance().getFormatFromFileName(Poco::URI(filename).getPath(), true); - StorageURL::collectHeaders(url_function_args, configuration.headers, context); + StorageURL::collectHeaders(args, configuration.headers, context); } else { - auto * headers_it = StorageURL::collectHeaders(url_function_args, configuration.headers, context); + auto * headers_it = StorageURL::collectHeaders(args, configuration.headers, context); /// ITableFunctionFileLike cannot parse headers argument, so remove it. - if (headers_it != url_function_args.end()) - url_function_args.erase(headers_it); + if (headers_it != args.end()) + args.erase(headers_it); - ITableFunctionFileLike::parseArguments(ast, context); + ITableFunctionFileLike::parseArgumentsImpl(args, context); + } +} + +void TableFunctionURL::addColumnsStructureToArguments(ASTs & args, const String & desired_structure, const ContextPtr & context) +{ + if (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(desired_structure)}; + auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); + args.push_back(equal_func); + } + else + { + /// If arguments contain headers, just remove it and add to the end of arguments later + /// (header argument can be at any position). + HTTPHeaderEntries tmp_headers; + auto * headers_it = StorageURL::collectHeaders(args, tmp_headers, context); + ASTPtr headers_ast; + if (headers_it != args.end()) + { + headers_ast = *headers_it; + args.erase(headers_it); + } + + ITableFunctionFileLike::addColumnsStructureToArguments(args, desired_structure, context); + + if (headers_ast) + args.push_back(headers_ast); } } diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index 32455eefaea..021eb71df53 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -10,21 +10,35 @@ namespace DB class Context; -/* url(source, format[, structure, compression]) - creates a temporary storage from url. +/* url(source, [format, structure, compression]) - creates a temporary storage from url. */ class TableFunctionURL : public ITableFunctionFileLike { public: static constexpr auto name = "url"; - std::string getName() const override + static constexpr auto signature = " - uri\n" + " - uri, format\n" + " - uri, format, structure\n" + " - uri, format, structure, compression_method\n" + "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; + + String getName() const override { return name; } + String getSignature() const override + { + return signature; + } + ColumnsDescription getActualTableStructure(ContextPtr context) const override; + static void addColumnsStructureToArguments(ASTs & args, const String & desired_structure, const ContextPtr & context); + protected: void parseArguments(const ASTPtr & ast, ContextPtr context) override; + void parseArgumentsImpl(ASTs & args, const ContextPtr & context) override; StorageURL::Configuration configuration; diff --git a/src/TableFunctions/TableFunctionURLCluster.cpp b/src/TableFunctions/TableFunctionURLCluster.cpp index fc37ecc7f1d..c94943db758 100644 --- a/src/TableFunctions/TableFunctionURLCluster.cpp +++ b/src/TableFunctions/TableFunctionURLCluster.cpp @@ -1,83 +1,11 @@ #include +#include #include "registerTableFunctions.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include namespace DB { -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_GET; -} - -void TableFunctionURLCluster::parseArguments(const ASTPtr & ast_function, ContextPtr context) -{ - auto ast_copy = ast_function->clone(); - /// Parse args - ASTs & args_func = ast_copy->children; - - if (args_func.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function {} must have arguments", getName()); - - ASTs & args = args_func.at(0)->children; - - if (args.size() < 2 || args.size() > 5) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "The signature of table function {} shall be the following:\n" - " - cluster, uri\n" - " - cluster, uri, format\n" - " - cluster, uri, format, structure\n" - " - cluster, uri, format, structure, compression_method", - getName()); - - for (auto & arg : args) - arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - - /// This argument is always the first - cluster_name = checkAndGetLiteralArgument(args[0], "cluster_name"); - - if (!context->tryGetCluster(cluster_name)) - throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", cluster_name); - - /// Just cut the first arg (cluster_name) and try to parse other table function arguments as is - args.erase(args.begin()); - - TableFunctionURL::parseArguments(ast_copy, context); -} - - -ColumnsDescription TableFunctionURLCluster::getActualTableStructure(ContextPtr context) const -{ - if (structure == "auto") - { - context->checkAccess(getSourceAccessType()); - return StorageURL::getTableStructureFromData(format, - filename, - chooseCompressionMethod(Poco::URI(filename).getPath(), compression_method), - configuration.headers, - std::nullopt, - context); - } - - return parseColumnsListFromString(structure, context); -} - - StoragePtr TableFunctionURLCluster::getStorage( const String & /*source*/, const String & /*format_*/, const ColumnsDescription & columns, ContextPtr context, const std::string & table_name, const String & /*compression_method_*/) const @@ -85,7 +13,7 @@ StoragePtr TableFunctionURLCluster::getStorage( StoragePtr storage; if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) { - //On worker node this uri won't contains globs + //On worker node this uri won't contain globs storage = std::make_shared( filename, StorageID(getDatabaseName(), table_name), diff --git a/src/TableFunctions/TableFunctionURLCluster.h b/src/TableFunctions/TableFunctionURLCluster.h index e7935e9a9be..be6992fcaaf 100644 --- a/src/TableFunctions/TableFunctionURLCluster.h +++ b/src/TableFunctions/TableFunctionURLCluster.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -20,29 +21,32 @@ class Context; * On worker node it asks initiator about next task to process, processes it. * This is repeated until the tasks are finished. */ -class TableFunctionURLCluster : public TableFunctionURL +class TableFunctionURLCluster : public ITableFunctionCluster { public: static constexpr auto name = "urlCluster"; - std::string getName() const override + static constexpr auto signature = " - cluster, uri\n" + " - cluster, uri, format\n" + " - cluster, uri, format, structure\n" + " - cluster, uri, format, structure, compression_method\n" + "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; + + String getName() const override { return name; } + String getSignature() const override + { + return signature; + } + protected: StoragePtr getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const String & compression_method_) const override; const char * getStorageTypeName() const override { return "URLCluster"; } - - AccessType getSourceAccessType() const override { return AccessType::URL; } - - ColumnsDescription getActualTableStructure(ContextPtr) const override; - void parseArguments(const ASTPtr &, ContextPtr) override; - - String cluster_name; - }; } diff --git a/tests/queries/0_stateless/02721_url_cluster.reference b/tests/queries/0_stateless/02721_url_cluster.reference index 36a5e136469..36ef7609920 100644 --- a/tests/queries/0_stateless/02721_url_cluster.reference +++ b/tests/queries/0_stateless/02721_url_cluster.reference @@ -75,6 +75,21 @@ c3 Nullable(Int64) c1 Nullable(Int64) c2 Nullable(Int64) c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) 0 0 0 0 0 0 1 2 3 diff --git a/tests/queries/0_stateless/02721_url_cluster.sql b/tests/queries/0_stateless/02721_url_cluster.sql index ea387b06dc3..49791db1a2c 100644 --- a/tests/queries/0_stateless/02721_url_cluster.sql +++ b/tests/queries/0_stateless/02721_url_cluster.sql @@ -20,6 +20,12 @@ desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://local desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto'); desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto', 'auto'); +desc urlCluster('test_cluster_one_shard_three_replicas_localhost', headers('X-ClickHouse-Database'='default'), 'http://localhost:11111/test/{a,b}.tsv'); +desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', headers('X-ClickHouse-Database'='default'), 'TSV'); +desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'auto', headers('X-ClickHouse-Database'='default')); +desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto', headers('X-ClickHouse-Database'='default')); +desc urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', headers('X-ClickHouse-Database'='default'), 'auto', 'auto'); + select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv') order by c1, c2, c3; select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV') order by c1, c2, c3; select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'auto') order by c1, c2, c3; diff --git a/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.reference b/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.reference new file mode 100644 index 00000000000..a753e633bab --- /dev/null +++ b/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.reference @@ -0,0 +1,16 @@ +1 2 3 +4 5 6 +7 8 9 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +0 0 0 diff --git a/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.sh b/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.sh new file mode 100755 index 00000000000..6526ca8961f --- /dev/null +++ b/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-s3-storage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "create named collection $CLICKHOUSE_TEST_UNIQUE_NAME as url='http://localhost:11111/test/a.tsv'" +$CLICKHOUSE_CLIENT -q "select * from s3Cluster(test_cluster_one_shard_three_replicas_localhost, $CLICKHOUSE_TEST_UNIQUE_NAME)" +$CLICKHOUSE_CLIENT -q "select * from s3Cluster(test_cluster_one_shard_three_replicas_localhost, $CLICKHOUSE_TEST_UNIQUE_NAME, structure='auto')" +$CLICKHOUSE_CLIENT -q "select * from urlCluster(test_cluster_one_shard_three_replicas_localhost, $CLICKHOUSE_TEST_UNIQUE_NAME)" +$CLICKHOUSE_CLIENT -q "select * from urlCluster(test_cluster_one_shard_three_replicas_localhost, $CLICKHOUSE_TEST_UNIQUE_NAME, structure='auto')" +$CLICKHOUSE_CLIENT -q "drop named collection $CLICKHOUSE_TEST_UNIQUE_NAME" + From 5d6a28b22c050f1c4578787d69813f3824f3b458 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 May 2023 14:00:22 +0000 Subject: [PATCH 0255/2223] Update test --- tests/queries/0_stateless/02721_url_cluster.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02721_url_cluster.sql b/tests/queries/0_stateless/02721_url_cluster.sql index 49791db1a2c..c30b03495cd 100644 --- a/tests/queries/0_stateless/02721_url_cluster.sql +++ b/tests/queries/0_stateless/02721_url_cluster.sql @@ -34,7 +34,7 @@ select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'htt drop table if exists test; create table test (x UInt32, y UInt32, z UInt32) engine=Memory(); -insert into test select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/a.tsv', 'TSV'); +insert into test select * from urlCluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/a.tsv', 'TSV'); select * from test; drop table test; From 3d00f4cdf6084374d2c156b4ec0f072876b7965d Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 May 2023 14:03:04 +0000 Subject: [PATCH 0256/2223] Remove unneded files --- ...lumnsStructureToQueryWithClusterEngine.cpp | 175 ------------------ ...ColumnsStructureToQueryWithClusterEngine.h | 13 -- 2 files changed, 188 deletions(-) delete mode 100644 src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp delete mode 100644 src/Storages/addColumnsStructureToQueryWithClusterEngine.h diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp deleted file mode 100644 index cb8b7dcea59..00000000000 --- a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp +++ /dev/null @@ -1,175 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -static ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query) -{ - auto * select_query = query->as(); - if (!select_query || !select_query->tables()) - return nullptr; - - auto * tables = select_query->tables()->as(); - auto * table_expression = tables->children[0]->as()->table_expression->as(); - if (!table_expression->table_function) - return nullptr; - - auto * table_function = table_expression->table_function->as(); - return table_function->arguments->as(); -} - -static ASTExpressionList * getExpressionListAndCheckArguments(ASTPtr & query, size_t max_arguments, const String & function_name) -{ - ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); - if (!expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function {}, got '{}'", function_name, queryToString(query)); - - if (expression_list->children.size() < 2 || expression_list->children.size() > max_arguments) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 2 to {} arguments in {} table functions, got {}", - max_arguments, function_name, expression_list->children.size()); - - return expression_list; -} - -static void addColumnsStructureToQueryWithHDFSOrURLClusterEngine(ASTPtr & query, const String & structure, const String & function_name) -{ - ASTExpressionList * expression_list = getExpressionListAndCheckArguments(query, 5, function_name); - auto structure_literal = std::make_shared(structure); - ASTs & args = expression_list->children; - - /// XCuster(cluster_name, source) - if (args.size() == 2) - { - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - /// XCuster(cluster_name, source, format) - else if (args.size() == 3) - { - args.push_back(structure_literal); - } - /// XCuster(cluster_name, source, format, 'auto') - else if (args.size() == 4) - { - args.back() = structure_literal; - } - /// XCuster(cluster_name, source, format, 'auto', compression) - else if (args.size() == 5) - { - args[args.size() - 2] = structure_literal; - } -} - - -void addColumnsStructureToQueryWithHDFSClusterEngine(ASTPtr & query, const String & structure) -{ - addColumnsStructureToQueryWithHDFSOrURLClusterEngine(query, structure, "hdfsCluster"); -} - -void addColumnsStructureToQueryWithURLClusterEngine(ASTPtr & query, const String & structure) -{ - addColumnsStructureToQueryWithHDFSOrURLClusterEngine(query, structure, "urlCluster"); -} - -void addColumnsStructureToQueryWithS3ClusterEngine(ASTPtr & query, const String & structure) -{ - ASTExpressionList * expression_list = getExpressionListAndCheckArguments(query, 7, "s3Cluster"); - auto structure_literal = std::make_shared(structure); - ASTs & args = expression_list->children; - - /// s3Cluster(cluster_name, s3_url) - if (args.size() == 2) - { - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - /// s3Cluster(cluster_name, s3_url, format) or s3Cluster(cluster_name, s3_url, NOSIGN) - /// We can distinguish them by looking at the 3-rd argument: check if it's NOSIGN or not. - else if (args.size() == 3) - { - auto third_arg = checkAndGetLiteralArgument(args[2], "format/NOSIGN"); - /// If there is NOSIGN, add format name before structure. - if (boost::iequals(third_arg, "NOSIGN")) - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - /// s3Cluster(cluster_name, source, format, structure) or - /// s3Cluster(cluster_name, source, access_key_id, access_key_id) or - /// s3Cluster(cluster_name, source, NOSIGN, format) - /// We can distinguish them by looking at the 3-nrd argument: check if it's NOSIGN, format name or neither. - else if (args.size() == 4) - { - auto third_arg = checkAndGetLiteralArgument(args[2], "format/NOSIGN"); - if (boost::iequals(third_arg, "NOSIGN")) - { - args.push_back(structure_literal); - } - else if (third_arg == "auto" || FormatFactory::instance().getAllFormats().contains(third_arg)) - { - args.back() = structure_literal; - } - else - { - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - } - /// s3Cluster(cluster_name, source, format, structure, compression_method) or - /// s3Cluster(cluster_name, source, access_key_id, access_key_id, format) or - /// s3Cluster(cluster_name, source, NOSIGN, format, structure) - /// We can distinguish them by looking at the 3-rd argument: check if it's NOSIGN, format name or neither. - else if (args.size() == 5) - { - auto third_arg = checkAndGetLiteralArgument(args[2], "format/NOSIGN"); - if (boost::iequals(third_arg, "NOSIGN")) - { - args.back() = structure_literal; - } - else if (third_arg == "auto" || FormatFactory::instance().getAllFormats().contains(third_arg)) - { - args[args.size() - 2] = structure_literal; - } - else - { - args.push_back(structure_literal); - } - } - /// s3Cluster(cluster_name, source, access_key_id, access_key_id, format, structure) or - /// s3Cluster(cluster_name, source, NOSIGN, format, structure, compression_method) - /// We can distinguish them by looking at the 3-rd argument: check if it's a NOSIGN keyword name or not. - else if (args.size() == 6) - { - auto third_arg = checkAndGetLiteralArgument(args[2], "format/NOSIGN"); - if (boost::iequals(third_arg, "NOSIGN")) - { - args[args.size() - 2] = structure_literal; - } - else - { - args.back() = structure_literal; - } - } - /// s3Cluster(cluster_name, source, access_key_id, access_key_id, format, structure, compression) - else if (args.size() == 7) - { - args[args.size() - 2] = structure_literal; - } -} - - -} diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.h b/src/Storages/addColumnsStructureToQueryWithClusterEngine.h deleted file mode 100644 index 44d85e00e24..00000000000 --- a/src/Storages/addColumnsStructureToQueryWithClusterEngine.h +++ /dev/null @@ -1,13 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/// Add structure argument for queries with urlCluster/hdfsCluster/s3CLuster table function. -void addColumnsStructureToQueryWithHDFSClusterEngine(ASTPtr & query, const String & structure); -void addColumnsStructureToQueryWithURLClusterEngine(ASTPtr & query, const String & structure); -void addColumnsStructureToQueryWithS3ClusterEngine(ASTPtr & query, const String & structure); - -} From 93dd3a8ec97ee61946faac2eeafecda196904791 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 May 2023 14:03:37 +0000 Subject: [PATCH 0257/2223] Remove not used cpp file --- src/TableFunctions/ITableFunctionCluster.cpp | 3 --- 1 file changed, 3 deletions(-) delete mode 100644 src/TableFunctions/ITableFunctionCluster.cpp diff --git a/src/TableFunctions/ITableFunctionCluster.cpp b/src/TableFunctions/ITableFunctionCluster.cpp deleted file mode 100644 index bde08294a81..00000000000 --- a/src/TableFunctions/ITableFunctionCluster.cpp +++ /dev/null @@ -1,3 +0,0 @@ -// -// Created by Павел Круглов on 12/05/2023. -// From be736956297990fa823a09af1b16f40e5c372445 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 May 2023 14:09:36 +0000 Subject: [PATCH 0258/2223] Add more tests for s3Cluster --- ...2457_s3_cluster_schema_inference.reference | 66 +++++++++++++++++++ .../02457_s3_cluster_schema_inference.sql | 13 +++- 2 files changed, 78 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference index c7ab9b4f989..8d744ba7b46 100644 --- a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference +++ b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference @@ -28,6 +28,72 @@ c3 Nullable(Int64) c1 Nullable(Int64) c2 Nullable(Int64) c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 0 0 0 0 0 0 1 2 3 diff --git a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql index d8cd3b0d34f..6182a1a222e 100644 --- a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql +++ b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql @@ -11,6 +11,12 @@ desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localh desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'auto'); desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV', 'auto'); desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV', 'auto', 'auto'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', NOSIGN); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', NOSIGN, 'TSV'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', NOSIGN, 'TSV', 'auto'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', NOSIGN, 'TSV', 'auto', 'auto'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', headers(MyCustomHeader = 'SomeValue')); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto', headers(MyCustomHeader = 'SomeValue'), 'auto'); select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv') order by c1, c2, c3; @@ -23,5 +29,10 @@ select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'auto') order by c1, c2, c3; select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV', 'auto') order by c1, c2, c3; select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV', 'auto', 'auto') order by c1, c2, c3; - +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', NOSIGN) order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', NOSIGN, 'TSV') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', NOSIGN, 'TSV', 'auto') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', NOSIGN, 'TSV', 'auto', 'auto') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', headers(MyCustomHeader = 'SomeValue')) order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV', 'auto', headers(MyCustomHeader = 'SomeValue'), 'auto') order by c1, c2, c3; From 1daa9811222c80f2d957556d32be4f3309034e4b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 12 May 2023 16:12:01 +0200 Subject: [PATCH 0259/2223] Fix special builds --- src/Formats/CapnProtoSerializer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index 00ccfc7717d..091e70da656 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -1007,7 +1007,7 @@ namespace catch (Exception & e) { e.addMessage("(while converting column {})", column_name); - throw e; + throw std::move(e); } } @@ -1015,7 +1015,7 @@ namespace { assert(builder); auto & struct_builder = assert_cast(*builder); - if (auto tuple_column = typeid_cast(column.get())) + if (auto * tuple_column = typeid_cast(column.get())) writeRow(tuple_column->getColumnsCopy(), struct_builder, row_num); else writeRow(Columns{column}, struct_builder, row_num); From d2e185d86df4f7e6a7b19134d7de2f960e61a644 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 12 May 2023 14:36:54 +0000 Subject: [PATCH 0260/2223] Analyzer: do not optimize GROUP BY keys with ROLLUP and CUBE --- src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp | 3 +++ .../queries/0_stateless/02734_optimize_group_by.reference | 8 ++++++++ tests/queries/0_stateless/02734_optimize_group_by.sql | 7 +++++++ 3 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02734_optimize_group_by.reference create mode 100644 tests/queries/0_stateless/02734_optimize_group_by.sql diff --git a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp index c97645219da..5ed52f1210b 100644 --- a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp +++ b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp @@ -38,6 +38,9 @@ public: if (!query->hasGroupBy()) return; + if (query->isGroupByWithCube() || query->isGroupByWithRollup()) + return; + auto & group_by = query->getGroupBy().getNodes(); if (query->isGroupByWithGroupingSets()) { diff --git a/tests/queries/0_stateless/02734_optimize_group_by.reference b/tests/queries/0_stateless/02734_optimize_group_by.reference new file mode 100644 index 00000000000..3f5ef03cb61 --- /dev/null +++ b/tests/queries/0_stateless/02734_optimize_group_by.reference @@ -0,0 +1,8 @@ +a b +a b +a b + b +a b +a b +a b +a b diff --git a/tests/queries/0_stateless/02734_optimize_group_by.sql b/tests/queries/0_stateless/02734_optimize_group_by.sql new file mode 100644 index 00000000000..28e86c04b0f --- /dev/null +++ b/tests/queries/0_stateless/02734_optimize_group_by.sql @@ -0,0 +1,7 @@ +SELECT 'a' AS key, 'b' as value GROUP BY key WITH CUBE SETTINGS allow_experimental_analyzer = 0; +SELECT 'a' AS key, 'b' as value GROUP BY key WITH CUBE SETTINGS allow_experimental_analyzer = 1; + +SELECT 'a' AS key, 'b' as value GROUP BY ignore(1) WITH CUBE; + +SELECT 'a' AS key, 'b' as value GROUP BY ignore(1); +SELECT 'a' AS key, 'b' as value GROUP BY key; From 792565d858341ff86174d0049240b5164f73eb5b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 12 May 2023 17:23:37 +0200 Subject: [PATCH 0261/2223] Updated to ULL --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index f6ce93b0688..2eb07d77e1b 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -620,7 +620,7 @@ public: const Settings & settings = context->getSettingsRef(); chassert(!storage->paths.empty()); - const auto max_parsing_threads = std::max(settings.max_threads / storage->paths.size(), 1UL); + const auto max_parsing_threads = std::max(settings.max_threads / storage->paths.size(), 1ULL); auto format = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, max_parsing_threads); From 24067ea977b6e4484f68efba8858ba8d0ad1cd6b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 12 May 2023 15:54:50 +0000 Subject: [PATCH 0262/2223] allow conversion for toIPv4OrDefault --- src/Functions/FunctionsCodingIP.h | 81 +++++++++++++++++++++++++++++ src/Functions/FunctionsConversion.h | 20 +++++++ 2 files changed, 101 insertions(+) diff --git a/src/Functions/FunctionsCodingIP.h b/src/Functions/FunctionsCodingIP.h index d02cc81f608..bd53fa7e043 100644 --- a/src/Functions/FunctionsCodingIP.h +++ b/src/Functions/FunctionsCodingIP.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -16,6 +17,7 @@ namespace ErrorCodes extern const int CANNOT_PARSE_IPV4; extern const int CANNOT_PARSE_IPV6; extern const int ILLEGAL_COLUMN; + extern const int CANNOT_CONVERT_TYPE; } enum class IPStringToNumExceptionMode : uint8_t @@ -296,4 +298,83 @@ ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray * null_map return col_res; } +template +ColumnPtr convertIPv6ToIPv4(ColumnPtr column, const PaddedPODArray * null_map = nullptr) +{ + const ColumnIPv6 * column_ipv6 = checkAndGetColumn(column.get()); + + if (!column_ipv6) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column type {}. Expected IPv6.", column->getName()); + + size_t column_size = column_ipv6->size(); + + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to = nullptr; + + if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + { + col_null_map_to = ColumnUInt8::create(column_size, false); + vec_null_map_to = &col_null_map_to->getData(); + } + + const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; + + auto col_res = ToColumn::create(); + auto & vec_res = col_res->getData(); + vec_res.resize(column_size); + const auto & vec_src = column_ipv6->getData(); + + for (size_t i = 0; i < vec_res.size(); ++i) + { + const uint8_t * src = reinterpret_cast(&vec_src[i]); + uint8_t * dst = reinterpret_cast(&vec_res[i]); + + if (null_map && (*null_map)[i]) + { + std::memset(dst, '\0', IPV4_BINARY_LENGTH); + if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + (*vec_null_map_to)[i] = true; + continue; + } + + if (!matchIPv6Subnet(src, ip4_cidr, 96)) + { + if constexpr (exception_mode == IPStringToNumExceptionMode::Throw) + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", column->getName()); + } + else if constexpr (exception_mode == IPStringToNumExceptionMode::Default) + { + std::memset(dst, '\0', IPV4_BINARY_LENGTH); + } + else if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + { + (*vec_null_map_to)[i] = true; + std::memset(dst, '\0', IPV4_BINARY_LENGTH); + } + continue; + } + + if constexpr (std::endian::native == std::endian::little) + { + dst[0] = src[15]; + dst[1] = src[14]; + dst[2] = src[13]; + dst[3] = src[12]; + } + else + { + dst[0] = src[12]; + dst[1] = src[13]; + dst[2] = src[14]; + dst[3] = src[15]; + } + } + + if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + return ColumnNullable::create(std::move(col_res), std::move(col_null_map_to)); + + return col_res; +} + } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 5bf59f33cb5..4d4efc84df1 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -4035,6 +4035,26 @@ private: return true; } } + else if constexpr (WhichDataType(FromDataType::type_id).isIPv6() && WhichDataType(ToDataType::type_id).isIPv4()) + { + ret = [cast_ipv4_ipv6_default_on_conversion_error_value, requested_result_is_nullable]( + ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t) + -> ColumnPtr + { + if (!WhichDataType(result_type).isIPv4()) + throw Exception( + ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); + + const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; + if (cast_ipv4_ipv6_default_on_conversion_error_value || requested_result_is_nullable) + return convertIPv6ToIPv4(arguments[0].column, null_map); + else + return convertIPv6ToIPv4(arguments[0].column, null_map); + }; + + return true; + } + if constexpr (WhichDataType(ToDataType::type_id).isStringOrFixedString()) { if (from_type->getCustomSerialization()) From d3c408aaf16356f59f91e95dac0b628e858f4640 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 12 May 2023 16:32:23 +0000 Subject: [PATCH 0263/2223] ClearableHashSet w/o zero storage for StringRef --- src/Common/HashTable/ClearableHashSet.h | 38 ----------------------- src/Common/HashTable/HashTableKeyHolder.h | 2 +- 2 files changed, 1 insertion(+), 39 deletions(-) diff --git a/src/Common/HashTable/ClearableHashSet.h b/src/Common/HashTable/ClearableHashSet.h index 006d45df7cd..13067c39d1b 100644 --- a/src/Common/HashTable/ClearableHashSet.h +++ b/src/Common/HashTable/ClearableHashSet.h @@ -48,30 +48,6 @@ struct ClearableHashTableCell : public BaseCell ClearableHashTableCell(const Key & key_, const State & state) : BaseCell(key_, state), version(state.version) {} }; -using StringRefBaseCell = HashSetCellWithSavedHash, ClearableHashSetState>; - -/// specialization for StringRef to allow zero size key (empty string) -template <> -struct ClearableHashTableCell : public StringRefBaseCell -{ - using State = ClearableHashSetState; - using value_type = typename StringRefBaseCell::value_type; - - UInt32 version; - - bool isZero(const State & state) const { return version != state.version; } - static bool isZero(const StringRef & key_, const State & state_) { return StringRefBaseCell::isZero(key_, state_); } - - /// Set the key value to zero. - void setZero() { version = 0; } - - /// Do I need to store the zero key separately (that is, can a zero key be inserted into the hash table). - static constexpr bool need_zero_value_storage = true; - - ClearableHashTableCell() { } /// NOLINT - ClearableHashTableCell(const StringRef & key_, const State & state) : StringRefBaseCell(key_, state), version(state.version) { } -}; - template < typename Key, typename Hash = DefaultHash, @@ -90,13 +66,6 @@ public: { ++this->version; this->m_size = 0; - - if constexpr (Cell::need_zero_value_storage) - { - /// clear ZeroValueStorage - if (this->hasZero()) - this->clearHasZero(); - } } }; @@ -119,13 +88,6 @@ public: { ++this->version; this->m_size = 0; - - if constexpr (Cell::need_zero_value_storage) - { - /// clear ZeroValueStorage - if (this->hasZero()) - this->clearHasZero(); - } } }; diff --git a/src/Common/HashTable/HashTableKeyHolder.h b/src/Common/HashTable/HashTableKeyHolder.h index d10ad093af6..994587e67ec 100644 --- a/src/Common/HashTable/HashTableKeyHolder.h +++ b/src/Common/HashTable/HashTableKeyHolder.h @@ -89,7 +89,7 @@ inline StringRef & ALWAYS_INLINE keyHolderGetKey(DB::ArenaKeyHolder & holder) inline void ALWAYS_INLINE keyHolderPersistKey(DB::ArenaKeyHolder & holder) { // Hash table shouldn't ask us to persist a zero key - assert(holder.key.size > 0); + // assert(holder.key.size > 0); holder.key.data = holder.pool.insert(holder.key.data, holder.key.size); } From 7698776d2aae48880e45d8a791b7474b5ef20999 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 12 May 2023 18:53:51 +0200 Subject: [PATCH 0264/2223] Fix special build --- src/Formats/CapnProtoSchema.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Formats/CapnProtoSchema.cpp b/src/Formats/CapnProtoSchema.cpp index 22518d5061a..f9ab88d39ed 100644 --- a/src/Formats/CapnProtoSchema.cpp +++ b/src/Formats/CapnProtoSchema.cpp @@ -151,7 +151,7 @@ namespace { template - static DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) + DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) { std::vector> values; for (auto enumerant : enumerants) @@ -159,7 +159,7 @@ namespace return std::make_shared>(std::move(values)); } - static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) + DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) { auto enumerants = enum_schema.getEnumerants(); if (enumerants.size() < 128) @@ -170,7 +170,7 @@ namespace throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "ClickHouse supports only 8 and 16-bit Enums"); } - static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) + DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) { switch (capnp_type.which()) { From c9f90fb9adba16ad1131a16d46a6efefdd532379 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 12 May 2023 18:54:38 +0200 Subject: [PATCH 0265/2223] Fix special build --- src/Formats/CapnProtoSerializer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index 091e70da656..ff3880976c7 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -158,7 +158,7 @@ namespace }; template - static std::unique_ptr createIntegerSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + std::unique_ptr createIntegerSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) { switch (capnp_type.which()) { @@ -1015,7 +1015,7 @@ namespace { assert(builder); auto & struct_builder = assert_cast(*builder); - if (auto * tuple_column = typeid_cast(column.get())) + if (const auto * tuple_column = typeid_cast(column.get())) writeRow(tuple_column->getColumnsCopy(), struct_builder, row_num); else writeRow(Columns{column}, struct_builder, row_num); From 51c69b6fbdefa577a5e8ee10819fc5ec91d6aa9b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 12 May 2023 17:36:35 +0000 Subject: [PATCH 0266/2223] Comments regarding zero values in clearable hash table --- src/Common/HashTable/ClearableHashSet.h | 4 ++++ src/Common/HashTable/HashTableKeyHolder.h | 6 +++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Common/HashTable/ClearableHashSet.h b/src/Common/HashTable/ClearableHashSet.h index 13067c39d1b..657d388cbfb 100644 --- a/src/Common/HashTable/ClearableHashSet.h +++ b/src/Common/HashTable/ClearableHashSet.h @@ -10,6 +10,10 @@ * Instead of this class, you could just use the pair (version, key) in the HashSet as the key * but then the table would accumulate all the keys that it ever stored, and it was unreasonably growing. * This class goes a step further and considers the keys with the old version empty in the hash table. + * + * Zero values note: + * A cell in ClearableHashSet can store a zero values as normal value + * If its version is equal to the version of the set itself, then it's not considered as empty even key's value is zero value of the corresponding type */ diff --git a/src/Common/HashTable/HashTableKeyHolder.h b/src/Common/HashTable/HashTableKeyHolder.h index 994587e67ec..d1b38744255 100644 --- a/src/Common/HashTable/HashTableKeyHolder.h +++ b/src/Common/HashTable/HashTableKeyHolder.h @@ -88,7 +88,11 @@ inline StringRef & ALWAYS_INLINE keyHolderGetKey(DB::ArenaKeyHolder & holder) inline void ALWAYS_INLINE keyHolderPersistKey(DB::ArenaKeyHolder & holder) { - // Hash table shouldn't ask us to persist a zero key + // Normally, our hash table shouldn't ask us to persist a zero key, + // but it can happened in the case of clearable hash table (ClearableHashSet, for example). + // The clearable hash table doesn't use zero storage and + // distinguishes empty keys by using cell version, not the value itself. + // So, when an empty StringRef is inserted in ClearableHashSet we'll get here key of zero size. // assert(holder.key.size > 0); holder.key.data = holder.pool.insert(holder.key.data, holder.key.size); } From 602b9a740ec2bf3064d2970c54fbc92da9304991 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 12 May 2023 19:39:33 +0000 Subject: [PATCH 0267/2223] Make better, allow generateRandom without structure argument --- .../sql-reference/table-functions/generate.md | 30 +- ...pp => FunctionGenerateRandomStructure.cpp} | 402 ++++++++++-------- .../FunctionGenerateRandomStructure.h | 45 ++ .../TableFunctionGenerateRandom.cpp | 67 ++- .../02586_generate_random_structure.reference | 16 +- .../02586_generate_random_structure.sql | 6 + 6 files changed, 369 insertions(+), 197 deletions(-) rename src/Functions/{generateRandomStructure.cpp => FunctionGenerateRandomStructure.cpp} (51%) create mode 100644 src/Functions/FunctionGenerateRandomStructure.h diff --git a/docs/en/sql-reference/table-functions/generate.md b/docs/en/sql-reference/table-functions/generate.md index b2777418e4f..6ceeb63cbb3 100644 --- a/docs/en/sql-reference/table-functions/generate.md +++ b/docs/en/sql-reference/table-functions/generate.md @@ -11,7 +11,7 @@ Allows to populate test tables with data. Not all types are supported. ``` sql -generateRandom('name TypeName[, name TypeName]...', [, 'random_seed'[, 'max_string_length'[, 'max_array_length']]]) +generateRandom(['name TypeName[, name TypeName]...', [, 'random_seed'[, 'max_string_length'[, 'max_array_length']]]]) ``` **Arguments** @@ -67,5 +67,33 @@ SELECT * FROM generateRandom(generateRandomStructure(4, 101), 101) LIMIT 3; └─────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────┘ ``` +With missing `structure` argument (in this case the structure is random): + +```sql +SELECT * FROM generateRandom() LIMIT 3; +``` + +```text +┌───c1─┬─────────c2─┬─────────────────────c3─┬──────────────────────c4─┬─c5───────┐ +│ -128 │ 317300854 │ 2030-08-16 08:22:20.65 │ 1994-08-16 12:08:56.745 │ R0qgiC46 │ +│ 40 │ -744906827 │ 2059-04-16 06:31:36.98 │ 1975-07-16 16:28:43.893 │ PuH4M*MZ │ +│ -55 │ 698652232 │ 2052-08-04 20:13:39.68 │ 1998-09-20 03:48:29.279 │ │ +└──────┴────────────┴────────────────────────┴─────────────────────────┴──────────┘ +``` + +With random seed both for random structure and random data: + +```sql +SELECT * FROM generateRandom(11) LIMIT 3; +``` + +```text +┌───────────────────────────────────────c1─┬─────────────────────────────────────────────────────────────────────────────c2─┬─────────────────────────────────────────────────────────────────────────────c3─┬─────────c4─┬─────────────────────────────────────────────────────────────────────────────c5─┬──────────────────────c6─┬─c7──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─c8──────────────────────────────────────┬─────────c9─┐ +│ -77422512305044606600216318673365695785 │ 636812099959807642229.503817849012019401335326013846687285151335352272727523 │ -34944452809785978175157829109276115789694605299387223845886143311647505037529 │ 544473976 │ 111220388331710079615337037674887514156741572807049614590010583571763691328563 │ 22016.22623506465 │ {'2052-01-31 20:25:33':4306400876908509081044405485378623663,'1993-04-16 15:58:49':164367354809499452887861212674772770279,'2101-08-19 03:07:18':-60676948945963385477105077735447194811,'2039-12-22 22:31:39':-59227773536703059515222628111999932330} │ a7b2:8f58:4d07:6707:4189:80cf:92f5:902d │ 1950-07-14 │ +│ -159940486888657488786004075627859832441 │ 629206527868163085099.8195700356331771569105231840157308480121506729741348442 │ -53203761250367440823323469081755775164053964440214841464405368882783634063735 │ 2187136525 │ 94881662451116595672491944222189810087991610568040618106057495823910493624275 │ 1.3095786748458954e-104 │ {} │ a051:e3da:2e0a:c69:7835:aed6:e8b:3817 │ 1943-03-25 │ +│ -5239084224358020595591895205940528518 │ -529937657954363597180.1709207212648004850138812370209091520162977548101577846 │ 47490343304582536176125359129223180987770215457970451211489086575421345731671 │ 1637451978 │ 101899445785010192893461828129714741298630410942962837910400961787305271699002 │ 2.4344456058391296e223 │ {'2013-12-22 17:42:43':80271108282641375975566414544777036006,'2041-03-08 10:28:17':169706054082247533128707458270535852845,'1986-08-31 23:07:38':-54371542820364299444195390357730624136,'2094-04-23 21:26:50':7944954483303909347454597499139023465} │ 1293:a726:e899:9bfc:8c6f:2aa1:22c9:b635 │ 1924-11-20 │ +└──────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────┴────────────┴────────────────────────────────────────────────────────────────────────────────┴─────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────┴────────────┘ +``` + ## Related content - Blog: [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse) diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp similarity index 51% rename from src/Functions/generateRandomStructure.cpp rename to src/Functions/FunctionGenerateRandomStructure.cpp index eccccdf563f..4cf783212cb 100644 --- a/src/Functions/generateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -1,9 +1,10 @@ -#include -#include -#include +#include #include #include #include +#include +#include +#include #include #include @@ -19,10 +20,18 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -class FunctionGenerateRandomStructure : public IFunction +namespace { -private: - static constexpr std::array simple_types + const size_t MAX_NUMBER_OF_COLUMNS = 128; + const size_t MAX_TUPLE_ELEMENTS = 16; + const size_t MAX_DATETIME64_PRECISION = 9; + const size_t MAX_DECIMAL32_PRECISION = 9; + const size_t MAX_DECIMAL64_PRECISION = 18; + const size_t MAX_DECIMAL128_PRECISION = 38; + const size_t MAX_DECIMAL256_PRECISION = 76; + const size_t MAX_DEPTH = 32; + + constexpr std::array simple_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -55,7 +64,7 @@ private: TypeIndex::UUID, }; - static constexpr std::array complex_types + constexpr std::array complex_types { TypeIndex::Nullable, TypeIndex::LowCardinality, @@ -64,7 +73,7 @@ private: TypeIndex::Map, }; - static constexpr std::array map_key_types + constexpr std::array map_key_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -84,119 +93,147 @@ private: TypeIndex::String, TypeIndex::FixedString, TypeIndex::IPv4, + TypeIndex::Enum8, + TypeIndex::Enum16, + TypeIndex::UUID, + TypeIndex::LowCardinality, + }; + + constexpr std::array suspicious_lc_types + { + TypeIndex::Int8, + TypeIndex::UInt8, + TypeIndex::Int16, + TypeIndex::UInt16, + TypeIndex::Int32, + TypeIndex::UInt32, + TypeIndex::Int64, + TypeIndex::UInt64, + TypeIndex::Int128, + TypeIndex::UInt128, + TypeIndex::Int256, + TypeIndex::UInt256, + TypeIndex::Float32, + TypeIndex::Float64, + TypeIndex::Date, + TypeIndex::Date32, + TypeIndex::DateTime, + TypeIndex::String, + TypeIndex::FixedString, + TypeIndex::IPv4, + TypeIndex::IPv6, TypeIndex::UUID, }; - static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; - static constexpr size_t MAX_TUPLE_ELEMENTS = 16; - static constexpr size_t MAX_DATETIME64_PRECISION = 9; - static constexpr size_t MAX_DECIMAL32_PRECISION = 9; - static constexpr size_t MAX_DECIMAL64_PRECISION = 18; - static constexpr size_t MAX_DECIMAL128_PRECISION = 38; - static constexpr size_t MAX_DECIMAL256_PRECISION = 76; - static constexpr size_t MAX_DEPTH = 32; - -public: - static constexpr auto name = "generateRandomStructure"; - - static FunctionPtr create(ContextPtr /*context*/) + template + constexpr auto getAllTypes() { - return std::make_shared(); + constexpr size_t complex_types_size = complex_types.size() * allow_complex_types; + constexpr size_t result_size = simple_types.size() + complex_types_size; + std::array result; + size_t index = 0; + + for (size_t i = 0; i != simple_types.size(); ++i, ++index) + result[index] = simple_types[i]; + + for (size_t i = 0; i != complex_types_size; ++i, ++index) + result[index] = complex_types[i]; + + return result; } - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 0; } - - bool isVariadic() const override { return true; } - bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } - bool useDefaultImplementationForConstants() const override { return false; } - bool useDefaultImplementationForNulls() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() > 2) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 2", - getName(), arguments.size()); - - - for (size_t i = 0; i != 2; ++i) - { - if (arguments.size() == i) - break; - - if (!isUnsignedInteger(arguments[i]) && !arguments[i]->onlyNull()) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the {} argument of function {}, expected unsigned integer or Null", - i + 1, - arguments[i]->getName(), - getName()); - } - } - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override - { - size_t seed = randomSeed(); - size_t number_of_columns = 0; - - if (!arguments.empty() && !arguments[0].column->onlyNull()) - { - number_of_columns = arguments[0].column->getUInt(0); - if (number_of_columns > MAX_NUMBER_OF_COLUMNS) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Maximum allowed number of columns is {}, got {}", - MAX_NUMBER_OF_COLUMNS, - number_of_columns); - } - - if (arguments.size() > 1 && !arguments[1].column->onlyNull()) - seed = arguments[1].column->getUInt(0); - - pcg64 rng(seed); - if (number_of_columns == 0) - number_of_columns = generateNumberOfColumns(rng); - - auto col_res = ColumnString::create(); - auto & string_column = assert_cast(*col_res); - auto & chars = string_column.getChars(); - WriteBufferFromVector buf(chars); - for (size_t i = 0; i != number_of_columns; ++i) - { - if (i != 0) - writeCString(", ", buf); - String column_name = "c" + std::to_string(i + 1); - writeString(column_name, buf); - writeChar(' ', buf); - writeRandomType(column_name, rng, buf); - } - - buf.finalize(); - chars.push_back(0); - string_column.getOffsets().push_back(chars.size()); - return ColumnConst::create(std::move(col_res), input_rows_count); - } - -private: - - size_t generateNumberOfColumns(pcg64 & rng) const + size_t generateNumberOfColumns(pcg64 & rng) { return rng() % MAX_NUMBER_OF_COLUMNS + 1; } - template - void writeRandomType(const String & column_name, pcg64 & rng, WriteBuffer & buf, size_t depth = 0) const + void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types, bool allow_nullable = true) { + bool make_nullable = allow_nullable & rng() % 2; + if (make_nullable) + writeCString("Nullable(", buf); + + if (allow_suspicious_lc_types) + { + TypeIndex type = suspicious_lc_types[rng() % map_key_types.size()]; + if (type == TypeIndex::FixedString) + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); + else + writeString(magic_enum::enum_name(type), buf); + } + else + { + /// Support only String and FixedString. + if (rng() % 2) + writeCString("String", buf); + else + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); + } + + if (make_nullable) + writeChar(')', buf); + } + + + void writeEnumValues(const String & column_name, pcg64 & rng, WriteBuffer & buf, ssize_t max_value) + { + /// Don't generate big enums, because it will lead to really big result + /// and slowness of this function, and it can lead to `Max query size exceeded` + /// while using this function with generateRandom. + size_t num_values = rng() % 16 + 1; + std::vector values(num_values); + + /// Generate random numbers from range [-(max_value + 1), max_value - num_values + 1] + for (Int16 & x : values) + x = rng() % (2 * (max_value + 1) - num_values) - max_value - 1; + /// Make all numbers unique. + std::sort(values.begin(), values.end()); + for (size_t i = 0; i < num_values; ++i) + values[i] += i; + std::shuffle(values.begin(), values.end(), rng); + for (size_t i = 0; i != num_values; ++i) + { + if (i != 0) + writeCString(", ", buf); + writeString("'" + column_name + "V" + std::to_string(i) + "' = " + std::to_string(values[i]), buf); + } + } + + void writeMapKeyType(const String & column_name, pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types) + { + TypeIndex type = map_key_types[rng() % map_key_types.size()]; + switch (type) + { + case TypeIndex::FixedString: + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); + break; + case TypeIndex::LowCardinality: + writeCString("LowCardinality(", buf); + writeLowCardinalityNestedType(rng, buf, allow_suspicious_lc_types, false); + writeChar(')', buf); + break; + case TypeIndex::Enum8: + writeCString("Enum8(", buf); + writeEnumValues(column_name, rng, buf, INT8_MAX); + writeChar(')', buf); + break; + case TypeIndex::Enum16: + writeCString("Enum16(", buf); + writeEnumValues(column_name, rng, buf, INT16_MAX); + writeChar(')', buf); + break; + default: + writeString(magic_enum::enum_name(type), buf); + break; + } + } + + template + void writeRandomType(const String & column_name, pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types, size_t depth = 0) + { + if (allow_complex_types && depth > MAX_DEPTH) + writeRandomType(column_name, rng, buf, depth); + constexpr auto all_types = getAllTypes(); auto type = all_types[rng() % all_types.size()]; @@ -212,55 +249,55 @@ private: writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); return; case TypeIndex::DateTime64: - writeString("DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")", buf); + writeString("DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION + 1) + ")", buf); return; case TypeIndex::Decimal32: - writeString("Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")", buf); + writeString("Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION + 1) + ")", buf); return; case TypeIndex::Decimal64: - writeString("Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")", buf); + writeString("Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION + 1) + ")", buf); return; case TypeIndex::Decimal128: - writeString("Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")", buf); + writeString("Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION + 1) + ")", buf); return; case TypeIndex::Decimal256: - writeString("Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")", buf); + writeString("Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION + 1) + ")", buf); return; case TypeIndex::Enum8: writeCString("Enum8(", buf); - writeEnumValues(column_name, rng, buf); + writeEnumValues(column_name, rng, buf, INT8_MAX); writeChar(')', buf); return; case TypeIndex::Enum16: writeCString("Enum16(", buf); - writeEnumValues(column_name, rng, buf); + writeEnumValues(column_name, rng, buf, INT16_MAX); writeChar(')', buf); return; case TypeIndex::LowCardinality: writeCString("LowCardinality(", buf); - writeLowCardinalityNestedType(rng, buf); + writeLowCardinalityNestedType(rng, buf, allow_suspicious_lc_types); writeChar(')', buf); return; case TypeIndex::Nullable: { writeCString("Nullable(", buf); - writeRandomType(column_name, rng, buf, depth + 1); + writeRandomType(column_name, rng, buf, allow_suspicious_lc_types, depth + 1); writeChar(')', buf); return; } case TypeIndex::Array: { writeCString("Array(", buf); - writeRandomType(column_name, rng, buf, depth + 1); + writeRandomType(column_name, rng, buf, allow_suspicious_lc_types, depth + 1); writeChar(')', buf); return; } case TypeIndex::Map: { writeCString("Map(", buf); - writeMapKeyType(rng, buf); + writeMapKeyType(column_name, rng, buf, allow_suspicious_lc_types); writeCString(", ", buf); - writeRandomType(column_name, rng, buf, depth + 1); + writeRandomType(column_name, rng, buf, allow_suspicious_lc_types, depth + 1); writeChar(')', buf); return; } @@ -285,7 +322,7 @@ private: writeString(element_name, buf); writeChar(' ', buf); } - writeRandomType(element_name, rng, buf, depth + 1); + writeRandomType(element_name, rng, buf, allow_suspicious_lc_types, depth + 1); } writeChar(')', buf); return; @@ -296,64 +333,87 @@ private: } } - void writeMapKeyType(pcg64 & rng, WriteBuffer & buf) const + void writeRandomStructure(pcg64 & rng, size_t number_of_columns, WriteBuffer & buf, bool allow_suspicious_lc_types) { - TypeIndex type = map_key_types[rng() % map_key_types.size()]; - if (type == TypeIndex::FixedString) - writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); - else - writeString(magic_enum::enum_name(type), buf); - } - - void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf) const - { - /// Support only String and FixedString (maybe Nullable). - String nested_type; - bool make_nullable = rng() % 2; - if (make_nullable) - writeCString("Nullable(", buf); - - if (rng() % 2) - writeCString("String", buf); - else - writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); - - if (make_nullable) - writeChar(')', buf); - } - - void writeEnumValues(const String & column_name, pcg64 & rng, WriteBuffer & buf) const - { - /// Don't generate big enums, because it will lead to really big result - /// and slowness of this function, and it can lead to `Max query size exceeded` - /// while using this function with generateRandom. - ssize_t num_values = rng() % 16 + 1; - for (ssize_t i = 0; i != num_values; ++i) + for (size_t i = 0; i != number_of_columns; ++i) { if (i != 0) writeCString(", ", buf); - writeString("'" + column_name + "V" + std::to_string(i) + "' = " + std::to_string(i), buf); + String column_name = "c" + std::to_string(i + 1); + writeString(column_name, buf); + writeChar(' ', buf); + writeRandomType(column_name, rng, buf, allow_suspicious_lc_types); + } + } +} + +DataTypePtr FunctionGenerateRandomStructure::getReturnTypeImpl(const DataTypes & arguments) const +{ + if (arguments.size() > 2) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 2", + getName(), arguments.size()); + + + for (size_t i = 0; i != arguments.size(); ++i) + { + if (!isUnsignedInteger(arguments[i]) && !arguments[i]->onlyNull()) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the {} argument of function {}, expected unsigned integer or Null", + arguments[i]->getName(), + i + 1, + getName()); } } - template - static constexpr auto getAllTypes() + return std::make_shared(); +} + +ColumnPtr FunctionGenerateRandomStructure::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const +{ + size_t seed = randomSeed(); + size_t number_of_columns = 0; + + if (!arguments.empty() && !arguments[0].column->onlyNull()) { - constexpr size_t complex_types_size = complex_types.size() * allow_complex_types; - constexpr size_t result_size = simple_types.size() + complex_types_size; - std::array result; - size_t index = 0; - - for (size_t i = 0; i != simple_types.size(); ++i, ++index) - result[index] = simple_types[i]; - - for (size_t i = 0; i != complex_types_size; ++i, ++index) - result[index] = complex_types[i]; - - return result; + number_of_columns = arguments[0].column->getUInt(0); + if (number_of_columns > MAX_NUMBER_OF_COLUMNS) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Maximum allowed number of columns is {}, got {}", + MAX_NUMBER_OF_COLUMNS, + number_of_columns); } -}; + if (arguments.size() > 1 && !arguments[1].column->onlyNull()) + seed = arguments[1].column->getUInt(0); + + pcg64 rng(seed); + if (number_of_columns == 0) + number_of_columns = generateNumberOfColumns(rng); + + auto col_res = ColumnString::create(); + auto & string_column = assert_cast(*col_res); + auto & chars = string_column.getChars(); + WriteBufferFromVector buf(chars); + writeRandomStructure(rng, number_of_columns, buf, allow_suspicious_lc_types); + buf.finalize(); + chars.push_back(0); + string_column.getOffsets().push_back(chars.size()); + return ColumnConst::create(std::move(col_res), input_rows_count); +} + +String FunctionGenerateRandomStructure::generateRandomStructure(size_t seed, const ContextPtr & context) +{ + pcg64 rng(seed); + size_t number_of_columns = generateNumberOfColumns(rng); + WriteBufferFromOwnString buf; + writeRandomStructure(rng, number_of_columns, buf, context->getSettingsRef().allow_suspicious_low_cardinality_types); + return buf.str(); +} REGISTER_FUNCTION(GenerateRandomStructure) { diff --git a/src/Functions/FunctionGenerateRandomStructure.h b/src/Functions/FunctionGenerateRandomStructure.h new file mode 100644 index 00000000000..1d1bcb1a0a8 --- /dev/null +++ b/src/Functions/FunctionGenerateRandomStructure.h @@ -0,0 +1,45 @@ +#include +#include + +#include + +namespace DB +{ + +class FunctionGenerateRandomStructure : public IFunction +{ +public: + static constexpr auto name = "generateRandomStructure"; + + explicit FunctionGenerateRandomStructure(bool allow_suspicious_lc_types_) : allow_suspicious_lc_types(allow_suspicious_lc_types_) + { + } + + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context->getSettingsRef().allow_suspicious_low_cardinality_types.value); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isVariadic() const override { return true; } + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } + bool useDefaultImplementationForConstants() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override; + + static String generateRandomStructure(size_t seed, const ContextPtr & context); + +private: + bool allow_suspicious_lc_types; +}; + +} diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 12cbda334a3..a9e81c78c07 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -1,20 +1,20 @@ -#include #include -#include #include #include #include #include -#include #include #include #include +#include #include #include +#include + #include "registerTableFunctions.h" @@ -41,13 +41,21 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co if (args.empty()) return; - if (args.size() > 4) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Table function '{}' requires at most four arguments: " - " structure, [random_seed, max_string_length, max_array_length].", getName()); + /// First, check if first argument is structure or seed. + const auto * first_arg_literal = args[0]->as(); + bool first_argument_is_structure = !first_arg_literal || first_arg_literal->value.getType() == Field::Types::String; + size_t max_args = first_argument_is_structure ? 4 : 3; - /// Allow constant expression for structure argument, it can be generated using generateRandomStructure function. - args[0] = evaluateConstantExpressionAsLiteral(args[0], context); + if (args.size() > max_args) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Table function '{}' requires at most four (or three if structure is missing) arguments: " + " [structure, random_seed, max_string_length, max_array_length].", getName()); + + if (first_argument_is_structure) + { + /// Allow constant expression for structure argument, it can be generated using generateRandomStructure function. + args[0] = evaluateConstantExpressionAsLiteral(args[0], context); + } // All the arguments must be literals. for (const auto & arg : args) @@ -55,26 +63,39 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co if (!arg->as()) { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "All arguments of table function '{}' must be literals. " + "All arguments of table function '{}' except structure argument must be literals. " "Got '{}' instead", getName(), arg->formatForErrorMessage()); } } - /// Parsing first argument as table structure and creating a sample block - structure = checkAndGetLiteralArgument(args[0], "structure"); + size_t arg_index = 0; - if (args.size() >= 2) + if (first_argument_is_structure) { - const auto & literal = args[1]->as(); + /// Parsing first argument as table structure and creating a sample block + structure = checkAndGetLiteralArgument(args[arg_index], "structure"); + ++arg_index; + } + + if (args.size() >= arg_index + 1) + { + const auto & literal = args[arg_index]->as(); + ++arg_index; if (!literal.value.isNull()) random_seed = checkAndGetLiteralArgument(literal, "random_seed"); } - if (args.size() >= 3) - max_string_length = checkAndGetLiteralArgument(args[2], "max_string_length"); + if (args.size() >= arg_index + 1) + { + max_string_length = checkAndGetLiteralArgument(args[arg_index], "max_string_length"); + ++arg_index; + } - if (args.size() == 4) - max_array_length = checkAndGetLiteralArgument(args[3], "max_string_length"); + if (args.size() == arg_index + 1) + { + max_array_length = checkAndGetLiteralArgument(args[arg_index], "max_string_length"); + ++arg_index; + } } ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextPtr context) const @@ -82,11 +103,11 @@ ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextP if (structure == "auto") { if (structure_hint.empty()) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Table function '{}' was used without structure argument but structure could not be determined automatically. Please, " - "provide structure manually", - getName()); + { + auto random_structure = FunctionGenerateRandomStructure::generateRandomStructure(random_seed.value_or(randomSeed()), context); + return parseColumnsListFromString(random_structure, context); + } + return structure_hint; } diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index 65bdc530f10..bd2009830f9 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -1,4 +1,16 @@ -c1 String, c2 UInt256, c3 String, c4 Decimal128(7), c5 UInt128 +c1 String, c2 UInt256, c3 String, c4 Decimal128(8), c5 UInt128 String Const(String) -` 90465455320735604871982424534384518837533904778028808627865442405232847164685 5& -3034771008825448884614719061068.2821046 75820566154622566322847299106656624693 +` 90465455320735604871982424534384518837533904778028808627865442405232847164685 5& -303477100882544888461471906106.82821046 75820566154622566322847299106656624693 +c1 Int128 +c2 Decimal(76, 55) +c3 Int256 +c4 UInt32 +c5 UInt256 +c6 Float64 +c7 Map(DateTime, Int128) +c8 IPv6 +c9 Date32 +-77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 +-77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 +-77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':166979754159728572703419507823025932071} 8eff:8d3c:5a2c:fa5f:b2bf:2b0e:ff23:beb2 2143-03-03 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index a28c159cff5..174e2be261e 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -9,3 +9,9 @@ select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMEN select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN} select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN} +desc generateRandom(11); +select * from generateRandom(11) limit 1; +select * from generateRandom(11, 2) limit 1; +select * from generateRandom(11, 2, 2) limit 1; +select * from generateRandom(11, 2, 2, 2) limit 1; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} + From fd58eac75a036eb12d3f185d02ae28c98fd0fa3f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 13 May 2023 10:22:37 +0200 Subject: [PATCH 0268/2223] Fixed max_threads datatype issue for builds --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 2eb07d77e1b..76900bd244a 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -620,7 +620,7 @@ public: const Settings & settings = context->getSettingsRef(); chassert(!storage->paths.empty()); - const auto max_parsing_threads = std::max(settings.max_threads / storage->paths.size(), 1ULL); + const auto max_parsing_threads = std::max(settings.max_threads/ storage->paths.size(), 1UL); auto format = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, max_parsing_threads); From c28bb963b57d87b762c6f39858b8701a9bc48d87 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 13 May 2023 15:47:40 +0000 Subject: [PATCH 0269/2223] Remove -fsanitize-blacklist --- cmake/sanitize.cmake | 5 ++++- tests/msan_suppressions.txt | 0 2 files changed, 4 insertions(+), 1 deletion(-) delete mode 100644 tests/msan_suppressions.txt diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index b2fbdb256fd..3e20625dfc9 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -8,6 +8,9 @@ option (SANITIZE "Enable one of the code sanitizers" "") set (SAN_FLAGS "${SAN_FLAGS} -g -fno-omit-frame-pointer -DSANITIZER") +# It's possible to pass an ignore list to sanitizers (-fsanitize-ignorelist). Intentionally not doing this because +# 1. out-of-source suppressions are awkward 2. it seems ignore lists don't work after the Clang v16 upgrade (#49829) + if (SANITIZE) if (SANITIZE STREQUAL "address") set (ASAN_FLAGS "-fsanitize=address -fsanitize-address-use-after-scope") @@ -29,7 +32,7 @@ if (SANITIZE) # Linking can fail due to relocation overflows (see #49145), caused by too big object files / libraries. # Work around this with position-independent builds (-fPIC and -fpie), this is slightly slower than non-PIC/PIE but that's okay. - set (MSAN_FLAGS "-fsanitize=memory -fsanitize-memory-use-after-dtor -fsanitize-memory-track-origins -fno-optimize-sibling-calls -fPIC -fpie -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/msan_suppressions.txt") + set (MSAN_FLAGS "-fsanitize=memory -fsanitize-memory-use-after-dtor -fsanitize-memory-track-origins -fno-optimize-sibling-calls -fPIC -fpie") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}") diff --git a/tests/msan_suppressions.txt b/tests/msan_suppressions.txt deleted file mode 100644 index e69de29bb2d..00000000000 From 9fb09469f003caf704ca1548a1039c6c98a2feb6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 13 May 2023 20:38:05 +0000 Subject: [PATCH 0270/2223] Fix expected results --- .../0_stateless/02734_entropy_learned_hashing.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference b/tests/queries/0_stateless/02734_entropy_learned_hashing.reference index 2240da7fea4..d9b014f3745 100644 --- a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference +++ b/tests/queries/0_stateless/02734_entropy_learned_hashing.reference @@ -7,6 +7,6 @@ 0 0 0 -2603192927274642682 -4947675599669400333 -10783339242466472992 +11160318154034397263 +11160318154034397263 +11160318154034397263 From 3355894631fea5cbe05851c2bad4e4b36f94bfca Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 13 May 2023 21:43:27 +0000 Subject: [PATCH 0271/2223] Update test (found bug in previous implementation) --- .../02730_with_fill_by_sorting_prefix.reference | 12 ++++++++++++ .../02730_with_fill_by_sorting_prefix.sql | 3 +++ 2 files changed, 15 insertions(+) diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 54a4faf46f2..26cf5200e87 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -1,4 +1,5 @@ -- { echoOn } +set use_with_fill_by_sorting_prefix=1; -- corner case with constant sort prefix SELECT number FROM numbers(1) @@ -72,3 +73,14 @@ select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 int 5 7 9999 5 8 9999 5 9 9999 +select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +1 6 9999 +1 7 9999 +1 8 9999 +1 9 9999 +1 10 1 +1 12 1 +3 5 1 +3 7 1 +5 1 1 +5 3 1 diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index 1b2288da323..3363dae35b8 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -1,4 +1,6 @@ -- { echoOn } +set use_with_fill_by_sorting_prefix=1; + -- corner case with constant sort prefix SELECT number FROM numbers(1) @@ -30,3 +32,4 @@ insert into ts VALUES (5, 1, 1), (5, 3, 1); select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); +select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; From 629a1a6668f8dc4a25106e72745cf6048181fc01 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 13 May 2023 22:11:23 +0000 Subject: [PATCH 0272/2223] Minor cleanup --- src/Interpreters/FillingRow.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 87cc91681c4..3ee3b681280 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -101,8 +101,7 @@ bool FillingRow::next(const FillingRow & to_row) if (equals(row[pos], to_row.row[pos])) { bool is_less = false; - size_t i = pos + 1; - for (; i < row_size; ++i) + for (size_t i = pos + 1; i < row_size; ++i) { const auto & fill_from = getFillDescription(i).fill_from; if (!fill_from.isNull()) From c87b33a24dde639144648ac9c648844ef4dc1c07 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Sat, 13 May 2023 22:55:27 -0700 Subject: [PATCH 0273/2223] Fix error on OS X regarding resetting timeouts. This happens when remote disconnects due to inactivity. It seems to work on Linux, likely due to difference in SO_LINGER, maybe a different default timeout on Darwin. Verified manually using clickhouse cloud using following process: 1. Connect to instance. 2. Run `show tables`. 3. Wait 6 minutes. 4. Run `show tables`. With this fix, the EINVAL is not reported, and client will simply reconnect. --- src/IO/TimeoutSetter.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/IO/TimeoutSetter.cpp b/src/IO/TimeoutSetter.cpp index 1beb26772e5..ed21383ccd4 100644 --- a/src/IO/TimeoutSetter.cpp +++ b/src/IO/TimeoutSetter.cpp @@ -31,13 +31,15 @@ TimeoutSetter::~TimeoutSetter() { try { + bool connected = socket.impl()->initialized(); + if (!connected) + return; + socket.setSendTimeout(old_send_timeout); socket.setReceiveTimeout(old_receive_timeout); } catch (...) { - /// Sometimes caught on Mac OS X. This message can be safely ignored. - /// If you are developer using Mac, please debug this error message by yourself. tryLogCurrentException("Client", "TimeoutSetter: Can't reset timeouts"); } } From 93384d6546ef0d7a5a023b40f7f4d2e411f1faea Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 14 May 2023 15:52:49 +0000 Subject: [PATCH 0274/2223] Fix suffix generation (see added test in commit 3355894) --- src/Interpreters/FillingRow.cpp | 12 ++-- src/Interpreters/FillingRow.h | 5 +- .../Transforms/FillingTransform.cpp | 61 ++++++++++++++----- src/Processors/Transforms/FillingTransform.h | 7 +++ 4 files changed, 63 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 3ee3b681280..7efbb62b0d2 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -59,7 +59,7 @@ bool FillingRow::isNull() const return true; } -bool FillingRow::next(const FillingRow & to_row) +std::pair FillingRow::next(const FillingRow & to_row) { const size_t row_size = size(); size_t pos = 0; @@ -70,7 +70,7 @@ bool FillingRow::next(const FillingRow & to_row) break; if (pos == row_size || less(to_row.row[pos], row[pos], getDirection(pos))) - return false; + return {false, false}; /// If we have any 'fill_to' value at position greater than 'pos', /// we need to generate rows up to 'fill_to' value. @@ -87,7 +87,7 @@ bool FillingRow::next(const FillingRow & to_row) { row[i] = next_value; initFromDefaults(i + 1); - return true; + return {true, true}; } } @@ -95,7 +95,7 @@ bool FillingRow::next(const FillingRow & to_row) getFillDescription(pos).step_func(next_value); if (less(to_row.row[pos], next_value, getDirection(pos)) || equals(next_value, getFillDescription(pos).fill_to)) - return false; + return {false, false}; row[pos] = next_value; if (equals(row[pos], to_row.row[pos])) @@ -111,11 +111,11 @@ bool FillingRow::next(const FillingRow & to_row) is_less |= less(row[i], to_row.row[i], getDirection(i)); } - return is_less; + return {is_less, true}; } initFromDefaults(pos + 1); - return true; + return {true, true}; } void FillingRow::initFromDefaults(size_t from_pos) diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index b1af851d792..004b417542c 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -19,7 +19,10 @@ public: explicit FillingRow(const SortDescription & sort_description); /// Generates next row according to fill 'from', 'to' and 'step' values. - bool next(const FillingRow & to_row); + /// Return pair of boolean + /// apply - true if filling values should be inserted into result set + /// value_changed - true if filling row value was changed + std::pair next(const FillingRow & to_row); void initFromDefaults(size_t from_pos = 0); diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 5b3200ac89e..e4bd689471a 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -337,8 +337,11 @@ void FillingTransform::interpolate(const MutableColumns & result_columns, Block using MutableColumnRawPtrs = std::vector; -static void insertFromFillingRow(const MutableColumnRawPtrs & filling_columns, const MutableColumnRawPtrs & interpolate_columns, const MutableColumnRawPtrs & other_columns, - const FillingRow & filling_row, const Block & interpolate_block) +void FillingTransform::insertFromFillingRow( + const MutableColumnRawPtrs & filling_columns, + const MutableColumnRawPtrs & interpolate_columns, + const MutableColumnRawPtrs & other_columns, + const Block & interpolate_block) { logDebug("insertFromFillingRow", filling_row); @@ -364,6 +367,8 @@ static void insertFromFillingRow(const MutableColumnRawPtrs & filling_columns, c for (auto * other_column : other_columns) other_column->insertDefault(); + + filling_row_inserted = true; } static void copyRowFromColumns(const MutableColumnRawPtrs & dest, const Columns & source, size_t row_num) @@ -452,8 +457,8 @@ bool FillingTransform::generateSuffixIfNeeded( logDebug("generateSuffixIfNeeded() filling_row", filling_row); logDebug("generateSuffixIfNeeded() next_row", next_row); - /// Determines should we insert filling row before start generating next rows. - bool should_insert_first = next_row < filling_row || next_row.isNull(); + /// Determines if we should insert filling row before start generating next rows + bool should_insert_first = (next_row < filling_row && !filling_row_inserted) || next_row.isNull(); logDebug("should_insert_first", should_insert_first); for (size_t i = 0, size = filling_row.size(); i < size; ++i) @@ -471,7 +476,7 @@ bool FillingTransform::generateSuffixIfNeeded( if (should_insert_first && filling_row < next_row) { interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); /// fulfill sort prefix columns with last row values or defaults if (!last_range_sort_prefix.empty()) copyRowFromColumns(res_sort_prefix_columns, last_range_sort_prefix, 0); @@ -480,10 +485,16 @@ bool FillingTransform::generateSuffixIfNeeded( sort_prefix_column->insertDefault(); } - while (filling_row.next(next_row)) + bool filling_row_changed = false; + while (true) { + const auto [apply, changed] = filling_row.next(next_row); + filling_row_changed = changed; + if (!apply) + break; + interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); /// fulfill sort prefix columns with last row values or defaults if (!last_range_sort_prefix.empty()) copyRowFromColumns(res_sort_prefix_columns, last_range_sort_prefix, 0); @@ -491,6 +502,9 @@ bool FillingTransform::generateSuffixIfNeeded( for (auto * sort_prefix_column : res_sort_prefix_columns) sort_prefix_column->insertDefault(); } + /// new valid filling row was generated but not inserted + if (filling_row_changed) + filling_row_inserted = false; return true; } @@ -546,18 +560,20 @@ void FillingTransform::transformRange( Block interpolate_block; if (new_sorting_prefix) { + logDebug("--- new range ---", range_end); for (size_t i = 0, size = filling_row.size(); i < size; ++i) { - auto current_value = (*input_fill_columns[i])[range_begin]; + const auto current_value = (*input_fill_columns[i])[range_begin]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; if (!fill_from.isNull() && !equals(current_value, fill_from)) { filling_row.initFromDefaults(i); + filling_row_inserted = false; if (less(fill_from, current_value, filling_row.getDirection(i))) { interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, range_begin); } break; @@ -577,7 +593,7 @@ void FillingTransform::transformRange( for (size_t i = 0, size = filling_row.size(); i < size; ++i) { - auto current_value = (*input_fill_columns[i])[row_ind]; + const auto current_value = (*input_fill_columns[i])[row_ind]; const auto & fill_to = filling_row.getFillDescription(i).fill_to; if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i))) @@ -587,21 +603,33 @@ void FillingTransform::transformRange( } logDebug("next_row updated", next_row); - /// A case, when at previous step row was initialized from defaults 'fill_from' values - /// and probably we need to insert it to block. + /// The condition is true when filling row is initialized by value(s) in FILL FROM, + /// and there are row(s) in current range with value(s) < then in the filling row. + /// It can happen only once for a range. if (should_insert_first && filling_row < next_row) { interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } - while (filling_row.next(next_row)) + bool filling_row_changed = false; + while(true) { + const auto [apply, changed] = filling_row.next(next_row); + filling_row_changed = changed; + if (!apply) + break; + interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } + /// new valid filling row was generated but not inserted, will use it during suffix generation + if (filling_row_changed) + filling_row_inserted = false; + + logDebug("filling_row after", filling_row); copyRowFromColumns(res_fill_columns, input_fill_columns, row_ind); copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); @@ -651,7 +679,10 @@ void FillingTransform::transform(Chunk & chunk) /// if no data was processed, then need to initialize filling_row if (last_row.empty()) + { filling_row.initFromDefaults(); + filling_row_inserted = false; + } if (generateSuffixIfNeeded(input.getHeader().getColumns(), result_columns)) { diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 2a4da05ae29..85da544ef2d 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -70,6 +70,12 @@ private: MutableColumnRawPtrs res_other_columns); bool generateSuffixIfNeeded(const Columns & input_columns, MutableColumns & result_columns); + void insertFromFillingRow( + const MutableColumnRawPtrs & filling_columns, + const MutableColumnRawPtrs & interpolate_columns, + const MutableColumnRawPtrs & other_columns, + const Block & interpolate_block); + const SortDescription sort_description; const SortDescription fill_description; /// Contains only columns with WITH FILL. SortDescription sort_prefix; @@ -77,6 +83,7 @@ private: FillingRow filling_row; /// Current row, which is used to fill gaps. FillingRow next_row; /// Row to which we need to generate filling rows. + bool filling_row_inserted = false; using Positions = std::vector; Positions fill_column_positions; From b2dddf7cf89e9b31c88b58b4cb78542432d71773 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 14 May 2023 16:06:35 +0000 Subject: [PATCH 0275/2223] Fix style --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index e4bd689471a..93bec15bae7 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -614,7 +614,7 @@ void FillingTransform::transformRange( } bool filling_row_changed = false; - while(true) + while (true) { const auto [apply, changed] = filling_row.next(next_row); filling_row_changed = changed; From de49a33cb33a9265a3c431bb9ea7d359bc4f9b27 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 14 May 2023 17:36:56 +0000 Subject: [PATCH 0276/2223] Fix tests, pt. II --- .../0_stateless/02734_entropy_learned_hashing.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference b/tests/queries/0_stateless/02734_entropy_learned_hashing.reference index d9b014f3745..2240da7fea4 100644 --- a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference +++ b/tests/queries/0_stateless/02734_entropy_learned_hashing.reference @@ -7,6 +7,6 @@ 0 0 0 -11160318154034397263 -11160318154034397263 -11160318154034397263 +2603192927274642682 +4947675599669400333 +10783339242466472992 From fc857aa2dbf297d2681af16ddfbafb47739db854 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 15 May 2023 03:06:03 +0000 Subject: [PATCH 0277/2223] tests added --- .../queries/0_stateless/02234_cast_to_ip_address.reference | 4 ++++ tests/queries/0_stateless/02234_cast_to_ip_address.sql | 7 +++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/02234_cast_to_ip_address.reference b/tests/queries/0_stateless/02234_cast_to_ip_address.reference index 9023b36a9bf..fa9c6bd0f94 100644 --- a/tests/queries/0_stateless/02234_cast_to_ip_address.reference +++ b/tests/queries/0_stateless/02234_cast_to_ip_address.reference @@ -11,6 +11,10 @@ IPv4 functions 127.0.0.1 127.0.0.1 -- +1.2.3.4 +1.2.3.4 +0.0.0.0 +-- 127.0.0.1 -- 0 diff --git a/tests/queries/0_stateless/02234_cast_to_ip_address.sql b/tests/queries/0_stateless/02234_cast_to_ip_address.sql index 6c65fe86cc9..28f1afff57f 100644 --- a/tests/queries/0_stateless/02234_cast_to_ip_address.sql +++ b/tests/queries/0_stateless/02234_cast_to_ip_address.sql @@ -20,6 +20,13 @@ SELECT toIPv4OrNull('127.0.0.1'); SELECT '--'; +SELECT toIPv4(toIPv6('::ffff:1.2.3.4')); +SELECT toIPv4(toIPv6('::afff:1.2.3.4')); --{serverError CANNOT_CONVERT_TYPE} +SELECT toIPv4OrDefault(toIPv6('::ffff:1.2.3.4')); +SELECT toIPv4OrDefault(toIPv6('::afff:1.2.3.4')); + +SELECT '--'; + SELECT cast('test' , 'IPv4'); --{serverError CANNOT_PARSE_IPV4} SELECT cast('127.0.0.1' , 'IPv4'); From aa7ab1f23badfc798116e7eeece8e62a0dfcfa9b Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 15 May 2023 11:20:03 +0000 Subject: [PATCH 0278/2223] Fix comments --- .../functions/other-functions.md | 2 ++ .../FunctionGenerateRandomStructure.cpp | 25 +++++++++++-------- .../02586_generate_random_structure.reference | 1 + .../02586_generate_random_structure.sql | 3 +++ 4 files changed, 21 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index e235a3db393..20e1168ed5a 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2677,5 +2677,7 @@ Result: └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +**Note**: the maximum nesting depth of complex types (Array, Tuple, Map, Nested) is limited to 16. + This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 4cf783212cb..75455cdda85 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -29,7 +30,7 @@ namespace const size_t MAX_DECIMAL64_PRECISION = 18; const size_t MAX_DECIMAL128_PRECISION = 38; const size_t MAX_DECIMAL256_PRECISION = 76; - const size_t MAX_DEPTH = 32; + const size_t MAX_DEPTH = 16; constexpr std::array simple_types { @@ -147,15 +148,16 @@ namespace return rng() % MAX_NUMBER_OF_COLUMNS + 1; } - void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types, bool allow_nullable = true) + void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types) { - bool make_nullable = allow_nullable & rng() % 2; + bool make_nullable = rng() % 2; if (make_nullable) writeCString("Nullable(", buf); if (allow_suspicious_lc_types) { - TypeIndex type = suspicious_lc_types[rng() % map_key_types.size()]; + TypeIndex type = suspicious_lc_types[rng() % suspicious_lc_types.size()]; + if (type == TypeIndex::FixedString) writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); else @@ -174,7 +176,6 @@ namespace writeChar(')', buf); } - void writeEnumValues(const String & column_name, pcg64 & rng, WriteBuffer & buf, ssize_t max_value) { /// Don't generate big enums, because it will lead to really big result @@ -183,9 +184,9 @@ namespace size_t num_values = rng() % 16 + 1; std::vector values(num_values); - /// Generate random numbers from range [-(max_value + 1), max_value - num_values + 1] + /// Generate random numbers from range [-(max_value + 1), max_value - num_values + 1]. for (Int16 & x : values) - x = rng() % (2 * (max_value + 1) - num_values) - max_value - 1; + x = rng() % (2 * max_value + 3 - num_values) - max_value - 1; /// Make all numbers unique. std::sort(values.begin(), values.end()); for (size_t i = 0; i < num_values; ++i) @@ -199,7 +200,7 @@ namespace } } - void writeMapKeyType(const String & column_name, pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types) + void writeMapKeyType(const String & column_name, pcg64 & rng, WriteBuffer & buf) { TypeIndex type = map_key_types[rng() % map_key_types.size()]; switch (type) @@ -209,7 +210,11 @@ namespace break; case TypeIndex::LowCardinality: writeCString("LowCardinality(", buf); - writeLowCardinalityNestedType(rng, buf, allow_suspicious_lc_types, false); + /// Map key supports only String and FixedString inside LowCardinality. + if (rng() % 2) + writeCString("String", buf); + else + writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf); writeChar(')', buf); break; case TypeIndex::Enum8: @@ -295,7 +300,7 @@ namespace case TypeIndex::Map: { writeCString("Map(", buf); - writeMapKeyType(column_name, rng, buf, allow_suspicious_lc_types); + writeMapKeyType(column_name, rng, buf); writeCString(", ", buf); writeRandomType(column_name, rng, buf, allow_suspicious_lc_types, depth + 1); writeChar(')', buf); diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index bd2009830f9..e6e2c73ad87 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -14,3 +14,4 @@ c9 Date32 -77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 -77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 -77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':166979754159728572703419507823025932071} 8eff:8d3c:5a2c:fa5f:b2bf:2b0e:ff23:beb2 2143-03-03 +c1 LowCardinality(Nullable(UInt64)), c2 Date32, c3 LowCardinality(Nullable(Float64)), c4 Int256, c5 Date32 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index 174e2be261e..b9cec1a436a 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -15,3 +15,6 @@ select * from generateRandom(11, 2) limit 1; select * from generateRandom(11, 2, 2) limit 1; select * from generateRandom(11, 2, 2, 2) limit 1; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +set allow_suspicious_low_cardinality_types=1; +select generateRandomStructure(5, 4); + From eef0a433e57598405009cf3f859da767f7415972 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 15 May 2023 11:24:18 +0000 Subject: [PATCH 0279/2223] Add note about possible huge output --- docs/en/sql-reference/table-functions/generate.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/table-functions/generate.md b/docs/en/sql-reference/table-functions/generate.md index 6ceeb63cbb3..724f6d4a1f2 100644 --- a/docs/en/sql-reference/table-functions/generate.md +++ b/docs/en/sql-reference/table-functions/generate.md @@ -95,5 +95,7 @@ SELECT * FROM generateRandom(11) LIMIT 3; └──────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────────────┴────────────┴────────────────────────────────────────────────────────────────────────────────┴─────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────┴────────────┘ ``` +**Note:** `generateRandom(generateRandomStructure(), [random seed], max_string_length, max_array_length)` with large enough `max_array_length` can generate really huge output due to possible big nesting depth (up to 16) of complex types (`Array`, `Tuple`, `Map`, `Nested`). + ## Related content - Blog: [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse) From c901d2a7be1f1791b75567f381bfddb7416c4beb Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 15 May 2023 13:46:18 +0200 Subject: [PATCH 0280/2223] Fix style --- src/Functions/FunctionGenerateRandomStructure.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/FunctionGenerateRandomStructure.h b/src/Functions/FunctionGenerateRandomStructure.h index 1d1bcb1a0a8..894096a6e07 100644 --- a/src/Functions/FunctionGenerateRandomStructure.h +++ b/src/Functions/FunctionGenerateRandomStructure.h @@ -1,3 +1,5 @@ +#pragma once + #include #include From a07db551a7f0233b74d47555b98ca46cffb8b2a3 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 15 May 2023 13:46:36 +0200 Subject: [PATCH 0281/2223] Fix style --- src/TableFunctions/TableFunctionGenerateRandom.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index a9e81c78c07..08059796660 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -26,7 +26,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int LOGICAL_ERROR; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr context) From b06e34a77fb597b53637ec9b37e0bbebbfd98a1f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 10 May 2023 20:54:59 -0300 Subject: [PATCH 0282/2223] Accept key value delimiter as part of value --- .../keyvaluepair/impl/NeedleFactory.h | 22 ++++++++++++++++++- .../keyvaluepair/impl/StateHandlerImpl.h | 18 +++++---------- ...t_key_value_pairs_multiple_input.reference | 16 ++++++++++++-- ...extract_key_value_pairs_multiple_input.sql | 14 +++++++++++- 4 files changed, 54 insertions(+), 16 deletions(-) diff --git a/src/Functions/keyvaluepair/impl/NeedleFactory.h b/src/Functions/keyvaluepair/impl/NeedleFactory.h index 007f970aedf..b5f0645abcc 100644 --- a/src/Functions/keyvaluepair/impl/NeedleFactory.h +++ b/src/Functions/keyvaluepair/impl/NeedleFactory.h @@ -38,7 +38,7 @@ public: return SearchSymbols {std::string{needles.data(), needles.size()}}; } - SearchSymbols getReadNeedles(const Configuration & extractor_configuration) + SearchSymbols getReadKeyNeedles(const Configuration & extractor_configuration) { const auto & [key_value_delimiter, quoting_character, pair_delimiters] = extractor_configuration; @@ -57,6 +57,26 @@ public: return SearchSymbols {std::string{needles.data(), needles.size()}}; } + + SearchSymbols getReadValueNeedles(const Configuration & extractor_configuration) + { + const auto & [key_value_delimiter, quoting_character, pair_delimiters] + = extractor_configuration; + + std::vector needles; + + needles.push_back(quoting_character); + + std::copy(pair_delimiters.begin(), pair_delimiters.end(), std::back_inserter(needles)); + + if constexpr (WITH_ESCAPING) + { + needles.push_back('\\'); + } + + return SearchSymbols {std::string{needles.data(), needles.size()}}; + } + SearchSymbols getReadQuotedNeedles(const Configuration & extractor_configuration) { const auto quoting_character = extractor_configuration.quoting_character; diff --git a/src/Functions/keyvaluepair/impl/StateHandlerImpl.h b/src/Functions/keyvaluepair/impl/StateHandlerImpl.h index 217a42b84ad..7fc3ba54833 100644 --- a/src/Functions/keyvaluepair/impl/StateHandlerImpl.h +++ b/src/Functions/keyvaluepair/impl/StateHandlerImpl.h @@ -41,7 +41,8 @@ public: NeedleFactory needle_factory; wait_needles = needle_factory.getWaitNeedles(configuration); - read_needles = needle_factory.getReadNeedles(configuration); + read_key_needles = needle_factory.getReadKeyNeedles(configuration); + read_value_needles = needle_factory.getReadValueNeedles(configuration); read_quoted_needles = needle_factory.getReadQuotedNeedles(configuration); } @@ -77,7 +78,7 @@ public: size_t pos = 0; - while (const auto * p = find_first_symbols_or_null({file.begin() + pos, file.end()}, read_needles)) + while (const auto * p = find_first_symbols_or_null({file.begin() + pos, file.end()}, read_key_needles)) { auto character_position = p - file.begin(); size_t next_pos = character_position + 1u; @@ -191,10 +192,6 @@ public: { return {pos + 1u, State::READING_QUOTED_VALUE}; } - else if (isKeyValueDelimiter(current_character)) - { - return {pos, State::WAITING_KEY}; - } if constexpr (WITH_ESCAPING) { @@ -218,7 +215,7 @@ public: size_t pos = 0; - while (const auto * p = find_first_symbols_or_null({file.begin() + pos, file.end()}, read_needles)) + while (const auto * p = find_first_symbols_or_null({file.begin() + pos, file.end()}, read_value_needles)) { const size_t character_position = p - file.begin(); size_t next_pos = character_position + 1u; @@ -237,10 +234,6 @@ public: } } } - else if (isKeyValueDelimiter(*p)) - { - return {next_pos, State::WAITING_KEY}; - } else if (isPairDelimiter(*p)) { value.append(file.begin() + pos, file.begin() + character_position); @@ -300,7 +293,8 @@ public: private: SearchSymbols wait_needles; - SearchSymbols read_needles; + SearchSymbols read_key_needles; + SearchSymbols read_value_needles; SearchSymbols read_quoted_needles; /* diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference index 125afa19427..e3bd1d7fc45 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference @@ -123,7 +123,7 @@ SELECT -- semi-colon as pair delimiter -- expected output: {'age':'31','name':'neymar','team':'psg'} WITH - extractKeyValuePairs('name:neymar;age:31;team:psg;invalid1:invalid1,invalid2:invalid2', ':', ';') AS s_map, + extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value', ':', ';') AS s_map, CAST( arrayMap( (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) @@ -132,7 +132,7 @@ WITH ) AS x SELECT x; -{'age':'31','name':'neymar','team':'psg'} +{'age':'31','name':'neymar','random_key':'value_with_comma,still_part_of_value:still_part_of_value','team':'psg'} -- both comma and semi-colon as pair delimiters -- expected output: {'age':'31','last_key':'last_value','name':'neymar','nationality':'brazil','team':'psg'} WITH @@ -304,6 +304,18 @@ WITH SELECT x; {} +-- key value delimiter should be considered valid part of value, char2 has a problem with == +WITH + extractKeyValuePairs('formula=1+2=3 argument1=1 argument2=2 result=3, char="=" char2== string="foo=bar"', '=') AS s_map, + CAST( + arrayMap( + (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) + ), + 'Map(String,String)' + ) AS x +SELECT + x; +{'argument1':'1','argument2':'2','char':'=','char2':'=','formula':'1+2=3','result':'3','string':'foo=bar'} -- check str_to_map alias (it is case-insensitive) WITH sTr_tO_mAp('name:neymar, age:31 team:psg,nationality:brazil') AS s_map, diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql index 0a1500b1796..578a30560c8 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql @@ -123,7 +123,7 @@ SELECT -- semi-colon as pair delimiter -- expected output: {'age':'31','name':'neymar','team':'psg'} WITH - extractKeyValuePairs('name:neymar;age:31;team:psg;invalid1:invalid1,invalid2:invalid2', ':', ';') AS s_map, + extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value', ':', ';') AS s_map, CAST( arrayMap( (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) @@ -427,6 +427,18 @@ WITH SELECT x; +-- key value delimiter should be considered valid part of value, char2 has a problem with == +WITH + extractKeyValuePairs('formula=1+2=3 argument1=1 argument2=2 result=3, char="=" char2== string="foo=bar"', '=') AS s_map, + CAST( + arrayMap( + (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) + ), + 'Map(String,String)' + ) AS x +SELECT + x; + -- check str_to_map alias (it is case-insensitive) WITH sTr_tO_mAp('name:neymar, age:31 team:psg,nationality:brazil') AS s_map, From fff0cf28b70054ca2ceb1e1f0bedd57fdf90a4f2 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 11 May 2023 09:30:09 -0300 Subject: [PATCH 0283/2223] address pr comments --- .../02499_extract_key_value_pairs_multiple_input.reference | 6 +++--- .../02499_extract_key_value_pairs_multiple_input.sql | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference index e3bd1d7fc45..ccccec10667 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference @@ -121,9 +121,9 @@ SELECT x; {} -- semi-colon as pair delimiter --- expected output: {'age':'31','name':'neymar','team':'psg'} +-- expected output: {'age':'31','anotherkey':'anothervalue','name':'neymar','random_key':'value_with_comma,still_part_of_value:still_part_of_value','team':'psg'} WITH - extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value', ':', ';') AS s_map, + extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value;anotherkey:anothervalue', ':', ';') AS s_map, CAST( arrayMap( (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) @@ -132,7 +132,7 @@ WITH ) AS x SELECT x; -{'age':'31','name':'neymar','random_key':'value_with_comma,still_part_of_value:still_part_of_value','team':'psg'} +{'age':'31','anotherkey':'anothervalue','name':'neymar','random_key':'value_with_comma,still_part_of_value:still_part_of_value','team':'psg'} -- both comma and semi-colon as pair delimiters -- expected output: {'age':'31','last_key':'last_value','name':'neymar','nationality':'brazil','team':'psg'} WITH diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql index 578a30560c8..58975bdfeec 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql @@ -121,9 +121,9 @@ SELECT x; -- semi-colon as pair delimiter --- expected output: {'age':'31','name':'neymar','team':'psg'} +-- expected output: {'age':'31','anotherkey':'anothervalue','name':'neymar','random_key':'value_with_comma,still_part_of_value:still_part_of_value','team':'psg'} WITH - extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value', ':', ';') AS s_map, + extractKeyValuePairs('name:neymar;age:31;team:psg;random_key:value_with_comma,still_part_of_value:still_part_of_value;anotherkey:anothervalue', ':', ';') AS s_map, CAST( arrayMap( (x) -> (x, s_map[x]), arraySort(mapKeys(s_map)) From c16cba48b1d3895aa29114309679c15aed9ca60e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 11 May 2023 09:32:26 -0300 Subject: [PATCH 0284/2223] remove old / debug comment in tests --- .../02499_extract_key_value_pairs_multiple_input.reference | 2 +- .../02499_extract_key_value_pairs_multiple_input.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference index ccccec10667..a56e0881471 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.reference @@ -304,7 +304,7 @@ WITH SELECT x; {} --- key value delimiter should be considered valid part of value, char2 has a problem with == +-- key value delimiter should be considered valid part of value WITH extractKeyValuePairs('formula=1+2=3 argument1=1 argument2=2 result=3, char="=" char2== string="foo=bar"', '=') AS s_map, CAST( diff --git a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql index 58975bdfeec..5a1727f91e2 100644 --- a/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql +++ b/tests/queries/0_stateless/02499_extract_key_value_pairs_multiple_input.sql @@ -427,7 +427,7 @@ WITH SELECT x; --- key value delimiter should be considered valid part of value, char2 has a problem with == +-- key value delimiter should be considered valid part of value WITH extractKeyValuePairs('formula=1+2=3 argument1=1 argument2=2 result=3, char="=" char2== string="foo=bar"', '=') AS s_map, CAST( From f2a023140e6134aca52fa1bd45e3a616f790be70 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 15:38:50 +0200 Subject: [PATCH 0285/2223] Fix processing pending batch for Distributed async INSERT after restart After abnormal server restart current_batch.txt (that contains list of files to send to the remote shard), may not have all files, if it was terminated between unlink .bin files and truncation of current_batch.txt But it should be fixed in a more reliable way, though to backport the patch I kept it simple. Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DistributedAsyncInsertBatch.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index ba5938e057d..b82cf1d7ffc 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -201,6 +201,14 @@ void DistributedAsyncInsertBatch::sendBatch() { for (const auto & file : files) { + /// In case of recovery it is possible that some of files will be + /// missing, if server had been restarted abnormally + if (recovered && !fs::exists(file)) + { + LOG_WARNING(parent.log, "File {} does not exists, likely due abnormal shutdown", file); + continue; + } + ReadBufferFromFile in(file); const auto & distributed_header = DistributedAsyncInsertHeader::read(in, parent.log); From 550d7a01bceba0fccfa948ef035466c348a38e68 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 15 May 2023 17:55:02 +0200 Subject: [PATCH 0286/2223] Try to fix build --- src/TableFunctions/ITableFunctionCluster.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/TableFunctions/ITableFunctionCluster.h b/src/TableFunctions/ITableFunctionCluster.h index f3aabc77706..e88e3921aac 100644 --- a/src/TableFunctions/ITableFunctionCluster.h +++ b/src/TableFunctions/ITableFunctionCluster.h @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -19,8 +20,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -class Context; - /// Base class for *Cluster table functions that require cluster_name for the first argument. template class ITableFunctionCluster : public Base From 362fa4849f0beccc988231ba3b107ca0868ccb16 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 15 May 2023 17:56:53 +0200 Subject: [PATCH 0287/2223] Try to fix build --- src/Functions/FunctionGenerateRandomStructure.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 75455cdda85..9d818350fac 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include @@ -431,12 +431,12 @@ the number of columns in the result structure (random by default) and random see The maximum number of columns is 128. The function returns a value of type String. )", - Documentation::Examples{ + FunctionDocumentation::Examples{ {"random", "SELECT generateRandomStructure()"}, {"with specified number of columns", "SELECT generateRandomStructure(10)"}, {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, }, - Documentation::Categories{"Random"} + FunctionDocumentation::Categories{"Random"} }, FunctionFactory::CaseSensitive); } From 05ae7b2c2d7337c2ade92634f859718061a31064 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 15 May 2023 18:28:12 +0200 Subject: [PATCH 0288/2223] fix some tests --- src/Interpreters/MutationsInterpreter.cpp | 1 + src/Interpreters/MutationsInterpreter.h | 4 +-- .../MergeTree/MutateFromLogEntryTask.cpp | 5 ++- src/Storages/MergeTree/MutateTask.cpp | 30 +++++++++++++++-- .../ReplicatedMergeTreeMutationEntry.cpp | 9 +++++ .../ReplicatedMergeTreeMutationEntry.h | 2 ++ .../MergeTree/ReplicatedMergeTreeQueue.cpp | 11 ++++--- .../MergeTree/ReplicatedMergeTreeQueue.h | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 9 +++-- tests/clickhouse-test | 8 ++--- .../test_default_compression_codec/test.py | 4 +++ .../02440_mutations_finalization.reference | 5 +++ .../02440_mutations_finalization.sql | 33 +++++++++++++++++++ ...441_alter_delete_and_drop_column.reference | 2 ++ .../02441_alter_delete_and_drop_column.sql | 20 +++++++++++ 15 files changed, 130 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/02440_mutations_finalization.reference create mode 100644 tests/queries/0_stateless/02440_mutations_finalization.sql create mode 100644 tests/queries/0_stateless/02441_alter_delete_and_drop_column.reference create mode 100644 tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 2f05ba5d7f8..2f9e647b99e 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -542,6 +542,7 @@ void MutationsInterpreter::prepare(bool dry_run) if (commands.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty mutation commands list"); + /// TODO Should we get columns, indices and projections from the part itself? Table metadata may be different const ColumnsDescription & columns_desc = metadata_snapshot->getColumns(); const IndicesDescription & indices_desc = metadata_snapshot->getSecondaryIndices(); const ProjectionsDescription & projections_desc = metadata_snapshot->getProjections(); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 4b965546408..8bb90365050 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -50,7 +50,7 @@ public: bool return_all_columns_ = false, bool return_mutated_rows_ = false); - /// Special case for MergeTree + /// Special case for *MergeTree MutationsInterpreter( MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, @@ -123,7 +123,7 @@ public: private: StoragePtr storage; - /// Special case for MergeTree. + /// Special case for *MergeTree. MergeTreeData * data = nullptr; MergeTreeData::DataPartPtr part; }; diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 35a503c658d..d0b85ee65b8 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -90,7 +90,10 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() } new_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, storage.format_version); - commands = std::make_shared(storage.queue.getMutationCommands(source_part, new_part_info.mutation)); + Strings mutation_ids; + commands = std::make_shared(storage.queue.getMutationCommands(source_part, new_part_info.mutation, mutation_ids)); + LOG_TRACE(log, "Mutating part {} with mutation commands from {} mutations ({}): {}", + entry.new_part_name, commands->size(), fmt::join(mutation_ids, ", "), commands->toString()); /// Once we mutate part, we must reserve space on the same disk, because mutations can possibly create hardlinks. /// Can throw an exception. diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a3fa210ac42..27e5319ed4f 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -58,7 +58,9 @@ static void splitAndModifyMutationCommands( MergeTreeData::DataPartPtr part, const MutationCommands & commands, MutationCommands & for_interpreter, - MutationCommands & for_file_renames) + MutationCommands & for_file_renames, + const StorageMetadataPtr & table_metadata_snapshot, + Poco::Logger * log) { auto part_columns = part->getColumnsDescription(); @@ -142,6 +144,29 @@ static void splitAndModifyMutationCommands( { if (!mutated_columns.contains(column.name)) { + if (!table_metadata_snapshot->getColumns().has(column.name)) + { + /// We cannot add the column because there's no such column in table. + /// It's okay if the column was dropped. It may also absent in dropped_columns + /// if the corresponding MUTATE_PART entry was not created yet or was created separately from current MUTATE_PART. + /// But we don't know for sure what happened. + auto part_metadata_version = part->getMetadataVersion(); + auto table_metadata_version = table_metadata_snapshot->getMetadataVersion(); + if (table_metadata_version <= part_metadata_version) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} with metadata version {} contains column {} that is absent " + "in table {} with metadata version {}", + part->name, part_metadata_version, column.name, + part->storage.getStorageID().getNameForLogs(), table_metadata_version); + + if (part_metadata_version < table_metadata_version) + { + LOG_WARNING(log, "Ignoring column {} from part {} with metadata version {} because there is no such column " + "in table {} with metadata version {}. Assuming the column was dropped", column.name, part->name, + part_metadata_version, part->storage.getStorageID().getNameForLogs(), table_metadata_version); + continue; + } + } + for_interpreter.emplace_back( MutationCommand{.type = MutationCommand::Type::READ_COLUMN, .column_name = column.name, .data_type = column.type}); } @@ -1776,7 +1801,8 @@ bool MutateTask::prepare() context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false); context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0)); - MutationHelpers::splitAndModifyMutationCommands(ctx->source_part, ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames); + MutationHelpers::splitAndModifyMutationCommands(ctx->source_part, ctx->commands_for_part, ctx->for_interpreter, + ctx->for_file_renames, ctx->metadata_snapshot, ctx->log); ctx->stage_progress = std::make_unique(1.0); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp index 1efb3f6826b..5e01cd96f6b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp @@ -99,4 +99,13 @@ std::shared_ptr ReplicatedMergeTreeMutationEntry::backup() c return std::make_shared(out.str()); } + +String ReplicatedMergeTreeMutationEntry::getBlockNumbersForLogs() const +{ + WriteBufferFromOwnString out; + for (const auto & kv : block_numbers) + out << kv.first << " = " << kv.second << "; "; + return out.str(); +} + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h index 8c359a57279..09787bd1731 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h @@ -51,6 +51,8 @@ struct ReplicatedMergeTreeMutationEntry bool isAlterMutation() const { return alter_version != -1; } std::shared_ptr backup() const; + + String getBlockNumbersForLogs() const; }; using ReplicatedMergeTreeMutationEntryPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 1762c7aabe9..9f7ae3222a4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -955,13 +955,14 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C const String & partition_id = pair.first; Int64 block_num = pair.second; mutations_by_partition[partition_id].emplace(block_num, &mutation); - LOG_TRACE(log, "Adding mutation {} for partition {} for all block numbers less than {}", entry->znode_name, partition_id, block_num); } + LOG_TRACE(log, "Adding mutation {} for {} partitions (data versions: {})", + entry->znode_name, entry->block_numbers.size(), entry->getBlockNumbersForLogs()); /// Initialize `mutation.parts_to_do`. We cannot use only current_parts + virtual_parts here so we /// traverse all the queue and build correct state of parts_to_do. auto queue_representation = getQueueRepresentation(queue, format_version); - mutation.parts_to_do = getPartNamesToMutate(*entry, virtual_parts, queue_representation, format_version); + mutation.parts_to_do = getPartNamesToMutate(*entry, current_parts, queue_representation, format_version); if (mutation.parts_to_do.size() == 0) some_mutations_are_probably_done = true; @@ -1801,7 +1802,7 @@ std::map ReplicatedMergeTreeQueue::getAlterMutationCo } MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( - const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version) const + const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const { /// NOTE: If the corresponding mutation is not found, the error is logged (and not thrown as an exception) /// to allow recovering from a mutation that cannot be executed. This way you can delete the mutation entry @@ -1840,6 +1841,8 @@ MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( MutationCommands commands; for (auto it = begin; it != end; ++it) { + chassert(mutation_pointer < it->second->entry->znode_name); + mutation_ids.push_back(it->second->entry->znode_name); const auto & commands_from_entry = it->second->entry->commands; commands.insert(commands.end(), commands_from_entry.begin(), commands_from_entry.end()); } @@ -2600,7 +2603,7 @@ void ReplicatedMergeTreeQueue::removeCurrentPartsFromMutations() { std::lock_guard state_lock(state_mutex); for (const auto & part_name : current_parts.getParts()) - removeCoveredPartsFromMutations(part_name, /*remove_part = */ true, /*remove_covered_parts = */ true); + removeCoveredPartsFromMutations(part_name, /*remove_part = */ false, /*remove_covered_parts = */ true); } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 368f2d4bc1f..3fefc341bbc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -392,7 +392,8 @@ public: /// Returns functor which used by MergeTreeMergerMutator to select parts for merge ReplicatedMergeTreeMergePredicate getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, PartitionIdsHint && partition_ids_hint); - MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version) const; + MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, + Strings & mutation_ids) const; /// Return mutation commands for part which could be not applied to /// it according to part mutation version. Used when we apply alter commands on fly, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b5e53950a02..5ca97f06cde 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5265,12 +5265,12 @@ void StorageReplicatedMergeTree::alter( fs::path(zookeeper_path) / "log/log-", alter_entry->toString(), zkutil::CreateMode::PersistentSequential)); PartitionBlockNumbersHolder partition_block_numbers_holder; + ReplicatedMergeTreeMutationEntry mutation_entry; if (have_mutation) { delayMutationOrThrowIfNeeded(&partial_shutdown_event, query_context); const String mutations_path(fs::path(zookeeper_path) / "mutations"); - ReplicatedMergeTreeMutationEntry mutation_entry; mutation_entry.alter_version = new_metadata_version; mutation_entry.source_replica = replica_name; mutation_entry.commands = std::move(maybe_mutation_commands); @@ -5322,12 +5322,16 @@ void StorageReplicatedMergeTree::alter( /// ReplicatedMergeTreeMutationEntry record in /mutations String mutation_path = dynamic_cast(*results[mutation_path_idx]).path_created; mutation_znode = mutation_path.substr(mutation_path.find_last_of('/') + 1); + LOG_DEBUG(log, "Created log entry {} to update table metadata to version {}, created a mutation {} (data versions: {})", + alter_entry->znode_name, alter_entry->alter_version, *mutation_znode, mutation_entry.getBlockNumbersForLogs()); } else { /// ALTER_METADATA record in replication /log String alter_path = dynamic_cast(*results[alter_path_idx]).path_created; alter_entry->znode_name = alter_path.substr(alter_path.find_last_of('/') + 1); + LOG_DEBUG(log, "Created log entry {} to update table metadata to version {}", + alter_entry->znode_name, alter_entry->alter_version); } break; } @@ -6493,7 +6497,8 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte const String & path_created = dynamic_cast(responses[1].get())->path_created; mutation_entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1); - LOG_TRACE(log, "Created mutation with ID {}", mutation_entry.znode_name); + LOG_TRACE(log, "Created mutation with ID {} (data versions: {})", + mutation_entry.znode_name, mutation_entry.getBlockNumbersForLogs()); break; } else if (rc == Coordination::Error::ZBADVERSION) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e279b899a93..cd9dde28fea 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -286,11 +286,11 @@ def get_processlist_with_stacktraces(args): -- NOTE: view() here to do JOIN on shards, instead of initiator FROM clusterAllReplicas('test_cluster_database_replicated', view( SELECT + p.*, arrayStringConcat(groupArray('Thread ID ' || toString(s.thread_id) || '\n' || arrayStringConcat(arrayMap( x -> concat(addressToLine(x), '::', demangle(addressToSymbol(x))), s.trace), '\n') AS stacktrace - )) AS stacktraces, - p.* + )) AS stacktraces FROM system.processes p JOIN system.stack_trace s USING (query_id) WHERE query NOT LIKE '%system.processes%' @@ -307,11 +307,11 @@ def get_processlist_with_stacktraces(args): args, """ SELECT + p.*, arrayStringConcat(groupArray('Thread ID ' || toString(s.thread_id) || '\n' || arrayStringConcat(arrayMap( x -> concat(addressToLine(x), '::', demangle(addressToSymbol(x))), s.trace), '\n') AS stacktrace - )) AS stacktraces, - p.* + )) AS stacktraces FROM system.processes p JOIN system.stack_trace s USING (query_id) WHERE query NOT LIKE '%system.processes%' diff --git a/tests/integration/test_default_compression_codec/test.py b/tests/integration/test_default_compression_codec/test.py index 5d033ac8f7e..c7c30f5eea4 100644 --- a/tests/integration/test_default_compression_codec/test.py +++ b/tests/integration/test_default_compression_codec/test.py @@ -262,6 +262,8 @@ def test_default_codec_multiple(start_cluster): ) ) + node2.query("SYSTEM SYNC REPLICA compression_table_multiple", timeout=15) + # Same codec for all assert ( get_compression_codec_byte(node1, "compression_table_multiple", "1_0_0_0") @@ -330,6 +332,8 @@ def test_default_codec_multiple(start_cluster): node1.query("OPTIMIZE TABLE compression_table_multiple FINAL") + node2.query("SYSTEM SYNC REPLICA compression_table_multiple", timeout=15) + assert ( get_compression_codec_byte(node1, "compression_table_multiple", "1_0_0_1") == CODECS_MAPPING["Multiple"] diff --git a/tests/queries/0_stateless/02440_mutations_finalization.reference b/tests/queries/0_stateless/02440_mutations_finalization.reference new file mode 100644 index 00000000000..a8b9c2acdce --- /dev/null +++ b/tests/queries/0_stateless/02440_mutations_finalization.reference @@ -0,0 +1,5 @@ +0000000000 UPDATE n = 2 WHERE n = 1 ['all_0_0_0'] 0 +1 +0000000000 UPDATE n = 2 WHERE n = 1 ['all_0_0_0'] 0 +2 +0000000000 UPDATE n = 2 WHERE n = 1 [] 1 diff --git a/tests/queries/0_stateless/02440_mutations_finalization.sql b/tests/queries/0_stateless/02440_mutations_finalization.sql new file mode 100644 index 00000000000..796dcde8e4e --- /dev/null +++ b/tests/queries/0_stateless/02440_mutations_finalization.sql @@ -0,0 +1,33 @@ + +create table mut (n int) engine=ReplicatedMergeTree('/test/02440/{database}/mut', '1') order by tuple(); +set insert_keeper_fault_injection_probability=0; +insert into mut values (1); +system stop merges mut; +alter table mut update n = 2 where n = 1; +-- it will create MUTATE_PART entry, but will not execute it + +select mutation_id, command, parts_to_do_names, is_done from system.mutations where database=currentDatabase() and table='mut'; + +-- merges (and mutations) will start again after detach/attach, we need to avoid this somehow... +create table tmp (n int) engine=MergeTree order by tuple() settings index_granularity=1; +insert into tmp select * from numbers(1000); +alter table tmp update n = sleepEachRow(1) where 1; +select sleepEachRow(2) as higher_probablility_of_reproducing_the_issue format Null; + +-- it will not execute MUTATE_PART, because another mutation is currently executing (in tmp) +alter table mut modify setting max_number_of_mutations_for_replica=1; +detach table mut; +attach table mut; + +-- mutation should not be finished yet +select * from mut; +select mutation_id, command, parts_to_do_names, is_done from system.mutations where database=currentDatabase() and table='mut'; + +alter table mut modify setting max_number_of_mutations_for_replica=100; +system sync replica mut; + +-- and now it should +select * from mut; +select mutation_id, command, parts_to_do_names, is_done from system.mutations where database=currentDatabase() and table='mut'; + +drop table tmp; -- btw, it will check that mutation can be cancelled between blocks on shutdown diff --git a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.reference b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.reference new file mode 100644 index 00000000000..e9858167301 --- /dev/null +++ b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.reference @@ -0,0 +1,2 @@ +MUTATE_PART all_0_0_0_1 ['all_0_0_0'] +1 2 diff --git a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql new file mode 100644 index 00000000000..d274fae1a4f --- /dev/null +++ b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql @@ -0,0 +1,20 @@ + +create table mut (n int, m int, k int) engine=ReplicatedMergeTree('/test/02441/{database}/mut', '1') order by n; +set insert_keeper_fault_injection_probability=0; +insert into mut values (1, 2, 3), (10, 20, 30); + +system stop merges mut; +alter table mut delete where n = 10; +alter table mut drop column k settings alter_sync=0; +system sync replica mut pull; + +-- a funny way to wait for ALTER_METADATA to disappear from the replication queue +select sleepEachRow(1) from url('http://localhost:8123/?param_tries={1..30}&query=' || encodeURLComponent( + 'select * from system.replication_queue where database=''' || currentDatabase() || ''' and table=''mut'' and type=''ALTER_METADATA''' + ), 'LineAsString', 's String') settings max_threads=1 format Null; + +select type, new_part_name, parts_to_merge from system.replication_queue where database=currentDatabase() and table='mut'; +system start merges mut; +set receive_timeout=30; +system sync replica mut; +select * from mut; From 334f062fa0d8159cb73fb4e5d869285305979ce6 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 15 May 2023 16:39:26 +0000 Subject: [PATCH 0289/2223] fix style --- src/CMakeLists.txt | 3 +-- src/Storages/HDFS/StorageHDFSCluster.cpp | 5 +++++ src/Storages/StorageS3Cluster.cpp | 5 +++++ src/Storages/StorageURLCluster.cpp | 5 +++++ src/TableFunctions/ITableFunctionDataLake.h | 4 ---- src/TableFunctions/ITableFunctionFileLike.cpp | 2 -- src/TableFunctions/TableFunctionS3.cpp | 14 ++++++++------ src/TableFunctions/TableFunctionURL.cpp | 6 ------ 8 files changed, 24 insertions(+), 20 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index c04b7acad3d..b3f4fbb7420 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -211,7 +211,6 @@ endif() if (TARGET ch_contrib::jemalloc) target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::jemalloc) endif() -target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::sparsehash) add_subdirectory(Access/Common) add_subdirectory(Common/ZooKeeper) @@ -464,7 +463,7 @@ endif () if (TARGET ch_contrib::ldap) dbms_target_link_libraries (PRIVATE ch_contrib::ldap ch_contrib::lber) endif () -dbms_target_link_libraries (PUBLIC ch_contrib::sparsehash) +dbms_target_link_libraries (PRIVATE ch_contrib::sparsehash) if (TARGET ch_contrib::protobuf) dbms_target_link_libraries (PRIVATE ch_contrib::protobuf) diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 90756d8be6d..46e67b623e2 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -29,6 +29,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + StorageHDFSCluster::StorageHDFSCluster( ContextPtr context_, const String & cluster_name_, diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index daf0af4f6e3..153a3b7f11b 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -27,6 +27,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + StorageS3Cluster::StorageS3Cluster( const String & cluster_name_, const StorageS3::Configuration & configuration_, diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 409291ec801..f652a40a561 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -28,6 +28,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + StorageURLCluster::StorageURLCluster( ContextPtr context_, const String & cluster_name_, diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 34331e86bc1..f87838cfb56 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -13,10 +13,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} template class ITableFunctionDataLake : public TableFunction diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 4b6a8f06741..9d8c36deea4 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -2,7 +2,6 @@ #include #include -#include #include @@ -19,7 +18,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; } diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 495f4fad7ce..ca87e7cd979 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -28,6 +28,8 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } @@ -369,9 +371,9 @@ void registerTableFunctionGCS(TableFunctionFactory & factory) { factory.registerFunction( {.documentation - = {.description=R"(The table function can be used to read the data stored on Google Cloud Storage.)", - .examples{{"gcs", "SELECT * FROM gcs(url, hmac_key, hmac_secret)", ""}}, - .categories{"DataLake"}}, + = {R"(The table function can be used to read the data stored on Google Cloud Storage.)", + Documentation::Examples{{"gcs", "SELECT * FROM gcs(url, hmac_key, hmac_secret)"}}, + Documentation::Categories{"DataLake"}}, .allow_readonly = false}); } @@ -379,9 +381,9 @@ void registerTableFunctionS3(TableFunctionFactory & factory) { factory.registerFunction( {.documentation - = {.description=R"(The table function can be used to read the data stored on AWS S3.)", - .examples{{"s3", "SELECT * FROM s3(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"}}, + = {R"(The table function can be used to read the data stored on AWS S3.)", + Documentation::Examples{{"s3", "SELECT * FROM s3(url, access_key_id, secret_access_key)"}}, + Documentation::Categories{"DataLake"}}, .allow_readonly = false}); } diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index b138d80a9ec..6e2675d5de1 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -12,18 +12,12 @@ #include #include #include -#include #include namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - std::vector TableFunctionURL::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const { auto & table_function_node = query_node_table_function->as(); From a551d1f042f061aba24fff9dfdc23a82c1cba194 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 15 May 2023 16:45:45 +0000 Subject: [PATCH 0290/2223] Fix build --- src/CMakeLists.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b3f4fbb7420..c04b7acad3d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -211,6 +211,7 @@ endif() if (TARGET ch_contrib::jemalloc) target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::jemalloc) endif() +target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::sparsehash) add_subdirectory(Access/Common) add_subdirectory(Common/ZooKeeper) @@ -463,7 +464,7 @@ endif () if (TARGET ch_contrib::ldap) dbms_target_link_libraries (PRIVATE ch_contrib::ldap ch_contrib::lber) endif () -dbms_target_link_libraries (PRIVATE ch_contrib::sparsehash) +dbms_target_link_libraries (PUBLIC ch_contrib::sparsehash) if (TARGET ch_contrib::protobuf) dbms_target_link_libraries (PRIVATE ch_contrib::protobuf) From 0d585746de23feee1b323b850f6c76331006e5c0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 15 May 2023 17:08:00 +0000 Subject: [PATCH 0291/2223] Fixes + disallow usage of the same column in sorting prefix and INTERPOLATE + sorting prefix is empty if use_with_fill_by_sorting_prefix is off --- .../Transforms/FillingTransform.cpp | 37 +++++++++++++------ ...2730_with_fill_by_sorting_prefix.reference | 10 +++-- .../02730_with_fill_by_sorting_prefix.sql | 3 ++ 3 files changed, 35 insertions(+), 15 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 93bec15bae7..052a5a1c183 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -239,21 +239,22 @@ FillingTransform::FillingTransform( if (!unique_positions.insert(pos).second) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Multiple WITH FILL for identical expressions is not supported in ORDER BY"); - /// build sorting prefix for first fill column - for (const auto & desc : sort_description) + if (use_with_fill_by_sorting_prefix) { - if (desc.column_name == fill_description[0].column_name) - break; + /// build sorting prefix for first fill column + for (const auto & desc : sort_description) + { + if (desc.column_name == fill_description[0].column_name) + break; - size_t pos = header_.getPositionByName(desc.column_name); - sort_prefix_positions.push_back(pos); + size_t pos = header_.getPositionByName(desc.column_name); + sort_prefix_positions.push_back(pos); - sort_prefix.push_back(desc); + sort_prefix.push_back(desc); + } + logDebug("sort prefix", dumpSortDescription(sort_prefix)); + last_range_sort_prefix.reserve(sort_prefix.size()); } - logDebug("sort prefix", dumpSortDescription(sort_prefix)); - last_range_sort_prefix.reserve(sort_prefix.size()); - - /// TODO: check conflict in positions between interpolate and sorting prefix columns size_t idx = 0; for (const ColumnWithTypeAndName & column : header_.getColumnsWithTypeAndName()) @@ -273,6 +274,20 @@ FillingTransform::FillingTransform( if (interpolate_description) for (const auto & name : interpolate_description->result_columns_order) interpolate_column_positions.push_back(header_.getPositionByName(name)); + + /// check conflict in positions between interpolate and sorting prefix columns + if (!sort_prefix_positions.empty() && !interpolate_column_positions.empty()) + { + std::unordered_set interpolate_positions(interpolate_column_positions.begin(), interpolate_column_positions.end()); + for (auto sort_prefix_pos : sort_prefix_positions) + { + if (interpolate_positions.contains(sort_prefix_pos)) + throw Exception( + ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "The same column in ORDER BY before WITH FILL (sorting prefix) and INTERPOLATE is not allowed. Column: {}", + (header_.begin() + sort_prefix_pos)->name); + } + } } /// prepare() is overrididen to call transform() after all chunks are processed diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 26cf5200e87..3e2f939af28 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -74,13 +74,15 @@ select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 int 5 8 9999 5 9 9999 select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; -1 6 9999 -1 7 9999 -1 8 9999 -1 9 9999 +0 6 9999 +0 7 9999 +0 8 9999 +0 9 9999 1 10 1 1 12 1 3 5 1 3 7 1 5 1 1 5 3 1 +-- checking that sorting prefix columns can't be used in INTERPOLATE +SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index 3363dae35b8..01fc125672a 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -33,3 +33,6 @@ select * from ts order by sensor_id, timestamp with fill step 1 settings max_blo select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; + +-- checking that sorting prefix columns can't be used in INTERPOLATE +SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } From 65bc702b0bfa2dc01c76fc1ba10007eff980fdd7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 15 May 2023 20:02:30 +0200 Subject: [PATCH 0292/2223] fix --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 27e5319ed4f..f7de6ed3d22 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -144,7 +144,7 @@ static void splitAndModifyMutationCommands( { if (!mutated_columns.contains(column.name)) { - if (!table_metadata_snapshot->getColumns().has(column.name)) + if (!table_metadata_snapshot->getColumns().has(column.name) && !part->storage.getVirtuals().contains(column.name)) { /// We cannot add the column because there's no such column in table. /// It's okay if the column was dropped. It may also absent in dropped_columns From e4a653b41cf109c7e4a885b3763a8f68f3080d89 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 15 May 2023 18:49:34 +0000 Subject: [PATCH 0293/2223] Multiple pools support for AsyncLoader --- src/Common/AsyncLoader.cpp | 218 +++++++++++++++++++++++------------- src/Common/AsyncLoader.h | 219 +++++++++++++++++++------------------ 2 files changed, 259 insertions(+), 178 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 6c19b1910eb..ccb4944b4c1 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -41,9 +42,9 @@ std::exception_ptr LoadJob::exception() const return load_exception; } -ssize_t LoadJob::priority() const +size_t LoadJob::pool() const { - return load_priority; + return pool_id; } void LoadJob::wait() const @@ -148,22 +149,35 @@ void LoadTask::remove() { loader.remove(jobs); jobs.clear(); + goal_jobs.clear(); } } void LoadTask::detach() { jobs.clear(); + goal_jobs.clear(); } -AsyncLoader::AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_, bool log_progress_) + +AsyncLoader::AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_) : log_failures(log_failures_) , log_progress(log_progress_) , log(&Poco::Logger::get("AsyncLoader")) - , max_threads(max_threads_) - , pool(metric_threads, metric_active_threads, max_threads) { - + pools.reserve(pool_initializers.size()); + for (auto && init : pool_initializers) + pools.push_back({ + .name = init.name, + .priority = init.priority, + .thread_pool = std::make_unique( + init.metric_threads, + init.metric_active_threads, + init.max_threads, + /* max_free_threads = */ 0, + init.max_threads), + .max_threads = init.max_threads + }); } AsyncLoader::~AsyncLoader() @@ -175,13 +189,20 @@ void AsyncLoader::start() { std::unique_lock lock{mutex}; is_running = true; - for (size_t i = 0; workers < max_threads && i < ready_queue.size(); i++) - spawn(lock); + updateCurrentPriorityAndSpawn(lock); } void AsyncLoader::wait() { - pool.wait(); + // Because job can create new jobs in other pools we have to recheck in cycle + std::unique_lock lock{mutex}; + while (!scheduled_jobs.empty()) + { + lock.unlock(); + for (auto & p : pools) + p.thread_pool->wait(); + lock.lock(); + } } void AsyncLoader::stop() @@ -191,7 +212,7 @@ void AsyncLoader::stop() is_running = false; // NOTE: there is no need to notify because workers never wait } - pool.wait(); + wait(); } void AsyncLoader::schedule(LoadTask & task) @@ -229,9 +250,9 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) old_jobs = finished_jobs.size(); } - // Make set of jobs to schedule: + // Pass 1. Make set of jobs to schedule: // 1) exclude already scheduled or finished jobs - // 2) include pending dependencies, that are not yet scheduled + // 2) include assigned job dependencies (that are not yet scheduled) LoadJobSet jobs; for (const auto & job : input_jobs) gatherNotScheduled(job, jobs, lock); @@ -242,17 +263,17 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) // We do not want any exception to be throws after this point, because the following code is not exception-safe DENY_ALLOCATIONS_IN_SCOPE; - // Schedule all incoming jobs + // Pass 2. Schedule all incoming jobs for (const auto & job : jobs) { NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - scheduled_jobs.emplace(job, Info{.initial_priority = job->load_priority, .priority = job->load_priority}); + scheduled_jobs.try_emplace(job); job->scheduled(); }); } - // Process dependencies on scheduled pending jobs + // Pass 3. Process dependencies on scheduled jobs, priority inheritance for (const auto & job : jobs) { Info & info = scheduled_jobs.find(job)->second; @@ -267,17 +288,18 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) }); info.dependencies_left++; - // Priority inheritance: prioritize deps to have at least given `priority` to avoid priority inversion - prioritize(dep, info.priority, lock); + // Priority inheritance: prioritize deps to have at least given `pool.priority` to avoid priority inversion + prioritize(dep, job->pool_id, lock); } } // Enqueue non-blocked jobs (w/o dependencies) to ready queue - if (!info.is_blocked()) + if (!info.isBlocked()) enqueue(info, job, lock); } - // Process dependencies on other jobs. It is done in a separate pass to facilitate propagation of cancel signals (if any). + // Pass 4: Process dependencies on other jobs. + // It is done in a separate pass to facilitate cancelling due to already failed dependencies. for (const auto & job : jobs) { if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) @@ -285,12 +307,12 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) for (const auto & dep : job->dependencies) { if (scheduled_jobs.contains(dep)) - continue; // Skip dependencies on scheduled pending jobs (already processed) + continue; // Skip dependencies on scheduled jobs (already processed in pass 3) LoadStatus dep_status = dep->status(); if (dep_status == LoadStatus::OK) continue; // Dependency on already successfully finished job -- it's okay. - // Dependency on not scheduled pending job -- it's bad. + // Dependency on assigned job -- it's bad. // Probably, there is an error in `jobs` set, `gatherNotScheduled()` should have fixed it. chassert(dep_status != LoadStatus::PENDING); @@ -305,7 +327,7 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) job->name, getExceptionMessage(dep->exception(), /* with_stacktrace = */ false))); }); - finish(lock, job, LoadStatus::CANCELED, e); + finish(job, LoadStatus::CANCELED, e, lock); break; // This job is now finished, stop its dependencies processing } } @@ -327,13 +349,13 @@ void AsyncLoader::gatherNotScheduled(const LoadJobPtr & job, LoadJobSet & jobs, } } -void AsyncLoader::prioritize(const LoadJobPtr & job, ssize_t new_priority) +void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool) { if (!job) return; DENY_ALLOCATIONS_IN_SCOPE; std::unique_lock lock{mutex}; - prioritize(job, new_priority, lock); + prioritize(job, new_pool, lock); } void AsyncLoader::remove(const LoadJobSet & jobs) @@ -347,14 +369,14 @@ void AsyncLoader::remove(const LoadJobSet & jobs) { if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) { - if (info->second.is_executing()) + if (info->second.isExecuting()) continue; // Skip executing jobs on the first pass std::exception_ptr e; NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, "Load job '{}' canceled", job->name)); }); - finish(lock, job, LoadStatus::CANCELED, e); + finish(job, LoadStatus::CANCELED, e, lock); } } // On the second pass wait for executing jobs to finish @@ -363,7 +385,7 @@ void AsyncLoader::remove(const LoadJobSet & jobs) if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) { // Job is currently executing - chassert(info->second.is_executing()); + chassert(info->second.isExecuting()); lock.unlock(); job->waitNoThrow(); // Wait for job to finish lock.lock(); @@ -379,25 +401,36 @@ void AsyncLoader::remove(const LoadJobSet & jobs) } } -void AsyncLoader::setMaxThreads(size_t value) +void AsyncLoader::setMaxThreads(size_t pool, size_t value) { std::unique_lock lock{mutex}; - pool.setMaxThreads(value); - pool.setMaxFreeThreads(value); - pool.setQueueSize(value); - max_threads = value; + auto & p = pools[pool]; + p.thread_pool->setMaxThreads(value); + p.thread_pool->setQueueSize(value); // Keep queue size equal max threads count to avoid blocking during spawning + p.max_threads = value; if (!is_running) return; - for (size_t i = 0; workers < max_threads && i < ready_queue.size(); i++) - spawn(lock); + for (size_t i = 0; canSpawnWorker(p, lock) && i < p.ready_queue.size(); i++) + spawn(p, lock); } -size_t AsyncLoader::getMaxThreads() const +size_t AsyncLoader::getMaxThreads(size_t pool) const { std::unique_lock lock{mutex}; - return max_threads; + return pools[pool].max_threads; } +const String & AsyncLoader::getPoolName(size_t pool) const +{ + return pools[pool].name; // NOTE: lock is not needed because `name` is const and `pools` are immutable +} + +ssize_t AsyncLoader::getPoolPriority(size_t pool) const +{ + return pools[pool].priority; // NOTE: lock is not needed because `priority` is const and `pools` are immutable +} + + size_t AsyncLoader::getScheduledJobCount() const { std::unique_lock lock{mutex}; @@ -412,11 +445,10 @@ std::vector AsyncLoader::getJobStates() const states.emplace(job->name, JobState{ .job = job, .dependencies_left = info.dependencies_left, - .is_executing = info.is_executing(), - .is_blocked = info.is_blocked(), - .is_ready = info.is_ready(), - .initial_priority = info.initial_priority, - .ready_seqno = last_ready_seqno + .ready_seqno = info.ready_seqno, + .is_blocked = info.isBlocked(), + .is_ready = info.isReady(), + .is_executing = info.isExecuting() }); for (const auto & job : finished_jobs) states.emplace(job->name, JobState{.job = job}); @@ -462,21 +494,21 @@ String AsyncLoader::checkCycleImpl(const LoadJobPtr & job, LoadJobSet & left, Lo return {}; } -void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job) +void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job, std::unique_lock & lock) { + chassert(scheduled_jobs.contains(job)); // Job was pending if (status == LoadStatus::OK) { // Notify waiters job->ok(); // Update dependent jobs and enqueue if ready - chassert(scheduled_jobs.contains(job)); // Job was pending for (const auto & dep : scheduled_jobs[job].dependent_jobs) { chassert(scheduled_jobs.contains(dep)); // All depended jobs must be pending Info & dep_info = scheduled_jobs[dep]; dep_info.dependencies_left--; - if (!dep_info.is_blocked()) + if (!dep_info.isBlocked()) enqueue(dep_info, dep, lock); } } @@ -488,11 +520,10 @@ void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & else if (status == LoadStatus::CANCELED) job->canceled(exception_from_job); - chassert(scheduled_jobs.contains(job)); // Job was pending Info & info = scheduled_jobs[job]; - if (info.is_ready()) + if (info.isReady()) { - ready_queue.erase(info.key()); + pools[job->pool_id].ready_queue.erase(info.ready_seqno); info.ready_seqno = 0; } @@ -512,7 +543,7 @@ void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & dep->name, getExceptionMessage(exception_from_job, /* with_stacktrace = */ false))); }); - finish(lock, dep, LoadStatus::CANCELED, e); + finish(dep, LoadStatus::CANCELED, e, lock); } // Clean dependency graph edges pointing to canceled jobs @@ -531,58 +562,98 @@ void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & }); } -void AsyncLoader::prioritize(const LoadJobPtr & job, ssize_t new_priority, std::unique_lock & lock) +void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::unique_lock & lock) { if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) { - if (info->second.priority >= new_priority) - return; // Never lower priority + Pool & old_pool = pools[job->pool_id]; + Pool & new_pool = pools[new_pool_id]; + if (old_pool.priority >= new_pool.priority) + return; // Never lower priority or change pool leaving the same priority // Update priority and push job forward through ready queue if needed - if (info->second.ready_seqno) - ready_queue.erase(info->second.key()); - info->second.priority = new_priority; - job->load_priority.store(new_priority); // Set user-facing priority (may affect executing jobs) - if (info->second.ready_seqno) + UInt64 ready_seqno = info->second.ready_seqno; + if (ready_seqno) + old_pool.ready_queue.erase(ready_seqno); + job->pool_id.store(new_pool_id); // Set user-facing pool and priority (may affect executing jobs) + if (ready_seqno) { NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - ready_queue.emplace(info->second.key(), job); + new_pool.ready_queue.emplace(ready_seqno, job); }); } // Recurse into dependencies for (const auto & dep : job->dependencies) - prioritize(dep, new_priority, lock); + prioritize(dep, new_pool_id, lock); } } void AsyncLoader::enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock) { - chassert(!info.is_blocked()); + chassert(!info.isBlocked()); chassert(info.ready_seqno == 0); info.ready_seqno = ++last_ready_seqno; + Pool & pool = pools[job->pool_id]; NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - ready_queue.emplace(info.key(), job); + pool.ready_queue.emplace(info.ready_seqno, job); }); job->enqueued(); - if (is_running && workers < max_threads) - spawn(lock); + if (canSpawnWorker(pool, lock)) + spawn(pool, lock); } -void AsyncLoader::spawn(std::unique_lock &) +bool AsyncLoader::canSpawnWorker(Pool & pool, std::unique_lock &) { - workers++; + return is_running + && !pool.ready_queue.empty() + && pool.workers < pool.max_threads + && (!current_priority || *current_priority <= pool.priority); +} + +bool AsyncLoader::canWorkerLive(Pool & pool, std::unique_lock &) +{ + return is_running + && !pool.ready_queue.empty() + && pool.workers <= pool.max_threads + && (!current_priority || *current_priority <= pool.priority); +} + +void AsyncLoader::updateCurrentPriorityAndSpawn(std::unique_lock & lock) +{ + // Find current priority. + // NOTE: We assume low number of pools, so O(N) scans are fine. + std::optional priority; + for (Pool & pool : pools) + { + if (pool.isActive() && (!priority || *priority < pool.priority)) + priority = pool.priority; + } + current_priority = priority; + + // Spawn workers in all pools with current priority + for (Pool & pool : pools) + { + for (size_t i = 0; canSpawnWorker(pool, lock) && i < pool.ready_queue.size(); i++) + spawn(pool, lock); + } +} + +void AsyncLoader::spawn(Pool & pool, std::unique_lock &) +{ + pool.workers++; + current_priority = pool.priority; // canSpawnWorker() ensures this would not decrease current_priority NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - pool.scheduleOrThrowOnError([this] { worker(); }); + pool.thread_pool->scheduleOrThrowOnError([this, &pool] { worker(pool); }); }); } -void AsyncLoader::worker() +void AsyncLoader::worker(Pool & pool) { DENY_ALLOCATIONS_IN_SCOPE; @@ -591,27 +662,28 @@ void AsyncLoader::worker() while (true) { // This is inside the loop to also reset previous thread names set inside the jobs - setThreadName("AsyncLoader"); + setThreadName(pool.name.c_str()); { std::unique_lock lock{mutex}; // Handle just executed job if (exception_from_job) - finish(lock, job, LoadStatus::FAILED, exception_from_job); + finish(job, LoadStatus::FAILED, exception_from_job, lock); else if (job) - finish(lock, job, LoadStatus::OK); + finish(job, LoadStatus::OK, {}, lock); - if (!is_running || ready_queue.empty() || workers > max_threads) + if (!canWorkerLive(pool, lock)) { - workers--; + if (--pool.workers == 0) + updateCurrentPriorityAndSpawn(lock); // It will spawn lower priority workers if needed return; } // Take next job to be executed from the ready queue - auto it = ready_queue.begin(); + auto it = pool.ready_queue.begin(); job = it->second; - ready_queue.erase(it); + pool.ready_queue.erase(it); scheduled_jobs.find(job)->second.ready_seqno = 0; // This job is no longer in the ready queue } diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index d4a3218a541..0c52c549a44 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include namespace Poco { class Logger; } @@ -46,22 +46,22 @@ class LoadJob : private boost::noncopyable { public: template - LoadJob(LoadJobSetType && dependencies_, String name_, Func && func_, ssize_t priority_ = 0) + LoadJob(LoadJobSetType && dependencies_, String name_, size_t pool_id_, Func && func_) : dependencies(std::forward(dependencies_)) , name(std::move(name_)) + , pool_id(pool_id_) , func(std::forward(func_)) - , load_priority(priority_) {} // Current job status. LoadStatus status() const; std::exception_ptr exception() const; - // Returns current value of a priority of the job. May differ from initial priority. - ssize_t priority() const; + // Returns current pool of the job. May differ from initial pool. + size_t pool() const; // Sync wait for a pending job to be finished: OK, FAILED or CANCELED status. - // Throws if job is FAILED or CANCELED. Returns or throws immediately on non-pending job. + // Throws if job is FAILED or CANCELED. Returns or throws immediately if called on non-pending job. void wait() const; // Wait for a job to reach any non PENDING status. @@ -92,8 +92,8 @@ private: void enqueued(); void execute(const LoadJobPtr & self); + std::atomic pool_id; std::function func; - std::atomic load_priority; mutable std::mutex mutex; mutable std::condition_variable finished; @@ -115,25 +115,25 @@ struct EmptyJobFunc template LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, Func && func = EmptyJobFunc()) { - return std::make_shared(std::move(dependencies), std::move(name), std::forward(func)); + return std::make_shared(std::move(dependencies), std::move(name), 0, std::forward(func)); } template LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, Func && func = EmptyJobFunc()) { - return std::make_shared(dependencies, std::move(name), std::forward(func)); + return std::make_shared(dependencies, std::move(name), 0, std::forward(func)); } template -LoadJobPtr makeLoadJob(LoadJobSet && dependencies, ssize_t priority, String name, Func && func = EmptyJobFunc()) +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, size_t pool_id, String name, Func && func = EmptyJobFunc()) { - return std::make_shared(std::move(dependencies), std::move(name), std::forward(func), priority); + return std::make_shared(std::move(dependencies), std::move(name), pool_id, std::forward(func)); } template -LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, ssize_t priority, String name, Func && func = EmptyJobFunc()) +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, Func && func = EmptyJobFunc()) { - return std::make_shared(dependencies, std::move(name), std::forward(func), priority); + return std::make_shared(dependencies, std::move(name), pool_id, std::forward(func)); } // Represents a logically connected set of LoadJobs required to achieve some goals (final LoadJob in the set). @@ -251,96 +251,112 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs & return result; } -// `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks dependencies and priorities of jobs. +// `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks job dependencies and priorities. // Basic usage example: +// // Start async_loader with two thread pools (0=bg, 1=fg): +// AsyncLoader async_loader({ +// {"BgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 1, .priority = 0} +// {"FgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 2, .priority = 1} +// }); +// +// // Create and schedule a task consisting of three jobs. Job1 has no dependencies and is run first. +// // Job2 and job3 depend on job1 and are run only after job1 completion. // auto job_func = [&] (const LoadJobPtr & self) { -// LOG_TRACE(log, "Executing load job '{}' with priority '{}'", self->name, self->priority()); +// LOG_TRACE(log, "Executing load job '{}' in pool '{}'", self->name, async_loader->getPoolName(self->pool())); // }; -// auto job1 = makeLoadJob({}, "job1", job_func); -// auto job2 = makeLoadJob({ job1 }, "job2", job_func); -// auto job3 = makeLoadJob({ job1 }, "job3", job_func); +// auto job1 = makeLoadJob({}, "job1", /* pool_id = */ 0, job_func); +// auto job2 = makeLoadJob({ job1 }, "job2", /* pool_id = */ 0, job_func); +// auto job3 = makeLoadJob({ job1 }, "job3", /* pool_id = */ 0, job_func); // auto task = makeLoadTask(async_loader, { job1, job2, job3 }); // task.schedule(); -// Here we have created and scheduled a task consisting of three jobs. Job1 has no dependencies and is run first. -// Job2 and job3 depend on job1 and are run only after job1 completion. Another thread may prioritize a job and wait for it: -// async_loader->prioritize(job3, /* priority = */ 1); // higher priority jobs are run first, default priority is zero. +// +// // Another thread may prioritize a job by changing its pool and wait for it: +// async_loader->prioritize(job3, /* pool_id = */ 1); // higher priority jobs are run first, default priority is zero. // job3->wait(); // blocks until job completion or cancellation and rethrow an exception (if any) // -// AsyncLoader tracks state of all scheduled jobs. Job lifecycle is the following: -// 1) Job is constructed with PENDING status and initial priority. The job is placed into a task. -// 2) The task is scheduled with all its jobs and their dependencies. A scheduled job may be ready (i.e. have all its dependencies finished) or blocked. -// 3a) When all dependencies are successfully executed, the job became ready. A ready job is enqueued into the ready queue. +// Every job has a pool associated with it. AsyncLoader starts every job in its thread pool. +// Each pool has a constant priority and a mutable maximum number of threads. +// Higher priority (greater `pool.priority` value) jobs are run first. +// No job with lower priority is started while there is at least one higher priority job ready or running. +// +// Job priority can be elevated (but cannot be lowered) +// (a) if either it has a dependent job with higher priority: +// in this case the priority and the pool of a dependent job is inherited during `schedule()` call; +// (b) or job was explicitly prioritized by `prioritize(job, higher_priority_pool)` call: +// this also leads to a priority inheritance for all the dependencies. +// Value stored in load job `pool_id` field is atomic and can be changed even during job execution. +// Job is, of course, not moved from its initial thread pool, but it should use `self->pool()` for +// all new jobs it create to avoid priority inversion. +// +// === IMPLEMENTATION DETAILS === +// All possible states and statuses of a job: +// .---------- scheduled ----------. +// ctor --> assigned --> blocked --> ready --> executing --> finished ------> removed --> dtor +// STATUS: '------------------ PENDING -----------------' '-- OK|FAILED|CANCELED --' +// +// AsyncLoader tracks state of all scheduled and finished jobs. Job lifecycle is the following: +// 1) A job is constructed with PENDING status and assigned to a pool. The job is placed into a task. +// 2) The task is scheduled with all its jobs and their dependencies. A scheduled job may be ready, blocked (and later executing). +// 3a) When all dependencies are successfully finished, the job became ready. A ready job is enqueued into the ready queue of its pool. // 3b) If at least one of the job dependencies is failed or canceled, then this job is canceled (with all it's dependent jobs as well). // On cancellation an ASYNC_LOAD_CANCELED exception is generated and saved inside LoadJob object. The job status is changed to CANCELED. // Exception is rethrown by any existing or new `wait()` call. The job is moved to the set of the finished jobs. -// 4) The scheduled pending ready job starts execution by a worker. The job is dequeued. Callback `job_func` is called. -// Status of an executing job is PENDING. And it is still considered as a scheduled job by AsyncLoader. -// Note that `job_func` of a CANCELED job is never executed. +// 4) The ready job starts execution by a worker. The job is dequeued. Callback `job_func` is called. +// Status of an executing job is PENDING. Note that `job_func` of a CANCELED job is never executed. // 5a) On successful execution the job status is changed to OK and all existing and new `wait()` calls finish w/o exceptions. // 5b) Any exception thrown out of `job_func` is wrapped into an ASYNC_LOAD_FAILED exception and saved inside LoadJob. // The job status is changed to FAILED. All the dependent jobs are canceled. The exception is rethrown from all existing and new `wait()` calls. // 6) The job is no longer considered as scheduled and is instead moved to the finished jobs set. This is just for introspection of the finished jobs. // 7) The task containing this job is destructed or `remove()` is explicitly called. The job is removed from the finished job set. // 8) The job is destructed. -// -// Every job has a priority associated with it. AsyncLoader runs higher priority (greater `priority` value) jobs first. Job priority can be elevated -// (a) if either it has a dependent job with higher priority (in this case priority of a dependent job is inherited); -// (b) or job was explicitly prioritized by `prioritize(job, higher_priority)` call (this also leads to a priority inheritance for all the dependencies). -// Note that to avoid priority inversion `job_func` should use `self->priority()` to schedule new jobs in AsyncLoader or any other pool. -// Value stored in load job priority field is atomic and can be increased even during job execution. -// -// When a task is scheduled it can contain dependencies on previously scheduled jobs. These jobs can have any status. If job A being scheduled depends on -// another job B that is not yet scheduled, then job B will also be scheduled (even if the task does not contain it). class AsyncLoader : private boost::noncopyable { private: - // Key of a pending job in the ready queue. - struct ReadyKey - { - ssize_t priority; // Ascending order - ssize_t initial_priority; // Ascending order - UInt64 ready_seqno; // Descending order + // Thread pool for job execution. + // Pools control the following aspects of job execution: + // 1) Concurrency: Amount of concurrently executing jobs in a pool is `max_threads`. + // 2) Priority: As long as there is executing worker with higher priority, workers with lower priorities are not started + // (although, they can finish last job started before higher priority jobs appeared) + struct Pool { + const String name; + const ssize_t priority; + std::unique_ptr thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools. + std::map ready_queue; // FIFO queue of jobs to be executed in this pool. Map is used for faster erasing. Key is `ready_seqno` + size_t max_threads; // Max number of workers to be spawn + size_t workers = 0; // Number of currently execution workers - bool operator<(const ReadyKey & rhs) const - { - if (priority > rhs.priority) - return true; - if (priority < rhs.priority) - return false; - if (initial_priority > rhs.initial_priority) - return true; - if (initial_priority < rhs.initial_priority) - return false; - return ready_seqno < rhs.ready_seqno; - } + bool isActive() const { return workers > 0 || !ready_queue.empty(); } }; // Scheduling information for a pending job. struct Info { - ssize_t initial_priority = 0; // Initial priority passed into schedule(). - ssize_t priority = 0; // Elevated priority, due to priority inheritance or prioritize(). size_t dependencies_left = 0; // Current number of dependencies on pending jobs. UInt64 ready_seqno = 0; // Zero means that job is not in ready queue. LoadJobSet dependent_jobs; // Set of jobs dependent on this job. - // Three independent states of a non-finished job. - bool is_blocked() const { return dependencies_left > 0; } - bool is_ready() const { return dependencies_left == 0 && ready_seqno > 0; } - bool is_executing() const { return dependencies_left == 0 && ready_seqno == 0; } - - // Get key of a ready job - ReadyKey key() const - { - return {.priority = priority, .initial_priority = initial_priority, .ready_seqno = ready_seqno}; - } + // Three independent states of a scheduled job. + bool isBlocked() const { return dependencies_left > 0; } + bool isReady() const { return dependencies_left == 0 && ready_seqno > 0; } + bool isExecuting() const { return dependencies_left == 0 && ready_seqno == 0; } }; public: using Metric = CurrentMetrics::Metric; - AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_, bool log_progress_); + // Helper struct for AsyncLoader construction + struct PoolInitializer + { + String name; + Metric metric_threads; + Metric metric_active_threads; + size_t max_threads; + ssize_t priority; + }; + AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_); + + // Stops AsyncLoader before destruction // WARNING: all tasks instances should be destructed before associated AsyncLoader. ~AsyncLoader(); @@ -356,10 +372,10 @@ public: // - or canceled using ~Task() or remove() later. void stop(); - // Schedule all jobs of given `task` and their dependencies (if any, not scheduled yet). - // Higher priority jobs (with greater `job->priority()` value) are executed earlier. - // All dependencies of a scheduled job inherit its priority if it is higher. This way higher priority job - // never wait for (blocked by) lower priority jobs. No priority inversion is possible. + // Schedule all jobs of given `task` and their dependencies (even if they are not in task). + // All dependencies of a scheduled job inherit its pool if it has higher priority. This way higher priority job + // never waits for (blocked by) lower priority jobs. No priority inversion is possible. + // Idempotent: multiple schedule() calls for the same job are no-op. // Note that `task` destructor ensures that all its jobs are finished (OK, FAILED or CANCELED) // and are removed from AsyncLoader, so it is thread-safe to destroy them. void schedule(LoadTask & task); @@ -369,15 +385,19 @@ public: void schedule(const std::vector & tasks); // Increase priority of a job and all its dependencies recursively. - void prioritize(const LoadJobPtr & job, ssize_t new_priority); + // Jobs from higher (than `new_pool`) priority pools are not changed. + void prioritize(const LoadJobPtr & job, size_t new_pool); // Remove finished jobs, cancel scheduled jobs, wait for executing jobs to finish and remove them. void remove(const LoadJobSet & jobs); - // Increase or decrease maximum number of simultaneously executing jobs. - void setMaxThreads(size_t value); + // Increase or decrease maximum number of simultaneously executing jobs in `pool`. + void setMaxThreads(size_t pool, size_t value); + + size_t getMaxThreads(size_t pool) const; + const String & getPoolName(size_t pool) const; + ssize_t getPoolPriority(size_t pool) const; - size_t getMaxThreads() const; size_t getScheduledJobCount() const; // Helper class for introspection @@ -385,11 +405,10 @@ public: { LoadJobPtr job; size_t dependencies_left = 0; - bool is_executing = false; + UInt64 ready_seqno = 0; bool is_blocked = false; bool is_ready = false; - std::optional initial_priority; - std::optional ready_seqno; + bool is_executing = false; }; // For introspection and debug only, see `system.async_loader` table @@ -398,42 +417,32 @@ public: private: void checkCycle(const LoadJobSet & jobs, std::unique_lock & lock); String checkCycleImpl(const LoadJobPtr & job, LoadJobSet & left, LoadJobSet & visited, std::unique_lock & lock); - void finish(std::unique_lock & lock, const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job = {}); + void finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job, std::unique_lock & lock); void scheduleImpl(const LoadJobSet & input_jobs); void gatherNotScheduled(const LoadJobPtr & job, LoadJobSet & jobs, std::unique_lock & lock); - void prioritize(const LoadJobPtr & job, ssize_t new_priority, std::unique_lock & lock); + void prioritize(const LoadJobPtr & job, size_t new_pool_id, std::unique_lock & lock); void enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock); - void spawn(std::unique_lock &); - void worker(); + bool canSpawnWorker(Pool & pool, std::unique_lock &); + bool canWorkerLive(Pool & pool, std::unique_lock &); + void updateCurrentPriorityAndSpawn(std::unique_lock &); + void spawn(Pool & pool, std::unique_lock &); + void worker(Pool & pool); // Logging const bool log_failures; // Worker should log all exceptions caught from job functions. const bool log_progress; // Periodically log total progress Poco::Logger * log; - std::chrono::system_clock::time_point busy_period_start_time; - AtomicStopwatch stopwatch; - size_t old_jobs = 0; // Number of jobs that were finished in previous busy period (for correct progress indication) mutable std::mutex mutex; // Guards all the fields below. - bool is_running = false; - - // Full set of scheduled pending jobs along with scheduling info. - std::unordered_map scheduled_jobs; - - // Subset of scheduled pending non-blocked jobs (waiting for a worker to be executed). - // Represent a queue of jobs in order of decreasing priority and FIFO for jobs with equal priorities. - std::map ready_queue; - - // Set of finished jobs (for introspection only, until jobs are removed). - LoadJobSet finished_jobs; - - // Increasing counter for `ReadyKey` assignment (to preserve FIFO order of the jobs with equal priorities). - UInt64 last_ready_seqno = 0; - - // For executing jobs. Note that we avoid using an internal queue of the pool to be able to prioritize jobs. - size_t max_threads; - size_t workers = 0; - ThreadPool pool; + bool is_running = true; + std::optional current_priority; // highest priority among active pools + UInt64 last_ready_seqno = 0; // Increasing counter for ready queue keys. + std::unordered_map scheduled_jobs; // Full set of scheduled pending jobs along with scheduling info. + std::vector pools; // Thread pools for job execution and ready queues + LoadJobSet finished_jobs; // Set of finished jobs (for introspection only, until jobs are removed). + AtomicStopwatch stopwatch; // For progress indication + size_t old_jobs = 0; // Number of jobs that were finished in previous busy period (for correct progress indication) + std::chrono::system_clock::time_point busy_period_start_time; }; } From d9be88a36a0a9221345e8300e954440d15605b8b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 15 May 2023 21:40:10 +0000 Subject: [PATCH 0294/2223] Add UUID data type to PostgreSQL --- src/Storages/StoragePostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index e013199c584..09198e5bdad 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -267,6 +267,7 @@ public: else if (which.isFloat64()) nested_column = ColumnFloat64::create(); else if (which.isDate()) nested_column = ColumnUInt16::create(); else if (which.isDateTime()) nested_column = ColumnUInt32::create(); + else if (which.isUUID()) nested_column = ColumnUUID::create(); else if (which.isDateTime64()) { nested_column = ColumnDecimal::create(0, 6); From f6d7145a723510572de153165f7a008176073ac6 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 15 May 2023 23:05:13 +0000 Subject: [PATCH 0295/2223] fix tests --- src/Common/tests/gtest_async_loader.cpp | 72 +++++++++++++++++++++---- 1 file changed, 62 insertions(+), 10 deletions(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 5666c4b923e..763f7d7972c 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -30,6 +30,11 @@ namespace DB::ErrorCodes extern const int ASYNC_LOAD_CANCELED; } +struct Initializer { + size_t max_threads = 1; + ssize_t priority = 0; +}; + struct AsyncLoaderTest { AsyncLoader loader; @@ -37,10 +42,32 @@ struct AsyncLoaderTest std::mutex rng_mutex; pcg64 rng{randomSeed()}; - explicit AsyncLoaderTest(size_t max_threads = 1) - : loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, max_threads, /* log_failures = */ false, /* log_progress = */ false) + explicit AsyncLoaderTest(std::vector initializers) + : loader(getPoolInitializers(initializers), /* log_failures = */ false, /* log_progress = */ false) {} + explicit AsyncLoaderTest(size_t max_threads = 1) + : AsyncLoaderTest({{.max_threads = max_threads}}) + {} + + std::vector getPoolInitializers(std::vector initializers) + { + std::vector result; + size_t pool_id = 0; + for (auto & desc : initializers) + { + result.push_back({ + .name = fmt::format("Pool{}", pool_id), + .metric_threads = CurrentMetrics::TablesLoaderThreads, + .metric_active_threads = CurrentMetrics::TablesLoaderThreadsActive, + .max_threads = desc.max_threads, + .priority = desc.priority + }); + pool_id++; + } + return result; + } + template T randomInt(T from, T to) { @@ -114,16 +141,19 @@ struct AsyncLoaderTest TEST(AsyncLoader, Smoke) { - AsyncLoaderTest t(2); + AsyncLoaderTest t({ + {.max_threads = 2, .priority = 0}, + {.max_threads = 2, .priority = -1}, + }); - static constexpr ssize_t low_priority = -1; + static constexpr ssize_t low_priority_pool = 1; std::atomic jobs_done{0}; std::atomic low_priority_jobs_done{0}; auto job_func = [&] (const LoadJobPtr & self) { jobs_done++; - if (self->priority() == low_priority) + if (self->pool() == low_priority_pool) low_priority_jobs_done++; }; @@ -135,7 +165,7 @@ TEST(AsyncLoader, Smoke) auto job3 = makeLoadJob({ job2 }, "job3", job_func); auto job4 = makeLoadJob({ job2 }, "job4", job_func); auto task2 = t.schedule({ job3, job4 }); - auto job5 = makeLoadJob({ job3, job4 }, low_priority, "job5", job_func); + auto job5 = makeLoadJob({ job3, job4 }, low_priority_pool, "job5", job_func); task2->merge(t.schedule({ job5 })); std::thread waiter_thread([=] { job5->wait(); }); @@ -562,13 +592,24 @@ TEST(AsyncLoader, TestOverload) TEST(AsyncLoader, StaticPriorities) { - AsyncLoaderTest t(1); + AsyncLoaderTest t({ + {.max_threads = 1, .priority = 0}, + {.max_threads = 1, .priority = 1}, + {.max_threads = 1, .priority = 2}, + {.max_threads = 1, .priority = 3}, + {.max_threads = 1, .priority = 4}, + {.max_threads = 1, .priority = 5}, + {.max_threads = 1, .priority = 6}, + {.max_threads = 1, .priority = 7}, + {.max_threads = 1, .priority = 8}, + {.max_threads = 1, .priority = 9}, + }); std::string schedule; auto job_func = [&] (const LoadJobPtr & self) { - schedule += fmt::format("{}{}", self->name, self->priority()); + schedule += fmt::format("{}{}", self->name, self->pool()); }; std::vector jobs; @@ -590,7 +631,18 @@ TEST(AsyncLoader, StaticPriorities) TEST(AsyncLoader, DynamicPriorities) { - AsyncLoaderTest t(1); + AsyncLoaderTest t({ + {.max_threads = 1, .priority = 0}, + {.max_threads = 1, .priority = 1}, + {.max_threads = 1, .priority = 2}, + {.max_threads = 1, .priority = 3}, + {.max_threads = 1, .priority = 4}, + {.max_threads = 1, .priority = 5}, + {.max_threads = 1, .priority = 6}, + {.max_threads = 1, .priority = 7}, + {.max_threads = 1, .priority = 8}, + {.max_threads = 1, .priority = 9}, + }); for (bool prioritize : {false, true}) { @@ -602,7 +654,7 @@ TEST(AsyncLoader, DynamicPriorities) { if (prioritize && self->name == "C") t.loader.prioritize(job_to_prioritize, 9); // dynamic prioritization - schedule += fmt::format("{}{}", self->name, self->priority()); + schedule += fmt::format("{}{}", self->name, self->pool()); }; // Job DAG with initial priorities. During execution of C4, job G0 priority is increased to G9, postponing B3 job executing. From 29aa96037758df616ec6253289e218abd7212626 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 09:07:35 +0200 Subject: [PATCH 0296/2223] refine docs for regexp tree dictionary --- docs/en/sql-reference/dictionaries/index.md | 107 ++++++++++++++++---- 1 file changed, 89 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 5801b7866cb..ad5a65df994 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2197,13 +2197,13 @@ Result: └─────────────────────────────────┴───────┘ ``` -## RegExp Tree Dictionary {#regexp-tree-dictionary} +## RegExpTree Dictionary {#regexp-tree-dictionary} -Regexp Tree dictionary stores multiple trees of regular expressions with attributions. Users can retrieve strings in the dictionary. If a string matches the root of the regexp tree, we will collect the corresponding attributes of the matched root and continue to walk the children. If any of the children matches the string, we will collect attributes and rewrite the old ones if conflicts occur, then continue the traverse until we reach leaf nodes. +RegExpTree Dictionary is designed to store multiple regular expressions in a dictionary, and query if a string could match one or multiple regular expressions. In some senarioes, for example, User Agent Parser, this data structure is very useful. We can use it in both local and cloud environment. -Example of the ddl query for creating Regexp Tree dictionary: +### Use RegExpTree Dictionary in local environment - +In local environment, we create RegexpTree dictionary by a yaml file: ```sql create dictionary regexp_dict @@ -2218,11 +2218,9 @@ LAYOUT(regexp_tree) ... ``` -**Source** +The dictionary source `YAMLRegExpTree` represents the structure of a Regexp Tree. For example: -We introduce a type of source called `YAMLRegExpTree` representing the structure of Regexp Tree dictionary. An Example of a valid yaml config is like: - -```xml +```yaml - regexp: 'Linux/(\d+[\.\d]*).+tlinux' name: 'TencentOS' version: '\1' @@ -2240,17 +2238,15 @@ We introduce a type of source called `YAMLRegExpTree` representing the structure version: '10' ``` -The key `regexp` represents the regular expression of a tree node. The name of key is same as the dictionary key. The `name` and `version` is user-defined attributions in the dicitionary. The `versions` (which can be any name that not appear in attributions or the key) indicates the children nodes of this tree. +This config consists of a list of RegExpTree nodes. Each node has following structure: -**Back Reference** +- **regexp** means the regular expression of this node. +- **user defined attributions** is a list of dictionary attributions defined in the dictionary structure. In this case, we have two attributions: `name` and `version`. The first nodes have the both attributions. The second node only have `name` attribution, because the `version` is defined in the children nodes. + - The value of an attribution could contain a **back reference** which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. During the query execution, the back reference in the value will be replaced by the matched capture group. +- **children nodes** is the secondary layer of the RegExpTree nodes, which also contains a list of RegExpTree nodes. If a string matches a regexp node in the first layer, the dictionary will check if the string matches the children nodes of it. If it matches, we assign the attributions of the matching nodes. If two or more nodes define the same attribution, chilren nodes have more priority. + - the name of **children nodes** in yaml files can be arbitrary. -The value of an attribution could contain a back reference which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. - -During the query execution, the back reference in the value will be replaced by the matched capture group. - -**Query** - -Due to the specialty of Regexp Tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull` work with it. +Due to the specialty of Regexp Tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. Example: @@ -2260,12 +2256,87 @@ SELECT dictGet('regexp_dict', ('name', 'version'), '31/tclwebkit1024'); Result: -``` +```text ┌─dictGet('regexp_dict', ('name', 'version'), '31/tclwebkit1024')─┐ │ ('Andriod','12') │ └─────────────────────────────────────────────────────────────────┘ ``` +Explain: + +In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the first layer, so the dictionary will continue to look into the children nodes in the second layer and find it matches `3[12]/tclwebkit`. As a result, the value of `name` is `Andriod` defined in the first layer and the value of `version` is `12` defined in the second layer. + +### Use RegExpTree Dictionary on cloud + +We have shown how RegExpTree work in the local enviroument, but we cannot use `YAMLRegExpTree` on cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local enviroment, then dump this dictionary to a csv file by `dictionary` table function and [INTO OUTFILE](../../statements/select/into-outfile.md) clause. + +```sql +select * from dictionary(regexp_dict) into outfile('regexp_dict.csv') +``` + +The content of csv file is: + +```text +1,0,"Linux/(\d+[\.\d]*).+tlinux","['version','name']","['\\1','TencentOS']" +2,0,"(\d+)/tclwebkit(\d+[\.\d]*)","['comment','version','name']","['test $1 and $2','$1','Andriod']" +3,2,"33/tclwebkit","['version']","['13']" +4,2,"3[12]/tclwebkit","['version']","['12']" +5,2,"3[12]/tclwebkit","['version']","['11']" +6,2,"3[12]/tclwebkit","['version']","['10']" +``` + +The schema of dumped file is always + +- `id UInt64` represents the identify number of the RegexpTree node. +- `parent_id UInt64` represents the id of the parent of a node. +- `regexp String` represents the regular expression string. +- `keys Array(String)` represents the names of user defined attributions. +- `values Array(String)` represents the values of user defined attributions. + +On the cloud, we can create a table `regexp_dictionary_source_table` with the above table structure. + +```sql +CREATE TABLE regexp_dictionary_source_table +( + id UInt64, + parent_id UInt64, + regexp String, + keys Array(String), + values Array(String) +) ENGINE=Memory; +``` + +Then update the local CSV by + +```bash +clickhouse client \ + --host MY_HOST \ + --secure \ + --password MY_PASSWORD \ + --query " + insert into regexp_dictionary_source_table + select * from input ('id UInt64, parent_id UInt64, regexp String, keys Array(String), values Array(String)') + FORMAT CSV" < regexp_dict.csv +``` + +You can see how to [Insert Local Files](https://clickhouse.com/docs/en/integrations/data-ingestion/insert-local-files) for more details. After we initialize the source table, we can create a RegexpTree by table source: + +``` sql +create dictionary regexp_dict +( + regexp String, + name String, + version String +PRIMARY KEY(regexp) +SOURCE(CLICKHOUSE(TABLE 'regexp_dictionary_source_table')) +LIFETIME(0) +LAYOUT(regexp_tree); +``` + +### Use RegexpTree Dictionary as a UA Parser + +With a powerful yaml configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demostrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) + ## Embedded Dictionaries {#embedded-dictionaries} From e1509ca59d879920d51ef7a08970a5d5f2e2202b Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 16 May 2023 08:23:06 +0000 Subject: [PATCH 0297/2223] fix style --- src/Common/AsyncLoader.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 0c52c549a44..d52cd03ef32 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -317,7 +317,8 @@ private: // 1) Concurrency: Amount of concurrently executing jobs in a pool is `max_threads`. // 2) Priority: As long as there is executing worker with higher priority, workers with lower priorities are not started // (although, they can finish last job started before higher priority jobs appeared) - struct Pool { + struct Pool + { const String name; const ssize_t priority; std::unique_ptr thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools. From e4e473ef30ff4a6ca3f198e3da6d21b1ff85ccbd Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 11:22:14 +0200 Subject: [PATCH 0298/2223] Update docs/en/sql-reference/dictionaries/index.md Co-authored-by: Sergei Trifonov --- docs/en/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index ad5a65df994..1ab6370c977 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2199,7 +2199,7 @@ Result: ## RegExpTree Dictionary {#regexp-tree-dictionary} -RegExpTree Dictionary is designed to store multiple regular expressions in a dictionary, and query if a string could match one or multiple regular expressions. In some senarioes, for example, User Agent Parser, this data structure is very useful. We can use it in both local and cloud environment. +RegExpTree Dictionary is designed to store multiple regular expressions in a dictionary, and query if a string could match one or multiple regular expressions. In some scenarios, for example, User Agent Parser, this data structure is very useful. We can use it in both local and cloud environments. ### Use RegExpTree Dictionary in local environment From 31b8e3c4892ff03340314274ab715506aff0e95b Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 11:22:24 +0200 Subject: [PATCH 0299/2223] Update docs/en/sql-reference/dictionaries/index.md Co-authored-by: Sergei Trifonov --- docs/en/sql-reference/dictionaries/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 1ab6370c977..b50ad4bf365 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2241,9 +2241,9 @@ The dictionary source `YAMLRegExpTree` represents the structure of a Regexp Tree This config consists of a list of RegExpTree nodes. Each node has following structure: - **regexp** means the regular expression of this node. -- **user defined attributions** is a list of dictionary attributions defined in the dictionary structure. In this case, we have two attributions: `name` and `version`. The first nodes have the both attributions. The second node only have `name` attribution, because the `version` is defined in the children nodes. +- **user defined attributions** is a list of dictionary attributions defined in the dictionary structure. In this case, we have two attributions: `name` and `version`. The first nodes have both attributions. The second node only has `name` attribution, because the `version` is defined in the children nodes. - The value of an attribution could contain a **back reference** which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. During the query execution, the back reference in the value will be replaced by the matched capture group. -- **children nodes** is the secondary layer of the RegExpTree nodes, which also contains a list of RegExpTree nodes. If a string matches a regexp node in the first layer, the dictionary will check if the string matches the children nodes of it. If it matches, we assign the attributions of the matching nodes. If two or more nodes define the same attribution, chilren nodes have more priority. +- **children nodes** is the secondary layer of the RegExpTree nodes, which also contains a list of RegExpTree nodes. If a string matches a regexp node in the first layer, the dictionary will check if the string matches the children nodes of it. If it matches, we assign the attributions of the matching nodes. If two or more nodes define the same attribution, children nodes have more priority. - the name of **children nodes** in yaml files can be arbitrary. Due to the specialty of Regexp Tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. From ed5906f15ded79823b9b4988cca6b335a6908100 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 11:22:31 +0200 Subject: [PATCH 0300/2223] Update docs/en/sql-reference/dictionaries/index.md Co-authored-by: Sergei Trifonov --- docs/en/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index b50ad4bf365..086c0b5c0ed 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2335,7 +2335,7 @@ LAYOUT(regexp_tree); ### Use RegexpTree Dictionary as a UA Parser -With a powerful yaml configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demostrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) +With a powerful yaml configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) ## Embedded Dictionaries {#embedded-dictionaries} From a40d86b921ac3a519b7c9d65b41afbdf6667d2b8 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 11:22:42 +0200 Subject: [PATCH 0301/2223] Update docs/en/sql-reference/dictionaries/index.md Co-authored-by: Sergei Trifonov --- docs/en/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 086c0b5c0ed..66f661cce60 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2268,7 +2268,7 @@ In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in t ### Use RegExpTree Dictionary on cloud -We have shown how RegExpTree work in the local enviroument, but we cannot use `YAMLRegExpTree` on cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local enviroment, then dump this dictionary to a csv file by `dictionary` table function and [INTO OUTFILE](../../statements/select/into-outfile.md) clause. +We have shown how RegExpTree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../../statements/select/into-outfile.md) clause. ```sql select * from dictionary(regexp_dict) into outfile('regexp_dict.csv') From b6d2a84e830dbcb74343c70e32d5de06ddea3c70 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 May 2023 12:01:55 +0200 Subject: [PATCH 0302/2223] Try to fix build --- src/Functions/FunctionGenerateRandomStructure.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 9d818350fac..4cbbdd84c4a 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -8,6 +8,8 @@ #include #include #include +#include +#include #include From a3dfa40eab3810aa8482ee37477dbcae6617510d Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 16 May 2023 10:07:21 +0000 Subject: [PATCH 0303/2223] Fix --- src/TableFunctions/ITableFunctionCluster.h | 4 ++-- src/TableFunctions/ITableFunctionFileLike.cpp | 3 ++- src/TableFunctions/TableFunctionS3.cpp | 15 ++++++++------- src/TableFunctions/TableFunctionURL.cpp | 3 ++- 4 files changed, 14 insertions(+), 11 deletions(-) diff --git a/src/TableFunctions/ITableFunctionCluster.h b/src/TableFunctions/ITableFunctionCluster.h index e88e3921aac..ad88d7b54f0 100644 --- a/src/TableFunctions/ITableFunctionCluster.h +++ b/src/TableFunctions/ITableFunctionCluster.h @@ -15,7 +15,7 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_GET; extern const int LOGICAL_ERROR; } @@ -49,7 +49,7 @@ protected: void parseArgumentsImpl(ASTs & args, const ContextPtr & context) override { if (args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); /// 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 9d8c36deea4..a60ab70d570 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -19,6 +19,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, const ContextPtr &) @@ -51,7 +52,7 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context void ITableFunctionFileLike::parseArgumentsImpl(ASTs & args, const ContextPtr & context) { if (args.empty() || args.size() > 4) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index ca87e7cd979..c2c23642edb 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -43,7 +43,7 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context else { if (args.empty() || args.size() > 6) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); auto * header_it = StorageURL::collectHeaders(args, configuration.headers_from_ast, context); if (header_it != args.end()) @@ -371,9 +371,9 @@ void registerTableFunctionGCS(TableFunctionFactory & factory) { factory.registerFunction( {.documentation - = {R"(The table function can be used to read the data stored on Google Cloud Storage.)", - Documentation::Examples{{"gcs", "SELECT * FROM gcs(url, hmac_key, hmac_secret)"}}, - Documentation::Categories{"DataLake"}}, + = {.description=R"(The table function can be used to read the data stored on Google Cloud Storage.)", + .examples{{"gcs", "SELECT * FROM gcs(url, hmac_key, hmac_secret)", ""}}, + .categories{"DataLake"}}, .allow_readonly = false}); } @@ -381,12 +381,13 @@ void registerTableFunctionS3(TableFunctionFactory & factory) { factory.registerFunction( {.documentation - = {R"(The table function can be used to read the data stored on AWS S3.)", - Documentation::Examples{{"s3", "SELECT * FROM s3(url, access_key_id, secret_access_key)"}}, - Documentation::Categories{"DataLake"}}, + = {.description=R"(The table function can be used to read the data stored on AWS S3.)", + .examples{{"s3", "SELECT * FROM s3(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, .allow_readonly = false}); } + void registerTableFunctionCOS(TableFunctionFactory & factory) { factory.registerFunction(); diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 6e2675d5de1..4ed204a2af3 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -14,7 +14,8 @@ #include #include - +#include +#include namespace DB { From 9dbe9507e740fdcad4f5882bcd6e74efd55ae862 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 May 2023 12:55:20 +0200 Subject: [PATCH 0304/2223] Fix style --- src/TableFunctions/TableFunctionS3.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index c2c23642edb..3b70acc3c02 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -29,7 +29,6 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; } From a4f84cecfde581b37a37589239040612e7022935 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 May 2023 13:05:36 +0200 Subject: [PATCH 0305/2223] Retry connection expired in test_rename_column/test.py --- tests/integration/test_rename_column/test.py | 32 ++++++++++++++++++-- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_rename_column/test.py b/tests/integration/test_rename_column/test.py index 6bab0a28259..a664f1050e1 100644 --- a/tests/integration/test_rename_column/test.py +++ b/tests/integration/test_rename_column/test.py @@ -225,7 +225,9 @@ def select( def rename_column( node, table_name, name, new_name, iterations=1, ignore_exception=False ): - for i in range(iterations): + i = 0 + while True: + i += 1 try: node.query( "ALTER TABLE {table_name} RENAME COLUMN {name} to {new_name}".format( @@ -233,14 +235,22 @@ def rename_column( ) ) except QueryRuntimeException as ex: + if 'Coordination::Exception' in str(ex): + continue + if not ignore_exception: raise + if i >= iterations: + break + def rename_column_on_cluster( node, table_name, name, new_name, iterations=1, ignore_exception=False ): - for i in range(iterations): + i = 0 + while True: + i += 1 try: node.query( "ALTER TABLE {table_name} ON CLUSTER test_cluster RENAME COLUMN {name} to {new_name}".format( @@ -248,12 +258,21 @@ def rename_column_on_cluster( ) ) except QueryRuntimeException as ex: + if 'Coordination::Exception' in str(ex): + continue + if not ignore_exception: raise + if i >= iterations: + break + + def alter_move(node, table_name, iterations=1, ignore_exception=False): - for i in range(iterations): + i = 0 + while True: + i += 1 move_part = random.randint(0, 99) move_volume = "external" try: @@ -263,9 +282,16 @@ def alter_move(node, table_name, iterations=1, ignore_exception=False): ) ) except QueryRuntimeException as ex: + if 'Coordination::Exception' in str(ex): + continue + if not ignore_exception: raise + if i >= iterations: + break + + def test_rename_parallel_same_node(started_cluster): table_name = "test_rename_parallel_same_node" From b3fc7d1a86264bb7c62b6b10026d5701e91753e1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 16 May 2023 11:15:15 +0000 Subject: [PATCH 0306/2223] Automatic style fix --- tests/integration/test_rename_column/test.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_rename_column/test.py b/tests/integration/test_rename_column/test.py index a664f1050e1..8dc57cf08ff 100644 --- a/tests/integration/test_rename_column/test.py +++ b/tests/integration/test_rename_column/test.py @@ -235,7 +235,7 @@ def rename_column( ) ) except QueryRuntimeException as ex: - if 'Coordination::Exception' in str(ex): + if "Coordination::Exception" in str(ex): continue if not ignore_exception: @@ -258,7 +258,7 @@ def rename_column_on_cluster( ) ) except QueryRuntimeException as ex: - if 'Coordination::Exception' in str(ex): + if "Coordination::Exception" in str(ex): continue if not ignore_exception: @@ -268,7 +268,6 @@ def rename_column_on_cluster( break - def alter_move(node, table_name, iterations=1, ignore_exception=False): i = 0 while True: @@ -282,7 +281,7 @@ def alter_move(node, table_name, iterations=1, ignore_exception=False): ) ) except QueryRuntimeException as ex: - if 'Coordination::Exception' in str(ex): + if "Coordination::Exception" in str(ex): continue if not ignore_exception: @@ -292,7 +291,6 @@ def alter_move(node, table_name, iterations=1, ignore_exception=False): break - def test_rename_parallel_same_node(started_cluster): table_name = "test_rename_parallel_same_node" drop_table(nodes, table_name) From 8436a093e78880983d4340e0b52822c1782dc84e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 May 2023 13:36:12 +0200 Subject: [PATCH 0307/2223] Fix build --- src/IO/WriteBufferFromPocoSocket.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 79534b0f030..df34e8003cb 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace ProfileEvents From d4ea3ea045a250c86859ebfcff953b13822f0a23 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 16 May 2023 13:49:32 +0200 Subject: [PATCH 0308/2223] Fix --- src/Interpreters/Cache/Metadata.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index a97b10ffbfa..01cdc7f1d1b 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -1,6 +1,7 @@ #include #include #include +#include "Common/Exception.h" #include #include @@ -257,8 +258,6 @@ void CacheMetadata::doCleanup() } locked_metadata->markAsRemoved(); - erase(it); - LOG_DEBUG(log, "Key {} is removed from metadata", cleanup_key); try { @@ -272,9 +271,17 @@ void CacheMetadata::doCleanup() } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + LOG_ERROR(log, "Error while removing key {}: {}", cleanup_key, getCurrentExceptionMessage(false)); chassert(false); } + + /// Remove key from metadata AFTER deleting key directory, because otherwise key lock is + /// released before we delete directory from fs and there might be a race: + /// a key, which we just removed, can be added back to cache before we start removing key directory, + /// which makes key directory either non-empty (and we get exception in try catch above) + /// or we removed directory while another thread thinks it exists. + erase(it); + LOG_DEBUG(log, "Key {} is removed from metadata", cleanup_key); } } From 9a2645a72904e1146e98ca58c0274bbe034a79ef Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 16 May 2023 14:09:38 +0200 Subject: [PATCH 0309/2223] Fixed clang build --- src/Backups/BackupCoordinationLocal.h | 2 +- src/Backups/BackupCoordinationRemote.h | 2 +- src/Backups/BackupsWorker.cpp | 6 +++--- src/Backups/IBackupCoordination.h | 2 +- src/Backups/IRestoreCoordination.h | 2 +- src/Backups/RestoreCoordinationLocal.h | 2 +- src/Backups/RestoreCoordinationRemote.h | 2 +- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index a7b05fbb83c..60fcc014720 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -22,7 +22,7 @@ public: BackupCoordinationLocal(bool plain_backup_); ~BackupCoordinationLocal() override; - void setStage(const String & new_stage, const String & message = "") override; + void setStage(const String & new_stage, const String & message) override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index 5671079fa27..949dd9c9bf0 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -33,7 +33,7 @@ public: ~BackupCoordinationRemote() override; - void setStage(const String & new_stage, const String & message = "") override; + void setStage(const String & new_stage, const String & message) override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 720ca994a40..0a6482fb7de 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -368,7 +368,7 @@ void BackupsWorker::doBackup( /// Wait until all the hosts have written their backup entries. backup_coordination->waitForStage(Stage::COMPLETED); - backup_coordination->setStage(Stage::COMPLETED); + backup_coordination->setStage(Stage::COMPLETED,""); } else { @@ -386,7 +386,7 @@ void BackupsWorker::doBackup( writeBackupEntries(backup, std::move(backup_entries), backup_id, backup_coordination, backup_settings.internal); /// We have written our backup entries, we need to tell other hosts (they could be waiting for it). - backup_coordination->setStage(Stage::COMPLETED); + backup_coordination->setStage(Stage::COMPLETED,""); } size_t num_files = 0; @@ -709,7 +709,7 @@ void BackupsWorker::doRestore( /// Wait until all the hosts have written their backup entries. restore_coordination->waitForStage(Stage::COMPLETED); - restore_coordination->setStage(Stage::COMPLETED); + restore_coordination->setStage(Stage::COMPLETED,""); } else { diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 68a13ab7846..75d9202374b 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -21,7 +21,7 @@ public: virtual ~IBackupCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & new_stage, const String & message = "") = 0; + virtual void setStage(const String & new_stage, const String & message) = 0; virtual void setError(const Exception & exception) = 0; virtual Strings waitForStage(const String & stage_to_wait) = 0; virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index b4df9491c4c..2f9e8d171f6 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -18,7 +18,7 @@ public: virtual ~IRestoreCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & new_stage, const String & message = "") = 0; + virtual void setStage(const String & new_stage, const String & message) = 0; virtual void setError(const Exception & exception) = 0; virtual Strings waitForStage(const String & stage_to_wait) = 0; virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index 2240a25ef3d..e27f0d1ef88 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -19,7 +19,7 @@ public: ~RestoreCoordinationLocal() override; /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & new_stage, const String & message = "") override; + void setStage(const String & new_stage, const String & message) override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 989b1c1b727..eb0fcff9c2d 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -26,7 +26,7 @@ public: ~RestoreCoordinationRemote() override; /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & new_stage, const String & message = "") override; + void setStage(const String & new_stage, const String & message) override; void setError(const Exception & exception) override; Strings waitForStage(const String & stage_to_wait) override; Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; From fd2731845c12d0ee82ebf2fb8b8aada523303581 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 15 Apr 2023 20:43:06 +0200 Subject: [PATCH 0310/2223] Simplify interface of IBackupWriter: Remove supportNativeCopy() function. --- src/Backups/BackupIO.cpp | 22 +++--- src/Backups/BackupIO.h | 29 +++++--- src/Backups/BackupIO_Disk.cpp | 54 +++++++------- src/Backups/BackupIO_Disk.h | 6 +- src/Backups/BackupIO_File.cpp | 47 ++++++------ src/Backups/BackupIO_File.h | 7 +- src/Backups/BackupIO_S3.cpp | 52 +++++++------- src/Backups/BackupIO_S3.h | 24 +------ src/Backups/BackupImpl.cpp | 72 ++++++++----------- .../test_backup_restore_new/test.py | 12 ---- .../test_backup_restore_s3/test.py | 6 +- 11 files changed, 146 insertions(+), 185 deletions(-) diff --git a/src/Backups/BackupIO.cpp b/src/Backups/BackupIO.cpp index f78e6df23a8..7b269bd965f 100644 --- a/src/Backups/BackupIO.cpp +++ b/src/Backups/BackupIO.cpp @@ -4,29 +4,30 @@ #include #include #include +#include namespace DB { -namespace ErrorCodes +IBackupReader::IBackupReader(Poco::Logger * log_) : log(log_) { - extern const int NOT_IMPLEMENTED; } void IBackupReader::copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) { + LOG_TRACE(log, "Copying file {} through buffers", file_name); auto read_buffer = readFile(file_name); auto write_buffer = destination_disk->writeFile(destination_path, std::min(size, DBMS_DEFAULT_BUFFER_SIZE), write_mode, write_settings); copyData(*read_buffer, *write_buffer, size); write_buffer->finalize(); } -IBackupWriter::IBackupWriter(const ContextPtr & context_) - : read_settings(context_->getBackupReadSettings()) - , has_throttling(static_cast(context_->getBackupsThrottler())) -{} +IBackupWriter::IBackupWriter(const ContextPtr & context_, Poco::Logger * log_) + : log(log_), read_settings(context_->getBackupReadSettings()) +{ +} void IBackupWriter::copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name) { @@ -38,9 +39,12 @@ void IBackupWriter::copyDataToFile(const CreateReadBufferFunction & create_read_ write_buffer->finalize(); } -void IBackupWriter::copyFileNative( - DiskPtr /* src_disk */, const String & /* src_file_name */, UInt64 /* src_offset */, UInt64 /* src_size */, const String & /* dest_file_name */) +void IBackupWriter::copyFileFromDisk( + DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Native copy not implemented for backup writer"); + LOG_TRACE(log, "Copying file {} through buffers", src_file_name); + auto create_read_buffer = [this, src_disk, src_file_name] { return src_disk->readFile(src_file_name, read_settings); }; + copyDataToFile(create_read_buffer, src_offset, src_size, dest_file_name); } + } diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index aef9c14e83e..dae13422bf2 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -15,6 +15,8 @@ class WriteBuffer; class IBackupReader /// BackupReaderFile, BackupReaderDisk { public: + explicit IBackupReader(Poco::Logger * log_); + virtual ~IBackupReader() = default; virtual bool fileExists(const String & file_name) = 0; virtual UInt64 getFileSize(const String & file_name) = 0; @@ -22,6 +24,9 @@ public: virtual void copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings); virtual DataSourceDescription getDataSourceDescription() const = 0; + +protected: + Poco::Logger * const log; }; /// Represents operations of storing to disk or uploading for writing a backup. @@ -30,29 +35,31 @@ class IBackupWriter /// BackupWriterFile, BackupWriterDisk public: using CreateReadBufferFunction = std::function()>; - explicit IBackupWriter(const ContextPtr & context_); + IBackupWriter(const ContextPtr & context_, Poco::Logger * log_); virtual ~IBackupWriter() = default; virtual bool fileExists(const String & file_name) = 0; virtual UInt64 getFileSize(const String & file_name) = 0; virtual bool fileContentsEqual(const String & file_name, const String & expected_file_contents) = 0; + virtual std::unique_ptr writeFile(const String & file_name) = 0; + + virtual void copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name); + + /// copyFileFromDisk() can be much faster than copyDataToFile() + /// (especially for S3 where it can use CopyObject to copy objects inside S3 instead of downloading and uploading them). + virtual void copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name); + virtual void removeFile(const String & file_name) = 0; virtual void removeFiles(const Strings & file_names) = 0; - virtual DataSourceDescription getDataSourceDescription() const = 0; - virtual void copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name); - virtual bool supportNativeCopy(DataSourceDescription /* data_source_description */) const { return false; } - /// Copy file using native copy (optimized for S3 to use CopyObject) - /// - /// NOTE: It still may fall back to copyDataToFile() if native copy is not possible: - /// - different buckets - /// - throttling had been requested - virtual void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name); + virtual DataSourceDescription getDataSourceDescription() const = 0; protected: + Poco::Logger * const log; + + /// These read settings are used to read from the source disk in copyFileFromDisk(). const ReadSettings read_settings; - const bool has_throttling; }; } diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index 10d7a572f6b..643922cf3d0 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -14,7 +15,7 @@ namespace ErrorCodes } BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & path_) - : disk(disk_), path(path_), log(&Poco::Logger::get("BackupReaderDisk")) + : IBackupReader(&Poco::Logger::get("BackupReaderDisk")), disk(disk_), path(path_) { } @@ -38,22 +39,29 @@ std::unique_ptr BackupReaderDisk::readFile(const String & fi void BackupReaderDisk::copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) { - if (write_mode == WriteMode::Rewrite) + if ((write_mode == WriteMode::Rewrite) && (destination_disk->getDataSourceDescription() == getDataSourceDescription())) { - LOG_TRACE(log, "Copying {}/{} from disk {} to {} by the disk", path, file_name, disk->getName(), destination_disk->getName()); + /// Use more optimal way. + LOG_TRACE(log, "Copying file {} using {} disk", file_name, toString(destination_disk->getDataSourceDescription().type)); disk->copyFile(path / file_name, *destination_disk, destination_path, write_settings); return; } - LOG_TRACE(log, "Copying {}/{} from disk {} to {} through buffers", path, file_name, disk->getName(), destination_disk->getName()); + /// Fallback to copy through buffers. IBackupReader::copyFileToDisk(file_name, size, destination_disk, destination_path, write_mode, write_settings); } +DataSourceDescription BackupReaderDisk::getDataSourceDescription() const +{ + return disk->getDataSourceDescription(); +} + BackupWriterDisk::BackupWriterDisk(const DiskPtr & disk_, const String & path_, const ContextPtr & context_) - : IBackupWriter(context_) + : IBackupWriter(context_, &Poco::Logger::get("BackupWriterDisk")) , disk(disk_) , path(path_) + , has_throttling(static_cast(context_->getBackupsThrottler())) { } @@ -115,31 +123,25 @@ DataSourceDescription BackupWriterDisk::getDataSourceDescription() const return disk->getDataSourceDescription(); } -DataSourceDescription BackupReaderDisk::getDataSourceDescription() const +void BackupWriterDisk::copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) { - return disk->getDataSourceDescription(); -} - -bool BackupWriterDisk::supportNativeCopy(DataSourceDescription data_source_description) const -{ - return data_source_description == disk->getDataSourceDescription(); -} - -void BackupWriterDisk::copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) -{ - if (!src_disk) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot natively copy data to disk without source disk"); - - if (has_throttling || (src_offset != 0) || (src_size != src_disk->getFileSize(src_file_name))) + /// IDisk::copyFile() can copy to the same disk only, and it cannot do the throttling. + if (!has_throttling && (getDataSourceDescription() == src_disk->getDataSourceDescription())) { - auto create_read_buffer = [this, src_disk, src_file_name] { return src_disk->readFile(src_file_name, read_settings); }; - copyDataToFile(create_read_buffer, src_offset, src_size, dest_file_name); - return; + /// IDisk::copyFile() can copy a file as a whole only. + if ((src_offset == 0) && (src_size == src_disk->getFileSize(src_file_name))) + { + /// Use more optimal way. + LOG_TRACE(log, "Copying file {} using {} disk", src_file_name, toString(src_disk->getDataSourceDescription().type)); + auto dest_file_path = path / dest_file_name; + disk->createDirectories(dest_file_path.parent_path()); + src_disk->copyFile(src_file_name, *disk, dest_file_path); + return; + } } - auto file_path = path / dest_file_name; - disk->createDirectories(file_path.parent_path()); - src_disk->copyFile(src_file_name, *disk, file_path); + /// Fallback to copy through buffers. + IBackupWriter::copyFileFromDisk(src_disk, src_file_name, src_offset, src_size, dest_file_name); } } diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index be34847000d..884282ea4e0 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -25,7 +25,6 @@ public: private: DiskPtr disk; std::filesystem::path path; - Poco::Logger * log; }; class BackupWriterDisk : public IBackupWriter @@ -38,16 +37,15 @@ public: UInt64 getFileSize(const String & file_name) override; bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; + void copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override; void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; DataSourceDescription getDataSourceDescription() const override; - bool supportNativeCopy(DataSourceDescription data_source_description) const override; - void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override; - private: DiskPtr disk; std::filesystem::path path; + const bool has_throttling; }; } diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index d4c9d0cb210..b569e65284b 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -12,7 +13,9 @@ namespace fs = std::filesystem; namespace DB { -BackupReaderFile::BackupReaderFile(const String & path_) : path(path_), log(&Poco::Logger::get("BackupReaderFile")) + +BackupReaderFile::BackupReaderFile(const String & path_) + : IBackupReader(&Poco::Logger::get("BackupReaderFile")), path(path_) { } @@ -39,19 +42,20 @@ void BackupReaderFile::copyFileToDisk(const String & file_name, size_t size, Dis if (destination_disk->getDataSourceDescription() == getDataSourceDescription()) { /// Use more optimal way. - LOG_TRACE(log, "Copying {}/{} to disk {} locally", path, file_name, destination_disk->getName()); + LOG_TRACE(log, "Copying file {} locally", file_name); fs::copy(path / file_name, fullPath(destination_disk, destination_path), fs::copy_options::overwrite_existing); return; } - LOG_TRACE(log, "Copying {}/{} to disk {} through buffers", path, file_name, destination_disk->getName()); + /// Fallback to copy through buffers. IBackupReader::copyFileToDisk(path / file_name, size, destination_disk, destination_path, write_mode, write_settings); } BackupWriterFile::BackupWriterFile(const String & path_, const ContextPtr & context_) - : IBackupWriter(context_) + : IBackupWriter(context_, &Poco::Logger::get("BackupWriterFile")) , path(path_) + , has_throttling(static_cast(context_->getBackupsThrottler())) { } @@ -141,29 +145,26 @@ DataSourceDescription BackupReaderFile::getDataSourceDescription() const } -bool BackupWriterFile::supportNativeCopy(DataSourceDescription data_source_description) const +void BackupWriterFile::copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) { - return data_source_description == getDataSourceDescription(); -} - -void BackupWriterFile::copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) -{ - std::string abs_source_path; - if (src_disk) - abs_source_path = fullPath(src_disk, src_file_name); - else - abs_source_path = fs::absolute(src_file_name); - - if (has_throttling || (src_offset != 0) || (src_size != fs::file_size(abs_source_path))) + /// std::filesystem::copy() can copy from the filesystem only, and it cannot do the throttling. + if (!has_throttling && (getDataSourceDescription() == src_disk->getDataSourceDescription())) { - auto create_read_buffer = [this, abs_source_path] { return createReadBufferFromFileBase(abs_source_path, read_settings); }; - copyDataToFile(create_read_buffer, src_offset, src_size, dest_file_name); - return; + std::string abs_source_path = fullPath(src_disk, src_file_name); + /// std::filesystem::copy() can copy a file as a whole only. + if ((src_offset == 0) && (src_size == fs::file_size(abs_source_path))) + { + /// Use more optimal way. + LOG_TRACE(log, "Copying file {} locally", src_file_name); + auto abs_dest_path = path / dest_file_name; + fs::create_directories(abs_dest_path.parent_path()); + fs::copy(abs_source_path, abs_dest_path, fs::copy_options::overwrite_existing); + return; + } } - auto file_path = path / dest_file_name; - fs::create_directories(file_path.parent_path()); - fs::copy(abs_source_path, file_path, fs::copy_options::overwrite_existing); + /// Fallback to copy through buffers. + IBackupWriter::copyFileFromDisk(src_disk, src_file_name, src_offset, src_size, dest_file_name); } } diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index d4b6e13d546..1f0a247c321 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -22,27 +22,26 @@ public: private: std::filesystem::path path; - Poco::Logger * log; }; class BackupWriterFile : public IBackupWriter { public: - explicit BackupWriterFile(const String & path_, const ContextPtr & context_); + BackupWriterFile(const String & path_, const ContextPtr & context_); ~BackupWriterFile() override; bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; + void copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override; void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; DataSourceDescription getDataSourceDescription() const override; - bool supportNativeCopy(DataSourceDescription data_source_description) const override; - void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override; private: std::filesystem::path path; + const bool has_throttling; }; } diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 84dba63ae4e..043075d55ec 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -102,11 +102,11 @@ namespace BackupReaderS3::BackupReaderS3( const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) - : s3_uri(s3_uri_) + : IBackupReader(&Poco::Logger::get("BackupReaderS3")) + , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , read_settings(context_->getReadSettings()) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) - , log(&Poco::Logger::get("BackupReaderS3")) { request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint } @@ -141,8 +141,6 @@ std::unique_ptr BackupReaderS3::readFile(const String & file void BackupReaderS3::copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) { - LOG_TRACE(log, "Copying {} to disk {}", file_name, destination_disk->getName()); - copyS3FileToDisk( client, s3_uri.bucket, @@ -162,11 +160,10 @@ void BackupReaderS3::copyFileToDisk(const String & file_name, size_t size, DiskP BackupWriterS3::BackupWriterS3( const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) - : IBackupWriter(context_) + : IBackupWriter(context_, &Poco::Logger::get("BackupWriterS3")) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) - , log(&Poco::Logger::get("BackupWriterS3")) { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint @@ -177,30 +174,31 @@ DataSourceDescription BackupWriterS3::getDataSourceDescription() const return DataSourceDescription{DataSourceType::S3, s3_uri.endpoint, false, false}; } -bool BackupWriterS3::supportNativeCopy(DataSourceDescription data_source_description) const +void BackupWriterS3::copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) { - return getDataSourceDescription() == data_source_description; -} - -void BackupWriterS3::copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) -{ - if (!src_disk) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot natively copy data to disk without source disk"); - - auto objects = src_disk->getStorageObjects(src_file_name); - if (objects.size() > 1) + /// copyS3File() can copy to another S3 bucket, but it requires the same S3 URI endpoint. + /// We don't check `has_throttling` here (compare with BackupWriterDisk::copyFileFromDisk()) because + /// copyS3File() almost doesn't use network so the throttling is not needed. + if (getDataSourceDescription() == src_disk->getDataSourceDescription()) { - auto create_read_buffer = [this, src_disk, src_file_name] { return src_disk->readFile(src_file_name, read_settings); }; - copyDataToFile(create_read_buffer, src_offset, src_size, dest_file_name); - } - else - { - auto object_storage = src_disk->getObjectStorage(); - std::string src_bucket = object_storage->getObjectsNamespace(); - auto file_path = fs::path(s3_uri.key) / dest_file_name; - copyS3File(client, src_bucket, objects[0].remote_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {}, - threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); + /// A single file can be represented as multiple objects in S3 bucket. + /// However copyS3File() can copy only a single file into a single file. + auto objects = src_disk->getStorageObjects(src_file_name); + if (objects.size() == 1) + { + /// Use more optimal way. + LOG_TRACE(log, "Copying file {} using native copy", src_file_name); + auto object_storage = src_disk->getObjectStorage(); + std::string src_bucket = object_storage->getObjectsNamespace(); + auto file_path = fs::path(s3_uri.key) / dest_file_name; + copyS3File(client, src_bucket, objects[0].remote_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {}, + threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); + return; + } } + + /// Fallback to BackupWriterS3::copyDataToFile(). + IBackupWriter::copyFileFromDisk(src_disk, src_file_name, src_offset, src_size, dest_file_name); } void BackupWriterS3::copyDataToFile( diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index c32a6b48660..7d53d30e8d6 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -32,7 +32,6 @@ private: std::shared_ptr client; ReadSettings read_settings; S3Settings::RequestSettings request_settings; - Poco::Logger * log; }; @@ -48,37 +47,18 @@ public: std::unique_ptr writeFile(const String & file_name) override; void copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name) override; + void copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override; + DataSourceDescription getDataSourceDescription() const override; void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; - DataSourceDescription getDataSourceDescription() const override; - bool supportNativeCopy(DataSourceDescription data_source_description) const override; - void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override; - private: - void copyObjectImpl( - const String & src_bucket, - const String & src_key, - const String & dst_bucket, - const String & dst_key, - size_t size, - const std::optional & metadata = std::nullopt) const; - - void copyObjectMultipartImpl( - const String & src_bucket, - const String & src_key, - const String & dst_bucket, - const String & dst_key, - size_t size, - const std::optional & metadata = std::nullopt) const; - void removeFilesBatch(const Strings & file_names); S3::URI s3_uri; std::shared_ptr client; S3Settings::RequestSettings request_settings; - Poco::Logger * log; std::optional supports_batch_delete; }; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index a8c6e16b268..55fb6dbfe03 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -805,72 +805,56 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) if (writing_finalized) throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup is already finalized"); - std::string from_file_name = "memory buffer"; - if (auto fname = entry->getFilePath(); !fname.empty()) - from_file_name = "file " + fname; - + bool should_check_lock_file = false; { std::lock_guard lock{mutex}; ++num_files; total_size += info.size; + if (!num_entries) + should_check_lock_file = true; } + auto src_disk = entry->tryGetDiskIfExists(); + auto src_file_path = entry->getFilePath(); + String src_file_desc = src_file_path.empty() ? "memory buffer" : ("file " + src_file_path); + if (info.data_file_name.empty()) { - LOG_TRACE(log, "Writing backup for file {} from {}: skipped, {}", info.data_file_name, from_file_name, !info.size ? "empty" : "base backup has it"); + LOG_TRACE(log, "Writing backup for file {} from {}: skipped, {}", info.data_file_name, src_file_desc, !info.size ? "empty" : "base backup has it"); return; } if (!coordination->startWritingFile(info.data_file_index)) { - LOG_TRACE(log, "Writing backup for file {} from {}: skipped, data file #{} is already being written", info.data_file_name, from_file_name, info.data_file_index); + LOG_TRACE(log, "Writing backup for file {} from {}: skipped, data file #{} is already being written", info.data_file_name, src_file_desc, info.data_file_index); return; } - LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}", info.data_file_name, from_file_name, info.data_file_index); + if (!should_check_lock_file) + checkLockFile(true); - auto writer_description = writer->getDataSourceDescription(); - auto reader_description = entry->getDataSourceDescription(); + /// NOTE: `mutex` must be unlocked during copying otherwise writing will be in one thread maximum and hence slow. - /// We need to copy whole file without archive, we can do it faster - /// if source and destination are compatible - if (!use_archive && writer->supportNativeCopy(reader_description)) + if (use_archive) { - /// Should be much faster than writing data through server. - LOG_TRACE(log, "Will copy file {} using native copy", info.data_file_name); - - /// NOTE: `mutex` must be unlocked here otherwise writing will be in one thread maximum and hence slow. - - writer->copyFileNative(entry->tryGetDiskIfExists(), entry->getFilePath(), info.base_size, info.size - info.base_size, info.data_file_name); + LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}, adding to archive", info.data_file_name, src_file_desc, info.data_file_index); + auto out = archive_writer->writeFile(info.data_file_name); + auto read_buffer = entry->getReadBuffer(); + if (info.base_size != 0) + read_buffer->seek(info.base_size, SEEK_SET); + copyData(*read_buffer, *out); + out->finalize(); + } + else if (src_disk) + { + LOG_TRACE(log, "Writing backup for file {} from {} (disk {}): data file #{}", info.data_file_name, src_file_desc, src_disk->getName(), info.data_file_index); + writer->copyFileFromDisk(src_disk, src_file_path, info.base_size, info.size - info.base_size, info.data_file_name); } else { - bool has_entries = false; - { - std::lock_guard lock{mutex}; - has_entries = num_entries > 0; - } - if (!has_entries) - checkLockFile(true); - - if (use_archive) - { - LOG_TRACE(log, "Adding file {} to archive", info.data_file_name); - auto out = archive_writer->writeFile(info.data_file_name); - auto read_buffer = entry->getReadBuffer(); - if (info.base_size != 0) - read_buffer->seek(info.base_size, SEEK_SET); - copyData(*read_buffer, *out); - out->finalize(); - } - else - { - LOG_TRACE(log, "Will copy file {}", info.data_file_name); - auto create_read_buffer = [entry] { return entry->getReadBuffer(); }; - - /// NOTE: `mutex` must be unlocked here otherwise writing will be in one thread maximum and hence slow. - writer->copyDataToFile(create_read_buffer, info.base_size, info.size - info.base_size, info.data_file_name); - } + LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}", info.data_file_name, src_file_desc, info.data_file_index); + auto create_read_buffer = [entry] { return entry->getReadBuffer(); }; + writer->copyDataToFile(create_read_buffer, info.base_size, info.size - info.base_size, info.data_file_name); } { diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 18f6ada6821..c19cca4126a 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -158,8 +158,6 @@ def test_restore_table(engine): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") - assert instance.contains_in_log("using native copy") - instance.query("DROP TABLE test.table") assert instance.query("EXISTS test.table") == "0\n" @@ -200,8 +198,6 @@ def test_restore_table_under_another_name(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") - assert instance.contains_in_log("using native copy") - assert instance.query("EXISTS test.table2") == "0\n" instance.query(f"RESTORE TABLE test.table AS test.table2 FROM {backup_name}") @@ -215,8 +211,6 @@ def test_backup_table_under_another_name(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table AS test.table2 TO {backup_name}") - assert instance.contains_in_log("using native copy") - assert instance.query("EXISTS test.table2") == "0\n" instance.query(f"RESTORE TABLE test.table2 FROM {backup_name}") @@ -245,8 +239,6 @@ def test_incremental_backup(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") - assert instance.contains_in_log("using native copy") - instance.query("INSERT INTO test.table VALUES (65, 'a'), (66, 'b')") assert instance.query("SELECT count(), sum(x) FROM test.table") == "102\t5081\n" @@ -524,8 +516,6 @@ def test_file_engine(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") - assert instance.contains_in_log("using native copy") - instance.query("DROP TABLE test.table") assert instance.query("EXISTS test.table") == "0\n" @@ -540,8 +530,6 @@ def test_database(): instance.query(f"BACKUP DATABASE test TO {backup_name}") - assert instance.contains_in_log("using native copy") - instance.query("DROP DATABASE test") instance.query(f"RESTORE DATABASE test FROM {backup_name}") diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index e42754dd2a3..6e9b6b8569e 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -141,7 +141,7 @@ def test_backup_to_s3_native_copy(): f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("BackupImpl.*using native copy") + assert node.contains_in_log("BackupWriterS3.*using native copy") assert node.contains_in_log("copyS3FileToDisk.*using native copy") assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" @@ -155,7 +155,7 @@ def test_backup_to_s3_native_copy_other_bucket(): f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("BackupImpl.*using native copy") + assert node.contains_in_log("BackupWriterS3.*using native copy") assert node.contains_in_log("copyS3FileToDisk.*using native copy") assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" @@ -167,7 +167,7 @@ def test_backup_to_s3_native_copy_multipart(): backup_name = new_backup_name() backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" check_backup_and_restore(storage_policy, backup_destination, size=1000000) - assert node.contains_in_log("BackupImpl.*using native copy") + assert node.contains_in_log("BackupWriterS3.*using native copy") assert node.contains_in_log("copyS3FileToDisk.*using native copy") assert node.contains_in_log( f"copyS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}/" From 69114cb550527443e555875a91f9fe1c2731e532 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 22 Apr 2023 13:48:03 +0200 Subject: [PATCH 0311/2223] Add function getBlobPath() to IDisk interface to allow copying to/from disks which are not built on top of IObjectStorage. --- src/Backups/BackupIO_S3.cpp | 14 ++++++-------- src/Disks/DiskEncrypted.h | 12 ++++++++++++ src/Disks/DiskLocal.cpp | 11 +++++++++++ src/Disks/DiskLocal.h | 3 +++ src/Disks/FakeDiskTransaction.h | 8 ++------ src/Disks/IDisk.cpp | 10 ---------- src/Disks/IDisk.h | 16 ++++++++++------ src/Disks/IDiskTransaction.h | 8 +++----- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 16 ++++++++++------ src/Disks/ObjectStorages/DiskObjectStorage.h | 7 ++----- .../DiskObjectStorageTransaction.cpp | 11 +++++------ .../DiskObjectStorageTransaction.h | 6 +----- .../ObjectStorages/S3/copyS3FileToDisk.cpp | 18 ++++++++++++++---- 13 files changed, 79 insertions(+), 61 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 043075d55ec..901e9de7576 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -181,17 +181,15 @@ void BackupWriterS3::copyFileFromDisk(DiskPtr src_disk, const String & src_file_ /// copyS3File() almost doesn't use network so the throttling is not needed. if (getDataSourceDescription() == src_disk->getDataSourceDescription()) { - /// A single file can be represented as multiple objects in S3 bucket. - /// However copyS3File() can copy only a single file into a single file. - auto objects = src_disk->getStorageObjects(src_file_name); - if (objects.size() == 1) + /// getBlobPath() can return std::nullopt if the file is stored as multiple objects in S3 bucket. + /// In this case we can't use the native copy. + if (auto blob_path = src_disk->getBlobPath(src_file_name)) { /// Use more optimal way. LOG_TRACE(log, "Copying file {} using native copy", src_file_name); - auto object_storage = src_disk->getObjectStorage(); - std::string src_bucket = object_storage->getObjectsNamespace(); - auto file_path = fs::path(s3_uri.key) / dest_file_name; - copyS3File(client, src_bucket, objects[0].remote_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {}, + const auto & [src_bucket, src_key] = *blob_path; + auto dest_key = fs::path(s3_uri.key) / dest_file_name; + copyS3File(client, src_bucket, src_key, src_offset, src_size, s3_uri.bucket, dest_key, request_settings, {}, threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); return; } diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 8e824a1f7e5..d9da320e505 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -131,6 +131,18 @@ public: WriteMode mode, const WriteSettings & settings) override; + std::optional> getBlobPath(const String & path) const override + { + auto wrapped_path = wrappedPath(path); + return delegate->getBlobPath(wrapped_path); + } + + void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override + { + auto wrapped_path = wrappedPath(path); + delegate->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function)); + } + void removeFile(const String & path) override { auto wrapped_path = wrappedPath(path); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 1abecb7af4e..a40368fae88 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -328,6 +328,17 @@ DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode, const fs::path(disk_path) / path, buf_size, flags, settings.local_throttler); } +std::optional> DiskLocal::getBlobPath(const String & path) const +{ + return std::make_pair(fs::path(disk_path) / path, ""); +} + +void DiskLocal::writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) +{ + auto blob_path = std::make_pair(fs::path(disk_path) / path, ""); + std::move(write_blob_function)(blob_path, mode, {}); +} + void DiskLocal::removeFile(const String & path) { auto fs_path = fs::path(disk_path) / path; diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 3112198aab3..b838654925d 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -81,6 +81,9 @@ public: WriteMode mode, const WriteSettings & settings) override; + std::optional> getBlobPath(const String & path) const override; + void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override; + void removeFile(const String & path) override; void removeFileIfExists(const String & path) override; void removeDirectory(const String & path) override; diff --git a/src/Disks/FakeDiskTransaction.h b/src/Disks/FakeDiskTransaction.h index 3a7ea4473b6..2cf540444be 100644 --- a/src/Disks/FakeDiskTransaction.h +++ b/src/Disks/FakeDiskTransaction.h @@ -69,13 +69,9 @@ public: return disk.writeFile(path, buf_size, mode, settings); } - void writeFileUsingCustomWriteObject( - const String & path, - WriteMode mode, - std::function & object_attributes)> - custom_write_object_function) override + void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override { - disk.writeFileUsingCustomWriteObject(path, mode, std::move(custom_write_object_function)); + disk.writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function)); } void removeFile(const std::string & path) override diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 22197760d88..9f39c8242f5 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -35,16 +35,6 @@ void IDisk::copyFile(const String & from_file_path, IDisk & to_disk, const Strin out->finalize(); } -void IDisk::writeFileUsingCustomWriteObject( - const String &, WriteMode, std::function &)>) -{ - throw Exception( - ErrorCodes::NOT_IMPLEMENTED, - "Method `writeFileUsingCustomWriteObject()` is not implemented for disk: {}", - getDataSourceDescription().type); -} - - DiskTransactionPtr IDisk::createTransaction() { return std::make_shared(*this); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index b0a57afe169..238f2258c3d 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -209,14 +209,18 @@ public: WriteMode mode = WriteMode::Rewrite, const WriteSettings & settings = {}) = 0; - /// Write a file using a custom function to write an object to the disk's object storage. + /// Returns the path to a blob representing a specified file. + /// The meaning of the returned path depends on disk's type. + /// For DiskLocal it the absolute path to the file and for DiskObjectStorage it's the name of a namespace + /// combined with StoredObject::absolute_path. + virtual std::optional> getBlobPath(const String & path) const = 0; + + using WriteBlobFunction = std::function & blob_path, WriteMode mode, const std::optional & object_attributes)>; + + /// Write a file using a custom function to write a blob representing the file. /// This method is alternative to writeFile(), the difference is that writeFile() calls IObjectStorage::writeObject() /// to write an object to the object storage while this method allows to specify a callback for that. - virtual void writeFileUsingCustomWriteObject( - const String & path, - WriteMode mode, - std::function & object_attributes)> - custom_write_object_function); + virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; /// Remove file. Throws exception if file doesn't exists or it's a directory. /// Return whether file was finally removed. (For remote disks it is not always removed). diff --git a/src/Disks/IDiskTransaction.h b/src/Disks/IDiskTransaction.h index 2edbe858c06..376d7bd78e6 100644 --- a/src/Disks/IDiskTransaction.h +++ b/src/Disks/IDiskTransaction.h @@ -68,12 +68,10 @@ public: const WriteSettings & settings = {}, bool autocommit = true) = 0; + using WriteBlobFunction = std::function & blob_path, WriteMode mode, const std::optional & object_attributes)>; + /// Write a file using a custom function to write an object to the disk's object storage. - virtual void writeFileUsingCustomWriteObject( - const String & path, - WriteMode mode, - std::function & object_attributes)> - custom_write_object_function) = 0; + virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; /// Remove file. Throws exception if file doesn't exists or it's a directory. virtual void removeFile(const std::string & path) = 0; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index f832ba5b7b6..b01f1b327e7 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -579,15 +579,19 @@ std::unique_ptr DiskObjectStorage::writeFile( return result; } -void DiskObjectStorage::writeFileUsingCustomWriteObject( - const String & path, - WriteMode mode, - std::function & object_attributes)> - custom_write_object_function) +std::optional> DiskObjectStorage::getBlobPath(const String & path) const +{ + auto objects = getStorageObjects(path); + if (objects.size() != 1) + return {}; + return std::make_pair(object_storage->getObjectsNamespace(), objects[0].absolute_path); +} + +void DiskObjectStorage::writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) { LOG_TEST(log, "Write file: {}", path); auto transaction = createObjectStorageTransaction(); - return transaction->writeFileUsingCustomWriteObject(path, mode, std::move(custom_write_object_function)); + return transaction->writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function)); } void DiskObjectStorage::applyNewSettings( diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 4372bc75950..97751edc3f5 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -149,11 +149,8 @@ public: WriteMode mode, const WriteSettings & settings) override; - void writeFileUsingCustomWriteObject( - const String & path, - WriteMode mode, - std::function & object_attributes)> - custom_write_object_function) override; + std::optional> getBlobPath(const String & path) const override; + void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override; void copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 6cafc35f8fa..f578bcb9772 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -669,11 +669,8 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile } -void DiskObjectStorageTransaction::writeFileUsingCustomWriteObject( - const String & path, - WriteMode mode, - std::function & object_attributes)> - custom_write_object_function) +void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction( + const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) { /// This function is a simplified and adapted version of DiskObjectStorageTransaction::writeFile(). auto blob_name = object_storage.generateBlobNameForPath(path); @@ -694,8 +691,10 @@ void DiskObjectStorageTransaction::writeFileUsingCustomWriteObject( operations_to_execute.emplace_back(std::move(write_operation)); + auto blob_path = std::make_pair(object_storage.getObjectsNamespace(), object.absolute_path); + /// We always use mode Rewrite because we simulate append using metadata and different files - size_t object_size = std::move(custom_write_object_function)(object, WriteMode::Rewrite, object_attributes); + size_t object_size = std::move(write_blob_function)(blob_path, WriteMode::Rewrite, object_attributes); /// Create metadata (see create_metadata_callback in DiskObjectStorageTransaction::writeFile()). if (mode == WriteMode::Rewrite) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h index 080a3e42057..a4cb0ed3739 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h @@ -100,11 +100,7 @@ public: bool autocommit = true) override; /// Write a file using a custom function to write an object to the disk's object storage. - void writeFileUsingCustomWriteObject( - const String & path, - WriteMode mode, - std::function & object_attributes)> - custom_write_object_function) override; + void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override; void removeFile(const std::string & path) override; void removeFileIfExists(const std::string & path) override; diff --git a/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp b/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp index f89415f789a..dea3266ad7a 100644 --- a/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp +++ b/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp @@ -50,18 +50,28 @@ void copyS3FileToDisk( String dest_bucket = destination_disk->getObjectStorage()->getObjectsNamespace(); - auto custom_write_object = [&](const StoredObject & object_, WriteMode write_mode_, const std::optional & object_attributes_) -> size_t + auto write_blob_function = [&](const std::pair & blob_path_, WriteMode write_mode_, const std::optional & object_attributes_) -> size_t { /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. chassert(write_mode_ == WriteMode::Rewrite); - copyS3File(s3_client, src_bucket, src_key, *src_offset, *src_size, dest_bucket, /* dest_key= */ object_.remote_path, - request_settings, object_attributes_, scheduler, /* for_disk_s3= */ true); + copyS3File( + s3_client, + src_bucket, + src_key, + *src_offset, + *src_size, + /* dest_bucket= */ blob_path_.first, + /* dest_key= */ blob_path_.second, + request_settings, + object_attributes_, + scheduler, + /* for_disk_s3= */ true); return *src_size; }; - destination_disk->writeFileUsingCustomWriteObject(destination_path, write_mode, custom_write_object); + destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function); } } From 101aa6eff09a06f3751f3f06c7fa22afb4d93134 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 22 Apr 2023 17:51:14 +0200 Subject: [PATCH 0312/2223] Add function copyS3FileFromDisk(). --- src/Backups/BackupIO_S3.cpp | 24 +---- .../ObjectStorages/S3/copyS3FileToDisk.cpp | 102 ++++++++++++------ .../ObjectStorages/S3/copyS3FileToDisk.h | 20 +++- .../test_backup_restore_s3/test.py | 6 +- 4 files changed, 94 insertions(+), 58 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 901e9de7576..d2861500159 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -176,27 +176,9 @@ DataSourceDescription BackupWriterS3::getDataSourceDescription() const void BackupWriterS3::copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) { - /// copyS3File() can copy to another S3 bucket, but it requires the same S3 URI endpoint. - /// We don't check `has_throttling` here (compare with BackupWriterDisk::copyFileFromDisk()) because - /// copyS3File() almost doesn't use network so the throttling is not needed. - if (getDataSourceDescription() == src_disk->getDataSourceDescription()) - { - /// getBlobPath() can return std::nullopt if the file is stored as multiple objects in S3 bucket. - /// In this case we can't use the native copy. - if (auto blob_path = src_disk->getBlobPath(src_file_name)) - { - /// Use more optimal way. - LOG_TRACE(log, "Copying file {} using native copy", src_file_name); - const auto & [src_bucket, src_key] = *blob_path; - auto dest_key = fs::path(s3_uri.key) / dest_file_name; - copyS3File(client, src_bucket, src_key, src_offset, src_size, s3_uri.bucket, dest_key, request_settings, {}, - threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); - return; - } - } - - /// Fallback to BackupWriterS3::copyDataToFile(). - IBackupWriter::copyFileFromDisk(src_disk, src_file_name, src_offset, src_size, dest_file_name); + copyS3FileFromDisk(src_disk, src_file_name, src_offset, src_size, + client, s3_uri.bucket, fs::path(s3_uri.key) / dest_file_name, read_settings, request_settings, + threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); } void BackupWriterS3::copyDataToFile( diff --git a/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp b/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp index dea3266ad7a..e43d88b2519 100644 --- a/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp +++ b/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp @@ -13,7 +13,7 @@ namespace DB { void copyS3FileToDisk( - const std::shared_ptr & s3_client, + const std::shared_ptr & src_s3_client, const String & src_bucket, const String & src_key, const std::optional & version_id, @@ -31,47 +31,85 @@ void copyS3FileToDisk( src_offset = 0; if (!src_size) - src_size = S3::getObjectSize(*s3_client, src_bucket, src_key, version_id.value_or(""), request_settings) - *src_offset; + src_size = S3::getObjectSize(*src_s3_client, src_bucket, src_key, version_id.value_or(""), request_settings) - *src_offset; auto destination_data_source_description = destination_disk->getDataSourceDescription(); - if (destination_data_source_description != DataSourceDescription{DataSourceType::S3, s3_client->getInitialEndpoint(), false, false}) + if (destination_data_source_description == DataSourceDescription{DataSourceType::S3, src_s3_client->getInitialEndpoint(), false, false}) { - LOG_TRACE(&Poco::Logger::get("copyS3FileToDisk"), "Copying {} to disk {} through buffers", src_key, destination_disk->getName()); - ReadBufferFromS3 read_buffer{s3_client, src_bucket, src_key, {}, request_settings, read_settings}; - if (*src_offset) - read_buffer.seek(*src_offset, SEEK_SET); - auto write_buffer = destination_disk->writeFile(destination_path, std::min(*src_size, DBMS_DEFAULT_BUFFER_SIZE), write_mode, write_settings); - copyData(read_buffer, *write_buffer, *src_size); - write_buffer->finalize(); + /// Use native copy, the more optimal way. + LOG_TRACE(&Poco::Logger::get("copyS3FileToDisk"), "Copying {} to disk {} using native copy", src_key, destination_disk->getName()); + auto write_blob_function = [&](const std::pair & blob_path_, WriteMode write_mode_, const std::optional & object_attributes_) -> size_t + { + /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. + chassert(write_mode_ == WriteMode::Rewrite); + + copyS3File( + src_s3_client, + src_bucket, + src_key, + *src_offset, + *src_size, + /* dest_bucket= */ blob_path_.first, + /* dest_key= */ blob_path_.second, + request_settings, + object_attributes_, + scheduler, + /* for_disk_s3= */ true); + + return *src_size; + }; + + destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function); return; } - LOG_TRACE(&Poco::Logger::get("copyS3FileToDisk"), "Copying {} to disk {} using native copy", src_key, destination_disk->getName()); + /// Fallback to copy through buffers. + LOG_TRACE(&Poco::Logger::get("copyS3FileToDisk"), "Copying {} to disk {} through buffers", src_key, destination_disk->getName()); + ReadBufferFromS3 read_buffer{src_s3_client, src_bucket, src_key, {}, request_settings, read_settings}; + if (*src_offset) + read_buffer.seek(*src_offset, SEEK_SET); + auto write_buffer = destination_disk->writeFile(destination_path, std::min(*src_size, DBMS_DEFAULT_BUFFER_SIZE), write_mode, write_settings); + copyData(read_buffer, *write_buffer, *src_size); + write_buffer->finalize(); +} - String dest_bucket = destination_disk->getObjectStorage()->getObjectsNamespace(); +void copyS3FileFromDisk( + DiskPtr src_disk, + const String & src_path, + std::optional src_offset, + std::optional src_size, + const std::shared_ptr & dest_s3_client, + const String & dest_bucket, + const String & dest_key, + const ReadSettings & read_settings, + const S3Settings::RequestSettings & request_settings, + ThreadPoolCallbackRunner scheduler) +{ + if (!src_offset) + src_offset = 0; - auto write_blob_function = [&](const std::pair & blob_path_, WriteMode write_mode_, const std::optional & object_attributes_) -> size_t + if (!src_size) + src_size = src_disk->getFileSize(src_path) - *src_offset; + + auto source_data_source_description = src_disk->getDataSourceDescription(); + if (source_data_source_description == DataSourceDescription{DataSourceType::S3, dest_s3_client->getInitialEndpoint(), false, false}) { - /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. - chassert(write_mode_ == WriteMode::Rewrite); + /// getBlobPath() can return std::nullopt if the file is stored as multiple objects in S3 bucket. + /// In this case we can't use native copy. + if (auto blob_path = src_disk->getBlobPath(src_path)) + { + /// Use native copy, the more optimal way. + LOG_TRACE(&Poco::Logger::get("copyS3FileFromDisk"), "Copying file {} to S3 using native copy", src_path); + const auto & [src_bucket, src_key] = *blob_path; + copyS3File(dest_s3_client, src_bucket, src_key, *src_offset, *src_size, dest_bucket, dest_key, request_settings, {}, scheduler); + return; + } + } - copyS3File( - s3_client, - src_bucket, - src_key, - *src_offset, - *src_size, - /* dest_bucket= */ blob_path_.first, - /* dest_key= */ blob_path_.second, - request_settings, - object_attributes_, - scheduler, - /* for_disk_s3= */ true); - - return *src_size; - }; - - destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function); + /// Fallback to copy through buffers. + LOG_TRACE(&Poco::Logger::get("copyS3FileFromDisk"), "Copying {} to S3 through buffers", src_path); + auto create_read_buffer = [src_disk, &src_path, &read_settings] { return src_disk->readFile(src_path, read_settings); }; + copyDataToS3File(create_read_buffer, *src_offset, *src_size, dest_s3_client, dest_bucket, dest_key, request_settings, {}, scheduler); } } diff --git a/src/Disks/ObjectStorages/S3/copyS3FileToDisk.h b/src/Disks/ObjectStorages/S3/copyS3FileToDisk.h index 21c92ec9623..78caf2f50c8 100644 --- a/src/Disks/ObjectStorages/S3/copyS3FileToDisk.h +++ b/src/Disks/ObjectStorages/S3/copyS3FileToDisk.h @@ -13,11 +13,11 @@ namespace DB { /// Copies an object from S3 bucket to a disk of any type. -/// Depending on the disk the function can either do copying though buffers +/// Depending on the disk the function can either do copying through buffers /// (i.e. download the object by portions and then write those portions to the specified disk), /// or perform a server-side copy. void copyS3FileToDisk( - const std::shared_ptr & s3_client, + const std::shared_ptr & src_s3_client, const String & src_bucket, const String & src_key, const std::optional & version_id, @@ -31,6 +31,22 @@ void copyS3FileToDisk( const S3Settings::RequestSettings & request_settings = {}, ThreadPoolCallbackRunner scheduler = {}); +/// Copies an object from a disk of any type to S3 bucket. +/// Depending on the disk the function can either do copying through buffers +/// (i.e. read the object by portions and then upload those portions to the specified disk), +/// or perform a server-side copy. +void copyS3FileFromDisk( + DiskPtr src_disk, + const String & src_path, + std::optional src_offset, + std::optional src_size, + const std::shared_ptr & dest_s3_client, + const String & dest_bucket, + const String & dest_key, + const ReadSettings & read_settings = {}, + const S3Settings::RequestSettings & request_settings = {}, + ThreadPoolCallbackRunner scheduler = {}); + } #endif diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 6e9b6b8569e..b5ac34f0b46 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -141,7 +141,7 @@ def test_backup_to_s3_native_copy(): f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("BackupWriterS3.*using native copy") + assert node.contains_in_log("copyS3FileFromDisk.*using native copy") assert node.contains_in_log("copyS3FileToDisk.*using native copy") assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" @@ -155,7 +155,7 @@ def test_backup_to_s3_native_copy_other_bucket(): f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("BackupWriterS3.*using native copy") + assert node.contains_in_log("copyS3FileFromDisk.*using native copy") assert node.contains_in_log("copyS3FileToDisk.*using native copy") assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" @@ -167,7 +167,7 @@ def test_backup_to_s3_native_copy_multipart(): backup_name = new_backup_name() backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" check_backup_and_restore(storage_policy, backup_destination, size=1000000) - assert node.contains_in_log("BackupWriterS3.*using native copy") + assert node.contains_in_log("copyS3FileFromDisk.*using native copy") assert node.contains_in_log("copyS3FileToDisk.*using native copy") assert node.contains_in_log( f"copyS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}/" From bc880db5d925a4ff57565d85b541e4d6c17c07d7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 23 Apr 2023 10:50:45 +0200 Subject: [PATCH 0313/2223] Add functions to read/write encrypted files from IDisk. --- src/Disks/DiskEncrypted.h | 23 +++++++++++++++++++++++ src/Disks/IDisk.cpp | 15 +++++++++++++++ src/Disks/IDisk.h | 14 ++++++++++++++ 3 files changed, 52 insertions(+) diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index d9da320e505..9c9e61275ce 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -198,6 +198,29 @@ public: delegate->removeSharedFileIfExists(wrapped_path, flag); } + std::unique_ptr readEncryptedFile( + const String & path, const ReadSettings & settings) const override + { + auto wrapped_path = wrappedPath(path); + return delegate->readFile(wrapped_path, settings); + } + + std::unique_ptr writeEncryptedFile( + const String & path, + size_t buf_size, + WriteMode mode, + const WriteSettings & settings) const override + { + auto wrapped_path = wrappedPath(path); + return delegate->writeFile(wrapped_path, buf_size, mode, settings); + } + + size_t getEncryptedFileSize(const String & path) const override + { + auto wrapped_path = wrappedPath(path); + return delegate->getFileSize(wrapped_path); + } + void setLastModified(const String & path, const Poco::Timestamp & timestamp) override { auto wrapped_path = wrappedPath(path); diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 9f39c8242f5..54e09b09d2f 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -52,6 +52,21 @@ void IDisk::removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_ba } } +std::unique_ptr IDisk::readEncryptedFile(const String &, const ReadSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "File encryption is not implemented for disk of type {}", getDataSourceDescription().type); +} + +std::unique_ptr IDisk::writeEncryptedFile(const String &, size_t, WriteMode, const WriteSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "File encryption is not implemented for disk of type {}", getDataSourceDescription().type); +} + +size_t IDisk::getEncryptedFileSize(const String &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "File encryption is not implemented for disk of type {}", getDataSourceDescription().type); +} + using ResultsCollector = std::vector>; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 238f2258c3d..b98d8a74308 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -251,6 +251,20 @@ public: /// Second bool param is a flag to remove (true) or keep (false) shared data on S3 virtual void removeSharedFileIfExists(const String & path, bool /* keep_shared_data */) { removeFileIfExists(path); } + /// Reads a file from an encrypted disk without decrypting it. + virtual std::unique_ptr readEncryptedFile( + const String & path, const ReadSettings & settings = ReadSettings{}) const; + + /// Writes an already encrypted file to an encrypted disk. + virtual std::unique_ptr writeEncryptedFile( + const String & path, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + WriteMode mode = WriteMode::Rewrite, + const WriteSettings & settings = {}) const; + + /// Returns the size of encrypted file on an encrypted disk. + virtual size_t getEncryptedFileSize(const String & path) const; + virtual const String & getCacheName() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no cache"); } virtual bool supportsCache() const { return false; } From cc50fcc60a4102b194f30b56896d045390833d8d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 23 Apr 2023 12:25:46 +0200 Subject: [PATCH 0314/2223] Remove the 'temporary_file_' argument from BackupEntryFromImmutableFile's constructor. --- src/Backups/BackupEntryFromAppendOnlyFile.cpp | 5 ++--- src/Backups/BackupEntryFromAppendOnlyFile.h | 3 +-- src/Backups/BackupEntryFromImmutableFile.cpp | 4 +--- src/Backups/BackupEntryFromImmutableFile.h | 5 +---- src/Backups/BackupEntryWrappedWith.h | 8 +++++++- .../MergeTree/DataPartStorageOnDiskBase.cpp | 10 +++++++--- src/Storages/StorageLog.cpp | 17 +++++++++-------- src/Storages/StorageStripeLog.cpp | 17 +++++++++-------- 8 files changed, 37 insertions(+), 32 deletions(-) diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index 5384a69d890..83117f686bf 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -10,9 +10,8 @@ BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile( const String & file_path_, const ReadSettings & settings_, const std::optional & file_size_, - const std::optional & checksum_, - const std::shared_ptr & temporary_file_) - : BackupEntryFromImmutableFile(disk_, file_path_, settings_, file_size_, checksum_, temporary_file_) + const std::optional & checksum_) + : BackupEntryFromImmutableFile(disk_, file_path_, settings_, file_size_, checksum_) , limit(BackupEntryFromImmutableFile::getSize()) { } diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.h b/src/Backups/BackupEntryFromAppendOnlyFile.h index b0cee38c6be..b7a39c935a9 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.h +++ b/src/Backups/BackupEntryFromAppendOnlyFile.h @@ -18,8 +18,7 @@ public: const String & file_path_, const ReadSettings & settings_, const std::optional & file_size_ = {}, - const std::optional & checksum_ = {}, - const std::shared_ptr & temporary_file_ = {}); + const std::optional & checksum_ = {}); UInt64 getSize() const override { return limit; } std::unique_ptr getReadBuffer() const override; diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index 48783a3bb63..790ea567496 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -13,14 +13,12 @@ BackupEntryFromImmutableFile::BackupEntryFromImmutableFile( const String & file_path_, const ReadSettings & settings_, const std::optional & file_size_, - const std::optional & checksum_, - const std::shared_ptr & temporary_file_) + const std::optional & checksum_) : disk(disk_) , file_path(file_path_) , settings(settings_) , file_size(file_size_) , checksum(checksum_) - , temporary_file_on_disk(temporary_file_) { } diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 66f1fade294..4f2f902d31e 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -7,7 +7,6 @@ namespace DB { -class TemporaryFileOnDisk; class IDisk; using DiskPtr = std::shared_ptr; @@ -22,8 +21,7 @@ public: const String & file_path_, const ReadSettings & settings_, const std::optional & file_size_ = {}, - const std::optional & checksum_ = {}, - const std::shared_ptr & temporary_file_ = {}); + const std::optional & checksum_ = {}); ~BackupEntryFromImmutableFile() override; @@ -43,7 +41,6 @@ private: mutable std::optional file_size TSA_GUARDED_BY(get_file_size_mutex); mutable std::mutex get_file_size_mutex; const std::optional checksum; - const std::shared_ptr temporary_file_on_disk; }; } diff --git a/src/Backups/BackupEntryWrappedWith.h b/src/Backups/BackupEntryWrappedWith.h index 97244650b6b..da3b70e9ba9 100644 --- a/src/Backups/BackupEntryWrappedWith.h +++ b/src/Backups/BackupEntryWrappedWith.h @@ -27,11 +27,17 @@ private: T custom_value; }; +template +BackupEntryPtr wrapBackupEntryWith(BackupEntryPtr && backup_entry, const T & custom_value) +{ + return std::make_shared>(std::move(backup_entry), custom_value); +} + template void wrapBackupEntriesWith(std::vector> & backup_entries, const T & custom_value) { for (auto & [_, backup_entry] : backup_entries) - backup_entry = std::make_shared>(std::move(backup_entry), custom_value); + backup_entry = wrapBackupEntryWith(std::move(backup_entry), custom_value); } } diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index ec00cc3d2b9..4df490f41fe 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -392,9 +393,12 @@ void DataPartStorageOnDiskBase::backup( file_hash = {it->second.file_hash.first, it->second.file_hash.second}; } - backup_entries.emplace_back( - filepath_in_backup, - std::make_unique(disk, filepath_on_disk, read_settings, file_size, file_hash, temp_dir_owner)); + BackupEntryPtr backup_entry = std::make_unique(disk, filepath_on_disk, read_settings, file_size, file_hash); + + if (temp_dir_owner) + backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); + + backup_entries.emplace_back(filepath_in_backup, std::move(backup_entry)); } } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 8264d67aaba..19887d6695e 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -951,10 +952,10 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c String data_file_name = fileName(data_file.path); String hardlink_file_path = temp_dir / data_file_name; disk->createHardLink(data_file.path, hardlink_file_path); - backup_entries_collector.addBackupEntry( - data_path_in_backup_fs / data_file_name, - std::make_unique( - disk, hardlink_file_path, read_settings, file_checker.getFileSize(data_file.path), std::nullopt, temp_dir_owner)); + BackupEntryPtr backup_entry = std::make_unique( + disk, hardlink_file_path, read_settings, file_checker.getFileSize(data_file.path)); + backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); + backup_entries_collector.addBackupEntry(data_path_in_backup_fs / data_file_name, std::move(backup_entry)); } /// __marks.mrk @@ -964,10 +965,10 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c String marks_file_name = fileName(marks_file_path); String hardlink_file_path = temp_dir / marks_file_name; disk->createHardLink(marks_file_path, hardlink_file_path); - backup_entries_collector.addBackupEntry( - data_path_in_backup_fs / marks_file_name, - std::make_unique( - disk, hardlink_file_path, read_settings, file_checker.getFileSize(marks_file_path), std::nullopt, temp_dir_owner)); + BackupEntryPtr backup_entry = std::make_unique( + disk, hardlink_file_path, read_settings, file_checker.getFileSize(marks_file_path)); + backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); + backup_entries_collector.addBackupEntry(data_path_in_backup_fs / marks_file_name, std::move(backup_entry)); } /// sizes.json diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index d54725b8b39..ddb55c119c4 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -551,10 +552,10 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec String data_file_name = fileName(data_file_path); String hardlink_file_path = temp_dir / data_file_name; disk->createHardLink(data_file_path, hardlink_file_path); - backup_entries_collector.addBackupEntry( - data_path_in_backup_fs / data_file_name, - std::make_unique( - disk, hardlink_file_path, read_settings, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner)); + BackupEntryPtr backup_entry = std::make_unique( + disk, hardlink_file_path, read_settings, file_checker.getFileSize(data_file_path)); + backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); + backup_entries_collector.addBackupEntry(data_path_in_backup_fs / data_file_name, std::move(backup_entry)); } /// index.mrk @@ -563,10 +564,10 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec String index_file_name = fileName(index_file_path); String hardlink_file_path = temp_dir / index_file_name; disk->createHardLink(index_file_path, hardlink_file_path); - backup_entries_collector.addBackupEntry( - data_path_in_backup_fs / index_file_name, - std::make_unique( - disk, hardlink_file_path, read_settings, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner)); + BackupEntryPtr backup_entry = std::make_unique( + disk, hardlink_file_path, read_settings, file_checker.getFileSize(index_file_path)); + backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); + backup_entries_collector.addBackupEntry(data_path_in_backup_fs / index_file_name, std::move(backup_entry)); } /// sizes.json From c92219f01b29c54c6a83b7b2ea63565ff67b7681 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 25 Apr 2023 19:44:03 +0200 Subject: [PATCH 0315/2223] BACKUP now writes encrypted data for tables on encrypted disks. --- src/Backups/BackupCoordinationRemote.cpp | 2 + src/Backups/BackupEntryFromAppendOnlyFile.h | 3 +- src/Backups/BackupEntryFromImmutableFile.cpp | 28 ++--- src/Backups/BackupEntryFromImmutableFile.h | 12 +- src/Backups/BackupEntryFromMemory.h | 7 -- src/Backups/BackupEntryFromSmallFile.cpp | 25 +++- src/Backups/BackupEntryFromSmallFile.h | 18 ++- src/Backups/BackupEntryWrappedWith.h | 5 +- src/Backups/BackupFileInfo.cpp | 2 + src/Backups/BackupFileInfo.h | 3 + src/Backups/BackupIO.cpp | 43 ++++--- src/Backups/BackupIO.h | 30 +++-- src/Backups/BackupIO_Disk.cpp | 46 +++---- src/Backups/BackupIO_Disk.h | 19 +-- src/Backups/BackupIO_File.cpp | 107 ++++++++-------- src/Backups/BackupIO_File.h | 18 +-- src/Backups/BackupIO_S3.cpp | 106 ++++++++++------ src/Backups/BackupIO_S3.h | 24 ++-- src/Backups/BackupImpl.cpp | 47 +++++-- src/Backups/BackupImpl.h | 2 + src/Backups/IBackupEntriesLazyBatch.cpp | 20 +-- src/Backups/IBackupEntry.h | 9 +- src/Common/ErrorCodes.cpp | 1 + src/Disks/DiskEncrypted.h | 24 ++-- src/Disks/DiskLocal.cpp | 32 +++-- src/Disks/DiskLocal.h | 3 +- src/Disks/DiskType.cpp | 5 + src/Disks/DiskType.h | 1 + src/Disks/IDisk.h | 27 ++-- src/Disks/IDiskTransaction.h | 2 +- .../ObjectStorages/DiskObjectStorage.cpp | 11 +- src/Disks/ObjectStorages/DiskObjectStorage.h | 2 +- .../DiskObjectStorageTransaction.cpp | 6 +- .../ObjectStorages/S3/copyS3FileToDisk.cpp | 117 ------------------ .../ObjectStorages/S3/copyS3FileToDisk.h | 52 -------- src/Storages/StorageLog.cpp | 1 + src/Storages/StorageMemory.cpp | 1 + src/Storages/StorageStripeLog.cpp | 1 + 38 files changed, 413 insertions(+), 449 deletions(-) delete mode 100644 src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp delete mode 100644 src/Disks/ObjectStorages/S3/copyS3FileToDisk.h diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index c1ef353c4eb..9a2fdf5dd6b 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -115,6 +115,7 @@ namespace writeBinary(info.checksum, out); writeBinary(info.base_size, out); writeBinary(info.base_checksum, out); + writeBinary(info.encrypted_by_disk, out); /// We don't store `info.data_file_name` and `info.data_file_index` because they're determined automalically /// after reading file infos for all the hosts (see the class BackupCoordinationFileInfos). } @@ -136,6 +137,7 @@ namespace readBinary(info.checksum, in); readBinary(info.base_size, in); readBinary(info.base_checksum, in); + readBinary(info.encrypted_by_disk, in); } return res; } diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.h b/src/Backups/BackupEntryFromAppendOnlyFile.h index b7a39c935a9..7c57e55923e 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.h +++ b/src/Backups/BackupEntryFromAppendOnlyFile.h @@ -11,7 +11,6 @@ namespace DB class BackupEntryFromAppendOnlyFile : public BackupEntryFromImmutableFile { public: - /// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data. BackupEntryFromAppendOnlyFile( const DiskPtr & disk_, @@ -23,6 +22,8 @@ public: UInt64 getSize() const override { return limit; } std::unique_ptr getReadBuffer() const override; + bool isFromImmutableFile() const override { return false; } + private: const UInt64 limit; }; diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index 790ea567496..7545134f638 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -16,9 +16,10 @@ BackupEntryFromImmutableFile::BackupEntryFromImmutableFile( const std::optional & checksum_) : disk(disk_) , file_path(file_path_) + , data_source_description(disk->getDataSourceDescription()) , settings(settings_) - , file_size(file_size_) - , checksum(checksum_) + , file_size(data_source_description.is_encrypted ? std::optional{} : file_size_) + , checksum(data_source_description.is_encrypted ? std::optional{} : checksum_) { } @@ -28,24 +29,21 @@ UInt64 BackupEntryFromImmutableFile::getSize() const { std::lock_guard lock{get_file_size_mutex}; if (!file_size) - file_size = disk->getFileSize(file_path); + { + if (data_source_description.is_encrypted) + file_size = disk->getEncryptedFileSize(file_path); + else + file_size = disk->getFileSize(file_path); + } return *file_size; } std::unique_ptr BackupEntryFromImmutableFile::getReadBuffer() const { - return disk->readFile(file_path, settings); -} - - -DataSourceDescription BackupEntryFromImmutableFile::getDataSourceDescription() const -{ - return disk->getDataSourceDescription(); -} - -String BackupEntryFromImmutableFile::getFilePath() const -{ - return file_path; + if (data_source_description.is_encrypted) + return disk->readEncryptedFile(file_path, settings); + else + return disk->readFile(file_path, settings); } } diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 4f2f902d31e..f2801b67df6 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -14,7 +14,6 @@ using DiskPtr = std::shared_ptr; class BackupEntryFromImmutableFile : public IBackupEntry { public: - /// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data. BackupEntryFromImmutableFile( const DiskPtr & disk_, @@ -28,15 +27,20 @@ public: UInt64 getSize() const override; std::optional getChecksum() const override { return checksum; } std::unique_ptr getReadBuffer() const override; + + bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; } - String getFilePath() const override; - DataSourceDescription getDataSourceDescription() const override; + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } - DiskPtr tryGetDiskIfExists() const override { return disk; } + bool isFromFile() const override { return true; } + bool isFromImmutableFile() const override { return true; } + DiskPtr getDisk() const override { return disk; } + String getFilePath() const override { return file_path; } private: const DiskPtr disk; const String file_path; + const DataSourceDescription data_source_description; ReadSettings settings; mutable std::optional file_size TSA_GUARDED_BY(get_file_size_mutex); mutable std::mutex get_file_size_mutex; diff --git a/src/Backups/BackupEntryFromMemory.h b/src/Backups/BackupEntryFromMemory.h index df3b9de40e3..64f46d68580 100644 --- a/src/Backups/BackupEntryFromMemory.h +++ b/src/Backups/BackupEntryFromMemory.h @@ -19,18 +19,11 @@ public: std::optional getChecksum() const override { return checksum; } std::unique_ptr getReadBuffer() const override; - String getFilePath() const override - { - return ""; - } - DataSourceDescription getDataSourceDescription() const override { return DataSourceDescription{DataSourceType::RAM, "", false, false}; } - DiskPtr tryGetDiskIfExists() const override { return nullptr; } - private: const String data; const std::optional checksum; diff --git a/src/Backups/BackupEntryFromSmallFile.cpp b/src/Backups/BackupEntryFromSmallFile.cpp index d24b3a6498d..6f7d2364031 100644 --- a/src/Backups/BackupEntryFromSmallFile.cpp +++ b/src/Backups/BackupEntryFromSmallFile.cpp @@ -1,6 +1,9 @@ #include +#include +#include #include #include +#include #include @@ -16,9 +19,9 @@ namespace return s; } - String readFile(const DiskPtr & disk, const String & file_path) + String readFile(const DiskPtr & disk, const String & file_path, bool read_encrypted) { - auto buf = disk->readFile(file_path); + auto buf = read_encrypted ? disk->readEncryptedFile(file_path) : disk->readFile(file_path); String s; readStringUntilEOF(s, *buf); return s; @@ -27,14 +30,26 @@ namespace BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_, const std::optional & checksum_) - : BackupEntryFromMemory(readFile(file_path_), checksum_), file_path(file_path_) + : file_path(file_path_) + , data_source_description(DiskLocal::getLocalDataSourceDescription(file_path_)) + , data(readFile(file_path_)) + , checksum(checksum_) { } - + BackupEntryFromSmallFile::BackupEntryFromSmallFile( const DiskPtr & disk_, const String & file_path_, const std::optional & checksum_) - : BackupEntryFromMemory(readFile(disk_, file_path_), checksum_), disk(disk_), file_path(file_path_) + : disk(disk_) + , file_path(file_path_) + , data_source_description(disk_->getDataSourceDescription()) + , data(readFile(disk_, file_path, data_source_description.is_encrypted)) + , checksum(data_source_description.is_encrypted ? std::optional{} : checksum_) { } +std::unique_ptr BackupEntryFromSmallFile::getReadBuffer() const +{ + return std::make_unique(data); +} + } diff --git a/src/Backups/BackupEntryFromSmallFile.h b/src/Backups/BackupEntryFromSmallFile.h index 99e319f07a0..2f7f3764571 100644 --- a/src/Backups/BackupEntryFromSmallFile.h +++ b/src/Backups/BackupEntryFromSmallFile.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -10,7 +10,7 @@ using DiskPtr = std::shared_ptr; /// Represents a file prepared to be included in a backup, /// assuming that the file is small and can be easily loaded into memory. -class BackupEntryFromSmallFile : public BackupEntryFromMemory +class BackupEntryFromSmallFile : public IBackupEntry { public: /// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data. @@ -23,12 +23,24 @@ public: const String & file_path_, const std::optional & checksum_ = {}); + UInt64 getSize() const override { return data.size(); } + std::optional getChecksum() const override { return checksum; } + std::unique_ptr getReadBuffer() const override; + + bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; } + + bool isFromFile() const override { return true; } + DiskPtr getDisk() const override { return disk; } String getFilePath() const override { return file_path; } - DiskPtr tryGetDiskIfExists() const override { return disk; } + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } + private: const DiskPtr disk; const String file_path; + const DataSourceDescription data_source_description; + const String data; + const std::optional checksum; }; } diff --git a/src/Backups/BackupEntryWrappedWith.h b/src/Backups/BackupEntryWrappedWith.h index da3b70e9ba9..933fec39634 100644 --- a/src/Backups/BackupEntryWrappedWith.h +++ b/src/Backups/BackupEntryWrappedWith.h @@ -18,8 +18,11 @@ public: UInt64 getSize() const override { return entry->getSize(); } std::optional getChecksum() const override { return entry->getChecksum(); } std::unique_ptr getReadBuffer() const override { return entry->getReadBuffer(); } + bool isEncryptedByDisk() const override { return entry->isEncryptedByDisk(); } + bool isFromFile() const override { return entry->isFromFile(); } + bool isFromImmutableFile() const override { return entry->isFromImmutableFile(); } String getFilePath() const override { return entry->getFilePath(); } - DiskPtr tryGetDiskIfExists() const override { return entry->tryGetDiskIfExists(); } + DiskPtr getDisk() const override { return entry->getDisk(); } DataSourceDescription getDataSourceDescription() const override { return entry->getDataSourceDescription(); } private: diff --git a/src/Backups/BackupFileInfo.cpp b/src/Backups/BackupFileInfo.cpp index 5a3076d1647..91ddc52ae44 100644 --- a/src/Backups/BackupFileInfo.cpp +++ b/src/Backups/BackupFileInfo.cpp @@ -111,6 +111,7 @@ String BackupFileInfo::describe() const result += fmt::format("base_checksum: {};\n", getHexUIntLowercase(checksum)); result += fmt::format("data_file_name: {};\n", data_file_name); result += fmt::format("data_file_index: {};\n", data_file_index); + result += fmt::format("encrypted_by_disk: {};\n", encrypted_by_disk); return result; } @@ -122,6 +123,7 @@ BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const Backu BackupFileInfo info; info.file_name = adjusted_path; info.size = backup_entry->getSize(); + info.encrypted_by_disk = backup_entry->isEncryptedByDisk(); /// We don't set `info.data_file_name` and `info.data_file_index` in this function because they're set during backup coordination /// (see the class BackupCoordinationFileInfos). diff --git a/src/Backups/BackupFileInfo.h b/src/Backups/BackupFileInfo.h index ae6ec83a37b..a925a1e81ac 100644 --- a/src/Backups/BackupFileInfo.h +++ b/src/Backups/BackupFileInfo.h @@ -35,6 +35,9 @@ struct BackupFileInfo /// This field is set during backup coordination (see the class BackupCoordinationFileInfos). size_t data_file_index = static_cast(-1); + /// Whether this file is encrypted by an encrypted disk. + bool encrypted_by_disk = false; + struct LessByFileName { bool operator()(const BackupFileInfo & lhs, const BackupFileInfo & rhs) const { return (lhs.file_name < rhs.file_name); } diff --git a/src/Backups/BackupIO.cpp b/src/Backups/BackupIO.cpp index 7b269bd965f..4d890ed7419 100644 --- a/src/Backups/BackupIO.cpp +++ b/src/Backups/BackupIO.cpp @@ -14,13 +14,18 @@ IBackupReader::IBackupReader(Poco::Logger * log_) : log(log_) { } -void IBackupReader::copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) +void IBackupReader::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) { - LOG_TRACE(log, "Copying file {} through buffers", file_name); - auto read_buffer = readFile(file_name); - auto write_buffer = destination_disk->writeFile(destination_path, std::min(size, DBMS_DEFAULT_BUFFER_SIZE), write_mode, write_settings); - copyData(*read_buffer, *write_buffer, size); + LOG_TRACE(log, "Copying file {} to disk {} through buffers", path_in_backup, destination_disk->getName()); + auto read_buffer = readFile(path_in_backup); + auto buf_size = std::min(file_size, DBMS_DEFAULT_BUFFER_SIZE); + std::unique_ptr write_buffer; + if (encrypted_in_backup) + write_buffer = destination_disk->writeEncryptedFile(destination_path, buf_size, write_mode, write_settings); + else + write_buffer = destination_disk->writeFile(destination_path, buf_size, write_mode, write_settings); + copyData(*read_buffer, *write_buffer, file_size); write_buffer->finalize(); } @@ -29,22 +34,28 @@ IBackupWriter::IBackupWriter(const ContextPtr & context_, Poco::Logger * log_) { } -void IBackupWriter::copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name) +void IBackupWriter::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { auto read_buffer = create_read_buffer(); - if (offset) - read_buffer->seek(offset, SEEK_SET); - auto write_buffer = writeFile(dest_file_name); - copyData(*read_buffer, *write_buffer, size); + if (start_pos) + read_buffer->seek(start_pos, SEEK_SET); + auto write_buffer = writeFile(path_in_backup); + copyData(*read_buffer, *write_buffer, length); write_buffer->finalize(); } -void IBackupWriter::copyFileFromDisk( - DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) +void IBackupWriter::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) { - LOG_TRACE(log, "Copying file {} through buffers", src_file_name); - auto create_read_buffer = [this, src_disk, src_file_name] { return src_disk->readFile(src_file_name, read_settings); }; - copyDataToFile(create_read_buffer, src_offset, src_size, dest_file_name); + LOG_TRACE(log, "Copying file {} from disk {} through buffers", src_path, src_disk->getName()); + auto create_read_buffer = [this, src_disk, src_path, copy_encrypted] + { + if (copy_encrypted) + return src_disk->readEncryptedFile(src_path, read_settings); + else + return src_disk->readFile(src_path, read_settings); + }; + copyDataToFile(path_in_backup, create_read_buffer, start_pos, length); } } diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index dae13422bf2..f65f0871698 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -16,14 +16,19 @@ class IBackupReader /// BackupReaderFile, BackupReaderDisk { public: explicit IBackupReader(Poco::Logger * log_); - virtual ~IBackupReader() = default; + virtual bool fileExists(const String & file_name) = 0; virtual UInt64 getFileSize(const String & file_name) = 0; + virtual std::unique_ptr readFile(const String & file_name) = 0; - virtual void copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings); - virtual DataSourceDescription getDataSourceDescription() const = 0; + + /// The function copyFileToDisk() can be much faster than reading the file with readFile() and then writing it to some disk. + /// (especially for S3 where it can use CopyObject to copy objects inside S3 instead of downloading and uploading them). + /// Parameters: + /// `encrypted_in_backup` specify if this file is encrypted in the backup, so it shouldn't be encrypted again while restoring to an encrypted disk. + virtual void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings); protected: Poco::Logger * const log; @@ -33,28 +38,29 @@ protected: class IBackupWriter /// BackupWriterFile, BackupWriterDisk { public: - using CreateReadBufferFunction = std::function()>; - IBackupWriter(const ContextPtr & context_, Poco::Logger * log_); - virtual ~IBackupWriter() = default; + virtual bool fileExists(const String & file_name) = 0; virtual UInt64 getFileSize(const String & file_name) = 0; virtual bool fileContentsEqual(const String & file_name, const String & expected_file_contents) = 0; virtual std::unique_ptr writeFile(const String & file_name) = 0; - virtual void copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name); + using CreateReadBufferFunction = std::function()>; + virtual void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length); - /// copyFileFromDisk() can be much faster than copyDataToFile() + /// The function copyFileFromDisk() can be much faster than copyDataToFile() /// (especially for S3 where it can use CopyObject to copy objects inside S3 instead of downloading and uploading them). - virtual void copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name); + /// Parameters: + /// `start_pos` and `length` specify a part of the file on `src_disk` to copy to the backup. + /// `copy_encrypted` specify whether this function should copy encrypted data of the file `src_path` to the backup. + virtual void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length); virtual void removeFile(const String & file_name) = 0; virtual void removeFiles(const Strings & file_names) = 0; - virtual DataSourceDescription getDataSourceDescription() const = 0; - protected: Poco::Logger * const log; diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index 643922cf3d0..d24d90ae7ae 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -36,24 +36,19 @@ std::unique_ptr BackupReaderDisk::readFile(const String & fi return disk->readFile(path / file_name); } -void BackupReaderDisk::copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) +void BackupReaderDisk::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) { - if ((write_mode == WriteMode::Rewrite) && (destination_disk->getDataSourceDescription() == getDataSourceDescription())) + if ((write_mode == WriteMode::Rewrite) && !encrypted_in_backup) { /// Use more optimal way. - LOG_TRACE(log, "Copying file {} using {} disk", file_name, toString(destination_disk->getDataSourceDescription().type)); - disk->copyFile(path / file_name, *destination_disk, destination_path, write_settings); + LOG_TRACE(log, "Copying file {} from disk {} to disk {}", path_in_backup, disk->getName(), destination_disk->getName()); + disk->copyFile(path / path_in_backup, *destination_disk, destination_path, write_settings); return; } /// Fallback to copy through buffers. - IBackupReader::copyFileToDisk(file_name, size, destination_disk, destination_path, write_mode, write_settings); -} - -DataSourceDescription BackupReaderDisk::getDataSourceDescription() const -{ - return disk->getDataSourceDescription(); + IBackupReader::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode, write_settings); } @@ -118,30 +113,21 @@ void BackupWriterDisk::removeFiles(const Strings & file_names) disk->removeDirectory(path); } -DataSourceDescription BackupWriterDisk::getDataSourceDescription() const +void BackupWriterDisk::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) { - return disk->getDataSourceDescription(); -} - -void BackupWriterDisk::copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) -{ - /// IDisk::copyFile() can copy to the same disk only, and it cannot do the throttling. - if (!has_throttling && (getDataSourceDescription() == src_disk->getDataSourceDescription())) + if (!copy_encrypted && !start_pos && (length == src_disk->getFileSize(src_path))) { - /// IDisk::copyFile() can copy a file as a whole only. - if ((src_offset == 0) && (src_size == src_disk->getFileSize(src_file_name))) - { - /// Use more optimal way. - LOG_TRACE(log, "Copying file {} using {} disk", src_file_name, toString(src_disk->getDataSourceDescription().type)); - auto dest_file_path = path / dest_file_name; - disk->createDirectories(dest_file_path.parent_path()); - src_disk->copyFile(src_file_name, *disk, dest_file_path); - return; - } + /// Use more optimal way. + LOG_TRACE(log, "Copying file {} from disk {} to disk {}", src_path, src_disk->getName(), disk->getName()); + auto dest_file_path = path / path_in_backup; + disk->createDirectories(dest_file_path.parent_path()); + src_disk->copyFile(src_path, *disk, dest_file_path); + return; } /// Fallback to copy through buffers. - IBackupWriter::copyFileFromDisk(src_disk, src_file_name, src_offset, src_size, dest_file_name); + IBackupWriter::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); } } diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index 884282ea4e0..d2af06668eb 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -18,13 +18,12 @@ public: bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; std::unique_ptr readFile(const String & file_name) override; - void copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) override; - DataSourceDescription getDataSourceDescription() const override; + void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) override; private: - DiskPtr disk; - std::filesystem::path path; + const DiskPtr disk; + const std::filesystem::path path; }; class BackupWriterDisk : public IBackupWriter @@ -37,14 +36,16 @@ public: UInt64 getFileSize(const String & file_name) override; bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; - void copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override; + + void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; - DataSourceDescription getDataSourceDescription() const override; private: - DiskPtr disk; - std::filesystem::path path; + const DiskPtr disk; + const std::filesystem::path path; const bool has_throttling; }; diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index b569e65284b..69d899528cd 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -15,7 +16,9 @@ namespace DB { BackupReaderFile::BackupReaderFile(const String & path_) - : IBackupReader(&Poco::Logger::get("BackupReaderFile")), path(path_) + : IBackupReader(&Poco::Logger::get("BackupReaderFile")) + , path(path_) + , data_source_description(DiskLocal::getLocalDataSourceDescription(path_)) { } @@ -36,25 +39,44 @@ std::unique_ptr BackupReaderFile::readFile(const String & fi return createReadBufferFromFileBase(path / file_name, {}); } -void BackupReaderFile::copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) +void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) { - if (destination_disk->getDataSourceDescription() == getDataSourceDescription()) + if (write_mode == WriteMode::Rewrite) { - /// Use more optimal way. - LOG_TRACE(log, "Copying file {} locally", file_name); - fs::copy(path / file_name, fullPath(destination_disk, destination_path), fs::copy_options::overwrite_existing); - return; + auto destination_data_source_description = destination_disk->getDataSourceDescription(); + if (destination_data_source_description.sameKind(data_source_description) + && (destination_data_source_description.is_encrypted == encrypted_in_backup)) + { + /// Use more optimal way. + LOG_TRACE(log, "Copying file {} to disk {} locally", path_in_backup, destination_disk->getName()); + + auto write_blob_function + = [abs_source_path = path / path_in_backup, file_size]( + const Strings & blob_path, WriteMode mode, const std::optional &) -> size_t + { + if (blob_path.size() != 1 || mode != WriteMode::Rewrite) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Blob writing function called with unexpected blob_path.size={} or mode={}", + blob_path.size(), mode); + fs::copy(abs_source_path, blob_path.at(0), fs::copy_options::overwrite_existing); + return file_size; + }; + + destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function); + return; + } } /// Fallback to copy through buffers. - IBackupReader::copyFileToDisk(path / file_name, size, destination_disk, destination_path, write_mode, write_settings); + IBackupReader::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode, write_settings); } BackupWriterFile::BackupWriterFile(const String & path_, const ContextPtr & context_) : IBackupWriter(context_, &Poco::Logger::get("BackupWriterFile")) , path(path_) + , data_source_description(DiskLocal::getLocalDataSourceDescription(path_)) , has_throttling(static_cast(context_->getBackupsThrottler())) { } @@ -112,59 +134,36 @@ void BackupWriterFile::removeFiles(const Strings & file_names) fs::remove(path); } -DataSourceDescription BackupWriterFile::getDataSourceDescription() const -{ - DataSourceDescription data_source_description; - - data_source_description.type = DataSourceType::Local; - - if (auto block_device_id = tryGetBlockDeviceId(path); block_device_id.has_value()) - data_source_description.description = *block_device_id; - else - data_source_description.description = path; - data_source_description.is_encrypted = false; - data_source_description.is_cached = false; - - return data_source_description; -} - -DataSourceDescription BackupReaderFile::getDataSourceDescription() const -{ - DataSourceDescription data_source_description; - - data_source_description.type = DataSourceType::Local; - - if (auto block_device_id = tryGetBlockDeviceId(path); block_device_id.has_value()) - data_source_description.description = *block_device_id; - else - data_source_description.description = path; - data_source_description.is_encrypted = false; - data_source_description.is_cached = false; - - return data_source_description; -} - - -void BackupWriterFile::copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) +void BackupWriterFile::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) { /// std::filesystem::copy() can copy from the filesystem only, and it cannot do the throttling. - if (!has_throttling && (getDataSourceDescription() == src_disk->getDataSourceDescription())) + if (!has_throttling) { - std::string abs_source_path = fullPath(src_disk, src_file_name); - /// std::filesystem::copy() can copy a file as a whole only. - if ((src_offset == 0) && (src_size == fs::file_size(abs_source_path))) + auto source_data_source_description = src_disk->getDataSourceDescription(); + if (source_data_source_description.sameKind(data_source_description) + && (source_data_source_description.is_encrypted == copy_encrypted)) { - /// Use more optimal way. - LOG_TRACE(log, "Copying file {} locally", src_file_name); - auto abs_dest_path = path / dest_file_name; - fs::create_directories(abs_dest_path.parent_path()); - fs::copy(abs_source_path, abs_dest_path, fs::copy_options::overwrite_existing); - return; + if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 1) + { + auto abs_source_path = blob_path[0]; + + /// std::filesystem::copy() can copy a file as a whole only. + if ((start_pos == 0) && (length == fs::file_size(abs_source_path))) + { + /// Use more optimal way. + LOG_TRACE(log, "Copying file {} from disk {} locally", src_path, src_disk->getName()); + auto abs_dest_path = path / path_in_backup; + fs::create_directories(abs_dest_path.parent_path()); + fs::copy(abs_source_path, abs_dest_path, fs::copy_options::overwrite_existing); + return; + } + } } } /// Fallback to copy through buffers. - IBackupWriter::copyFileFromDisk(src_disk, src_file_name, src_offset, src_size, dest_file_name); + IBackupWriter::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); } } diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index 1f0a247c321..a8ada969ca4 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -16,12 +16,13 @@ public: bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; std::unique_ptr readFile(const String & file_name) override; - void copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) override; - DataSourceDescription getDataSourceDescription() const override; + + void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) override; private: - std::filesystem::path path; + const std::filesystem::path path; + const DataSourceDescription data_source_description; }; class BackupWriterFile : public IBackupWriter @@ -34,13 +35,16 @@ public: UInt64 getFileSize(const String & file_name) override; bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; - void copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override; + + void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; - DataSourceDescription getDataSourceDescription() const override; private: - std::filesystem::path path; + const std::filesystem::path path; + const DataSourceDescription data_source_description; const bool has_throttling; }; diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index d2861500159..00694a5e01d 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -2,7 +2,6 @@ #if USE_AWS_S3 #include -#include #include #include #include @@ -107,16 +106,11 @@ BackupReaderS3::BackupReaderS3( , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , read_settings(context_->getReadSettings()) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) + , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} { request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint } -DataSourceDescription BackupReaderS3::getDataSourceDescription() const -{ - return DataSourceDescription{DataSourceType::S3, s3_uri.endpoint, false, false}; -} - - BackupReaderS3::~BackupReaderS3() = default; bool BackupReaderS3::fileExists(const String & file_name) @@ -138,23 +132,45 @@ std::unique_ptr BackupReaderS3::readFile(const String & file client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, request_settings, read_settings); } -void BackupReaderS3::copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) +void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) { - copyS3FileToDisk( - client, - s3_uri.bucket, - fs::path(s3_uri.key) / file_name, - s3_uri.version_id, - 0, - size, - destination_disk, - destination_path, - write_mode, - read_settings, - write_settings, - request_settings, - threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupReaderS3")); + auto destination_data_source_description = destination_disk->getDataSourceDescription(); + if (destination_data_source_description.sameKind(data_source_description) + && (destination_data_source_description.is_encrypted == encrypted_in_backup)) + { + /// Use native copy, the more optimal way. + LOG_TRACE(log, "Copying {} from S3 to disk {} using native copy", path_in_backup, destination_disk->getName()); + auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional & object_attributes) -> size_t + { + /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. + if (blob_path.size() != 2 || mode != WriteMode::Rewrite) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Blob writing function called with unexpected blob_path.size={} or mode={}", + blob_path.size(), mode); + + copyS3File( + client, + s3_uri.bucket, + fs::path(s3_uri.key) / path_in_backup, + 0, + file_size, + /* dest_bucket= */ blob_path[0], + /* dest_key= */ blob_path[1], + request_settings, + object_attributes, + threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupReaderS3"), + /* for_disk_s3= */ true); + + return file_size; + }; + + destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function); + return; + } + + /// Fallback to copy through buffers. + IBackupReader::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode, write_settings); } @@ -164,27 +180,47 @@ BackupWriterS3::BackupWriterS3( , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) + , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint } -DataSourceDescription BackupWriterS3::getDataSourceDescription() const +void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) { - return DataSourceDescription{DataSourceType::S3, s3_uri.endpoint, false, false}; + auto source_data_source_description = src_disk->getDataSourceDescription(); + if (source_data_source_description.sameKind(data_source_description) + && (source_data_source_description.is_encrypted == copy_encrypted)) + { + /// getBlobPath() can return std::nullopt if the file is stored as multiple objects in S3 bucket. + /// In this case we can't use native copy. + if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) + { + /// Use native copy, the more optimal way. + LOG_TRACE(log, "Copying file {} from disk {} to S3 using native copy", src_path, src_disk->getName()); + copyS3File( + client, + /* src_bucket */ blob_path[0], + /* src_key= */ blob_path[1], + start_pos, + length, + s3_uri.endpoint, + fs::path(s3_uri.key) / path_in_backup, + request_settings, + {}, + threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); + return; + } + } + + /// Fallback to copy through buffers. + IBackupWriter::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); } -void BackupWriterS3::copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) +void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { - copyS3FileFromDisk(src_disk, src_file_name, src_offset, src_size, - client, s3_uri.bucket, fs::path(s3_uri.key) / dest_file_name, read_settings, request_settings, - threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); -} - -void BackupWriterS3::copyDataToFile( - const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name) -{ - copyDataToS3File(create_read_buffer, offset, size, client, s3_uri.bucket, fs::path(s3_uri.key) / dest_file_name, request_settings, {}, + copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {}, threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); } diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 7d53d30e8d6..1db9d5f8c4c 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -23,15 +23,16 @@ public: bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; std::unique_ptr readFile(const String & file_name) override; - void copyFileToDisk(const String & file_name, size_t size, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) override; - DataSourceDescription getDataSourceDescription() const override; + + void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) override; private: - S3::URI s3_uri; - std::shared_ptr client; - ReadSettings read_settings; + const S3::URI s3_uri; + const std::shared_ptr client; + const ReadSettings read_settings; S3Settings::RequestSettings request_settings; + const DataSourceDescription data_source_description; }; @@ -46,9 +47,9 @@ public: bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; - void copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name) override; - void copyFileFromDisk(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override; - DataSourceDescription getDataSourceDescription() const override; + void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) override; + void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) override; void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; @@ -56,10 +57,11 @@ public: private: void removeFilesBatch(const Strings & file_names); - S3::URI s3_uri; - std::shared_ptr client; + const S3::URI s3_uri; + const std::shared_ptr client; S3Settings::RequestSettings request_settings; std::optional supports_batch_delete; + const DataSourceDescription data_source_description; }; } diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 55fb6dbfe03..9bfa2d77353 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -36,6 +36,7 @@ namespace ErrorCodes extern const int WRONG_BASE_BACKUP; extern const int BACKUP_ENTRY_NOT_FOUND; extern const int BACKUP_IS_EMPTY; + extern const int CANNOT_RESTORE_TO_NONENCRYPTED_DISK; extern const int FAILED_TO_SYNC_BACKUP_OR_RESTORE; extern const int LOGICAL_ERROR; } @@ -339,6 +340,8 @@ void BackupImpl::writeBackupMetadata() } if (!info.data_file_name.empty() && (info.data_file_name != info.file_name)) *out << "" << xml << info.data_file_name << ""; + if (info.encrypted_by_disk) + *out << "true"; } total_size += info.size; @@ -444,6 +447,7 @@ void BackupImpl::readBackupMetadata() { info.data_file_name = getString(file_config, "data_file", info.file_name); } + info.encrypted_by_disk = getBool(file_config, "encrypted_by_disk", false); } file_names.emplace(info.file_name, std::pair{info.size, info.checksum}); @@ -633,6 +637,11 @@ std::unique_ptr BackupImpl::readFile(const String & file_nam } std::unique_ptr BackupImpl::readFile(const SizeAndChecksum & size_and_checksum) const +{ + return readFileImpl(size_and_checksum, /* read_encrypted= */ false); +} + +std::unique_ptr BackupImpl::readFileImpl(const SizeAndChecksum & size_and_checksum, bool read_encrypted) const { if (open_mode != OpenMode::READ) throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup is not opened for reading"); @@ -660,6 +669,14 @@ std::unique_ptr BackupImpl::readFile(const SizeAndChecksum & info = it->second; } + if (info.encrypted_by_disk != read_encrypted) + { + throw Exception( + ErrorCodes::CANNOT_RESTORE_TO_NONENCRYPTED_DISK, + "File {} is encrypted in the backup, it can be restored only to an encrypted disk", + info.data_file_name); + } + std::unique_ptr read_buffer; std::unique_ptr base_read_buffer; @@ -760,14 +777,21 @@ size_t BackupImpl::copyFileToDisk(const SizeAndChecksum & size_and_checksum, Dis info = it->second; } + if (info.encrypted_by_disk && !destination_disk->getDataSourceDescription().is_encrypted) + { + throw Exception( + ErrorCodes::CANNOT_RESTORE_TO_NONENCRYPTED_DISK, + "File {} is encrypted in the backup, it can be restored only to an encrypted disk", + info.data_file_name); + } + bool file_copied = false; if (info.size && !info.base_size && !use_archive) { /// Data comes completely from this backup. - reader->copyFileToDisk(info.data_file_name, info.size, destination_disk, destination_path, write_mode, write_settings); + reader->copyFileToDisk(info.data_file_name, info.size, info.encrypted_by_disk, destination_disk, destination_path, write_mode, write_settings); file_copied = true; - } else if (info.size && (info.size == info.base_size)) { @@ -786,9 +810,13 @@ size_t BackupImpl::copyFileToDisk(const SizeAndChecksum & size_and_checksum, Dis else { /// Use the generic way to copy data. `readFile()` will update `num_read_files`. - auto read_buffer = readFile(size_and_checksum); - auto write_buffer = destination_disk->writeFile(destination_path, std::min(info.size, DBMS_DEFAULT_BUFFER_SIZE), - write_mode, write_settings); + auto read_buffer = readFileImpl(size_and_checksum, /* read_encrypted= */ info.encrypted_by_disk); + std::unique_ptr write_buffer; + size_t buf_size = std::min(info.size, DBMS_DEFAULT_BUFFER_SIZE); + if (info.encrypted_by_disk) + write_buffer = destination_disk->writeEncryptedFile(destination_path, buf_size, write_mode, write_settings); + else + write_buffer = destination_disk->writeFile(destination_path, buf_size, write_mode, write_settings); copyData(*read_buffer, *write_buffer, info.size); write_buffer->finalize(); } @@ -814,8 +842,9 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) should_check_lock_file = true; } - auto src_disk = entry->tryGetDiskIfExists(); + auto src_disk = entry->getDisk(); auto src_file_path = entry->getFilePath(); + bool from_immutable_file = entry->isFromImmutableFile(); String src_file_desc = src_file_path.empty() ? "memory buffer" : ("file " + src_file_path); if (info.data_file_name.empty()) @@ -845,16 +874,16 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) copyData(*read_buffer, *out); out->finalize(); } - else if (src_disk) + else if (src_disk && from_immutable_file) { LOG_TRACE(log, "Writing backup for file {} from {} (disk {}): data file #{}", info.data_file_name, src_file_desc, src_disk->getName(), info.data_file_index); - writer->copyFileFromDisk(src_disk, src_file_path, info.base_size, info.size - info.base_size, info.data_file_name); + writer->copyFileFromDisk(info.data_file_name, src_disk, src_file_path, info.encrypted_by_disk, info.base_size, info.size - info.base_size); } else { LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}", info.data_file_name, src_file_desc, info.data_file_index); auto create_read_buffer = [entry] { return entry->getReadBuffer(); }; - writer->copyDataToFile(create_read_buffer, info.base_size, info.size - info.base_size, info.data_file_name); + writer->copyDataToFile(info.data_file_name, create_read_buffer, info.base_size, info.size - info.base_size); } { diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index bf94926c46c..511b100c557 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -109,6 +109,8 @@ private: /// Calculates and sets `compressed_size`. void setCompressedSize(); + std::unique_ptr readFileImpl(const SizeAndChecksum & size_and_checksum, bool read_encrypted) const; + const String backup_name_for_logging; const bool use_archive; const ArchiveParams archive_params; diff --git a/src/Backups/IBackupEntriesLazyBatch.cpp b/src/Backups/IBackupEntriesLazyBatch.cpp index 78086015e7b..cd4d470967b 100644 --- a/src/Backups/IBackupEntriesLazyBatch.cpp +++ b/src/Backups/IBackupEntriesLazyBatch.cpp @@ -20,20 +20,12 @@ public: UInt64 getSize() const override { return getInternalBackupEntry()->getSize(); } std::optional getChecksum() const override { return getInternalBackupEntry()->getChecksum(); } std::unique_ptr getReadBuffer() const override { return getInternalBackupEntry()->getReadBuffer(); } - String getFilePath() const override - { - return getInternalBackupEntry()->getFilePath(); - } - - DiskPtr tryGetDiskIfExists() const override - { - return getInternalBackupEntry()->tryGetDiskIfExists(); - } - - DataSourceDescription getDataSourceDescription() const override - { - return getInternalBackupEntry()->getDataSourceDescription(); - } + bool isEncryptedByDisk() const override { return getInternalBackupEntry()->isEncryptedByDisk(); } + DataSourceDescription getDataSourceDescription() const override { return getInternalBackupEntry()->getDataSourceDescription(); } + bool isFromFile() const override { return getInternalBackupEntry()->isFromFile(); } + bool isFromImmutableFile() const override { return getInternalBackupEntry()->isFromImmutableFile(); } + String getFilePath() const override { return getInternalBackupEntry()->getFilePath(); } + DiskPtr getDisk() const override { return getInternalBackupEntry()->getDisk(); } private: BackupEntryPtr getInternalBackupEntry() const diff --git a/src/Backups/IBackupEntry.h b/src/Backups/IBackupEntry.h index 2a71a1e9756..7a93d4035df 100644 --- a/src/Backups/IBackupEntry.h +++ b/src/Backups/IBackupEntry.h @@ -27,9 +27,14 @@ public: /// Returns a read buffer for reading the data. virtual std::unique_ptr getReadBuffer() const = 0; - virtual String getFilePath() const = 0; + /// Returns true if the data returned by getReadBuffer() is encrypted by an encrypted disk. + virtual bool isEncryptedByDisk() const { return false; } - virtual DiskPtr tryGetDiskIfExists() const = 0; + /// Returns information about disk and file if this backup entry is generated from a file. + virtual bool isFromFile() const { return false; } + virtual bool isFromImmutableFile() const { return false; } + virtual String getFilePath() const { return ""; } + virtual DiskPtr getDisk() const { return nullptr; } virtual DataSourceDescription getDataSourceDescription() const = 0; }; diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 24e6114b26c..83a7314ac7a 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -579,6 +579,7 @@ M(694, ASYNC_LOAD_CYCLE) \ M(695, ASYNC_LOAD_FAILED) \ M(696, ASYNC_LOAD_CANCELED) \ + M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 9c9e61275ce..be726ef46b4 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -131,18 +131,6 @@ public: WriteMode mode, const WriteSettings & settings) override; - std::optional> getBlobPath(const String & path) const override - { - auto wrapped_path = wrappedPath(path); - return delegate->getBlobPath(wrapped_path); - } - - void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override - { - auto wrapped_path = wrappedPath(path); - delegate->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function)); - } - void removeFile(const String & path) override { auto wrapped_path = wrappedPath(path); @@ -198,6 +186,18 @@ public: delegate->removeSharedFileIfExists(wrapped_path, flag); } + Strings getBlobPath(const String & path) const override + { + auto wrapped_path = wrappedPath(path); + return delegate->getBlobPath(wrapped_path); + } + + void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override + { + auto wrapped_path = wrappedPath(path); + delegate->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function)); + } + std::unique_ptr readEncryptedFile( const String & path, const ReadSettings & settings) const override { diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index a40368fae88..c76ea289101 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -328,15 +328,16 @@ DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode, const fs::path(disk_path) / path, buf_size, flags, settings.local_throttler); } -std::optional> DiskLocal::getBlobPath(const String & path) const +std::vector DiskLocal::getBlobPath(const String & path) const { - return std::make_pair(fs::path(disk_path) / path, ""); + auto fs_path = fs::path(disk_path) / path; + return {fs_path}; } void DiskLocal::writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) { - auto blob_path = std::make_pair(fs::path(disk_path) / path, ""); - std::move(write_blob_function)(blob_path, mode, {}); + auto fs_path = fs::path(disk_path) / path; + std::move(write_blob_function)({fs_path}, mode, {}); } void DiskLocal::removeFile(const String & path) @@ -466,15 +467,8 @@ DiskLocal::DiskLocal(const String & name_, const String & path_, UInt64 keep_fre , disk_path(path_) , keep_free_space_bytes(keep_free_space_bytes_) , logger(&Poco::Logger::get("DiskLocal")) + , data_source_description(getLocalDataSourceDescription(disk_path)) { - data_source_description.type = DataSourceType::Local; - - if (auto block_device_id = tryGetBlockDeviceId(disk_path); block_device_id.has_value()) - data_source_description.description = *block_device_id; - else - data_source_description.description = disk_path; - data_source_description.is_encrypted = false; - data_source_description.is_cached = false; } DiskLocal::DiskLocal( @@ -490,6 +484,20 @@ DataSourceDescription DiskLocal::getDataSourceDescription() const return data_source_description; } +DataSourceDescription DiskLocal::getLocalDataSourceDescription(const String & path) +{ + DataSourceDescription res; + res.type = DataSourceType::Local; + + if (auto block_device_id = tryGetBlockDeviceId(path); block_device_id.has_value()) + res.description = *block_device_id; + else + res.description = path; + res.is_encrypted = false; + res.is_cached = false; + return res; +} + void DiskLocal::shutdown() { if (disk_checker) diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index b838654925d..3d340ae40b7 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -81,7 +81,7 @@ public: WriteMode mode, const WriteSettings & settings) override; - std::optional> getBlobPath(const String & path) const override; + Strings getBlobPath(const String & path) const override; void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override; void removeFile(const String & path) override; @@ -102,6 +102,7 @@ public: void truncateFile(const String & path, size_t size) override; DataSourceDescription getDataSourceDescription() const override; + static DataSourceDescription getLocalDataSourceDescription(const String & path); bool isRemote() const override { return false; } diff --git a/src/Disks/DiskType.cpp b/src/Disks/DiskType.cpp index 92979ab505c..aa18cc6e0cb 100644 --- a/src/Disks/DiskType.cpp +++ b/src/Disks/DiskType.cpp @@ -8,4 +8,9 @@ bool DataSourceDescription::operator==(const DataSourceDescription & other) cons return std::tie(type, description, is_encrypted) == std::tie(other.type, other.description, other.is_encrypted); } +bool DataSourceDescription::sameKind(const DataSourceDescription & other) const +{ + return std::tie(type, description) == std::tie(other.type, other.description); +} + } diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index 840ed5549e6..7d47fa8da78 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -51,6 +51,7 @@ struct DataSourceDescription bool is_cached = false; bool operator==(const DataSourceDescription & other) const; + bool sameKind(const DataSourceDescription & other) const; }; } diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index b98d8a74308..006f2d882a0 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -209,19 +209,6 @@ public: WriteMode mode = WriteMode::Rewrite, const WriteSettings & settings = {}) = 0; - /// Returns the path to a blob representing a specified file. - /// The meaning of the returned path depends on disk's type. - /// For DiskLocal it the absolute path to the file and for DiskObjectStorage it's the name of a namespace - /// combined with StoredObject::absolute_path. - virtual std::optional> getBlobPath(const String & path) const = 0; - - using WriteBlobFunction = std::function & blob_path, WriteMode mode, const std::optional & object_attributes)>; - - /// Write a file using a custom function to write a blob representing the file. - /// This method is alternative to writeFile(), the difference is that writeFile() calls IObjectStorage::writeObject() - /// to write an object to the object storage while this method allows to specify a callback for that. - virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; - /// Remove file. Throws exception if file doesn't exists or it's a directory. /// Return whether file was finally removed. (For remote disks it is not always removed). virtual void removeFile(const String & path) = 0; @@ -251,6 +238,20 @@ public: /// Second bool param is a flag to remove (true) or keep (false) shared data on S3 virtual void removeSharedFileIfExists(const String & path, bool /* keep_shared_data */) { removeFileIfExists(path); } + /// Returns the path to a blob representing a specified file. + /// The meaning of the returned path depends on disk's type. + /// E.g. for DiskLocal it the absolute path to the file and for DiskObjectStorage it's the name of the objects' namespace + /// combined with StoredObject::absolute_path for each stored object representing a specified file. + virtual Strings getBlobPath(const String & path) const = 0; + + using WriteBlobFunction = std::function & object_attributes)>; + + /// Write a file using a custom function to write a blob representing the file. + /// This method is alternative to writeFile(), the difference is that for example for DiskObjectStorage + /// writeFile() calls IObjectStorage::writeObject() to write an object to the object storage while + /// this method allows to specify a callback for that. + virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; + /// Reads a file from an encrypted disk without decrypting it. virtual std::unique_ptr readEncryptedFile( const String & path, const ReadSettings & settings = ReadSettings{}) const; diff --git a/src/Disks/IDiskTransaction.h b/src/Disks/IDiskTransaction.h index 376d7bd78e6..f0c32e04f48 100644 --- a/src/Disks/IDiskTransaction.h +++ b/src/Disks/IDiskTransaction.h @@ -68,7 +68,7 @@ public: const WriteSettings & settings = {}, bool autocommit = true) = 0; - using WriteBlobFunction = std::function & blob_path, WriteMode mode, const std::optional & object_attributes)>; + using WriteBlobFunction = std::function & object_attributes)>; /// Write a file using a custom function to write an object to the disk's object storage. virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index b01f1b327e7..c080c2cd92d 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -579,12 +579,15 @@ std::unique_ptr DiskObjectStorage::writeFile( return result; } -std::optional> DiskObjectStorage::getBlobPath(const String & path) const +Strings DiskObjectStorage::getBlobPath(const String & path) const { auto objects = getStorageObjects(path); - if (objects.size() != 1) - return {}; - return std::make_pair(object_storage->getObjectsNamespace(), objects[0].absolute_path); + Strings res; + res.reserve(objects.size() + 1); + res.emplace_back(object_storage->getObjectsNamespace()); + for (const auto & object : objects) + res.emplace_back(object.absolute_path); + return res; } void DiskObjectStorage::writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 97751edc3f5..b7dfaf67cf2 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -149,7 +149,7 @@ public: WriteMode mode, const WriteSettings & settings) override; - std::optional> getBlobPath(const String & path) const override; + Strings getBlobPath(const String & path) const override; void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override; void copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index f578bcb9772..f98ac55889b 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -691,7 +691,11 @@ void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction( operations_to_execute.emplace_back(std::move(write_operation)); - auto blob_path = std::make_pair(object_storage.getObjectsNamespace(), object.absolute_path); + /// See DiskObjectStorage::getBlobPath(). + Strings blob_path; + blob_path.reserve(2); + blob_path.emplace_back(object_storage.getObjectsNamespace()); + blob_path.emplace_back(object.absolute_path); /// We always use mode Rewrite because we simulate append using metadata and different files size_t object_size = std::move(write_blob_function)(blob_path, WriteMode::Rewrite, object_attributes); diff --git a/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp b/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp deleted file mode 100644 index e43d88b2519..00000000000 --- a/src/Disks/ObjectStorages/S3/copyS3FileToDisk.cpp +++ /dev/null @@ -1,117 +0,0 @@ -#include - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include - - -namespace DB -{ - -void copyS3FileToDisk( - const std::shared_ptr & src_s3_client, - const String & src_bucket, - const String & src_key, - const std::optional & version_id, - std::optional src_offset, - std::optional src_size, - DiskPtr destination_disk, - const String & destination_path, - WriteMode write_mode, - const ReadSettings & read_settings, - const WriteSettings & write_settings, - const S3Settings::RequestSettings & request_settings, - ThreadPoolCallbackRunner scheduler) -{ - if (!src_offset) - src_offset = 0; - - if (!src_size) - src_size = S3::getObjectSize(*src_s3_client, src_bucket, src_key, version_id.value_or(""), request_settings) - *src_offset; - - auto destination_data_source_description = destination_disk->getDataSourceDescription(); - if (destination_data_source_description == DataSourceDescription{DataSourceType::S3, src_s3_client->getInitialEndpoint(), false, false}) - { - /// Use native copy, the more optimal way. - LOG_TRACE(&Poco::Logger::get("copyS3FileToDisk"), "Copying {} to disk {} using native copy", src_key, destination_disk->getName()); - auto write_blob_function = [&](const std::pair & blob_path_, WriteMode write_mode_, const std::optional & object_attributes_) -> size_t - { - /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. - chassert(write_mode_ == WriteMode::Rewrite); - - copyS3File( - src_s3_client, - src_bucket, - src_key, - *src_offset, - *src_size, - /* dest_bucket= */ blob_path_.first, - /* dest_key= */ blob_path_.second, - request_settings, - object_attributes_, - scheduler, - /* for_disk_s3= */ true); - - return *src_size; - }; - - destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function); - return; - } - - /// Fallback to copy through buffers. - LOG_TRACE(&Poco::Logger::get("copyS3FileToDisk"), "Copying {} to disk {} through buffers", src_key, destination_disk->getName()); - ReadBufferFromS3 read_buffer{src_s3_client, src_bucket, src_key, {}, request_settings, read_settings}; - if (*src_offset) - read_buffer.seek(*src_offset, SEEK_SET); - auto write_buffer = destination_disk->writeFile(destination_path, std::min(*src_size, DBMS_DEFAULT_BUFFER_SIZE), write_mode, write_settings); - copyData(read_buffer, *write_buffer, *src_size); - write_buffer->finalize(); -} - -void copyS3FileFromDisk( - DiskPtr src_disk, - const String & src_path, - std::optional src_offset, - std::optional src_size, - const std::shared_ptr & dest_s3_client, - const String & dest_bucket, - const String & dest_key, - const ReadSettings & read_settings, - const S3Settings::RequestSettings & request_settings, - ThreadPoolCallbackRunner scheduler) -{ - if (!src_offset) - src_offset = 0; - - if (!src_size) - src_size = src_disk->getFileSize(src_path) - *src_offset; - - auto source_data_source_description = src_disk->getDataSourceDescription(); - if (source_data_source_description == DataSourceDescription{DataSourceType::S3, dest_s3_client->getInitialEndpoint(), false, false}) - { - /// getBlobPath() can return std::nullopt if the file is stored as multiple objects in S3 bucket. - /// In this case we can't use native copy. - if (auto blob_path = src_disk->getBlobPath(src_path)) - { - /// Use native copy, the more optimal way. - LOG_TRACE(&Poco::Logger::get("copyS3FileFromDisk"), "Copying file {} to S3 using native copy", src_path); - const auto & [src_bucket, src_key] = *blob_path; - copyS3File(dest_s3_client, src_bucket, src_key, *src_offset, *src_size, dest_bucket, dest_key, request_settings, {}, scheduler); - return; - } - } - - /// Fallback to copy through buffers. - LOG_TRACE(&Poco::Logger::get("copyS3FileFromDisk"), "Copying {} to S3 through buffers", src_path); - auto create_read_buffer = [src_disk, &src_path, &read_settings] { return src_disk->readFile(src_path, read_settings); }; - copyDataToS3File(create_read_buffer, *src_offset, *src_size, dest_s3_client, dest_bucket, dest_key, request_settings, {}, scheduler); -} - -} - -#endif diff --git a/src/Disks/ObjectStorages/S3/copyS3FileToDisk.h b/src/Disks/ObjectStorages/S3/copyS3FileToDisk.h deleted file mode 100644 index 78caf2f50c8..00000000000 --- a/src/Disks/ObjectStorages/S3/copyS3FileToDisk.h +++ /dev/null @@ -1,52 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include - - -namespace DB -{ - -/// Copies an object from S3 bucket to a disk of any type. -/// Depending on the disk the function can either do copying through buffers -/// (i.e. download the object by portions and then write those portions to the specified disk), -/// or perform a server-side copy. -void copyS3FileToDisk( - const std::shared_ptr & src_s3_client, - const String & src_bucket, - const String & src_key, - const std::optional & version_id, - std::optional src_offset, - std::optional src_size, - DiskPtr destination_disk, - const String & destination_path, - WriteMode write_mode = WriteMode::Rewrite, - const ReadSettings & read_settings = {}, - const WriteSettings & write_settings = {}, - const S3Settings::RequestSettings & request_settings = {}, - ThreadPoolCallbackRunner scheduler = {}); - -/// Copies an object from a disk of any type to S3 bucket. -/// Depending on the disk the function can either do copying through buffers -/// (i.e. read the object by portions and then upload those portions to the specified disk), -/// or perform a server-side copy. -void copyS3FileFromDisk( - DiskPtr src_disk, - const String & src_path, - std::optional src_offset, - std::optional src_size, - const std::shared_ptr & dest_s3_client, - const String & dest_bucket, - const String & dest_key, - const ReadSettings & read_settings = {}, - const S3Settings::RequestSettings & request_settings = {}, - ThreadPoolCallbackRunner scheduler = {}); - -} - -#endif diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 19887d6695e..31f499a7d96 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -27,6 +27,7 @@ #include #include +#include #include #include #include diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 37d9e3bc32c..68c888a2d23 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index ddb55c119c4..5b22db91631 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -31,6 +31,7 @@ #include #include +#include #include #include #include From 002fd19cb7833f1babc6345329d293ba6445155a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 28 Apr 2023 13:15:29 +0200 Subject: [PATCH 0316/2223] Move the common part of BackupIO_* to BackupIO_Default. --- src/Backups/BackupIO.cpp | 61 ------------ src/Backups/BackupIO.h | 39 ++++---- src/Backups/BackupIO_Default.cpp | 95 +++++++++++++++++++ src/Backups/BackupIO_Default.h | 73 ++++++++++++++ src/Backups/BackupIO_Disk.cpp | 70 ++++++-------- src/Backups/BackupIO_Disk.h | 25 ++--- src/Backups/BackupIO_File.cpp | 84 +++++++--------- src/Backups/BackupIO_File.h | 25 ++--- src/Backups/BackupIO_S3.cpp | 40 +++----- src/Backups/BackupIO_S3.h | 13 ++- src/Backups/BackupImpl.cpp | 20 ++-- src/Backups/BackupImpl.h | 6 +- src/Backups/IBackup.h | 4 +- .../registerBackupEnginesFileAndDisk.cpp | 4 +- src/Disks/DiskEncrypted.h | 6 +- src/Disks/IDisk.cpp | 2 +- src/Disks/IDisk.h | 5 +- .../ObjectStorages/DiskObjectStorage.cpp | 4 +- 18 files changed, 324 insertions(+), 252 deletions(-) delete mode 100644 src/Backups/BackupIO.cpp create mode 100644 src/Backups/BackupIO_Default.cpp create mode 100644 src/Backups/BackupIO_Default.h diff --git a/src/Backups/BackupIO.cpp b/src/Backups/BackupIO.cpp deleted file mode 100644 index 4d890ed7419..00000000000 --- a/src/Backups/BackupIO.cpp +++ /dev/null @@ -1,61 +0,0 @@ -#include - -#include -#include -#include -#include -#include - - -namespace DB -{ - -IBackupReader::IBackupReader(Poco::Logger * log_) : log(log_) -{ -} - -void IBackupReader::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) -{ - LOG_TRACE(log, "Copying file {} to disk {} through buffers", path_in_backup, destination_disk->getName()); - auto read_buffer = readFile(path_in_backup); - auto buf_size = std::min(file_size, DBMS_DEFAULT_BUFFER_SIZE); - std::unique_ptr write_buffer; - if (encrypted_in_backup) - write_buffer = destination_disk->writeEncryptedFile(destination_path, buf_size, write_mode, write_settings); - else - write_buffer = destination_disk->writeFile(destination_path, buf_size, write_mode, write_settings); - copyData(*read_buffer, *write_buffer, file_size); - write_buffer->finalize(); -} - -IBackupWriter::IBackupWriter(const ContextPtr & context_, Poco::Logger * log_) - : log(log_), read_settings(context_->getBackupReadSettings()) -{ -} - -void IBackupWriter::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) -{ - auto read_buffer = create_read_buffer(); - if (start_pos) - read_buffer->seek(start_pos, SEEK_SET); - auto write_buffer = writeFile(path_in_backup); - copyData(*read_buffer, *write_buffer, length); - write_buffer->finalize(); -} - -void IBackupWriter::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, - bool copy_encrypted, UInt64 start_pos, UInt64 length) -{ - LOG_TRACE(log, "Copying file {} from disk {} through buffers", src_path, src_disk->getName()); - auto create_read_buffer = [this, src_disk, src_path, copy_encrypted] - { - if (copy_encrypted) - return src_disk->readEncryptedFile(src_path, read_settings); - else - return src_disk->readFile(src_path, read_settings); - }; - copyDataToFile(path_in_backup, create_read_buffer, start_pos, length); -} - -} diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index f65f0871698..d522387deba 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -1,21 +1,23 @@ #pragma once #include -#include -#include -#include -#include + namespace DB { +class IDisk; +using DiskPtr = std::shared_ptr; class SeekableReadBuffer; class WriteBuffer; +enum class WriteMode; +struct WriteSettings; +struct ReadSettings; /// Represents operations of loading from disk or downloading for reading a backup. -class IBackupReader /// BackupReaderFile, BackupReaderDisk +/// See also implementations: BackupReaderFile, BackupReaderDisk. +class IBackupReader { public: - explicit IBackupReader(Poco::Logger * log_); virtual ~IBackupReader() = default; virtual bool fileExists(const String & file_name) = 0; @@ -28,17 +30,18 @@ public: /// Parameters: /// `encrypted_in_backup` specify if this file is encrypted in the backup, so it shouldn't be encrypted again while restoring to an encrypted disk. virtual void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings); - -protected: - Poco::Logger * const log; + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) = 0; + + virtual const ReadSettings & getReadSettings() const = 0; + virtual const WriteSettings & getWriteSettings() const = 0; + virtual size_t getWriteBufferSize() const = 0; }; /// Represents operations of storing to disk or uploading for writing a backup. -class IBackupWriter /// BackupWriterFile, BackupWriterDisk +/// See also implementations: BackupWriterFile, BackupWriterDisk +class IBackupWriter { public: - IBackupWriter(const ContextPtr & context_, Poco::Logger * log_); virtual ~IBackupWriter() = default; virtual bool fileExists(const String & file_name) = 0; @@ -48,7 +51,7 @@ public: virtual std::unique_ptr writeFile(const String & file_name) = 0; using CreateReadBufferFunction = std::function()>; - virtual void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length); + virtual void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) = 0; /// The function copyFileFromDisk() can be much faster than copyDataToFile() /// (especially for S3 where it can use CopyObject to copy objects inside S3 instead of downloading and uploading them). @@ -56,16 +59,14 @@ public: /// `start_pos` and `length` specify a part of the file on `src_disk` to copy to the backup. /// `copy_encrypted` specify whether this function should copy encrypted data of the file `src_path` to the backup. virtual void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, - bool copy_encrypted, UInt64 start_pos, UInt64 length); + bool copy_encrypted, UInt64 start_pos, UInt64 length) = 0; virtual void removeFile(const String & file_name) = 0; virtual void removeFiles(const Strings & file_names) = 0; -protected: - Poco::Logger * const log; - - /// These read settings are used to read from the source disk in copyFileFromDisk(). - const ReadSettings read_settings; + virtual const ReadSettings & getReadSettings() const = 0; + virtual const WriteSettings & getWriteSettings() const = 0; + virtual size_t getWriteBufferSize() const = 0; }; } diff --git a/src/Backups/BackupIO_Default.cpp b/src/Backups/BackupIO_Default.cpp new file mode 100644 index 00000000000..3b4851e9441 --- /dev/null +++ b/src/Backups/BackupIO_Default.cpp @@ -0,0 +1,95 @@ +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +BackupReaderDefault::BackupReaderDefault(Poco::Logger * log_, const ContextPtr & context_) + : log(log_) + , read_settings(context_->getBackupReadSettings()) + , write_settings(context_->getWriteSettings()) + , write_buffer_size(DBMS_DEFAULT_BUFFER_SIZE) +{ +} + +void BackupReaderDefault::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) +{ + LOG_TRACE(log, "Copying file {} to disk {} through buffers", path_in_backup, destination_disk->getName()); + + auto read_buffer = readFile(path_in_backup); + + std::unique_ptr write_buffer; + auto buf_size = std::min(file_size, write_buffer_size); + if (encrypted_in_backup) + write_buffer = destination_disk->writeEncryptedFile(destination_path, buf_size, write_mode, write_settings); + else + write_buffer = destination_disk->writeFile(destination_path, buf_size, write_mode, write_settings); + + copyData(*read_buffer, *write_buffer, file_size); + write_buffer->finalize(); +} + +BackupWriterDefault::BackupWriterDefault(Poco::Logger * log_, const ContextPtr & context_) + : log(log_) + , read_settings(context_->getBackupReadSettings()) + , write_settings(context_->getWriteSettings()) + , write_buffer_size(DBMS_DEFAULT_BUFFER_SIZE) +{ +} + +bool BackupWriterDefault::fileContentsEqual(const String & file_name, const String & expected_file_contents) +{ + if (!fileExists(file_name)) + return false; + + try + { + auto in = readFile(file_name, expected_file_contents.size()); + String actual_file_contents(expected_file_contents.size(), ' '); + return (in->read(actual_file_contents.data(), actual_file_contents.size()) == actual_file_contents.size()) + && (actual_file_contents == expected_file_contents) && in->eof(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + return false; + } +} + +void BackupWriterDefault::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) +{ + auto read_buffer = create_read_buffer(); + + if (start_pos) + read_buffer->seek(start_pos, SEEK_SET); + + auto write_buffer = writeFile(path_in_backup); + + copyData(*read_buffer, *write_buffer, length); + write_buffer->finalize(); +} + +void BackupWriterDefault::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) +{ + LOG_TRACE(log, "Copying file {} from disk {} through buffers", src_path, src_disk->getName()); + + auto create_read_buffer = [this, src_disk, src_path, file_size = start_pos + length, copy_encrypted] + { + if (copy_encrypted) + return src_disk->readEncryptedFile(src_path, read_settings, {}, file_size); + else + return src_disk->readFile(src_path, read_settings, {}, file_size); + }; + + copyDataToFile(path_in_backup, create_read_buffer, start_pos, length); +} +} diff --git a/src/Backups/BackupIO_Default.h b/src/Backups/BackupIO_Default.h new file mode 100644 index 00000000000..0fc510f9361 --- /dev/null +++ b/src/Backups/BackupIO_Default.h @@ -0,0 +1,73 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +class IDisk; +using DiskPtr = std::shared_ptr; +class ReadBuffer; +class SeekableReadBuffer; +class WriteBuffer; +enum class WriteMode; + +/// Represents operations of loading from disk or downloading for reading a backup. +class BackupReaderDefault : public IBackupReader +{ +public: + BackupReaderDefault(Poco::Logger * log_, const ContextPtr & context_); + ~BackupReaderDefault() override = default; + + /// The function copyFileToDisk() can be much faster than reading the file with readFile() and then writing it to some disk. + /// (especially for S3 where it can use CopyObject to copy objects inside S3 instead of downloading and uploading them). + /// Parameters: + /// `encrypted_in_backup` specify if this file is encrypted in the backup, so it shouldn't be encrypted again while restoring to an encrypted disk. + void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override; + + const ReadSettings & getReadSettings() const override { return read_settings; } + const WriteSettings & getWriteSettings() const override { return write_settings; } + size_t getWriteBufferSize() const override { return write_buffer_size; } + +protected: + Poco::Logger * const log; + const ReadSettings read_settings; + + /// The write settings are used to write to the source disk in copyFileToDisk(). + const WriteSettings write_settings; + const size_t write_buffer_size; +}; + +/// Represents operations of storing to disk or uploading for writing a backup. +class BackupWriterDefault : public IBackupWriter +{ +public: + BackupWriterDefault(Poco::Logger * log_, const ContextPtr & context_); + ~BackupWriterDefault() override = default; + + bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; + void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) override; + void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + + const ReadSettings & getReadSettings() const override { return read_settings; } + const WriteSettings & getWriteSettings() const override { return write_settings; } + size_t getWriteBufferSize() const override { return write_buffer_size; } + +protected: + /// Here readFile() is used only to implement fileContentsEqual(). + virtual std::unique_ptr readFile(const String & file_name, size_t expected_file_size) = 0; + + Poco::Logger * const log; + + /// The read settings are used to read from the source disk in copyFileFromDisk(). + const ReadSettings read_settings; + + const WriteSettings write_settings; + const size_t write_buffer_size; +}; + +} diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index d24d90ae7ae..f64e929131c 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB @@ -14,8 +13,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & path_) - : IBackupReader(&Poco::Logger::get("BackupReaderDisk")), disk(disk_), path(path_) +BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & root_path_, const ContextPtr & context_) + : BackupReaderDefault(&Poco::Logger::get("BackupReaderDisk"), context_) + , disk(disk_) + , root_path(root_path_) { } @@ -23,40 +24,39 @@ BackupReaderDisk::~BackupReaderDisk() = default; bool BackupReaderDisk::fileExists(const String & file_name) { - return disk->exists(path / file_name); + return disk->exists(root_path / file_name); } UInt64 BackupReaderDisk::getFileSize(const String & file_name) { - return disk->getFileSize(path / file_name); + return disk->getFileSize(root_path / file_name); } std::unique_ptr BackupReaderDisk::readFile(const String & file_name) { - return disk->readFile(path / file_name); + return disk->readFile(root_path / file_name, read_settings); } void BackupReaderDisk::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { if ((write_mode == WriteMode::Rewrite) && !encrypted_in_backup) { /// Use more optimal way. LOG_TRACE(log, "Copying file {} from disk {} to disk {}", path_in_backup, disk->getName(), destination_disk->getName()); - disk->copyFile(path / path_in_backup, *destination_disk, destination_path, write_settings); + disk->copyFile(root_path / path_in_backup, *destination_disk, destination_path, write_settings); return; } /// Fallback to copy through buffers. - IBackupReader::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode, write_settings); + BackupReaderDefault::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode); } -BackupWriterDisk::BackupWriterDisk(const DiskPtr & disk_, const String & path_, const ContextPtr & context_) - : IBackupWriter(context_, &Poco::Logger::get("BackupWriterDisk")) +BackupWriterDisk::BackupWriterDisk(const DiskPtr & disk_, const String & root_path_, const ContextPtr & context_) + : BackupWriterDefault(&Poco::Logger::get("BackupWriterDisk"), context_) , disk(disk_) - , path(path_) - , has_throttling(static_cast(context_->getBackupsThrottler())) + , root_path(root_path_) { } @@ -64,53 +64,39 @@ BackupWriterDisk::~BackupWriterDisk() = default; bool BackupWriterDisk::fileExists(const String & file_name) { - return disk->exists(path / file_name); + return disk->exists(root_path / file_name); } UInt64 BackupWriterDisk::getFileSize(const String & file_name) { - return disk->getFileSize(path / file_name); + return disk->getFileSize(root_path / file_name); } -bool BackupWriterDisk::fileContentsEqual(const String & file_name, const String & expected_file_contents) +std::unique_ptr BackupWriterDisk::readFile(const String & file_name, size_t expected_file_size) { - if (!disk->exists(path / file_name)) - return false; - - try - { - auto in = disk->readFile(path / file_name); - String actual_file_contents(expected_file_contents.size(), ' '); - return (in->read(actual_file_contents.data(), actual_file_contents.size()) == actual_file_contents.size()) - && (actual_file_contents == expected_file_contents) && in->eof(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - return false; - } + return disk->readFile(file_name, read_settings, {}, expected_file_size); } std::unique_ptr BackupWriterDisk::writeFile(const String & file_name) { - auto file_path = path / file_name; + auto file_path = root_path / file_name; disk->createDirectories(file_path.parent_path()); - return disk->writeFile(file_path); + return disk->writeFile(file_path, write_buffer_size, WriteMode::Rewrite, write_settings); } void BackupWriterDisk::removeFile(const String & file_name) { - disk->removeFileIfExists(path / file_name); - if (disk->isDirectory(path) && disk->isDirectoryEmpty(path)) - disk->removeDirectory(path); + disk->removeFileIfExists(root_path / file_name); + if (disk->isDirectory(root_path) && disk->isDirectoryEmpty(root_path)) + disk->removeDirectory(root_path); } void BackupWriterDisk::removeFiles(const Strings & file_names) { for (const auto & file_name : file_names) - disk->removeFileIfExists(path / file_name); - if (disk->isDirectory(path) && disk->isDirectoryEmpty(path)) - disk->removeDirectory(path); + disk->removeFileIfExists(root_path / file_name); + if (disk->isDirectory(root_path) && disk->isDirectoryEmpty(root_path)) + disk->removeDirectory(root_path); } void BackupWriterDisk::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, @@ -120,14 +106,14 @@ void BackupWriterDisk::copyFileFromDisk(const String & path_in_backup, DiskPtr s { /// Use more optimal way. LOG_TRACE(log, "Copying file {} from disk {} to disk {}", src_path, src_disk->getName(), disk->getName()); - auto dest_file_path = path / path_in_backup; + auto dest_file_path = root_path / path_in_backup; disk->createDirectories(dest_file_path.parent_path()); - src_disk->copyFile(src_path, *disk, dest_file_path); + src_disk->copyFile(src_path, *disk, dest_file_path, write_settings); return; } /// Fallback to copy through buffers. - IBackupWriter::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); + BackupWriterDefault::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); } } diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index d2af06668eb..399ebeaa227 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -1,40 +1,42 @@ #pragma once +#include #include -#include -#include + namespace DB { class IDisk; using DiskPtr = std::shared_ptr; -class BackupReaderDisk : public IBackupReader +class BackupReaderDisk : public BackupReaderDefault { public: - BackupReaderDisk(const DiskPtr & disk_, const String & path_); + BackupReaderDisk(const DiskPtr & disk_, const String & root_path_, const ContextPtr & context_); ~BackupReaderDisk() override; bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; + std::unique_ptr readFile(const String & file_name) override; + void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) override; + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override; private: const DiskPtr disk; - const std::filesystem::path path; + const std::filesystem::path root_path; }; -class BackupWriterDisk : public IBackupWriter +class BackupWriterDisk : public BackupWriterDefault { public: - BackupWriterDisk(const DiskPtr & disk_, const String & path_, const ContextPtr & context_); + BackupWriterDisk(const DiskPtr & disk_, const String & root_path_, const ContextPtr & context_); ~BackupWriterDisk() override; bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; - bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; + std::unique_ptr writeFile(const String & file_name) override; void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, @@ -44,9 +46,10 @@ public: void removeFiles(const Strings & file_names) override; private: + std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; + const DiskPtr disk; - const std::filesystem::path path; - const bool has_throttling; + const std::filesystem::path root_path; }; } diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 69d899528cd..69265799793 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -1,12 +1,11 @@ #include #include -#include +//#include #include #include -#include -#include +//#include +//#include #include -#include namespace fs = std::filesystem; @@ -15,32 +14,30 @@ namespace fs = std::filesystem; namespace DB { -BackupReaderFile::BackupReaderFile(const String & path_) - : IBackupReader(&Poco::Logger::get("BackupReaderFile")) - , path(path_) - , data_source_description(DiskLocal::getLocalDataSourceDescription(path_)) +BackupReaderFile::BackupReaderFile(const String & root_path_, const ContextPtr & context_) + : BackupReaderDefault(&Poco::Logger::get("BackupReaderFile"), context_) + , root_path(root_path_) + , data_source_description(DiskLocal::getLocalDataSourceDescription(root_path)) { } -BackupReaderFile::~BackupReaderFile() = default; - bool BackupReaderFile::fileExists(const String & file_name) { - return fs::exists(path / file_name); + return fs::exists(root_path / file_name); } UInt64 BackupReaderFile::getFileSize(const String & file_name) { - return fs::file_size(path / file_name); + return fs::file_size(root_path / file_name); } std::unique_ptr BackupReaderFile::readFile(const String & file_name) { - return createReadBufferFromFileBase(path / file_name, {}); + return createReadBufferFromFileBase(root_path / file_name, read_settings); } void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { if (write_mode == WriteMode::Rewrite) { @@ -52,7 +49,7 @@ void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file LOG_TRACE(log, "Copying file {} to disk {} locally", path_in_backup, destination_disk->getName()); auto write_blob_function - = [abs_source_path = path / path_in_backup, file_size]( + = [abs_source_path = root_path / path_in_backup, file_size]( const Strings & blob_path, WriteMode mode, const std::optional &) -> size_t { if (blob_path.size() != 1 || mode != WriteMode::Rewrite) @@ -69,69 +66,53 @@ void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file } /// Fallback to copy through buffers. - IBackupReader::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode, write_settings); + BackupReaderDefault::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode); } -BackupWriterFile::BackupWriterFile(const String & path_, const ContextPtr & context_) - : IBackupWriter(context_, &Poco::Logger::get("BackupWriterFile")) - , path(path_) - , data_source_description(DiskLocal::getLocalDataSourceDescription(path_)) - , has_throttling(static_cast(context_->getBackupsThrottler())) +BackupWriterFile::BackupWriterFile(const String & root_path_, const ContextPtr & context_) + : BackupWriterDefault(&Poco::Logger::get("BackupWriterFile"), context_) + , root_path(root_path_) + , data_source_description(DiskLocal::getLocalDataSourceDescription(root_path)) + , has_throttling(static_cast(read_settings.local_throttler)) { } -BackupWriterFile::~BackupWriterFile() = default; - bool BackupWriterFile::fileExists(const String & file_name) { - return fs::exists(path / file_name); + return fs::exists(root_path / file_name); } UInt64 BackupWriterFile::getFileSize(const String & file_name) { - return fs::file_size(path / file_name); + return fs::file_size(root_path / file_name); } -bool BackupWriterFile::fileContentsEqual(const String & file_name, const String & expected_file_contents) +std::unique_ptr BackupWriterFile::readFile(const String & file_name, size_t expected_file_size) { - if (!fs::exists(path / file_name)) - return false; - - try - { - auto in = createReadBufferFromFileBase(path / file_name, {}); - String actual_file_contents(expected_file_contents.size(), ' '); - return (in->read(actual_file_contents.data(), actual_file_contents.size()) == actual_file_contents.size()) - && (actual_file_contents == expected_file_contents) && in->eof(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - return false; - } + return createReadBufferFromFileBase(root_path / file_name, read_settings, {}, expected_file_size); } std::unique_ptr BackupWriterFile::writeFile(const String & file_name) { - auto file_path = path / file_name; + auto file_path = root_path / file_name; fs::create_directories(file_path.parent_path()); - return std::make_unique(file_path); + return std::make_unique(file_path, write_buffer_size, -1, write_settings.local_throttler); } void BackupWriterFile::removeFile(const String & file_name) { - fs::remove(path / file_name); - if (fs::is_directory(path) && fs::is_empty(path)) - fs::remove(path); + fs::remove(root_path / file_name); + if (fs::is_directory(root_path) && fs::is_empty(root_path)) + fs::remove(root_path); } void BackupWriterFile::removeFiles(const Strings & file_names) { for (const auto & file_name : file_names) - fs::remove(path / file_name); - if (fs::is_directory(path) && fs::is_empty(path)) - fs::remove(path); + fs::remove(root_path / file_name); + if (fs::is_directory(root_path) && fs::is_empty(root_path)) + fs::remove(root_path); } void BackupWriterFile::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, @@ -144,6 +125,7 @@ void BackupWriterFile::copyFileFromDisk(const String & path_in_backup, DiskPtr s if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) { + /// std::filesystem::copy() can copy from a single file only. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 1) { auto abs_source_path = blob_path[0]; @@ -153,7 +135,7 @@ void BackupWriterFile::copyFileFromDisk(const String & path_in_backup, DiskPtr s { /// Use more optimal way. LOG_TRACE(log, "Copying file {} from disk {} locally", src_path, src_disk->getName()); - auto abs_dest_path = path / path_in_backup; + auto abs_dest_path = root_path / path_in_backup; fs::create_directories(abs_dest_path.parent_path()); fs::copy(abs_source_path, abs_dest_path, fs::copy_options::overwrite_existing); return; @@ -163,7 +145,7 @@ void BackupWriterFile::copyFileFromDisk(const String & path_in_backup, DiskPtr s } /// Fallback to copy through buffers. - IBackupWriter::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); + BackupWriterDefault::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); } } diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index a8ada969ca4..45fc0d47115 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -1,39 +1,38 @@ #pragma once +#include +#include #include -#include -#include + namespace DB { -class BackupReaderFile : public IBackupReader +class BackupReaderFile : public BackupReaderDefault { public: - explicit BackupReaderFile(const String & path_); - ~BackupReaderFile() override; + explicit BackupReaderFile(const String & root_path_, const ContextPtr & context_); bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; + std::unique_ptr readFile(const String & file_name) override; void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) override; + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override; private: - const std::filesystem::path path; + const std::filesystem::path root_path; const DataSourceDescription data_source_description; }; -class BackupWriterFile : public IBackupWriter +class BackupWriterFile : public BackupWriterDefault { public: - BackupWriterFile(const String & path_, const ContextPtr & context_); - ~BackupWriterFile() override; + BackupWriterFile(const String & root_path_, const ContextPtr & context_); bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; - bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, @@ -43,7 +42,9 @@ public: void removeFiles(const Strings & file_names) override; private: - const std::filesystem::path path; + std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; + + const std::filesystem::path root_path; const DataSourceDescription data_source_description; const bool has_throttling; }; diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 00694a5e01d..8d9d34bf9b5 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -101,10 +102,9 @@ namespace BackupReaderS3::BackupReaderS3( const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) - : IBackupReader(&Poco::Logger::get("BackupReaderS3")) + : BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) - , read_settings(context_->getReadSettings()) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} { @@ -133,7 +133,7 @@ std::unique_ptr BackupReaderS3::readFile(const String & file } void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { auto destination_data_source_description = destination_disk->getDataSourceDescription(); if (destination_data_source_description.sameKind(data_source_description) @@ -170,13 +170,13 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s } /// Fallback to copy through buffers. - IBackupReader::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode, write_settings); + BackupReaderDefault::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode); } BackupWriterS3::BackupWriterS3( const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) - : IBackupWriter(context_, &Poco::Logger::get("BackupWriterS3")) + : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) @@ -193,8 +193,8 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) { - /// getBlobPath() can return std::nullopt if the file is stored as multiple objects in S3 bucket. - /// In this case we can't use native copy. + /// getBlobPath() can return more than 2 elements if the file is stored as multiple objects in S3 bucket. + /// In this case we can't use the native copy. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) { /// Use native copy, the more optimal way. @@ -215,7 +215,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src } /// Fallback to copy through buffers. - IBackupWriter::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); + BackupWriterDefault::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); } void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) @@ -239,24 +239,11 @@ UInt64 BackupWriterS3::getFileSize(const String & file_name) return objects[0].GetSize(); } -bool BackupWriterS3::fileContentsEqual(const String & file_name, const String & expected_file_contents) +std::unique_ptr BackupWriterS3::readFile(const String & file_name, size_t expected_file_size) { - if (listObjects(*client, s3_uri, file_name).empty()) - return false; - - try - { - auto in = std::make_unique( - client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, request_settings, read_settings); - String actual_file_contents(expected_file_contents.size(), ' '); - return (in->read(actual_file_contents.data(), actual_file_contents.size()) == actual_file_contents.size()) - && (actual_file_contents == expected_file_contents) && in->eof(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - return false; - } + return std::make_unique( + client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, request_settings, read_settings, + false, 0, 0, false, expected_file_size); } std::unique_ptr BackupWriterS3::writeFile(const String & file_name) @@ -267,7 +254,8 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) fs::path(s3_uri.key) / file_name, request_settings, std::nullopt, - threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); + threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3"), + write_settings); } void BackupWriterS3::removeFile(const String & file_name) diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 1db9d5f8c4c..cca56bae6bc 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -3,8 +3,8 @@ #include "config.h" #if USE_AWS_S3 -#include -#include +#include +#include #include #include #include @@ -14,7 +14,7 @@ namespace DB { /// Represents a backup stored to AWS S3. -class BackupReaderS3 : public IBackupReader +class BackupReaderS3 : public BackupReaderDefault { public: BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_); @@ -25,18 +25,17 @@ public: std::unique_ptr readFile(const String & file_name) override; void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode, const WriteSettings & write_settings) override; + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override; private: const S3::URI s3_uri; const std::shared_ptr client; - const ReadSettings read_settings; S3Settings::RequestSettings request_settings; const DataSourceDescription data_source_description; }; -class BackupWriterS3 : public IBackupWriter +class BackupWriterS3 : public BackupWriterDefault { public: BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_); @@ -44,7 +43,6 @@ public: bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; - bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) override; @@ -55,6 +53,7 @@ public: void removeFiles(const Strings & file_names) override; private: + std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; void removeFilesBatch(const Strings & file_names); const S3::URI s3_uri; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 9bfa2d77353..715fe9e0d13 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -737,14 +737,14 @@ std::unique_ptr BackupImpl::readFileImpl(const SizeAndChecks } } -size_t BackupImpl::copyFileToDisk(const String & file_name, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) const +size_t BackupImpl::copyFileToDisk(const String & file_name, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const { - return copyFileToDisk(getFileSizeAndChecksum(file_name), destination_disk, destination_path, write_mode, write_settings); + return copyFileToDisk(getFileSizeAndChecksum(file_name), destination_disk, destination_path, write_mode); } -size_t BackupImpl::copyFileToDisk(const SizeAndChecksum & size_and_checksum, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) const +size_t BackupImpl::copyFileToDisk(const SizeAndChecksum & size_and_checksum, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const { if (open_mode != OpenMode::READ) throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup is not opened for reading"); @@ -790,13 +790,13 @@ size_t BackupImpl::copyFileToDisk(const SizeAndChecksum & size_and_checksum, Dis if (info.size && !info.base_size && !use_archive) { /// Data comes completely from this backup. - reader->copyFileToDisk(info.data_file_name, info.size, info.encrypted_by_disk, destination_disk, destination_path, write_mode, write_settings); + reader->copyFileToDisk(info.data_file_name, info.size, info.encrypted_by_disk, destination_disk, destination_path, write_mode); file_copied = true; } else if (info.size && (info.size == info.base_size)) { /// Data comes completely from the base backup (nothing comes from this backup). - base_backup->copyFileToDisk(std::pair{info.base_size, info.base_checksum}, destination_disk, destination_path, write_mode, write_settings); + base_backup->copyFileToDisk(std::pair{info.base_size, info.base_checksum}, destination_disk, destination_path, write_mode); file_copied = true; } @@ -812,11 +812,11 @@ size_t BackupImpl::copyFileToDisk(const SizeAndChecksum & size_and_checksum, Dis /// Use the generic way to copy data. `readFile()` will update `num_read_files`. auto read_buffer = readFileImpl(size_and_checksum, /* read_encrypted= */ info.encrypted_by_disk); std::unique_ptr write_buffer; - size_t buf_size = std::min(info.size, DBMS_DEFAULT_BUFFER_SIZE); + size_t buf_size = std::min(info.size, reader->getWriteBufferSize()); if (info.encrypted_by_disk) - write_buffer = destination_disk->writeEncryptedFile(destination_path, buf_size, write_mode, write_settings); + write_buffer = destination_disk->writeEncryptedFile(destination_path, buf_size, write_mode, reader->getWriteSettings()); else - write_buffer = destination_disk->writeFile(destination_path, buf_size, write_mode, write_settings); + write_buffer = destination_disk->writeFile(destination_path, buf_size, write_mode, reader->getWriteSettings()); copyData(*read_buffer, *write_buffer, info.size); write_buffer->finalize(); } diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index 511b100c557..7e95d156162 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -76,10 +76,8 @@ public: SizeAndChecksum getFileSizeAndChecksum(const String & file_name) const override; std::unique_ptr readFile(const String & file_name) const override; std::unique_ptr readFile(const SizeAndChecksum & size_and_checksum) const override; - size_t copyFileToDisk(const String & file_name, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) const override; - size_t copyFileToDisk(const SizeAndChecksum & size_and_checksum, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode, const WriteSettings & write_settings) const override; + size_t copyFileToDisk(const String & file_name, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const override; + size_t copyFileToDisk(const SizeAndChecksum & size_and_checksum, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const override; void writeFile(const BackupFileInfo & info, BackupEntryPtr entry) override; void finalizeWriting() override; bool supportsWritingInMultipleThreads() const override { return !use_archive; } diff --git a/src/Backups/IBackup.h b/src/Backups/IBackup.h index 031130fa3b4..660f7d5da22 100644 --- a/src/Backups/IBackup.h +++ b/src/Backups/IBackup.h @@ -109,10 +109,10 @@ public: /// Copies a file from the backup to a specified destination disk. Returns the number of bytes written. virtual size_t copyFileToDisk(const String & file_name, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode = WriteMode::Rewrite, const WriteSettings & write_settings = {}) const = 0; + WriteMode write_mode = WriteMode::Rewrite) const = 0; virtual size_t copyFileToDisk(const SizeAndChecksum & size_and_checksum, DiskPtr destination_disk, const String & destination_path, - WriteMode write_mode = WriteMode::Rewrite, const WriteSettings & write_settings = {}) const = 0; + WriteMode write_mode = WriteMode::Rewrite) const = 0; /// Puts a new entry to the backup. virtual void writeFile(const BackupFileInfo & file_info, BackupEntryPtr entry) = 0; diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index 46f44471e6f..daae9627759 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -169,9 +169,9 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) { std::shared_ptr reader; if (engine_name == "File") - reader = std::make_shared(path); + reader = std::make_shared(path, params.context); else - reader = std::make_shared(disk, path); + reader = std::make_shared(disk, path, params.context); return std::make_unique(backup_name_for_logging, archive_params, params.base_backup_info, reader, params.context); } else diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index be726ef46b4..db53accc062 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -199,10 +199,12 @@ public: } std::unique_ptr readEncryptedFile( - const String & path, const ReadSettings & settings) const override + const String & path, const ReadSettings & settings, + std::optional read_hint, + std::optional file_size) const override { auto wrapped_path = wrappedPath(path); - return delegate->readFile(wrapped_path, settings); + return delegate->readFile(wrapped_path, settings, read_hint, file_size); } std::unique_ptr writeEncryptedFile( diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 54e09b09d2f..3c8ca454463 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -52,7 +52,7 @@ void IDisk::removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_ba } } -std::unique_ptr IDisk::readEncryptedFile(const String &, const ReadSettings &) const +std::unique_ptr IDisk::readEncryptedFile(const String &, const ReadSettings &, std::optional, std::optional) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "File encryption is not implemented for disk of type {}", getDataSourceDescription().type); } diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 006f2d882a0..770eec46ce6 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -254,7 +254,10 @@ public: /// Reads a file from an encrypted disk without decrypting it. virtual std::unique_ptr readEncryptedFile( - const String & path, const ReadSettings & settings = ReadSettings{}) const; + const String & path, + const ReadSettings & settings = ReadSettings{}, + std::optional read_hint = {}, + std::optional file_size = {}) const; /// Writes an already encrypted file to an encrypted disk. virtual std::unique_ptr writeEncryptedFile( diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index c080c2cd92d..bfa1ed1ab26 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -584,7 +584,9 @@ Strings DiskObjectStorage::getBlobPath(const String & path) const auto objects = getStorageObjects(path); Strings res; res.reserve(objects.size() + 1); - res.emplace_back(object_storage->getObjectsNamespace()); + String objects_namespace = object_storage->getObjectsNamespace(); + if (!objects_namespace.empty()) + res.emplace_back(objects_namespace); for (const auto & object : objects) res.emplace_back(object.absolute_path); return res; From 517e119e0385079b22531032ece33f4a8ed45977 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 1 May 2023 14:23:59 +0200 Subject: [PATCH 0317/2223] Move checksum calculation to IBackupEntry. --- src/Backups/BackupEntryFromAppendOnlyFile.cpp | 45 +++++++++++--- src/Backups/BackupEntryFromAppendOnlyFile.h | 26 +++++--- src/Backups/BackupEntryFromImmutableFile.cpp | 60 +++++++++++++------ src/Backups/BackupEntryFromImmutableFile.h | 23 +++---- src/Backups/BackupEntryFromMemory.cpp | 7 +-- src/Backups/BackupEntryFromMemory.h | 18 ++---- src/Backups/BackupEntryFromSmallFile.cpp | 9 +-- src/Backups/BackupEntryFromSmallFile.h | 22 ++----- .../BackupEntryWithChecksumCalculation.cpp | 48 +++++++++++++++ .../BackupEntryWithChecksumCalculation.h | 22 +++++++ src/Backups/BackupEntryWrappedWith.h | 7 ++- src/Backups/BackupFileInfo.cpp | 39 +++--------- src/Backups/BackupIO_File.cpp | 3 - src/Backups/IBackupEntriesLazyBatch.cpp | 7 ++- src/Backups/IBackupEntry.h | 9 ++- src/Disks/DiskEncrypted.cpp | 7 +++ src/Disks/DiskEncrypted.h | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 3 +- 18 files changed, 228 insertions(+), 129 deletions(-) create mode 100644 src/Backups/BackupEntryWithChecksumCalculation.cpp create mode 100644 src/Backups/BackupEntryWithChecksumCalculation.h diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index 83117f686bf..e3b79695a0c 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -1,25 +1,52 @@ #include +#include #include namespace DB { +namespace +{ + UInt64 calculateSize(const DiskPtr & disk, const String & file_path, const std::optional & file_size, bool disk_is_encrypted) + { + if (file_size) + { + if (disk_is_encrypted) + return DiskEncrypted::convertFileSizeToEncryptedFileSize(*file_size); + else + return *file_size; + } + else + { + if (disk_is_encrypted) + return disk->getEncryptedFileSize(file_path); + else + return disk->getFileSize(file_path); + } + } +} + BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile( - const DiskPtr & disk_, - const String & file_path_, - const ReadSettings & settings_, - const std::optional & file_size_, - const std::optional & checksum_) - : BackupEntryFromImmutableFile(disk_, file_path_, settings_, file_size_, checksum_) - , limit(BackupEntryFromImmutableFile::getSize()) + const DiskPtr & disk_, const String & file_path_, const ReadSettings & settings_, const std::optional & file_size_) + : disk(disk_) + , file_path(file_path_) + , data_source_description(disk->getDataSourceDescription()) + , settings(settings_) + , size(calculateSize(disk_, file_path_, file_size_, data_source_description.is_encrypted)) { } +BackupEntryFromAppendOnlyFile::~BackupEntryFromAppendOnlyFile() = default; + std::unique_ptr BackupEntryFromAppendOnlyFile::getReadBuffer() const { - auto buf = BackupEntryFromImmutableFile::getReadBuffer(); - return std::make_unique(std::move(buf), 0, limit); + std::unique_ptr buf; + if (data_source_description.is_encrypted) + buf = disk->readEncryptedFile(file_path, settings); + else + buf = disk->readFile(file_path, settings); + return std::make_unique(std::move(buf), 0, size); } } diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.h b/src/Backups/BackupEntryFromAppendOnlyFile.h index 7c57e55923e..e6814c4c345 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.h +++ b/src/Backups/BackupEntryFromAppendOnlyFile.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -8,24 +8,34 @@ namespace DB /// Represents a file prepared to be included in a backup, assuming that until this backup entry is destroyed /// the file can be appended with new data, but the bytes which are already in the file won't be changed. -class BackupEntryFromAppendOnlyFile : public BackupEntryFromImmutableFile +class BackupEntryFromAppendOnlyFile : public BackupEntryWithChecksumCalculation { public: - /// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data. + /// The constructor is allowed to not set `file_size_`, in that case it will be calculated from the data. BackupEntryFromAppendOnlyFile( const DiskPtr & disk_, const String & file_path_, const ReadSettings & settings_, - const std::optional & file_size_ = {}, - const std::optional & checksum_ = {}); + const std::optional & file_size_ = {}); + + ~BackupEntryFromAppendOnlyFile() override; - UInt64 getSize() const override { return limit; } std::unique_ptr getReadBuffer() const override; + UInt64 getSize() const override { return size; } - bool isFromImmutableFile() const override { return false; } + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } + bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; } + + bool isFromFile() const override { return true; } + DiskPtr getDisk() const override { return disk; } + String getFilePath() const override { return file_path; } private: - const UInt64 limit; + const DiskPtr disk; + const String file_path; + const DataSourceDescription data_source_description; + const ReadSettings settings; + const UInt64 size; }; } diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index 7545134f638..ad0ce477600 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -1,8 +1,6 @@ #include #include -#include -#include -#include +#include namespace DB @@ -18,26 +16,13 @@ BackupEntryFromImmutableFile::BackupEntryFromImmutableFile( , file_path(file_path_) , data_source_description(disk->getDataSourceDescription()) , settings(settings_) - , file_size(data_source_description.is_encrypted ? std::optional{} : file_size_) - , checksum(data_source_description.is_encrypted ? std::optional{} : checksum_) + , file_size(file_size_) + , checksum(checksum_) { } BackupEntryFromImmutableFile::~BackupEntryFromImmutableFile() = default; -UInt64 BackupEntryFromImmutableFile::getSize() const -{ - std::lock_guard lock{get_file_size_mutex}; - if (!file_size) - { - if (data_source_description.is_encrypted) - file_size = disk->getEncryptedFileSize(file_path); - else - file_size = disk->getFileSize(file_path); - } - return *file_size; -} - std::unique_ptr BackupEntryFromImmutableFile::getReadBuffer() const { if (data_source_description.is_encrypted) @@ -46,4 +31,43 @@ std::unique_ptr BackupEntryFromImmutableFile::getReadBuffer( return disk->readFile(file_path, settings); } +UInt64 BackupEntryFromImmutableFile::getSize() const +{ + std::lock_guard lock{size_and_checksum_mutex}; + if (!file_size_adjusted) + { + if (!file_size) + file_size = disk->getFileSize(file_path); + if (data_source_description.is_encrypted) + *file_size = DiskEncrypted::convertFileSizeToEncryptedFileSize(*file_size); + file_size_adjusted = true; + } + return *file_size; +} + +UInt128 BackupEntryFromImmutableFile::getChecksum() const +{ + std::lock_guard lock{size_and_checksum_mutex}; + if (!checksum_adjusted) + { + /// TODO: We should not just ignore `checksum` if `data_source_description.is_encrypted == true`, we should use it somehow. + if (!checksum || data_source_description.is_encrypted) + checksum = BackupEntryWithChecksumCalculation::getChecksum(); + checksum_adjusted = true; + } + return *checksum; +} + +std::optional BackupEntryFromImmutableFile::getPartialChecksum(size_t prefix_length) const +{ + if (prefix_length == 0) + return 0; + + if (prefix_length >= getSize()) + return getChecksum(); + + /// For immutable files we don't use partial checksums. + return std::nullopt; +} + } diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index f2801b67df6..23f48e8335a 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -11,7 +11,7 @@ class IDisk; using DiskPtr = std::shared_ptr; /// Represents a file prepared to be included in a backup, assuming that until this backup entry is destroyed the file won't be changed. -class BackupEntryFromImmutableFile : public IBackupEntry +class BackupEntryFromImmutableFile : public BackupEntryWithChecksumCalculation { public: /// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data. @@ -24,13 +24,14 @@ public: ~BackupEntryFromImmutableFile() override; - UInt64 getSize() const override; - std::optional getChecksum() const override { return checksum; } std::unique_ptr getReadBuffer() const override; - - bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; } + + UInt64 getSize() const override; + UInt128 getChecksum() const override; + std::optional getPartialChecksum(size_t prefix_length) const override; DataSourceDescription getDataSourceDescription() const override { return data_source_description; } + bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; } bool isFromFile() const override { return true; } bool isFromImmutableFile() const override { return true; } @@ -41,10 +42,12 @@ private: const DiskPtr disk; const String file_path; const DataSourceDescription data_source_description; - ReadSettings settings; - mutable std::optional file_size TSA_GUARDED_BY(get_file_size_mutex); - mutable std::mutex get_file_size_mutex; - const std::optional checksum; + const ReadSettings settings; + mutable std::optional file_size; + mutable std::optional checksum; + mutable bool file_size_adjusted = false; + mutable bool checksum_adjusted = false; + mutable std::mutex size_and_checksum_mutex; }; } diff --git a/src/Backups/BackupEntryFromMemory.cpp b/src/Backups/BackupEntryFromMemory.cpp index f59eadc2d7f..82e033caca0 100644 --- a/src/Backups/BackupEntryFromMemory.cpp +++ b/src/Backups/BackupEntryFromMemory.cpp @@ -5,13 +5,12 @@ namespace DB { -BackupEntryFromMemory::BackupEntryFromMemory(const void * data_, size_t size_, const std::optional & checksum_) - : BackupEntryFromMemory(String{reinterpret_cast(data_), size_}, checksum_) +BackupEntryFromMemory::BackupEntryFromMemory(const void * data_, size_t size_) + : BackupEntryFromMemory(String{reinterpret_cast(data_), size_}) { } -BackupEntryFromMemory::BackupEntryFromMemory(String data_, const std::optional & checksum_) - : data(std::move(data_)), checksum(checksum_) +BackupEntryFromMemory::BackupEntryFromMemory(String data_) : data(std::move(data_)) { } diff --git a/src/Backups/BackupEntryFromMemory.h b/src/Backups/BackupEntryFromMemory.h index 64f46d68580..19b72c16e53 100644 --- a/src/Backups/BackupEntryFromMemory.h +++ b/src/Backups/BackupEntryFromMemory.h @@ -1,32 +1,26 @@ #pragma once -#include -#include +#include namespace DB { /// Represents small preloaded data to be included in a backup. -class BackupEntryFromMemory : public IBackupEntry +class BackupEntryFromMemory : public BackupEntryWithChecksumCalculation { public: /// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data. - BackupEntryFromMemory(const void * data_, size_t size_, const std::optional & checksum_ = {}); - explicit BackupEntryFromMemory(String data_, const std::optional & checksum_ = {}); + BackupEntryFromMemory(const void * data_, size_t size_); + explicit BackupEntryFromMemory(String data_); - UInt64 getSize() const override { return data.size(); } - std::optional getChecksum() const override { return checksum; } std::unique_ptr getReadBuffer() const override; + UInt64 getSize() const override { return data.size(); } - DataSourceDescription getDataSourceDescription() const override - { - return DataSourceDescription{DataSourceType::RAM, "", false, false}; - } + DataSourceDescription getDataSourceDescription() const override { return DataSourceDescription{DataSourceType::RAM, "", false, false}; } private: const String data; - const std::optional checksum; }; } diff --git a/src/Backups/BackupEntryFromSmallFile.cpp b/src/Backups/BackupEntryFromSmallFile.cpp index 6f7d2364031..3662522c243 100644 --- a/src/Backups/BackupEntryFromSmallFile.cpp +++ b/src/Backups/BackupEntryFromSmallFile.cpp @@ -29,21 +29,18 @@ namespace } -BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_, const std::optional & checksum_) +BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_) : file_path(file_path_) , data_source_description(DiskLocal::getLocalDataSourceDescription(file_path_)) , data(readFile(file_path_)) - , checksum(checksum_) { } - -BackupEntryFromSmallFile::BackupEntryFromSmallFile( - const DiskPtr & disk_, const String & file_path_, const std::optional & checksum_) + +BackupEntryFromSmallFile::BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_) : disk(disk_) , file_path(file_path_) , data_source_description(disk_->getDataSourceDescription()) , data(readFile(disk_, file_path, data_source_description.is_encrypted)) - , checksum(data_source_description.is_encrypted ? std::optional{} : checksum_) { } diff --git a/src/Backups/BackupEntryFromSmallFile.h b/src/Backups/BackupEntryFromSmallFile.h index 2f7f3764571..6846ca71a55 100644 --- a/src/Backups/BackupEntryFromSmallFile.h +++ b/src/Backups/BackupEntryFromSmallFile.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -10,37 +10,27 @@ using DiskPtr = std::shared_ptr; /// Represents a file prepared to be included in a backup, /// assuming that the file is small and can be easily loaded into memory. -class BackupEntryFromSmallFile : public IBackupEntry +class BackupEntryFromSmallFile : public BackupEntryWithChecksumCalculation { public: - /// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data. - explicit BackupEntryFromSmallFile( - const String & file_path_, - const std::optional & checksum_ = {}); + explicit BackupEntryFromSmallFile(const String & file_path_); + BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_); - BackupEntryFromSmallFile( - const DiskPtr & disk_, - const String & file_path_, - const std::optional & checksum_ = {}); - - UInt64 getSize() const override { return data.size(); } - std::optional getChecksum() const override { return checksum; } std::unique_ptr getReadBuffer() const override; + UInt64 getSize() const override { return data.size(); } + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; } bool isFromFile() const override { return true; } DiskPtr getDisk() const override { return disk; } String getFilePath() const override { return file_path; } - DataSourceDescription getDataSourceDescription() const override { return data_source_description; } - private: const DiskPtr disk; const String file_path; const DataSourceDescription data_source_description; const String data; - const std::optional checksum; }; } diff --git a/src/Backups/BackupEntryWithChecksumCalculation.cpp b/src/Backups/BackupEntryWithChecksumCalculation.cpp new file mode 100644 index 00000000000..bc97a907cff --- /dev/null +++ b/src/Backups/BackupEntryWithChecksumCalculation.cpp @@ -0,0 +1,48 @@ +#include +#include + + +namespace DB +{ + +template +UInt128 BackupEntryWithChecksumCalculation::getChecksum() const +{ + std::lock_guard lock{checksum_calculation_mutex}; + if (!calculated_checksum) + { + auto read_buffer = this->getReadBuffer(); + HashingReadBuffer hashing_read_buffer(*read_buffer); + hashing_read_buffer.ignoreAll(); + calculated_checksum = hashing_read_buffer.getHash(); + } + return *calculated_checksum; +} + +template +std::optional BackupEntryWithChecksumCalculation::getPartialChecksum(size_t prefix_length) const +{ + if (prefix_length == 0) + return 0; + + if (prefix_length >= this->getSize()) + return this->getChecksum(); + + auto read_buffer = this->getReadBuffer(); + HashingReadBuffer hashing_read_buffer(*read_buffer); + hashing_read_buffer.ignore(prefix_length); + auto partial_checksum = hashing_read_buffer.getHash(); + + std::lock_guard lock{checksum_calculation_mutex}; + if (!calculated_checksum) + { + hashing_read_buffer.ignoreAll(); + calculated_checksum = hashing_read_buffer.getHash(); + } + + return partial_checksum; +} + +template class BackupEntryWithChecksumCalculation; + +} diff --git a/src/Backups/BackupEntryWithChecksumCalculation.h b/src/Backups/BackupEntryWithChecksumCalculation.h new file mode 100644 index 00000000000..32701ab9952 --- /dev/null +++ b/src/Backups/BackupEntryWithChecksumCalculation.h @@ -0,0 +1,22 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Calculates the checksum and the partial checksum for a backup entry based on ReadBuffer returned by getReadBuffer(). +template +class BackupEntryWithChecksumCalculation : public Base +{ +public: + UInt128 getChecksum() const override; + std::optional getPartialChecksum(size_t prefix_length) const override; + +private: + mutable std::optional calculated_checksum; + mutable std::mutex checksum_calculation_mutex; +}; + +} diff --git a/src/Backups/BackupEntryWrappedWith.h b/src/Backups/BackupEntryWrappedWith.h index 933fec39634..01b6163039b 100644 --- a/src/Backups/BackupEntryWrappedWith.h +++ b/src/Backups/BackupEntryWrappedWith.h @@ -15,15 +15,16 @@ public: BackupEntryWrappedWith(BackupEntryPtr entry_, T && custom_value_) : entry(entry_), custom_value(std::move(custom_value_)) { } ~BackupEntryWrappedWith() override = default; - UInt64 getSize() const override { return entry->getSize(); } - std::optional getChecksum() const override { return entry->getChecksum(); } std::unique_ptr getReadBuffer() const override { return entry->getReadBuffer(); } + UInt64 getSize() const override { return entry->getSize(); } + UInt128 getChecksum() const override { return entry->getChecksum(); } + std::optional getPartialChecksum(size_t prefix_length) const override { return entry->getPartialChecksum(prefix_length); } + DataSourceDescription getDataSourceDescription() const override { return entry->getDataSourceDescription(); } bool isEncryptedByDisk() const override { return entry->isEncryptedByDisk(); } bool isFromFile() const override { return entry->isFromFile(); } bool isFromImmutableFile() const override { return entry->isFromImmutableFile(); } String getFilePath() const override { return entry->getFilePath(); } DiskPtr getDisk() const override { return entry->getDisk(); } - DataSourceDescription getDataSourceDescription() const override { return entry->getDataSourceDescription(); } private: BackupEntryPtr entry; diff --git a/src/Backups/BackupFileInfo.cpp b/src/Backups/BackupFileInfo.cpp index 91ddc52ae44..42546d1b1b8 100644 --- a/src/Backups/BackupFileInfo.cpp +++ b/src/Backups/BackupFileInfo.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB @@ -49,44 +49,19 @@ namespace struct ChecksumsForNewEntry { UInt128 full_checksum; - UInt128 prefix_checksum; + std::optional prefix_checksum; }; /// Calculate checksum for backup entry if it's empty. /// Also able to calculate additional checksum of some prefix. ChecksumsForNewEntry calculateNewEntryChecksumsIfNeeded(const BackupEntryPtr & entry, size_t prefix_size) { + ChecksumsForNewEntry res; + /// The partial checksum should be calculated before the full checksum to enable optimization in BackupEntryWithChecksumCalculation. if (prefix_size > 0) - { - auto read_buffer = entry->getReadBuffer(); - HashingReadBuffer hashing_read_buffer(*read_buffer); - hashing_read_buffer.ignore(prefix_size); - auto prefix_checksum = hashing_read_buffer.getHash(); - if (entry->getChecksum() == std::nullopt) - { - hashing_read_buffer.ignoreAll(); - auto full_checksum = hashing_read_buffer.getHash(); - return ChecksumsForNewEntry{full_checksum, prefix_checksum}; - } - else - { - return ChecksumsForNewEntry{*(entry->getChecksum()), prefix_checksum}; - } - } - else - { - if (entry->getChecksum() == std::nullopt) - { - auto read_buffer = entry->getReadBuffer(); - HashingReadBuffer hashing_read_buffer(*read_buffer); - hashing_read_buffer.ignoreAll(); - return ChecksumsForNewEntry{hashing_read_buffer.getHash(), 0}; - } - else - { - return ChecksumsForNewEntry{*(entry->getChecksum()), 0}; - } - } + res.prefix_checksum = entry->getPartialChecksum(prefix_size); + res.full_checksum = entry->getChecksum(); + return res; } /// We store entries' file names in the backup without leading slashes. diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 69265799793..4268f653e30 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -1,10 +1,7 @@ #include #include -//#include #include #include -//#include -//#include #include diff --git a/src/Backups/IBackupEntriesLazyBatch.cpp b/src/Backups/IBackupEntriesLazyBatch.cpp index cd4d470967b..8913d60f2eb 100644 --- a/src/Backups/IBackupEntriesLazyBatch.cpp +++ b/src/Backups/IBackupEntriesLazyBatch.cpp @@ -17,11 +17,12 @@ class IBackupEntriesLazyBatch::BackupEntryFromBatch : public IBackupEntry public: BackupEntryFromBatch(const std::shared_ptr & batch_, size_t index_) : batch(batch_), index(index_) { } - UInt64 getSize() const override { return getInternalBackupEntry()->getSize(); } - std::optional getChecksum() const override { return getInternalBackupEntry()->getChecksum(); } std::unique_ptr getReadBuffer() const override { return getInternalBackupEntry()->getReadBuffer(); } - bool isEncryptedByDisk() const override { return getInternalBackupEntry()->isEncryptedByDisk(); } + UInt64 getSize() const override { return getInternalBackupEntry()->getSize(); } + UInt128 getChecksum() const override { return getInternalBackupEntry()->getChecksum(); } + std::optional getPartialChecksum(size_t prefix_length) const override { return getInternalBackupEntry()->getPartialChecksum(prefix_length); } DataSourceDescription getDataSourceDescription() const override { return getInternalBackupEntry()->getDataSourceDescription(); } + bool isEncryptedByDisk() const override { return getInternalBackupEntry()->isEncryptedByDisk(); } bool isFromFile() const override { return getInternalBackupEntry()->isFromFile(); } bool isFromImmutableFile() const override { return getInternalBackupEntry()->isFromImmutableFile(); } String getFilePath() const override { return getInternalBackupEntry()->getFilePath(); } diff --git a/src/Backups/IBackupEntry.h b/src/Backups/IBackupEntry.h index 7a93d4035df..249c290cbe7 100644 --- a/src/Backups/IBackupEntry.h +++ b/src/Backups/IBackupEntry.h @@ -20,9 +20,12 @@ public: /// Returns the size of the data. virtual UInt64 getSize() const = 0; - /// Returns the checksum of the data if it's precalculated. - /// Can return nullopt which means the checksum should be calculated from the read buffer. - virtual std::optional getChecksum() const { return {}; } + /// Returns the checksum of the data. + virtual UInt128 getChecksum() const = 0; + + /// Returns a partial checksum, i.e. the checksum calculated for a prefix part of the data. + /// Can return nullopt if the partial checksum is too difficult to calculate. + virtual std::optional getPartialChecksum(size_t /* prefix_length */) const { return {}; } /// Returns a read buffer for reading the data. virtual std::unique_ptr getReadBuffer() const = 0; diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index db18e9652e7..15f314dcfc9 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -348,6 +348,13 @@ size_t DiskEncrypted::getFileSize(const String & path) const return size > FileEncryption::Header::kSize ? (size - FileEncryption::Header::kSize) : 0; } +size_t DiskEncrypted::convertFileSizeToEncryptedFileSize(size_t file_size) +{ + if (file_size) + return file_size + FileEncryption::Header::kSize; + return 0; +} + void DiskEncrypted::truncateFile(const String & path, size_t size) { auto wrapped_path = wrappedPath(path); diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index db53accc062..295e70a2c66 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -223,6 +223,8 @@ public: return delegate->getFileSize(wrapped_path); } + static size_t convertFileSizeToEncryptedFileSize(size_t file_size); + void setLastModified(const String & path, const Poco::Timestamp & timestamp) override { auto wrapped_path = wrappedPath(path); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1dde057569e..fba6e6cbafc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9278,10 +9278,9 @@ void StorageReplicatedMergeTree::backupData( auto & hash = part_names_with_hashes_calculating[part_name]; if (relative_path.ends_with(".bin")) { - auto checksum = backup_entry->getChecksum(); hash.update(relative_path); hash.update(backup_entry->getSize()); - hash.update(*checksum); + hash.update(backup_entry->getChecksum()); } continue; } From c48c20fac83a0f7014a97e74fa81ce8ab49bdd1d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 1 May 2023 18:43:20 +0200 Subject: [PATCH 0318/2223] Use combined checksums for encrypted immutable files. --- src/Backups/BackupEntryFromAppendOnlyFile.cpp | 25 +++++++------------ src/Backups/BackupEntryFromAppendOnlyFile.h | 2 +- src/Backups/BackupEntryFromImmutableFile.cpp | 22 ++++++++++++---- src/Backups/BackupIO_Disk.cpp | 2 +- src/Backups/BackupIO_File.cpp | 2 +- src/Backups/BackupIO_S3.cpp | 2 +- src/Disks/DiskEncrypted.cpp | 16 +++++++++--- src/Disks/DiskEncrypted.h | 4 +++ src/Disks/IDisk.cpp | 10 ++++++++ src/Disks/IDisk.h | 10 +++++--- .../test_backup_restore_s3/test.py | 12 ++++----- 11 files changed, 70 insertions(+), 37 deletions(-) diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index e3b79695a0c..2c28abc2e2f 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -8,22 +8,15 @@ namespace DB namespace { - UInt64 calculateSize(const DiskPtr & disk, const String & file_path, const std::optional & file_size, bool disk_is_encrypted) + /// For append-only files we must calculate its size on the construction of a backup entry. + UInt64 calculateSize(const DiskPtr & disk, const String & file_path, bool is_encrypted, std::optional unencrypted_file_size) { - if (file_size) - { - if (disk_is_encrypted) - return DiskEncrypted::convertFileSizeToEncryptedFileSize(*file_size); - else - return *file_size; - } + if (!unencrypted_file_size) + return is_encrypted ? disk->getEncryptedFileSize(file_path) : disk->getFileSize(file_path); + else if (is_encrypted) + return disk->getEncryptedFileSize(*unencrypted_file_size); else - { - if (disk_is_encrypted) - return disk->getEncryptedFileSize(file_path); - else - return disk->getFileSize(file_path); - } + return *unencrypted_file_size; } } @@ -32,8 +25,8 @@ BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile( : disk(disk_) , file_path(file_path_) , data_source_description(disk->getDataSourceDescription()) - , settings(settings_) - , size(calculateSize(disk_, file_path_, file_size_, data_source_description.is_encrypted)) + , size(calculateSize(disk_, file_path_, data_source_description.is_encrypted, file_size_)) + , settings(settings_.adjustBufferSize(size)) { } diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.h b/src/Backups/BackupEntryFromAppendOnlyFile.h index e6814c4c345..fac8c319d8c 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.h +++ b/src/Backups/BackupEntryFromAppendOnlyFile.h @@ -34,8 +34,8 @@ private: const DiskPtr disk; const String file_path; const DataSourceDescription data_source_description; - const ReadSettings settings; const UInt64 size; + const ReadSettings settings; }; } diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index ad0ce477600..f377e54d9b0 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -6,6 +6,17 @@ namespace DB { +namespace +{ + /// We mix the checksum calculated for non-encrypted data with IV generated to encrypt the file + /// to generate kind of a checksum for encrypted data. Of course it differs from the CityHash properly calculated for encrypted data. + UInt128 combineChecksums(UInt128 checksum1, UInt128 checksum2) + { + chassert(std::size(checksum2.items) == 2); + return CityHash_v1_0_2::CityHash128WithSeed(reinterpret_cast(&checksum1), sizeof(checksum1), {checksum2.items[0], checksum2.items[1]}); + } +} + BackupEntryFromImmutableFile::BackupEntryFromImmutableFile( const DiskPtr & disk_, const String & file_path_, @@ -37,9 +48,9 @@ UInt64 BackupEntryFromImmutableFile::getSize() const if (!file_size_adjusted) { if (!file_size) - file_size = disk->getFileSize(file_path); - if (data_source_description.is_encrypted) - *file_size = DiskEncrypted::convertFileSizeToEncryptedFileSize(*file_size); + file_size = data_source_description.is_encrypted ? disk->getEncryptedFileSize(file_path) : disk->getFileSize(file_path); + else if (data_source_description.is_encrypted) + file_size = disk->getEncryptedFileSize(*file_size); file_size_adjusted = true; } return *file_size; @@ -50,9 +61,10 @@ UInt128 BackupEntryFromImmutableFile::getChecksum() const std::lock_guard lock{size_and_checksum_mutex}; if (!checksum_adjusted) { - /// TODO: We should not just ignore `checksum` if `data_source_description.is_encrypted == true`, we should use it somehow. - if (!checksum || data_source_description.is_encrypted) + if (!checksum) checksum = BackupEntryWithChecksumCalculation::getChecksum(); + else if (data_source_description.is_encrypted) + checksum = combineChecksums(*checksum, disk->getEncryptedFileIV(file_path)); checksum_adjusted = true; } return *checksum; diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index f64e929131c..1e1af63cdc2 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -74,7 +74,7 @@ UInt64 BackupWriterDisk::getFileSize(const String & file_name) std::unique_ptr BackupWriterDisk::readFile(const String & file_name, size_t expected_file_size) { - return disk->readFile(file_name, read_settings, {}, expected_file_size); + return disk->readFile(root_path / file_name, read_settings.adjustBufferSize(expected_file_size)); } std::unique_ptr BackupWriterDisk::writeFile(const String & file_name) diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 4268f653e30..649637e567d 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -87,7 +87,7 @@ UInt64 BackupWriterFile::getFileSize(const String & file_name) std::unique_ptr BackupWriterFile::readFile(const String & file_name, size_t expected_file_size) { - return createReadBufferFromFileBase(root_path / file_name, read_settings, {}, expected_file_size); + return createReadBufferFromFileBase(root_path / file_name, read_settings.adjustBufferSize(expected_file_size)); } std::unique_ptr BackupWriterFile::writeFile(const String & file_name) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 8d9d34bf9b5..09d2bdbcfaf 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -205,7 +205,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src /* src_key= */ blob_path[1], start_pos, length, - s3_uri.endpoint, + s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {}, diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 15f314dcfc9..72f668db00b 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -348,10 +348,20 @@ size_t DiskEncrypted::getFileSize(const String & path) const return size > FileEncryption::Header::kSize ? (size - FileEncryption::Header::kSize) : 0; } -size_t DiskEncrypted::convertFileSizeToEncryptedFileSize(size_t file_size) +UInt128 DiskEncrypted::getEncryptedFileIV(const String & path) const { - if (file_size) - return file_size + FileEncryption::Header::kSize; + auto wrapped_path = wrappedPath(path); + auto read_buffer = delegate->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize)); + if (read_buffer->eof()) + return 0; + auto header = readHeader(*read_buffer); + return header.init_vector.get(); +} + +size_t DiskEncrypted::getEncryptedFileSize(size_t unencrypted_size) const +{ + if (unencrypted_size) + return unencrypted_size + FileEncryption::Header::kSize; return 0; } diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 295e70a2c66..c494dd6a216 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -223,6 +223,10 @@ public: return delegate->getFileSize(wrapped_path); } + size_t getEncryptedFileSize(size_t unencrypted_size) const override; + + UInt128 getEncryptedFileIV(const String & path) const override; + static size_t convertFileSizeToEncryptedFileSize(size_t file_size); void setLastModified(const String & path, const Poco::Timestamp & timestamp) override diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 3c8ca454463..9a5ae997b46 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -67,6 +67,16 @@ size_t IDisk::getEncryptedFileSize(const String &) const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "File encryption is not implemented for disk of type {}", getDataSourceDescription().type); } +size_t IDisk::getEncryptedFileSize(size_t) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "File encryption is not implemented for disk of type {}", getDataSourceDescription().type); +} + +UInt128 IDisk::getEncryptedFileIV(const String &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "File encryption is not implemented for disk of type {}", getDataSourceDescription().type); +} + using ResultsCollector = std::vector>; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 770eec46ce6..47b4ed80ebf 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -252,22 +252,26 @@ public: /// this method allows to specify a callback for that. virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; - /// Reads a file from an encrypted disk without decrypting it. + /// Reads a file from an encrypted disk without decrypting it (only for encrypted disks). virtual std::unique_ptr readEncryptedFile( const String & path, const ReadSettings & settings = ReadSettings{}, std::optional read_hint = {}, std::optional file_size = {}) const; - /// Writes an already encrypted file to an encrypted disk. + /// Writes an already encrypted file to the disk (only for encrypted disks). virtual std::unique_ptr writeEncryptedFile( const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, WriteMode mode = WriteMode::Rewrite, const WriteSettings & settings = {}) const; - /// Returns the size of encrypted file on an encrypted disk. + /// Returns the size of an encrypted file (only for encrypted disks). virtual size_t getEncryptedFileSize(const String & path) const; + virtual size_t getEncryptedFileSize(size_t unencrypted_size) const; + + /// Returns IV of an encrypted file (only for encrypted disks). + virtual UInt128 getEncryptedFileIV(const String & path) const; virtual const String & getCacheName() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no cache"); } diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index b5ac34f0b46..2f60575b634 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -141,8 +141,8 @@ def test_backup_to_s3_native_copy(): f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("copyS3FileFromDisk.*using native copy") - assert node.contains_in_log("copyS3FileToDisk.*using native copy") + assert node.contains_in_log("BackupWriterS3.*using native copy") + assert node.contains_in_log("BackupReaderS3.*using native copy") assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" ) @@ -155,8 +155,8 @@ def test_backup_to_s3_native_copy_other_bucket(): f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("copyS3FileFromDisk.*using native copy") - assert node.contains_in_log("copyS3FileToDisk.*using native copy") + assert node.contains_in_log("BackupWriterS3.*using native copy") + assert node.contains_in_log("BackupReaderS3.*using native copy") assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" ) @@ -167,8 +167,8 @@ def test_backup_to_s3_native_copy_multipart(): backup_name = new_backup_name() backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" check_backup_and_restore(storage_policy, backup_destination, size=1000000) - assert node.contains_in_log("copyS3FileFromDisk.*using native copy") - assert node.contains_in_log("copyS3FileToDisk.*using native copy") + assert node.contains_in_log("BackupWriterS3.*using native copy") + assert node.contains_in_log("BackupReaderS3.*using native copy") assert node.contains_in_log( f"copyS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}/" ) From d0fd81fa27514ff18001a25bd2a3e2b631ca9e0d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 2 May 2023 01:55:56 +0200 Subject: [PATCH 0319/2223] Add tests. --- .../configs/allow_backup_path.xml | 5 + tests/integration/test_encrypted_disk/test.py | 126 +++++++++++++++++- 2 files changed, 130 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_encrypted_disk/configs/allow_backup_path.xml diff --git a/tests/integration/test_encrypted_disk/configs/allow_backup_path.xml b/tests/integration/test_encrypted_disk/configs/allow_backup_path.xml new file mode 100644 index 00000000000..eade3bfb744 --- /dev/null +++ b/tests/integration/test_encrypted_disk/configs/allow_backup_path.xml @@ -0,0 +1,5 @@ + + + /backups/ + + \ No newline at end of file diff --git a/tests/integration/test_encrypted_disk/test.py b/tests/integration/test_encrypted_disk/test.py index 8187f2ff6a8..90da5849c7f 100644 --- a/tests/integration/test_encrypted_disk/test.py +++ b/tests/integration/test_encrypted_disk/test.py @@ -1,6 +1,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException +import os.path from helpers.test_tools import assert_eq_with_retry @@ -9,8 +10,9 @@ FIRST_PART_NAME = "all_1_1_0" cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", - main_configs=["configs/storage.xml"], + main_configs=["configs/storage.xml", "configs/allow_backup_path.xml"], tmpfs=["/disk:size=100M"], + external_dirs=["/backups/"], with_minio=True, stay_alive=True, ) @@ -33,6 +35,15 @@ def cleanup_after_test(): node.query("DROP TABLE IF EXISTS encrypted_test SYNC") +backup_id_counter = 0 + + +def new_backup_name(): + global backup_id_counter + backup_id_counter += 1 + return f"backup{backup_id_counter}" + + @pytest.mark.parametrize( "policy", ["encrypted_policy", "encrypted_policy_key192b", "local_policy", "s3_policy"], @@ -295,3 +306,116 @@ def test_restart(): assert node.query(select_query) == "(0,'data'),(1,'data')" node.query("DROP TABLE encrypted_test SYNC;") + + +@pytest.mark.parametrize( + "storage_policy,backup_type,storage_policy2", + [ + ("encrypted_policy", "S3", "encrypted_policy"), + ("encrypted_policy", "S3", "s3_encrypted_default_path"), + ("s3_encrypted_default_path", "S3", "s3_encrypted_default_path"), + ("s3_encrypted_default_path", "S3", "encrypted_policy"), + ("s3_encrypted_default_path", "File", "encrypted_policy"), + ("local_policy", "File", "encrypted_policy"), + ], +) +def test_backup_restore(storage_policy, backup_type, storage_policy2): + node.query( + f""" + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='{storage_policy}' + """ + ) + + node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')") + select_query = "SELECT * FROM encrypted_test ORDER BY id FORMAT Values" + assert node.query(select_query) == "(0,'data'),(1,'data')" + + backup_name = new_backup_name() + if backup_type == "S3": + backup_destination = ( + f"S3('http://minio1:9001/root/backups/{backup_name}', 'minio', 'minio123')" + ) + elif backup_type == "File": + backup_destination = f"File('/backups/{backup_name}/')" + + node.query(f"BACKUP TABLE encrypted_test TO {backup_destination}") + + if backup_type == "File" and storage_policy.find("encrypted") != -1: + root_path = os.path.join(node.cluster.instances_dir, "backups", backup_name) + with open( + f"{root_path}/data/default/encrypted_test/all_1_1_0/data.bin", "rb" + ) as file: + assert file.read().startswith(b"ENC") + with open(f"{root_path}/metadata/default/encrypted_test.sql") as file: + assert file.read().startswith("CREATE TABLE default.encrypted_test") + with open(f"{root_path}/.backup") as file: + assert file.read().find("true") != -1 + + node.query(f"DROP TABLE encrypted_test SYNC") + + if storage_policy != storage_policy2: + node.query( + f""" + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='{storage_policy2}' + """ + ) + + node.query( + f"RESTORE TABLE encrypted_test FROM {backup_destination} SETTINGS allow_different_table_def={int(storage_policy != storage_policy2)}" + ) + + assert node.query(select_query) == "(0,'data'),(1,'data')" + + +def test_cannot_restore_encrypted_files_to_unencrypted_disk(): + node.query( + """ + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='encrypted_policy' + """ + ) + + node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')") + assert ( + node.query("SELECT * FROM encrypted_test ORDER BY id FORMAT Values") + == "(0,'data'),(1,'data')" + ) + + backup_name = new_backup_name() + backup_destination = ( + f"S3('http://minio1:9001/root/backups/{backup_name}', 'minio', 'minio123')" + ) + node.query(f"BACKUP TABLE encrypted_test TO {backup_destination}") + + node.query(f"DROP TABLE encrypted_test SYNC") + + node.query( + f""" + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='local_policy' + """ + ) + + expected_error = "can be restored only to an encrypted disk" + assert expected_error in node.query_and_get_error( + f"RESTORE TABLE encrypted_test FROM {backup_destination} SETTINGS allow_different_table_def=1" + ) +>>>>>>> 9c08fb30995 (Add tests.) From 7cea26423010a09ef6777c17ac210cd11cfb14c7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 3 May 2023 12:33:35 +0200 Subject: [PATCH 0320/2223] Fix whitespaces. --- src/Backups/BackupIO.h | 4 ++-- src/Backups/BackupIO_Default.h | 4 ++-- src/Backups/BackupIO_Disk.cpp | 5 ----- src/Backups/BackupIO_File.cpp | 5 +++++ src/Backups/IBackupEntry.h | 2 +- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index d522387deba..e4a82a604e8 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -28,10 +28,10 @@ public: /// The function copyFileToDisk() can be much faster than reading the file with readFile() and then writing it to some disk. /// (especially for S3 where it can use CopyObject to copy objects inside S3 instead of downloading and uploading them). /// Parameters: - /// `encrypted_in_backup` specify if this file is encrypted in the backup, so it shouldn't be encrypted again while restoring to an encrypted disk. + /// `encrypted_in_backup` specify if this file is encrypted in the backup, so it shouldn't be encrypted again while restoring to an encrypted disk. virtual void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) = 0; - + virtual const ReadSettings & getReadSettings() const = 0; virtual const WriteSettings & getWriteSettings() const = 0; virtual size_t getWriteBufferSize() const = 0; diff --git a/src/Backups/BackupIO_Default.h b/src/Backups/BackupIO_Default.h index 0fc510f9361..ad7bdf15d9f 100644 --- a/src/Backups/BackupIO_Default.h +++ b/src/Backups/BackupIO_Default.h @@ -25,10 +25,10 @@ public: /// The function copyFileToDisk() can be much faster than reading the file with readFile() and then writing it to some disk. /// (especially for S3 where it can use CopyObject to copy objects inside S3 instead of downloading and uploading them). /// Parameters: - /// `encrypted_in_backup` specify if this file is encrypted in the backup, so it shouldn't be encrypted again while restoring to an encrypted disk. + /// `encrypted_in_backup` specify if this file is encrypted in the backup, so it shouldn't be encrypted again while restoring to an encrypted disk. void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override; - + const ReadSettings & getReadSettings() const override { return read_settings; } const WriteSettings & getWriteSettings() const override { return write_settings; } size_t getWriteBufferSize() const override { return write_buffer_size; } diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index 1e1af63cdc2..b58aa1832a5 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -8,11 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & root_path_, const ContextPtr & context_) : BackupReaderDefault(&Poco::Logger::get("BackupReaderDisk"), context_) , disk(disk_) diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 649637e567d..eb079623919 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -11,6 +11,11 @@ namespace fs = std::filesystem; namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + BackupReaderFile::BackupReaderFile(const String & root_path_, const ContextPtr & context_) : BackupReaderDefault(&Poco::Logger::get("BackupReaderFile"), context_) , root_path(root_path_) diff --git a/src/Backups/IBackupEntry.h b/src/Backups/IBackupEntry.h index 249c290cbe7..ca176c740c0 100644 --- a/src/Backups/IBackupEntry.h +++ b/src/Backups/IBackupEntry.h @@ -33,7 +33,7 @@ public: /// Returns true if the data returned by getReadBuffer() is encrypted by an encrypted disk. virtual bool isEncryptedByDisk() const { return false; } - /// Returns information about disk and file if this backup entry is generated from a file. + /// Returns information about disk and file if this backup entry is generated from a file. virtual bool isFromFile() const { return false; } virtual bool isFromImmutableFile() const { return false; } virtual String getFilePath() const { return ""; } From 5198997fd84dc3d2071f4b80b6888ebf8f800b81 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 3 May 2023 13:51:36 +0200 Subject: [PATCH 0321/2223] Remove ReadSettings from backup entries. --- src/Backups/BackupEntryFromAppendOnlyFile.cpp | 9 ++++----- src/Backups/BackupEntryFromAppendOnlyFile.h | 4 +--- src/Backups/BackupEntryFromImmutableFile.cpp | 8 +++----- src/Backups/BackupEntryFromImmutableFile.h | 6 ++---- src/Backups/BackupEntryFromMemory.cpp | 2 +- src/Backups/BackupEntryFromMemory.h | 2 +- src/Backups/BackupEntryFromSmallFile.cpp | 2 +- src/Backups/BackupEntryFromSmallFile.h | 2 +- src/Backups/BackupEntryWithChecksumCalculation.cpp | 4 ++-- src/Backups/BackupEntryWrappedWith.h | 2 +- src/Backups/BackupImpl.cpp | 4 ++-- src/Backups/IBackupEntriesLazyBatch.cpp | 2 +- src/Backups/IBackupEntry.h | 2 +- src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp | 3 +-- src/Storages/MergeTree/DataPartStorageOnDiskBase.h | 1 - src/Storages/MergeTree/IDataPartStorage.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 2 -- src/Storages/StorageLog.cpp | 8 +++----- src/Storages/StorageMemory.cpp | 6 ++---- src/Storages/StorageStripeLog.cpp | 8 +++----- 20 files changed, 30 insertions(+), 48 deletions(-) diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index 2c28abc2e2f..331a4a69d06 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -21,24 +21,23 @@ namespace } BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile( - const DiskPtr & disk_, const String & file_path_, const ReadSettings & settings_, const std::optional & file_size_) + const DiskPtr & disk_, const String & file_path_, const std::optional & file_size_) : disk(disk_) , file_path(file_path_) , data_source_description(disk->getDataSourceDescription()) , size(calculateSize(disk_, file_path_, data_source_description.is_encrypted, file_size_)) - , settings(settings_.adjustBufferSize(size)) { } BackupEntryFromAppendOnlyFile::~BackupEntryFromAppendOnlyFile() = default; -std::unique_ptr BackupEntryFromAppendOnlyFile::getReadBuffer() const +std::unique_ptr BackupEntryFromAppendOnlyFile::getReadBuffer(const ReadSettings & read_settings) const { std::unique_ptr buf; if (data_source_description.is_encrypted) - buf = disk->readEncryptedFile(file_path, settings); + buf = disk->readEncryptedFile(file_path, read_settings.adjustBufferSize(size)); else - buf = disk->readFile(file_path, settings); + buf = disk->readFile(file_path, read_settings.adjustBufferSize(size)); return std::make_unique(std::move(buf), 0, size); } diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.h b/src/Backups/BackupEntryFromAppendOnlyFile.h index fac8c319d8c..8a78478dcc5 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.h +++ b/src/Backups/BackupEntryFromAppendOnlyFile.h @@ -15,12 +15,11 @@ public: BackupEntryFromAppendOnlyFile( const DiskPtr & disk_, const String & file_path_, - const ReadSettings & settings_, const std::optional & file_size_ = {}); ~BackupEntryFromAppendOnlyFile() override; - std::unique_ptr getReadBuffer() const override; + std::unique_ptr getReadBuffer(const ReadSettings & read_settings) const override; UInt64 getSize() const override { return size; } DataSourceDescription getDataSourceDescription() const override { return data_source_description; } @@ -35,7 +34,6 @@ private: const String file_path; const DataSourceDescription data_source_description; const UInt64 size; - const ReadSettings settings; }; } diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index f377e54d9b0..24965f09356 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -20,13 +20,11 @@ namespace BackupEntryFromImmutableFile::BackupEntryFromImmutableFile( const DiskPtr & disk_, const String & file_path_, - const ReadSettings & settings_, const std::optional & file_size_, const std::optional & checksum_) : disk(disk_) , file_path(file_path_) , data_source_description(disk->getDataSourceDescription()) - , settings(settings_) , file_size(file_size_) , checksum(checksum_) { @@ -34,12 +32,12 @@ BackupEntryFromImmutableFile::BackupEntryFromImmutableFile( BackupEntryFromImmutableFile::~BackupEntryFromImmutableFile() = default; -std::unique_ptr BackupEntryFromImmutableFile::getReadBuffer() const +std::unique_ptr BackupEntryFromImmutableFile::getReadBuffer(const ReadSettings & read_settings) const { if (data_source_description.is_encrypted) - return disk->readEncryptedFile(file_path, settings); + return disk->readEncryptedFile(file_path, read_settings); else - return disk->readFile(file_path, settings); + return disk->readFile(file_path, read_settings); } UInt64 BackupEntryFromImmutableFile::getSize() const diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 23f48e8335a..5cdb4adc7c4 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -1,10 +1,10 @@ #pragma once #include -#include #include #include + namespace DB { class IDisk; @@ -18,13 +18,12 @@ public: BackupEntryFromImmutableFile( const DiskPtr & disk_, const String & file_path_, - const ReadSettings & settings_, const std::optional & file_size_ = {}, const std::optional & checksum_ = {}); ~BackupEntryFromImmutableFile() override; - std::unique_ptr getReadBuffer() const override; + std::unique_ptr getReadBuffer(const ReadSettings & read_settings) const override; UInt64 getSize() const override; UInt128 getChecksum() const override; @@ -42,7 +41,6 @@ private: const DiskPtr disk; const String file_path; const DataSourceDescription data_source_description; - const ReadSettings settings; mutable std::optional file_size; mutable std::optional checksum; mutable bool file_size_adjusted = false; diff --git a/src/Backups/BackupEntryFromMemory.cpp b/src/Backups/BackupEntryFromMemory.cpp index 82e033caca0..1dd911afbbb 100644 --- a/src/Backups/BackupEntryFromMemory.cpp +++ b/src/Backups/BackupEntryFromMemory.cpp @@ -14,7 +14,7 @@ BackupEntryFromMemory::BackupEntryFromMemory(String data_) : data(std::move(data { } -std::unique_ptr BackupEntryFromMemory::getReadBuffer() const +std::unique_ptr BackupEntryFromMemory::getReadBuffer(const ReadSettings &) const { return std::make_unique(data); } diff --git a/src/Backups/BackupEntryFromMemory.h b/src/Backups/BackupEntryFromMemory.h index 19b72c16e53..d8bc0eb966d 100644 --- a/src/Backups/BackupEntryFromMemory.h +++ b/src/Backups/BackupEntryFromMemory.h @@ -14,7 +14,7 @@ public: BackupEntryFromMemory(const void * data_, size_t size_); explicit BackupEntryFromMemory(String data_); - std::unique_ptr getReadBuffer() const override; + std::unique_ptr getReadBuffer(const ReadSettings &) const override; UInt64 getSize() const override { return data.size(); } DataSourceDescription getDataSourceDescription() const override { return DataSourceDescription{DataSourceType::RAM, "", false, false}; } diff --git a/src/Backups/BackupEntryFromSmallFile.cpp b/src/Backups/BackupEntryFromSmallFile.cpp index 3662522c243..3dcee7147fc 100644 --- a/src/Backups/BackupEntryFromSmallFile.cpp +++ b/src/Backups/BackupEntryFromSmallFile.cpp @@ -44,7 +44,7 @@ BackupEntryFromSmallFile::BackupEntryFromSmallFile(const DiskPtr & disk_, const { } -std::unique_ptr BackupEntryFromSmallFile::getReadBuffer() const +std::unique_ptr BackupEntryFromSmallFile::getReadBuffer(const ReadSettings &) const { return std::make_unique(data); } diff --git a/src/Backups/BackupEntryFromSmallFile.h b/src/Backups/BackupEntryFromSmallFile.h index 6846ca71a55..4f936718fbb 100644 --- a/src/Backups/BackupEntryFromSmallFile.h +++ b/src/Backups/BackupEntryFromSmallFile.h @@ -16,7 +16,7 @@ public: explicit BackupEntryFromSmallFile(const String & file_path_); BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_); - std::unique_ptr getReadBuffer() const override; + std::unique_ptr getReadBuffer(const ReadSettings &) const override; UInt64 getSize() const override { return data.size(); } DataSourceDescription getDataSourceDescription() const override { return data_source_description; } diff --git a/src/Backups/BackupEntryWithChecksumCalculation.cpp b/src/Backups/BackupEntryWithChecksumCalculation.cpp index bc97a907cff..2c0c2eab8ff 100644 --- a/src/Backups/BackupEntryWithChecksumCalculation.cpp +++ b/src/Backups/BackupEntryWithChecksumCalculation.cpp @@ -11,7 +11,7 @@ UInt128 BackupEntryWithChecksumCalculation::getChecksum() const std::lock_guard lock{checksum_calculation_mutex}; if (!calculated_checksum) { - auto read_buffer = this->getReadBuffer(); + auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(this->getSize())); HashingReadBuffer hashing_read_buffer(*read_buffer); hashing_read_buffer.ignoreAll(); calculated_checksum = hashing_read_buffer.getHash(); @@ -28,7 +28,7 @@ std::optional BackupEntryWithChecksumCalculation::getPartialCheck if (prefix_length >= this->getSize()) return this->getChecksum(); - auto read_buffer = this->getReadBuffer(); + auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(prefix_length)); HashingReadBuffer hashing_read_buffer(*read_buffer); hashing_read_buffer.ignore(prefix_length); auto partial_checksum = hashing_read_buffer.getHash(); diff --git a/src/Backups/BackupEntryWrappedWith.h b/src/Backups/BackupEntryWrappedWith.h index 01b6163039b..f865d529206 100644 --- a/src/Backups/BackupEntryWrappedWith.h +++ b/src/Backups/BackupEntryWrappedWith.h @@ -15,7 +15,7 @@ public: BackupEntryWrappedWith(BackupEntryPtr entry_, T && custom_value_) : entry(entry_), custom_value(std::move(custom_value_)) { } ~BackupEntryWrappedWith() override = default; - std::unique_ptr getReadBuffer() const override { return entry->getReadBuffer(); } + std::unique_ptr getReadBuffer(const ReadSettings & read_settings) const override { return entry->getReadBuffer(read_settings); } UInt64 getSize() const override { return entry->getSize(); } UInt128 getChecksum() const override { return entry->getChecksum(); } std::optional getPartialChecksum(size_t prefix_length) const override { return entry->getPartialChecksum(prefix_length); } diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 715fe9e0d13..7fcb42ec378 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -868,7 +868,7 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) { LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}, adding to archive", info.data_file_name, src_file_desc, info.data_file_index); auto out = archive_writer->writeFile(info.data_file_name); - auto read_buffer = entry->getReadBuffer(); + auto read_buffer = entry->getReadBuffer(writer->getReadSettings()); if (info.base_size != 0) read_buffer->seek(info.base_size, SEEK_SET); copyData(*read_buffer, *out); @@ -882,7 +882,7 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) else { LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}", info.data_file_name, src_file_desc, info.data_file_index); - auto create_read_buffer = [entry] { return entry->getReadBuffer(); }; + auto create_read_buffer = [entry, read_settings = writer->getReadSettings()] { return entry->getReadBuffer(read_settings); }; writer->copyDataToFile(info.data_file_name, create_read_buffer, info.base_size, info.size - info.base_size); } diff --git a/src/Backups/IBackupEntriesLazyBatch.cpp b/src/Backups/IBackupEntriesLazyBatch.cpp index 8913d60f2eb..7c6bb891981 100644 --- a/src/Backups/IBackupEntriesLazyBatch.cpp +++ b/src/Backups/IBackupEntriesLazyBatch.cpp @@ -17,7 +17,7 @@ class IBackupEntriesLazyBatch::BackupEntryFromBatch : public IBackupEntry public: BackupEntryFromBatch(const std::shared_ptr & batch_, size_t index_) : batch(batch_), index(index_) { } - std::unique_ptr getReadBuffer() const override { return getInternalBackupEntry()->getReadBuffer(); } + std::unique_ptr getReadBuffer(const ReadSettings & read_settings) const override { return getInternalBackupEntry()->getReadBuffer(read_settings); } UInt64 getSize() const override { return getInternalBackupEntry()->getSize(); } UInt128 getChecksum() const override { return getInternalBackupEntry()->getChecksum(); } std::optional getPartialChecksum(size_t prefix_length) const override { return getInternalBackupEntry()->getPartialChecksum(prefix_length); } diff --git a/src/Backups/IBackupEntry.h b/src/Backups/IBackupEntry.h index ca176c740c0..7e952e9b568 100644 --- a/src/Backups/IBackupEntry.h +++ b/src/Backups/IBackupEntry.h @@ -28,7 +28,7 @@ public: virtual std::optional getPartialChecksum(size_t /* prefix_length */) const { return {}; } /// Returns a read buffer for reading the data. - virtual std::unique_ptr getReadBuffer() const = 0; + virtual std::unique_ptr getReadBuffer(const ReadSettings & read_settings) const = 0; /// Returns true if the data returned by getReadBuffer() is encrypted by an encrypted disk. virtual bool isEncryptedByDisk() const { return false; } diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 4df490f41fe..c1e2c5b8cf8 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -319,7 +319,6 @@ DataPartStorageOnDiskBase::getReplicatedFilesDescriptionForRemoteDisk(const Name } void DataPartStorageOnDiskBase::backup( - const ReadSettings & read_settings, const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, const String & path_in_backup, @@ -393,7 +392,7 @@ void DataPartStorageOnDiskBase::backup( file_hash = {it->second.file_hash.first, it->second.file_hash.second}; } - BackupEntryPtr backup_entry = std::make_unique(disk, filepath_on_disk, read_settings, file_size, file_hash); + BackupEntryPtr backup_entry = std::make_unique(disk, filepath_on_disk, file_size, file_hash); if (temp_dir_owner) backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index fcc771f1250..52544bb2457 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -50,7 +50,6 @@ public: ReplicatedFilesDescription getReplicatedFilesDescriptionForRemoteDisk(const NameSet & file_names) const override; void backup( - const ReadSettings & read_settings, const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, const String & path_in_backup, diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index b0b42b331cd..9c267d94e63 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -197,7 +197,6 @@ public: /// Also creates a new tmp_dir for internal disk (if disk is mentioned the first time). using TemporaryFilesOnDisks = std::map>; virtual void backup( - const ReadSettings & read_settings, const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, const String & path_in_backup, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a53eb417fcd..e36bc8baeb4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5086,7 +5086,6 @@ BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, con BackupEntries backup_entries_from_part; part->getDataPartStorage().backup( - read_settings, part->checksums, part->getFileNamesWithoutChecksums(), data_path_in_backup, @@ -5098,7 +5097,6 @@ BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, con for (const auto & [projection_name, projection_part] : projection_parts) { projection_part->getDataPartStorage().backup( - read_settings, projection_part->checksums, projection_part->getFileNamesWithoutChecksums(), fs::path{data_path_in_backup} / part->name, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 31f499a7d96..6d77d2b97b0 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -928,10 +928,8 @@ std::optional StorageLog::totalBytes(const Settings &) const void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { - auto local_context = backup_entries_collector.getContext(); - ReadSettings read_settings = local_context->getBackupReadSettings(); + auto lock_timeout = getLockTimeout(backup_entries_collector.getContext()); - auto lock_timeout = getLockTimeout(local_context); loadMarks(lock_timeout); ReadLock lock{rwlock, lock_timeout}; @@ -954,7 +952,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c String hardlink_file_path = temp_dir / data_file_name; disk->createHardLink(data_file.path, hardlink_file_path); BackupEntryPtr backup_entry = std::make_unique( - disk, hardlink_file_path, read_settings, file_checker.getFileSize(data_file.path)); + disk, hardlink_file_path, file_checker.getFileSize(data_file.path)); backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); backup_entries_collector.addBackupEntry(data_path_in_backup_fs / data_file_name, std::move(backup_entry)); } @@ -967,7 +965,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c String hardlink_file_path = temp_dir / marks_file_name; disk->createHardLink(marks_file_path, hardlink_file_path); BackupEntryPtr backup_entry = std::make_unique( - disk, hardlink_file_path, read_settings, file_checker.getFileSize(marks_file_path)); + disk, hardlink_file_path, file_checker.getFileSize(marks_file_path)); backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); backup_entries_collector.addBackupEntry(data_path_in_backup_fs / marks_file_name, std::move(backup_entry)); } diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 68c888a2d23..ebc780f5ab1 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -308,8 +308,6 @@ namespace BackupEntries generate() override { - ReadSettings read_settings = context->getBackupReadSettings(); - BackupEntries backup_entries; backup_entries.resize(file_paths.size()); @@ -326,7 +324,7 @@ namespace NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index}; for (const auto & block : *blocks) block_out.write(block); - backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared(temp_disk, data_file_path, read_settings)}; + backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared(temp_disk, data_file_path)}; } /// Writing index.mrk @@ -335,7 +333,7 @@ namespace auto index_mrk_out_compressed = temp_disk->writeFile(index_mrk_path); CompressedWriteBuffer index_mrk_out{*index_mrk_out_compressed}; index.write(index_mrk_out); - backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared(temp_disk, index_mrk_path, read_settings)}; + backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared(temp_disk, index_mrk_path)}; } /// Writing columns.txt diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 5b22db91631..23bf88b9db5 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -529,10 +529,8 @@ std::optional StorageStripeLog::totalBytes(const Settings &) const void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { - auto local_context = backup_entries_collector.getContext(); - ReadSettings read_settings = local_context->getBackupReadSettings(); + auto lock_timeout = getLockTimeout(backup_entries_collector.getContext()); - auto lock_timeout = getLockTimeout(local_context); loadIndices(lock_timeout); ReadLock lock{rwlock, lock_timeout}; @@ -554,7 +552,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec String hardlink_file_path = temp_dir / data_file_name; disk->createHardLink(data_file_path, hardlink_file_path); BackupEntryPtr backup_entry = std::make_unique( - disk, hardlink_file_path, read_settings, file_checker.getFileSize(data_file_path)); + disk, hardlink_file_path, file_checker.getFileSize(data_file_path)); backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); backup_entries_collector.addBackupEntry(data_path_in_backup_fs / data_file_name, std::move(backup_entry)); } @@ -566,7 +564,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec String hardlink_file_path = temp_dir / index_file_name; disk->createHardLink(index_file_path, hardlink_file_path); BackupEntryPtr backup_entry = std::make_unique( - disk, hardlink_file_path, read_settings, file_checker.getFileSize(index_file_path)); + disk, hardlink_file_path, file_checker.getFileSize(index_file_path)); backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); backup_entries_collector.addBackupEntry(data_path_in_backup_fs / index_file_name, std::move(backup_entry)); } From 019493efa370f4c7aceadc6cb6a33485bf022254 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 3 May 2023 15:56:28 +0200 Subject: [PATCH 0322/2223] Fix throttling in backups. --- src/Backups/BackupIO_Disk.cpp | 39 ++++++++++++++++++++++++----------- src/Backups/BackupIO_Disk.h | 3 +++ src/Backups/BackupIO_File.cpp | 18 +++++++++------- src/Backups/BackupIO_File.h | 1 - src/Backups/BackupIO_S3.cpp | 11 ++++++---- 5 files changed, 47 insertions(+), 25 deletions(-) diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index b58aa1832a5..3b1651bb223 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -12,6 +12,7 @@ BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & root_pa : BackupReaderDefault(&Poco::Logger::get("BackupReaderDisk"), context_) , disk(disk_) , root_path(root_path_) + , data_source_description(disk->getDataSourceDescription()) { } @@ -35,12 +36,18 @@ std::unique_ptr BackupReaderDisk::readFile(const String & fi void BackupReaderDisk::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { - if ((write_mode == WriteMode::Rewrite) && !encrypted_in_backup) + /// Use IDisk::copyFile() as a more optimal way to copy a file if it's possible. + bool has_throttling = disk->isRemote() ? static_cast(read_settings.remote_throttler) : static_cast(read_settings.local_throttler); + if (!has_throttling && (write_mode == WriteMode::Rewrite) && !encrypted_in_backup) { - /// Use more optimal way. - LOG_TRACE(log, "Copying file {} from disk {} to disk {}", path_in_backup, disk->getName(), destination_disk->getName()); - disk->copyFile(root_path / path_in_backup, *destination_disk, destination_path, write_settings); - return; + auto destination_data_source_description = destination_disk->getDataSourceDescription(); + if (destination_data_source_description.sameKind(data_source_description) && !data_source_description.is_encrypted) + { + /// Use more optimal way. + LOG_TRACE(log, "Copying file {} from disk {} to disk {}", path_in_backup, disk->getName(), destination_disk->getName()); + disk->copyFile(root_path / path_in_backup, *destination_disk, destination_path, write_settings); + return; /// copied! + } } /// Fallback to copy through buffers. @@ -52,6 +59,7 @@ BackupWriterDisk::BackupWriterDisk(const DiskPtr & disk_, const String & root_pa : BackupWriterDefault(&Poco::Logger::get("BackupWriterDisk"), context_) , disk(disk_) , root_path(root_path_) + , data_source_description(disk->getDataSourceDescription()) { } @@ -97,14 +105,21 @@ void BackupWriterDisk::removeFiles(const Strings & file_names) void BackupWriterDisk::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, bool copy_encrypted, UInt64 start_pos, UInt64 length) { - if (!copy_encrypted && !start_pos && (length == src_disk->getFileSize(src_path))) + /// Use IDisk::copyFile() as a more optimal way to copy a file if it's possible. + bool has_throttling = src_disk->isRemote() ? static_cast(read_settings.remote_throttler) : static_cast(read_settings.local_throttler); + if (!has_throttling && !start_pos && !copy_encrypted) { - /// Use more optimal way. - LOG_TRACE(log, "Copying file {} from disk {} to disk {}", src_path, src_disk->getName(), disk->getName()); - auto dest_file_path = root_path / path_in_backup; - disk->createDirectories(dest_file_path.parent_path()); - src_disk->copyFile(src_path, *disk, dest_file_path, write_settings); - return; + auto source_data_source_description = src_disk->getDataSourceDescription(); + if (source_data_source_description.sameKind(data_source_description) && !source_data_source_description.is_encrypted + && (length == src_disk->getFileSize(src_path))) + { + /// Use more optimal way. + LOG_TRACE(log, "Copying file {} from disk {} to disk {}", src_path, src_disk->getName(), disk->getName()); + auto dest_file_path = root_path / path_in_backup; + disk->createDirectories(dest_file_path.parent_path()); + src_disk->copyFile(src_path, *disk, dest_file_path, write_settings); + return; /// copied! + } } /// Fallback to copy through buffers. diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index 399ebeaa227..faf4ef03447 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -26,6 +27,7 @@ public: private: const DiskPtr disk; const std::filesystem::path root_path; + const DataSourceDescription data_source_description; }; class BackupWriterDisk : public BackupWriterDefault @@ -50,6 +52,7 @@ private: const DiskPtr disk; const std::filesystem::path root_path; + const DataSourceDescription data_source_description; }; } diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index eb079623919..7ffae26d16f 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -41,7 +41,9 @@ std::unique_ptr BackupReaderFile::readFile(const String & fi void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { - if (write_mode == WriteMode::Rewrite) + /// std::filesystem::copy() can copy from the filesystem only, and it can't do the throttling or appending. + bool has_throttling = static_cast(read_settings.local_throttler); + if (!has_throttling && (write_mode == WriteMode::Rewrite)) { auto destination_data_source_description = destination_disk->getDataSourceDescription(); if (destination_data_source_description.sameKind(data_source_description) @@ -50,10 +52,10 @@ void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file /// Use more optimal way. LOG_TRACE(log, "Copying file {} to disk {} locally", path_in_backup, destination_disk->getName()); - auto write_blob_function - = [abs_source_path = root_path / path_in_backup, file_size]( - const Strings & blob_path, WriteMode mode, const std::optional &) -> size_t + auto write_blob_function = [abs_source_path = root_path / path_in_backup, file_size]( + const Strings & blob_path, WriteMode mode, const std::optional &) -> size_t { + /// For local disks the size of a blob path is expected to be 1. if (blob_path.size() != 1 || mode != WriteMode::Rewrite) throw Exception(ErrorCodes::LOGICAL_ERROR, "Blob writing function called with unexpected blob_path.size={} or mode={}", @@ -63,7 +65,7 @@ void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file }; destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function); - return; + return; /// copied! } } @@ -76,7 +78,6 @@ BackupWriterFile::BackupWriterFile(const String & root_path_, const ContextPtr & : BackupWriterDefault(&Poco::Logger::get("BackupWriterFile"), context_) , root_path(root_path_) , data_source_description(DiskLocal::getLocalDataSourceDescription(root_path)) - , has_throttling(static_cast(read_settings.local_throttler)) { } @@ -120,7 +121,8 @@ void BackupWriterFile::removeFiles(const Strings & file_names) void BackupWriterFile::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, bool copy_encrypted, UInt64 start_pos, UInt64 length) { - /// std::filesystem::copy() can copy from the filesystem only, and it cannot do the throttling. + /// std::filesystem::copy() can copy from the filesystem only, and it can't do the throttling or copy a part of the file. + bool has_throttling = static_cast(read_settings.local_throttler); if (!has_throttling) { auto source_data_source_description = src_disk->getDataSourceDescription(); @@ -140,7 +142,7 @@ void BackupWriterFile::copyFileFromDisk(const String & path_in_backup, DiskPtr s auto abs_dest_path = root_path / path_in_backup; fs::create_directories(abs_dest_path.parent_path()); fs::copy(abs_source_path, abs_dest_path, fs::copy_options::overwrite_existing); - return; + return; /// copied! } } } diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index 45fc0d47115..fd2c0b07158 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -46,7 +46,6 @@ private: const std::filesystem::path root_path; const DataSourceDescription data_source_description; - const bool has_throttling; }; } diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 09d2bdbcfaf..69f56078f9d 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -135,6 +135,8 @@ std::unique_ptr BackupReaderS3::readFile(const String & file void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { + /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. + /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto destination_data_source_description = destination_disk->getDataSourceDescription(); if (destination_data_source_description.sameKind(data_source_description) && (destination_data_source_description.is_encrypted == encrypted_in_backup)) @@ -166,7 +168,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s }; destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function); - return; + return; /// copied! } /// Fallback to copy through buffers. @@ -189,9 +191,10 @@ BackupWriterS3::BackupWriterS3( void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, bool copy_encrypted, UInt64 start_pos, UInt64 length) { + /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. + /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto source_data_source_description = src_disk->getDataSourceDescription(); - if (source_data_source_description.sameKind(data_source_description) - && (source_data_source_description.is_encrypted == copy_encrypted)) + if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) { /// getBlobPath() can return more than 2 elements if the file is stored as multiple objects in S3 bucket. /// In this case we can't use the native copy. @@ -210,7 +213,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src request_settings, {}, threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); - return; + return; /// copied! } } From 943707963feff55dbf07ae47a7955253cc6adde7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 4 May 2023 01:27:16 +0200 Subject: [PATCH 0323/2223] Add backup setting "decrypt_files_from_encrypted_disks" --- src/Backups/BackupEntryFromAppendOnlyFile.cpp | 15 +-- src/Backups/BackupEntryFromAppendOnlyFile.h | 4 +- src/Backups/BackupEntryFromImmutableFile.cpp | 11 +- src/Backups/BackupEntryFromImmutableFile.h | 4 +- src/Backups/BackupEntryFromSmallFile.cpp | 9 +- src/Backups/BackupEntryFromSmallFile.h | 5 +- .../BackupEntryWithChecksumCalculation.cpp | 12 +- src/Backups/BackupIO_Default.cpp | 6 +- src/Backups/BackupSettings.cpp | 1 + src/Backups/BackupSettings.h | 3 + src/Disks/IDisk.h | 4 +- .../MergeTree/DataPartStorageOnDiskBase.cpp | 10 +- .../MergeTree/DataPartStorageOnDiskBase.h | 3 +- src/Storages/MergeTree/IDataPartStorage.h | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 12 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageLog.cpp | 8 +- src/Storages/StorageMemory.cpp | 26 ++-- src/Storages/StorageMergeTree.cpp | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- src/Storages/StorageStripeLog.cpp | 8 +- tests/integration/test_encrypted_disk/test.py | 119 ++++++++---------- 22 files changed, 152 insertions(+), 121 deletions(-) diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index 331a4a69d06..1d73ab52820 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include @@ -9,11 +9,11 @@ namespace DB namespace { /// For append-only files we must calculate its size on the construction of a backup entry. - UInt64 calculateSize(const DiskPtr & disk, const String & file_path, bool is_encrypted, std::optional unencrypted_file_size) + UInt64 calculateSize(const DiskPtr & disk, const String & file_path, bool copy_encrypted, std::optional unencrypted_file_size) { if (!unencrypted_file_size) - return is_encrypted ? disk->getEncryptedFileSize(file_path) : disk->getFileSize(file_path); - else if (is_encrypted) + return copy_encrypted ? disk->getEncryptedFileSize(file_path) : disk->getFileSize(file_path); + else if (copy_encrypted) return disk->getEncryptedFileSize(*unencrypted_file_size); else return *unencrypted_file_size; @@ -21,11 +21,12 @@ namespace } BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile( - const DiskPtr & disk_, const String & file_path_, const std::optional & file_size_) + const DiskPtr & disk_, const String & file_path_, bool copy_encrypted_, const std::optional & file_size_) : disk(disk_) , file_path(file_path_) , data_source_description(disk->getDataSourceDescription()) - , size(calculateSize(disk_, file_path_, data_source_description.is_encrypted, file_size_)) + , copy_encrypted(copy_encrypted_ && data_source_description.is_encrypted) + , size(calculateSize(disk_, file_path_, copy_encrypted, file_size_)) { } @@ -34,7 +35,7 @@ BackupEntryFromAppendOnlyFile::~BackupEntryFromAppendOnlyFile() = default; std::unique_ptr BackupEntryFromAppendOnlyFile::getReadBuffer(const ReadSettings & read_settings) const { std::unique_ptr buf; - if (data_source_description.is_encrypted) + if (copy_encrypted) buf = disk->readEncryptedFile(file_path, read_settings.adjustBufferSize(size)); else buf = disk->readFile(file_path, read_settings.adjustBufferSize(size)); diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.h b/src/Backups/BackupEntryFromAppendOnlyFile.h index 8a78478dcc5..257c392f24c 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.h +++ b/src/Backups/BackupEntryFromAppendOnlyFile.h @@ -15,6 +15,7 @@ public: BackupEntryFromAppendOnlyFile( const DiskPtr & disk_, const String & file_path_, + bool copy_encrypted_ = false, const std::optional & file_size_ = {}); ~BackupEntryFromAppendOnlyFile() override; @@ -23,7 +24,7 @@ public: UInt64 getSize() const override { return size; } DataSourceDescription getDataSourceDescription() const override { return data_source_description; } - bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; } + bool isEncryptedByDisk() const override { return copy_encrypted; } bool isFromFile() const override { return true; } DiskPtr getDisk() const override { return disk; } @@ -33,6 +34,7 @@ private: const DiskPtr disk; const String file_path; const DataSourceDescription data_source_description; + const bool copy_encrypted; const UInt64 size; }; diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index 24965f09356..cc635dd8541 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -1,6 +1,5 @@ #include #include -#include namespace DB @@ -20,11 +19,13 @@ namespace BackupEntryFromImmutableFile::BackupEntryFromImmutableFile( const DiskPtr & disk_, const String & file_path_, + bool copy_encrypted_, const std::optional & file_size_, const std::optional & checksum_) : disk(disk_) , file_path(file_path_) , data_source_description(disk->getDataSourceDescription()) + , copy_encrypted(copy_encrypted_ && data_source_description.is_encrypted) , file_size(file_size_) , checksum(checksum_) { @@ -34,7 +35,7 @@ BackupEntryFromImmutableFile::~BackupEntryFromImmutableFile() = default; std::unique_ptr BackupEntryFromImmutableFile::getReadBuffer(const ReadSettings & read_settings) const { - if (data_source_description.is_encrypted) + if (copy_encrypted) return disk->readEncryptedFile(file_path, read_settings); else return disk->readFile(file_path, read_settings); @@ -46,8 +47,8 @@ UInt64 BackupEntryFromImmutableFile::getSize() const if (!file_size_adjusted) { if (!file_size) - file_size = data_source_description.is_encrypted ? disk->getEncryptedFileSize(file_path) : disk->getFileSize(file_path); - else if (data_source_description.is_encrypted) + file_size = copy_encrypted ? disk->getEncryptedFileSize(file_path) : disk->getFileSize(file_path); + else if (copy_encrypted) file_size = disk->getEncryptedFileSize(*file_size); file_size_adjusted = true; } @@ -61,7 +62,7 @@ UInt128 BackupEntryFromImmutableFile::getChecksum() const { if (!checksum) checksum = BackupEntryWithChecksumCalculation::getChecksum(); - else if (data_source_description.is_encrypted) + else if (copy_encrypted) checksum = combineChecksums(*checksum, disk->getEncryptedFileIV(file_path)); checksum_adjusted = true; } diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 5cdb4adc7c4..850a86a3264 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -18,6 +18,7 @@ public: BackupEntryFromImmutableFile( const DiskPtr & disk_, const String & file_path_, + bool copy_encrypted_ = false, const std::optional & file_size_ = {}, const std::optional & checksum_ = {}); @@ -30,7 +31,7 @@ public: std::optional getPartialChecksum(size_t prefix_length) const override; DataSourceDescription getDataSourceDescription() const override { return data_source_description; } - bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; } + bool isEncryptedByDisk() const override { return copy_encrypted; } bool isFromFile() const override { return true; } bool isFromImmutableFile() const override { return true; } @@ -41,6 +42,7 @@ private: const DiskPtr disk; const String file_path; const DataSourceDescription data_source_description; + const bool copy_encrypted; mutable std::optional file_size; mutable std::optional checksum; mutable bool file_size_adjusted = false; diff --git a/src/Backups/BackupEntryFromSmallFile.cpp b/src/Backups/BackupEntryFromSmallFile.cpp index 3dcee7147fc..22487767689 100644 --- a/src/Backups/BackupEntryFromSmallFile.cpp +++ b/src/Backups/BackupEntryFromSmallFile.cpp @@ -19,9 +19,9 @@ namespace return s; } - String readFile(const DiskPtr & disk, const String & file_path, bool read_encrypted) + String readFile(const DiskPtr & disk, const String & file_path, bool copy_encrypted) { - auto buf = read_encrypted ? disk->readEncryptedFile(file_path) : disk->readFile(file_path); + auto buf = copy_encrypted ? disk->readEncryptedFile(file_path) : disk->readFile(file_path); String s; readStringUntilEOF(s, *buf); return s; @@ -36,11 +36,12 @@ BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_) { } -BackupEntryFromSmallFile::BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_) +BackupEntryFromSmallFile::BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_, bool copy_encrypted_) : disk(disk_) , file_path(file_path_) , data_source_description(disk_->getDataSourceDescription()) - , data(readFile(disk_, file_path, data_source_description.is_encrypted)) + , copy_encrypted(copy_encrypted_ && data_source_description.is_encrypted) + , data(readFile(disk_, file_path, copy_encrypted)) { } diff --git a/src/Backups/BackupEntryFromSmallFile.h b/src/Backups/BackupEntryFromSmallFile.h index 4f936718fbb..d6651ab8cb5 100644 --- a/src/Backups/BackupEntryFromSmallFile.h +++ b/src/Backups/BackupEntryFromSmallFile.h @@ -14,13 +14,13 @@ class BackupEntryFromSmallFile : public BackupEntryWithChecksumCalculation getReadBuffer(const ReadSettings &) const override; UInt64 getSize() const override { return data.size(); } DataSourceDescription getDataSourceDescription() const override { return data_source_description; } - bool isEncryptedByDisk() const override { return data_source_description.is_encrypted; } + bool isEncryptedByDisk() const override { return copy_encrypted; } bool isFromFile() const override { return true; } DiskPtr getDisk() const override { return disk; } @@ -30,6 +30,7 @@ private: const DiskPtr disk; const String file_path; const DataSourceDescription data_source_description; + const bool copy_encrypted = false; const String data; }; diff --git a/src/Backups/BackupEntryWithChecksumCalculation.cpp b/src/Backups/BackupEntryWithChecksumCalculation.cpp index 2c0c2eab8ff..1e634e6bb73 100644 --- a/src/Backups/BackupEntryWithChecksumCalculation.cpp +++ b/src/Backups/BackupEntryWithChecksumCalculation.cpp @@ -25,15 +25,21 @@ std::optional BackupEntryWithChecksumCalculation::getPartialCheck if (prefix_length == 0) return 0; - if (prefix_length >= this->getSize()) + size_t size = this->getSize(); + if (prefix_length >= size) return this->getChecksum(); - auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(prefix_length)); + std::lock_guard lock{checksum_calculation_mutex}; + + ReadSettings read_settings; + if (calculated_checksum) + read_settings.adjustBufferSize(calculated_checksum ? prefix_length : size); + + auto read_buffer = this->getReadBuffer(read_settings); HashingReadBuffer hashing_read_buffer(*read_buffer); hashing_read_buffer.ignore(prefix_length); auto partial_checksum = hashing_read_buffer.getHash(); - std::lock_guard lock{checksum_calculation_mutex}; if (!calculated_checksum) { hashing_read_buffer.ignoreAll(); diff --git a/src/Backups/BackupIO_Default.cpp b/src/Backups/BackupIO_Default.cpp index 3b4851e9441..f7ba061cf3a 100644 --- a/src/Backups/BackupIO_Default.cpp +++ b/src/Backups/BackupIO_Default.cpp @@ -82,12 +82,12 @@ void BackupWriterDefault::copyFileFromDisk(const String & path_in_backup, DiskPt { LOG_TRACE(log, "Copying file {} from disk {} through buffers", src_path, src_disk->getName()); - auto create_read_buffer = [this, src_disk, src_path, file_size = start_pos + length, copy_encrypted] + auto create_read_buffer = [src_disk, src_path, copy_encrypted, settings = read_settings.adjustBufferSize(start_pos + length)] { if (copy_encrypted) - return src_disk->readEncryptedFile(src_path, read_settings, {}, file_size); + return src_disk->readEncryptedFile(src_path, settings); else - return src_disk->readFile(src_path, read_settings, {}, file_size); + return src_disk->readFile(src_path, settings); }; copyDataToFile(path_in_backup, create_read_buffer, start_pos, length); diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 57d85305e25..882342467fe 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes M(String, password) \ M(Bool, structure_only) \ M(Bool, async) \ + M(Bool, decrypt_files_from_encrypted_disks) \ M(Bool, deduplicate_files) \ M(UInt64, shard_num) \ M(UInt64, replica_num) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 1b97256c75b..2c899687e6e 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -32,6 +32,9 @@ struct BackupSettings /// Whether the BACKUP command must return immediately without waiting until the backup has completed. bool async = false; + /// Whether the BACKUP command should decrypt files stored on encrypted disks. + bool decrypt_files_from_encrypted_disks = false; + /// Whether the BACKUP will omit similar files (within one backup only). bool deduplicate_files = true; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 47b4ed80ebf..4d74fe8bbab 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -253,14 +253,14 @@ public: virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; /// Reads a file from an encrypted disk without decrypting it (only for encrypted disks). - virtual std::unique_ptr readEncryptedFile( + virtual std::unique_ptr readEncryptedFile( /// NOLINT const String & path, const ReadSettings & settings = ReadSettings{}, std::optional read_hint = {}, std::optional file_size = {}) const; /// Writes an already encrypted file to the disk (only for encrypted disks). - virtual std::unique_ptr writeEncryptedFile( + virtual std::unique_ptr writeEncryptedFile( /// NOLINT const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, WriteMode mode = WriteMode::Rewrite, diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index c1e2c5b8cf8..ebe55ea7dc7 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -322,8 +323,9 @@ void DataPartStorageOnDiskBase::backup( const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, const String & path_in_backup, - BackupEntries & backup_entries, + const BackupSettings & backup_settings, bool make_temporary_hard_links, + BackupEntries & backup_entries, TemporaryFilesOnDisks * temp_dirs) const { fs::path part_path_on_disk = fs::path{root_path} / part_dir; @@ -364,6 +366,8 @@ void DataPartStorageOnDiskBase::backup( files_to_backup = getActualFileNamesOnDisk(files_to_backup); + bool copy_encrypted = !backup_settings.decrypt_files_from_encrypted_disks; + for (const auto & filepath : files_to_backup) { auto filepath_on_disk = part_path_on_disk / filepath; @@ -371,7 +375,7 @@ void DataPartStorageOnDiskBase::backup( if (files_without_checksums.contains(filepath)) { - backup_entries.emplace_back(filepath_in_backup, std::make_unique(disk, filepath_on_disk)); + backup_entries.emplace_back(filepath_in_backup, std::make_unique(disk, filepath_on_disk, copy_encrypted)); continue; } @@ -392,7 +396,7 @@ void DataPartStorageOnDiskBase::backup( file_hash = {it->second.file_hash.first, it->second.file_hash.second}; } - BackupEntryPtr backup_entry = std::make_unique(disk, filepath_on_disk, file_size, file_hash); + BackupEntryPtr backup_entry = std::make_unique(disk, filepath_on_disk, copy_encrypted, file_size, file_hash); if (temp_dir_owner) backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 52544bb2457..09eb7f008bc 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -53,8 +53,9 @@ public: const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, const String & path_in_backup, - BackupEntries & backup_entries, + const BackupSettings & backup_settings, bool make_temporary_hard_links, + BackupEntries & backup_entries, TemporaryFilesOnDisks * temp_dirs) const override; MutableDataPartStoragePtr freeze( diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 9c267d94e63..7c85469d890 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -65,6 +65,7 @@ using SyncGuardPtr = std::unique_ptr; class IBackupEntry; using BackupEntryPtr = std::shared_ptr; using BackupEntries = std::vector>; +struct BackupSettings; struct WriteSettings; @@ -200,8 +201,9 @@ public: const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, const String & path_in_backup, - BackupEntries & backup_entries, + const BackupSettings & backup_settings, bool make_temporary_hard_links, + BackupEntries & backup_entries, TemporaryFilesOnDisks * temp_dirs) const = 0; /// Creates hardlinks into 'to/dir_path' for every file in data part. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e36bc8baeb4..b8208052f19 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5051,7 +5051,11 @@ Pipe MergeTreeData::alterPartition( } -BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup, const ContextPtr & local_context) +BackupEntries MergeTreeData::backupParts( + const DataPartsVector & data_parts, + const String & data_path_in_backup, + const BackupSettings & backup_settings, + const ContextPtr & local_context) { BackupEntries backup_entries; std::map> temp_dirs; @@ -5089,8 +5093,9 @@ BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, con part->checksums, part->getFileNamesWithoutChecksums(), data_path_in_backup, - backup_entries_from_part, + backup_settings, make_temporary_hard_links, + backup_entries_from_part, &temp_dirs); auto projection_parts = part->getProjectionParts(); @@ -5100,8 +5105,9 @@ BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, con projection_part->checksums, projection_part->getFileNamesWithoutChecksums(), fs::path{data_path_in_backup} / part->name, - backup_entries_from_part, + backup_settings, make_temporary_hard_links, + backup_entries_from_part, &temp_dirs); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ce5c6a730e9..04b008b623c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1322,7 +1322,7 @@ protected: MovePartsOutcome movePartsToSpace(const DataPartsVector & parts, SpacePtr space); /// Makes backup entries to backup the parts of this table. - BackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup, const ContextPtr & local_context); + BackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup, const BackupSettings & backup_settings, const ContextPtr & local_context); class RestoredPartsHolder; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 6d77d2b97b0..f698f1881fa 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -944,6 +944,8 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c fs::path temp_dir = temp_dir_owner->getPath(); disk->createDirectories(temp_dir); + bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks; + /// *.bin for (const auto & data_file : data_files) { @@ -952,7 +954,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c String hardlink_file_path = temp_dir / data_file_name; disk->createHardLink(data_file.path, hardlink_file_path); BackupEntryPtr backup_entry = std::make_unique( - disk, hardlink_file_path, file_checker.getFileSize(data_file.path)); + disk, hardlink_file_path, copy_encrypted, file_checker.getFileSize(data_file.path)); backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); backup_entries_collector.addBackupEntry(data_path_in_backup_fs / data_file_name, std::move(backup_entry)); } @@ -965,7 +967,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c String hardlink_file_path = temp_dir / marks_file_name; disk->createHardLink(marks_file_path, hardlink_file_path); BackupEntryPtr backup_entry = std::make_unique( - disk, hardlink_file_path, file_checker.getFileSize(marks_file_path)); + disk, hardlink_file_path, copy_encrypted, file_checker.getFileSize(marks_file_path)); backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); backup_entries_collector.addBackupEntry(data_path_in_backup_fs / marks_file_name, std::move(backup_entry)); } @@ -973,7 +975,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c /// sizes.json String files_info_path = file_checker.getPath(); backup_entries_collector.addBackupEntry( - data_path_in_backup_fs / fileName(files_info_path), std::make_unique(disk, files_info_path)); + data_path_in_backup_fs / fileName(files_info_path), std::make_unique(disk, files_info_path, copy_encrypted)); /// columns.txt backup_entries_collector.addBackupEntry( diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index ebc780f5ab1..c9654cfd105 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include #include #include @@ -319,21 +319,25 @@ namespace IndexForNativeFormat index; { auto data_file_path = temp_dir / fs::path{file_paths[data_bin_pos]}.filename(); - auto data_out_compressed = temp_disk->writeFile(data_file_path); - CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size}; - NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index}; - for (const auto & block : *blocks) - block_out.write(block); - backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared(temp_disk, data_file_path)}; + { + auto data_out_compressed = temp_disk->writeFile(data_file_path); + CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size}; + NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index}; + for (const auto & block : *blocks) + block_out.write(block); + } + backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared(temp_disk, data_file_path)}; } /// Writing index.mrk { auto index_mrk_path = temp_dir / fs::path{file_paths[index_mrk_pos]}.filename(); - auto index_mrk_out_compressed = temp_disk->writeFile(index_mrk_path); - CompressedWriteBuffer index_mrk_out{*index_mrk_out_compressed}; - index.write(index_mrk_out); - backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared(temp_disk, index_mrk_path)}; + { + auto index_mrk_out_compressed = temp_disk->writeFile(index_mrk_path); + CompressedWriteBuffer index_mrk_out{*index_mrk_out_compressed}; + index.write(index_mrk_out); + } + backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared(temp_disk, index_mrk_path)}; } /// Writing columns.txt diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b4dc2830bd6..2c19d3ba122 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2142,6 +2142,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) { + const auto & backup_settings = backup_entries_collector.getBackupSettings(); auto local_context = backup_entries_collector.getContext(); DataPartsVector data_parts; @@ -2154,7 +2155,7 @@ void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collec for (const auto & data_part : data_parts) min_data_version = std::min(min_data_version, data_part->info.getDataVersion() + 1); - backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup, local_context)); + backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup, backup_settings, local_context)); backup_entries_collector.addBackupEntries(backupMutations(min_data_version, data_path_in_backup)); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fba6e6cbafc..fcb7adbd69f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9252,6 +9252,8 @@ void StorageReplicatedMergeTree::backupData( /// First we generate backup entries in the same way as an ordinary MergeTree does. /// But then we don't add them to the BackupEntriesCollector right away, /// because we need to coordinate them with other replicas (other replicas can have better parts). + + const auto & backup_settings = backup_entries_collector.getBackupSettings(); auto local_context = backup_entries_collector.getContext(); DataPartsVector data_parts; @@ -9260,7 +9262,7 @@ void StorageReplicatedMergeTree::backupData( else data_parts = getVisibleDataPartsVector(local_context); - auto backup_entries = backupParts(data_parts, /* data_path_in_backup */ "", local_context); + auto backup_entries = backupParts(data_parts, /* data_path_in_backup */ "", backup_settings, local_context); auto coordination = backup_entries_collector.getBackupCoordination(); String shared_id = getTableSharedID(); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 23bf88b9db5..b2e7c202800 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -545,6 +545,8 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec fs::path temp_dir = temp_dir_owner->getPath(); disk->createDirectories(temp_dir); + bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks; + /// data.bin { /// We make a copy of the data file because it can be changed later in write() or in truncate(). @@ -552,7 +554,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec String hardlink_file_path = temp_dir / data_file_name; disk->createHardLink(data_file_path, hardlink_file_path); BackupEntryPtr backup_entry = std::make_unique( - disk, hardlink_file_path, file_checker.getFileSize(data_file_path)); + disk, hardlink_file_path, copy_encrypted, file_checker.getFileSize(data_file_path)); backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); backup_entries_collector.addBackupEntry(data_path_in_backup_fs / data_file_name, std::move(backup_entry)); } @@ -564,7 +566,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec String hardlink_file_path = temp_dir / index_file_name; disk->createHardLink(index_file_path, hardlink_file_path); BackupEntryPtr backup_entry = std::make_unique( - disk, hardlink_file_path, file_checker.getFileSize(index_file_path)); + disk, hardlink_file_path, copy_encrypted, file_checker.getFileSize(index_file_path)); backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); backup_entries_collector.addBackupEntry(data_path_in_backup_fs / index_file_name, std::move(backup_entry)); } @@ -572,7 +574,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec /// sizes.json String files_info_path = file_checker.getPath(); backup_entries_collector.addBackupEntry( - data_path_in_backup_fs / fileName(files_info_path), std::make_unique(disk, files_info_path)); + data_path_in_backup_fs / fileName(files_info_path), std::make_unique(disk, files_info_path, copy_encrypted)); /// columns.txt backup_entries_collector.addBackupEntry( diff --git a/tests/integration/test_encrypted_disk/test.py b/tests/integration/test_encrypted_disk/test.py index 90da5849c7f..66ff073f02b 100644 --- a/tests/integration/test_encrypted_disk/test.py +++ b/tests/integration/test_encrypted_disk/test.py @@ -309,17 +309,24 @@ def test_restart(): @pytest.mark.parametrize( - "storage_policy,backup_type,storage_policy2", + "backup_type,old_storage_policy,new_storage_policy,decrypt_files_from_encrypted_disks", [ - ("encrypted_policy", "S3", "encrypted_policy"), - ("encrypted_policy", "S3", "s3_encrypted_default_path"), - ("s3_encrypted_default_path", "S3", "s3_encrypted_default_path"), - ("s3_encrypted_default_path", "S3", "encrypted_policy"), - ("s3_encrypted_default_path", "File", "encrypted_policy"), - ("local_policy", "File", "encrypted_policy"), + ("S3", "encrypted_policy", "encrypted_policy", False), + ("S3", "encrypted_policy", "s3_encrypted_default_path", False), + ("S3", "s3_encrypted_default_path", "s3_encrypted_default_path", False), + ("S3", "s3_encrypted_default_path", "encrypted_policy", False), + ("File", "s3_encrypted_default_path", "encrypted_policy", False), + ("File", "local_policy", "encrypted_policy", False), + ("File", "encrypted_policy", "local_policy", False), + ("File", "encrypted_policy", "local_policy", True), ], ) -def test_backup_restore(storage_policy, backup_type, storage_policy2): +def test_backup_restore( + backup_type, + old_storage_policy, + new_storage_policy, + decrypt_files_from_encrypted_disks, +): node.query( f""" CREATE TABLE encrypted_test ( @@ -327,7 +334,7 @@ def test_backup_restore(storage_policy, backup_type, storage_policy2): data String ) ENGINE=MergeTree() ORDER BY id - SETTINGS storage_policy='{storage_policy}' + SETTINGS storage_policy='{old_storage_policy}' """ ) @@ -343,22 +350,38 @@ def test_backup_restore(storage_policy, backup_type, storage_policy2): elif backup_type == "File": backup_destination = f"File('/backups/{backup_name}/')" - node.query(f"BACKUP TABLE encrypted_test TO {backup_destination}") + node.query( + f"BACKUP TABLE encrypted_test TO {backup_destination} SETTINGS decrypt_files_from_encrypted_disks={int(decrypt_files_from_encrypted_disks)}" + ) - if backup_type == "File" and storage_policy.find("encrypted") != -1: + storage_policy_changed = old_storage_policy != new_storage_policy + old_disk_encrypted = old_storage_policy.find("encrypted") != -1 + new_disk_encrypted = new_storage_policy.find("encrypted") != -1 + + if backup_type == "File": root_path = os.path.join(node.cluster.instances_dir, "backups", backup_name) + expect_encrypted_in_backup = ( + old_disk_encrypted and not decrypt_files_from_encrypted_disks + ) + + with open(f"{root_path}/metadata/default/encrypted_test.sql") as file: + assert file.read().startswith("CREATE TABLE default.encrypted_test") + + with open(f"{root_path}/.backup") as file: + found_encrypted_in_backup = ( + file.read().find("true") != -1 + ) + assert found_encrypted_in_backup == expect_encrypted_in_backup + with open( f"{root_path}/data/default/encrypted_test/all_1_1_0/data.bin", "rb" ) as file: - assert file.read().startswith(b"ENC") - with open(f"{root_path}/metadata/default/encrypted_test.sql") as file: - assert file.read().startswith("CREATE TABLE default.encrypted_test") - with open(f"{root_path}/.backup") as file: - assert file.read().find("true") != -1 + found_encrypted_in_backup = file.read().startswith(b"ENC") + assert found_encrypted_in_backup == expect_encrypted_in_backup node.query(f"DROP TABLE encrypted_test SYNC") - if storage_policy != storage_policy2: + if storage_policy_changed: node.query( f""" CREATE TABLE encrypted_test ( @@ -366,56 +389,22 @@ def test_backup_restore(storage_policy, backup_type, storage_policy2): data String ) ENGINE=MergeTree() ORDER BY id - SETTINGS storage_policy='{storage_policy2}' + SETTINGS storage_policy='{new_storage_policy}' """ ) - node.query( - f"RESTORE TABLE encrypted_test FROM {backup_destination} SETTINGS allow_different_table_def={int(storage_policy != storage_policy2)}" - ) + restore_command = f"RESTORE TABLE encrypted_test FROM {backup_destination} SETTINGS allow_different_table_def={int(storage_policy_changed)}" - assert node.query(select_query) == "(0,'data'),(1,'data')" + expect_error = None + if ( + old_disk_encrypted + and not new_disk_encrypted + and not decrypt_files_from_encrypted_disks + ): + expect_error = "can be restored only to an encrypted disk" - -def test_cannot_restore_encrypted_files_to_unencrypted_disk(): - node.query( - """ - CREATE TABLE encrypted_test ( - id Int64, - data String - ) ENGINE=MergeTree() - ORDER BY id - SETTINGS storage_policy='encrypted_policy' - """ - ) - - node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')") - assert ( - node.query("SELECT * FROM encrypted_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - - backup_name = new_backup_name() - backup_destination = ( - f"S3('http://minio1:9001/root/backups/{backup_name}', 'minio', 'minio123')" - ) - node.query(f"BACKUP TABLE encrypted_test TO {backup_destination}") - - node.query(f"DROP TABLE encrypted_test SYNC") - - node.query( - f""" - CREATE TABLE encrypted_test ( - id Int64, - data String - ) ENGINE=MergeTree() - ORDER BY id - SETTINGS storage_policy='local_policy' - """ - ) - - expected_error = "can be restored only to an encrypted disk" - assert expected_error in node.query_and_get_error( - f"RESTORE TABLE encrypted_test FROM {backup_destination} SETTINGS allow_different_table_def=1" - ) ->>>>>>> 9c08fb30995 (Add tests.) + if expect_error: + assert expect_error in node.query_and_get_error(restore_command) + else: + node.query(restore_command) + assert node.query(select_query) == "(0,'data'),(1,'data')" From 2ec94a42b728a7daa6ab0c2c72fab1119e5f2a1d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 5 May 2023 10:40:12 +0200 Subject: [PATCH 0324/2223] Remove default parameters from virtual functions. --- src/Disks/DiskEncrypted.h | 7 ++----- src/Disks/IDisk.cpp | 2 +- src/Disks/IDisk.h | 13 +++---------- 3 files changed, 6 insertions(+), 16 deletions(-) diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index c494dd6a216..530d9b2dc02 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -198,13 +198,10 @@ public: delegate->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function)); } - std::unique_ptr readEncryptedFile( - const String & path, const ReadSettings & settings, - std::optional read_hint, - std::optional file_size) const override + std::unique_ptr readEncryptedFile(const String & path, const ReadSettings & settings) const override { auto wrapped_path = wrappedPath(path); - return delegate->readFile(wrapped_path, settings, read_hint, file_size); + return delegate->readFile(wrapped_path, settings); } std::unique_ptr writeEncryptedFile( diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 9a5ae997b46..88dd65bfde7 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -52,7 +52,7 @@ void IDisk::removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_ba } } -std::unique_ptr IDisk::readEncryptedFile(const String &, const ReadSettings &, std::optional, std::optional) const +std::unique_ptr IDisk::readEncryptedFile(const String &, const ReadSettings &) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "File encryption is not implemented for disk of type {}", getDataSourceDescription().type); } diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 4d74fe8bbab..e3ac790d2b7 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -253,18 +253,11 @@ public: virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; /// Reads a file from an encrypted disk without decrypting it (only for encrypted disks). - virtual std::unique_ptr readEncryptedFile( /// NOLINT - const String & path, - const ReadSettings & settings = ReadSettings{}, - std::optional read_hint = {}, - std::optional file_size = {}) const; + virtual std::unique_ptr readEncryptedFile(const String & path, const ReadSettings & settings) const; /// Writes an already encrypted file to the disk (only for encrypted disks). - virtual std::unique_ptr writeEncryptedFile( /// NOLINT - const String & path, - size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, - WriteMode mode = WriteMode::Rewrite, - const WriteSettings & settings = {}) const; + virtual std::unique_ptr writeEncryptedFile( + const String & path, size_t buf_size, WriteMode mode, const WriteSettings & settings) const; /// Returns the size of an encrypted file (only for encrypted disks). virtual size_t getEncryptedFileSize(const String & path) const; From b068f0b619ceb83023ba116c66a3fcafec7e8305 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 5 May 2023 11:40:33 +0200 Subject: [PATCH 0325/2223] Fix build. --- src/Backups/BackupEntryFromSmallFile.cpp | 2 +- src/Backups/BackupIO_S3.cpp | 10 +++++----- src/Disks/IDisk.h | 4 ++-- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 4 ++-- .../ObjectStorages/DiskObjectStorageTransaction.cpp | 6 ++++-- 5 files changed, 14 insertions(+), 12 deletions(-) diff --git a/src/Backups/BackupEntryFromSmallFile.cpp b/src/Backups/BackupEntryFromSmallFile.cpp index 22487767689..d0a99056b59 100644 --- a/src/Backups/BackupEntryFromSmallFile.cpp +++ b/src/Backups/BackupEntryFromSmallFile.cpp @@ -21,7 +21,7 @@ namespace String readFile(const DiskPtr & disk, const String & file_path, bool copy_encrypted) { - auto buf = copy_encrypted ? disk->readEncryptedFile(file_path) : disk->readFile(file_path); + auto buf = copy_encrypted ? disk->readEncryptedFile(file_path, {}) : disk->readFile(file_path); String s; readStringUntilEOF(s, *buf); return s; diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 69f56078f9d..40ecde71173 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -157,8 +157,8 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s fs::path(s3_uri.key) / path_in_backup, 0, file_size, - /* dest_bucket= */ blob_path[0], - /* dest_key= */ blob_path[1], + /* dest_bucket= */ blob_path[1], + /* dest_key= */ blob_path[0], request_settings, object_attributes, threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupReaderS3"), @@ -196,7 +196,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src auto source_data_source_description = src_disk->getDataSourceDescription(); if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) { - /// getBlobPath() can return more than 2 elements if the file is stored as multiple objects in S3 bucket. + /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in S3 bucket. /// In this case we can't use the native copy. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) { @@ -204,8 +204,8 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src LOG_TRACE(log, "Copying file {} from disk {} to S3 using native copy", src_path, src_disk->getName()); copyS3File( client, - /* src_bucket */ blob_path[0], - /* src_key= */ blob_path[1], + /* src_bucket */ blob_path[1], + /* src_key= */ blob_path[0], start_pos, length, s3_uri.bucket, diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index e3ac790d2b7..6bbd7c26bec 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -240,8 +240,8 @@ public: /// Returns the path to a blob representing a specified file. /// The meaning of the returned path depends on disk's type. - /// E.g. for DiskLocal it the absolute path to the file and for DiskObjectStorage it's the name of the objects' namespace - /// combined with StoredObject::absolute_path for each stored object representing a specified file. + /// E.g. for DiskLocal it's the absolute path to the file and for DiskObjectStorage it's + /// StoredObject::remote_path for each stored object combined with the name of the objects' namespace. virtual Strings getBlobPath(const String & path) const = 0; using WriteBlobFunction = std::function & object_attributes)>; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index bfa1ed1ab26..129f1ab1ef7 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -584,11 +584,11 @@ Strings DiskObjectStorage::getBlobPath(const String & path) const auto objects = getStorageObjects(path); Strings res; res.reserve(objects.size() + 1); + for (const auto & object : objects) + res.emplace_back(object.remote_path); String objects_namespace = object_storage->getObjectsNamespace(); if (!objects_namespace.empty()) res.emplace_back(objects_namespace); - for (const auto & object : objects) - res.emplace_back(object.absolute_path); return res; } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index f98ac55889b..2c22df64d90 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -694,8 +694,10 @@ void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction( /// See DiskObjectStorage::getBlobPath(). Strings blob_path; blob_path.reserve(2); - blob_path.emplace_back(object_storage.getObjectsNamespace()); - blob_path.emplace_back(object.absolute_path); + blob_path.emplace_back(object.remote_path); + String objects_namespace = object_storage.getObjectsNamespace(); + if (!objects_namespace.empty()) + blob_path.emplace_back(objects_namespace); /// We always use mode Rewrite because we simulate append using metadata and different files size_t object_size = std::move(write_blob_function)(blob_path, WriteMode::Rewrite, object_attributes); From 68138395ebb4b7c129f9ef8c6c8fbd2a8dfc1c9b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 5 May 2023 15:31:12 +0200 Subject: [PATCH 0326/2223] Fix parameterized views when query parameter used multiple times in the query Example: CREATE VIEW view AS SELECT * FROM system.one WHERE dummy = {k1:Int}+1 OR dummy = {k1:Int}+2 ^^ ^^ Signed-off-by: Azat Khuzhin --- src/Interpreters/ActionsVisitor.cpp | 15 +++++++++++++-- .../02428_parameterized_view.reference | 3 ++- .../0_stateless/02428_parameterized_view.sh | 5 +++-- 3 files changed, 18 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index b3bb6e9db90..28ba2e3e838 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1216,11 +1216,22 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & else if (data.is_create_parameterized_view && query_parameter) { const auto data_type = DataTypeFactory::instance().get(query_parameter->type); - ColumnWithTypeAndName column(data_type,query_parameter->getColumnName()); + /// Use getUniqueName() to allow multiple use of query parameter in the query: + /// + /// CREATE VIEW view AS + /// SELECT * + /// FROM system.one + /// WHERE dummy = {k1:Int}+1 OR dummy = {k1:Int}+2 + /// ^^ ^^ + /// + /// NOTE: query in the VIEW will not be modified this is needed + /// only during analysis for CREATE VIEW to avoid duplicated + /// column names. + ColumnWithTypeAndName column(data_type, data.getUniqueName("__" + query_parameter->getColumnName())); data.addColumn(column); argument_types.push_back(data_type); - argument_names.push_back(query_parameter->name); + argument_names.push_back(column.name); } else { diff --git a/tests/queries/0_stateless/02428_parameterized_view.reference b/tests/queries/0_stateless/02428_parameterized_view.reference index 59a3f4d67dd..b73c52c478f 100644 --- a/tests/queries/0_stateless/02428_parameterized_view.reference +++ b/tests/queries/0_stateless/02428_parameterized_view.reference @@ -19,7 +19,8 @@ ERROR 50 ERROR 10 -ERROR +10 +20 20 ERROR 30 diff --git a/tests/queries/0_stateless/02428_parameterized_view.sh b/tests/queries/0_stateless/02428_parameterized_view.sh index 57a43ce08a0..3abfbfc22fc 100755 --- a/tests/queries/0_stateless/02428_parameterized_view.sh +++ b/tests/queries/0_stateless/02428_parameterized_view.sh @@ -62,7 +62,8 @@ $CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv2(price=50)" 2>&1 | grep $CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv3 AS SELECT * FROM test_02428_Catalog WHERE Price={price:UInt64} AND Quantity=3" $CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv3(price=10)" -$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv4 AS SELECT * FROM test_02428_Catalog WHERE Price={price:UInt64} AND Quantity={price:UInt64}" 2>&1 | grep -Fq "DUPLICATE_COLUMN" && echo 'ERROR' || echo 'OK' +$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv4 AS SELECT * FROM test_02428_Catalog WHERE Price={price:UInt64} OR Price={price:UInt64}*2" +$CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv4(price=10) ORDER BY Price" $CLICKHOUSE_CLIENT -q "CREATE DATABASE ${CLICKHOUSE_TEST_UNIQUE_NAME}" $CLICKHOUSE_CLIENT -q "CREATE TABLE ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog (Name String, Price UInt64, Quantity UInt64) ENGINE = Memory" @@ -130,4 +131,4 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog" $CLICKHOUSE_CLIENT -q "DROP DATABASE ${CLICKHOUSE_TEST_UNIQUE_NAME}" $CLICKHOUSE_CLIENT -q "DROP VIEW 02428_trace_view" $CLICKHOUSE_CLIENT -q "DROP TABLE 02428_otel_traces_trace_id_ts" -$CLICKHOUSE_CLIENT -q "DROP TABLE 02428_otel_traces" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "DROP TABLE 02428_otel_traces" From ea5976180969fafb7dc7db8b5be5ff2d9c2976f0 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 15:25:04 +0200 Subject: [PATCH 0327/2223] fix OptimizeRegularExpression --- src/Common/OptimizedRegularExpression.cpp | 33 +++++++++++++++-------- src/Common/tests/gtest_optimize_re.cpp | 32 ++++++++++++---------- 2 files changed, 40 insertions(+), 25 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 68f5b86877e..92b3ad32ecd 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -63,12 +63,13 @@ const char * analyzeImpl( bool is_first_call = begin == regexp.data(); int depth = 0; is_trivial = true; + bool is_prefix = true; required_substring.clear(); bool has_alternative_on_depth_0 = false; bool has_case_insensitive_flag = false; /// Substring with a position. - using Substring = std::pair; + using Substring = std::pair; using Substrings = std::vector; Substrings trivial_substrings(1); @@ -98,6 +99,9 @@ const char * analyzeImpl( auto finish_non_trivial_char = [&](bool create_new_substr = true) { + is_trivial = false; + if (create_new_substr) + is_prefix = false; if (depth != 0) return; @@ -106,6 +110,7 @@ const char * analyzeImpl( if (alter.suffix) { alter.literal += last_substring->first; + alter.suffix = false; } } @@ -126,16 +131,24 @@ const char * analyzeImpl( if (alter.prefix) { alter.literal = last_substring->first + alter.literal; + alter.prefix = is_prefix; } } if (group_required_string.prefix) + { last_substring->first += group_required_string.literal; + last_substring->second = is_prefix; + } else { finish_non_trivial_char(); last_substring->first = group_required_string.literal; + last_substring->second = false; } + + is_prefix = is_prefix && group_required_string.prefix && group_required_string.suffix; + /// if we can still append, no need to finish it. e.g. abc(de)fg should capture abcdefg if (!last_substring->first.empty() && !group_required_string.suffix) { @@ -185,7 +198,6 @@ const char * analyzeImpl( goto ordinary; default: /// all other escape sequences are not supported - is_trivial = false; finish_non_trivial_char(); break; } @@ -196,6 +208,7 @@ const char * analyzeImpl( case '|': is_trivial = false; + is_prefix = false; ++pos; if (depth == 0) { @@ -205,6 +218,7 @@ const char * analyzeImpl( break; case '(': + /// bracket does not break is_prefix. for example abc(d) has a prefix 'abcd' is_trivial = false; if (!in_square_braces) { @@ -258,7 +272,6 @@ const char * analyzeImpl( case '[': in_square_braces = true; ++depth; - is_trivial = false; finish_non_trivial_char(); ++pos; break; @@ -270,7 +283,6 @@ const char * analyzeImpl( --depth; if (depth == 0) in_square_braces = false; - is_trivial = false; finish_non_trivial_char(); ++pos; break; @@ -284,7 +296,6 @@ const char * analyzeImpl( break; case '^': case '$': case '.': case '+': - is_trivial = false; finish_non_trivial_char(); ++pos; break; @@ -296,7 +307,6 @@ const char * analyzeImpl( case '?': [[fallthrough]]; case '*': - is_trivial = false; if (depth == 0 && !last_substring->first.empty() && !in_square_braces) { last_substring->first.resize(last_substring->first.size() - 1); @@ -318,8 +328,9 @@ const char * analyzeImpl( default: if (depth == 0 && !in_curly_braces && !in_square_braces) { + /// record the first position of last string. if (last_substring->first.empty()) - last_substring->second = pos - begin; + last_substring->second = is_prefix; last_substring->first.push_back(*pos); } ++pos; @@ -328,10 +339,9 @@ const char * analyzeImpl( } finish: - finish_non_trivial_char(false); - if (!is_trivial) { + finish_non_trivial_char(false); /// we calculate required substring even though has_alternative_on_depth_0. /// we will clear the required substring after putting it to alternatives. if (!has_case_insensitive_flag) @@ -357,7 +367,7 @@ finish: if (max_length >= MIN_LENGTH_FOR_STRSTR || (!is_first_call && max_length > 0)) { required_substring.literal = candidate_it->first; - required_substring.prefix = candidate_it->second == 0; + required_substring.prefix = candidate_it->second; required_substring.suffix = candidate_it + 1 == trivial_substrings.end(); } } @@ -365,7 +375,8 @@ finish: else if (!trivial_substrings.empty()) { required_substring.literal = trivial_substrings.front().first; - required_substring.prefix = trivial_substrings.front().second == 0; + /// trivial string means the whole regex is a simple string literal, so the prefix and suffix should be true. + required_substring.prefix = true; required_substring.suffix = true; } diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp index 556700f1fcc..3710666d336 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -4,37 +4,40 @@ TEST(OptimizeRE, analyze) { - auto test_f = [](const std::string & regexp, const std::string & answer, std::vector expect_alternatives = {}, bool trival_expected = false) + auto test_f = [](const std::string & regexp, const std::string & required, std::vector expect_alternatives = {}, bool trival_expected = false, bool prefix_expected = false) { - std::string required; + std::string answer; bool is_trivial; bool is_prefix; std::vector alternatives; - OptimizedRegularExpression::analyze(regexp, required, is_trivial, is_prefix, alternatives); + OptimizedRegularExpression::analyze(regexp, answer, is_trivial, is_prefix, alternatives); std::cerr << regexp << std::endl; EXPECT_EQ(required, answer); EXPECT_EQ(alternatives, expect_alternatives); EXPECT_EQ(is_trivial, trival_expected); + EXPECT_EQ(is_prefix, prefix_expected); }; - test_f("abc", "abc", {}, true); + test_f("abc", "abc", {}, true, true); test_f("c([^k]*)de", ""); - test_f("abc(de)fg", "abcdefg"); - test_f("abc(de|xyz)fg", "abc", {"abcdefg", "abcxyzfg"}); - test_f("abc(de?f|xyz)fg", "abc", {"abcd", "abcxyzfg"}); + test_f("abc(de)fg", "abcdefg", {}, false, true); + test_f("abc(de|xyz)fg", "abc", {"abcdefg", "abcxyzfg"}, false, true); + test_f("abc(de?f|xyz)fg", "abc", {"abcd", "abcxyzfg"}, false, true); test_f("abc|fgk|xyz", "", {"abc","fgk", "xyz"}); - test_f("(abc)", "abc"); + test_f("(abc)", "abc", {}, false, true); test_f("(abc|fgk)", "", {"abc","fgk"}); test_f("(abc|fgk)(e|f|zkh|)", "", {"abc","fgk"}); test_f("abc(abc|fg)xyzz", "xyzz", {"abcabcxyzz","abcfgxyzz"}); + test_f("((abc|fg)kkk*)xyzz", "xyzz", {"abckk", "fgkk"}); + test_f("abc(*(abc|fg)*)xyzz", "xyzz"); test_f("abc[k]xyzz", "xyzz"); test_f("(abc[k]xyzz)", "xyzz"); - test_f("abc((de)fg(hi))jk", "abcdefghijk"); - test_f("abc((?:de)fg(?:hi))jk", "abcdefghijk"); - test_f("abc((de)fghi+zzz)jk", "abcdefghi"); - test_f("abc((de)fg(hi))?jk", "abc"); - test_f("abc((de)fghi?zzz)jk", "abcdefgh"); + test_f("abc((de)fg(hi))jk", "abcdefghijk", {}, false, true); + test_f("abc((?:de)fg(?:hi))jk", "abcdefghijk", {}, false, true); + test_f("abc((de)fghi+zzz)jk", "abcdefghi", {}, false, true); + test_f("abc((de)fg(hi))?jk", "abc", {}, false, true); + test_f("abc((de)fghi?zzz)jk", "abcdefgh", {}, false, true); test_f("abc(*cd)jk", "cdjk"); - test_f(R"(abc(de|xyz|(\{xx\}))fg)", "abc", {"abcdefg", "abcxyzfg", "abc{xx}fg"}); + test_f(R"(abc(de|xyz|(\{xx\}))fg)", "abc", {"abcdefg", "abcxyzfg", "abc{xx}fg"}, false, true); test_f("abc(abc|fg)?xyzz", "xyzz"); test_f("abc(abc|fg){0,1}xyzz", "xyzz"); test_f("abc(abc|fg)xyzz|bcdd?k|bc(f|g|h?)z", "", {"abcabcxyzz", "abcfgxyzz", "bcd", "bc"}); @@ -43,4 +46,5 @@ TEST(OptimizeRE, analyze) test_f(R"([Bb]ai[Dd]u[Ss]pider(?:-[A-Za-z]{1,30})(?:-[A-Za-z]{1,30}|)|bingbot|\bYeti(?:-[a-z]{1,30}|)|Catchpoint(?: bot|)|[Cc]harlotte|Daumoa(?:-feedfetcher|)|(?:[a-zA-Z]{1,30}-|)Googlebot(?:-[a-zA-Z]{1,30}|))", "", {"pider-", "bingbot", "Yeti-", "Yeti", "Catchpoint bot", "Catchpoint", "harlotte", "Daumoa-feedfetcher", "Daumoa", "-Googlebot", "Googlebot"}); test_f("abc|(:?xx|yy|zz|x?)def", "", {"abc", "def"}); test_f("abc|(:?xx|yy|zz|x?){1,2}def", "", {"abc", "def"}); + test_f(R"(\\A(?:(?:[-0-9_a-z]+(?:\\.[-0-9_a-z]+)*)/k8s1)\\z)", "/k8s1"); } From 7df0e9d933267d7a2595e7d970c190650e09954c Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 15:33:08 +0200 Subject: [PATCH 0328/2223] fix broken link --- docs/en/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 66f661cce60..4abc41cdf42 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2268,7 +2268,7 @@ In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in t ### Use RegExpTree Dictionary on cloud -We have shown how RegExpTree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../../statements/select/into-outfile.md) clause. +We have shown how RegExpTree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../statements/select/into-outfile.md) clause. ```sql select * from dictionary(regexp_dict) into outfile('regexp_dict.csv') From d50e6fe8682a7a77572dfb3ea11541fecad25702 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 May 2023 15:35:16 +0200 Subject: [PATCH 0329/2223] Fix build after bad conflicts resolution --- src/Functions/FunctionGenerateRandomStructure.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 4cbbdd84c4a..16dac4f5112 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -426,19 +426,19 @@ REGISTER_FUNCTION(GenerateRandomStructure) { factory.registerFunction( { - R"( + .description=R"( Generates a random table structure. This function takes 2 optional constant arguments: the number of columns in the result structure (random by default) and random seed (random by default) The maximum number of columns is 128. The function returns a value of type String. )", - FunctionDocumentation::Examples{ - {"random", "SELECT generateRandomStructure()"}, - {"with specified number of columns", "SELECT generateRandomStructure(10)"}, - {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, + .examples{ + {"random", "SELECT generateRandomStructure()", "c1 UInt32, c2 FixedString(25)"}, + {"with specified number of columns", "SELECT generateRandomStructure(3)", "c1 String, c2 Array(Int32), c3 LowCardinality(String)"}, + {"with specified seed", "SELECT generateRandomStructure(1, 42)", "c1 UInt128"}, }, - FunctionDocumentation::Categories{"Random"} + .categories{"Random"} }, FunctionFactory::CaseSensitive); } From dea5cbcf4e01af9793ba1beac777c88a7567cc8c Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Tue, 16 May 2023 16:39:00 +0200 Subject: [PATCH 0330/2223] Slightly update comment --- src/Common/HashTable/HashTableKeyHolder.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/HashTable/HashTableKeyHolder.h b/src/Common/HashTable/HashTableKeyHolder.h index d1b38744255..e2f9d80db53 100644 --- a/src/Common/HashTable/HashTableKeyHolder.h +++ b/src/Common/HashTable/HashTableKeyHolder.h @@ -88,7 +88,7 @@ inline StringRef & ALWAYS_INLINE keyHolderGetKey(DB::ArenaKeyHolder & holder) inline void ALWAYS_INLINE keyHolderPersistKey(DB::ArenaKeyHolder & holder) { - // Normally, our hash table shouldn't ask us to persist a zero key, + // Normally, our hash table shouldn't ask to persist a zero key, // but it can happened in the case of clearable hash table (ClearableHashSet, for example). // The clearable hash table doesn't use zero storage and // distinguishes empty keys by using cell version, not the value itself. From 633c37e3e197d35857daa1509904a46c6aa1a966 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 16 May 2023 14:54:52 +0000 Subject: [PATCH 0331/2223] Update tests --- .../configs/named_collections.xml | 10 ++++++++ tests/integration/test_s3_cluster/test.py | 24 ++++++++++++++++--- .../configs/named_collections.xml | 8 +++++++ tests/integration/test_storage_url/test.py | 16 +++++++++++-- ...l_cluster_with_named_collections.reference | 16 ------------- ...7_s3_url_cluster_with_named_collections.sh | 14 ----------- 6 files changed, 53 insertions(+), 35 deletions(-) create mode 100644 tests/integration/test_s3_cluster/configs/named_collections.xml create mode 100644 tests/integration/test_storage_url/configs/named_collections.xml delete mode 100644 tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.reference delete mode 100755 tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.sh diff --git a/tests/integration/test_s3_cluster/configs/named_collections.xml b/tests/integration/test_s3_cluster/configs/named_collections.xml new file mode 100644 index 00000000000..64d1bd98df2 --- /dev/null +++ b/tests/integration/test_s3_cluster/configs/named_collections.xml @@ -0,0 +1,10 @@ + + + + http://minio1:9001/root/data/{clickhouse,database}/* + minio + minio123 + CSV> + + + diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index 69e36dbf9b4..41f19cdd12d 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -67,20 +67,20 @@ def started_cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "s0_0_0", - main_configs=["configs/cluster.xml"], + main_configs=["configs/cluster.xml", "configs/named_collections.xml"], macros={"replica": "node1", "shard": "shard1"}, with_minio=True, with_zookeeper=True, ) cluster.add_instance( "s0_0_1", - main_configs=["configs/cluster.xml"], + main_configs=["configs/cluster.xml", "configs/named_collections.xml"], macros={"replica": "replica2", "shard": "shard1"}, with_zookeeper=True, ) cluster.add_instance( "s0_1_0", - main_configs=["configs/cluster.xml"], + main_configs=["configs/cluster.xml", "configs/named_collections.xml"], macros={"replica": "replica1", "shard": "shard2"}, with_zookeeper=True, ) @@ -406,3 +406,21 @@ def test_cluster_with_header(started_cluster): ) == "SomeValue\n" ) + + +def test_cluster_with_named_collection(started_cluster): + node = started_cluster.instances["s0_0_0"] + + pure_s3 = node.query("""SELECT * from s3(test_s3) ORDER BY (c1, c2, c3)""") + + s3_cluster = node.query( + """SELECT * from s3Cluster(cluster_simple, test_s3) ORDER BY (c1, c2, c3)""" + ) + + assert TSV(pure_s3) == TSV(s3_cluster) + + s3_cluster = node.query( + """SELECT * from s3Cluster(cluster_simple, test_s3, structure='auto') ORDER BY (c1, c2, c3)""" + ) + + assert TSV(pure_s3) == TSV(s3_cluster) diff --git a/tests/integration/test_storage_url/configs/named_collections.xml b/tests/integration/test_storage_url/configs/named_collections.xml new file mode 100644 index 00000000000..e8b0aa91f53 --- /dev/null +++ b/tests/integration/test_storage_url/configs/named_collections.xml @@ -0,0 +1,8 @@ + + + + http://nginx:80/test_1 + TSV> + + + diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index 8dd502f1bde..d7f9125e29c 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -4,7 +4,7 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/conf.xml"], with_nginx=True + "node1", main_configs=["configs/conf.xml", "configs/named_collections.xml"], with_nginx=True ) @@ -35,7 +35,7 @@ def test_partition_by(): assert result.strip() == "1\t2\t3" -def test_urlCluster(): +def test_url_cluster(): result = node1.query( f"select * from urlCluster('test_cluster_two_shards', 'http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')" ) @@ -50,6 +50,18 @@ def test_urlCluster(): assert result.strip() == "1\t2\t3" +def test_url_cluster_with_named_collection(): + result = node1.query( + f"select * from urlCluster(test_cluster_one_shard_three_replicas_localhost, test_url)" + ) + assert result.strip() == "3\t2\t1" + + result = node1.query( + f"select * from urlCluster(test_cluster_one_shard_three_replicas_localhost, test_url, structure='auto')" + ) + assert result.strip() == "3\t2\t1" + + def test_table_function_url_access_rights(): node1.query("CREATE USER OR REPLACE u1") diff --git a/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.reference b/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.reference deleted file mode 100644 index a753e633bab..00000000000 --- a/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.reference +++ /dev/null @@ -1,16 +0,0 @@ -1 2 3 -4 5 6 -7 8 9 -0 0 0 -1 2 3 -4 5 6 -7 8 9 -0 0 0 -1 2 3 -4 5 6 -7 8 9 -0 0 0 -1 2 3 -4 5 6 -7 8 9 -0 0 0 diff --git a/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.sh b/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.sh deleted file mode 100755 index 6526ca8961f..00000000000 --- a/tests/queries/0_stateless/02737_s3_url_cluster_with_named_collections.sh +++ /dev/null @@ -1,14 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, no-s3-storage - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT -q "create named collection $CLICKHOUSE_TEST_UNIQUE_NAME as url='http://localhost:11111/test/a.tsv'" -$CLICKHOUSE_CLIENT -q "select * from s3Cluster(test_cluster_one_shard_three_replicas_localhost, $CLICKHOUSE_TEST_UNIQUE_NAME)" -$CLICKHOUSE_CLIENT -q "select * from s3Cluster(test_cluster_one_shard_three_replicas_localhost, $CLICKHOUSE_TEST_UNIQUE_NAME, structure='auto')" -$CLICKHOUSE_CLIENT -q "select * from urlCluster(test_cluster_one_shard_three_replicas_localhost, $CLICKHOUSE_TEST_UNIQUE_NAME)" -$CLICKHOUSE_CLIENT -q "select * from urlCluster(test_cluster_one_shard_three_replicas_localhost, $CLICKHOUSE_TEST_UNIQUE_NAME, structure='auto')" -$CLICKHOUSE_CLIENT -q "drop named collection $CLICKHOUSE_TEST_UNIQUE_NAME" - From 77adb7c8bc4c680503d2937a8448f097fef1af3d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 16 May 2023 17:00:05 +0200 Subject: [PATCH 0332/2223] Update 02535_analyzer_group_by_use_nulls reference --- .../02535_analyzer_group_by_use_nulls.reference | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference index 50755627996..63610604ddd 100644 --- a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference +++ b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference @@ -5,15 +5,25 @@ GROUP BY ROLLUP(number, number % 2) ORDER BY (number, number % 2, val) SETTINGS group_by_use_nulls=1; 0 0 0 +0 \N 0 1 1 1 +1 \N 1 2 0 2 +2 \N 2 3 1 3 +3 \N 3 4 0 4 +4 \N 4 5 1 5 +5 \N 5 6 0 6 +6 \N 6 7 1 7 +7 \N 7 8 0 8 +8 \N 8 9 1 9 +9 \N 9 \N \N 45 set optimize_group_by_function_keys = 0; SELECT number, number % 2, sum(number) AS val From 724949927b0da0d1150f271f51e0fe11e214e472 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 16 May 2023 17:36:48 +0200 Subject: [PATCH 0333/2223] Add logging --- src/Interpreters/Cache/FileCache.cpp | 15 ++++++++++++--- src/Interpreters/Cache/LRUFileCachePriority.cpp | 7 ++++++- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 2d5744b630e..9ab7943e263 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -573,8 +573,6 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) else queue_size += 1; - size_t removed_size = 0; - class EvictionCandidates final : public std::vector { public: @@ -600,6 +598,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) std::unordered_map to_delete; + size_t removed_size = 0; auto iterate_func = [&](LockedKey & locked_key, FileSegmentMetadataPtr segment_metadata) { chassert(segment_metadata->file_segment->assertCorrectness()); @@ -655,8 +654,18 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) { /// max_size == 0 means unlimited cache size, /// max_element_size means unlimited number of cache elements. - return (main_priority->getSizeLimit() != 0 && main_priority->getSize(cache_lock) + size - removed_size > main_priority->getSizeLimit()) + const bool is_overflow = (main_priority->getSizeLimit() != 0 + && main_priority->getSize(cache_lock) + size - removed_size > main_priority->getSizeLimit()) || (main_priority->getElementsLimit() != 0 && queue_size > main_priority->getElementsLimit()); + + LOG_TEST( + log, "Overflow: {}, size: {}, ready to remove: {}, current cache size: {}/{}, elements: {}/{}, while reserving for {}:{}", + is_overflow, size, removed_size, + main_priority->getSize(cache_lock), main_priority->getSizeLimit(), + main_priority->getElementsCount(cache_lock), main_priority->getElementsLimit(), + file_segment.key(), file_segment.offset()); + + return is_overflow; }; main_priority->iterate( diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 5e05acd9680..3c36962a0e5 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -44,7 +44,7 @@ IFileCachePriority::Iterator LRUFileCachePriority::add( throw Exception( ErrorCodes::LOGICAL_ERROR, "Not enough space to add {}:{} with size {}: current size: {}/{}", - key, offset, size, current_size, getSizeLimit()); + key, offset, size, current_size, size_limit); } auto iter = queue.insert(queue.end(), Entry(key, offset, size, key_metadata)); @@ -161,6 +161,11 @@ void LRUFileCachePriority::LRUFileCacheIterator::annul() void LRUFileCachePriority::LRUFileCacheIterator::updateSize(int64_t size) { + LOG_TEST( + cache_priority->log, + "Update size with {} in LRU queue for key: {}, offset: {}, previous size: {}", + size, queue_iter->key, queue_iter->offset, queue_iter->size); + cache_priority->current_size += size; queue_iter->size += size; From 0da82945ac2df6c79d7a47ae628346b3d1fda6e0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 16 May 2023 18:18:48 +0200 Subject: [PATCH 0334/2223] fix --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 3 ++- .../0_stateless/02440_mutations_finalization.reference | 2 +- .../queries/0_stateless/02440_mutations_finalization.sql | 5 +++-- .../0_stateless/02441_alter_delete_and_drop_column.sql | 8 +++++++- 4 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 9f7ae3222a4..e3c9a54023c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1841,7 +1841,8 @@ MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( MutationCommands commands; for (auto it = begin; it != end; ++it) { - chassert(mutation_pointer < it->second->entry->znode_name); + /// FIXME uncomment this assertion after relesing 23.5 (currently it fails in Upgrade check) + /// chassert(mutation_pointer < it->second->entry->znode_name); mutation_ids.push_back(it->second->entry->znode_name); const auto & commands_from_entry = it->second->entry->commands; commands.insert(commands.end(), commands_from_entry.begin(), commands_from_entry.end()); diff --git a/tests/queries/0_stateless/02440_mutations_finalization.reference b/tests/queries/0_stateless/02440_mutations_finalization.reference index a8b9c2acdce..c4bad0a3806 100644 --- a/tests/queries/0_stateless/02440_mutations_finalization.reference +++ b/tests/queries/0_stateless/02440_mutations_finalization.reference @@ -2,4 +2,4 @@ 1 0000000000 UPDATE n = 2 WHERE n = 1 ['all_0_0_0'] 0 2 -0000000000 UPDATE n = 2 WHERE n = 1 [] 1 +0000000000 UPDATE n = 2 WHERE n = 1 [] diff --git a/tests/queries/0_stateless/02440_mutations_finalization.sql b/tests/queries/0_stateless/02440_mutations_finalization.sql index 796dcde8e4e..c522d8ab9df 100644 --- a/tests/queries/0_stateless/02440_mutations_finalization.sql +++ b/tests/queries/0_stateless/02440_mutations_finalization.sql @@ -6,6 +6,7 @@ system stop merges mut; alter table mut update n = 2 where n = 1; -- it will create MUTATE_PART entry, but will not execute it +system sync replica mut pull; select mutation_id, command, parts_to_do_names, is_done from system.mutations where database=currentDatabase() and table='mut'; -- merges (and mutations) will start again after detach/attach, we need to avoid this somehow... @@ -26,8 +27,8 @@ select mutation_id, command, parts_to_do_names, is_done from system.mutations wh alter table mut modify setting max_number_of_mutations_for_replica=100; system sync replica mut; --- and now it should +-- and now it should (is_done may be 0, but it's okay) select * from mut; -select mutation_id, command, parts_to_do_names, is_done from system.mutations where database=currentDatabase() and table='mut'; +select mutation_id, command, parts_to_do_names from system.mutations where database=currentDatabase() and table='mut'; drop table tmp; -- btw, it will check that mutation can be cancelled between blocks on shutdown diff --git a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql index d274fae1a4f..b9b1b645e8e 100644 --- a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql +++ b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql @@ -5,11 +5,17 @@ insert into mut values (1, 2, 3), (10, 20, 30); system stop merges mut; alter table mut delete where n = 10; + +-- a funny way to wait for a MUTATE_PART to be assigned +select sleepEachRow(2) from url('http://localhost:8123/?param_tries={1..10}&query=' || encodeURLComponent( + 'select 1 where ''MUTATE_PART'' not in (select type from system.replication_queue where database=''' || currentDatabase() || ''' and table=''mut'')' + ), 'LineAsString', 's String') settings max_threads=1 format Null; + alter table mut drop column k settings alter_sync=0; system sync replica mut pull; -- a funny way to wait for ALTER_METADATA to disappear from the replication queue -select sleepEachRow(1) from url('http://localhost:8123/?param_tries={1..30}&query=' || encodeURLComponent( +select sleepEachRow(2) from url('http://localhost:8123/?param_tries={1..10}&query=' || encodeURLComponent( 'select * from system.replication_queue where database=''' || currentDatabase() || ''' and table=''mut'' and type=''ALTER_METADATA''' ), 'LineAsString', 's String') settings max_threads=1 format Null; From 26488185a03abf7a313a281d2ea36d0df267a4cc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 16 May 2023 18:48:12 +0200 Subject: [PATCH 0335/2223] fix test --- .../0_stateless/02439_merge_selecting_partitions.reference | 2 +- .../queries/0_stateless/02439_merge_selecting_partitions.sql | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02439_merge_selecting_partitions.reference b/tests/queries/0_stateless/02439_merge_selecting_partitions.reference index 7dae4cb5eff..e836994b3aa 100644 --- a/tests/queries/0_stateless/02439_merge_selecting_partitions.reference +++ b/tests/queries/0_stateless/02439_merge_selecting_partitions.reference @@ -1 +1 @@ -/test/02439/default/block_numbers/123 +/test/02439/s1/default/block_numbers/123 diff --git a/tests/queries/0_stateless/02439_merge_selecting_partitions.sql b/tests/queries/0_stateless/02439_merge_selecting_partitions.sql index 28bb0014cca..de57e35a5a7 100644 --- a/tests/queries/0_stateless/02439_merge_selecting_partitions.sql +++ b/tests/queries/0_stateless/02439_merge_selecting_partitions.sql @@ -1,5 +1,5 @@ -create table rmt (n int, m int) engine=ReplicatedMergeTree('/test/02439/{database}', '1') partition by n order by n; +create table rmt (n int, m int) engine=ReplicatedMergeTree('/test/02439/{shard}/{database}', '{replica}') partition by n order by n; insert into rmt select number, number from numbers(50); insert into rmt values (1, 2); insert into rmt values (1, 3); @@ -20,5 +20,5 @@ select sleepEachRow(3) as higher_probability_of_reproducing_the_issue format Nul system flush logs; -- it should not list unneeded partitions where we cannot merge anything -select distinct path from system.zookeeper_log where path like '/test/02439/' || currentDatabase() || '/block_numbers/%' +select distinct path from system.zookeeper_log where path like '/test/02439/s1/' || currentDatabase() || '/block_numbers/%' and op_num in ('List', 'SimpleList', 'FilteredList') and path not like '%/block_numbers/1' From b12eefc69444c7171ea3c0f2ba4c96bc51deece8 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 16 May 2023 18:57:04 +0000 Subject: [PATCH 0336/2223] fix timeout units and log message --- src/Common/PoolBase.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index a9c595c440c..ba19c3e2150 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -146,12 +146,12 @@ public: if (timeout < 0) { - LOG_INFO(log, "No free connections in pool. Waiting undefinitelly."); + LOG_INFO(log, "No free connections in pool. Waiting indefinitely."); available.wait(lock); } else { - auto timeout_ms = std::chrono::microseconds(timeout); + auto timeout_ms = std::chrono::milliseconds(timeout); LOG_INFO(log, "No free connections in pool. Waiting {} ms.", timeout_ms.count()); available.wait_for(lock, timeout_ms); } From 64bf477ff29c6541d5c96c6ea333fb371d40f1c4 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 16 May 2023 14:58:04 -0400 Subject: [PATCH 0337/2223] add DDL --- docs/en/operations/named-collections.md | 152 ++++++++++++++++++++---- 1 file changed, 127 insertions(+), 25 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 0189cc0a8e2..b65438b97f3 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -2,34 +2,89 @@ slug: /en/operations/named-collections sidebar_position: 69 sidebar_label: "Named collections" +title: "Named collections" --- -# Storing details for connecting to external sources in configuration files +Named collections provide a way to store collections of key-value pairs to be +used to configure integrations with external sources. You can use named collections with +dictionaries, tables, table functions, and object storage. + +Named collections can be configured with DDL or in configuration files and are applied +when ClickHouse starts. They simplify the creation of objects and the hiding of credentials +from users without administrative access. + +## Storing details for connecting to external sources in the system database + +### Permissions to create named collections with DDL + +To manage named collections with DDL a user must have the `named_control_collection` privilege. This can be assigned by adding a file to `/etc/clickhouse-server/users.d/`. The example gives the user `default` both the `access_management` and `named_collection_control` privileges: + +```xml title='/etc/clickhouse-server/users.d/user_default.xml' + + + + 65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5 + 1 + + 1 + + + + +``` + +:::tip +In the above example the `passowrd_sha256_hex` value is the hexadecimal representation of the SHA256 hash of the password. This configuration for the user `default` has the attribute `replace=true` as in the default configuration has a plain text `password` set, and it is not possible to have both plaintext and sha256 hex passwords set for a user. +::: + +## Storing details for connecting to external sources in configuration files -Details for connecting to external sources (dictionaries, tables, table functions) can be saved -in configuration files and thus simplify the creation of objects and hide credentials -from users with only SQL access. Parameters can be set in XML `CSV` and overridden in SQL `, format = 'TSV'`. The parameters in SQL can be overridden using format `key` = `value`: `compression_method = 'gzip'`. -Named collections are stored in the `config.xml` file of the ClickHouse server in the `` section and are applied when ClickHouse starts. +Named collections are stored in the system table `system.named_collections` or in `config.xml` file of the ClickHouse server in the `` section and are applied when ClickHouse starts. + +### DDL example + +```sql +CREATE NAMED COLLECTION name AS +key_1 = 'value', +key_2 = 'value2', +url = 'https://connection.url/' +``` + +### XML example -Example of configuration: ```xml $ cat /etc/clickhouse-server/config.d/named_collections.xml - ... + + value + value_2 + https://connection.url/ + ``` -## Named collections for accessing S3. +## Named collections for accessing S3 The description of parameters see [s3 Table Function](../sql-reference/table-functions/s3.md). -Example of configuration: +### DDL example + +```sql +CREATE NAMED COLLECTION s3_mydata AS +access_key_id = 'AKIAIOSFODNN7EXAMPLE', +secret_access_key = 'wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY', +format = 'CSV', +url = 'https://s3.us-east-1.amazonaws.com/yourbucket/mydata/' +``` + +### XML example + ```xml @@ -43,23 +98,23 @@ Example of configuration: ``` -### Example of using named collections with the s3 function +### s3() function and S3 Table Named collection examples + +Both of the following examples use the same named collection `s3_mydata`: + +#### s3() function ```sql INSERT INTO FUNCTION s3(s3_mydata, filename = 'test_file.tsv.gz', format = 'TSV', structure = 'number UInt64', compression_method = 'gzip') SELECT * FROM numbers(10000); - -SELECT count() -FROM s3(s3_mydata, filename = 'test_file.tsv.gz') - -┌─count()─┐ -│ 10000 │ -└─────────┘ -1 rows in set. Elapsed: 0.279 sec. Processed 10.00 thousand rows, 90.00 KB (35.78 thousand rows/s., 322.02 KB/s.) ``` -### Example of using named collections with an S3 table +:::tip +The first argument to the `s3()` function above is the name of the collection, `s3_mydata`. Without named collections, the access key ID, secret, format, and URL would all be passed in every call to the `s3()` function. +::: + +#### S3 table ```sql CREATE TABLE s3_engine_table (number Int64) @@ -78,7 +133,22 @@ SELECT * FROM s3_engine_table LIMIT 3; The description of parameters see [mysql](../sql-reference/table-functions/mysql.md). -Example of configuration: +### DDL example + +```sql +CREATE NAMED COLLECTION mymysql AS +user = 'myuser', +password = 'mypass', +host = '127.0.0.1', +port = 3306, +database = 'test' +connection_pool_size = 8 +on_duplicate_clause = 1 +replace_query = 1 +``` + +### XML example + ```xml @@ -96,7 +166,11 @@ Example of configuration: ``` -### Example of using named collections with the mysql function +### mysql() function, MySQL table, MySQL database, and Dictionary named collection examples + +The four following examples use the same named collection `mymysql`: + +#### mysql() function ```sql SELECT count() FROM mysql(mymysql, table = 'test'); @@ -105,8 +179,11 @@ SELECT count() FROM mysql(mymysql, table = 'test'); │ 3 │ └─────────┘ ``` +:::note +The named collection does not specify the `table` parameter, so it is specified in the function call as `table = 'test'`. +::: -### Example of using named collections with an MySQL table +#### MySQL table ```sql CREATE TABLE mytable(A Int64) ENGINE = MySQL(mymysql, table = 'test', connection_pool_size=3, replace_query=0); @@ -117,7 +194,11 @@ SELECT count() FROM mytable; └─────────┘ ``` -### Example of using named collections with database with engine MySQL +:::note +The DDL overrides the named collection setting for connection_pool_size. +::: + +#### MySQL database ```sql CREATE DATABASE mydatabase ENGINE = MySQL(mymysql); @@ -130,7 +211,7 @@ SHOW TABLES FROM mydatabase; └────────┘ ``` -### Example of using named collections with a dictionary with source MySQL +#### MySQL Dictionary ```sql CREATE DICTIONARY dict (A Int64, B String) @@ -150,6 +231,17 @@ SELECT dictGet('dict', 'B', 2); The description of parameters see [postgresql](../sql-reference/table-functions/postgresql.md). +```sql +CREATE NAMED COLLECTION mypg AS +user = 'pguser', +password = 'jw8s0F4', +host = '127.0.0.1', +port = 5432, +database = 'test', +schema = 'test_schema', +connection_pool_size = 8 +``` + Example of configuration: ```xml @@ -229,12 +321,22 @@ SELECT dictGet('dict', 'b', 2); └─────────────────────────┘ ``` -## Named collections for accessing remote ClickHouse database +## Named collections for accessing a remote ClickHouse database The description of parameters see [remote](../sql-reference/table-functions/remote.md/#parameters). Example of configuration: +```sql +CREATE NAMED COLLECTION remote1 AS +host = 'remote_host', +port = 9000, +database = 'system', +user = 'foo', +password = 'secret', +secure = 1 +``` + ```xml From abacf1f99088ee52b46173537a65aab6b0e6306c Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 16 May 2023 19:14:54 +0000 Subject: [PATCH 0338/2223] add missing `quota_key` in operator== for connections --- src/Client/ConnectionPool.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index c3d0955019e..9b1f7dc5b9d 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -192,6 +192,7 @@ inline bool operator==(const ConnectionPoolFactory::Key & lhs, const ConnectionP { return lhs.max_connections == rhs.max_connections && lhs.host == rhs.host && lhs.port == rhs.port && lhs.default_database == rhs.default_database && lhs.user == rhs.user && lhs.password == rhs.password + && lhs.quota_key == rhs.quota_key && lhs.cluster == rhs.cluster && lhs.cluster_secret == rhs.cluster_secret && lhs.client_name == rhs.client_name && lhs.compression == rhs.compression && lhs.secure == rhs.secure && lhs.priority == rhs.priority; } From 35f00f72b3bad1025453181f785c20c9945dda36 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 May 2023 22:42:43 +0200 Subject: [PATCH 0339/2223] add functional test --- tests/queries/0_stateless/02751_match_constant_needle.reference | 1 + tests/queries/0_stateless/02751_match_constant_needle.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02751_match_constant_needle.reference create mode 100644 tests/queries/0_stateless/02751_match_constant_needle.sql diff --git a/tests/queries/0_stateless/02751_match_constant_needle.reference b/tests/queries/0_stateless/02751_match_constant_needle.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02751_match_constant_needle.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02751_match_constant_needle.sql b/tests/queries/0_stateless/02751_match_constant_needle.sql new file mode 100644 index 00000000000..71bdcc7cb0a --- /dev/null +++ b/tests/queries/0_stateless/02751_match_constant_needle.sql @@ -0,0 +1 @@ +select match('default/k8s1', '\\A(?:(?:[-0-9_a-z]+(?:\\.[-0-9_a-z]+)*)/k8s1)\\z'); From 15cb6276883288ce57966e067bd1efdae428ed54 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Tue, 16 May 2023 16:35:41 -0600 Subject: [PATCH 0340/2223] Update postgresql.md The type cannot be optional, so we need to be clear in the docs that [type1] [type2] cannot be in square brackets, so removed them --- docs/en/engines/table-engines/integrations/postgresql.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index fbd6d944363..f27d4d48f75 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -13,8 +13,8 @@ The PostgreSQL engine allows to perform `SELECT` and `INSERT` queries on data th ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], + name1 type1 [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], + name2 type2 [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], ... ) ENGINE = PostgreSQL('host:port', 'database', 'table', 'user', 'password'[, `schema`]); ``` From 282297b677d0650b5d268f895d1eaa5c233e3eb1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 16 May 2023 23:46:01 +0000 Subject: [PATCH 0341/2223] binary encoding of IPv6 in protobuf --- src/Formats/ProtobufSerializer.cpp | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 8bdef0e7d3f..ee33d2d9991 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -1,4 +1,5 @@ #include +#include "Common/formatIPv6.h" #if USE_PROTOBUF # include @@ -1852,25 +1853,26 @@ namespace write_function = [this](IPv6 value) { - ipToString(value, text_buffer); + text_buffer = String(IPV6_BINARY_LENGTH, '\0'); + memcpy(text_buffer.data(), &value.toUnderType(), IPV6_BINARY_LENGTH); writeStr(text_buffer); }; read_function = [this]() -> IPv6 { readStr(text_buffer); - return parse(text_buffer); + if (text_buffer.size() != IPV6_BINARY_LENGTH) + throw Exception(ErrorCodes::PROTOBUF_BAD_CAST, + "Could not convert bytes field {} to IPv6 for inserting into column {} - field size {} is not equal to IPv6 size {}", + field_descriptor.full_name(), column_name, text_buffer.size(), IPV6_BINARY_LENGTH); + IPv6 value; + memcpy(&value.toUnderType(), text_buffer.data(), IPV6_BINARY_LENGTH); + return value; }; default_function = [this]() -> IPv6 { return parse(field_descriptor.default_value_string()); }; } - static void ipToString(const IPv6 & ip, String & str) - { - WriteBufferFromString buf{str}; - writeText(ip, buf); - } - std::function write_function; std::function read_function; std::function default_function; From 0a44a69dc82bac8f490399b8197c6220d673e1d3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 17 May 2023 00:22:13 +0000 Subject: [PATCH 0342/2223] remove unnecessary header --- src/Formats/ProtobufSerializer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index ee33d2d9991..f690800d145 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -1,5 +1,4 @@ #include -#include "Common/formatIPv6.h" #if USE_PROTOBUF # include From f4ac4c3f9d783a83e86185373d90cdf4bbe6e1ba Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 17 May 2023 03:14:30 +0200 Subject: [PATCH 0343/2223] Corrections after review. --- src/Backups/BackupFileInfo.cpp | 12 +++++++----- src/Backups/BackupIO_Disk.cpp | 2 ++ src/Backups/BackupIO_File.cpp | 4 ++-- src/Storages/StorageMemory.cpp | 28 ++++++++++++++++------------ 4 files changed, 27 insertions(+), 19 deletions(-) diff --git a/src/Backups/BackupFileInfo.cpp b/src/Backups/BackupFileInfo.cpp index 42546d1b1b8..d539ada55c4 100644 --- a/src/Backups/BackupFileInfo.cpp +++ b/src/Backups/BackupFileInfo.cpp @@ -36,7 +36,7 @@ namespace { /// We cannot reuse base backup because our file is smaller /// than file stored in previous backup - if (new_entry_info.size < base_backup_info.first) + if ((new_entry_info.size < base_backup_info.first) || !base_backup_info.first) return CheckBackupResult::HasNothing; if (base_backup_info.first == new_entry_info.size) @@ -48,7 +48,10 @@ namespace struct ChecksumsForNewEntry { - UInt128 full_checksum; + /// 0 is the valid checksum of empty data. + UInt128 full_checksum = 0; + + /// std::nullopt here means that it's too difficult to calculate a partial checksum so it shouldn't be used. std::optional prefix_checksum; }; @@ -58,8 +61,7 @@ namespace { ChecksumsForNewEntry res; /// The partial checksum should be calculated before the full checksum to enable optimization in BackupEntryWithChecksumCalculation. - if (prefix_size > 0) - res.prefix_checksum = entry->getPartialChecksum(prefix_size); + res.prefix_checksum = entry->getPartialChecksum(prefix_size); res.full_checksum = entry->getChecksum(); return res; } @@ -116,7 +118,7 @@ BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const Backu /// We have info about this file in base backup /// If file has no checksum -- calculate and fill it. - if (base_backup_file_info.has_value()) + if (base_backup_file_info) { LOG_TRACE(log, "File {} found in base backup, checking for equality", adjusted_path); CheckBackupResult check_base = checkBaseBackupForFile(*base_backup_file_info, info); diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index 3b1651bb223..1514b4c24c7 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -37,6 +37,7 @@ void BackupReaderDisk::copyFileToDisk(const String & path_in_backup, size_t file DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { /// Use IDisk::copyFile() as a more optimal way to copy a file if it's possible. + /// However IDisk::copyFile() can't use throttling for reading, and can't copy an encrypted file or do appending. bool has_throttling = disk->isRemote() ? static_cast(read_settings.remote_throttler) : static_cast(read_settings.local_throttler); if (!has_throttling && (write_mode == WriteMode::Rewrite) && !encrypted_in_backup) { @@ -106,6 +107,7 @@ void BackupWriterDisk::copyFileFromDisk(const String & path_in_backup, DiskPtr s bool copy_encrypted, UInt64 start_pos, UInt64 length) { /// Use IDisk::copyFile() as a more optimal way to copy a file if it's possible. + /// However IDisk::copyFile() can't use throttling for reading, and can't copy an encrypted file or copy a part of the file. bool has_throttling = src_disk->isRemote() ? static_cast(read_settings.remote_throttler) : static_cast(read_settings.local_throttler); if (!has_throttling && !start_pos && !copy_encrypted) { diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 7ffae26d16f..e1a3f336521 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -41,7 +41,7 @@ std::unique_ptr BackupReaderFile::readFile(const String & fi void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { - /// std::filesystem::copy() can copy from the filesystem only, and it can't do the throttling or appending. + /// std::filesystem::copy() can copy from the filesystem only, and can't do throttling or appending. bool has_throttling = static_cast(read_settings.local_throttler); if (!has_throttling && (write_mode == WriteMode::Rewrite)) { @@ -121,7 +121,7 @@ void BackupWriterFile::removeFiles(const Strings & file_names) void BackupWriterFile::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, bool copy_encrypted, UInt64 start_pos, UInt64 length) { - /// std::filesystem::copy() can copy from the filesystem only, and it can't do the throttling or copy a part of the file. + /// std::filesystem::copy() can copy from the filesystem only, and can't do throttling or copy a part of the file. bool has_throttling = static_cast(read_settings.local_throttler); if (!has_throttling) { diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index c9654cfd105..31e45db55cb 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -319,24 +319,28 @@ namespace IndexForNativeFormat index; { auto data_file_path = temp_dir / fs::path{file_paths[data_bin_pos]}.filename(); - { - auto data_out_compressed = temp_disk->writeFile(data_file_path); - CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size}; - NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index}; - for (const auto & block : *blocks) - block_out.write(block); - } + auto data_out_compressed = temp_disk->writeFile(data_file_path); + auto data_out = std::make_unique(*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size); + NativeWriter block_out{*data_out, 0, metadata_snapshot->getSampleBlock(), false, &index}; + for (const auto & block : *blocks) + block_out.write(block); + data_out->finalize(); + data_out.reset(); + data_out_compressed->finalize(); + data_out_compressed.reset(); backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared(temp_disk, data_file_path)}; } /// Writing index.mrk { auto index_mrk_path = temp_dir / fs::path{file_paths[index_mrk_pos]}.filename(); - { - auto index_mrk_out_compressed = temp_disk->writeFile(index_mrk_path); - CompressedWriteBuffer index_mrk_out{*index_mrk_out_compressed}; - index.write(index_mrk_out); - } + auto index_mrk_out_compressed = temp_disk->writeFile(index_mrk_path); + auto index_mrk_out = std::make_unique(*index_mrk_out_compressed); + index.write(*index_mrk_out); + index_mrk_out->finalize(); + index_mrk_out.reset(); + index_mrk_out_compressed->finalize(); + index_mrk_out_compressed.reset(); backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared(temp_disk, index_mrk_path)}; } From 4a6c7254e874fa839feeb80b46930fa84b6ee19e Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 17 May 2023 03:43:35 +0000 Subject: [PATCH 0344/2223] --multiquery -> -n -q syntax sugar --- programs/client/Client.cpp | 9 ++++++++- programs/local/LocalServer.cpp | 12 ++++++++++-- src/Client/ClientBase.cpp | 21 +++++++++++++++------ src/Client/ClientBase.h | 1 + 4 files changed, 34 insertions(+), 9 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index fb30f60a0b8..b28740ab80e 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1181,7 +1181,7 @@ void Client::processOptions(const OptionsDescription & options_description, void Client::processConfig() { /// Batch mode is enabled if one of the following is true: - /// - -e (--query) command line option is present. + /// - -q (--query) command line option is present. /// The value of the option is used as the text of query (or of multiple queries). /// If stdin is not a terminal, INSERT data for the first query is read from it. /// - stdin is not a terminal. In this case queries are read from it. @@ -1381,6 +1381,13 @@ void Client::readArguments( allow_repeated_settings = true; else if (arg == "--allow_merge_tree_settings") allow_merge_tree_settings = true; + else if ((arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-') && arg == "--multiquery") + { + /** Transforms from '--multiquery ' into '--multiquery -q ' */ + ++arg_num; + arg = argv[arg_num]; + addMultiquery(arg, common_arguments); + } else common_arguments.emplace_back(arg); } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 8925f50fe97..91e4e077ac7 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -818,8 +818,16 @@ void LocalServer::readArguments(int argc, char ** argv, Arguments & common_argum { for (int arg_num = 1; arg_num < argc; ++arg_num) { - const char * arg = argv[arg_num]; - common_arguments.emplace_back(arg); + std::string_view arg = argv[arg_num]; + if ((arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-') && arg == "--multiquery") + { + /** Transforms from '--multiquery ' into '--multiquery -q ' */ + ++arg_num; + arg = argv[arg_num]; + addMultiquery(arg, common_arguments); + } + else + common_arguments.emplace_back(arg); } } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 77a93a25e9b..f0bd70ad798 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1246,6 +1246,14 @@ void ClientBase::setInsertionTable(const ASTInsertQuery & insert_query) } +void ClientBase::addMultiquery(std::string_view query, Arguments & common_arguments) const +{ + common_arguments.emplace_back("--multiquery"); + common_arguments.emplace_back("-q"); + common_arguments.emplace_back(query); +} + + void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr parsed_query) { auto query = query_to_execute; @@ -2592,15 +2600,19 @@ void ClientBase::init(int argc, char ** argv) ("version-clean", "print version in machine-readable format and exit") ("config-file,C", po::value(), "config-file path") + + ("query,q", po::value(), "query") ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") + + ("multiquery,n", "Indicates that --query can execute several SQL statements. if --query was not specified then SQL statement can be set right after --multiquery. Example --multiquery \"select 1\"") + ("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)") + ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") + ("query_id", po::value(), "query_id") ("database,d", po::value(), "database") ("history_file", po::value(), "path to history file") - ("query,q", po::value(), "query") ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") - ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") - ("query_id", po::value(), "query_id") ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off") ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") @@ -2612,9 +2624,6 @@ void ClientBase::init(int argc, char ** argv) ("log-level", po::value(), "log level") ("server_logs_file", po::value(), "put server logs into specified file") - ("multiline,m", "multiline") - ("multiquery,n", "multiquery") - ("suggestion_limit", po::value()->default_value(10000), "Suggestion limit for how many databases, tables and columns to fetch.") diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 5926f73f51a..8d56c7eb051 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -129,6 +129,7 @@ protected: void setInsertionTable(const ASTInsertQuery & insert_query); + void addMultiquery(std::string_view query, Arguments & common_arguments) const; private: void receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool partial_result_on_first_cancel); From 502fcace7e480360656c0240d8038e12d04f61e5 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 17 May 2023 03:43:53 +0000 Subject: [PATCH 0345/2223] Added functional tests for updated --multiquery --- .../02751_multiquery_with_argument.reference | 24 +++++++++++++ .../02751_multiquery_with_argument.sh | 34 +++++++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 tests/queries/0_stateless/02751_multiquery_with_argument.reference create mode 100755 tests/queries/0_stateless/02751_multiquery_with_argument.sh diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.reference b/tests/queries/0_stateless/02751_multiquery_with_argument.reference new file mode 100644 index 00000000000..33288ec5bcb --- /dev/null +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.reference @@ -0,0 +1,24 @@ +100 +101 +102 +103 +Syntax error +Empty query +Empty query +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +Bad arguments +Bad arguments +Bad arguments +Bad arguments +Bad arguments +Bad arguments +Bad arguments +BAD_ARGUMENTS +Bad arguments +BAD_ARGUMENTS +Bad arguments +Bad arguments +Bad arguments +Bad arguments diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.sh b/tests/queries/0_stateless/02751_multiquery_with_argument.sh new file mode 100755 index 00000000000..153004c752e --- /dev/null +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL --multiquery "SELECT 100" +$CLICKHOUSE_LOCAL --multiquery "SELECT 101;" +$CLICKHOUSE_LOCAL --multiquery "SELECT 102;SELECT 103;" + +# Invalid SQL. +$CLICKHOUSE_LOCAL --multiquery "SELECT 200; S" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_LOCAL --multiquery "; SELECT 201;" 2>&1 | grep -o 'Empty query' +$CLICKHOUSE_LOCAL --multiquery "; S; SELECT 202" 2>&1 | grep -o 'Empty query' + +# Error expectation cases. +# -n is prohibited +$CLICKHOUSE_LOCAL -n "SELECT 301" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_LOCAL -n "SELECT 302;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_LOCAL -n "SELECT 304;SELECT 305;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_LOCAL --multiquery --multiquery 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL -n --multiquery 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL --multiquery -n 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL --multiquery --multiquery "SELECT 306; SELECT 307;" 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL -n --multiquery "SELECT 307; SELECT 308;" 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL --multiquery "SELECT 309; SELECT 310;" --multiquery 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL --multiquery "SELECT 311;" --multiquery "SELECT 312;" 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL --multiquery "SELECT 313;" -n "SELECT 314;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_LOCAL --multiquery "SELECT 315;" --query "SELECT 316;" 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL -n "SELECT 320" --query "SELECT 317;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_LOCAL --query --multiquery --multiquery "SELECT 318;" 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL --query --multiquery "SELECT 319;" 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL --query -n "SELECT 400;" 2>&1 | grep -o 'Bad arguments' +$CLICKHOUSE_LOCAL --query -n --multiquery "SELECT 401;" 2>&1 | grep -o 'Bad arguments' \ No newline at end of file From 61cd36ce85ba557f165b1a1579e17e30832ff68c Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 17 May 2023 03:44:38 +0000 Subject: [PATCH 0346/2223] fixed 02048_parallel_reading_from_infile.sh --- .../0_stateless/02048_parallel_reading_from_infile.sh | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh b/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh index d53fe8dd305..f055ea304b2 100755 --- a/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh +++ b/tests/queries/0_stateless/02048_parallel_reading_from_infile.sh @@ -26,13 +26,12 @@ SELECT count() FROM test_infile_parallel WHERE Value='first'; SELECT count() FROM test_infile_parallel WHERE Value='second'; EOF -# Error code is 36 (BAD_ARGUMENTS). It is not ignored. -${CLICKHOUSE_CLIENT} --multiquery " -DROP TABLE IF EXISTS test_infile_parallel; +# Error code is 27 (DB::ParsingException). It is not ignored. +${CLICKHOUSE_CLIENT} -m --multiquery --query "DROP TABLE IF EXISTS test_infile_parallel; CREATE TABLE test_infile_parallel (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory(); SET input_format_allow_errors_num=0; INSERT INTO test_infile_parallel FROM INFILE '${CLICKHOUSE_TMP}/test_infile_parallel*' FORMAT TSV; -" 2>&1 | grep -q "36" && echo "Correct" || echo 'Fail' +" 2>&1 | grep -q "27" && echo "Correct" || echo 'Fail' ${CLICKHOUSE_LOCAL} --multiquery < Date: Wed, 17 May 2023 03:45:09 +0000 Subject: [PATCH 0347/2223] Updated documentation for --multiquery --- docs/en/interfaces/cli.md | 2 +- docs/ru/interfaces/cli.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 9bf4a465962..6d5395d46e3 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -181,7 +181,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--queries-file` – file path with queries to execute. You must specify either `query` or `queries-file` option. - `--database, -d` – Select the current default database. Default value: the current database from the server settings (‘default’ by default). - `--multiline, -m` – If specified, allow multiline queries (do not send the query on Enter). -- `--multiquery, -n` – If specified, allow processing multiple queries separated by semicolons. +- `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`. - `--format, -f` – Use the specified default format to output the result. - `--vertical, -E` – If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `–format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables. - `--time, -t` – If specified, print the query execution time to ‘stderr’ in non-interactive mode. diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 8a4ec083242..4c22eae0207 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -132,7 +132,7 @@ $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="numbe - `--queries-file` - путь к файлу с запросами для выполнения. Необходимо указать только одну из опций: `query` или `queries-file`. - `--database, -d` — выбрать текущую БД. Без указания значение берется из настроек сервера (по умолчанию — БД ‘default’). - `--multiline, -m` — если указано — разрешить многострочные запросы, не отправлять запрос по нажатию Enter. -- `--multiquery, -n` — если указано — разрешить выполнять несколько запросов, разделённых точкой с запятой. +- `--multiquery, -n` — Если указано, то после опции `--query` могут быть перечислены несколько запросов, разделенных точкой с запятой. Для удобства можно также опустить `--query` и передавать запросы непосредственно после `--multiquery`. - `--format, -f` — использовать указанный формат по умолчанию для вывода результата. - `--vertical, -E` — если указано, использовать по умолчанию формат [Vertical](../interfaces/formats.md#vertical) для вывода результата. То же самое, что `–format=Vertical`. В этом формате каждое значение выводится на отдельной строке, что удобно для отображения широких таблиц. - `--time, -t` — если указано, в неинтерактивном режиме вывести время выполнения запроса в поток ‘stderr’. From 29b10ae33625ae3db0aa42aea99ce585f9c8409a Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 17 May 2023 04:03:52 +0000 Subject: [PATCH 0348/2223] reordered options --- src/Client/ClientBase.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f0bd70ad798..c978115790e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2604,12 +2604,12 @@ void ClientBase::init(int argc, char ** argv) ("query,q", po::value(), "query") ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") - ("multiquery,n", "Indicates that --query can execute several SQL statements. if --query was not specified then SQL statement can be set right after --multiquery. Example --multiquery \"select 1\"") ("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)") + ("database,d", po::value(), "database") ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") ("query_id", po::value(), "query_id") - ("database,d", po::value(), "database") + ("history_file", po::value(), "path to history file") ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") From c10435489487626448b8e861074ef9b41ecb9f4e Mon Sep 17 00:00:00 2001 From: fhbai Date: Wed, 17 May 2023 14:39:30 +0800 Subject: [PATCH 0349/2223] fix --- src/Functions/array/arrayDotProduct.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index e3c80775f1b..d17c223cc2f 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -57,7 +57,7 @@ public: } template - static ResultType apply( + static inline NO_SANITIZE_UNDEFINED ResultType apply( const T * left, const U * right, size_t size) From 3c80e30f02f40acd372942506d8d57ec803ef2b6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 11 May 2023 16:59:19 +0200 Subject: [PATCH 0350/2223] Fix per-query IO/BACKUPs throttling settings (when default profile has them) When some of this settings was set for default profile (in users.xml/users.yml), then it will be always used regardless of what user passed. Fix this by not inherit per-query throttlers, for this they should be reset before making query context and they should not be initialized as before in Context::makeQueryContext(), since makeQueryContext() called too early, when user settings was not read yet. But there we had also initialization of per-server throttling, move this into the ContextSharedPart::configureServerWideThrottling(), and call it once we have ServerSettings set. Also note, that this patch makes the following settings - server settings: - max_replicated_fetches_network_bandwidth_for_server - max_replicated_sends_network_bandwidth_for_server But this change should not affect anybody, since it is done with compatiblity (i.e. if this setting is set in users profile it will be read from it as well as a fallback). Signed-off-by: Azat Khuzhin --- src/Core/ServerSettings.h | 2 + src/Core/Settings.h | 4 +- src/Interpreters/Context.cpp | 150 +++++++----------- .../configs/limit_replication_config.xml | 8 +- .../test_replicated_fetches_bandwidth/test.py | 2 +- 5 files changed, 65 insertions(+), 101 deletions(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index ee3482414af..2a73930836a 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -23,6 +23,8 @@ namespace DB M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The maximum number of threads that would be used for loading outdated data parts on startup", 0) \ M(UInt64, outdated_part_loading_thread_pool_queue_size, 10000, "Queue size for parts loading thread pool.", 0) \ + M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ + M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \ M(UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0) \ M(UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 610c7135a75..874e31b8d37 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -101,8 +101,6 @@ class IColumn; M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, false, "Whether to use the cache of uncompressed blocks.", 0) \ M(Bool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ - M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited. Only has meaning at server startup.", 0) \ - M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited. Only has meaning at server startup.", 0) \ M(UInt64, max_remote_read_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for read.", 0) \ M(UInt64, max_remote_write_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for write.", 0) \ M(UInt64, max_local_read_bandwidth, 0, "The maximum speed of local reads in bytes per second.", 0) \ @@ -791,6 +789,8 @@ class IColumn; MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_distributed_schedule_pool_size, 16) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_remote_read_network_bandwidth_for_server, 0) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_remote_write_network_bandwidth_for_server, 0) \ + MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0) \ + MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0) \ /* ---- */ \ MAKE_OBSOLETE(M, DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic) \ MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b4bdb7cf233..0ef9ea53ee8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -631,6 +631,30 @@ struct ContextSharedPart : boost::noncopyable log->warning(message); warnings.push_back(message); } + + void configureServerWideThrottling() + { + if (auto bandwidth = server_settings.max_replicated_fetches_network_bandwidth_for_server) + replicated_fetches_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_replicated_sends_network_bandwidth_for_server) + replicated_sends_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_remote_read_network_bandwidth_for_server) + remote_read_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_remote_write_network_bandwidth_for_server) + remote_write_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_local_read_bandwidth_for_server) + local_read_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_local_write_bandwidth_for_server) + local_write_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings.max_backup_bandwidth_for_server) + backups_server_throttler = std::make_shared(bandwidth); + } }; @@ -1897,16 +1921,22 @@ void Context::makeQueryContext() { query_context = shared_from_this(); - /// Create throttlers, to inherit the ThrottlePtr in the context copies. - { - getRemoteReadThrottler(); - getRemoteWriteThrottler(); - - getLocalReadThrottler(); - getLocalWriteThrottler(); - - getBackupsThrottler(); - } + /// Throttling should not be inherited, otherwise if you will set + /// throttling for default profile you will not able to overwrite it + /// per-user/query. + /// + /// Note, that if you need to set it server-wide, you should use + /// per-server settings, i.e.: + /// - max_backup_bandwidth_for_server + /// - max_remote_read_network_bandwidth_for_server + /// - max_remote_write_network_bandwidth_for_server + /// - max_local_read_bandwidth_for_server + /// - max_local_write_bandwidth_for_server + remote_read_query_throttler.reset(); + remote_write_query_throttler.reset(); + local_read_query_throttler.reset(); + local_write_query_throttler.reset(); + backups_query_throttler.reset(); } void Context::makeSessionContext() @@ -2438,143 +2468,76 @@ BackgroundSchedulePool & Context::getMessageBrokerSchedulePool() const ThrottlerPtr Context::getReplicatedFetchesThrottler() const { - auto lock = getLock(); - if (!shared->replicated_fetches_throttler) - shared->replicated_fetches_throttler = std::make_shared( - settings.max_replicated_fetches_network_bandwidth_for_server); - return shared->replicated_fetches_throttler; } ThrottlerPtr Context::getReplicatedSendsThrottler() const { - auto lock = getLock(); - if (!shared->replicated_sends_throttler) - shared->replicated_sends_throttler = std::make_shared( - settings.max_replicated_sends_network_bandwidth_for_server); - return shared->replicated_sends_throttler; } ThrottlerPtr Context::getRemoteReadThrottler() const { - ThrottlerPtr throttler; - - const auto & query_settings = getSettingsRef(); - UInt64 bandwidth_for_server = shared->server_settings.max_remote_read_network_bandwidth_for_server; - if (bandwidth_for_server) - { - auto lock = getLock(); - if (!shared->remote_read_throttler) - shared->remote_read_throttler = std::make_shared(bandwidth_for_server); - throttler = shared->remote_read_throttler; - } - - if (query_settings.max_remote_read_network_bandwidth) + ThrottlerPtr throttler = shared->remote_read_throttler; + if (auto bandwidth = getSettingsRef().max_remote_read_network_bandwidth) { auto lock = getLock(); if (!remote_read_query_throttler) - remote_read_query_throttler = std::make_shared(query_settings.max_remote_read_network_bandwidth, throttler); + remote_read_query_throttler = std::make_shared(bandwidth, throttler); throttler = remote_read_query_throttler; } - return throttler; } ThrottlerPtr Context::getRemoteWriteThrottler() const { - ThrottlerPtr throttler; - - const auto & query_settings = getSettingsRef(); - UInt64 bandwidth_for_server = shared->server_settings.max_remote_write_network_bandwidth_for_server; - if (bandwidth_for_server) - { - auto lock = getLock(); - if (!shared->remote_write_throttler) - shared->remote_write_throttler = std::make_shared(bandwidth_for_server); - throttler = shared->remote_write_throttler; - } - - if (query_settings.max_remote_write_network_bandwidth) + ThrottlerPtr throttler = shared->remote_write_throttler; + if (auto bandwidth = getSettingsRef().max_remote_write_network_bandwidth) { auto lock = getLock(); if (!remote_write_query_throttler) - remote_write_query_throttler = std::make_shared(query_settings.max_remote_write_network_bandwidth, throttler); + remote_write_query_throttler = std::make_shared(bandwidth, throttler); throttler = remote_write_query_throttler; } - return throttler; } ThrottlerPtr Context::getLocalReadThrottler() const { - ThrottlerPtr throttler; - - if (shared->server_settings.max_local_read_bandwidth_for_server) - { - auto lock = getLock(); - if (!shared->local_read_throttler) - shared->local_read_throttler = std::make_shared(shared->server_settings.max_local_read_bandwidth_for_server); - throttler = shared->local_read_throttler; - } - - const auto & query_settings = getSettingsRef(); - if (query_settings.max_local_read_bandwidth) + ThrottlerPtr throttler = shared->local_read_throttler; + if (auto bandwidth = getSettingsRef().max_local_read_bandwidth) { auto lock = getLock(); if (!local_read_query_throttler) - local_read_query_throttler = std::make_shared(query_settings.max_local_read_bandwidth, throttler); + local_read_query_throttler = std::make_shared(bandwidth, throttler); throttler = local_read_query_throttler; } - return throttler; } ThrottlerPtr Context::getLocalWriteThrottler() const { - ThrottlerPtr throttler; - - if (shared->server_settings.max_local_write_bandwidth_for_server) - { - auto lock = getLock(); - if (!shared->local_write_throttler) - shared->local_write_throttler = std::make_shared(shared->server_settings.max_local_write_bandwidth_for_server); - throttler = shared->local_write_throttler; - } - - const auto & query_settings = getSettingsRef(); - if (query_settings.max_local_write_bandwidth) + ThrottlerPtr throttler = shared->local_write_throttler; + if (auto bandwidth = getSettingsRef().max_local_write_bandwidth) { auto lock = getLock(); if (!local_write_query_throttler) - local_write_query_throttler = std::make_shared(query_settings.max_local_write_bandwidth, throttler); + local_write_query_throttler = std::make_shared(bandwidth, throttler); throttler = local_write_query_throttler; } - return throttler; } ThrottlerPtr Context::getBackupsThrottler() const { - ThrottlerPtr throttler; - - if (shared->server_settings.max_backup_bandwidth_for_server) - { - auto lock = getLock(); - if (!shared->backups_server_throttler) - shared->backups_server_throttler = std::make_shared(shared->server_settings.max_backup_bandwidth_for_server); - throttler = shared->backups_server_throttler; - } - - const auto & query_settings = getSettingsRef(); - if (query_settings.max_backup_bandwidth) + ThrottlerPtr throttler = shared->backups_server_throttler; + if (auto bandwidth = getSettingsRef().max_backup_bandwidth) { auto lock = getLock(); if (!backups_query_throttler) - backups_query_throttler = std::make_shared(query_settings.max_backup_bandwidth, throttler); + backups_query_throttler = std::make_shared(bandwidth, throttler); throttler = backups_query_throttler; } - return throttler; } @@ -3633,7 +3596,10 @@ void Context::setApplicationType(ApplicationType type) shared->application_type = type; if (type == ApplicationType::SERVER) + { shared->server_settings.loadSettingsFromConfig(Poco::Util::Application::instance().config()); + shared->configureServerWideThrottling(); + } } void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config) diff --git a/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml b/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml index b18f0fbc93f..2a7b47e3560 100644 --- a/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml +++ b/tests/integration/test_replicated_fetches_bandwidth/configs/limit_replication_config.xml @@ -1,8 +1,4 @@ - - - 5242880 - 10485760 - - + 5242880 + 10485760 diff --git a/tests/integration/test_replicated_fetches_bandwidth/test.py b/tests/integration/test_replicated_fetches_bandwidth/test.py index 059102f8683..cd969746c31 100644 --- a/tests/integration/test_replicated_fetches_bandwidth/test.py +++ b/tests/integration/test_replicated_fetches_bandwidth/test.py @@ -12,7 +12,7 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance("node1", with_zookeeper=True) node2 = cluster.add_instance("node2", with_zookeeper=True) node3 = cluster.add_instance( - "node3", user_configs=["configs/limit_replication_config.xml"], with_zookeeper=True + "node3", main_configs=["configs/limit_replication_config.xml"], with_zookeeper=True ) From 7383da0c526399963ff3496c22b6a59f062ff98f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 11 May 2023 21:43:43 +0200 Subject: [PATCH 0351/2223] Fix per-query remote throttler remote throttler by some reason had been overwritten by the global one during reloads, likely this is for graceful reload of this option, but it breaks per-query throttling, remove this logic. Signed-off-by: Azat Khuzhin --- .../AzureBlobStorage/AzureObjectStorage.cpp | 1 - src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp | 5 ----- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h | 5 ----- src/Disks/ObjectStorages/IObjectStorage.cpp | 11 ----------- src/Disks/ObjectStorages/IObjectStorage.h | 11 ++++------- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 1 - 6 files changed, 4 insertions(+), 30 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 9db5d13a7f8..62c3216ad3f 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -282,7 +282,6 @@ void AzureObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguratio { auto new_settings = getAzureBlobStorageSettings(config, config_prefix, context); settings.set(std::move(new_settings)); - applyRemoteThrottlingSettings(context); /// We don't update client } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index e50e410823d..38c088ab213 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -156,11 +156,6 @@ void HDFSObjectStorage::copyObject( /// NOLINT } -void HDFSObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr context) -{ - applyRemoteThrottlingSettings(context); -} - std::unique_ptr HDFSObjectStorage::cloneObjectStorage(const std::string &, const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr) { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS object storage doesn't support cloning"); diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 4064a5c5b7f..fdc47ad16a6 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -105,11 +105,6 @@ public: void startup() override; - void applyNewSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - String getObjectsNamespace() const override { return ""; } std::unique_ptr cloneObjectStorage( diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 52e8b1a465d..1ee55a7b342 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -55,27 +55,16 @@ const std::string & IObjectStorage::getCacheName() const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getCacheName() is not implemented for object storage"); } -void IObjectStorage::applyRemoteThrottlingSettings(ContextPtr context) -{ - std::unique_lock lock{throttlers_mutex}; - remote_read_throttler = context->getRemoteReadThrottler(); - remote_write_throttler = context->getRemoteWriteThrottler(); -} - ReadSettings IObjectStorage::patchSettings(const ReadSettings & read_settings) const { - std::unique_lock lock{throttlers_mutex}; ReadSettings settings{read_settings}; - settings.remote_throttler = remote_read_throttler; settings.for_object_storage = true; return settings; } WriteSettings IObjectStorage::patchSettings(const WriteSettings & write_settings) const { - std::unique_lock lock{throttlers_mutex}; WriteSettings settings{write_settings}; - settings.remote_throttler = remote_write_throttler; settings.for_object_storage = true; return settings; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index f4f1b063ade..58a31fdc8c3 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -165,9 +165,10 @@ public: /// Apply new settings, in most cases reiniatilize client and some other staff virtual void applyNewSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) = 0; + const Poco::Util::AbstractConfiguration &, + const std::string & /*config_prefix*/, + ContextPtr) + {} /// Sometimes object storages have something similar to chroot or namespace, for example /// buckets in S3. If object storage doesn't have any namepaces return empty string. @@ -205,10 +206,6 @@ public: virtual WriteSettings patchSettings(const WriteSettings & write_settings) const; -protected: - /// Should be called from implementation of applyNewSettings() - void applyRemoteThrottlingSettings(ContextPtr context); - private: mutable std::mutex throttlers_mutex; ThrottlerPtr remote_read_throttler; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 79b3d3a2b8b..9faae3a6c62 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -443,7 +443,6 @@ void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & auto new_client = getClient(config, config_prefix, context, *new_s3_settings); s3_settings.set(std::move(new_s3_settings)); client.set(std::move(new_client)); - applyRemoteThrottlingSettings(context); } std::unique_ptr S3ObjectStorage::cloneObjectStorage( From fdfb1eda55b0c2ee95dc7284629f7ded6f4c9196 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 11 May 2023 22:15:05 +0200 Subject: [PATCH 0352/2223] Fix {Local,Remote}ReadThrottlerSleepMicroseconds metric values And also update the test, since now you could have slightly less sleep intervals, if query spend some time in other places. But what is important is that query_duration_ms does not exceeded calculated delay. Signed-off-by: Azat Khuzhin --- src/Common/Throttler.cpp | 16 ++++++++-------- src/Common/Throttler.h | 8 ++++---- .../02703_max_local_read_bandwidth.sh | 2 +- .../02703_max_local_write_bandwidth.sh | 2 +- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 4f99c24cc8d..4c1320db27a 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -61,20 +61,20 @@ UInt64 Throttler::add(size_t amount) throw Exception::createDeprecated(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED); /// Wait unless there is positive amount of tokens - throttling - Int64 sleep_time = 0; + Int64 sleep_time_ns = 0; if (max_speed && tokens_value < 0) { - sleep_time = static_cast(-tokens_value / max_speed * NS); - accumulated_sleep += sleep_time; - sleepForNanoseconds(sleep_time); - accumulated_sleep -= sleep_time; - ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_time / 1000UL); + sleep_time_ns = static_cast(-tokens_value / max_speed * NS); + accumulated_sleep += sleep_time_ns; + sleepForNanoseconds(sleep_time_ns); + accumulated_sleep -= sleep_time_ns; + ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_time_ns / 1000UL); } if (parent) - sleep_time += parent->add(amount); + sleep_time_ns += parent->add(amount); - return static_cast(sleep_time); + return static_cast(sleep_time_ns); } void Throttler::reset() diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index 4b117ae7637..7508065096b 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -34,15 +34,15 @@ public: const std::shared_ptr & parent_ = nullptr); /// Use `amount` tokens, sleeps if required or throws exception on limit overflow. - /// Returns duration of sleep in microseconds (to distinguish sleeping on different kinds of throttlers for metrics) + /// Returns duration of sleep in nanoseconds (to distinguish sleeping on different kinds of throttlers for metrics) UInt64 add(size_t amount); UInt64 add(size_t amount, ProfileEvents::Event event_amount, ProfileEvents::Event event_sleep_us) { - UInt64 sleep_us = add(amount); + UInt64 sleep_ns = add(amount); ProfileEvents::increment(event_amount, amount); - ProfileEvents::increment(event_sleep_us, sleep_us); - return sleep_us; + ProfileEvents::increment(event_sleep_us, sleep_ns / 1000UL); + return sleep_ns; } /// Not thread safe diff --git a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh index 130f3a29ade..d47e2f363bd 100755 --- a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh @@ -32,7 +32,7 @@ for read_method in "${read_methods[@]}"; do query_duration_ms >= 7e3, ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] > 8e6, ProfileEvents['LocalReadThrottlerBytes'] > 8e6, - ProfileEvents['LocalReadThrottlerSleepMicroseconds'] > 7e6 + ProfileEvents['LocalReadThrottlerSleepMicroseconds'] > 7e6*0.9 FROM system.query_log WHERE current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' AND type != 'QueryStart' " diff --git a/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh index 80713e90169..41165d35d37 100755 --- a/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -nm -q " query_duration_ms >= 7e3, ProfileEvents['WriteBufferFromFileDescriptorWriteBytes'] > 8e6, ProfileEvents['LocalWriteThrottlerBytes'] > 8e6, - ProfileEvents['LocalWriteThrottlerSleepMicroseconds'] > 7e6 + ProfileEvents['LocalWriteThrottlerSleepMicroseconds'] > 7e6*0.9 FROM system.query_log WHERE current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' AND type != 'QueryStart' " From 9fe4f1a934c86d5d42fef6bbd7c3d19db4e7f97b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 11 May 2023 21:58:07 +0200 Subject: [PATCH 0353/2223] Extensive coverage for bandwidth limiting settings Signed-off-by: Azat Khuzhin --- tests/integration/test_throttling/__init__.py | 0 .../configs/server_backups.xml | 34 ++ .../configs/server_overrides.xml | 3 + .../configs/users_overrides.xml | 3 + tests/integration/test_throttling/test.py | 413 ++++++++++++++++++ 5 files changed, 453 insertions(+) create mode 100644 tests/integration/test_throttling/__init__.py create mode 100644 tests/integration/test_throttling/configs/server_backups.xml create mode 100644 tests/integration/test_throttling/configs/server_overrides.xml create mode 100644 tests/integration/test_throttling/configs/users_overrides.xml create mode 100644 tests/integration/test_throttling/test.py diff --git a/tests/integration/test_throttling/__init__.py b/tests/integration/test_throttling/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_throttling/configs/server_backups.xml b/tests/integration/test_throttling/configs/server_backups.xml new file mode 100644 index 00000000000..d25c67a779c --- /dev/null +++ b/tests/integration/test_throttling/configs/server_backups.xml @@ -0,0 +1,34 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + +
+ s3 +
+
+
+
+
+ + + + http://minio1:9001/root/data/ + minio + minio123 + + + + + default + /backups/ + +
diff --git a/tests/integration/test_throttling/configs/server_overrides.xml b/tests/integration/test_throttling/configs/server_overrides.xml new file mode 100644 index 00000000000..197bf660500 --- /dev/null +++ b/tests/integration/test_throttling/configs/server_overrides.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_throttling/configs/users_overrides.xml b/tests/integration/test_throttling/configs/users_overrides.xml new file mode 100644 index 00000000000..197bf660500 --- /dev/null +++ b/tests/integration/test_throttling/configs/users_overrides.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py new file mode 100644 index 00000000000..ff8e7154d0d --- /dev/null +++ b/tests/integration/test_throttling/test.py @@ -0,0 +1,413 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +# pylint: disable=line-too-long + +# This test covers the following options: +# - max_backup_bandwidth +# - max_backup_bandwidth_for_server +# - max_local_read_bandwidth +# - max_local_read_bandwidth_for_server +# - max_local_write_bandwidth +# - max_local_write_bandwidth_for_server +# - max_remote_read_network_bandwidth +# - max_remote_read_network_bandwidth_for_server +# - max_remote_write_network_bandwidth +# - max_remote_write_network_bandwidth_for_server +# - and that max_backup_bandwidth from the query will override setting from the user profile + +import time +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + + +def elapsed(func, *args, **kwargs): + start = time.time() + ret = func(*args, **kwargs) + end = time.time() + return ret, end - start + + +node = cluster.add_instance( + "node", + stay_alive=True, + main_configs=["configs/server_backups.xml", "configs/server_overrides.xml"], + user_configs=["configs/users_overrides.xml"], + with_minio=True, +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield + finally: + cluster.shutdown() + + +@pytest.fixture(scope="function", autouse=True) +def revert_config(): + # Revert configs after the test, not before + yield + node.exec_in_container( + [ + "bash", + "-c", + f"echo '' > /etc/clickhouse-server/config.d/server_overrides.xml", + ] + ) + node.exec_in_container( + [ + "bash", + "-c", + f"echo '' > /etc/clickhouse-server/users.d/users_overrides.xml", + ] + ) + node.restart_clickhouse() + + +backup_id_counter = 0 + + +def next_backup_name(storage): + global backup_id_counter + if storage == "local": + backup_id_counter += 1 + return f"Disk('default', '{backup_id_counter}/')" + elif storage == "remote": + backup_id_counter += 1 + return f"S3(s3, '{backup_id_counter}/')" + else: + raise Exception(storage) + + +def node_update_config(mode, setting, value=None): + if mode is None: + return + if mode == "server": + config_path = "/etc/clickhouse-server/config.d/server_overrides.xml" + config_content = f""" + <{setting}>{value} + """ + else: + config_path = "/etc/clickhouse-server/users.d/users_overrides.xml" + config_content = f""" + + + + <{setting}>{value} + + + + """ + node.exec_in_container( + [ + "bash", + "-c", + f"echo '{config_content}' > {config_path}", + ] + ) + node.restart_clickhouse() + + +def assert_took(took, should_took): + assert took >= should_took[0] * 0.9 and took < should_took[1] + + +@pytest.mark.parametrize( + "policy,backup_name,mode,setting,value,should_took", + [ + # + # Local -> Local + # + pytest.param( + "default", + next_backup_name("local"), + None, + None, + None, + (0, 3), + id="no_local_throttling", + ), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "default", + next_backup_name("local"), + "user", + "max_backup_bandwidth", + "1M", + (7, 14), + id="user_local_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "default", + next_backup_name("local"), + "server", + "max_backup_bandwidth_for_server", + "2M", + (3, 7), + id="server_local_throttling", + ), + # + # Remote -> Local + # + pytest.param( + "s3", + next_backup_name("local"), + None, + None, + None, + (0, 3), + id="no_remote_to_local_throttling", + ), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "s3", + next_backup_name("local"), + "user", + "max_backup_bandwidth", + "1M", + (7, 14), + id="user_remote_to_local_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "s3", + next_backup_name("local"), + "server", + "max_backup_bandwidth_for_server", + "2M", + (3, 7), + id="server_remote_to_local_throttling", + ), + # + # Remote -> Remote + # + pytest.param( + "s3", + next_backup_name("remote"), + None, + None, + None, + (0, 3), + id="no_remote_to_remote_throttling", + ), + # No throttling for S3-to-S3, uses native copy + pytest.param( + "s3", + next_backup_name("remote"), + "user", + "max_backup_bandwidth", + "1M", + (0, 3), + id="user_remote_to_remote_throttling", + ), + # No throttling for S3-to-S3, uses native copy + pytest.param( + "s3", + next_backup_name("remote"), + "server", + "max_backup_bandwidth_for_server", + "2M", + (0, 3), + id="server_remote_to_remote_throttling", + ), + # + # Local -> Remote + # + # NOTE: S3 is complex, it will read file 3 times: + # - first for calculating the checksum + # - second for calculating the signature + # - and finally to write the payload to S3 + # Hence the value should be multipled by 3. + pytest.param( + "default", + next_backup_name("remote"), + None, + None, + None, + (0, 3), + id="no_local_to_remote_throttling", + ), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds, but for S3Client it is 2x more + pytest.param( + "default", + next_backup_name("remote"), + "user", + "max_backup_bandwidth", + "1M", + (7 * 3, 7 * 4 - 1), + id="user_local_to_remote_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds, but for S3Client it is 2x more + pytest.param( + "default", + next_backup_name("remote"), + "server", + "max_backup_bandwidth_for_server", + "2M", + (3 * 3, 3 * 5), + id="server_local_to_remote_throttling", + ), + ], +) +def test_backup_throttling(policy, backup_name, mode, setting, value, should_took): + node_update_config(mode, setting, value) + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='{policy}'; + insert into data select * from numbers(1e6); + """ + ) + _, took = elapsed(node.query, f"backup table data to {backup_name}") + assert_took(took, should_took) + + +def test_backup_throttling_override(): + node_update_config("user", "max_backup_bandwidth", "1M") + node.query( + """ + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9; + insert into data select * from numbers(1e6); + """ + ) + + backup_name = next_backup_name("local") + _, took = elapsed( + node.query, + f"backup table data to {backup_name}", + settings={ + "max_backup_bandwidth": "500K", + }, + ) + # reading 1e6*8 bytes with 500Ki default bandwith should take (8-0.5)/0.5=15 seconds + assert_took(took, (15, 20)) + + +@pytest.mark.parametrize( + "policy,mode,setting,value,should_took", + [ + # + # Local + # + pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "default", + "user", + "max_local_read_bandwidth", + "1M", + (7, 14), + id="user_local_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "default", + "server", + "max_local_read_bandwidth_for_server", + "2M", + (3, 7), + id="server_local_throttling", + ), + # + # Remote + # + pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "s3", + "user", + "max_remote_read_network_bandwidth", + "1M", + (7, 14), + id="user_remote_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "s3", + "server", + "max_remote_read_network_bandwidth_for_server", + "2M", + (3, 7), + id="server_remote_throttling", + ), + ], +) +def test_read_throttling(policy, mode, setting, value, should_took): + node_update_config(mode, setting, value) + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='{policy}'; + insert into data select * from numbers(1e6); + """ + ) + _, took = elapsed(node.query, f"select * from data") + assert_took(took, should_took) + + +@pytest.mark.parametrize( + "policy,mode,setting,value,should_took", + [ + # + # Local + # + pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "default", + "user", + "max_local_write_bandwidth", + "1M", + (7, 14), + id="local_user_throttling", + ), + # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "default", + "server", + "max_local_write_bandwidth_for_server", + "2M", + (3, 7), + id="local_server_throttling", + ), + # + # Remote + # + pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + # writeing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param( + "s3", + "user", + "max_remote_write_network_bandwidth", + "1M", + (7, 14), + id="user_remote_throttling", + ), + # writeing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + pytest.param( + "s3", + "server", + "max_remote_write_network_bandwidth_for_server", + "2M", + (3, 7), + id="server_remote_throttling", + ), + ], +) +def test_write_throttling(policy, mode, setting, value, should_took): + node_update_config(mode, setting, value) + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='{policy}'; + """ + ) + _, took = elapsed(node.query, f"insert into data select * from numbers(1e6)") + assert_took(took, should_took) From 3787b7f127750b573a98920f8e7243e67fe15d68 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 17 May 2023 12:16:18 +0200 Subject: [PATCH 0354/2223] Update Metadata.cpp --- src/Interpreters/Cache/Metadata.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 01cdc7f1d1b..c87eaabbbf2 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "Common/Exception.h" #include #include From 3d8b6b708fd655c4012afd1702092a36a3fb2e86 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 17 May 2023 10:20:29 +0000 Subject: [PATCH 0355/2223] Fix for analyzer: 02377_ ptimize_sorting_by_input_stream_properties_explain --- ..._input_stream_properties_explain.reference | 74 +++++++++++++++++++ ...ting_by_input_stream_properties_explain.sh | 5 +- 2 files changed, 78 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference index 0b34e878cee..69c325c21a9 100644 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference @@ -5,6 +5,12 @@ Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (None) Sorting (None) +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a +Sorting (Global): a ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): default.optimize_sorting.a_0 ASC +Sorting (None) +Sorting (None) -- disable optimization -> sorting order is NOT propagated from subquery -> full sort -- QUERY: set optimize_sorting_by_input_stream_properties=0;set query_plan_read_in_order=0;set max_threads=3;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a MergingSortedTransform 3 → 1 @@ -27,12 +33,24 @@ Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (Stream): a ASC Sorting (Stream): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a +Sorting (Global): a ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): default.optimize_sorting.a_0 ASC +Sorting (Stream): default.optimize_sorting.a_0 ASC +Sorting (Stream): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 +Sorting (None) +Sorting (Sorting for ORDER BY) +Sorting (Global): plus(default.optimize_sorting.a_0, 1_UInt8) ASC +Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a ASC -- ExpressionStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 Sorting (Global): plus(a, 1) ASC @@ -40,28 +58,54 @@ Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC Sorting (None) Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 +Sorting (Global): plus(a, 1) ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): plus(default.optimize_sorting.a_0, 1_UInt8) ASC +Sorting (None) +Sorting (Chunk): a ASC -- FilterStep preserves sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 +Sorting (Chunk): a ASC +Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 +Sorting (Chunk): a ASC +Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 +Sorting (Chunk): a ASC +Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a ASC -- FilterStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 Sorting (None) Sorting (None) Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 +Sorting (None) +Sorting (None) +Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (None) Sorting (None) Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0 +Sorting (None) +Sorting (None) +Sorting (Chunk): a ASC -- aliases break sorting order -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a Sorting (Global): a ASC @@ -72,6 +116,15 @@ Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (Stream): a ASC Sorting (Stream): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a +Sorting (Global): a ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): a_0 ASC +Sorting (None) +Sorting (Sorting for ORDER BY) +Sorting (Global): default.optimize_sorting.a_2 ASC +Sorting (Stream): default.optimize_sorting.a_2 ASC +Sorting (Stream): a ASC -- aliases DONT break sorting order -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) Sorting (Global): a ASC, b ASC @@ -79,6 +132,12 @@ Sorting (Sorting for ORDER BY) Sorting (Global): x ASC, y ASC Sorting (Stream): x ASC, y ASC Sorting (Stream): a ASC, b ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) +Sorting (Global): a ASC, b ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): x_2 ASC, y_3 ASC +Sorting (Stream): x_2 ASC, y_3 ASC +Sorting (Stream): a ASC, b ASC -- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1) -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 Sorting (None) @@ -89,6 +148,15 @@ Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 +Sorting (None) +Sorting (Sorting for ORDER BY) +Sorting (Global): plus(a_0, 1_UInt8) ASC +Sorting (Global): plus(default.optimize_sorting.a_3, 1_UInt8) ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): plus(default.optimize_sorting.a_3, 1_UInt8) ASC +Sorting (Chunk): default.optimize_sorting.a_3 ASC +Sorting (Chunk): a ASC -- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a Sorting (Global): a ASC @@ -96,3 +164,9 @@ Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (Stream): a ASC Sorting (Stream): a ASC +-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a +Sorting (Global): a ASC +Sorting (Sorting for ORDER BY) +Sorting (Global): default.optimize_sorting.a_0 ASC +Sorting (Stream): default.optimize_sorting.a_0 ASC +Sorting (Stream): a ASC diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh index e4cab987d35..7e937ac42b6 100755 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh @@ -17,9 +17,12 @@ function explain_sorting { echo "-- QUERY: "$1 $CLICKHOUSE_CLIENT -nq "$1" | eval $FIND_SORTING } + function explain_sortmode { echo "-- QUERY: "$1 - $CLICKHOUSE_CLIENT -nq "$1" | eval $FIND_SORTMODE + $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -nq "$1" | eval $FIND_SORTMODE + echo "-- QUERY (analyzer): "$1 + $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -nq "$1" | eval $FIND_SORTMODE } $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" From 36c31e1d795412b2d3107f58409d4c8806505a1a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 17 May 2023 14:07:34 +0300 Subject: [PATCH 0356/2223] Improve concurrent parts removal with zero copy replication (#49630) * improve concurrent parts removal * fix * fix --- src/Storages/MergeTree/MergeTreeData.cpp | 154 ++++++++++++++++++--- src/Storages/MergeTree/MergeTreeSettings.h | 2 + 2 files changed, 133 insertions(+), 23 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b8208052f19..6ddfc3b806e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -2429,9 +2430,13 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t } /// Parallel parts removal. - size_t num_threads = std::min(settings->max_part_removal_threads, parts_to_remove.size()); + size_t num_threads = settings->max_part_removal_threads; + if (!num_threads) + num_threads = getNumberOfPhysicalCPUCores() * 2; + num_threads = std::min(num_threads, parts_to_remove.size()); std::mutex part_names_mutex; - ThreadPool pool(CurrentMetrics::MergeTreePartsCleanerThreads, CurrentMetrics::MergeTreePartsCleanerThreadsActive, num_threads); + ThreadPool pool(CurrentMetrics::MergeTreePartsCleanerThreads, CurrentMetrics::MergeTreePartsCleanerThreadsActive, + num_threads, num_threads, /* unlimited queue size */ 0); /// This flag disallow straightforward concurrent parts removal. It's required only in case /// when we have parts on zero-copy disk + at least some of them were mutated. @@ -2490,29 +2495,62 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t /// We remove disjoint subsets of parts in parallel. /// The problem is that it's not trivial to divide Outdated parts into disjoint subsets, /// because Outdated parts legally can be intersecting (but intersecting parts must be separated by a DROP_RANGE). - /// So we ignore level and version and use block numbers only. - ActiveDataPartSet independent_ranges_set(format_version); - for (const auto & part : parts_to_remove) + /// So we ignore level and version and use block numbers only (they cannot intersect by block numbers unless we have a bug). + + struct RemovalRanges { - MergeTreePartInfo range_info = part->info; - range_info.level = static_cast(range_info.max_block - range_info.min_block); - range_info.mutation = 0; - independent_ranges_set.add(range_info, range_info.getPartNameV1()); - } + std::vector infos; + std::vector parts; + std::vector split_times; + }; - auto independent_ranges_infos = independent_ranges_set.getPartInfos(); - size_t sum_of_ranges = 0; - for (auto range : independent_ranges_infos) + auto split_into_independent_ranges = [this](const DataPartsVector & parts_to_remove_, size_t split_times) -> RemovalRanges { - range.level = MergeTreePartInfo::MAX_LEVEL; - range.mutation = MergeTreePartInfo::MAX_BLOCK_NUMBER; + if (parts_to_remove_.empty()) + return {}; - DataPartsVector parts_in_range; - for (const auto & part : parts_to_remove) - if (range.contains(part->info)) - parts_in_range.push_back(part); - sum_of_ranges += parts_in_range.size(); + ActiveDataPartSet independent_ranges_set(format_version); + for (const auto & part : parts_to_remove_) + { + MergeTreePartInfo range_info = part->info; + range_info.level = static_cast(range_info.max_block - range_info.min_block); + range_info.mutation = 0; + independent_ranges_set.add(range_info, range_info.getPartNameV1()); + } + RemovalRanges independent_ranges; + independent_ranges.infos = independent_ranges_set.getPartInfos(); + size_t num_ranges = independent_ranges.infos.size(); + independent_ranges.parts.resize(num_ranges); + independent_ranges.split_times.resize(num_ranges, split_times); + size_t avg_range_size = parts_to_remove_.size() / num_ranges; + + size_t sum_of_ranges = 0; + for (size_t i = 0; i < num_ranges; ++i) + { + MergeTreePartInfo & range = independent_ranges.infos[i]; + DataPartsVector & parts_in_range = independent_ranges.parts[i]; + range.level = MergeTreePartInfo::MAX_LEVEL; + range.mutation = MergeTreePartInfo::MAX_BLOCK_NUMBER; + + parts_in_range.reserve(avg_range_size * 2); + for (const auto & part : parts_to_remove_) + if (range.contains(part->info)) + parts_in_range.push_back(part); + sum_of_ranges += parts_in_range.size(); + } + + if (parts_to_remove_.size() != sum_of_ranges) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of removed parts is not equal to number of parts in independent ranges " + "({} != {}), it's a bug", parts_to_remove_.size(), sum_of_ranges); + + return independent_ranges; + }; + + auto schedule_parts_removal = [this, &pool, &part_names_mutex, part_names_succeed]( + const MergeTreePartInfo & range, DataPartsVector && parts_in_range) + { + /// Below, range should be captured by copy to avoid use-after-scope on exception from pool pool.scheduleOrThrowOnError( [this, range, &part_names_mutex, part_names_succeed, thread_group = CurrentThread::getGroup(), batch = std::move(parts_in_range)] { @@ -2535,13 +2573,83 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t } } }); + }; + + RemovalRanges independent_ranges = split_into_independent_ranges(parts_to_remove, /* split_times */ 0); + DataPartsVector excluded_parts; + size_t num_ranges = independent_ranges.infos.size(); + size_t sum_of_ranges = 0; + for (size_t i = 0; i < num_ranges; ++i) + { + MergeTreePartInfo & range = independent_ranges.infos[i]; + DataPartsVector & parts_in_range = independent_ranges.parts[i]; + UInt64 split_times = independent_ranges.split_times[i]; + + /// It may happen that we have a huge part covering thousands small parts. + /// In this case, we will get a huge range that will be process by only one thread causing really long tail latency. + /// Let's try to exclude such parts in order to get smaller tasks for thread pool and more uniform distribution. + if (settings->concurrent_part_removal_threshold < parts_in_range.size() && + split_times < settings->zero_copy_concurrent_part_removal_max_split_times) + { + auto smaller_parts_pred = [&range](const DataPartPtr & part) + { + return !(part->info.min_block == range.min_block && part->info.max_block == range.max_block); + }; + + size_t covered_parts_count = std::count_if(parts_in_range.begin(), parts_in_range.end(), smaller_parts_pred); + size_t top_level_count = parts_in_range.size() - covered_parts_count; + chassert(top_level_count); + Float32 parts_to_exclude_ratio = static_cast(top_level_count) / parts_in_range.size(); + if (settings->zero_copy_concurrent_part_removal_max_postpone_ratio < parts_to_exclude_ratio) + { + /// Most likely we have a long mutations chain here + LOG_DEBUG(log, "Block range {} contains {} parts including {} top-level parts, will not try to split it", + range.getPartNameForLogs(), parts_in_range.size(), top_level_count); + } + else + { + auto new_end_it = std::partition(parts_in_range.begin(), parts_in_range.end(), smaller_parts_pred); + std::move(new_end_it, parts_in_range.end(), std::back_inserter(excluded_parts)); + parts_in_range.erase(new_end_it, parts_in_range.end()); + + RemovalRanges subranges = split_into_independent_ranges(parts_in_range, split_times + 1); + + LOG_DEBUG(log, "Block range {} contained {} parts, it was split into {} independent subranges after excluding {} top-level parts", + range.getPartNameForLogs(), parts_in_range.size() + top_level_count, subranges.infos.size(), top_level_count); + + std::move(subranges.infos.begin(), subranges.infos.end(), std::back_inserter(independent_ranges.infos)); + std::move(subranges.parts.begin(), subranges.parts.end(), std::back_inserter(independent_ranges.parts)); + std::move(subranges.split_times.begin(), subranges.split_times.end(), std::back_inserter(independent_ranges.split_times)); + num_ranges += subranges.infos.size(); + continue; + } + } + + sum_of_ranges += parts_in_range.size(); + + schedule_parts_removal(range, std::move(parts_in_range)); + } + + /// Remove excluded parts as well. They were reordered, so sort them again + std::sort(excluded_parts.begin(), excluded_parts.end(), [](const auto & x, const auto & y) { return x->info < y->info; }); + LOG_TRACE(log, "Will remove {} big parts separately: {}", excluded_parts.size(), fmt::join(excluded_parts, ", ")); + + independent_ranges = split_into_independent_ranges(excluded_parts, /* split_times */ 0); + pool.wait(); + + for (size_t i = 0; i < independent_ranges.infos.size(); ++i) + { + MergeTreePartInfo & range = independent_ranges.infos[i]; + DataPartsVector & parts_in_range = independent_ranges.parts[i]; + schedule_parts_removal(range, std::move(parts_in_range)); } pool.wait(); - if (parts_to_remove.size() != sum_of_ranges) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of removed parts is not equal to number of parts in independent ranges " - "({} != {}), it's a bug", parts_to_remove.size(), sum_of_ranges); + if (parts_to_remove.size() != sum_of_ranges + excluded_parts.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Number of parts to remove was not equal to number of parts in independent ranges and excluded parts" + "({} != {} + {}), it's a bug", parts_to_remove.size(), sum_of_ranges, excluded_parts.size()); } size_t MergeTreeData::clearOldBrokenPartsFromDetachedDirectory() diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 9c7488cb6a6..c9e81ce9103 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -146,6 +146,8 @@ struct Settings; M(MaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \ M(MaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \ M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ + M(UInt64, zero_copy_concurrent_part_removal_max_split_times, 5, "Max recursion depth for splitting independent Outdated parts ranges into smaller subranges (highly not recommended to change)", 0) \ + M(Float, zero_copy_concurrent_part_removal_max_postpone_ratio, static_cast(0.05), "Max percentage of top level parts to postpone removal in order to get smaller independent ranges (highly not recommended to change)", 0) \ M(String, storage_policy, "default", "Name of storage disk policy", 0) \ M(String, disk, "", "Name of storage disk. Can be specified instead of storage policy.", 0) \ M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ From 846804fed085680d4d4ae1ac5f34329e39006486 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 17 May 2023 11:39:04 +0000 Subject: [PATCH 0357/2223] Add separate handshake_timeout for receiving Hello packet from replica --- docs/en/operations/settings/settings.md | 6 +++ src/Client/Connection.cpp | 6 ++- src/Client/Connection.h | 2 +- src/Client/ConnectionParameters.cpp | 3 +- src/Core/Settings.h | 1 + src/IO/ConnectionTimeouts.cpp | 58 +++++++++++++++---------- src/IO/ConnectionTimeouts.h | 14 ++++-- src/Server/TCPHandler.cpp | 7 +++ 8 files changed, 65 insertions(+), 32 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cddde2090f8..20779eba360 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1048,6 +1048,12 @@ Timeouts in seconds on the socket used for communicating with the client. Default value: 10, 300, 300. +## handshake_timeout_ms {#handshake-timeout-ms} + +Timeout in milliseconds for receiving Hello packet from replicas during handshake. + +Default value: 300000. + ## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index d39148d3016..09145bcdf1b 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -190,7 +190,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) connected = true; sendHello(); - receiveHello(); + receiveHello(timeouts.handshake_timeout); if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) sendAddendum(); @@ -305,8 +305,10 @@ void Connection::sendAddendum() } -void Connection::receiveHello() +void Connection::receiveHello(const Poco::Timespan & handshake_timeout) { + TimeoutSetter timeout_setter(*socket, socket->getSendTimeout(), handshake_timeout); + /// Receive hello packet. UInt64 packet_type = 0; diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 77dbe5e3398..cb3f2507cb9 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -256,7 +256,7 @@ private: void connect(const ConnectionTimeouts & timeouts); void sendHello(); void sendAddendum(); - void receiveHello(); + void receiveHello(const Poco::Timespan & handshake_timeout); #if USE_SSL void sendClusterNameAndSalt(); diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 2031036eb58..c47d217d432 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -67,7 +67,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati Poco::Timespan(config.getInt("connect_timeout", DBMS_DEFAULT_CONNECT_TIMEOUT_SEC), 0), Poco::Timespan(config.getInt("send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0), Poco::Timespan(config.getInt("receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0), - Poco::Timespan(config.getInt("tcp_keep_alive_timeout", 0), 0)); + Poco::Timespan(config.getInt("tcp_keep_alive_timeout", 0), 0), + Poco::Timespan(config.getInt("handshake_timeout_ms", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000), 0)); timeouts.sync_request_timeout = Poco::Timespan(config.getInt("sync_request_timeout", DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC), 0); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 610c7135a75..d5fb864ca6b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -55,6 +55,7 @@ class IColumn; M(UInt64, max_query_size, DBMS_DEFAULT_MAX_QUERY_SIZE, "The maximum number of bytes of a query string parsed by the SQL parser. Data in the VALUES clause of INSERT queries is processed by a separate stream parser (that consumes O(1) RAM) and not affected by this restriction.", 0) \ M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \ M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \ + M(Milliseconds, handshake_timeout_ms, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000, "Timeout for receiving HELLO packet from replicas.", 0) \ M(Milliseconds, connect_timeout_with_failover_ms, 1000, "Connection timeout for selecting first healthy replica.", 0) \ M(Milliseconds, connect_timeout_with_failover_secure_ms, 1000, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \ M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Timeout for receiving data from network, in seconds. If no bytes were received in this interval, exception is thrown. If you set this setting on client, the 'send_timeout' for the socket will be also set on the corresponding connection end on the server.", 0) \ diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index 401afb7baac..01fbaa4f817 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -17,22 +17,7 @@ ConnectionTimeouts::ConnectionTimeouts( , secure_connection_timeout(connection_timeout) , hedged_connection_timeout(receive_timeout_) , receive_data_timeout(receive_timeout_) -{ -} - -ConnectionTimeouts::ConnectionTimeouts( - Poco::Timespan connection_timeout_, - Poco::Timespan send_timeout_, - Poco::Timespan receive_timeout_, - Poco::Timespan tcp_keep_alive_timeout_) - : connection_timeout(connection_timeout_) - , send_timeout(send_timeout_) - , receive_timeout(receive_timeout_) - , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) - , http_keep_alive_timeout(0) - , secure_connection_timeout(connection_timeout) - , hedged_connection_timeout(receive_timeout_) - , receive_data_timeout(receive_timeout_) + , handshake_timeout(receive_timeout_) { } @@ -41,7 +26,26 @@ ConnectionTimeouts::ConnectionTimeouts( Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, Poco::Timespan tcp_keep_alive_timeout_, - Poco::Timespan http_keep_alive_timeout_) + Poco::Timespan handshake_timeout_) + : connection_timeout(connection_timeout_) + , send_timeout(send_timeout_) + , receive_timeout(receive_timeout_) + , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) + , http_keep_alive_timeout(0) + , secure_connection_timeout(connection_timeout) + , hedged_connection_timeout(receive_timeout_) + , receive_data_timeout(receive_timeout_) + , handshake_timeout(handshake_timeout_) +{ +} + +ConnectionTimeouts::ConnectionTimeouts( + Poco::Timespan connection_timeout_, + Poco::Timespan send_timeout_, + Poco::Timespan receive_timeout_, + Poco::Timespan tcp_keep_alive_timeout_, + Poco::Timespan http_keep_alive_timeout_, + Poco::Timespan handshake_timeout_) : connection_timeout(connection_timeout_) , send_timeout(send_timeout_) , receive_timeout(receive_timeout_) @@ -50,6 +54,7 @@ ConnectionTimeouts::ConnectionTimeouts( , secure_connection_timeout(connection_timeout) , hedged_connection_timeout(receive_timeout_) , receive_data_timeout(receive_timeout_) + , handshake_timeout(handshake_timeout_) { } @@ -60,16 +65,18 @@ ConnectionTimeouts::ConnectionTimeouts( Poco::Timespan tcp_keep_alive_timeout_, Poco::Timespan http_keep_alive_timeout_, Poco::Timespan secure_connection_timeout_, - Poco::Timespan receive_hello_timeout_, - Poco::Timespan receive_data_timeout_) + Poco::Timespan hedged_connection_timeout_, + Poco::Timespan receive_data_timeout_, + Poco::Timespan handshake_timeout_) : connection_timeout(connection_timeout_) , send_timeout(send_timeout_) , receive_timeout(receive_timeout_) , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) , http_keep_alive_timeout(http_keep_alive_timeout_) , secure_connection_timeout(secure_connection_timeout_) - , hedged_connection_timeout(receive_hello_timeout_) + , hedged_connection_timeout(hedged_connection_timeout_) , receive_data_timeout(receive_data_timeout_) + , handshake_timeout(handshake_timeout_) { } @@ -90,13 +97,14 @@ ConnectionTimeouts ConnectionTimeouts::getSaturated(Poco::Timespan limit) const saturate(http_keep_alive_timeout, limit), saturate(secure_connection_timeout, limit), saturate(hedged_connection_timeout, limit), - saturate(receive_data_timeout, limit)); + saturate(receive_data_timeout, limit), + saturate(handshake_timeout, limit)); } /// Timeouts for the case when we have just single attempt to connect. ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithoutFailover(const Settings & settings) { - return ConnectionTimeouts(settings.connect_timeout, settings.send_timeout, settings.receive_timeout, settings.tcp_keep_alive_timeout); + return ConnectionTimeouts(settings.connect_timeout, settings.send_timeout, settings.receive_timeout, settings.tcp_keep_alive_timeout, settings.handshake_timeout_ms); } /// Timeouts for the case when we will try many addresses in a loop. @@ -110,7 +118,8 @@ ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const Settings 0, settings.connect_timeout_with_failover_secure_ms, settings.hedged_connection_timeout_ms, - settings.receive_data_timeout_ms); + settings.receive_data_timeout_ms, + settings.handshake_timeout_ms); } ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout) @@ -120,7 +129,8 @@ ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings settings.http_send_timeout, settings.http_receive_timeout, settings.tcp_keep_alive_timeout, - http_keep_alive_timeout); + http_keep_alive_timeout, + settings.http_receive_timeout); } } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 368288ee022..684af42827f 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -23,6 +23,9 @@ struct ConnectionTimeouts Poco::Timespan hedged_connection_timeout; Poco::Timespan receive_data_timeout; + /// Timeout for receiving HELLO packet + Poco::Timespan handshake_timeout; + /// Timeout for synchronous request-result protocol call (like Ping or TablesStatus) Poco::Timespan sync_request_timeout = Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0); @@ -35,13 +38,15 @@ struct ConnectionTimeouts ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, - Poco::Timespan tcp_keep_alive_timeout_); + Poco::Timespan tcp_keep_alive_timeout_, + Poco::Timespan handshake_timeout_); ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, Poco::Timespan tcp_keep_alive_timeout_, - Poco::Timespan http_keep_alive_timeout_); + Poco::Timespan http_keep_alive_timeout_, + Poco::Timespan handshake_timeout_); ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, @@ -49,8 +54,9 @@ struct ConnectionTimeouts Poco::Timespan tcp_keep_alive_timeout_, Poco::Timespan http_keep_alive_timeout_, Poco::Timespan secure_connection_timeout_, - Poco::Timespan receive_hello_timeout_, - Poco::Timespan receive_data_timeout_); + Poco::Timespan hedged_connection_timeout_, + Poco::Timespan receive_data_timeout_, + Poco::Timespan handshake_timeout_); static Poco::Timespan saturate(Poco::Timespan timespan, Poco::Timespan limit); ConnectionTimeouts getSaturated(Poco::Timespan limit) const; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 0522b6d8a48..a076e248a9f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1296,6 +1296,13 @@ void TCPHandler::receiveUnexpectedHello() void TCPHandler::sendHello() { + if (unlikely(sleep_in_send_tables_status.totalMilliseconds())) + { + out->next(); + std::chrono::milliseconds ms(sleep_in_send_tables_status.totalMilliseconds()); + std::this_thread::sleep_for(ms); + } + writeVarUInt(Protocol::Server::Hello, *out); writeStringBinary(DBMS_NAME, *out); writeVarUInt(DBMS_VERSION_MAJOR, *out); From 2ff3c8badd5a5c18f14ca76438978a415fe73d74 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 17 May 2023 11:41:00 +0000 Subject: [PATCH 0358/2223] Remove testing code --- src/Server/TCPHandler.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a076e248a9f..0522b6d8a48 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1296,13 +1296,6 @@ void TCPHandler::receiveUnexpectedHello() void TCPHandler::sendHello() { - if (unlikely(sleep_in_send_tables_status.totalMilliseconds())) - { - out->next(); - std::chrono::milliseconds ms(sleep_in_send_tables_status.totalMilliseconds()); - std::this_thread::sleep_for(ms); - } - writeVarUInt(Protocol::Server::Hello, *out); writeStringBinary(DBMS_NAME, *out); writeVarUInt(DBMS_VERSION_MAJOR, *out); From 7df4820af7f84f850433e80e970e0763194310b5 Mon Sep 17 00:00:00 2001 From: AVMusorin Date: Tue, 16 May 2023 21:42:56 +0200 Subject: [PATCH 0359/2223] Fix metrics WriteBufferFromS3Bytes, WriteBufferFromS3Microseconds and WriteBufferFromS3RequestsErrors Ref: https://github.com/ClickHouse/ClickHouse/pull/45188 --- src/IO/S3/copyS3File.cpp | 24 +++- .../test_backup_restore_s3/test.py | 124 ++++++++++++++++-- 2 files changed, 134 insertions(+), 14 deletions(-) diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 3a2fd513392..20490ef9a19 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -15,6 +15,10 @@ namespace ProfileEvents { + extern const Event WriteBufferFromS3Bytes; + extern const Event WriteBufferFromS3Microseconds; + extern const Event WriteBufferFromS3RequestsErrors; + extern const Event S3CreateMultipartUpload; extern const Event S3CompleteMultipartUpload; extern const Event S3PutObject; @@ -135,7 +139,10 @@ namespace LOG_TRACE(log, "Multipart upload has created. Bucket: {}, Key: {}, Upload id: {}", dest_bucket, dest_key, multipart_upload_id); } else + { + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); + } } void completeMultipartUpload() @@ -184,7 +191,7 @@ namespace LOG_INFO(log, "Multipart upload failed with NO_SUCH_KEY error for Bucket: {}, Key: {}, Upload_id: {}, Parts: {}, will retry", dest_bucket, dest_key, multipart_upload_id, part_tags.size()); continue; /// will retry } - + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); throw S3Exception( outcome.GetError().GetErrorType(), "Message: {}, Key: {}, Bucket: {}, Tags: {}", @@ -228,7 +235,12 @@ namespace size_t next_position = std::min(position + normal_part_size, end_position); size_t part_size = next_position - position; /// `part_size` is either `normal_part_size` or smaller if it's the final part. + Stopwatch watch; uploadPart(part_number, position, part_size); + watch.stop(); + + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Bytes, part_size); + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); position = next_position; } @@ -485,16 +497,21 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3PutObject); + Stopwatch watch; auto outcome = client_ptr->PutObject(request); + watch.stop(); if (outcome.IsSuccess()) { + Int64 object_size = request.GetContentLength(); + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Bytes, object_size); + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); LOG_TRACE( log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}", dest_bucket, dest_key, - request.GetContentLength()); + object_size); break; } @@ -523,7 +540,7 @@ namespace request.GetContentLength()); continue; /// will retry } - + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); throw S3Exception( outcome.GetError().GetErrorType(), "Message: {}, Key: {}, Bucket: {}, Object size: {}", @@ -567,6 +584,7 @@ namespace if (!outcome.IsSuccess()) { abortMultipartUpload(); + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 2f60575b634..0285500d044 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -1,5 +1,8 @@ +from typing import Dict, Iterable import pytest from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + cluster = ClickHouseCluster(__file__) node = cluster.add_instance( @@ -34,23 +37,116 @@ def new_backup_name(): return f"backup{backup_id_counter}" -def check_backup_and_restore(storage_policy, backup_destination, size=1000): +def get_events(events_names: Iterable[str]) -> Dict[str, int]: + _events = TSV( + node.query( + f"SELECT event, value FROM system.events WHERE event in {events_names} SETTINGS system_events_show_zero_values = 1;" + ) + ) + return { + event: int(value) + for event, value in [line.split("\t") for line in _events.lines] + } + + +def check_backup_and_restore( + storage_policy, backup_destination, size=1000, backup_name=None, check_events=False +): + s3_backup_events = ( + "WriteBufferFromS3Microseconds", + "WriteBufferFromS3Bytes", + "WriteBufferFromS3RequestsErrors", + ) + s3_restore_events = ( + "ReadBufferFromS3Microseconds", + "ReadBufferFromS3Bytes", + "ReadBufferFromS3RequestsErrors", + ) + node.query( f""" DROP TABLE IF EXISTS data SYNC; CREATE TABLE data (key Int, value String, array Array(String)) Engine=MergeTree() ORDER BY tuple() SETTINGS storage_policy='{storage_policy}'; INSERT INTO data SELECT * FROM generateRandom('key Int, value String, array Array(String)') LIMIT {size}; - BACKUP TABLE data TO {backup_destination}; - RESTORE TABLE data AS data_restored FROM {backup_destination}; - SELECT throwIf( - (SELECT count(), sum(sipHash64(*)) FROM data) != - (SELECT count(), sum(sipHash64(*)) FROM data_restored), - 'Data does not matched after BACKUP/RESTORE' - ); - DROP TABLE data SYNC; - DROP TABLE data_restored SYNC; + OPTIMIZE TABLE data FINAL; """ ) + try: + events_before_backups = get_events(s3_backup_events) + node.query(f"BACKUP TABLE data TO {backup_destination}") + events_after_backups = get_events(s3_backup_events) + events_before_restore = get_events(s3_restore_events) + node.query( + f""" + RESTORE TABLE data AS data_restored FROM {backup_destination}; + """ + ) + events_after_restore = get_events(s3_restore_events) + node.query( + """ + SELECT throwIf( + (SELECT count(), sum(sipHash64(*)) FROM data) != + (SELECT count(), sum(sipHash64(*)) FROM data_restored), + 'Data does not matched after BACKUP/RESTORE' + ); + """ + ) + if check_events and backup_name: + objects = node.cluster.minio_client.list_objects( + "root", f"data/backups/multipart/{backup_name}/" + ) + backup_meta_size = 0 + for obj in objects: + if ".backup" in obj.object_name: + backup_meta_size = obj.size + break + backup_total_size = int( + node.query( + f"SELECT sum(total_size) FROM system.backups WHERE status = 'BACKUP_CREATED' AND name like '%{backup_name}%'" + ).strip() + ) + restore_total_size = int( + node.query( + f"SELECT sum(total_size) FROM system.backups WHERE status = 'RESTORED' AND name like '%{backup_name}%'" + ).strip() + ) + # backup + # NOTE: ~35 bytes is used by .lock file, so set up 100 bytes to avoid flaky test + assert ( + abs( + backup_total_size + - ( + events_after_backups["WriteBufferFromS3Bytes"] + - events_before_backups["WriteBufferFromS3Bytes"] + - backup_meta_size + ) + ) + < 100 + ) + assert ( + events_after_backups["WriteBufferFromS3Microseconds"] + > events_before_backups["WriteBufferFromS3Microseconds"] + ) + assert events_after_backups["WriteBufferFromS3RequestsErrors"] == 0 + # restore + assert ( + events_after_restore["ReadBufferFromS3Bytes"] + - events_before_restore["ReadBufferFromS3Bytes"] + - backup_meta_size + == restore_total_size + ) + assert ( + events_after_restore["ReadBufferFromS3Microseconds"] + > events_before_restore["ReadBufferFromS3Microseconds"] + ) + assert events_after_restore["ReadBufferFromS3RequestsErrors"] == 0 + finally: + node.query( + """ + DROP TABLE data SYNC; + DROP TABLE IF EXISTS data_restored SYNC; + """ + ) def check_system_tables(): @@ -128,7 +224,13 @@ def test_backup_to_s3_multipart(): storage_policy = "default" backup_name = new_backup_name() backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" - check_backup_and_restore(storage_policy, backup_destination, size=1000000) + check_backup_and_restore( + storage_policy, + backup_destination, + size=1000000, + backup_name=backup_name, + check_events=True, + ) assert node.contains_in_log( f"copyDataToS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}" ) From b5cc77512a5a02ee2f31591f2af66a0cb1bb15cd Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 17 May 2023 15:02:15 +0200 Subject: [PATCH 0360/2223] Fix style --- tests/integration/test_storage_url/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index d7f9125e29c..85491656db5 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -4,7 +4,9 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/conf.xml", "configs/named_collections.xml"], with_nginx=True + "node1", + main_configs=["configs/conf.xml", "configs/named_collections.xml"], + with_nginx=True ) From 7443dc925cb52eb19a6b42116e3acc3d8336a942 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 17 May 2023 13:12:00 +0000 Subject: [PATCH 0361/2223] Fix possible Logical error on bad Nullable parsing for text formats --- .../Serializations/SerializationNullable.cpp | 8 ++++++++ ...51_text_formats_bad_nullable_parsing.reference | 3 +++ .../02751_text_formats_bad_nullable_parsing.sh | 15 +++++++++++++++ 3 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/02751_text_formats_bad_nullable_parsing.reference create mode 100755 tests/queries/0_stateless/02751_text_formats_bad_nullable_parsing.sh diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 20188f7cec5..6e4402740d9 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -355,6 +355,9 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col /// It can happen only if there is a string instead of a number /// or if someone uses tab or LF in TSV null_representation. /// In the first case we cannot continue reading anyway. The second case seems to be unlikely. + /// We also should delete incorrectly deserialized value from nested column. + nested_column.popBack(1); + if (null_representation.find('\t') != std::string::npos || null_representation.find('\n') != std::string::npos) throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "TSV custom null representation " "containing '\\t' or '\\n' may not work correctly for large input."); @@ -447,6 +450,8 @@ ReturnType SerializationNullable::deserializeTextQuotedImpl(IColumn & column, Re /// We have some unread data in PeekableReadBuffer own memory. /// It can happen only if there is an unquoted string instead of a number. + /// We also should delete incorrectly deserialized value from nested column. + nested_column.popBack(1); throw DB::ParsingException( ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing Nullable: got an unquoted string {} instead of a number", @@ -579,6 +584,9 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB /// It can happen only if there is an unquoted string instead of a number /// or if someone uses csv delimiter, LF or CR in CSV null representation. /// In the first case we cannot continue reading anyway. The second case seems to be unlikely. + /// We also should delete incorrectly deserialized value from nested column. + nested_column.popBack(1); + if (null_representation.find(settings.csv.delimiter) != std::string::npos || null_representation.find('\r') != std::string::npos || null_representation.find('\n') != std::string::npos) throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "CSV custom null representation containing " diff --git a/tests/queries/0_stateless/02751_text_formats_bad_nullable_parsing.reference b/tests/queries/0_stateless/02751_text_formats_bad_nullable_parsing.reference new file mode 100644 index 00000000000..65e15e19c8b --- /dev/null +++ b/tests/queries/0_stateless/02751_text_formats_bad_nullable_parsing.reference @@ -0,0 +1,3 @@ +1 \N +1 \N +1 \N diff --git a/tests/queries/0_stateless/02751_text_formats_bad_nullable_parsing.sh b/tests/queries/0_stateless/02751_text_formats_bad_nullable_parsing.sh new file mode 100755 index 00000000000..e51079071ec --- /dev/null +++ b/tests/queries/0_stateless/02751_text_formats_bad_nullable_parsing.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +# NOTE: this sh wrapper is required because of shell_config + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT -q "create table test (x UInt32, y Nullable(UInt32)) engine=MergeTree order by x" +$CLICKHOUSE_CLIENT -q "select '1\t\\\N\n2\t\\\' format RawBLOB" | $CLICKHOUSE_CLIENT -q "insert into test settings input_format_allow_errors_num=1 format TSV" +$CLICKHOUSE_CLIENT -q "select '1,\\\N\n2,\\\' format RawBLOB" | $CLICKHOUSE_CLIENT -q "insert into test settings input_format_allow_errors_num=1 format CSV" +$CLICKHOUSE_CLIENT -q "select '1\tNULL\n2\tN' format RawBLOB" | $CLICKHOUSE_CLIENT -q "insert into test settings input_format_allow_errors_num=2, format_custom_escaping_rule='Quoted' format CustomSeparated" +$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "drop table test"; From 194ce2d881aa6c3598f24e93cce29671ec9f67c3 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 17 May 2023 13:13:57 +0000 Subject: [PATCH 0362/2223] Better --- src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 2 + src/Storages/StoragePostgreSQL.cpp | 40 ++++++++++++++----- 2 files changed, 31 insertions(+), 11 deletions(-) diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index 08b67a470f1..d2e8071c5de 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -202,6 +202,8 @@ void preparePostgreSQLArrayInfo( parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isFloat64()) parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isUUID()) + parser = [](std::string & field) -> Field { return parse(field); }; else if (which.isString() || which.isFixedString()) parser = [](std::string & field) -> Field { return field; }; else if (which.isDate()) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 09198e5bdad..007f83165fd 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -197,7 +197,7 @@ public: /// Cannot just use serializeAsText for array data type even though it converts perfectly /// any dimension number array into text format, because it encloses in '[]' and for postgres it must be '{}'. /// Check if array[...] syntax from PostgreSQL will be applicable. - void parseArray(const Field & array_field, const DataTypePtr & data_type, WriteBuffer & ostr) + static void parseArray(const Field & array_field, const DataTypePtr & data_type, WriteBuffer & ostr) { const auto * array_type = typeid_cast(data_type.get()); const auto & nested = array_type->getNestedType(); @@ -205,7 +205,7 @@ public: if (!isArray(nested)) { - writeText(clickhouseToPostgresArray(array, data_type), ostr); + parseArrayContent(array, data_type, ostr); return; } @@ -219,7 +219,7 @@ public: if (!isArray(nested_array_type->getNestedType())) { - writeText(clickhouseToPostgresArray(iter->get(), nested), ostr); + parseArrayContent(iter->get(), nested, ostr); } else { @@ -232,17 +232,35 @@ public: /// Conversion is done via column casting because with writeText(Array..) got incorrect conversion /// of Date and DateTime data types and it added extra quotes for values inside array. - static std::string clickhouseToPostgresArray(const Array & array_field, const DataTypePtr & data_type) + static void parseArrayContent(const Array & array_field, const DataTypePtr & data_type, WriteBuffer & ostr) { - auto nested = typeid_cast(data_type.get())->getNestedType(); - auto array_column = ColumnArray::create(createNested(nested)); + auto nested_type = typeid_cast(data_type.get())->getNestedType(); + auto array_column = ColumnArray::create(createNested(nested_type)); array_column->insert(array_field); - WriteBufferFromOwnString ostr; - data_type->getDefaultSerialization()->serializeText(*array_column, 0, ostr, FormatSettings{}); - /// ostr is guaranteed to be at least '[]', i.e. size is at least 2 and 2 only if ostr.str() == '[]' - assert(ostr.str().size() >= 2); - return '{' + std::string(ostr.str().begin() + 1, ostr.str().end() - 1) + '}'; + const IColumn & nested_column = array_column->getData(); + const auto serialization = nested_type->getDefaultSerialization(); + + FormatSettings settings; + settings.pretty.charset = FormatSettings::Pretty::Charset::ASCII; + + if (nested_type->isNullable()) + nested_type = static_cast(nested_type.get())->getNestedType(); + + const bool quoted = !isUUID(nested_type); + + writeChar('{', ostr); + for (size_t i = 0, size = array_field.size(); i < size; ++i) + { + if (i != 0) + writeChar(',', ostr); + + if (quoted) + serialization->serializeTextQuoted(nested_column, i, ostr, settings); + else + serialization->serializeText(nested_column, i, ostr, settings); + } + writeChar('}', ostr); } static MutableColumnPtr createNested(DataTypePtr nested) From ad637c682418caa8fdffd28795a2edb415f2bfce Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 17 May 2023 13:14:10 +0000 Subject: [PATCH 0363/2223] Add test --- .../test_storage_postgresql/test.py | 41 +++++++++++-------- 1 file changed, 25 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 3b7aae1ccdc..6ceddfe831b 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -113,7 +113,9 @@ def test_postgres_conversions(started_cluster): g Text[][][][][] NOT NULL, -- String h Integer[][][], -- Nullable(Int32) i Char(2)[][][][], -- Nullable(String) - k Char(2)[] -- Nullable(String) + j Char(2)[], -- Nullable(String) + k UUID[], -- Nullable(UUID) + l UUID[][] -- Nullable(UUID) )""" ) @@ -123,15 +125,18 @@ def test_postgres_conversions(started_cluster): ) expected = ( "a\tArray(Date)\t\t\t\t\t\n" - + "b\tArray(DateTime64(6))\t\t\t\t\t\n" - + "c\tArray(Array(Float32))\t\t\t\t\t\n" - + "d\tArray(Array(Float64))\t\t\t\t\t\n" - + "e\tArray(Array(Array(Decimal(5, 5))))\t\t\t\t\t\n" - + "f\tArray(Array(Array(Int32)))\t\t\t\t\t\n" - + "g\tArray(Array(Array(Array(Array(String)))))\t\t\t\t\t\n" - + "h\tArray(Array(Array(Nullable(Int32))))\t\t\t\t\t\n" - + "i\tArray(Array(Array(Array(Nullable(String)))))\t\t\t\t\t\n" - + "k\tArray(Nullable(String))" + "b\tArray(DateTime64(6))\t\t\t\t\t\n" + "c\tArray(Array(Float32))\t\t\t\t\t\n" + "d\tArray(Array(Float64))\t\t\t\t\t\n" + "e\tArray(Array(Array(Decimal(5, 5))))\t\t\t\t\t\n" + "f\tArray(Array(Array(Int32)))\t\t\t\t\t\n" + "g\tArray(Array(Array(Array(Array(String)))))\t\t\t\t\t\n" + "h\tArray(Array(Array(Nullable(Int32))))\t\t\t\t\t\n" + "i\tArray(Array(Array(Array(Nullable(String)))))\t\t\t\t\t\n" + "j\tArray(Nullable(String))\t\t\t\t\t\n" + "k\tArray(Nullable(UUID))\t\t\t\t\t\n" + "l\tArray(Array(Nullable(UUID)))" + "" ) assert result.rstrip() == expected @@ -147,7 +152,9 @@ def test_postgres_conversions(started_cluster): "[[[[['winx', 'winx', 'winx']]]]], " "[[[1, NULL], [NULL, 1]], [[NULL, NULL], [NULL, NULL]], [[4, 4], [5, 5]]], " "[[[[NULL]]]], " - "[]" + "[], " + "['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a', '42209d53-d641-4d73-a8b6-c038db1e75d6', NULL], " + "[[NULL, '42209d53-d641-4d73-a8b6-c038db1e75d6'], ['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a', NULL], [NULL, NULL]]" ")" ) @@ -157,15 +164,17 @@ def test_postgres_conversions(started_cluster): ) expected = ( "['2000-05-12','2000-05-12']\t" - + "['2000-05-12 12:12:12.012345','2000-05-12 12:12:12.012345']\t" - + "[[1.12345],[1.12345],[1.12345]]\t" - + "[[1.1234567891],[1.1234567891],[1.1234567891]]\t" - + "[[[0.11111,0.11111]],[[0.22222,0.22222]],[[0.33333,0.33333]]]\t" + "['2000-05-12 12:12:12.012345','2000-05-12 12:12:12.012345']\t" + "[[1.12345],[1.12345],[1.12345]]\t" + "[[1.1234567891],[1.1234567891],[1.1234567891]]\t" + "[[[0.11111,0.11111]],[[0.22222,0.22222]],[[0.33333,0.33333]]]\t" "[[[1,1],[1,1]],[[3,3],[3,3]],[[4,4],[5,5]]]\t" "[[[[['winx','winx','winx']]]]]\t" "[[[1,NULL],[NULL,1]],[[NULL,NULL],[NULL,NULL]],[[4,4],[5,5]]]\t" "[[[[NULL]]]]\t" - "[]\n" + "[]\t" + "['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a','42209d53-d641-4d73-a8b6-c038db1e75d6',NULL]\t" + "[[NULL,'42209d53-d641-4d73-a8b6-c038db1e75d6'],['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a',NULL],[NULL,NULL]]\n" ) assert result == expected From 0f26a82000d184fd073a019be7e3db721cb279bd Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 17 May 2023 15:32:40 +0200 Subject: [PATCH 0364/2223] Fix style --- tests/integration/test_storage_url/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index 85491656db5..f360ec105ec 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -6,7 +6,7 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", main_configs=["configs/conf.xml", "configs/named_collections.xml"], - with_nginx=True + with_nginx=True, ) From f2dbcb514673e439b0b11efa16575dd4fb8be164 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 May 2023 16:25:31 +0200 Subject: [PATCH 0365/2223] Better fix --- src/Interpreters/Cache/Metadata.cpp | 38 +++++++++++++++++++---------- 1 file changed, 25 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index c87eaabbbf2..e0b82763a08 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -257,30 +257,42 @@ void CacheMetadata::doCleanup() } locked_metadata->markAsRemoved(); + erase(it); + LOG_DEBUG(log, "Key {} is removed from metadata", cleanup_key); + + const fs::path key_directory = getPathInLocalCache(cleanup_key); + const fs::path key_prefix_directory = key_directory.parent_path(); try { - const fs::path key_directory = getPathInLocalCache(cleanup_key); if (fs::exists(key_directory)) fs::remove_all(key_directory); - - const fs::path key_prefix_directory = key_directory.parent_path(); - if (fs::exists(key_prefix_directory) && fs::is_empty(key_prefix_directory)) - fs::remove_all(key_prefix_directory); } catch (...) { - LOG_ERROR(log, "Error while removing key {}: {}", cleanup_key, getCurrentExceptionMessage(false)); + LOG_ERROR(log, "Error while removing key {}: {}", cleanup_key, getCurrentExceptionMessage(true)); chassert(false); + continue; } - /// Remove key from metadata AFTER deleting key directory, because otherwise key lock is - /// released before we delete directory from fs and there might be a race: - /// a key, which we just removed, can be added back to cache before we start removing key directory, - /// which makes key directory either non-empty (and we get exception in try catch above) - /// or we removed directory while another thread thinks it exists. - erase(it); - LOG_DEBUG(log, "Key {} is removed from metadata", cleanup_key); + try + { + if (fs::exists(key_prefix_directory) && fs::is_empty(key_prefix_directory)) + fs::remove_all(key_prefix_directory); + } + catch (const fs::filesystem_error & e) + { + /// Key prefix directory can become non-empty just now, it is expected. + if (e.code() == std::errc::directory_not_empty) + return; + LOG_ERROR(log, "Error while removing key {}: {}", cleanup_key, getCurrentExceptionMessage(true)); + chassert(false); + } + catch (...) + { + LOG_ERROR(log, "Error while removing key {}: {}", cleanup_key, getCurrentExceptionMessage(true)); + chassert(false); + } } } From 1c04085e8fcbb916292635a6297ae45fde7da49d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 17 May 2023 18:15:51 +0200 Subject: [PATCH 0366/2223] Update MergeTreeWriteAheadLog.h --- src/Storages/MergeTree/MergeTreeWriteAheadLog.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index eba7698b9f9..f5398a24e7d 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -60,7 +60,6 @@ public: ~MergeTreeWriteAheadLog(); - void addPart(DataPartInMemoryPtr & part); void dropPart(const String & part_name); std::vector restore( const StorageMetadataPtr & metadata_snapshot, @@ -77,7 +76,6 @@ public: private: void init(); void rotate(const std::unique_lock & lock); - void sync(std::unique_lock & lock); const MergeTreeData & storage; DiskPtr disk; From c7ab59302f3ae1850619a79f138dee865a92bbb6 Mon Sep 17 00:00:00 2001 From: Timur Solodovnikov Date: Wed, 17 May 2023 11:03:42 -0700 Subject: [PATCH 0367/2223] Set allow_experimental_query_cache setting as obsolete (#49934) * set allow_experimental_query_cache as obsolete * add tsolodov to trusted contributors * CI linter --------- Co-authored-by: Nikita Mikhaylov --- src/Core/Settings.h | 1 + tests/ci/workflow_approve_rerun_lambda/app.py | 1 + ...726_set_allow_experimental_query_cache_as_obsolete.reference | 0 .../02726_set_allow_experimental_query_cache_as_obsolete.sql | 2 ++ 4 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference create mode 100644 tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 610c7135a75..c65958b86b7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -779,6 +779,7 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ MAKE_OBSOLETE(M, Bool, allow_experimental_projection_optimization, true) \ + MAKE_OBSOLETE(M, Bool, allow_experimental_query_cache, true) \ /* moved to config.xml: see also src/Core/ServerSettings.h */ \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_buffer_flush_schedule_pool_size, 16) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_pool_size, 16) \ diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 7d234bdcfc4..2cdbdecca6e 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -125,6 +125,7 @@ TRUSTED_CONTRIBUTORS = { "thevar1able", # ClickHouse Employee "aalexfvk", "MikhailBurdukov", + "tsolodov", # ClickHouse Employee ] } diff --git a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference b/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql b/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql new file mode 100644 index 00000000000..244ba4e959a --- /dev/null +++ b/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql @@ -0,0 +1,2 @@ +SET allow_experimental_query_cache = 0; +SET allow_experimental_query_cache = 1; From a1fc96953f41818a1e661af49f14d4cad8d7226f Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 17 May 2023 14:48:16 -0400 Subject: [PATCH 0368/2223] reorder --- docs/en/operations/named-collections.md | 37 +++++++++++++------------ 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index b65438b97f3..1866315529e 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -13,7 +13,23 @@ Named collections can be configured with DDL or in configuration files and are a when ClickHouse starts. They simplify the creation of objects and the hiding of credentials from users without administrative access. -## Storing details for connecting to external sources in the system database +The keys in a named collection must match the parameter names of the corresponding +function, table engine, database, etc. In the examples below the parameter list is +linked to for each type. + +Parameters set in a named collection can be overridden in SQL, this is shown in the examples +below. + +## Storing named collections in the system database + +### DDL example + +```sql +CREATE NAMED COLLECTION name AS +key_1 = 'value', +key_2 = 'value2', +url = 'https://connection.url/' +``` ### Permissions to create named collections with DDL @@ -37,22 +53,7 @@ To manage named collections with DDL a user must have the `named_control_collect In the above example the `passowrd_sha256_hex` value is the hexadecimal representation of the SHA256 hash of the password. This configuration for the user `default` has the attribute `replace=true` as in the default configuration has a plain text `password` set, and it is not possible to have both plaintext and sha256 hex passwords set for a user. ::: -## Storing details for connecting to external sources in configuration files - - -Parameters can be set in XML `CSV` and overridden in SQL `, format = 'TSV'`. -The parameters in SQL can be overridden using format `key` = `value`: `compression_method = 'gzip'`. - -Named collections are stored in the system table `system.named_collections` or in `config.xml` file of the ClickHouse server in the `` section and are applied when ClickHouse starts. - -### DDL example - -```sql -CREATE NAMED COLLECTION name AS -key_1 = 'value', -key_2 = 'value2', -url = 'https://connection.url/' -``` +## Storing named collections in configuration files ### XML example @@ -98,7 +99,7 @@ url = 'https://s3.us-east-1.amazonaws.com/yourbucket/mydata/'
``` -### s3() function and S3 Table Named collection examples +### s3() function and S3 Table named collection examples Both of the following examples use the same named collection `s3_mydata`: From e5aa3fcc8fdf8bbc45756728f11a9a55c2f17fb6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 17 May 2023 19:09:13 +0000 Subject: [PATCH 0369/2223] Add queries with enabled analyzer --- .../0_stateless/01655_plan_optimizations.sh | 44 ++++++++++++++++--- 1 file changed, 39 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index ec856c9bf27..7c299f9cc26 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -26,11 +26,17 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> filter should be pushed down after aggregating, column after aggregation is const" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " explain actions = 1 select s, y, y != 0 from (select sum(x) as s, y from ( select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(y, 0)" +echo "> (analyzer) filter should be pushed down after aggregating, column after aggregation is const" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " + explain actions = 1 select s, y, y != 0 from (select sum(x) as s, y from ( + select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 + settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(y_1, 0_UInt8)" $CLICKHOUSE_CLIENT -q " select s, y, y != 0 from (select sum(x) as s, y from ( select number as x, number + 1 as y from numbers(10)) group by y @@ -38,12 +44,19 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> one condition of filter should be pushed down after aggregating, other condition is aliased" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4))" +echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " + explain actions = 1 select s, y from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s != 4 + settings enable_optimize_predicate_expression=0" | + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|ALIAS notEquals(s_0, 4_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -51,12 +64,19 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> one condition of filter should be pushed down after aggregating, other condition is casted" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2" +echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " + explain actions = 1 select s, y from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s - 4 + settings enable_optimize_predicate_expression=0" | + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|FUNCTION and(minus(s_0, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 4_UInt8)) UInt8 : 2" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -64,12 +84,19 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> one condition of filter should be pushed down after aggregating, other two conditions are ANDed" -$CLICKHOUSE_CLIENT --convert_query_to_cnf=0 -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" +echo "> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " + explain actions = 1 select s, y from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s - 8 and s - 4 + settings enable_optimize_predicate_expression=0" | + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|FUNCTION and(minus(s_0, 8_UInt8) :: 0, minus(s_0, 4_UInt8) :: 2) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 8_UInt8), minus(s_0, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -77,12 +104,19 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased" -$CLICKHOUSE_CLIENT --convert_query_to_cnf=0 -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" +echo "> (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " + explain actions = 1 select s, y from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s != 8 and y - 4 + settings enable_optimize_predicate_expression=0" | + grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y_1, 0_UInt8), minus(y_1, 4_UInt8))\|ALIAS notEquals(s_0, 8_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 8_UInt8), minus(y_1, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y From 549af4d35112f598f3bbaaf196d0d994ff9076dc Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 17 May 2023 21:23:32 +0200 Subject: [PATCH 0370/2223] address comments --- docs/en/sql-reference/dictionaries/index.md | 50 +++++++++---------- ...4_regexp_dictionary_table_source.reference | 40 +++++++-------- .../02504_regexp_dictionary_table_source.sql | 14 +++--- 3 files changed, 50 insertions(+), 54 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 4abc41cdf42..eb45247e74a 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2197,13 +2197,13 @@ Result: └─────────────────────────────────┴───────┘ ``` -## RegExpTree Dictionary {#regexp-tree-dictionary} +## Regular Expression Tree Dictionary {#regexp-tree-dictionary} -RegExpTree Dictionary is designed to store multiple regular expressions in a dictionary, and query if a string could match one or multiple regular expressions. In some scenarios, for example, User Agent Parser, this data structure is very useful. We can use it in both local and cloud environments. +Regular expression tree dictionaries are a special type of dictionary which represent the mapping from key to attributes using a tree of regular expressions. There are some use cases, e.g. parsing of (user agent)[https://en.wikipedia.org/wiki/User_agent] strings, which can be expressed elegantly with regexp tree dictionaries. -### Use RegExpTree Dictionary in local environment +### Use Regular Expression Tree Dictionary in ClickHouse Open-Source Environment -In local environment, we create RegexpTree dictionary by a yaml file: +Regular expression tree dictionaries are defined in ClickHouse open-source using the YAMLRegExpTree source which is provided the path to a YAML file containing the regular expression tree. ```sql create dictionary regexp_dict @@ -2218,7 +2218,7 @@ LAYOUT(regexp_tree) ... ``` -The dictionary source `YAMLRegExpTree` represents the structure of a Regexp Tree. For example: +The dictionary source `YAMLRegExpTree` represents the structure of a regexp tree. For example: ```yaml - regexp: 'Linux/(\d+[\.\d]*).+tlinux' @@ -2226,7 +2226,7 @@ The dictionary source `YAMLRegExpTree` represents the structure of a Regexp Tree version: '\1' - regexp: '\d+/tclwebkit(?:\d+[\.\d]*)' - name: 'Andriod' + name: 'Android' versions: - regexp: '33/tclwebkit' version: '13' @@ -2238,15 +2238,15 @@ The dictionary source `YAMLRegExpTree` represents the structure of a Regexp Tree version: '10' ``` -This config consists of a list of RegExpTree nodes. Each node has following structure: +This config consists of a list of Regular Expression Tree nodes. Each node has following structure: - **regexp** means the regular expression of this node. -- **user defined attributions** is a list of dictionary attributions defined in the dictionary structure. In this case, we have two attributions: `name` and `version`. The first nodes have both attributions. The second node only has `name` attribution, because the `version` is defined in the children nodes. - - The value of an attribution could contain a **back reference** which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. During the query execution, the back reference in the value will be replaced by the matched capture group. -- **children nodes** is the secondary layer of the RegExpTree nodes, which also contains a list of RegExpTree nodes. If a string matches a regexp node in the first layer, the dictionary will check if the string matches the children nodes of it. If it matches, we assign the attributions of the matching nodes. If two or more nodes define the same attribution, children nodes have more priority. - - the name of **children nodes** in yaml files can be arbitrary. +- **user defined attributes** is a list of dictionary attributes defined in the dictionary structure. In this case, we have two attributes: `name` and `version`. The first node has both attributes. The second node only has `name` attribute, because the `version` is defined in the children nodes. + - The value of an attribute could contain a **back reference** which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. During the query execution, the back reference in the value will be replaced by the matched capture group. +- **children nodes** is the children of a regexp tree node, which has their own attributes and children nodes. String matching preceeds in a depth-first fasion. If a string matches any regexp node in the top layer, the dictionary checks if the string matches the children nodes of it. If it matches, we assign the attributes of the matching nodes. If two or more nodes define the same attribute, children nodes have more priority. + - the name of **children nodes** in YAML files can be arbitrary. -Due to the specialty of Regexp Tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. +Due to the specialty of regexp tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. Example: @@ -2258,17 +2258,17 @@ Result: ```text ┌─dictGet('regexp_dict', ('name', 'version'), '31/tclwebkit1024')─┐ -│ ('Andriod','12') │ +│ ('Android','12') │ └─────────────────────────────────────────────────────────────────┘ ``` -Explain: +In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the top layer's second node, so the dictionary continues to look into the children nodes and find it matches `3[12]/tclwebkit`. As a result, the value of `name` is `Android` defined in the first layer and the value of `version` is `12` defined the child node. -In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the first layer, so the dictionary will continue to look into the children nodes in the second layer and find it matches `3[12]/tclwebkit`. As a result, the value of `name` is `Andriod` defined in the first layer and the value of `version` is `12` defined in the second layer. +With a powerful YAML configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) -### Use RegExpTree Dictionary on cloud +### Use Regular Expression Tree Dictionary in ClickHouse Cloud -We have shown how RegExpTree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local yaml file, we can use this file to create RegExpTree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../statements/select/into-outfile.md) clause. +We have shown how Regular Expression Tree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local YAML file, we can use this file to create Regular Expression Tree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../statements/select/into-outfile.md) clause. ```sql select * from dictionary(regexp_dict) into outfile('regexp_dict.csv') @@ -2278,7 +2278,7 @@ The content of csv file is: ```text 1,0,"Linux/(\d+[\.\d]*).+tlinux","['version','name']","['\\1','TencentOS']" -2,0,"(\d+)/tclwebkit(\d+[\.\d]*)","['comment','version','name']","['test $1 and $2','$1','Andriod']" +2,0,"(\d+)/tclwebkit(\d+[\.\d]*)","['comment','version','name']","['test $1 and $2','$1','Android']" 3,2,"33/tclwebkit","['version']","['13']" 4,2,"3[12]/tclwebkit","['version']","['12']" 5,2,"3[12]/tclwebkit","['version']","['11']" @@ -2287,11 +2287,11 @@ The content of csv file is: The schema of dumped file is always -- `id UInt64` represents the identify number of the RegexpTree node. +- `id UInt64` represents the id of the RegexpTree node. - `parent_id UInt64` represents the id of the parent of a node. - `regexp String` represents the regular expression string. -- `keys Array(String)` represents the names of user defined attributions. -- `values Array(String)` represents the values of user defined attributions. +- `keys Array(String)` represents the names of user defined attributes. +- `values Array(String)` represents the values of user defined attributes. On the cloud, we can create a table `regexp_dictionary_source_table` with the above table structure. @@ -2314,8 +2314,8 @@ clickhouse client \ --secure \ --password MY_PASSWORD \ --query " - insert into regexp_dictionary_source_table - select * from input ('id UInt64, parent_id UInt64, regexp String, keys Array(String), values Array(String)') + INSERT INTO regexp_dictionary_source_table + SELECT * FROM input ('id UInt64, parent_id UInt64, regexp String, keys Array(String), values Array(String)') FORMAT CSV" < regexp_dict.csv ``` @@ -2333,10 +2333,6 @@ LIFETIME(0) LAYOUT(regexp_tree); ``` -### Use RegexpTree Dictionary as a UA Parser - -With a powerful yaml configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) - ## Embedded Dictionaries {#embedded-dictionaries} diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference index 86a74291b07..4e72cf4ce37 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference +++ b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.reference @@ -1,11 +1,11 @@ 1 0 Linux/(\\d+[\\.\\d]*).+tlinux ['version','name'] ['\\1','TencentOS'] -2 0 (\\d+)/tclwebkit(\\d+[\\.\\d]*) ['comment','version','name'] ['test $1 and $2','$1','Andriod'] +2 0 (\\d+)/tclwebkit(\\d+[\\.\\d]*) ['comment','version','name'] ['test $1 and $2','$1','Android'] 3 2 33/tclwebkit ['version'] ['13'] 4 2 3[12]/tclwebkit ['version'] ['12'] 5 2 3[12]/tclwebkit ['version'] ['11'] 6 2 3[12]/tclwebkit ['version'] ['10'] ('TencentOS',101,'nothing') -('Andriod',13,'test 33 and 11.10') +('Android',13,'test 33 and 11.10') ('',NULL,'nothing') ('',0,'default') 30/tclwebkit0 @@ -23,22 +23,22 @@ 42/tclwebkit12 43/tclwebkit13 44/tclwebkit14 -('Andriod',30) -('Andriod',12) -('Andriod',12) -('Andriod',13) -('Andriod',34) -('Andriod',35) -('Andriod',36) -('Andriod',37) -('Andriod',38) -('Andriod',39) -('Andriod',40) -('Andriod',41) -('Andriod',42) -('Andriod',43) -('Andriod',44) -('Andriod1',33,'matched 3') -1 0 (\\d+)/tclwebkit ['version','name'] ['$1','Andriod'] +('Android',30) +('Android',12) +('Android',12) +('Android',13) +('Android',34) +('Android',35) +('Android',36) +('Android',37) +('Android',38) +('Android',39) +('Android',40) +('Android',41) +('Android',42) +('Android',43) +('Android',44) +('Android1',33,'matched 3') +1 0 (\\d+)/tclwebkit ['version','name'] ['$1','Android'] 2 0 33/tclwebkit ['comment','version'] ['matched 3','13'] -3 1 33/tclwebkit ['name'] ['Andriod1'] +3 1 33/tclwebkit ['name'] ['Android1'] diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql index 15e8adce403..42d7acbf057 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql +++ b/tests/queries/0_stateless/02504_regexp_dictionary_table_source.sql @@ -1,7 +1,7 @@ -- Tags: use-vectorscan -DROP TABLE IF EXISTS regexp_dictionary_source_table; DROP DICTIONARY IF EXISTS regexp_dict1; +DROP TABLE IF EXISTS regexp_dictionary_source_table; CREATE TABLE regexp_dictionary_source_table ( @@ -15,7 +15,7 @@ CREATE TABLE regexp_dictionary_source_table -- test back reference. INSERT INTO regexp_dictionary_source_table VALUES (1, 0, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']) -INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Andriod', '$1', 'test $1 and $2']) +INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']) INSERT INTO regexp_dictionary_source_table VALUES (3, 2, '33/tclwebkit', ['version'], ['13']) INSERT INTO regexp_dictionary_source_table VALUES (4, 2, '3[12]/tclwebkit', ['version'], ['12']) INSERT INTO regexp_dictionary_source_table VALUES (5, 2, '3[12]/tclwebkit', ['version'], ['11']) @@ -65,14 +65,14 @@ SYSTEM RELOAD dictionary regexp_dict1; -- { serverError 489 } truncate table regexp_dictionary_source_table; INSERT INTO regexp_dictionary_source_table VALUES (1, 2, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']) -INSERT INTO regexp_dictionary_source_table VALUES (2, 3, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Andriod', '$1', 'test $1 and $2']) -INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Andriod', '$1', 'test $1 and $2']) +INSERT INTO regexp_dictionary_source_table VALUES (2, 3, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']) +INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']) SYSTEM RELOAD dictionary regexp_dict1; -- { serverError 489 } -- test priority truncate table regexp_dictionary_source_table; -INSERT INTO regexp_dictionary_source_table VALUES (1, 0, '(\d+)/tclwebkit', ['name', 'version'], ['Andriod', '$1']); -INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '33/tclwebkit', ['name'], ['Andriod1']); -- child has more priority than parents. +INSERT INTO regexp_dictionary_source_table VALUES (1, 0, '(\d+)/tclwebkit', ['name', 'version'], ['Android', '$1']); +INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '33/tclwebkit', ['name'], ['Android1']); -- child has more priority than parents. INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '33/tclwebkit', ['version', 'comment'], ['13', 'matched 3']); -- larger id has lower priority than small id. SYSTEM RELOAD dictionary regexp_dict1; select dictGet(regexp_dict1, ('name', 'version', 'comment'), '33/tclwebkit'); @@ -82,6 +82,6 @@ SYSTEM RELOAD dictionary regexp_dict1; -- { serverError 489 } select * from dictionary(regexp_dict1); +DROP DICTIONARY IF EXISTS regexp_dict1; DROP TABLE IF EXISTS regexp_dictionary_source_table; DROP TABLE IF EXISTS needle_table; -DROP DICTIONARY IF EXISTS regexp_dict1; From 6a136897e375e61bf59a25c6dc65e4e2570e7641 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Wed, 17 May 2023 13:23:53 -0600 Subject: [PATCH 0371/2223] Create reddit-comments.md --- .../example-datasets/reddit-comments.md | 636 ++++++++++++++++++ 1 file changed, 636 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/reddit-comments.md diff --git a/docs/en/getting-started/example-datasets/reddit-comments.md b/docs/en/getting-started/example-datasets/reddit-comments.md new file mode 100644 index 00000000000..e1e372746c9 --- /dev/null +++ b/docs/en/getting-started/example-datasets/reddit-comments.md @@ -0,0 +1,636 @@ +--- +slug: /en/getting-started/example-datasets/reddit-comments +sidebar_label: Reddit comments +--- + +# Reddit comments dataset + +This dataset contains publicly-available comments on Reddit that go back to December, 2005, to March, 2023, and contains over 7B rows of data. The raw data is in JSON format in compressed `.zst` files and the rows look like the following: + +```json +{"controversiality":0,"body":"A look at Vietnam and Mexico exposes the myth of market liberalisation.","subreddit_id":"t5_6","link_id":"t3_17863","stickied":false,"subreddit":"reddit.com","score":2,"ups":2,"author_flair_css_class":null,"created_utc":1134365188,"author_flair_text":null,"author":"frjo","id":"c13","edited":false,"parent_id":"t3_17863","gilded":0,"distinguished":null,"retrieved_on":1473738411} +{"created_utc":1134365725,"author_flair_css_class":null,"score":1,"ups":1,"subreddit":"reddit.com","stickied":false,"link_id":"t3_17866","subreddit_id":"t5_6","controversiality":0,"body":"The site states \"What can I use it for? Meeting notes, Reports, technical specs Sign-up sheets, proposals and much more...\", just like any other new breeed of sites that want us to store everything we have on the web. And they even guarantee multiple levels of security and encryption etc. But what prevents these web site operators fom accessing and/or stealing Meeting notes, Reports, technical specs Sign-up sheets, proposals and much more, for competitive or personal gains...? I am pretty sure that most of them are honest, but what's there to prevent me from setting up a good useful site and stealing all your data? Call me paranoid - I am.","retrieved_on":1473738411,"distinguished":null,"gilded":0,"id":"c14","edited":false,"parent_id":"t3_17866","author":"zse7zse","author_flair_text":null} +{"gilded":0,"distinguished":null,"retrieved_on":1473738411,"author":"[deleted]","author_flair_text":null,"edited":false,"id":"c15","parent_id":"t3_17869","subreddit":"reddit.com","score":0,"ups":0,"created_utc":1134366848,"author_flair_css_class":null,"body":"Jython related topics by Frank Wierzbicki","controversiality":0,"subreddit_id":"t5_6","stickied":false,"link_id":"t3_17869"} +{"gilded":0,"retrieved_on":1473738411,"distinguished":null,"author_flair_text":null,"author":"[deleted]","edited":false,"parent_id":"t3_17870","id":"c16","subreddit":"reddit.com","created_utc":1134367660,"author_flair_css_class":null,"score":1,"ups":1,"body":"[deleted]","controversiality":0,"stickied":false,"link_id":"t3_17870","subreddit_id":"t5_6"} +{"gilded":0,"retrieved_on":1473738411,"distinguished":null,"author_flair_text":null,"author":"rjoseph","edited":false,"id":"c17","parent_id":"t3_17817","subreddit":"reddit.com","author_flair_css_class":null,"created_utc":1134367754,"score":1,"ups":1,"body":"Saft is by far the best extension you could tak onto your Safari","controversiality":0,"link_id":"t3_17817","stickied":false,"subreddit_id":"t5_6"} +``` + +A shoutout to Percona for the [motivation behind ingesting this dataset](https://www.percona.com/blog/big-data-set-reddit-comments-analyzing-clickhouse/), which we have downloaded and stored in an S3 bucket. + +:::note +The following commands were executed on ClickHouse Cloud. To run this on your own cluster, replace `default` in the `s3Cluster` function call with the name of your cluster. If you do not have a cluster, then replace the `s3Cluster` function with the `s3` function. +::: + +1. Let's create a table for the Reddit data: + +```sql +CREATE TABLE reddit +( + subreddit LowCardinality(String), + subreddit_id LowCardinality(String), + subreddit_type Enum('public' = 1, 'restricted' = 2, 'user' = 3, 'archived' = 4, 'gold_restricted' = 5, 'private' = 6), + author LowCardinality(String), + body String CODEC(ZSTD(6)), + created_date Date DEFAULT toDate(created_utc), + created_utc DateTime, + retrieved_on DateTime, + id String, + parent_id String, + link_id String, + score Int32, + total_awards_received UInt16, + controversiality UInt8, + gilded UInt8, + collapsed_because_crowd_control UInt8, + collapsed_reason Enum('' = 0, 'comment score below threshold' = 1, 'may be sensitive content' = 2, 'potentially toxic' = 3, 'potentially toxic content' = 4), + distinguished Enum('' = 0, 'moderator' = 1, 'admin' = 2, 'special' = 3), + removal_reason Enum('' = 0, 'legal' = 1), + author_created_utc DateTime, + author_fullname LowCardinality(String), + author_patreon_flair UInt8, + author_premium UInt8, + can_gild UInt8, + can_mod_post UInt8, + collapsed UInt8, + is_submitter UInt8, + _edited String, + locked UInt8, + quarantined UInt8, + no_follow UInt8, + send_replies UInt8, + stickied UInt8, + author_flair_text LowCardinality(String) +) +ENGINE = MergeTree +ORDER BY (subreddit, created_date, author); +``` + +:::note +The names of the files in S3 start with `RC_YYYY-MM` where `YYYY-MM` goes from `2005-12` to `2023-02`. The compression changes a couple of times though, so the file extensions are not consistent. For example: + +- the file names are initially `RC_2005-12.bz2` to `RC_2017-11.bz2` +- then they look like `RC_2017-12.xz` to `RC_2018-09.xz` +- and finally `RC_2018-10.zst` to `RC_2023-02.zst` +::: + +2. We are going to start with one month of data, but if you want to simply insert every row - skip ahead to step 8 below. The following file has 86M records from December, 2017: + +```sql +INSERT INTO reddit + SELECT * + FROM s3Cluster( + 'default', + 'https://clickhouse-public-datasets.s3.eu-central-1.amazonaws.com/reddit/original/RC_2017-12.xz', + 'JSONEachRow' + ); +``` + +If you do not have a cluster, use `s3` instead of `s3Cluster`: + +```sql +INSERT INTO reddit + SELECT * + FROM s3( + 'https://clickhouse-public-datasets.s3.eu-central-1.amazonaws.com/reddit/original/RC_2017-12.xz', + 'JSONEachRow' + ); +``` + +3. It will take a while depending on your resources, but when it's done verify it worked: + +```sql +SELECT formatReadableQuantity(count()) +FROM reddit; +``` + +```response +┌─formatReadableQuantity(count())─┐ +│ 85.97 million │ +└─────────────────────────────────┘ +``` + +4. Let's see how many unique subreddits were in December of 2017: + +```sql +SELECT uniqExact(subreddit) +FROM reddit; +``` + +```response +┌─uniqExact(subreddit)─┐ +│ 91613 │ +└──────────────────────┘ + +1 row in set. Elapsed: 1.572 sec. Processed 85.97 million rows, 367.43 MB (54.71 million rows/s., 233.80 MB/s.) +``` + +5. This query returns the top 10 subreddits (in terms of number of comments): + +```sql +SELECT + subreddit, + count() AS c +FROM reddit +GROUP BY subreddit +ORDER BY c DESC +LIMIT 20; +``` + +```response +┌─subreddit───────┬───────c─┐ +│ AskReddit │ 5245881 │ +│ politics │ 1753120 │ +│ nfl │ 1220266 │ +│ nba │ 960388 │ +│ The_Donald │ 931857 │ +│ news │ 796617 │ +│ worldnews │ 765709 │ +│ CFB │ 710360 │ +│ gaming │ 602761 │ +│ movies │ 601966 │ +│ soccer │ 590628 │ +│ Bitcoin │ 583783 │ +│ pics │ 563408 │ +│ StarWars │ 562514 │ +│ funny │ 547563 │ +│ leagueoflegends │ 517213 │ +│ teenagers │ 492020 │ +│ DestinyTheGame │ 477377 │ +│ todayilearned │ 472650 │ +│ videos │ 450581 │ +└─────────────────┴─────────┘ + +20 rows in set. Elapsed: 0.368 sec. Processed 85.97 million rows, 367.43 MB (233.34 million rows/s., 997.25 MB/s.) +``` + +6. Here are the top 10 authors in December of 2017, in terms of number of comments posted: + +```sql +SELECT + author, + count() AS c +FROM reddit +GROUP BY author +ORDER BY c DESC +LIMIT 10; +``` + +```response +┌─author──────────┬───────c─┐ +│ [deleted] │ 5913324 │ +│ AutoModerator │ 784886 │ +│ ImagesOfNetwork │ 83241 │ +│ BitcoinAllBot │ 54484 │ +│ imguralbumbot │ 45822 │ +│ RPBot │ 29337 │ +│ WikiTextBot │ 25982 │ +│ Concise_AMA_Bot │ 19974 │ +│ MTGCardFetcher │ 19103 │ +│ TotesMessenger │ 19057 │ +└─────────────────┴─────────┘ + +10 rows in set. Elapsed: 8.143 sec. Processed 85.97 million rows, 711.05 MB (10.56 million rows/s., 87.32 MB/s.) +``` + +7. We already inserted some data, but we will start over: + +```sql +TRUNCATE TABLE reddit; +``` + +8. This is a fun dataset and it looks like we can find some great information, so let's go ahead and insert the entire dataset from 2005 to 2023. When you're ready, run this command to insert all the rows. (It takes a while - up to 17 hours!) + +```sql +INSERT INTO reddit +SELECT * +FROM s3Cluster( + 'default', + 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC*', + 'JSONEachRow' + ) +SETTINGS zstd_window_log_max = 31; +``` + +The response looks like: + +```response +0 rows in set. Elapsed: 61187.839 sec. Processed 6.74 billion rows, 2.06 TB (110.17 thousand rows/s., 33.68 MB/s.) +``` + +8. Let's see how many rows were inserted and how much disk space the table is using: + + +```sql +SELECT + sum(rows) AS count, + formatReadableQuantity(count), + formatReadableSize(sum(bytes)) AS disk_size, + formatReadableSize(sum(data_uncompressed_bytes)) AS uncompressed_size +FROM system.parts +WHERE (table = 'reddit') AND active +``` + +Notice the compression of disk storage is about 1/3 of the uncompressed size: + +```response +┌──────count─┬─formatReadableQuantity(sum(rows))─┬─disk_size──┬─uncompressed_size─┐ +│ 6739503568 │ 6.74 billion │ 501.10 GiB │ 1.51 TiB │ +└────────────┴───────────────────────────────────┴────────────┴───────────────────┘ + +1 row in set. Elapsed: 0.010 sec. +``` + +9. The following query shows how many comments, authors and subreddits we have for each month: + +```sql +SELECT + toStartOfMonth(created_utc) AS firstOfMonth, + count() AS c, + bar(c, 0, 50000000, 25) AS bar_count, + uniq(author) AS authors, + bar(authors, 0, 5000000, 25) AS bar_authors, + uniq(subreddit) AS subreddits, + bar(subreddits, 0, 100000, 25) AS bar_subreddits +FROM reddit +GROUP BY firstOfMonth +ORDER BY firstOfMonth ASC; +``` + +This is a substantial query that has to process all 6.74 billion rows, but we still get an impressive response time (about 3 minutes): + +```response +┌─firstOfMonth─┬─────────c─┬─bar_count─────────────────┬─authors─┬─bar_authors───────────────┬─subreddits─┬─bar_subreddits────────────┐ +│ 2005-12-01 │ 1075 │ │ 394 │ │ 1 │ │ +│ 2006-01-01 │ 3666 │ │ 791 │ │ 2 │ │ +│ 2006-02-01 │ 9095 │ │ 1464 │ │ 18 │ │ +│ 2006-03-01 │ 13859 │ │ 1958 │ │ 15 │ │ +│ 2006-04-01 │ 19090 │ │ 2334 │ │ 21 │ │ +│ 2006-05-01 │ 26859 │ │ 2698 │ │ 21 │ │ +│ 2006-06-01 │ 29163 │ │ 3043 │ │ 19 │ │ +│ 2006-07-01 │ 37031 │ │ 3532 │ │ 22 │ │ +│ 2006-08-01 │ 50559 │ │ 4750 │ │ 24 │ │ +│ 2006-09-01 │ 50675 │ │ 4908 │ │ 21 │ │ +│ 2006-10-01 │ 54148 │ │ 5654 │ │ 31 │ │ +│ 2006-11-01 │ 62021 │ │ 6490 │ │ 23 │ │ +│ 2006-12-01 │ 61018 │ │ 6707 │ │ 24 │ │ +│ 2007-01-01 │ 81341 │ │ 7931 │ │ 23 │ │ +│ 2007-02-01 │ 95634 │ │ 9020 │ │ 21 │ │ +│ 2007-03-01 │ 112444 │ │ 10842 │ │ 23 │ │ +│ 2007-04-01 │ 126773 │ │ 10701 │ │ 26 │ │ +│ 2007-05-01 │ 170097 │ │ 11365 │ │ 25 │ │ +│ 2007-06-01 │ 178800 │ │ 11267 │ │ 22 │ │ +│ 2007-07-01 │ 203319 │ │ 12482 │ │ 25 │ │ +│ 2007-08-01 │ 225111 │ │ 14124 │ │ 30 │ │ +│ 2007-09-01 │ 259497 │ ▏ │ 15416 │ │ 33 │ │ +│ 2007-10-01 │ 274170 │ ▏ │ 15302 │ │ 36 │ │ +│ 2007-11-01 │ 372983 │ ▏ │ 15134 │ │ 43 │ │ +│ 2007-12-01 │ 363390 │ ▏ │ 15915 │ │ 31 │ │ +│ 2008-01-01 │ 452990 │ ▏ │ 18857 │ │ 126 │ │ +│ 2008-02-01 │ 441768 │ ▏ │ 18266 │ │ 173 │ │ +│ 2008-03-01 │ 463728 │ ▏ │ 18947 │ │ 292 │ │ +│ 2008-04-01 │ 468317 │ ▏ │ 18590 │ │ 323 │ │ +│ 2008-05-01 │ 536380 │ ▎ │ 20861 │ │ 375 │ │ +│ 2008-06-01 │ 577684 │ ▎ │ 22557 │ │ 575 │ ▏ │ +│ 2008-07-01 │ 592610 │ ▎ │ 23123 │ │ 657 │ ▏ │ +│ 2008-08-01 │ 595959 │ ▎ │ 23729 │ │ 707 │ ▏ │ +│ 2008-09-01 │ 680892 │ ▎ │ 26374 │ ▏ │ 801 │ ▏ │ +│ 2008-10-01 │ 789874 │ ▍ │ 28970 │ ▏ │ 893 │ ▏ │ +│ 2008-11-01 │ 792310 │ ▍ │ 30272 │ ▏ │ 1024 │ ▎ │ +│ 2008-12-01 │ 850359 │ ▍ │ 34073 │ ▏ │ 1103 │ ▎ │ +│ 2009-01-01 │ 1051649 │ ▌ │ 38978 │ ▏ │ 1316 │ ▎ │ +│ 2009-02-01 │ 944711 │ ▍ │ 43390 │ ▏ │ 1132 │ ▎ │ +│ 2009-03-01 │ 1048643 │ ▌ │ 46516 │ ▏ │ 1203 │ ▎ │ +│ 2009-04-01 │ 1094599 │ ▌ │ 48284 │ ▏ │ 1334 │ ▎ │ +│ 2009-05-01 │ 1201257 │ ▌ │ 52512 │ ▎ │ 1395 │ ▎ │ +│ 2009-06-01 │ 1258750 │ ▋ │ 57728 │ ▎ │ 1473 │ ▎ │ +│ 2009-07-01 │ 1470290 │ ▋ │ 60098 │ ▎ │ 1686 │ ▍ │ +│ 2009-08-01 │ 1750688 │ ▉ │ 67347 │ ▎ │ 1777 │ ▍ │ +│ 2009-09-01 │ 2032276 │ █ │ 78051 │ ▍ │ 1784 │ ▍ │ +│ 2009-10-01 │ 2242017 │ █ │ 93409 │ ▍ │ 2071 │ ▌ │ +│ 2009-11-01 │ 2207444 │ █ │ 95940 │ ▍ │ 2141 │ ▌ │ +│ 2009-12-01 │ 2560510 │ █▎ │ 104239 │ ▌ │ 2141 │ ▌ │ +│ 2010-01-01 │ 2884096 │ █▍ │ 114314 │ ▌ │ 2313 │ ▌ │ +│ 2010-02-01 │ 2687779 │ █▎ │ 115683 │ ▌ │ 2522 │ ▋ │ +│ 2010-03-01 │ 3228254 │ █▌ │ 125775 │ ▋ │ 2890 │ ▋ │ +│ 2010-04-01 │ 3209898 │ █▌ │ 128936 │ ▋ │ 3170 │ ▊ │ +│ 2010-05-01 │ 3267363 │ █▋ │ 131851 │ ▋ │ 3166 │ ▊ │ +│ 2010-06-01 │ 3532867 │ █▊ │ 139522 │ ▋ │ 3301 │ ▊ │ +│ 2010-07-01 │ 4032737 │ ██ │ 153451 │ ▊ │ 3662 │ ▉ │ +│ 2010-08-01 │ 4247982 │ ██ │ 164071 │ ▊ │ 3653 │ ▉ │ +│ 2010-09-01 │ 4704069 │ ██▎ │ 186613 │ ▉ │ 4009 │ █ │ +│ 2010-10-01 │ 5032368 │ ██▌ │ 203800 │ █ │ 4154 │ █ │ +│ 2010-11-01 │ 5689002 │ ██▊ │ 226134 │ █▏ │ 4383 │ █ │ +│ 2010-12-01 │ 5972642 │ ██▉ │ 245824 │ █▏ │ 4692 │ █▏ │ +│ 2011-01-01 │ 6603329 │ ███▎ │ 270025 │ █▎ │ 5141 │ █▎ │ +│ 2011-02-01 │ 6363114 │ ███▏ │ 277593 │ █▍ │ 5202 │ █▎ │ +│ 2011-03-01 │ 7556165 │ ███▊ │ 314748 │ █▌ │ 5445 │ █▎ │ +│ 2011-04-01 │ 7571398 │ ███▊ │ 329920 │ █▋ │ 6128 │ █▌ │ +│ 2011-05-01 │ 8803949 │ ████▍ │ 365013 │ █▊ │ 6834 │ █▋ │ +│ 2011-06-01 │ 9766511 │ ████▉ │ 393945 │ █▉ │ 7519 │ █▉ │ +│ 2011-07-01 │ 10557466 │ █████▎ │ 424235 │ ██ │ 8293 │ ██ │ +│ 2011-08-01 │ 12316144 │ ██████▏ │ 475326 │ ██▍ │ 9657 │ ██▍ │ +│ 2011-09-01 │ 12150412 │ ██████ │ 503142 │ ██▌ │ 10278 │ ██▌ │ +│ 2011-10-01 │ 13470278 │ ██████▋ │ 548801 │ ██▋ │ 10922 │ ██▋ │ +│ 2011-11-01 │ 13621533 │ ██████▊ │ 574435 │ ██▊ │ 11572 │ ██▉ │ +│ 2011-12-01 │ 14509469 │ ███████▎ │ 622849 │ ███ │ 12335 │ ███ │ +│ 2012-01-01 │ 16350205 │ ████████▏ │ 696110 │ ███▍ │ 14281 │ ███▌ │ +│ 2012-02-01 │ 16015695 │ ████████ │ 722892 │ ███▌ │ 14949 │ ███▋ │ +│ 2012-03-01 │ 17881943 │ ████████▉ │ 789664 │ ███▉ │ 15795 │ ███▉ │ +│ 2012-04-01 │ 19044534 │ █████████▌ │ 842491 │ ████▏ │ 16440 │ ████ │ +│ 2012-05-01 │ 20388260 │ ██████████▏ │ 886176 │ ████▍ │ 16974 │ ████▏ │ +│ 2012-06-01 │ 21897913 │ ██████████▉ │ 946798 │ ████▋ │ 17952 │ ████▍ │ +│ 2012-07-01 │ 24087517 │ ████████████ │ 1018636 │ █████ │ 19069 │ ████▊ │ +│ 2012-08-01 │ 25703326 │ ████████████▊ │ 1094445 │ █████▍ │ 20553 │ █████▏ │ +│ 2012-09-01 │ 23419524 │ ███████████▋ │ 1088491 │ █████▍ │ 20831 │ █████▏ │ +│ 2012-10-01 │ 24788236 │ ████████████▍ │ 1131885 │ █████▋ │ 21868 │ █████▍ │ +│ 2012-11-01 │ 24648302 │ ████████████▎ │ 1167608 │ █████▊ │ 21791 │ █████▍ │ +│ 2012-12-01 │ 26080276 │ █████████████ │ 1218402 │ ██████ │ 22622 │ █████▋ │ +│ 2013-01-01 │ 30365867 │ ███████████████▏ │ 1341703 │ ██████▋ │ 24696 │ ██████▏ │ +│ 2013-02-01 │ 27213960 │ █████████████▌ │ 1304756 │ ██████▌ │ 24514 │ ██████▏ │ +│ 2013-03-01 │ 30771274 │ ███████████████▍ │ 1391703 │ ██████▉ │ 25730 │ ██████▍ │ +│ 2013-04-01 │ 33259557 │ ████████████████▋ │ 1485971 │ ███████▍ │ 27294 │ ██████▊ │ +│ 2013-05-01 │ 33126225 │ ████████████████▌ │ 1506473 │ ███████▌ │ 27299 │ ██████▊ │ +│ 2013-06-01 │ 32648247 │ ████████████████▎ │ 1506650 │ ███████▌ │ 27450 │ ██████▊ │ +│ 2013-07-01 │ 34922133 │ █████████████████▍ │ 1561771 │ ███████▊ │ 28294 │ ███████ │ +│ 2013-08-01 │ 34766579 │ █████████████████▍ │ 1589781 │ ███████▉ │ 28943 │ ███████▏ │ +│ 2013-09-01 │ 31990369 │ ███████████████▉ │ 1570342 │ ███████▊ │ 29408 │ ███████▎ │ +│ 2013-10-01 │ 35940040 │ █████████████████▉ │ 1683770 │ ████████▍ │ 30273 │ ███████▌ │ +│ 2013-11-01 │ 37396497 │ ██████████████████▋ │ 1757467 │ ████████▊ │ 31173 │ ███████▊ │ +│ 2013-12-01 │ 39810216 │ ███████████████████▉ │ 1846204 │ █████████▏ │ 32326 │ ████████ │ +│ 2014-01-01 │ 42420655 │ █████████████████████▏ │ 1927229 │ █████████▋ │ 35603 │ ████████▉ │ +│ 2014-02-01 │ 38703362 │ ███████████████████▎ │ 1874067 │ █████████▎ │ 37007 │ █████████▎ │ +│ 2014-03-01 │ 42459956 │ █████████████████████▏ │ 1959888 │ █████████▊ │ 37948 │ █████████▍ │ +│ 2014-04-01 │ 42440735 │ █████████████████████▏ │ 1951369 │ █████████▊ │ 38362 │ █████████▌ │ +│ 2014-05-01 │ 42514094 │ █████████████████████▎ │ 1970197 │ █████████▊ │ 39078 │ █████████▊ │ +│ 2014-06-01 │ 41990650 │ ████████████████████▉ │ 1943850 │ █████████▋ │ 38268 │ █████████▌ │ +│ 2014-07-01 │ 46868899 │ ███████████████████████▍ │ 2059346 │ ██████████▎ │ 40634 │ ██████████▏ │ +│ 2014-08-01 │ 46990813 │ ███████████████████████▍ │ 2117335 │ ██████████▌ │ 41764 │ ██████████▍ │ +│ 2014-09-01 │ 44992201 │ ██████████████████████▍ │ 2124708 │ ██████████▌ │ 41890 │ ██████████▍ │ +│ 2014-10-01 │ 47497520 │ ███████████████████████▋ │ 2206535 │ ███████████ │ 43109 │ ██████████▊ │ +│ 2014-11-01 │ 46118074 │ ███████████████████████ │ 2239747 │ ███████████▏ │ 43718 │ ██████████▉ │ +│ 2014-12-01 │ 48807699 │ ████████████████████████▍ │ 2372945 │ ███████████▊ │ 43823 │ ██████████▉ │ +│ 2015-01-01 │ 53851542 │ █████████████████████████ │ 2499536 │ ████████████▍ │ 47172 │ ███████████▊ │ +│ 2015-02-01 │ 48342747 │ ████████████████████████▏ │ 2448496 │ ████████████▏ │ 47229 │ ███████████▊ │ +│ 2015-03-01 │ 54564441 │ █████████████████████████ │ 2550534 │ ████████████▊ │ 48156 │ ████████████ │ +│ 2015-04-01 │ 55005780 │ █████████████████████████ │ 2609443 │ █████████████ │ 49865 │ ████████████▍ │ +│ 2015-05-01 │ 54504410 │ █████████████████████████ │ 2585535 │ ████████████▉ │ 50137 │ ████████████▌ │ +│ 2015-06-01 │ 54258492 │ █████████████████████████ │ 2595129 │ ████████████▉ │ 49598 │ ████████████▍ │ +│ 2015-07-01 │ 58451788 │ █████████████████████████ │ 2720026 │ █████████████▌ │ 55022 │ █████████████▊ │ +│ 2015-08-01 │ 58075327 │ █████████████████████████ │ 2743994 │ █████████████▋ │ 55302 │ █████████████▊ │ +│ 2015-09-01 │ 55574825 │ █████████████████████████ │ 2672793 │ █████████████▎ │ 53960 │ █████████████▍ │ +│ 2015-10-01 │ 59494045 │ █████████████████████████ │ 2816426 │ ██████████████ │ 70210 │ █████████████████▌ │ +│ 2015-11-01 │ 57117500 │ █████████████████████████ │ 2847146 │ ██████████████▏ │ 71363 │ █████████████████▊ │ +│ 2015-12-01 │ 58523312 │ █████████████████████████ │ 2854840 │ ██████████████▎ │ 94559 │ ███████████████████████▋ │ +│ 2016-01-01 │ 61991732 │ █████████████████████████ │ 2920366 │ ██████████████▌ │ 108438 │ █████████████████████████ │ +│ 2016-02-01 │ 59189875 │ █████████████████████████ │ 2854683 │ ██████████████▎ │ 109916 │ █████████████████████████ │ +│ 2016-03-01 │ 63918864 │ █████████████████████████ │ 2969542 │ ██████████████▊ │ 84787 │ █████████████████████▏ │ +│ 2016-04-01 │ 64271256 │ █████████████████████████ │ 2999086 │ ██████████████▉ │ 61647 │ ███████████████▍ │ +│ 2016-05-01 │ 65212004 │ █████████████████████████ │ 3034674 │ ███████████████▏ │ 67465 │ ████████████████▊ │ +│ 2016-06-01 │ 65867743 │ █████████████████████████ │ 3057604 │ ███████████████▎ │ 75170 │ ██████████████████▊ │ +│ 2016-07-01 │ 66974735 │ █████████████████████████ │ 3199374 │ ███████████████▉ │ 77732 │ ███████████████████▍ │ +│ 2016-08-01 │ 69654819 │ █████████████████████████ │ 3239957 │ ████████████████▏ │ 63080 │ ███████████████▊ │ +│ 2016-09-01 │ 67024973 │ █████████████████████████ │ 3190864 │ ███████████████▉ │ 62324 │ ███████████████▌ │ +│ 2016-10-01 │ 71826553 │ █████████████████████████ │ 3284340 │ ████████████████▍ │ 62549 │ ███████████████▋ │ +│ 2016-11-01 │ 71022319 │ █████████████████████████ │ 3300822 │ ████████████████▌ │ 69718 │ █████████████████▍ │ +│ 2016-12-01 │ 72942967 │ █████████████████████████ │ 3430324 │ █████████████████▏ │ 71705 │ █████████████████▉ │ +│ 2017-01-01 │ 78946585 │ █████████████████████████ │ 3572093 │ █████████████████▊ │ 78198 │ ███████████████████▌ │ +│ 2017-02-01 │ 70609487 │ █████████████████████████ │ 3421115 │ █████████████████ │ 69823 │ █████████████████▍ │ +│ 2017-03-01 │ 79723106 │ █████████████████████████ │ 3638122 │ ██████████████████▏ │ 73865 │ ██████████████████▍ │ +│ 2017-04-01 │ 77478009 │ █████████████████████████ │ 3620591 │ ██████████████████ │ 74387 │ ██████████████████▌ │ +│ 2017-05-01 │ 79810360 │ █████████████████████████ │ 3650820 │ ██████████████████▎ │ 74356 │ ██████████████████▌ │ +│ 2017-06-01 │ 79901711 │ █████████████████████████ │ 3737614 │ ██████████████████▋ │ 72114 │ ██████████████████ │ +│ 2017-07-01 │ 81798725 │ █████████████████████████ │ 3872330 │ ███████████████████▎ │ 76052 │ ███████████████████ │ +│ 2017-08-01 │ 84658503 │ █████████████████████████ │ 3960093 │ ███████████████████▊ │ 77798 │ ███████████████████▍ │ +│ 2017-09-01 │ 83165192 │ █████████████████████████ │ 3880501 │ ███████████████████▍ │ 78402 │ ███████████████████▌ │ +│ 2017-10-01 │ 85828912 │ █████████████████████████ │ 3980335 │ ███████████████████▉ │ 80685 │ ████████████████████▏ │ +│ 2017-11-01 │ 84965681 │ █████████████████████████ │ 4026749 │ ████████████████████▏ │ 82659 │ ████████████████████▋ │ +│ 2017-12-01 │ 85973810 │ █████████████████████████ │ 4196354 │ ████████████████████▉ │ 91984 │ ██████████████████████▉ │ +│ 2018-01-01 │ 91558594 │ █████████████████████████ │ 4364443 │ █████████████████████▊ │ 102577 │ █████████████████████████ │ +│ 2018-02-01 │ 86467179 │ █████████████████████████ │ 4277899 │ █████████████████████▍ │ 104610 │ █████████████████████████ │ +│ 2018-03-01 │ 96490262 │ █████████████████████████ │ 4422470 │ ██████████████████████ │ 112559 │ █████████████████████████ │ +│ 2018-04-01 │ 98101232 │ █████████████████████████ │ 4572434 │ ██████████████████████▊ │ 105284 │ █████████████████████████ │ +│ 2018-05-01 │ 100109100 │ █████████████████████████ │ 4698908 │ ███████████████████████▍ │ 103910 │ █████████████████████████ │ +│ 2018-06-01 │ 100009462 │ █████████████████████████ │ 4697426 │ ███████████████████████▍ │ 101107 │ █████████████████████████ │ +│ 2018-07-01 │ 108151359 │ █████████████████████████ │ 5099492 │ █████████████████████████ │ 106184 │ █████████████████████████ │ +│ 2018-08-01 │ 107330940 │ █████████████████████████ │ 5084082 │ █████████████████████████ │ 109985 │ █████████████████████████ │ +│ 2018-09-01 │ 104473929 │ █████████████████████████ │ 5011953 │ █████████████████████████ │ 109710 │ █████████████████████████ │ +│ 2018-10-01 │ 112346556 │ █████████████████████████ │ 5320405 │ █████████████████████████ │ 112533 │ █████████████████████████ │ +│ 2018-11-01 │ 112573001 │ █████████████████████████ │ 5353282 │ █████████████████████████ │ 112211 │ █████████████████████████ │ +│ 2018-12-01 │ 121953600 │ █████████████████████████ │ 5611543 │ █████████████████████████ │ 118291 │ █████████████████████████ │ +│ 2019-01-01 │ 129386587 │ █████████████████████████ │ 6016687 │ █████████████████████████ │ 125725 │ █████████████████████████ │ +│ 2019-02-01 │ 120645639 │ █████████████████████████ │ 5974488 │ █████████████████████████ │ 125420 │ █████████████████████████ │ +│ 2019-03-01 │ 137650471 │ █████████████████████████ │ 6410197 │ █████████████████████████ │ 135924 │ █████████████████████████ │ +│ 2019-04-01 │ 138473643 │ █████████████████████████ │ 6416384 │ █████████████████████████ │ 139844 │ █████████████████████████ │ +│ 2019-05-01 │ 142463421 │ █████████████████████████ │ 6574836 │ █████████████████████████ │ 142012 │ █████████████████████████ │ +│ 2019-06-01 │ 134172939 │ █████████████████████████ │ 6601267 │ █████████████████████████ │ 140997 │ █████████████████████████ │ +│ 2019-07-01 │ 145965083 │ █████████████████████████ │ 6901822 │ █████████████████████████ │ 147802 │ █████████████████████████ │ +│ 2019-08-01 │ 146854393 │ █████████████████████████ │ 6993882 │ █████████████████████████ │ 151888 │ █████████████████████████ │ +│ 2019-09-01 │ 137540219 │ █████████████████████████ │ 7001362 │ █████████████████████████ │ 148839 │ █████████████████████████ │ +│ 2019-10-01 │ 129771456 │ █████████████████████████ │ 6825690 │ █████████████████████████ │ 144453 │ █████████████████████████ │ +│ 2019-11-01 │ 107990259 │ █████████████████████████ │ 6368286 │ █████████████████████████ │ 141768 │ █████████████████████████ │ +│ 2019-12-01 │ 112895934 │ █████████████████████████ │ 6640902 │ █████████████████████████ │ 148277 │ █████████████████████████ │ +│ 2020-01-01 │ 54354879 │ █████████████████████████ │ 4782339 │ ███████████████████████▉ │ 111658 │ █████████████████████████ │ +│ 2020-02-01 │ 22696923 │ ███████████▎ │ 3135175 │ ███████████████▋ │ 79521 │ ███████████████████▉ │ +│ 2020-03-01 │ 3466677 │ █▋ │ 987960 │ ████▉ │ 40901 │ ██████████▏ │ +└──────────────┴───────────┴───────────────────────────┴─────────┴───────────────────────────┴────────────┴───────────────────────────┘ + +172 rows in set. Elapsed: 184.809 sec. Processed 6.74 billion rows, 89.56 GB (36.47 million rows/s., 484.62 MB/s.) +``` + +10. Here are the top 10 subreddits of 2022: + +```sql +SELECT + subreddit, + count() AS count +FROM reddit +WHERE toYear(created_utc) = 2022 +GROUP BY subreddit +ORDER BY count DESC +LIMIT 10; +``` + +The response is: + +```response +┌─subreddit────────┬───count─┐ +│ AskReddit │ 3858203 │ +│ politics │ 1356782 │ +│ memes │ 1249120 │ +│ nfl │ 883667 │ +│ worldnews │ 866065 │ +│ teenagers │ 777095 │ +│ AmItheAsshole │ 752720 │ +│ dankmemes │ 657932 │ +│ nba │ 514184 │ +│ unpopularopinion │ 473649 │ +└──────────────────┴─────────┘ + +10 rows in set. Elapsed: 27.824 sec. Processed 6.74 billion rows, 53.26 GB (242.22 million rows/s., 1.91 GB/s.) +``` + +11. Let's see which subreddits had the biggest increase in commnents from 2018 to 2019: + +```sql +SELECT + subreddit, + newcount - oldcount AS diff +FROM +( + SELECT + subreddit, + count(*) AS newcount + FROM reddit + WHERE toYear(created_utc) = 2019 + GROUP BY subreddit +) +ALL INNER JOIN +( + SELECT + subreddit, + count(*) AS oldcount + FROM reddit + WHERE toYear(created_utc) = 2018 + GROUP BY subreddit +) USING (subreddit) +ORDER BY diff DESC +LIMIT 50 +SETTINGS joined_subquery_requires_alias = 0; +``` + +It looks like memes and teenagers were busy on Reddit in 2019: + +```response +┌─subreddit────────────┬─────diff─┐ +│ memes │ 15368369 │ +│ AskReddit │ 14663662 │ +│ teenagers │ 12266991 │ +│ AmItheAsshole │ 11561538 │ +│ dankmemes │ 11305158 │ +│ unpopularopinion │ 6332772 │ +│ PewdiepieSubmissions │ 5930818 │ +│ Market76 │ 5014668 │ +│ relationship_advice │ 3776383 │ +│ freefolk │ 3169236 │ +│ Minecraft │ 3160241 │ +│ classicwow │ 2907056 │ +│ Animemes │ 2673398 │ +│ gameofthrones │ 2402835 │ +│ PublicFreakout │ 2267605 │ +│ ShitPostCrusaders │ 2207266 │ +│ RoastMe │ 2195715 │ +│ gonewild │ 2148649 │ +│ AnthemTheGame │ 1803818 │ +│ entitledparents │ 1706270 │ +│ MortalKombat │ 1679508 │ +│ Cringetopia │ 1620555 │ +│ pokemon │ 1615266 │ +│ HistoryMemes │ 1608289 │ +│ Brawlstars │ 1574977 │ +│ iamatotalpieceofshit │ 1558315 │ +│ trashy │ 1518549 │ +│ ChapoTrapHouse │ 1505748 │ +│ Pikabu │ 1501001 │ +│ Showerthoughts │ 1475101 │ +│ cursedcomments │ 1465607 │ +│ ukpolitics │ 1386043 │ +│ wallstreetbets │ 1384431 │ +│ interestingasfuck │ 1378900 │ +│ wholesomememes │ 1353333 │ +│ AskOuija │ 1233263 │ +│ borderlands3 │ 1197192 │ +│ aww │ 1168257 │ +│ insanepeoplefacebook │ 1155473 │ +│ FortniteCompetitive │ 1122778 │ +│ EpicSeven │ 1117380 │ +│ FreeKarma4U │ 1116423 │ +│ YangForPresidentHQ │ 1086700 │ +│ SquaredCircle │ 1044089 │ +│ MurderedByWords │ 1042511 │ +│ AskMen │ 1024434 │ +│ thedivision │ 1016634 │ +│ barstoolsports │ 985032 │ +│ nfl │ 978340 │ +│ BattlefieldV │ 971408 │ +└──────────────────────┴──────────┘ + +50 rows in set. Elapsed: 65.954 sec. Processed 13.48 billion rows, 79.67 GB (204.37 million rows/s., 1.21 GB/s.) +``` + +12. One more query: let's compare ClickHouse mentions to other technologies like Snowflake and Postgres. This query is a big one because it has to search all the comments three times for a substring, and unfortunately ClickHouse user are obviously not very active on Reddit yet: + +```sql +SELECT + toStartOfQuarter(created_utc) AS quarter, + sum(if(positionCaseInsensitive(body, 'clickhouse') > 0, 1, 0)) AS clickhouse, + sum(if(positionCaseInsensitive(body, 'snowflake') > 0, 1, 0)) AS snowflake, + sum(if(positionCaseInsensitive(body, 'postgres') > 0, 1, 0)) AS postgres +FROM reddit +GROUP BY quarter +ORDER BY quarter ASC; +``` + +```response +┌────Quarter─┬─clickhouse─┬─snowflake─┬─postgres─┐ +│ 2005-10-01 │ 0 │ 0 │ 0 │ +│ 2006-01-01 │ 0 │ 2 │ 23 │ +│ 2006-04-01 │ 0 │ 2 │ 24 │ +│ 2006-07-01 │ 0 │ 4 │ 13 │ +│ 2006-10-01 │ 0 │ 23 │ 73 │ +│ 2007-01-01 │ 0 │ 14 │ 91 │ +│ 2007-04-01 │ 0 │ 10 │ 59 │ +│ 2007-07-01 │ 0 │ 39 │ 116 │ +│ 2007-10-01 │ 0 │ 45 │ 125 │ +│ 2008-01-01 │ 0 │ 53 │ 234 │ +│ 2008-04-01 │ 0 │ 79 │ 303 │ +│ 2008-07-01 │ 0 │ 102 │ 174 │ +│ 2008-10-01 │ 0 │ 156 │ 323 │ +│ 2009-01-01 │ 0 │ 206 │ 208 │ +│ 2009-04-01 │ 0 │ 178 │ 417 │ +│ 2009-07-01 │ 0 │ 300 │ 295 │ +│ 2009-10-01 │ 0 │ 633 │ 589 │ +│ 2010-01-01 │ 0 │ 555 │ 501 │ +│ 2010-04-01 │ 0 │ 587 │ 469 │ +│ 2010-07-01 │ 0 │ 770 │ 821 │ +│ 2010-10-01 │ 0 │ 1480 │ 550 │ +│ 2011-01-01 │ 0 │ 1482 │ 568 │ +│ 2011-04-01 │ 0 │ 1558 │ 406 │ +│ 2011-07-01 │ 0 │ 2163 │ 628 │ +│ 2011-10-01 │ 0 │ 4064 │ 566 │ +│ 2012-01-01 │ 0 │ 4621 │ 662 │ +│ 2012-04-01 │ 0 │ 5737 │ 785 │ +│ 2012-07-01 │ 0 │ 6097 │ 1127 │ +│ 2012-10-01 │ 0 │ 7986 │ 600 │ +│ 2013-01-01 │ 0 │ 9704 │ 839 │ +│ 2013-04-01 │ 0 │ 8161 │ 853 │ +│ 2013-07-01 │ 0 │ 9704 │ 1028 │ +│ 2013-10-01 │ 0 │ 12879 │ 1404 │ +│ 2014-01-01 │ 0 │ 12317 │ 1548 │ +│ 2014-04-01 │ 0 │ 13181 │ 1577 │ +│ 2014-07-01 │ 0 │ 15640 │ 1710 │ +│ 2014-10-01 │ 0 │ 19479 │ 1959 │ +│ 2015-01-01 │ 0 │ 20411 │ 2104 │ +│ 2015-04-01 │ 1 │ 20309 │ 9112 │ +│ 2015-07-01 │ 0 │ 20325 │ 4771 │ +│ 2015-10-01 │ 0 │ 25087 │ 3030 │ +│ 2016-01-01 │ 0 │ 23462 │ 3126 │ +│ 2016-04-01 │ 3 │ 25496 │ 2757 │ +│ 2016-07-01 │ 4 │ 28233 │ 2928 │ +│ 2016-10-01 │ 2 │ 45445 │ 2449 │ +│ 2017-01-01 │ 9 │ 76019 │ 2808 │ +│ 2017-04-01 │ 9 │ 67919 │ 2803 │ +│ 2017-07-01 │ 13 │ 68974 │ 2771 │ +│ 2017-10-01 │ 12 │ 69730 │ 2906 │ +│ 2018-01-01 │ 17 │ 67476 │ 3152 │ +│ 2018-04-01 │ 3 │ 67139 │ 3986 │ +│ 2018-07-01 │ 14 │ 67979 │ 3609 │ +│ 2018-10-01 │ 28 │ 74147 │ 3850 │ +│ 2019-01-01 │ 14 │ 80250 │ 4305 │ +│ 2019-04-01 │ 30 │ 70307 │ 3872 │ +│ 2019-07-01 │ 33 │ 77149 │ 4164 │ +│ 2019-10-01 │ 13 │ 76746 │ 3541 │ +│ 2020-01-01 │ 16 │ 54475 │ 846 │ +└────────────┴────────────┴───────────┴──────────┘ + +58 rows in set. Elapsed: 2663.751 sec. Processed 6.74 billion rows, 1.21 TB (2.53 million rows/s., 454.37 MB/s.) +``` \ No newline at end of file From e2e3a03dbe1e1dc5a4f2d39532bf072c9e6bffa3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 17 May 2023 22:33:30 +0200 Subject: [PATCH 0372/2223] Revert "`groupArray` returns cannot be nullable" --- src/AggregateFunctions/AggregateFunctionGroupArray.cpp | 2 +- src/AggregateFunctions/AggregateFunctionNull.cpp | 2 +- tests/queries/0_stateless/00529_orantius.reference | 2 +- tests/queries/0_stateless/01664_array_slice_ubsan.reference | 2 +- .../queries/0_stateless/02713_group_array_nullable.reference | 3 --- tests/queries/0_stateless/02713_group_array_nullable.sql | 5 ----- 6 files changed, 4 insertions(+), 12 deletions(-) delete mode 100644 tests/queries/0_stateless/02713_group_array_nullable.reference delete mode 100644 tests/queries/0_stateless/02713_group_array_nullable.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 67cfa3f7356..15f500b8bb6 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -121,7 +121,7 @@ AggregateFunctionPtr createAggregateFunctionGroupArraySample( void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory) { - AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = true }; + AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("groupArray", { createAggregateFunctionGroupArray, properties }); factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties }); diff --git a/src/AggregateFunctions/AggregateFunctionNull.cpp b/src/AggregateFunctions/AggregateFunctionNull.cpp index 19c66db98cd..3d3d7af3026 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.cpp +++ b/src/AggregateFunctions/AggregateFunctionNull.cpp @@ -72,7 +72,7 @@ public: { /// Currently the only functions that returns not-NULL on all NULL arguments are count and uniq, and they returns UInt64. if (properties.returns_default_when_only_null) - return std::make_shared(arguments, params, nested_function->getResultType()); + return std::make_shared(arguments, params, std::make_shared()); else return std::make_shared(arguments, params, std::make_shared(std::make_shared())); } diff --git a/tests/queries/0_stateless/00529_orantius.reference b/tests/queries/0_stateless/00529_orantius.reference index 1deecf44752..865659048cc 100644 --- a/tests/queries/0_stateless/00529_orantius.reference +++ b/tests/queries/0_stateless/00529_orantius.reference @@ -3,7 +3,7 @@ 1 1 [[1],[-1]] -[] +\N 1 42 42 [NULL,'','',NULL] diff --git a/tests/queries/0_stateless/01664_array_slice_ubsan.reference b/tests/queries/0_stateless/01664_array_slice_ubsan.reference index beb31c4a02b..abae410d95c 100644 --- a/tests/queries/0_stateless/01664_array_slice_ubsan.reference +++ b/tests/queries/0_stateless/01664_array_slice_ubsan.reference @@ -1 +1 @@ -['\0','\0','\0'] +[0,0,0] diff --git a/tests/queries/0_stateless/02713_group_array_nullable.reference b/tests/queries/0_stateless/02713_group_array_nullable.reference deleted file mode 100644 index a163ccade3a..00000000000 --- a/tests/queries/0_stateless/02713_group_array_nullable.reference +++ /dev/null @@ -1,3 +0,0 @@ -[1] -[0,1,2,3,4,5,6,7,8,9] -[8,9] diff --git a/tests/queries/0_stateless/02713_group_array_nullable.sql b/tests/queries/0_stateless/02713_group_array_nullable.sql deleted file mode 100644 index a0cb3075252..00000000000 --- a/tests/queries/0_stateless/02713_group_array_nullable.sql +++ /dev/null @@ -1,5 +0,0 @@ -SET aggregate_functions_null_for_empty = 1; - -SELECT groupArray(1); -SELECT groupArray(number) FROM numbers(10); -SELECT groupArrayLast(2)(number) FROM numbers(10); From 612b79868b6a5e9f8c91ddf0caea6f72e97c0320 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 17 May 2023 20:40:51 +0000 Subject: [PATCH 0373/2223] test added --- .../0_stateless/02751_protobuf_ipv6.reference | 2 ++ tests/queries/0_stateless/02751_protobuf_ipv6.sh | 14 ++++++++++++++ .../format_schemas/02751_protobuf_ipv6.proto | 6 ++++++ 3 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02751_protobuf_ipv6.reference create mode 100755 tests/queries/0_stateless/02751_protobuf_ipv6.sh create mode 100644 tests/queries/0_stateless/format_schemas/02751_protobuf_ipv6.proto diff --git a/tests/queries/0_stateless/02751_protobuf_ipv6.reference b/tests/queries/0_stateless/02751_protobuf_ipv6.reference new file mode 100644 index 00000000000..0318b49c77e --- /dev/null +++ b/tests/queries/0_stateless/02751_protobuf_ipv6.reference @@ -0,0 +1,2 @@ +::ffff:1.2.3.4 +::ffff:1.2.3.4 diff --git a/tests/queries/0_stateless/02751_protobuf_ipv6.sh b/tests/queries/0_stateless/02751_protobuf_ipv6.sh new file mode 100755 index 00000000000..ecf565d9db4 --- /dev/null +++ b/tests/queries/0_stateless/02751_protobuf_ipv6.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SCHEMADIR=$CURDIR/format_schemas + + +echo 121a1000000000000000000000ffff01020304 | xxd -r -p | $CLICKHOUSE_LOCAL --input-format Protobuf --format_schema="$SCHEMADIR/02751_protobuf_ipv6:Message" --structure="ipv6_bytes IPv6" -q "select * from table" + +$CLICKHOUSE_LOCAL -q "select '::ffff:1.2.3.4'::IPv6 as ipv6_bytes format Protobuf settings format_schema = '$SCHEMADIR/02751_protobuf_ipv6:Message'" | $CLICKHOUSE_LOCAL --input-format Protobuf --format_schema="$SCHEMADIR/02751_protobuf_ipv6:Message" --structure="ipv6_bytes IPv6" -q "select * from table" + diff --git a/tests/queries/0_stateless/format_schemas/02751_protobuf_ipv6.proto b/tests/queries/0_stateless/format_schemas/02751_protobuf_ipv6.proto new file mode 100644 index 00000000000..8e6f115f2d7 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02751_protobuf_ipv6.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + +message Message +{ + bytes ipv6_bytes = 3; +} From 855c95f6268b1f31cdadb4b55b4b4ada5db7847a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 17 May 2023 22:46:09 +0200 Subject: [PATCH 0374/2223] Update src/Interpreters/Cache/Metadata.cpp Co-authored-by: Igor Nikonov <954088+devcrafter@users.noreply.github.com> --- src/Interpreters/Cache/Metadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index e0b82763a08..843ffd45b63 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -284,7 +284,7 @@ void CacheMetadata::doCleanup() { /// Key prefix directory can become non-empty just now, it is expected. if (e.code() == std::errc::directory_not_empty) - return; + continue; LOG_ERROR(log, "Error while removing key {}: {}", cleanup_key, getCurrentExceptionMessage(true)); chassert(false); } From 2893f14c54b4414261858fe45c39b9477dd55daf Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 18 May 2023 09:20:38 +0200 Subject: [PATCH 0375/2223] Updated test_ssl_cert_authentication to use urllib3 --- .../test_ssl_cert_authentication/test.py | 42 +++++++++---------- 1 file changed, 20 insertions(+), 22 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index ec7111099c9..727abc4f3b0 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -2,10 +2,10 @@ import pytest from helpers.client import Client from helpers.cluster import ClickHouseCluster from helpers.ssl_context import WrapSSLContextWithSNI -import urllib.request, urllib.parse import ssl import os.path from os import remove +import urllib3 # The test cluster is configured with certificate for that host name, see 'server-ext.cnf'. @@ -164,24 +164,22 @@ def get_ssl_context(cert_name): def execute_query_https( query, user, enable_ssl_auth=True, cert_name=None, password=None ): - retries = 10 - while True: - try: - url = f"https://{instance.ip_address}:{HTTPS_PORT}/?query={urllib.parse.quote(query)}" - request = urllib.request.Request(url) - request.add_header("X-ClickHouse-User", user) - if enable_ssl_auth: - request.add_header("X-ClickHouse-SSL-Certificate-Auth", "on") - if password: - request.add_header("X-ClickHouse-Key", password) - response = urllib.request.urlopen( - request, context=get_ssl_context(cert_name) - ).read() - return response.decode("utf-8") - except BrokenPipeError: - retries -= 1 - if retries == 0: - raise + try: + url = f"https://{instance.ip_address}:{HTTPS_PORT}/?query={query}" + headers = {"X-ClickHouse-User":user} + if enable_ssl_auth: + headers["X-ClickHouse-SSL-Certificate-Auth"] = "on" + if password: + headers["X-ClickHouse-Key"] = password + http_client=urllib3.PoolManager(ssl_context=get_ssl_context(cert_name)) + response = http_client.request('GET', url, headers=headers) + if response.status != 200: + raise Exception(response.status) + return response.data.decode("utf-8") + except: + raise + + def test_https(): @@ -203,7 +201,7 @@ def test_https_wrong_cert(): # Wrong certificate: different user's certificate with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="john", cert_name="client2") - assert "HTTP Error 403" in str(err.value) + assert "403" in str(err.value) # Wrong certificate: self-signed certificate. with pytest.raises(Exception) as err: @@ -213,7 +211,7 @@ def test_https_wrong_cert(): # No certificate. with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="john") - assert "HTTP Error 403" in str(err.value) + assert "403" in str(err.value) # No header enabling SSL authentication. with pytest.raises(Exception) as err: @@ -341,7 +339,7 @@ def test_create_user(): with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="emma", cert_name="client3") - assert "HTTP Error 403" in str(err.value) + assert "403" in str(err.value) assert ( instance.query("SHOW CREATE USER lucy") From bf6afd27a655bb159583af2020113e101ca82e00 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 17 May 2023 13:32:51 +0000 Subject: [PATCH 0376/2223] build with disks --- programs/keeper-converter/KeeperConverter.cpp | 4 +- programs/keeper/CMakeLists.txt | 58 +++- programs/keeper/Keeper.cpp | 13 + programs/server/config.d/users.xml | 26 ++ src/Coordination/Context/Context.cpp | 259 ++++++++++++++++++ src/Coordination/Context/Context.h | 111 ++++++++ src/Coordination/Context/Settings.cpp | 24 ++ src/Coordination/Context/ThreadStatusExt.cpp | 14 + src/Coordination/CoordinationSettings.cpp | 60 ---- src/Coordination/CoordinationSettings.h | 9 - src/Coordination/KeeperContext.cpp | 104 +++++++ src/Coordination/KeeperContext.h | 37 ++- src/Coordination/KeeperDispatcher.cpp | 6 +- src/Coordination/KeeperServer.cpp | 33 ++- src/Coordination/KeeperSnapshotManager.cpp | 14 +- src/Coordination/KeeperStateMachine.cpp | 6 +- src/Coordination/KeeperStorage.cpp | 46 ++-- src/Coordination/ZooKeeperDataReader.cpp | 4 +- src/Coordination/pathUtils.cpp | 4 +- src/Coordination/pathUtils.h | 4 +- src/Coordination/tests/gtest_coordination.cpp | 2 +- src/Core/SettingsFields.cpp | 29 ++ src/Core/SettingsFields.h | 16 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 + src/Disks/IO/getThreadPoolReader.cpp | 24 -- .../ObjectStorages/DiskObjectStorage.cpp | 19 -- src/Disks/ObjectStorages/DiskObjectStorage.h | 14 +- .../ObjectStorages/DiskObjectStorageCache.cpp | 28 ++ src/Disks/registerDisks.cpp | 17 ++ src/Interpreters/Context.h | 8 + 30 files changed, 807 insertions(+), 188 deletions(-) create mode 100644 programs/server/config.d/users.xml create mode 100644 src/Coordination/Context/Context.cpp create mode 100644 src/Coordination/Context/Context.h create mode 100644 src/Coordination/Context/Settings.cpp create mode 100644 src/Coordination/Context/ThreadStatusExt.cpp create mode 100644 src/Coordination/KeeperContext.cpp create mode 100644 src/Disks/ObjectStorages/DiskObjectStorageCache.cpp diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index c81e61685fd..58d090ca8b9 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -40,8 +40,8 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) try { - auto keeper_context = std::make_shared(); - keeper_context->digest_enabled = true; + auto keeper_context = std::make_shared(true); + keeper_context->setDigestEnabled(true); DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index e5d56023f7b..c0c0a6dd1b0 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -46,6 +46,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManagerS3.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperContext.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStorage.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperAsynchronousMetrics.cpp @@ -58,10 +59,14 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/SettingsFields.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/BaseSettings.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/ServerSettings.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/Field.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/SettingsEnums.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/ServerUUID.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/UUID.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/BackgroundSchedulePool.cpp + + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/IO/ReadBuffer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperTCPHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp @@ -92,6 +97,10 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/ICompressionCodec.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/LZ4_decompress_faster.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/CurrentThread.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/NamedCollections/NamedCollections.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/NamedCollections/NamedCollectionConfiguration.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/IKeeper.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/TestKeeper.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -102,11 +111,56 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/ZooKeeperLock.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/ZooKeeperNodeCache.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/registerDisks.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IDisk.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskFactory.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskSelector.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskLocal.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskLocalCheckThread.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/LocalDirectorySyncGuard.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/TemporaryFileOnDisk.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/loadLocalDiskConfig.cpp + + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/IObjectStorage.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataFromDiskTransactionState.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorage.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp + + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/registerDiskS3.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/S3Capabilities.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/diskSettings.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/ProxyListConfiguration.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp + + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/createReadBufferFromFileBase.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadBufferFromRemoteFSGather.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/IOUringReader.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferFromTemporaryFile.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/getThreadPoolReader.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolRemoteFSReader.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolReader.cpp + + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Storages/StorageS3Settings.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/BaseDaemon.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/SentryWriter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/GraphiteWriter.cpp ${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/Context.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/Settings.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/ThreadStatusExt.cpp + Keeper.cpp clickhouse-keeper.cpp ) @@ -117,10 +171,6 @@ if (BUILD_STANDALONE_KEEPER) target_compile_definitions (clickhouse-keeper PRIVATE -DCLICKHOUSE_PROGRAM_STANDALONE_BUILD) target_compile_definitions (clickhouse-keeper PUBLIC -DWITHOUT_TEXT_LOG) - target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/../../src") # uses includes from src directory - target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src/Core/include") # uses some includes from core - target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src") # uses some includes from common - target_link_libraries(clickhouse-keeper PRIVATE ch_contrib::abseil_swiss_tables diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 3d1773260f5..996c4678450 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -21,6 +21,8 @@ #include #include +#include + #include #include @@ -40,6 +42,8 @@ #include #include +#include + int mainEntryClickHouseKeeper(int argc, char ** argv) { @@ -408,6 +412,15 @@ try std::mutex servers_lock; auto servers = std::make_shared>(); + auto shared_context = Context::createShared(); + auto global_context = Context::createGlobal(shared_context.get()); + + global_context->makeGlobalContext(); + global_context->setPath(path); + global_context->setRemoteHostFilter(config()); + + registerDisks(/*global_skip_access_check=*/false); + tiny_context = std::make_shared(); /// This object will periodically calculate some metrics. KeeperAsynchronousMetrics async_metrics( diff --git a/programs/server/config.d/users.xml b/programs/server/config.d/users.xml new file mode 100644 index 00000000000..d8a62b45baa --- /dev/null +++ b/programs/server/config.d/users.xml @@ -0,0 +1,26 @@ + + + + 10000000000 + 0 + 2 + + + 5000000000 + 20000000000 + + + + + + + + + + + + + default + + + diff --git a/src/Coordination/Context/Context.cpp b/src/Coordination/Context/Context.cpp new file mode 100644 index 00000000000..aeb4e405938 --- /dev/null +++ b/src/Coordination/Context/Context.cpp @@ -0,0 +1,259 @@ +#include + +#include +#include +#include + +#include + +#include + +#include +#include + +namespace ProfileEvents +{ + extern const Event ContextLock; +} + +namespace CurrentMetrics +{ + extern const Metric ContextLockWait; + extern const Metric BackgroundSchedulePoolTask; + extern const Metric BackgroundSchedulePoolSize; + extern const Metric IOWriterThreads; + extern const Metric IOWriterThreadsActive; +} + +namespace DB +{ + + +struct ContextSharedPart : boost::noncopyable +{ + ContextSharedPart() + : macros(std::make_unique()) + {} + + /// For access of most of shared objects. Recursive mutex. + mutable std::recursive_mutex mutex; + + ServerSettings server_settings; + + String path; /// Path to the data directory, with a slash at the end. + ConfigurationPtr config; /// Global configuration settings. + MultiVersion macros; /// Substitutions extracted from config. + mutable std::unique_ptr schedule_pool; /// A thread pool that can run different jobs in background + RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml + /// + mutable std::unique_ptr asynchronous_remote_fs_reader; + mutable std::unique_ptr asynchronous_local_fs_reader; + mutable std::unique_ptr synchronous_local_fs_reader; + + mutable std::unique_ptr threadpool_writer; + + mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads + mutable ThrottlerPtr remote_write_throttler; /// A server-wide throttler for remote IO writes + + mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads + mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes + +}; + +Context::Context() = default; +Context::~Context() = default; +Context::Context(const Context &) = default; +Context & Context::operator=(const Context &) = default; + +SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default; +SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) noexcept = default; +SharedContextHolder::SharedContextHolder() = default; +SharedContextHolder::~SharedContextHolder() = default; +SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) + : shared(std::move(shared_context)) {} + +void SharedContextHolder::reset() { shared.reset(); } + +void Context::makeGlobalContext() +{ + initGlobal(); + global_context = shared_from_this(); +} + +ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) +{ + auto res = std::shared_ptr(new Context); + res->shared = shared; + return res; +} + +void Context::initGlobal() +{ + assert(!global_context_instance); + global_context_instance = shared_from_this(); +} + +SharedContextHolder Context::createShared() +{ + return SharedContextHolder(std::make_unique()); +} + +ContextMutablePtr Context::getGlobalContext() const +{ + auto ptr = global_context.lock(); + if (!ptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no global context or global context has expired"); + return ptr; +} + +std::unique_lock Context::getLock() const +{ + ProfileEvents::increment(ProfileEvents::ContextLock); + CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; + return std::unique_lock(shared->mutex); +} + +String Context::getPath() const +{ + auto lock = getLock(); + return shared->path; +} + +void Context::setPath(const String & path) +{ + auto lock = getLock(); + shared->path = path; +} + +MultiVersion::Version Context::getMacros() const +{ + return shared->macros.get(); +} + +BackgroundSchedulePool & Context::getSchedulePool() const +{ + auto lock = getLock(); + if (!shared->schedule_pool) + { + shared->schedule_pool = std::make_unique( + shared->server_settings.background_schedule_pool_size, + CurrentMetrics::BackgroundSchedulePoolTask, + CurrentMetrics::BackgroundSchedulePoolSize, + "BgSchPool"); + } + + return *shared->schedule_pool; +} + +void Context::setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config) +{ + shared->remote_host_filter.setValuesFromConfig(config); +} + +const RemoteHostFilter & Context::getRemoteHostFilter() const +{ + return shared->remote_host_filter; +} + +IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const +{ + auto lock = getLock(); + + switch (type) + { + case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER: + { + if (!shared->asynchronous_remote_fs_reader) + shared->asynchronous_remote_fs_reader = createThreadPoolReader(type, getConfigRef()); + return *shared->asynchronous_remote_fs_reader; + } + case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER: + { + if (!shared->asynchronous_local_fs_reader) + shared->asynchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef()); + + return *shared->asynchronous_local_fs_reader; + } + case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER: + { + if (!shared->synchronous_local_fs_reader) + shared->synchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef()); + + return *shared->synchronous_local_fs_reader; + } + } +} + +std::shared_ptr Context::getFilesystemCacheLog() const +{ + return nullptr; +} + +std::shared_ptr Context::getFilesystemReadPrefetchesLog() const +{ + return nullptr; +} + +void Context::setConfig(const ConfigurationPtr & config) +{ + auto lock = getLock(); + shared->config = config; +} + +const Poco::Util::AbstractConfiguration & Context::getConfigRef() const +{ + auto lock = getLock(); + return shared->config ? *shared->config : Poco::Util::Application::instance().config(); +} + +std::shared_ptr Context::getAsyncReadCounters() const +{ + auto lock = getLock(); + if (!async_read_counters) + async_read_counters = std::make_shared(); + return async_read_counters; +} + +ThreadPool & Context::getThreadPoolWriter() const +{ + const auto & config = getConfigRef(); + + auto lock = getLock(); + + if (!shared->threadpool_writer) + { + auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); + auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); + + shared->threadpool_writer = std::make_unique( + CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, pool_size, pool_size, queue_size); + } + + return *shared->threadpool_writer; +} + +ThrottlerPtr Context::getRemoteReadThrottler() const +{ + return nullptr; +} + +ThrottlerPtr Context::getRemoteWriteThrottler() const +{ + return nullptr; +} + +ThrottlerPtr Context::getLocalReadThrottler() const +{ + return nullptr; +} + +ThrottlerPtr Context::getLocalWriteThrottler() const +{ + return nullptr; +} + +ReadSettings Context::getReadSettings() const +{ + return ReadSettings{}; +} + +} diff --git a/src/Coordination/Context/Context.h b/src/Coordination/Context/Context.h new file mode 100644 index 00000000000..683209d942e --- /dev/null +++ b/src/Coordination/Context/Context.h @@ -0,0 +1,111 @@ +#pragma once + +#include + +#include +#include + +#include + +#include +#include + +#include + +#include + +#include + +namespace DB +{ + +struct ContextSharedPart; +class Macros; +class FilesystemCacheLog; +class FilesystemReadPrefetchesLog; + +/// A small class which owns ContextShared. +/// We don't use something like unique_ptr directly to allow ContextShared type to be incomplete. +struct SharedContextHolder +{ + ~SharedContextHolder(); + SharedContextHolder(); + explicit SharedContextHolder(std::unique_ptr shared_context); + SharedContextHolder(SharedContextHolder &&) noexcept; + + SharedContextHolder & operator=(SharedContextHolder &&) noexcept; + + ContextSharedPart * get() const { return shared.get(); } + void reset(); +private: + std::unique_ptr shared; +}; + + +class Context : public std::enable_shared_from_this +{ +private: + /// Use copy constructor or createGlobal() instead + Context(); + Context(const Context &); + Context & operator=(const Context &); + + std::unique_lock getLock() const; + + ContextWeakMutablePtr global_context; + inline static ContextPtr global_context_instance; + ContextSharedPart * shared; + + /// Query metrics for reading data asynchronously with IAsynchronousReader. + mutable std::shared_ptr async_read_counters; + + Settings settings; /// Setting for query execution. +public: + /// Create initial Context with ContextShared and etc. + static ContextMutablePtr createGlobal(ContextSharedPart * shared); + static SharedContextHolder createShared(); + + ContextMutablePtr getGlobalContext() const; + static ContextPtr getGlobalContextInstance() { return global_context_instance; } + + void makeGlobalContext(); + void initGlobal(); + + ~Context(); + + using ConfigurationPtr = Poco::AutoPtr; + + /// Global application configuration settings. + void setConfig(const ConfigurationPtr & config); + const Poco::Util::AbstractConfiguration & getConfigRef() const; + + const Settings & getSettingsRef() const { return settings; } + + String getPath() const; + void setPath(const String & path); + + MultiVersion::Version getMacros() const; + + BackgroundSchedulePool & getSchedulePool() const; + + /// Storage of allowed hosts from config.xml + void setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config); + const RemoteHostFilter & getRemoteHostFilter() const; + + std::shared_ptr getFilesystemCacheLog() const; + std::shared_ptr getFilesystemReadPrefetchesLog() const; + + IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; + std::shared_ptr getAsyncReadCounters() const; + ThreadPool & getThreadPoolWriter() const; + + ThrottlerPtr getRemoteReadThrottler() const; + ThrottlerPtr getRemoteWriteThrottler() const; + + ThrottlerPtr getLocalReadThrottler() const; + ThrottlerPtr getLocalWriteThrottler() const; + + ReadSettings getReadSettings() const; +}; + +} diff --git a/src/Coordination/Context/Settings.cpp b/src/Coordination/Context/Settings.cpp new file mode 100644 index 00000000000..12a7a42ffac --- /dev/null +++ b/src/Coordination/Context/Settings.cpp @@ -0,0 +1,24 @@ +#include + +namespace DB +{ + +IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS) + +std::vector Settings::getAllRegisteredNames() const +{ + std::vector all_settings; + for (const auto & setting_field : all()) + { + all_settings.push_back(setting_field.getName()); + } + return all_settings; +} + +void Settings::set(std::string_view name, const Field & value) +{ + BaseSettings::set(name, value); +} + + +} diff --git a/src/Coordination/Context/ThreadStatusExt.cpp b/src/Coordination/Context/ThreadStatusExt.cpp new file mode 100644 index 00000000000..97f7287be8c --- /dev/null +++ b/src/Coordination/Context/ThreadStatusExt.cpp @@ -0,0 +1,14 @@ +#include + +namespace DB +{ + +void CurrentThread::detachFromGroupIfNotDetached() +{ +} + +void CurrentThread::attachToGroup(const ThreadGroupPtr &) +{ +} + +} diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 7a66134f43f..833ff7be7ae 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -85,14 +85,6 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const writeText(four_letter_word_allow_list, buf); buf.write('\n'); - writeText("log_storage_path=", buf); - writeText(log_storage_path, buf); - buf.write('\n'); - - writeText("snapshot_storage_path=", buf); - writeText(snapshot_storage_path, buf); - buf.write('\n'); - /// coordination_settings writeText("max_requests_batch_size=", buf); @@ -188,61 +180,9 @@ KeeperConfigurationAndSettings::loadFromConfig(const Poco::Util::AbstractConfigu DEFAULT_FOUR_LETTER_WORD_CMD)); - ret->log_storage_path = getLogsPathFromConfig(config, standalone_keeper_); - ret->snapshot_storage_path = getSnapshotsPathFromConfig(config, standalone_keeper_); - - ret->state_file_path = getStateFilePathFromConfig(config, standalone_keeper_); - ret->coordination_settings->loadFromConfig("keeper_server.coordination_settings", config); return ret; } -String KeeperConfigurationAndSettings::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) -{ - /// the most specialized path - if (config.has("keeper_server.log_storage_path")) - return config.getString("keeper_server.log_storage_path"); - - if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs"; - - if (standalone_keeper_) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs"; - else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs"; -} - -String KeeperConfigurationAndSettings::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) -{ - /// the most specialized path - if (config.has("keeper_server.snapshot_storage_path")) - return config.getString("keeper_server.snapshot_storage_path"); - - if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots"; - - if (standalone_keeper_) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots"; - else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots"; -} - -String KeeperConfigurationAndSettings::getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) -{ - if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "state"; - - if (config.has("keeper_server.snapshot_storage_path")) - return std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path() / "state"; - - if (config.has("keeper_server.log_storage_path")) - return std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path() / "state"; - - if (standalone_keeper_) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "state"; - else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/state"; -} - } diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 90f66ccc09b..16da8404b44 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -81,17 +81,8 @@ struct KeeperConfigurationAndSettings bool standalone_keeper; CoordinationSettingsPtr coordination_settings; - String log_storage_path; - String snapshot_storage_path; - String state_file_path; - void dump(WriteBufferFromOwnString & buf) const; static std::shared_ptr loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); - -private: - static String getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); - static String getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); - static String getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); }; using KeeperConfigurationAndSettingsPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp new file mode 100644 index 00000000000..9e504f5aa07 --- /dev/null +++ b/src/Coordination/KeeperContext.cpp @@ -0,0 +1,104 @@ +#include + +#include +#include +#include + +namespace DB +{ + + +KeeperContext::KeeperContext(bool standalone_keeper_) + : disk_selector(std::make_shared()) + , standalone_keeper(standalone_keeper_) +{} + +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +{ + digest_enabled = config.getBool("keeper_server.digest_enabled", false); + ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); + + disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance()); + + log_storage_path = getLogsPathFromConfig(config); + snapshot_storage_path = getSnapshotsPathFromConfig(config); + + state_file_path = getStateFilePathFromConfig(config); +} + +KeeperContext::Phase KeeperContext::getServerState() const +{ + return server_state; +} + +void KeeperContext::setServerState(KeeperContext::Phase server_state_) +{ + server_state = server_state_; +} + +bool KeeperContext::ignoreSystemPathOnStartup() const +{ + return ignore_system_path_on_startup; +} + +bool KeeperContext::digestEnabled() const +{ + return digest_enabled; +} + +void KeeperContext::setDigestEnabled(bool digest_enabled_) +{ + digest_enabled = digest_enabled_; +} + +KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const +{ + /// the most specialized path + if (config.has("keeper_server.log_storage_path")) + return std::make_shared("LogDisk", config.getString("keeper_server.log_storage_path"), 0); + + if (config.has("keeper_server.log_storage_disk")) + return config.getString("keeper_server.log_storage_disk"); + + if (config.has("keeper_server.storage_path")) + return std::make_shared("LogDisk", std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs", 0); + + if (standalone_keeper) + return std::make_shared("LogDisk", std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs", 0); + else + return std::make_shared("LogDisk", std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs", 0); +} + +std::string KeeperContext::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + /// the most specialized path + if (config.has("keeper_server.snapshot_storage_path")) + return config.getString("keeper_server.snapshot_storage_path"); + + if (config.has("keeper_server.storage_path")) + return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots"; + + if (standalone_keeper) + return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots"; + else + return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots"; +} + +std::string KeeperContext::getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + if (config.has("keeper_server.storage_path")) + return std::filesystem::path{config.getString("keeper_server.storage_path")} / "state"; + + if (config.has("keeper_server.snapshot_storage_path")) + return std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path() / "state"; + + if (config.has("keeper_server.log_storage_path")) + return std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path() / "state"; + + if (standalone_keeper) + return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "state"; + else + return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/state"; +} + +} diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 64fa8cea6ec..2a215f9d58f 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,10 +1,20 @@ #pragma once +#include + +#include + +#include +#include + namespace DB { -struct KeeperContext +class KeeperContext { +public: + explicit KeeperContext(bool standalone_keeper_); + enum class Phase : uint8_t { INIT, @@ -12,10 +22,35 @@ struct KeeperContext SHUTDOWN }; + void initialize(const Poco::Util::AbstractConfiguration & config); + + Phase getServerState() const; + void setServerState(Phase server_state_); + + bool ignoreSystemPathOnStartup() const; + + bool digestEnabled() const; + void setDigestEnabled(bool digest_enabled_); +private: + /// local disk defined using path or disk name + using Storage = std::variant; + + Storage getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const; + std::string getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config); + std::string getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config); + Phase server_state{Phase::INIT}; bool ignore_system_path_on_startup{false}; bool digest_enabled{true}; + + std::shared_ptr disk_selector; + + Storage log_storage_path; + Storage snapshot_storage_path; + Storage state_file_path; + + bool standalone_keeper; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index d64134f3024..6632e58782f 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -789,12 +789,14 @@ static uint64_t getDirSize(const fs::path & dir) uint64_t KeeperDispatcher::getLogDirSize() const { - return getDirSize(configuration_and_settings->log_storage_path); + //return getDirSize(configuration_and_settings->log_storage_path); + return 0; } uint64_t KeeperDispatcher::getSnapDirSize() const { - return getDirSize(configuration_and_settings->snapshot_storage_path); + //return getDirSize(configuration_and_settings->snapshot_storage_path); + return 0; } Keeper4LWInfo KeeperDispatcher::getKeeper4LWInfo() const diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 9ca792eecdd..d63593436f4 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -114,18 +114,17 @@ KeeperServer::KeeperServer( , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) , is_recovering(config.getBool("keeper_server.force_recovery", false)) - , keeper_context{std::make_shared()} + , keeper_context{std::make_shared(true)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); - keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); - keeper_context->ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); + keeper_context->initialize(config); - if (!fs::exists(configuration_and_settings_->snapshot_storage_path)) - fs::create_directories(configuration_and_settings_->snapshot_storage_path); - auto snapshots_disk = std::make_shared("Keeper-snapshots", configuration_and_settings_->snapshot_storage_path, 0); + //if (!fs::exists(keeper_context->snapshot_storage_path)) + // fs::create_directories(keeper_context->snapshot_storage_path); + auto snapshots_disk = std::make_shared("Keeper-snapshots", "", 0); state_machine = nuraft::cs_new( responses_queue_, @@ -137,23 +136,23 @@ KeeperServer::KeeperServer( commit_callback, checkAndGetSuperdigest(configuration_and_settings_->super_digest)); - auto state_path = fs::path(configuration_and_settings_->state_file_path).parent_path().generic_string(); - auto state_file_name = fs::path(configuration_and_settings_->state_file_path).filename().generic_string(); + //auto state_path = fs::path(keeper_context->state_file_path).parent_path().generic_string(); + //auto state_file_name = fs::path(configuration_and_settings_->state_file_path).filename().generic_string(); - if (!fs::exists(state_path)) - fs::create_directories(state_path); - auto state_disk = std::make_shared("Keeper-state", state_path, 0); + //if (!fs::exists(state_path)) + // fs::create_directories(state_path); + auto state_disk = std::make_shared("Keeper-state", "", 0); - if (!fs::exists(configuration_and_settings_->log_storage_path)) - fs::create_directories(configuration_and_settings_->log_storage_path); - auto logs_disk = std::make_shared("Keeper-logs", configuration_and_settings_->log_storage_path, 0); + //if (!fs::exists(configuration_and_settings_->log_storage_path)) + // fs::create_directories(configuration_and_settings_->log_storage_path); + auto logs_disk = std::make_shared("Keeper-logs", "", 0); state_manager = nuraft::cs_new( server_id, "keeper_server", logs_disk, state_disk, - state_file_name, + "state", config, coordination_settings); } @@ -431,7 +430,7 @@ void KeeperServer::startup(const Poco::Util::AbstractConfiguration & config, boo launchRaftServer(config, enable_ipv6); - keeper_context->server_state = KeeperContext::Phase::RUNNING; + keeper_context->setServerState(KeeperContext::Phase::RUNNING); } void KeeperServer::shutdownRaftServer() @@ -446,7 +445,7 @@ void KeeperServer::shutdownRaftServer() raft_instance->shutdown(); - keeper_context->server_state = KeeperContext::Phase::SHUTDOWN; + keeper_context->setServerState(KeeperContext::Phase::SHUTDOWN); if (create_snapshot_on_exit) raft_instance->create_snapshot(); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index b7fca5c1eab..e1c0c034cff 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -9,11 +9,11 @@ #include #include #include -#include #include #include #include #include +#include #include #include #include @@ -157,7 +157,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr if (snapshot.version >= SnapshotVersion::V5) { writeBinary(snapshot.zxid, out); - if (keeper_context->digest_enabled) + if (keeper_context->digestEnabled()) { writeBinary(static_cast(KeeperStorage::CURRENT_DIGEST_VERSION), out); writeBinary(snapshot.nodes_digest, out); @@ -268,7 +268,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial deserialization_result.snapshot_meta = deserializeSnapshotMetadata(in); KeeperStorage & storage = *deserialization_result.storage; - bool recalculate_digest = keeper_context->digest_enabled; + bool recalculate_digest = keeper_context->digestEnabled(); if (version >= SnapshotVersion::V5) { readBinary(storage.zxid, in); @@ -350,7 +350,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial const std::string error_msg = fmt::format("Cannot read node on path {} from a snapshot because it is used as a system node", path); if (match_result == IS_CHILD) { - if (keeper_context->ignore_system_path_on_startup || keeper_context->server_state != KeeperContext::Phase::INIT) + if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT) { LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); continue; @@ -366,7 +366,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { if (!is_node_empty(node)) { - if (keeper_context->ignore_system_path_on_startup || keeper_context->server_state != KeeperContext::Phase::INIT) + if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT) { LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); node = KeeperStorage::Node{}; @@ -395,9 +395,9 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { if (itr.key != "/") { - auto parent_path = parentPath(itr.key); + auto parent_path = parentNodePath(itr.key); storage.container.updateValue( - parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseName(path), /*update_size*/ version < SnapshotVersion::V4); }); + parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); }); } } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 53657d9b0b9..a7c845e5017 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -223,7 +223,7 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req throw; } - if (keeper_context->digest_enabled && request_for_session.digest) + if (keeper_context->digestEnabled() && request_for_session.digest) assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false); return true; @@ -271,7 +271,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response_for_session.session_id); } - if (keeper_context->digest_enabled && request_for_session.digest) + if (keeper_context->digestEnabled() && request_for_session.digest) assertDigest(*request_for_session.digest, storage->getNodesDigest(true), *request_for_session.request, true); } @@ -429,7 +429,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res }; - if (keeper_context->server_state == KeeperContext::Phase::SHUTDOWN) + if (keeper_context->getServerState() == KeeperContext::Phase::SHUTDOWN) { LOG_INFO(log, "Creating a snapshot during shutdown because 'create_snapshot_on_exit' is enabled."); auto snapshot_path = snapshot_task.create_snapshot(std::move(snapshot_task.snapshot)); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 7a1a5e42632..64e785693e4 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -128,7 +128,7 @@ KeeperStorage::ResponsesForSessions processWatchesImpl( watches.erase(watch_it); } - auto parent_path = parentPath(path); + auto parent_path = parentNodePath(path); Strings paths_to_check_for_list_watches; if (event_type == Coordination::Event::CREATED) @@ -276,7 +276,7 @@ void KeeperStorage::initializeSystemNodes() [](auto & node) { ++node.stat.numChildren; - node.addChild(getBaseName(keeper_system_path)); + node.addChild(getBaseNodeName(keeper_system_path)); } ); addDigest(updated_root_it->value, "/"); @@ -290,9 +290,9 @@ void KeeperStorage::initializeSystemNodes() child_system_node.setData(data); auto [map_key, _] = container.insert(std::string{path}, child_system_node); /// Take child path from key owned by map. - auto child_path = getBaseName(map_key->getKey()); + auto child_path = getBaseNodeName(map_key->getKey()); container.updateValue( - parentPath(StringRef(path)), + parentNodePath(StringRef(path)), [child_path](auto & parent) { // don't update stats so digest is okay @@ -705,7 +705,7 @@ bool KeeperStorage::createNode( bool is_sequental, Coordination::ACLs node_acls) { - auto parent_path = parentPath(path); + auto parent_path = parentNodePath(path); auto node_it = container.find(parent_path); if (node_it == container.end()) @@ -728,7 +728,7 @@ bool KeeperStorage::createNode( created_node.is_sequental = is_sequental; auto [map_key, _] = container.insert(path, created_node); /// Take child path from key owned by map. - auto child_path = getBaseName(map_key->getKey()); + auto child_path = getBaseNodeName(map_key->getKey()); container.updateValue( parent_path, [child_path](KeeperStorage::Node & parent) @@ -758,8 +758,8 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t version) acl_map.removeUsage(prev_node.acl_id); container.updateValue( - parentPath(path), - [child_basename = getBaseName(node_it->key)](KeeperStorage::Node & parent) + parentNodePath(path), + [child_basename = getBaseNodeName(node_it->key)](KeeperStorage::Node & parent) { parent.removeChild(child_basename); chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); @@ -843,7 +843,7 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_ void handleSystemNodeModification(const KeeperContext & keeper_context, std::string_view error_msg) { - if (keeper_context.server_state == KeeperContext::Phase::INIT && !keeper_context.ignore_system_path_on_startup) + if (keeper_context.getServerState() == KeeperContext::Phase::INIT && !keeper_context.ignoreSystemPathOnStartup()) throw Exception( ErrorCodes::LOGICAL_ERROR, "{}. Ignoring it can lead to data loss. " @@ -906,7 +906,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override { auto path = zk_request->getPath(); - return storage.checkACL(parentPath(path), Coordination::ACL::Create, session_id, is_local); + return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local); } std::vector @@ -917,7 +917,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr std::vector new_deltas; - auto parent_path = parentPath(request.path); + auto parent_path = parentNodePath(request.path); auto parent_node = storage.uncommitted_state.getNode(parent_path); if (parent_node == nullptr) return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; @@ -948,7 +948,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr if (storage.uncommitted_state.getNode(path_created)) return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; - if (getBaseName(path_created).size == 0) + if (getBaseNodeName(path_created).size == 0) return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; Coordination::ACLs node_acls; @@ -1098,7 +1098,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr { bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(parentPath(zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); + return storage.checkACL(parentNodePath(zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); } using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; @@ -1120,7 +1120,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr const auto update_parent_pzxid = [&]() { - auto parent_path = parentPath(request.path); + auto parent_path = parentNodePath(request.path); if (!storage.uncommitted_state.getNode(parent_path)) return; @@ -1155,7 +1155,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr update_parent_pzxid(); new_deltas.emplace_back( - std::string{parentPath(request.path)}, + std::string{parentNodePath(request.path)}, zxid, KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent) { @@ -1298,7 +1298,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce request.version}); new_deltas.emplace_back( - parentPath(request.path).toString(), + parentNodePath(request.path).toString(), zxid, KeeperStorage::UpdateNodeDelta { @@ -1458,7 +1458,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override { auto path = zk_request->getPath(); - return storage.checkACL(check_not_exists ? parentPath(path) : path, Coordination::ACL::Read, session_id, is_local); + return storage.checkACL(check_not_exists ? parentNodePath(path) : path, Coordination::ACL::Read, session_id, is_local); } std::vector @@ -2001,7 +2001,7 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vector & new_deltas) const { - if (!keeper_context->digest_enabled) + if (!keeper_context->digestEnabled()) return current_digest; std::unordered_map> updated_nodes; @@ -2099,7 +2099,7 @@ void KeeperStorage::preprocessRequest( TransactionInfo transaction{.zxid = new_last_zxid}; uint64_t new_digest = getNodesDigest(false).value; SCOPE_EXIT({ - if (keeper_context->digest_enabled) + if (keeper_context->digestEnabled()) // if the version of digest we got from the leader is the same as the one this instances has, we can simply copy the value // and just check the digest on the commit // a mistake can happen while applying the changes to the uncommitted_state so for now let's just recalculate the digest here also @@ -2122,7 +2122,7 @@ void KeeperStorage::preprocessRequest( { new_deltas.emplace_back ( - parentPath(ephemeral_path).toString(), + parentNodePath(ephemeral_path).toString(), new_last_zxid, UpdateNodeDelta { @@ -2315,7 +2315,7 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const { - if (!keeper_context->digest_enabled) + if (!keeper_context->digestEnabled()) return {.version = DigestVersion::NO_DIGEST}; if (committed || uncommitted_transactions.empty()) @@ -2326,13 +2326,13 @@ KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const void KeeperStorage::removeDigest(const Node & node, const std::string_view path) { - if (keeper_context->digest_enabled) + if (keeper_context->digestEnabled()) nodes_digest -= node.getDigest(path); } void KeeperStorage::addDigest(const Node & node, const std::string_view path) { - if (keeper_context->digest_enabled) + if (keeper_context->digestEnabled()) { node.invalidateDigestCache(); nodes_digest += node.getDigest(path); diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index bdc462f3ea0..94fc07bcc4a 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -139,8 +139,8 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L { if (itr.key != "/") { - auto parent_path = parentPath(itr.key); - storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseName(my_path)); ++value.stat.numChildren; }); + auto parent_path = parentNodePath(itr.key); + storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); ++value.stat.numChildren; }); } } diff --git a/src/Coordination/pathUtils.cpp b/src/Coordination/pathUtils.cpp index 1e1da339d2e..afa42b4a639 100644 --- a/src/Coordination/pathUtils.cpp +++ b/src/Coordination/pathUtils.cpp @@ -21,7 +21,7 @@ static size_t findLastSlash(StringRef path) return std::string::npos; } -StringRef parentPath(StringRef path) +StringRef parentNodePath(StringRef path) { auto rslash_pos = findLastSlash(path); if (rslash_pos > 0) @@ -29,7 +29,7 @@ StringRef parentPath(StringRef path) return "/"; } -StringRef getBaseName(StringRef path) +StringRef getBaseNodeName(StringRef path) { size_t basename_start = findLastSlash(path); return StringRef{path.data + basename_start + 1, path.size - basename_start - 1}; diff --git a/src/Coordination/pathUtils.h b/src/Coordination/pathUtils.h index 69ed2d8b177..b2b79b14110 100644 --- a/src/Coordination/pathUtils.h +++ b/src/Coordination/pathUtils.h @@ -6,8 +6,8 @@ namespace DB { -StringRef parentPath(StringRef path); +StringRef parentNodePath(StringRef path); -StringRef getBaseName(StringRef path); +StringRef getBaseNodeName(StringRef path); } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 62217fb2dd3..de5f2da262b 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2366,7 +2366,7 @@ TEST_P(CoordinationTest, TestSystemNodeModify) int64_t zxid{0}; // On INIT we abort when a system path is modified - keeper_context->server_state = KeeperContext::Phase::RUNNING; + keeper_context->setServerState(KeeperContext::Phase::RUNNING); KeeperStorage storage{500, "", keeper_context}; const auto assert_create = [&](const std::string_view path, const auto expected_code) { diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 8cd4efb68c6..c1f9fa00f2a 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -379,6 +379,35 @@ void SettingFieldMap::readBinary(ReadBuffer & in) *this = map; } +#else + +SettingFieldMap::SettingFieldMap(const Field &) : value(Map()) {} +String SettingFieldMap::toString() const +{ + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); +} + + +SettingFieldMap & SettingFieldMap::operator =(const Field &) +{ + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); +} + +void SettingFieldMap::parseFromString(const String &) +{ + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); +} + +void SettingFieldMap::writeBinary(WriteBuffer &) const +{ + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); +} + +void SettingFieldMap::readBinary(ReadBuffer &) +{ + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); +} + #endif namespace diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 2cd55e6b4c5..8cbce71b094 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -245,6 +245,12 @@ struct SettingFieldString void readBinary(ReadBuffer & in); }; +#ifdef CLICKHOUSE_PROGRAM_STANDALONE_BUILD +#define NORETURN [[noreturn]] +#else +#define NORETURN +#endif + struct SettingFieldMap { public: @@ -261,13 +267,15 @@ public: operator const Map &() const { return value; } /// NOLINT explicit operator Field() const { return value; } - String toString() const; - void parseFromString(const String & str); + NORETURN String toString() const; + NORETURN void parseFromString(const String & str); - void writeBinary(WriteBuffer & out) const; - void readBinary(ReadBuffer & in); + NORETURN void writeBinary(WriteBuffer & out) const; + NORETURN void readBinary(ReadBuffer & in); }; +#undef NORETURN + struct SettingFieldChar { public: diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 68b5a9c9d96..6c3556580a2 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -50,6 +50,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c size_t current_read_until_position = read_until_position ? read_until_position : object.bytes_size; auto current_read_buffer_creator = [=, this]() { return read_buffer_creator(object_path, current_read_until_position); }; +#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD if (with_cache) { auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path); @@ -66,6 +67,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c read_until_position ? std::optional(read_until_position) : std::nullopt, cache_log); } +#endif return current_read_buffer_creator(); } diff --git a/src/Disks/IO/getThreadPoolReader.cpp b/src/Disks/IO/getThreadPoolReader.cpp index deb8f66106c..7dbff9ffe76 100644 --- a/src/Disks/IO/getThreadPoolReader.cpp +++ b/src/Disks/IO/getThreadPoolReader.cpp @@ -7,9 +7,7 @@ #include #include -#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD #include -#endif namespace DB { @@ -21,32 +19,10 @@ namespace ErrorCodes IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) { -#ifdef CLICKHOUSE_PROGRAM_STANDALONE_BUILD - const auto & config = Poco::Util::Application::instance().config(); - switch (type) - { - case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER: - { - static auto asynchronous_remote_fs_reader = createThreadPoolReader(type, config); - return *asynchronous_remote_fs_reader; - } - case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER: - { - static auto asynchronous_local_fs_reader = createThreadPoolReader(type, config); - return *asynchronous_local_fs_reader; - } - case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER: - { - static auto synchronous_local_fs_reader = createThreadPoolReader(type, config); - return *synchronous_local_fs_reader; - } - } -#else auto context = Context::getGlobalContextInstance(); if (!context) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized"); return context->getThreadPoolReader(type); -#endif } std::unique_ptr createThreadPoolReader( diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index f832ba5b7b6..2a9cb86f91c 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -530,24 +529,6 @@ DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage() threadpool_size); } -void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name) -{ - object_storage = std::make_shared(object_storage, cache, cache_settings, layer_name); -} - -NameSet DiskObjectStorage::getCacheLayersNames() const -{ - NameSet cache_layers; - auto current_object_storage = object_storage; - while (current_object_storage->supportsCache()) - { - auto * cached_object_storage = assert_cast(current_object_storage.get()); - cache_layers.insert(cached_object_storage->getCacheConfigName()); - current_object_storage = cached_object_storage->getWrappedObjectStorage(); - } - return cache_layers; -} - std::unique_ptr DiskObjectStorage::readFile( const String & path, const ReadSettings & settings, diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 4372bc75950..2d942eda1d2 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -184,20 +184,22 @@ public: /// MergeTree table on this disk. bool isWriteOnce() const override; - /// Add a cache layer. - /// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage)) - /// There can be any number of cache layers: - /// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...)) - void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name); - /// Get structure of object storage this disk works with. Examples: /// DiskObjectStorage(S3ObjectStorage) /// DiskObjectStorage(CachedObjectStorage(S3ObjectStorage)) /// DiskObjectStorage(CachedObjectStorage(CachedObjectStorage(S3ObjectStorage))) String getStructure() const { return fmt::format("DiskObjectStorage-{}({})", getName(), object_storage->getName()); } +#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD + /// Add a cache layer. + /// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage)) + /// There can be any number of cache layers: + /// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...)) + void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name); + /// Get names of all cache layers. Name is how cache is defined in configuration file. NameSet getCacheLayersNames() const override; +#endif static std::shared_ptr getAsyncExecutor(const std::string & log_name, size_t size); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageCache.cpp b/src/Disks/ObjectStorages/DiskObjectStorageCache.cpp new file mode 100644 index 00000000000..9e5012dec54 --- /dev/null +++ b/src/Disks/ObjectStorages/DiskObjectStorageCache.cpp @@ -0,0 +1,28 @@ +#include + +#include + +#include + +namespace DB +{ + +void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name) +{ + object_storage = std::make_shared(object_storage, cache, cache_settings, layer_name); +} + +NameSet DiskObjectStorage::getCacheLayersNames() const +{ + NameSet cache_layers; + auto current_object_storage = object_storage; + while (current_object_storage->supportsCache()) + { + auto * cached_object_storage = assert_cast(current_object_storage.get()); + cache_layers.insert(cached_object_storage->getCacheConfigName()); + current_object_storage = cached_object_storage->getWrappedObjectStorage(); + } + return cache_layers; +} + +} diff --git a/src/Disks/registerDisks.cpp b/src/Disks/registerDisks.cpp index 48d5a19fb61..676744a8e79 100644 --- a/src/Disks/registerDisks.cpp +++ b/src/Disks/registerDisks.cpp @@ -32,6 +32,8 @@ void registerDiskCache(DiskFactory & factory, bool global_skip_access_check); void registerDiskLocalObjectStorage(DiskFactory & factory, bool global_skip_access_check); +#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD + void registerDisks(bool global_skip_access_check) { auto & factory = DiskFactory::instance(); @@ -61,4 +63,19 @@ void registerDisks(bool global_skip_access_check) registerDiskLocalObjectStorage(factory, global_skip_access_check); } +#else + +void registerDisks(bool global_skip_access_check) +{ + auto & factory = DiskFactory::instance(); + + registerDiskLocal(factory, global_skip_access_check); + +#if USE_AWS_S3 + registerDiskS3(factory, global_skip_access_check); +#endif +} + +#endif + } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 87843a458e8..23f3281330a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,5 +1,7 @@ #pragma once +#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD + #include #include #include @@ -1226,3 +1228,9 @@ struct HTTPContext : public IHTTPContext }; } + +#else + +#include + +#endif From 378beb20e8253c09e21713f4b018ffe384242da1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 18 May 2023 07:46:02 +0000 Subject: [PATCH 0377/2223] Automatic style fix --- tests/integration/test_ssl_cert_authentication/test.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 727abc4f3b0..c601bcb1db8 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -166,22 +166,20 @@ def execute_query_https( ): try: url = f"https://{instance.ip_address}:{HTTPS_PORT}/?query={query}" - headers = {"X-ClickHouse-User":user} + headers = {"X-ClickHouse-User": user} if enable_ssl_auth: headers["X-ClickHouse-SSL-Certificate-Auth"] = "on" if password: headers["X-ClickHouse-Key"] = password - http_client=urllib3.PoolManager(ssl_context=get_ssl_context(cert_name)) - response = http_client.request('GET', url, headers=headers) + http_client = urllib3.PoolManager(ssl_context=get_ssl_context(cert_name)) + response = http_client.request("GET", url, headers=headers) if response.status != 200: - raise Exception(response.status) + raise Exception(response.status) return response.data.decode("utf-8") except: raise - - def test_https(): assert ( execute_query_https("SELECT currentUser()", user="john", cert_name="client1") From d294ecbc162e129e6f63a1d9ec4c9c80b994cddb Mon Sep 17 00:00:00 2001 From: libin Date: Thu, 18 May 2023 15:50:19 +0800 Subject: [PATCH 0378/2223] Update grant.md docs: Modifying grant example --- docs/zh/sql-reference/statements/grant.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/statements/grant.md b/docs/zh/sql-reference/statements/grant.md index 12ad2e0fe25..7e7cdbff350 100644 --- a/docs/zh/sql-reference/statements/grant.md +++ b/docs/zh/sql-reference/statements/grant.md @@ -55,7 +55,7 @@ GRANT SELECT(x,y) ON db.table TO john WITH GRANT OPTION 同样 `john` 有权执行 `GRANT OPTION`,因此他能给其它账号进行和自己账号权限范围相同的授权。 -可以使用`*` 号代替表或库名进行授权操作。例如, `GRANT SELECT ONdb.* TO john` 操作运行 `john`对 `db`库的所有表执行 `SELECT`查询。同样,你可以忽略库名。在这种情形下,权限将指向当前的数据库。例如, `GRANT SELECT ON* to john` 对当前数据库的所有表指定授权, `GARNT SELECT ON mytable to john`对当前数据库的 `mytable`表进行授权。 +可以使用`*` 号代替表或库名进行授权操作。例如, `GRANT SELECT ONdb.* TO john` 操作运行 `john`对 `db`库的所有表执行 `SELECT`查询。同样,你可以忽略库名。在这种情形下,权限将指向当前的数据库。例如, `GRANT SELECT ON* to john` 对当前数据库的所有表指定授权, `GRANT SELECT ON mytable to john`对当前数据库的 `mytable`表进行授权。 访问 `systen`数据库总是被允许的(因为这个数据库用来处理sql操作) 可以一次给多个账号进行多种授权操作。 `GRANT SELECT,INSERT ON *.* TO john,robin` 允许 `john`和`robin` 账号对任意数据库的任意表执行 `INSERT`和 `SELECT`操作。 From 83d066e5cfad4c0b471efe907dd8a24ae2475e13 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 18 May 2023 09:07:27 +0000 Subject: [PATCH 0379/2223] Re-enable Date and Date32 as parameters of toUnixTimestamp function --- src/Functions/FunctionsConversion.h | 9 ++------- .../0_stateless/01592_toUnixTimestamp_Date.reference | 4 ++++ tests/queries/0_stateless/01592_toUnixTimestamp_Date.sql | 5 ++++- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9c4085f9745..2e21932d0e2 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -145,13 +145,6 @@ struct ConvertImpl using ColVecFrom = typename FromDataType::ColumnType; using ColVecTo = typename ToDataType::ColumnType; - if (std::is_same_v) - { - if (isDateOrDate32(named_from.type)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal type {} of first argument of function {}", - named_from.type->getName(), Name::name); - } - if constexpr ((IsDataTypeDecimal || IsDataTypeDecimal) && !(std::is_same_v || std::is_same_v)) { @@ -306,6 +299,8 @@ struct ConvertImpl { if constexpr (std::is_same_v && std::is_same_v) vec_to[i] = static_cast(static_cast(vec_from[i])); + else if constexpr (std::is_same_v && (std::is_same_v || std::is_same_v)) + vec_to[i] = static_cast(vec_from[i] * 86400); else vec_to[i] = static_cast(vec_from[i]); } diff --git a/tests/queries/0_stateless/01592_toUnixTimestamp_Date.reference b/tests/queries/0_stateless/01592_toUnixTimestamp_Date.reference index e69de29bb2d..f7d86529347 100644 --- a/tests/queries/0_stateless/01592_toUnixTimestamp_Date.reference +++ b/tests/queries/0_stateless/01592_toUnixTimestamp_Date.reference @@ -0,0 +1,4 @@ +1683676800 +1683676800 +1683676800 +1683676800 diff --git a/tests/queries/0_stateless/01592_toUnixTimestamp_Date.sql b/tests/queries/0_stateless/01592_toUnixTimestamp_Date.sql index e8411484d71..f2ba18a3b55 100644 --- a/tests/queries/0_stateless/01592_toUnixTimestamp_Date.sql +++ b/tests/queries/0_stateless/01592_toUnixTimestamp_Date.sql @@ -1 +1,4 @@ -select toUnixTimestamp(today()); -- { serverError 44 } +select toUnixTimestamp(makeDate(2023, 5, 10)); +select toUnixTimestamp(makeDate32(2023, 5, 10)); +select toUnixTimestamp(makeDate(2023, 5, 10), 'Pacific/Auckland'); +select toUnixTimestamp(makeDate32(2023, 5, 10), 'Pacific/Auckland'); \ No newline at end of file From 3c68d632134bac87504eda33d28e589f24e48982 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 18 May 2023 09:11:48 +0000 Subject: [PATCH 0380/2223] Amend the toUnixTimestamp function documentation --- .../functions/date-time-functions.md | 26 ++++++++++++++----- .../functions/date-time-functions.md | 26 ++++++++++++++----- 2 files changed, 38 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4710a5617c3..342f64620b7 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -357,14 +357,14 @@ Alias: `SECOND`. ## toUnixTimestamp -For DateTime arguments: converts the value to the number with type UInt32 -- Unix Timestamp (https://en.wikipedia.org/wiki/Unix_time). +Converts a string, date or date with time to a `UInt32` number -- [Unix Timestamp](https://en.wikipedia.org/wiki/Unix_time). -For String argument: converts the input string to the datetime according to the timezone (optional second argument, server timezone is used by default) and returns the corresponding unix timestamp. +The string argument is converted according to the timezone (optional second argument, server timezone is used by default). **Syntax** ``` sql -toUnixTimestamp(datetime) +toUnixTimestamp(date) toUnixTimestamp(str, [timezone]) ``` @@ -377,15 +377,27 @@ Type: `UInt32`. **Example** ``` sql -SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp +SELECT + '2017-11-05 08:07:47' AS dt_str, + toUnixTimestamp(dt_str) AS from_str, + toUnixTimestamp(dt_str, 'Asia/Tokyo') AS from_str_tokyo, + toUnixTimestamp(toDateTime(dt_str)) AS from_datetime, + toUnixTimestamp(toDate(dt_str)) AS from_date, + toUnixTimestamp(toDate32(dt_str)) AS from_date32 +FORMAT Vertical; ``` Result: ``` text -┌─unix_timestamp─┐ -│ 1509836867 │ -└────────────────┘ +Row 1: +────── +dt_str: 2017-11-05 08:07:47 +from_str: 1509869267 +from_str_tokyo: 1509836867 +from_datetime: 1509869267 +from_date: 1509840000 +from_date32: 1509840000 ``` :::note diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index a7e8a478edb..f0a8109786f 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -235,13 +235,13 @@ SELECT toDateTime('2021-04-21 10:20:30', 'Europe/Moscow') AS Time, toTypeName(Ti ## toUnixTimestamp {#to-unix-timestamp} -Переводит дату-с-временем в число типа UInt32 -- Unix Timestamp (https://en.wikipedia.org/wiki/Unix_time). -Для аргумента String, строка конвертируется в дату и время в соответствии с часовым поясом (необязательный второй аргумент, часовой пояс сервера используется по умолчанию). +Переводит строку, дату или дату-с-временем в число типа `UInt32` -- [Unix Timestamp](https://en.wikipedia.org/wiki/Unix_time). +Строковый аргумент конвертируется в дату и время в соответствии с часовым поясом (необязательный второй аргумент, часовой пояс сервера используется по умолчанию). **Синтаксис** ``` sql -toUnixTimestamp(datetime) +toUnixTimestamp(date) toUnixTimestamp(str, [timezone]) ``` @@ -256,15 +256,27 @@ toUnixTimestamp(str, [timezone]) Запрос: ``` sql -SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; +SELECT + '2017-11-05 08:07:47' AS dt_str, + toUnixTimestamp(dt_str) AS from_str, + toUnixTimestamp(dt_str, 'Asia/Tokyo') AS from_str_tokyo, + toUnixTimestamp(toDateTime(dt_str)) AS from_datetime, + toUnixTimestamp(toDate(dt_str)) AS from_date, + toUnixTimestamp(toDate32(dt_str)) AS from_date32 +FORMAT Vertical; ``` Результат: ``` text -┌─unix_timestamp─┐ -│ 1509836867 │ -└────────────────┘ +Row 1: +────── +dt_str: 2017-11-05 08:07:47 +from_str: 1509869267 +from_str_tokyo: 1509836867 +from_datetime: 1509869267 +from_date: 1509840000 +from_date32: 1509840000 ``` :::note From f98c337d2f9580fa65c8d21de447ab6e8fe3d781 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 18 May 2023 14:53:46 +0200 Subject: [PATCH 0381/2223] Fix stack-use-after-scope in resource manager test (#49908) * Fix stack-use-after-scope in resource manager test * fix --- .../gtest_resource_manager_hierarchical.cpp | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/src/IO/Resource/tests/gtest_resource_manager_hierarchical.cpp b/src/IO/Resource/tests/gtest_resource_manager_hierarchical.cpp index b113da31d59..43773559f03 100644 --- a/src/IO/Resource/tests/gtest_resource_manager_hierarchical.cpp +++ b/src/IO/Resource/tests/gtest_resource_manager_hierarchical.cpp @@ -47,9 +47,18 @@ TEST(IOResourceDynamicResourceManager, Smoke) TEST(IOResourceDynamicResourceManager, Fairness) { - constexpr size_t T = 3; // threads per queue - int N = 100; // requests per thread - ResourceTest t(2 * T + 1); + // Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1). + // Requests from A use `value = 1` and from B `value = -1` is used. + std::atomic unfairness = 0; + auto fairness_diff = [&] (Int64 value) + { + Int64 cur_unfairness = unfairness.fetch_add(value, std::memory_order_relaxed) + value; + EXPECT_NEAR(cur_unfairness, 0, 1); + }; + + constexpr size_t threads_per_queue = 3; + int requests_per_thread = 100; + ResourceTest t(2 * threads_per_queue + 1); t.update(R"CONFIG( @@ -70,24 +79,14 @@ TEST(IOResourceDynamicResourceManager, Fairness) )CONFIG"); - - // Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1). - // Requests from A use `value = 1` and from B `value = -1` is used. - std::atomic unfairness = 0; - auto fairness_diff = [&] (Int64 value) - { - Int64 cur_unfairness = unfairness.fetch_add(value, std::memory_order_relaxed) + value; - EXPECT_NEAR(cur_unfairness, 0, 1); - }; - - for (int thr = 0; thr < T; thr++) + for (int thread = 0; thread < threads_per_queue; thread++) { t.threads.emplace_back([&] { ClassifierPtr c = t.manager->acquire("A"); ResourceLink link = c->get("res1"); - t.startBusyPeriod(link, 1, N); - for (int req = 0; req < N; req++) + t.startBusyPeriod(link, 1, requests_per_thread); + for (int request = 0; request < requests_per_thread; request++) { TestGuard g(t, link, 1); fairness_diff(1); @@ -95,14 +94,14 @@ TEST(IOResourceDynamicResourceManager, Fairness) }); } - for (int thr = 0; thr < T; thr++) + for (int thread = 0; thread < threads_per_queue; thread++) { t.threads.emplace_back([&] { ClassifierPtr c = t.manager->acquire("B"); ResourceLink link = c->get("res1"); - t.startBusyPeriod(link, 1, N); - for (int req = 0; req < N; req++) + t.startBusyPeriod(link, 1, requests_per_thread); + for (int request = 0; request < requests_per_thread; request++) { TestGuard g(t, link, 1); fairness_diff(-1); From 94fe22493527634853febfb848260c084c17a368 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 18 May 2023 10:06:59 -0300 Subject: [PATCH 0382/2223] Update partition.md --- docs/en/sql-reference/statements/alter/partition.md | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index ce0bc1ea528..a0aa74e6d25 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -284,13 +284,17 @@ Manipulates data in the specifies partition matching the specified filtering exp Syntax: ``` sql -ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] [IN PARTITION partition_id] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] [IN PARTITION partition_expr] WHERE filter_expr ``` ### Example ``` sql +-- using partition name ALTER TABLE mt UPDATE x = x + 1 IN PARTITION 2 WHERE p = 2; + +-- using partition id +ALTER TABLE mt UPDATE x = x + 1 IN PARTITION ID '2' WHERE p = 2; ``` ### See Also @@ -304,13 +308,17 @@ Deletes data in the specifies partition matching the specified filtering express Syntax: ``` sql -ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE [IN PARTITION partition_id] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE [IN PARTITION partition_expr] WHERE filter_expr ``` ### Example ``` sql +-- using partition name ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; + +-- using partition id +ALTER TABLE mt DELETE IN PARTITION ID '2' WHERE p = 2; ``` ### See Also From 971cc092d4da472fa6a3a0726616218d6a783b58 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 18 May 2023 15:16:47 +0200 Subject: [PATCH 0383/2223] Update src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp --- src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 2510d6f2d19..30f31910bee 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -366,9 +366,6 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf part_info->column_name_set = {required_column_names.begin(), required_column_names.end()}; part_info->task_columns = task_columns; - if (settings.prefetch_buffer_size < DBMS_DEFAULT_BUFFER_SIZE) - throw Exception(ErrorCodes::LOGICAL_ERROR, "remove me"); - /// adjustBufferSize(), which is done in MergeTreeReaderStream and MergeTreeReaderCompact, /// lowers buffer size if file size (or required read range) is less. So we know that the /// settings.prefetch_buffer_size will be lowered there, therefore we account it here as well. From 30083351f5769d781a2dacccc8c259e846518956 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 18 May 2023 14:42:48 +0000 Subject: [PATCH 0384/2223] test fix --- tests/queries/0_stateless/02751_protobuf_ipv6.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02751_protobuf_ipv6.sh b/tests/queries/0_stateless/02751_protobuf_ipv6.sh index ecf565d9db4..f93963aa6c6 100755 --- a/tests/queries/0_stateless/02751_protobuf_ipv6.sh +++ b/tests/queries/0_stateless/02751_protobuf_ipv6.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) SCHEMADIR=$CURDIR/format_schemas -echo 121a1000000000000000000000ffff01020304 | xxd -r -p | $CLICKHOUSE_LOCAL --input-format Protobuf --format_schema="$SCHEMADIR/02751_protobuf_ipv6:Message" --structure="ipv6_bytes IPv6" -q "select * from table" +echo -ne '\x12\x1a\x10\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\xff\xff\x01\x02\x03\x04' | $CLICKHOUSE_LOCAL --input-format Protobuf --format_schema="$SCHEMADIR/02751_protobuf_ipv6:Message" --structure="ipv6_bytes IPv6" -q "select * from table" $CLICKHOUSE_LOCAL -q "select '::ffff:1.2.3.4'::IPv6 as ipv6_bytes format Protobuf settings format_schema = '$SCHEMADIR/02751_protobuf_ipv6:Message'" | $CLICKHOUSE_LOCAL --input-format Protobuf --format_schema="$SCHEMADIR/02751_protobuf_ipv6:Message" --structure="ipv6_bytes IPv6" -q "select * from table" From 6b4dcbd3ed5f3f00322b86cf82780509f93ea038 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 18 May 2023 23:23:39 +0800 Subject: [PATCH 0385/2223] Use PROJECT_*_DIR instead of CMAKE_*_DIR. --- CMakeLists.txt | 8 ++++---- cmake/add_check.cmake | 4 ++-- cmake/git.cmake | 10 +++++----- cmake/print_flags.cmake | 6 +++--- cmake/sanitize.cmake | 6 +++--- cmake/version.cmake | 2 +- contrib/avro-cmake/CMakeLists.txt | 2 +- contrib/cassandra-cmake/CMakeLists.txt | 2 +- contrib/cctz-cmake/CMakeLists.txt | 2 +- contrib/libuv-cmake/CMakeLists.txt | 4 ++-- contrib/mariadb-connector-c-cmake/CMakeLists.txt | 2 +- contrib/snappy-cmake/CMakeLists.txt | 2 +- contrib/zlib-ng-cmake/CMakeLists.txt | 2 +- programs/self-extracting/CMakeLists.txt | 6 +++--- src/CMakeLists.txt | 2 +- 15 files changed, 30 insertions(+), 30 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 26188cb7110..56bf3e1c3f8 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -259,8 +259,8 @@ endif () option (ENABLE_BUILD_PATH_MAPPING "Enable remapping of file source paths in debug info, predefined preprocessor macros, and __builtin_FILE(). It's used to generate reproducible builds. See https://reproducible-builds.org/docs/build-path" ${ENABLE_BUILD_PATH_MAPPING_DEFAULT}) if (ENABLE_BUILD_PATH_MAPPING) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -ffile-prefix-map=${CMAKE_SOURCE_DIR}=.") - set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -ffile-prefix-map=${CMAKE_SOURCE_DIR}=.") + set (COMPILER_FLAGS "${COMPILER_FLAGS} -ffile-prefix-map=${PROJECT_SOURCE_DIR}=.") + set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -ffile-prefix-map=${PROJECT_SOURCE_DIR}=.") endif () option (ENABLE_BUILD_PROFILING "Enable profiling of build time" OFF) @@ -557,7 +557,7 @@ if (NATIVE_BUILD_TARGETS ) message (STATUS "Building native targets...") - set (NATIVE_BUILD_DIR "${CMAKE_BINARY_DIR}/native") + set (NATIVE_BUILD_DIR "${PROJECT_BINARY_DIR}/native") execute_process( COMMAND ${CMAKE_COMMAND} -E make_directory "${NATIVE_BUILD_DIR}" @@ -571,7 +571,7 @@ if (NATIVE_BUILD_TARGETS # Avoid overriding .cargo/config.toml with native toolchain. "-DENABLE_RUST=OFF" "-DENABLE_CLICKHOUSE_SELF_EXTRACTING=${ENABLE_CLICKHOUSE_SELF_EXTRACTING}" - ${CMAKE_SOURCE_DIR} + ${PROJECT_SOURCE_DIR} WORKING_DIRECTORY "${NATIVE_BUILD_DIR}" COMMAND_ECHO STDOUT) diff --git a/cmake/add_check.cmake b/cmake/add_check.cmake index c6abbcdb321..ba30ee8676f 100644 --- a/cmake/add_check.cmake +++ b/cmake/add_check.cmake @@ -5,11 +5,11 @@ if (NOT TARGET check) if (CMAKE_CONFIGURATION_TYPES) add_custom_target (check COMMAND ${CMAKE_CTEST_COMMAND} --force-new-ctest-process --output-on-failure --build-config "$" - WORKING_DIRECTORY ${CMAKE_BINARY_DIR}) + WORKING_DIRECTORY ${PROJECT_BINARY_DIR}) else () add_custom_target (check COMMAND ${CMAKE_CTEST_COMMAND} --force-new-ctest-process --output-on-failure - WORKING_DIRECTORY ${CMAKE_BINARY_DIR}) + WORKING_DIRECTORY ${PROJECT_BINARY_DIR}) endif () endif () diff --git a/cmake/git.cmake b/cmake/git.cmake index 397ec3cd081..a4b3bd4bdab 100644 --- a/cmake/git.cmake +++ b/cmake/git.cmake @@ -5,14 +5,14 @@ if (Git_FOUND) # Commit hash + whether the building workspace was dirty or not execute_process(COMMAND "${GIT_EXECUTABLE}" rev-parse HEAD - WORKING_DIRECTORY ${CMAKE_SOURCE_DIR} + WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_VARIABLE GIT_HASH ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) # Branch name execute_process(COMMAND "${GIT_EXECUTABLE}" rev-parse --abbrev-ref HEAD - WORKING_DIRECTORY ${CMAKE_SOURCE_DIR} + WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_VARIABLE GIT_BRANCH ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) @@ -20,14 +20,14 @@ if (Git_FOUND) SET(ENV{TZ} "UTC") execute_process(COMMAND "${GIT_EXECUTABLE}" log -1 --format=%ad --date=iso-local - WORKING_DIRECTORY ${CMAKE_SOURCE_DIR} + WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_VARIABLE GIT_DATE ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) # Subject of the commit execute_process(COMMAND "${GIT_EXECUTABLE}" log -1 --format=%s - WORKING_DIRECTORY ${CMAKE_SOURCE_DIR} + WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_VARIABLE GIT_COMMIT_SUBJECT ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) @@ -35,7 +35,7 @@ if (Git_FOUND) execute_process( COMMAND ${GIT_EXECUTABLE} status - WORKING_DIRECTORY ${CMAKE_SOURCE_DIR} OUTPUT_STRIP_TRAILING_WHITESPACE) + WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_STRIP_TRAILING_WHITESPACE) else() message(STATUS "Git could not be found.") endif() diff --git a/cmake/print_flags.cmake b/cmake/print_flags.cmake index 66f2a8bfbc7..869764602d4 100644 --- a/cmake/print_flags.cmake +++ b/cmake/print_flags.cmake @@ -7,6 +7,6 @@ message (STATUS "compiler CXX = ${CMAKE_CXX_COMPILER} ${FULL_CXX_FLAGS}") message (STATUS "LINKER_FLAGS = ${FULL_EXE_LINKER_FLAGS}") # Reproducible builds -string (REPLACE "${CMAKE_SOURCE_DIR}" "." FULL_C_FLAGS_NORMALIZED "${FULL_C_FLAGS}") -string (REPLACE "${CMAKE_SOURCE_DIR}" "." FULL_CXX_FLAGS_NORMALIZED "${FULL_CXX_FLAGS}") -string (REPLACE "${CMAKE_SOURCE_DIR}" "." FULL_EXE_LINKER_FLAGS_NORMALIZED "${FULL_EXE_LINKER_FLAGS}") +string (REPLACE "${PROJECT_SOURCE_DIR}" "." FULL_C_FLAGS_NORMALIZED "${FULL_C_FLAGS}") +string (REPLACE "${PROJECT_SOURCE_DIR}" "." FULL_CXX_FLAGS_NORMALIZED "${FULL_CXX_FLAGS}") +string (REPLACE "${PROJECT_SOURCE_DIR}" "." FULL_EXE_LINKER_FLAGS_NORMALIZED "${FULL_EXE_LINKER_FLAGS}") diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index b2fbdb256fd..17ce8a7db29 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -29,14 +29,14 @@ if (SANITIZE) # Linking can fail due to relocation overflows (see #49145), caused by too big object files / libraries. # Work around this with position-independent builds (-fPIC and -fpie), this is slightly slower than non-PIC/PIE but that's okay. - set (MSAN_FLAGS "-fsanitize=memory -fsanitize-memory-use-after-dtor -fsanitize-memory-track-origins -fno-optimize-sibling-calls -fPIC -fpie -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/msan_suppressions.txt") + set (MSAN_FLAGS "-fsanitize=memory -fsanitize-memory-use-after-dtor -fsanitize-memory-track-origins -fno-optimize-sibling-calls -fPIC -fpie -fsanitize-blacklist=${PROJECT_SOURCE_DIR}/tests/msan_suppressions.txt") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${MSAN_FLAGS}") elseif (SANITIZE STREQUAL "thread") set (TSAN_FLAGS "-fsanitize=thread") if (COMPILER_CLANG) - set (TSAN_FLAGS "${TSAN_FLAGS} -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt") + set (TSAN_FLAGS "${TSAN_FLAGS} -fsanitize-blacklist=${PROJECT_SOURCE_DIR}/tests/tsan_suppressions.txt") endif() set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}") @@ -54,7 +54,7 @@ if (SANITIZE) set(UBSAN_FLAGS "${UBSAN_FLAGS} -fno-sanitize=unsigned-integer-overflow") endif() if (COMPILER_CLANG) - set (UBSAN_FLAGS "${UBSAN_FLAGS} -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/ubsan_suppressions.txt") + set (UBSAN_FLAGS "${UBSAN_FLAGS} -fsanitize-blacklist=${PROJECT_SOURCE_DIR}/tests/ubsan_suppressions.txt") endif() set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${UBSAN_FLAGS}") diff --git a/cmake/version.cmake b/cmake/version.cmake index acaa772ff2f..9ca21556f4d 100644 --- a/cmake/version.cmake +++ b/cmake/version.cmake @@ -1,4 +1,4 @@ -include(${CMAKE_SOURCE_DIR}/cmake/autogenerated_versions.txt) +include(${PROJECT_SOURCE_DIR}/cmake/autogenerated_versions.txt) set(VERSION_EXTRA "" CACHE STRING "") set(VERSION_TWEAK "" CACHE STRING "") diff --git a/contrib/avro-cmake/CMakeLists.txt b/contrib/avro-cmake/CMakeLists.txt index 25474650d0e..63b3854eef9 100644 --- a/contrib/avro-cmake/CMakeLists.txt +++ b/contrib/avro-cmake/CMakeLists.txt @@ -6,7 +6,7 @@ if (NOT ENABLE_AVRO) return() endif() -set(AVROCPP_ROOT_DIR "${CMAKE_SOURCE_DIR}/contrib/avro/lang/c++") +set(AVROCPP_ROOT_DIR "${PROJECT_SOURCE_DIR}/contrib/avro/lang/c++") set(AVROCPP_INCLUDE_DIR "${AVROCPP_ROOT_DIR}/api") set(AVROCPP_SOURCE_DIR "${AVROCPP_ROOT_DIR}/impl") diff --git a/contrib/cassandra-cmake/CMakeLists.txt b/contrib/cassandra-cmake/CMakeLists.txt index 59ff908b63a..32611e0e151 100644 --- a/contrib/cassandra-cmake/CMakeLists.txt +++ b/contrib/cassandra-cmake/CMakeLists.txt @@ -18,7 +18,7 @@ endif() # Need to use C++17 since the compilation is not possible with C++20 currently. set (CMAKE_CXX_STANDARD 17) -set(CASS_ROOT_DIR ${CMAKE_SOURCE_DIR}/contrib/cassandra) +set(CASS_ROOT_DIR ${PROJECT_SOURCE_DIR}/contrib/cassandra) set(CASS_SRC_DIR "${CASS_ROOT_DIR}/src") set(CASS_INCLUDE_DIR "${CASS_ROOT_DIR}/include") diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index f1ef9b53f7d..10070fbd949 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -26,7 +26,7 @@ endif () # StorageSystemTimeZones.generated.cpp is autogenerated each time during a build # data in this file will be used to populate the system.time_zones table, this is specific to OS_LINUX # as the library that's built using embedded tzdata is also specific to OS_LINUX -set(SYSTEM_STORAGE_TZ_FILE "${CMAKE_BINARY_DIR}/src/Storages/System/StorageSystemTimeZones.generated.cpp") +set(SYSTEM_STORAGE_TZ_FILE "${PROJECT_BINARY_DIR}/src/Storages/System/StorageSystemTimeZones.generated.cpp") # remove existing copies so that its generated fresh on each build. file(REMOVE ${SYSTEM_STORAGE_TZ_FILE}) diff --git a/contrib/libuv-cmake/CMakeLists.txt b/contrib/libuv-cmake/CMakeLists.txt index fb88799ed38..928fdcdd7e6 100644 --- a/contrib/libuv-cmake/CMakeLists.txt +++ b/contrib/libuv-cmake/CMakeLists.txt @@ -1,7 +1,7 @@ # This file is a modified version of contrib/libuv/CMakeLists.txt -set (SOURCE_DIR "${CMAKE_SOURCE_DIR}/contrib/libuv") -set (BINARY_DIR "${CMAKE_BINARY_DIR}/contrib/libuv") +set (SOURCE_DIR "${PROJECT_SOURCE_DIR}/contrib/libuv") +set (BINARY_DIR "${PROJECT_BINARY_DIR}/contrib/libuv") set(uv_sources src/fs-poll.c diff --git a/contrib/mariadb-connector-c-cmake/CMakeLists.txt b/contrib/mariadb-connector-c-cmake/CMakeLists.txt index 50287c54ac1..18d1510a57b 100644 --- a/contrib/mariadb-connector-c-cmake/CMakeLists.txt +++ b/contrib/mariadb-connector-c-cmake/CMakeLists.txt @@ -15,7 +15,7 @@ endif() # This is the LGPL libmariadb project. -set(CC_SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/mariadb-connector-c) +set(CC_SOURCE_DIR ${PROJECT_SOURCE_DIR}/contrib/mariadb-connector-c) set(CC_BINARY_DIR ${CMAKE_CURRENT_BINARY_DIR}) set(WITH_SSL ON) diff --git a/contrib/snappy-cmake/CMakeLists.txt b/contrib/snappy-cmake/CMakeLists.txt index 50cdc8732a1..f406de0e343 100644 --- a/contrib/snappy-cmake/CMakeLists.txt +++ b/contrib/snappy-cmake/CMakeLists.txt @@ -1,4 +1,4 @@ -set (SOURCE_DIR "${CMAKE_SOURCE_DIR}/contrib/snappy") +set (SOURCE_DIR "${PROJECT_SOURCE_DIR}/contrib/snappy") if (ARCH_S390X) set (SNAPPY_IS_BIG_ENDIAN 1) diff --git a/contrib/zlib-ng-cmake/CMakeLists.txt b/contrib/zlib-ng-cmake/CMakeLists.txt index aa067ba37e0..79f343bfc75 100644 --- a/contrib/zlib-ng-cmake/CMakeLists.txt +++ b/contrib/zlib-ng-cmake/CMakeLists.txt @@ -1,4 +1,4 @@ -set (SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/zlib-ng) +set (SOURCE_DIR ${PROJECT_SOURCE_DIR}/contrib/zlib-ng) add_definitions(-DZLIB_COMPAT) add_definitions(-DWITH_GZFILEOP) diff --git a/programs/self-extracting/CMakeLists.txt b/programs/self-extracting/CMakeLists.txt index 2cc26926b38..f3ff0bbcd78 100644 --- a/programs/self-extracting/CMakeLists.txt +++ b/programs/self-extracting/CMakeLists.txt @@ -4,10 +4,10 @@ if (NOT( AND CMAKE_HOST_SYSTEM_PROCESSOR STREQUAL CMAKE_SYSTEM_PROCESSOR ) ) - set (COMPRESSOR "${CMAKE_BINARY_DIR}/native/utils/self-extracting-executable/pre_compressor") - set (DECOMPRESSOR "--decompressor=${CMAKE_BINARY_DIR}/utils/self-extracting-executable/decompressor") + set (COMPRESSOR "${PROJECT_BINARY_DIR}/native/utils/self-extracting-executable/pre_compressor") + set (DECOMPRESSOR "--decompressor=${PROJECT_BINARY_DIR}/utils/self-extracting-executable/decompressor") else () - set (COMPRESSOR "${CMAKE_BINARY_DIR}/utils/self-extracting-executable/compressor") + set (COMPRESSOR "${PROJECT_BINARY_DIR}/utils/self-extracting-executable/compressor") endif () add_custom_target (self-extracting ALL diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b3f4fbb7420..87a2979ecd1 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -527,7 +527,7 @@ target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::fast_float) if (USE_ORC) dbms_target_link_libraries(PUBLIC ${ORC_LIBRARIES}) - dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} "${CMAKE_BINARY_DIR}/contrib/orc/c++/include") + dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} "${PROJECT_BINARY_DIR}/contrib/orc/c++/include") endif () if (TARGET ch_contrib::rocksdb) From 73661c3a4635398de2d4783f0e2c17fe59265258 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 18 May 2023 18:18:30 +0200 Subject: [PATCH 0386/2223] Move tunnings for woboq codebrowser to cmake out from build.sh Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 6 ++++++ docker/test/codebrowser/build.sh | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 26188cb7110..54d1ae2bfcb 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -102,6 +102,12 @@ if (ENABLE_FUZZING) set (ENABLE_PROTOBUF 1) endif() +option (ENABLE_WOBOQ_CODEBROWSER "Build for woboq codebrowser" OFF) + +if (ENABLE_WOBOQ_CODEBROWSER) + set (ENABLE_EMBEDDED_COMPILER 0) +endif() + # Global libraries # See: # - default_libs.cmake diff --git a/docker/test/codebrowser/build.sh b/docker/test/codebrowser/build.sh index 5ab9de5a453..d76d0c3a039 100755 --- a/docker/test/codebrowser/build.sh +++ b/docker/test/codebrowser/build.sh @@ -15,7 +15,7 @@ nproc=$(($(nproc) + 2)) # increase parallelism read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}" mkdir -p "$BUILD_DIRECTORY" && cd "$BUILD_DIRECTORY" -cmake "$SOURCE_DIRECTORY" -DCMAKE_CXX_COMPILER="/usr/bin/clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="/usr/bin/clang-${LLVM_VERSION}" -DCMAKE_EXPORT_COMPILE_COMMANDS=ON -DENABLE_EMBEDDED_COMPILER=0 "${CMAKE_FLAGS[@]}" +cmake "$SOURCE_DIRECTORY" -DCMAKE_CXX_COMPILER="/usr/bin/clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="/usr/bin/clang-${LLVM_VERSION}" -DENABLE_WOBOQ_CODEBROWSER=ON "${CMAKE_FLAGS[@]}" mkdir -p "$HTML_RESULT_DIRECTORY" echo 'Filter out too noisy "Error: filename" lines and keep them in full codebrowser_generator.log' /woboq_codebrowser/generator/codebrowser_generator -b "$BUILD_DIRECTORY" -a \ From 0f7a310a6775728fd71c4784c5e7a8e776338ca5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 18 May 2023 18:21:19 +0200 Subject: [PATCH 0387/2223] Fix woboq codebrowser build with -Wno-poison-system-directories woboq codebrowser uses clang tooling, which adds clang system includes (in Linux::AddClangSystemIncludeArgs()), because none of (-nostdinc, -nobuiltininc) is set. And later it will complain with -Wpoison-system-directories for added by itself includes in InitHeaderSearch::AddUnmappedPath(), because they are starts from one of the following: - /usr/include - /usr/local/include The interesting thing here is that it got broken only after upgrading to llvm 16 (in #49678), and the reason for this is that clang 15 build has system includes that does not trigger the warning - "/usr/lib/clang/15.0.7/include", while clang 16 has "/usr/include/clang/16.0.4/include" So let's simply disable this warning, but only for woboq. Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 54d1ae2bfcb..ef3f34204b2 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -106,6 +106,11 @@ option (ENABLE_WOBOQ_CODEBROWSER "Build for woboq codebrowser" OFF) if (ENABLE_WOBOQ_CODEBROWSER) set (ENABLE_EMBEDDED_COMPILER 0) + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-poison-system-directories") + # woboq codebrowser uses clang tooling, and they could add default system + # clang includes, and later clang will warn for those added by itself + # includes. + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-poison-system-directories") endif() # Global libraries From e7b6056bbbab56b2fe8d0e9f6243bba355744f31 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 18 May 2023 15:18:55 -0300 Subject: [PATCH 0388/2223] test for #46128 --- ...with_short_circuit_functins_mutations.reference | 3 +++ ...ality_with_short_circuit_functins_mutations.sql | 14 ++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02481_low_cardinality_with_short_circuit_functins_mutations.reference create mode 100644 tests/queries/0_stateless/02481_low_cardinality_with_short_circuit_functins_mutations.sql diff --git a/tests/queries/0_stateless/02481_low_cardinality_with_short_circuit_functins_mutations.reference b/tests/queries/0_stateless/02481_low_cardinality_with_short_circuit_functins_mutations.reference new file mode 100644 index 00000000000..9a6e97d4503 --- /dev/null +++ b/tests/queries/0_stateless/02481_low_cardinality_with_short_circuit_functins_mutations.reference @@ -0,0 +1,3 @@ +0 xxxx yyyy +1 yyyy yyyy +2 xxxx yyyy diff --git a/tests/queries/0_stateless/02481_low_cardinality_with_short_circuit_functins_mutations.sql b/tests/queries/0_stateless/02481_low_cardinality_with_short_circuit_functins_mutations.sql new file mode 100644 index 00000000000..9d183dde91d --- /dev/null +++ b/tests/queries/0_stateless/02481_low_cardinality_with_short_circuit_functins_mutations.sql @@ -0,0 +1,14 @@ +drop table if exists issue_46128; + +create table issue_46128 ( + id Int64, + a LowCardinality(Nullable(String)), + b LowCardinality(Nullable(String)) +) Engine = MergeTree order by id +as select number%100, 'xxxx', 'yyyy' from numbers(10); + +ALTER TABLE issue_46128 UPDATE a = b WHERE id= 1 settings mutations_sync=2; + +select * from issue_46128 where id <= 2 order by id; + +drop table issue_46128; From b21e15fc185519d113c1647d6515c4151659cf34 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 18 May 2023 20:32:55 +0000 Subject: [PATCH 0389/2223] Align 00921_datetime64_compatibility_long test with the new version of toUnixTimestamp() function --- .../0_stateless/00921_datetime64_compatibility_long.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference index 4f964f2478f..55bcf000d10 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference +++ b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference @@ -49,7 +49,7 @@ Code: 43 "UInt8",11 ------------------------------------------ SELECT toUnixTimestamp(N) -Code: 44 +"UInt32",1568650811 "UInt32",1568650811 "UInt32",1568650811 ------------------------------------------ From 8dc59c1efea2dfaa56ec5a0728a362714e2ab206 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 18 May 2023 21:40:20 +0000 Subject: [PATCH 0390/2223] Fix test_insert_same_partition_and_merge failing if one Azure request attempt fails --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- tests/integration/test_merge_tree_azure_blob_storage/test.py | 4 ++-- tests/integration/test_merge_tree_s3/test.py | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index f8ca6b9ab07..44185f74f60 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -51,7 +51,7 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, if (i == num_tries - 1) throw; - LOG_DEBUG(log, "Write at attempt {} for blob `{}` failed: {}", i + 1, blob_path, e.Message); + LOG_DEBUG(log, "Write at attempt {} for blob `{}` failed: {} {}", i + 1, blob_path, e.what(), e.Message); }; for (size_t i = 0; i < num_tries; ++i) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index bcb62c3181d..8bf4df17c39 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -203,7 +203,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical): node.query(f"SYSTEM START MERGES {TABLE_NAME}") # Wait for merges and old parts deletion - for attempt in range(0, 10): + for attempt in range(0, 60): parts_count = azure_query( node, f"SELECT COUNT(*) FROM system.parts WHERE table = '{TABLE_NAME}' FORMAT Values", @@ -211,7 +211,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical): if parts_count == "(1)": break - if attempt == 9: + if attempt == 59: assert parts_count == "(1)" time.sleep(1) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 7fbe8c8e99b..ee774f6632b 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -232,7 +232,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name): node.query("SYSTEM START MERGES s3_test") # Wait for merges and old parts deletion - for attempt in range(0, 10): + for attempt in range(0, 60): parts_count = node.query( "SELECT COUNT(*) FROM system.parts WHERE table = 's3_test' and active = 1 FORMAT Values" ) @@ -240,7 +240,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name): if parts_count == "(1)": break - if attempt == 9: + if attempt == 59: assert parts_count == "(1)" time.sleep(1) From 3a3e41355239a27d1748f0d6c220d0f6ccabae15 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 18 May 2023 21:47:52 +0000 Subject: [PATCH 0391/2223] Implement toLastDayWeek function --- src/Common/DateLUTImpl.h | 51 +++++++++++++++++++++++++--- src/Functions/CustomWeekTransforms.h | 32 +++++++++++++++++ src/Functions/toCustomWeek.cpp | 2 ++ 3 files changed, 80 insertions(+), 5 deletions(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index b40b4d7c65b..93af04456b2 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -311,11 +311,6 @@ public: /// All functions below are thread-safe; arguments are not checked. - static ExtendedDayNum toDayNum(ExtendedDayNum d) - { - return d; - } - static UInt32 saturateMinus(UInt32 x, UInt32 y) { UInt32 res = x - y; @@ -323,6 +318,11 @@ public: return res; } + static ExtendedDayNum toDayNum(ExtendedDayNum d) + { + return d; + } + static ExtendedDayNum toDayNum(LUTIndex d) { return ExtendedDayNum{static_cast(d.toUnderType() - daynum_offset_epoch)}; @@ -363,6 +363,27 @@ public: return toDayNum(LUTIndex(i - (lut[i].day_of_week - 1))); } + /// Round up to the last day of week. + template + inline Time toLastDayOfWeek(DateOrTime v) const + { + const LUTIndex i = toLUTIndex(v); + if constexpr (std::is_unsigned_v || std::is_same_v) + return lut_saturated[i + (7 - lut[i].day_of_week)].date; + else + return lut[i + (7 - lut[i].day_of_week)].date; + } + + template + inline auto toLastDayNumOfWeek(DateOrTime v) const + { + const LUTIndex i = toLUTIndex(v); + if constexpr (std::is_unsigned_v || std::is_same_v) + return toDayNum(LUTIndexWithSaturation(i + (7 - lut[i].day_of_week))); + else + return toDayNum(LUTIndex(i + (7 - lut[i].day_of_week))); + } + /// Round down to start of month. template inline Time toFirstDayOfMonth(DateOrTime v) const @@ -863,6 +884,26 @@ public: } } + /// Get last day of week with week_mode, return Saturday or Sunday + template + inline auto toLastDayNumOfWeek(DateOrTime v, UInt8 week_mode) const + { + bool monday_first_mode = week_mode & static_cast(WeekModeFlag::MONDAY_FIRST); + if (monday_first_mode) + { + return toLastDayNumOfWeek(v); + } + else + { + const auto day_of_week = toDayOfWeek(v); + v += 6; + if constexpr (std::is_unsigned_v || std::is_same_v) + return (day_of_week != 7) ? DayNum(saturateMinus(v, day_of_week)) : toDayNum(v); + else + return (day_of_week != 7) ? ExtendedDayNum(v - day_of_week) : toDayNum(v); + } + } + /// Check and change mode to effective. inline UInt8 check_week_mode(UInt8 mode) const /// NOLINT { diff --git a/src/Functions/CustomWeekTransforms.h b/src/Functions/CustomWeekTransforms.h index 413c81d1400..2a82912d5d6 100644 --- a/src/Functions/CustomWeekTransforms.h +++ b/src/Functions/CustomWeekTransforms.h @@ -88,6 +88,38 @@ struct ToStartOfWeekImpl using FactorTransform = ZeroTransform; }; +struct ToLastDayOfWeekImpl +{ + static constexpr auto name = "toLastDayOfWeek"; + + static inline UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static inline UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode); + } + static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(DayNum(d), week_mode); + } + static inline Int64 executeExtendedResult(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static inline Int32 executeExtendedResult(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode); + } + + using FactorTransform = ZeroTransform; +}; + struct ToWeekImpl { static constexpr auto name = "toWeek"; diff --git a/src/Functions/toCustomWeek.cpp b/src/Functions/toCustomWeek.cpp index b773cc7df96..98e7aaf1d6b 100644 --- a/src/Functions/toCustomWeek.cpp +++ b/src/Functions/toCustomWeek.cpp @@ -11,12 +11,14 @@ namespace DB using FunctionToWeek = FunctionCustomWeekToSomething; using FunctionToYearWeek = FunctionCustomWeekToSomething; using FunctionToStartOfWeek = FunctionCustomWeekToDateOrDate32; +using FunctionToLastDayOfWeek = FunctionCustomWeekToDateOrDate32; REGISTER_FUNCTION(ToCustomWeek) { factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); /// Compatibility aliases for mysql. factory.registerAlias("week", "toWeek", FunctionFactory::CaseInsensitive); From b8be714830a0b27766f7f50bb63ddb95b549c65c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 19 May 2023 00:44:27 +0000 Subject: [PATCH 0392/2223] Add schema inference to more table engines --- .../PostgreSQL/DatabasePostgreSQL.cpp | 5 +-- .../MeiliSearch/StorageMeiliSearch.cpp | 19 +++++++++++- src/Storages/MeiliSearch/StorageMeiliSearch.h | 4 ++- src/Storages/StorageExternalDistributed.cpp | 2 +- src/Storages/StorageMySQL.cpp | 28 ++++++++++++++++- src/Storages/StorageMySQL.h | 6 ++++ src/Storages/StoragePostgreSQL.cpp | 31 ++++++++++++++++++- src/Storages/StoragePostgreSQL.h | 7 +++++ src/Storages/StorageSQLite.cpp | 25 ++++++++++++++- src/Storages/StorageSQLite.h | 4 +++ .../TableFunctionMeiliSearch.cpp | 11 ++----- src/TableFunctions/TableFunctionMySQL.cpp | 17 ++-------- .../TableFunctionPostgreSQL.cpp | 16 ++-------- src/TableFunctions/TableFunctionSQLite.cpp | 12 ++----- .../test_storage_meilisearch/test.py | 22 ++++++++++++- tests/integration/test_storage_mysql/test.py | 27 ++++++++++++++++ .../test_storage_postgresql/test.py | 13 +++++++- .../01889_sqlite_read_write.reference | 5 +++ .../0_stateless/01889_sqlite_read_write.sh | 8 +++++ 19 files changed, 207 insertions(+), 55 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 4b42d799661..f4d750f85d4 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -176,7 +176,7 @@ StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, ContextPtr } -StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr, bool table_checked) const +StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr context_, bool table_checked) const { if (!cache_tables || !cached_tables.contains(table_name)) { @@ -191,7 +191,8 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr, auto storage = std::make_shared( StorageID(database_name, table_name), pool, table_name, - ColumnsDescription{columns_info->columns}, ConstraintsDescription{}, String{}, configuration.schema, configuration.on_conflict); + ColumnsDescription{columns_info->columns}, ConstraintsDescription{}, String{}, + context_, configuration.schema, configuration.on_conflict); if (cache_tables) { diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp index fe8cc74f577..e7350d38a20 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB { @@ -37,12 +38,27 @@ StorageMeiliSearch::StorageMeiliSearch( : IStorage(table_id), config{config_}, log(&Poco::Logger::get("StorageMeiliSearch (" + table_id.table_name + ")")) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + + if (columns_.empty()) + { + auto columns = getTableStructureFromData(config); + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); } +ColumnsDescription StorageMeiliSearch::getTableStructureFromData(const MeiliSearchConfiguration & config_) +{ + MeiliSearchColumnDescriptionFetcher fetcher(config_); + fetcher.addParam(doubleQuoteString("limit"), "1"); + return fetcher.fetchColumnsDescription(); +} + String convertASTtoStr(ASTPtr ptr) { WriteBufferFromOwnString out; @@ -175,6 +191,7 @@ void registerStorageMeiliSearch(StorageFactory & factory) return std::make_shared(args.table_id, config, args.columns, args.constraints, args.comment); }, { + .supports_schema_inference = true, .source_access_type = AccessType::MEILISEARCH, }); } diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.h b/src/Storages/MeiliSearch/StorageMeiliSearch.h index 30ff2f9b9fa..41c1db53437 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.h +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.h @@ -28,7 +28,9 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; - MeiliSearchConfiguration static getConfiguration(ASTs engine_args, ContextPtr context); + static MeiliSearchConfiguration getConfiguration(ASTs engine_args, ContextPtr context); + + static ColumnsDescription getTableStructureFromData(const MeiliSearchConfiguration & config_); private: MeiliSearchConfiguration config; diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index db1f33193ac..d493fead993 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -170,7 +170,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, settings.postgresql_connection_pool_auto_close_connection); shards.insert(std::make_shared( - args.table_id, std::move(pool), configuration.table, args.columns, args.constraints, String{})); + args.table_id, std::move(pool), configuration.table, args.columns, args.constraints, String{}, context)); } } #endif diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 232ff87d9ed..506b9659e3a 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -21,6 +21,7 @@ #include #include #include +#include namespace DB @@ -65,12 +66,36 @@ StorageMySQL::StorageMySQL( , log(&Poco::Logger::get("StorageMySQL (" + table_id_.table_name + ")")) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + + if (columns_.empty()) + { + auto columns = getTableStructureFromData(*pool, remote_database_name, remote_table_name, context_); + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); } +ColumnsDescription StorageMySQL::getTableStructureFromData( + mysqlxx::PoolWithFailover & pool_, + const String & database, + const String & table, + const ContextPtr & context_) +{ + const auto & settings = context_->getSettingsRef(); + const auto tables_and_columns = fetchTablesColumnsList(pool_, database, {table}, settings, settings.mysql_datatypes_support_level); + + const auto columns = tables_and_columns.find(table); + if (columns == tables_and_columns.end()) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", + (database.empty() ? "" : (backQuote(database) + "." + backQuote(table)))); + + return columns->second; +} Pipe StorageMySQL::read( const Names & column_names_, @@ -354,6 +379,7 @@ void registerStorageMySQL(StorageFactory & factory) }, { .supports_settings = true, + .supports_schema_inference = true, .source_access_type = AccessType::MYSQL, }); } diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index ae3d2f935b6..9f47f9925d5 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -75,6 +75,12 @@ public: const NamedCollection & named_collection, MySQLSettings & storage_settings, ContextPtr context_, bool require_table = true); + static ColumnsDescription getTableStructureFromData( + mysqlxx::PoolWithFailover & pool_, + const String & database, + const String & table, + const ContextPtr & context_); + private: friend class StorageMySQLSink; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index e013199c584..2285099d00b 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -43,6 +43,8 @@ #include #include +#include + namespace DB { @@ -60,6 +62,7 @@ StoragePostgreSQL::StoragePostgreSQL( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment, + ContextPtr context_, const String & remote_table_schema_, const String & on_conflict_) : IStorage(table_id_) @@ -70,12 +73,36 @@ StoragePostgreSQL::StoragePostgreSQL( , log(&Poco::Logger::get("StoragePostgreSQL (" + table_id_.table_name + ")")) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + + if (columns_.empty()) + { + auto columns = getTableStructureFromData(pool, remote_table_name, remote_table_schema, context_); + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); } +ColumnsDescription StoragePostgreSQL::getTableStructureFromData( + const postgres::PoolWithFailoverPtr & pool_, + const String & table, + const String & schema, + const ContextPtr & context_) +{ + const bool use_nulls = context_->getSettingsRef().external_table_functions_use_nulls; + auto connection_holder = pool_->get(); + auto columns_info = fetchPostgreSQLTableStructure( + connection_holder->get(), table, schema, use_nulls).physical_columns; + + if (!columns_info) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table structure not returned"); + + return ColumnsDescription{columns_info->columns}; +} Pipe StoragePostgreSQL::read( const Names & column_names_, @@ -504,10 +531,12 @@ void registerStoragePostgreSQL(StorageFactory & factory) args.columns, args.constraints, args.comment, + args.getContext(), configuration.schema, configuration.on_conflict); }, { + .supports_schema_inference = true, .source_access_type = AccessType::POSTGRES, }); } diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index b3ff342da10..be6bbc5ec63 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -31,6 +31,7 @@ public: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment, + ContextPtr context_, const String & remote_table_schema_ = "", const String & on_conflict = ""); @@ -66,6 +67,12 @@ public: static Configuration processNamedCollectionResult(const NamedCollection & named_collection, bool require_table = true); + static ColumnsDescription getTableStructureFromData( + const postgres::PoolWithFailoverPtr & pool_, + const String & table, + const String & schema, + const ContextPtr & context_); + private: String remote_table_name; String remote_table_schema; diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 10eba370d26..bd445217979 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -44,12 +45,33 @@ StorageSQLite::StorageSQLite( , log(&Poco::Logger::get("StorageSQLite (" + table_id_.table_name + ")")) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + + if (columns_.empty()) + { + auto columns = getTableStructureFromData(sqlite_db, remote_table_name); + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); } +ColumnsDescription StorageSQLite::getTableStructureFromData( + const SQLitePtr & sqlite_db_, + const String & table) +{ + auto columns = fetchSQLiteTableStructure(sqlite_db_.get(), table); + + if (!columns) + throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, "Failed to fetch table structure for {}", table); + + return ColumnsDescription{*columns}; +} + + Pipe StorageSQLite::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -176,6 +198,7 @@ void registerStorageSQLite(StorageFactory & factory) table_name, args.columns, args.constraints, args.getContext()); }, { + .supports_schema_inference = true, .source_access_type = AccessType::SQLITE, }); } diff --git a/src/Storages/StorageSQLite.h b/src/Storages/StorageSQLite.h index a021c00f627..323c29ac8bb 100644 --- a/src/Storages/StorageSQLite.h +++ b/src/Storages/StorageSQLite.h @@ -42,6 +42,10 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + static ColumnsDescription getTableStructureFromData( + const SQLitePtr & sqlite_db_, + const String & table); + private: String remote_table_name; String database_path; diff --git a/src/TableFunctions/TableFunctionMeiliSearch.cpp b/src/TableFunctions/TableFunctionMeiliSearch.cpp index 32dd73a37a1..41ae5bb9ee2 100644 --- a/src/TableFunctions/TableFunctionMeiliSearch.cpp +++ b/src/TableFunctions/TableFunctionMeiliSearch.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -9,19 +8,15 @@ namespace DB { StoragePtr TableFunctionMeiliSearch::executeImpl( - const ASTPtr & /* ast_function */, ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/) const + const ASTPtr & /* ast_function */, ContextPtr /*context*/, const String & table_name, ColumnsDescription /*cached_columns*/) const { - auto columns = getActualTableStructure(context); - return std::make_shared( - StorageID(getDatabaseName(), table_name), configuration.value(), columns, ConstraintsDescription{}, String{}); + StorageID(getDatabaseName(), table_name), configuration.value(), ColumnsDescription{}, ConstraintsDescription{}, String{}); } ColumnsDescription TableFunctionMeiliSearch::getActualTableStructure(ContextPtr /* context */) const { - MeiliSearchColumnDescriptionFetcher fetcher(configuration.value()); - fetcher.addParam(doubleQuoteString("limit"), "1"); - return fetcher.fetchColumnsDescription(); + return StorageMeiliSearch::getTableStructureFromData(configuration.value()); } diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index 1080f12021f..e1b38f3df41 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -1,7 +1,6 @@ #include "config.h" #if USE_MYSQL -#include #include #include #include @@ -16,7 +15,7 @@ #include #include "registerTableFunctions.h" -#include // for fetchTablesColumnsList +#include #include @@ -61,15 +60,7 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr ColumnsDescription TableFunctionMySQL::getActualTableStructure(ContextPtr context) const { - const auto & settings = context->getSettingsRef(); - const auto tables_and_columns = fetchTablesColumnsList(*pool, configuration->database, {configuration->table}, settings, settings.mysql_datatypes_support_level); - - const auto columns = tables_and_columns.find(configuration->table); - if (columns == tables_and_columns.end()) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", - (configuration->database.empty() ? "" : (backQuote(configuration->database) + "." + backQuote(configuration->table)))); - - return columns->second; + return StorageMySQL::getTableStructureFromData(*pool, configuration->database, configuration->table, context); } StoragePtr TableFunctionMySQL::executeImpl( @@ -78,8 +69,6 @@ StoragePtr TableFunctionMySQL::executeImpl( const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - auto columns = getActualTableStructure(context); - auto res = std::make_shared( StorageID(getDatabaseName(), table_name), std::move(*pool), @@ -87,7 +76,7 @@ StoragePtr TableFunctionMySQL::executeImpl( configuration->table, configuration->replace_query, configuration->on_duplicate_clause, - columns, + ColumnsDescription{}, ConstraintsDescription{}, String{}, context, diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index ab6212d0e30..2b17a1b2c2b 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -1,8 +1,6 @@ #include #if USE_LIBPQXX -#include - #include #include #include @@ -24,14 +22,14 @@ namespace ErrorCodes StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - auto columns = getActualTableStructure(context); auto result = std::make_shared( StorageID(getDatabaseName(), table_name), connection_pool, configuration->table, - columns, + ColumnsDescription{}, ConstraintsDescription{}, String{}, + context, configuration->schema, configuration->on_conflict); @@ -42,15 +40,7 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(ContextPtr context) const { - const bool use_nulls = context->getSettingsRef().external_table_functions_use_nulls; - auto connection_holder = connection_pool->get(); - auto columns_info = fetchPostgreSQLTableStructure( - connection_holder->get(), configuration->table, configuration->schema, use_nulls).physical_columns; - - if (!columns_info) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table structure not returned"); - - return ColumnsDescription{columns_info->columns}; + return StoragePostgreSQL::getTableStructureFromData(connection_pool, configuration->table, configuration->schema, context); } diff --git a/src/TableFunctions/TableFunctionSQLite.cpp b/src/TableFunctions/TableFunctionSQLite.cpp index 13c6fcea60c..fb3b348b966 100644 --- a/src/TableFunctions/TableFunctionSQLite.cpp +++ b/src/TableFunctions/TableFunctionSQLite.cpp @@ -5,7 +5,6 @@ #include #include -#include #include #include "registerTableFunctions.h" @@ -33,13 +32,11 @@ namespace ErrorCodes StoragePtr TableFunctionSQLite::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/) const { - auto columns = getActualTableStructure(context); - auto storage = std::make_shared(StorageID(getDatabaseName(), table_name), sqlite_db, database_path, remote_table_name, - columns, ConstraintsDescription{}, context); + ColumnsDescription{}, ConstraintsDescription{}, context); storage->startup(); return storage; @@ -48,12 +45,7 @@ StoragePtr TableFunctionSQLite::executeImpl(const ASTPtr & /*ast_function*/, ColumnsDescription TableFunctionSQLite::getActualTableStructure(ContextPtr /* context */) const { - auto columns = fetchSQLiteTableStructure(sqlite_db.get(), remote_table_name); - - if (!columns) - throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, "Failed to fetch table structure for {}", remote_table_name); - - return ColumnsDescription{*columns}; + return StorageSQLite::getTableStructureFromData(sqlite_db, remote_table_name); } diff --git a/tests/integration/test_storage_meilisearch/test.py b/tests/integration/test_storage_meilisearch/test.py index 66c32459251..ddcd7154154 100644 --- a/tests/integration/test_storage_meilisearch/test.py +++ b/tests/integration/test_storage_meilisearch/test.py @@ -57,10 +57,12 @@ def test_simple_select(started_cluster): push_data(client, table, data) + parameters = "'http://meili1:7700', 'new_table', ''" + node = started_cluster.instances["meili"] node.query("DROP TABLE IF EXISTS simple_meili_table") node.query( - "CREATE TABLE simple_meili_table(id UInt64, data String) ENGINE = MeiliSearch('http://meili1:7700', 'new_table', '')" + f"CREATE TABLE simple_meili_table(id UInt64, data String) ENGINE = MeiliSearch({parameters})" ) assert node.query("SELECT COUNT() FROM simple_meili_table") == "100\n" @@ -73,7 +75,25 @@ def test_simple_select(started_cluster): node.query("SELECT data FROM simple_meili_table WHERE id = 42") == hex(42 * 42) + "\n" ) + node.query( + f"CREATE TABLE simple_meili_table_auto_schema_engine ENGINE=MeiliSearch({parameters})" + ) + node.query( + f"CREATE TABLE simple_meili_table_auto_schema_function AS meilisearch({parameters})" + ) + + expected = "id\tInt64\t\t\t\t\t\ndata\tString\t\t\t\t\t\n" + assert ( + node.query("DESCRIBE TABLE simple_meili_table_auto_schema_engine") == expected + ) + assert ( + node.query("DESCRIBE TABLE simple_meili_table_auto_schema_function") == expected + ) + node.query("DROP TABLE simple_meili_table") + node.query("DROP TABLE simple_meili_table_auto_schema_engine") + node.query("DROP TABLE simple_meili_table_auto_schema_function") + table.delete() diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index 084e2eaa976..49629575ec7 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -307,6 +307,32 @@ def test_table_function(started_cluster): conn.close() +def test_schema_inference(started_cluster): + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) + drop_mysql_table(conn, "inference_table") + + with conn.cursor() as cursor: + cursor.execute( + "CREATE TABLE clickhouse.inference_table (id INT PRIMARY KEY, data BINARY(16) NOT NULL)" + ) + + parameters = "'mysql57:3306', 'clickhouse', 'inference_table', 'root', 'clickhouse'" + + node1.query( + f"CREATE TABLE mysql_schema_inference_engine ENGINE=MySQL({parameters})" + ) + node1.query(f"CREATE TABLE mysql_schema_inference_function AS mysql({parameters})") + + expected = "id\tInt32\t\t\t\t\t\ndata\tFixedString(16)\t\t\t\t\t\n" + assert node1.query("DESCRIBE TABLE mysql_schema_inference_engine") == expected + assert node1.query("DESCRIBE TABLE mysql_schema_inference_function") == expected + + node1.query("DROP TABLE mysql_schema_inference_engine") + node1.query("DROP TABLE mysql_schema_inference_function") + + drop_mysql_table(conn, "inference_table") + + def test_binary_type(started_cluster): conn = get_mysql_conn(started_cluster, cluster.mysql_ip) drop_mysql_table(conn, "binary_type") @@ -329,6 +355,7 @@ def test_binary_type(started_cluster): node1.query("SELECT * FROM {}".format(table_function)) == "42\tclickhouse\\0\\0\\0\\0\\0\\0\n" ) + drop_mysql_table(conn, "binary_type") def test_enum_type(started_cluster): diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 3b7aae1ccdc..696a962eefe 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -198,7 +198,9 @@ def test_non_default_scema(started_cluster): expected = node1.query("SELECT number FROM numbers(100)") assert result == expected - table_function = """postgresql('postgres1:5432', 'postgres', 'test_table', 'postgres', 'mysecretpassword', 'test_schema')""" + parameters = "'postgres1:5432', 'postgres', 'test_table', 'postgres', 'mysecretpassword', 'test_schema'" + table_function = f"postgresql({parameters})" + table_engine = f"PostgreSQL({parameters})" result = node1.query(f"SELECT * FROM {table_function}") assert result == expected @@ -224,10 +226,19 @@ def test_non_default_scema(started_cluster): expected = node1.query("SELECT number FROM numbers(200)") assert result == expected + node1.query(f"CREATE TABLE test.test_pg_auto_schema_engine ENGINE={table_engine}") + node1.query(f"CREATE TABLE test.test_pg_auto_schema_function AS {table_function}") + + expected = "a\tNullable(Int32)\t\t\t\t\t\n" + assert node1.query("DESCRIBE TABLE test.test_pg_auto_schema_engine") == expected + assert node1.query("DESCRIBE TABLE test.test_pg_auto_schema_function") == expected + cursor.execute("DROP SCHEMA test_schema CASCADE") cursor.execute('DROP SCHEMA "test.nice.schema" CASCADE') node1.query("DROP TABLE test.test_pg_table_schema") node1.query("DROP TABLE test.test_pg_table_schema_with_dots") + node1.query("DROP TABLE test.test_pg_auto_schema_engine") + node1.query("DROP TABLE test.test_pg_auto_schema_function") def test_concurrent_queries(started_cluster): diff --git a/tests/queries/0_stateless/01889_sqlite_read_write.reference b/tests/queries/0_stateless/01889_sqlite_read_write.reference index d6b1e6f41c1..9f2b382e41e 100644 --- a/tests/queries/0_stateless/01889_sqlite_read_write.reference +++ b/tests/queries/0_stateless/01889_sqlite_read_write.reference @@ -36,6 +36,11 @@ line1 1 line2 2 line3 3 line4 4 +test schema inference +col1 Nullable(String) +col2 Nullable(Int32) +col1 Nullable(String) +col2 Nullable(Int32) test path in clickhouse-local line1 1 line2 2 diff --git a/tests/queries/0_stateless/01889_sqlite_read_write.sh b/tests/queries/0_stateless/01889_sqlite_read_write.sh index e732155dbcc..7691d2e3c2c 100755 --- a/tests/queries/0_stateless/01889_sqlite_read_write.sh +++ b/tests/queries/0_stateless/01889_sqlite_read_write.sh @@ -87,6 +87,14 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO TABLE FUNCTION sqlite('${DB_PATH}', 't ${CLICKHOUSE_CLIENT} --query="SELECT * FROM sqlite('${DB_PATH}', 'table1') ORDER BY col2" +${CLICKHOUSE_CLIENT} --query="select 'test schema inference'"; +${CLICKHOUSE_CLIENT} --query="CREATE TABLE sqlite_table3_inferred_engine ENGINE = SQLite('${DB_PATH}', 'table3')" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE sqlite_table3_inferred_function AS sqlite('${DB_PATH}', 'table3')" +${CLICKHOUSE_CLIENT} --query="DESCRIBE TABLE sqlite_table3_inferred_engine;" +${CLICKHOUSE_CLIENT} --query="DESCRIBE TABLE sqlite_table3_inferred_function;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE sqlite_table3_inferred_engine;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE sqlite_table3_inferred_function;" + sqlite3 "${DB_PATH2}" 'DROP TABLE IF EXISTS table1' sqlite3 "${DB_PATH2}" 'CREATE TABLE table1 (col1 text, col2 smallint);' sqlite3 "${DB_PATH2}" "INSERT INTO table1 VALUES ('line1', 1), ('line2', 2), ('line3', 3)" From ef459567133de08446312e46aabd630bbb072eaa Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 19 May 2023 01:31:45 +0000 Subject: [PATCH 0393/2223] Fix style --- src/Storages/StorageMySQL.cpp | 1 + src/Storages/StoragePostgreSQL.cpp | 1 + src/TableFunctions/TableFunctionMySQL.cpp | 1 - src/TableFunctions/TableFunctionSQLite.cpp | 1 - 4 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 506b9659e3a..2a96581d591 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -31,6 +31,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_TABLE; } static String backQuoteMySQL(const String & x) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 2285099d00b..3ef73edde6f 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -53,6 +53,7 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; } StoragePostgreSQL::StoragePostgreSQL( diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index e1b38f3df41..1705dfcbfa5 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -25,7 +25,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int UNKNOWN_TABLE; } void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr context) diff --git a/src/TableFunctions/TableFunctionSQLite.cpp b/src/TableFunctions/TableFunctionSQLite.cpp index fb3b348b966..a9831363bd9 100644 --- a/src/TableFunctions/TableFunctionSQLite.cpp +++ b/src/TableFunctions/TableFunctionSQLite.cpp @@ -25,7 +25,6 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; - extern const int SQLITE_ENGINE_ERROR; } From e84f0895e72f25c7200993a1fd8ac8e11bd7f9b9 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 16 May 2023 19:27:30 +0000 Subject: [PATCH 0394/2223] Support hardlinking parts transactionally --- .../MergeTree/DataPartStorageOnDiskBase.cpp | 27 +++++++--- .../MergeTree/DataPartStorageOnDiskBase.h | 3 +- src/Storages/MergeTree/IDataPartStorage.h | 8 ++- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/localBackup.cpp | 50 ++++++++++++++----- src/Storages/MergeTree/localBackup.h | 8 +-- tests/integration/test_grpc_protocol/test.py | 2 +- .../test_grpc_protocol_ssl/test.py | 2 +- tests/integration/test_server_reload/test.py | 2 +- 9 files changed, 76 insertions(+), 28 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index ebe55ea7dc7..cfc3ff58f81 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -411,25 +411,38 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::freeze( bool make_source_readonly, std::function save_metadata_callback, bool copy_instead_of_hardlink, - const NameSet & files_to_copy_instead_of_hardlinks) const + const NameSet & files_to_copy_instead_of_hardlinks, + DiskTransactionPtr external_transaction) const { auto disk = volume->getDisk(); - disk->createDirectories(to); + if (external_transaction) + external_transaction->createDirectories(to); + else + disk->createDirectories(to); - localBackup(disk, getRelativePath(), fs::path(to) / dir_path, make_source_readonly, {}, copy_instead_of_hardlink, files_to_copy_instead_of_hardlinks); + localBackup(disk, getRelativePath(), fs::path(to) / dir_path, make_source_readonly, {}, copy_instead_of_hardlink, files_to_copy_instead_of_hardlinks, external_transaction); if (save_metadata_callback) save_metadata_callback(disk); - disk->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt"); - disk->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt"); - disk->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME); + if (external_transaction) + { + external_transaction->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt"); + external_transaction->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt"); + external_transaction->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME); + } + else + { + disk->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt"); + disk->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt"); + disk->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME); + } auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); /// Do not initialize storage in case of DETACH because part may be broken. bool to_detached = dir_path.starts_with("detached/"); - return create(single_disk_volume, to, dir_path, /*initialize=*/ !to_detached); + return create(single_disk_volume, to, dir_path, /*initialize=*/ !to_detached && !external_transaction); } MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart( diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 09eb7f008bc..6b27b7296fc 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -64,7 +64,8 @@ public: bool make_source_readonly, std::function save_metadata_callback, bool copy_instead_of_hardlink, - const NameSet & files_to_copy_instead_of_hardlinks) const override; + const NameSet & files_to_copy_instead_of_hardlinks, + DiskTransactionPtr external_transaction) const override; MutableDataPartStoragePtr clonePart( const std::string & to, diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 7c85469d890..f160254350d 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB { @@ -212,13 +213,18 @@ public: /// implementation which relies on paths of some blobs in S3. For example if we want to hardlink /// the whole part during mutation we shouldn't hardlink checksums.txt, because otherwise /// zero-copy locks for different parts will be on the same path in zookeeper. + /// + /// If `external_transaction` is provided, the disk operations (creating directories, hardlinking, + /// etc) won't be applied immediately; instead, they'll be added to external_transaction, which the + /// caller then needs to commit. virtual std::shared_ptr freeze( const std::string & to, const std::string & dir_path, bool make_source_readonly, std::function save_metadata_callback, bool copy_instead_of_hardlink, - const NameSet & files_to_copy_instead_of_hardlinks) const = 0; + const NameSet & files_to_copy_instead_of_hardlinks, + DiskTransactionPtr external_transaction = nullptr) const = 0; /// Make a full copy of a data part into 'to/dir_path' (possibly to a different disk). virtual std::shared_ptr clonePart( diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ff40c1da8d1..388d96314c0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -161,7 +161,7 @@ public: void remove(); /// Initialize columns (from columns.txt if exists, or create from column files if not). - /// Load checksums from checksums.txt if exists. Load index if required. + /// Load various metadata into memory: checksums from checksums.txt, index if required, etc. void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency); void appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection = false) const; diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index 3b05e3df8d3..6faacf3c066 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -17,9 +17,10 @@ namespace { void localBackupImpl( - const DiskPtr & disk, const String & source_path, + const DiskPtr & disk, IDiskTransaction * transaction, const String & source_path, const String & destination_path, bool make_source_readonly, size_t level, - std::optional max_level, const NameSet & files_to_copy_instead_of_hardlinks) + std::optional max_level, bool copy_instead_of_hardlinks, + const NameSet & files_to_copy_instead_of_hardlinks) { if (max_level && level > *max_level) return; @@ -27,7 +28,10 @@ void localBackupImpl( if (level >= 1000) throw DB::Exception(DB::ErrorCodes::TOO_DEEP_RECURSION, "Too deep recursion"); - disk->createDirectories(destination_path); + if (transaction) + transaction->createDirectories(destination_path); + else + disk->createDirectories(destination_path); for (auto it = disk->iterateDirectory(source_path); it->isValid(); it->next()) { @@ -37,15 +41,36 @@ void localBackupImpl( if (!disk->isDirectory(source)) { if (make_source_readonly) - disk->setReadOnly(source); - if (files_to_copy_instead_of_hardlinks.contains(it->name())) - disk->copyFile(source, *disk, destination); + { + if (transaction) + transaction->setReadOnly(source); + else + disk->setReadOnly(source); + } + if (copy_instead_of_hardlinks || files_to_copy_instead_of_hardlinks.contains(it->name())) + { + if (transaction) + { + transaction->copyFile(source, destination); + } + else + { + disk->copyFile(source, *disk, destination); + } + } else - disk->createHardLink(source, destination); + { + if (transaction) + transaction->createHardLink(source, destination); + else + disk->createHardLink(source, destination); + } } else { - localBackupImpl(disk, source, destination, make_source_readonly, level + 1, max_level, files_to_copy_instead_of_hardlinks); + localBackupImpl( + disk, transaction, source, destination, make_source_readonly, level + 1, max_level, + copy_instead_of_hardlinks, files_to_copy_instead_of_hardlinks); } } } @@ -89,7 +114,7 @@ private: void localBackup( const DiskPtr & disk, const String & source_path, const String & destination_path, bool make_source_readonly, - std::optional max_level, bool copy_instead_of_hardlinks, const NameSet & files_to_copy_intead_of_hardlinks) + std::optional max_level, bool copy_instead_of_hardlinks, const NameSet & files_to_copy_intead_of_hardlinks, DiskTransactionPtr disk_transaction) { if (disk->exists(destination_path) && !disk->isDirectoryEmpty(destination_path)) { @@ -100,7 +125,8 @@ void localBackup( size_t try_no = 0; const size_t max_tries = 10; - CleanupOnFail cleanup([disk, destination_path]() { disk->removeRecursive(destination_path); }); + CleanupOnFail cleanup(disk_transaction ? std::function([]{}) : + [disk, destination_path]() { disk->removeRecursive(destination_path); }); /** Files in the directory can be permanently added and deleted. * If some file is deleted during an attempt to make a backup, then try again, @@ -110,10 +136,10 @@ void localBackup( { try { - if (copy_instead_of_hardlinks) + if (copy_instead_of_hardlinks && !disk_transaction) disk->copyDirectoryContent(source_path, disk, destination_path); else - localBackupImpl(disk, source_path, destination_path, make_source_readonly, 0, max_level, files_to_copy_intead_of_hardlinks); + localBackupImpl(disk, disk_transaction.get(), source_path, destination_path, make_source_readonly, 0, max_level, copy_instead_of_hardlinks, files_to_copy_intead_of_hardlinks); } catch (const DB::ErrnoException & e) { diff --git a/src/Storages/MergeTree/localBackup.h b/src/Storages/MergeTree/localBackup.h index 74b188daff6..89906bf1d75 100644 --- a/src/Storages/MergeTree/localBackup.h +++ b/src/Storages/MergeTree/localBackup.h @@ -9,7 +9,7 @@ namespace DB /** Creates a local (at the same mount point) backup (snapshot) directory. * - * In the specified destination directory, it creates a hard links on all source-directory files + * In the specified destination directory, it creates hard links on all source-directory files * and in all nested directories, with saving (creating) all relative paths; * and also `chown`, removing the write permission. * @@ -17,9 +17,11 @@ namespace DB * and is intended to be used as a simple means of protection against a human or program error, * but not from a hardware failure. * - * If max_level is specified, than only files which depth relative source_path less or equal max_level will be copied. + * If max_level is specified, than only files with depth relative source_path less or equal max_level will be copied. * So, if max_level=0 than only direct file child are copied. + * + * If `transaction` is provided, the changes will be added to it instead of performend on disk. */ - void localBackup(const DiskPtr & disk, const String & source_path, const String & destination_path, bool make_source_readonly = true, std::optional max_level = {}, bool copy_instead_of_hardlinks = false, const NameSet & files_to_copy_intead_of_hardlinks = {}); + void localBackup(const DiskPtr & disk, const String & source_path, const String & destination_path, bool make_source_readonly = true, std::optional max_level = {}, bool copy_instead_of_hardlinks = false, const NameSet & files_to_copy_intead_of_hardlinks = {}, DiskTransactionPtr disk_transaction = nullptr); } diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 254b78667d5..dead4d447ec 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -42,7 +42,7 @@ node = cluster.add_instance( main_configs=["configs/grpc_config.xml"], # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 env_variables={ - "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS") + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") }, ) main_channel = None diff --git a/tests/integration/test_grpc_protocol_ssl/test.py b/tests/integration/test_grpc_protocol_ssl/test.py index f1a4475c1a5..4c7fe99f49d 100644 --- a/tests/integration/test_grpc_protocol_ssl/test.py +++ b/tests/integration/test_grpc_protocol_ssl/test.py @@ -44,7 +44,7 @@ node = cluster.add_instance( ], # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 env_variables={ - "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS") + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") }, ) diff --git a/tests/integration/test_server_reload/test.py b/tests/integration/test_server_reload/test.py index 1429713cb84..da5208799ac 100644 --- a/tests/integration/test_server_reload/test.py +++ b/tests/integration/test_server_reload/test.py @@ -36,7 +36,7 @@ instance = cluster.add_instance( with_zookeeper=True, # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 env_variables={ - "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS") + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") }, ) From c4f23e87f10be8f48c99be671a272347715676f9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 18:56:56 +0200 Subject: [PATCH 0395/2223] Export grower_type in HashTable Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashTable.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 8542ddae144..439b96d6976 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -753,6 +753,7 @@ protected: public: using key_type = Key; + using grower_type = Grower; using mapped_type = typename Cell::mapped_type; using value_type = typename Cell::value_type; using cell_type = Cell; From b44497fd4c1908b656bdd22538595084be63c9fe Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 16:33:46 +0200 Subject: [PATCH 0396/2223] Introduce PackedHashMap (HashMap with structure without padding) In case of you have HashMap with as the overhead of 38% can be crutial, especially if you have tons of keys. Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashMap.h | 18 +++++----- src/Common/HashTable/PackedHashMap.h | 46 ++++++++++++++++++++++++++ src/Interpreters/examples/hash_map.cpp | 31 +++++++++++++++++ 3 files changed, 87 insertions(+), 8 deletions(-) create mode 100644 src/Common/HashTable/PackedHashMap.h diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index f0045306fe8..5f4cb396822 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -9,6 +9,8 @@ /** NOTE HashMap could only be used for memmoveable (position independent) types. * Example: std::string is not position independent in libstdc++ with C++11 ABI or in libc++. * Also, key in hash table must be of type, that zero bytes is compared equals to zero key. + * + * Please keep in sync with PackedHashMap.h */ namespace DB @@ -53,13 +55,13 @@ PairNoInit, std::decay_t> makePairNoInit(First && fi } -template +template > struct HashMapCell { using Mapped = TMapped; using State = TState; - using value_type = PairNoInit; + using value_type = Pair; using mapped_type = Mapped; using key_type = Key; @@ -151,14 +153,14 @@ struct HashMapCell namespace std { - template - struct tuple_size> : std::integral_constant { }; + template + struct tuple_size> : std::integral_constant { }; - template - struct tuple_element<0, HashMapCell> { using type = Key; }; + template + struct tuple_element<0, HashMapCell> { using type = Key; }; - template - struct tuple_element<1, HashMapCell> { using type = TMapped; }; + template + struct tuple_element<1, HashMapCell> { using type = TMapped; }; } template diff --git a/src/Common/HashTable/PackedHashMap.h b/src/Common/HashTable/PackedHashMap.h new file mode 100644 index 00000000000..6b52a04aa49 --- /dev/null +++ b/src/Common/HashTable/PackedHashMap.h @@ -0,0 +1,46 @@ +#pragma once + +/// Packed versions HashMap, please keep in sync with HashMap.h + +#include + +/// A pair that does not initialize the elements, if not needed. +/// +/// NOTE: makePairNoInit() is omitted for PackedPairNoInit since it is not +/// required for PackedHashMap (see mergeBlockWithPipe() for details) +template +struct PackedPairNoInit +{ + First first; + Second second; + + PackedPairNoInit() {} /// NOLINT + + template + PackedPairNoInit(FirstValue && first_, NoInitTag) + : first(std::forward(first_)) + { + } + + template + PackedPairNoInit(FirstValue && first_, SecondValue && second_) + : first(std::forward(first_)) + , second(std::forward(second_)) + { + } +} __attribute__((packed)); + +/// Packed HashMap - HashMap with structure without padding +/// +/// Sometimes padding in structure can be crucial, consider the following +/// example as in this case the padding overhead +/// is 0.375, and this can be major in case of lots of keys. +/// +/// Note, there is no need to provide PackedHashSet, since it cannot have padding. +template < + typename Key, + typename Mapped, + typename Hash = DefaultHash, + typename Grower = HashTableGrower<>, + typename Allocator = HashTableAllocator> +using PackedHashMap = HashMapTable>, Hash, Grower, Allocator>; diff --git a/src/Interpreters/examples/hash_map.cpp b/src/Interpreters/examples/hash_map.cpp index fe2cbda019d..c0016bf8cd8 100644 --- a/src/Interpreters/examples/hash_map.cpp +++ b/src/Interpreters/examples/hash_map.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -278,5 +279,35 @@ int main(int argc, char ** argv) << std::endl; } + if (argc < 3 || std::stol(argv[2]) == 7) + { + Stopwatch watch; + + PackedHashMap map; + PackedHashMap::LookupResult it; + bool inserted; + + for (size_t i = 0; i < n; ++i) + { + map.emplace(data[i], it, inserted); + if (inserted) + { + new (&it->getMapped()) Value; + std::swap(it->getMapped(), value); + INIT + } + } + + watch.stop(); + std::cerr << std::fixed << std::setprecision(2) + << "PackedHashMap. Size: " << map.size() + << ", elapsed: " << watch.elapsedSeconds() + << " (" << n / watch.elapsedSeconds() << " elem/sec.)" +#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS + << ", collisions: " << map.getCollisions() +#endif + << std::endl; + } + return 0; } From 7b5d156cc5e4d53091a1d2930fe6def0fafe8b5e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 18:01:01 +0200 Subject: [PATCH 0397/2223] Optimize SPARSE_HASHED layout (by using PackedHashMap) In case you want dictionary optimized for memory, SPARSE_HASHED is not always gives you what you need. Consider the following example as , but this pair will also have a 6 byte padding (on amd64), so this is almost 40% of space wastage. And because of this padding, even google::sparse_hash_map, does not make picture better, in fact, sparse_hash_map is not very friendly to memory allocators (especially jemalloc). Here are some numbers for dictionary with 1e9 elements and UInt64 as key, and UInt16 as value: settings | load (sec) | read (sec) | read (million rows/s) | bytes_allocated | RSS HASHED upstream | - | - | - | - | 35GiB SPARSE_HASHED upstream | - | - | - | - | 26GiB - | - | - | - | - | - sparse_hash_map glibc hashbench | - | - | - | - | 17.5GiB sparse_hash_map packed allocator | 101.878 | 231.48 | 4.32 | - | 17.7GiB PackedHashMap | 15.514 | 42.35 | 23.61 | 20GiB | 22GiB As you can see PackedHashMap looks way more better then HASHED, and even better then SPARSE_HASHED, but slightly worse then sparse_hash_map with packed allocator (it is done with a custom patch to google sparse_hash_map). v2: rebase on top of bucket_count fix Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 74 ++-------- src/Dictionaries/HashedDictionary.h | 44 +----- .../HashedDictionaryCollectionTraits.h | 102 +++++++++++++ .../HashedDictionaryCollectionType.h | 134 ++++++++++++++++++ .../performance/hashed_dictionary_sharded.xml | 2 +- 5 files changed, 249 insertions(+), 107 deletions(-) create mode 100644 src/Dictionaries/HashedDictionaryCollectionTraits.h create mode 100644 src/Dictionaries/HashedDictionaryCollectionType.h diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 191adab8983..048c378b917 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -21,6 +21,7 @@ #include #include #include +#include namespace CurrentMetrics { @@ -28,21 +29,6 @@ namespace CurrentMetrics extern const Metric HashedDictionaryThreadsActive; } -namespace -{ - -/// NOTE: Trailing return type is explicitly specified for SFINAE. - -/// google::sparse_hash_map -template auto getKeyFromCell(const T & value) -> decltype(value->first) { return value->first; } // NOLINT -template auto getValueFromCell(const T & value) -> decltype(value->second) { return value->second; } // NOLINT - -/// HashMap -template auto getKeyFromCell(const T & value) -> decltype(value->getKey()) { return value->getKey(); } // NOLINT -template auto getValueFromCell(const T & value) -> decltype(value->getMapped()) { return value->getMapped(); } // NOLINT - -} - namespace DB { @@ -242,10 +228,7 @@ HashedDictionary::~HashedDictionary() CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("HashedDictDtor"); - if constexpr (sparse) - container.clear(); - else - container.clearAndShrink(); + clearContainer(container); }); ++hash_tables_count; @@ -834,12 +817,7 @@ void HashedDictionary::resize(size_t added if (unlikely(attributes_size == 0)) { size_t reserve_size = added_rows + no_attributes_containers.front().size(); - - if constexpr (sparse) - no_attributes_containers.front().resize(reserve_size); - else - no_attributes_containers.front().reserve(reserve_size); - + resizeContainer(no_attributes_containers.front(), reserve_size); return; } @@ -849,11 +827,7 @@ void HashedDictionary::resize(size_t added { auto & container = containers.front(); size_t reserve_size = added_rows + container.size(); - - if constexpr (sparse) - container.resize(reserve_size); - else - container.reserve(reserve_size); + resizeContainer(container, reserve_size); }); } } @@ -973,25 +947,9 @@ void HashedDictionary::calculateBytesAlloc { for (const auto & container : containers) { - using ContainerType = std::decay_t; - using AttributeValueType = typename ContainerType::mapped_type; - bytes_allocated += sizeof(container); - - if constexpr (sparse || std::is_same_v) - { - /// bucket_count() - Returns table size, that includes empty and deleted - /// size() - Returns table size, without empty and deleted - /// and since this is sparsehash, empty cells should not be significant, - /// and since items cannot be removed from the dictionary, deleted is also not important. - bytes_allocated += container.size() * (sizeof(KeyType) + sizeof(AttributeValueType)); - bucket_count += container.bucket_count(); - } - else - { - bytes_allocated += container.getBufferSizeInBytes(); - bucket_count += container.getBufferSizeInCells(); - } + bytes_allocated += getBufferSizeInBytes(container); + bucket_count += getBufferSizeInCells(container); } }); @@ -1010,17 +968,8 @@ void HashedDictionary::calculateBytesAlloc for (const auto & container : no_attributes_containers) { bytes_allocated += sizeof(container); - - if constexpr (sparse) - { - bytes_allocated += container.size() * (sizeof(KeyType)); - bucket_count += container.bucket_count(); - } - else - { - bytes_allocated += container.getBufferSizeInBytes(); - bucket_count += container.getBufferSizeInCells(); - } + bytes_allocated += getBufferSizeInBytes(container); + bucket_count += getBufferSizeInCells(container); } } @@ -1078,12 +1027,7 @@ Pipe HashedDictionary::read(const Names & keys.reserve(keys.size() + container.size()); for (const auto & key : container) - { - if constexpr (sparse) - keys.emplace_back(key); - else - keys.emplace_back(key.getKey()); - } + keys.emplace_back(getSetKeyFromCell(key)); } } diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 67683679647..08f5dfaa40d 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -4,17 +4,14 @@ #include #include #include -#include -#include -#include -#include #include #include #include #include #include +#include /** This dictionary stores all content in a hash table in memory * (a separate Key -> Value map for each attribute) @@ -136,42 +133,7 @@ public: private: template - using CollectionTypeNonSparse = std::conditional_t< - dictionary_key_type == DictionaryKeyType::Simple, - HashMap>, - HashMapWithSavedHash>>; - - using NoAttributesCollectionTypeNonSparse = std::conditional_t< - dictionary_key_type == DictionaryKeyType::Simple, - HashSet>, - HashSetWithSavedHash>>; - - /// Here we use sparse_hash_map with DefaultHash<> for the following reasons: - /// - /// - DefaultHash<> is used for HashMap - /// - DefaultHash<> (from HashTable/Hash.h> works better then std::hash<> - /// in case of sequential set of keys, but with random access to this set, i.e. - /// - /// SELECT number FROM numbers(3000000) ORDER BY rand() - /// - /// And even though std::hash<> works better in some other cases, - /// DefaultHash<> is preferred since the difference for this particular - /// case is significant, i.e. it can be 10x+. - template - using CollectionTypeSparse = std::conditional_t< - dictionary_key_type == DictionaryKeyType::Simple, - google::sparse_hash_map>, - google::sparse_hash_map>>; - - using NoAttributesCollectionTypeSparse = google::sparse_hash_set>; - - template - using CollectionType = std::conditional_t, CollectionTypeNonSparse>; - - template - using CollectionsHolder = std::vector>; - - using NoAttributesCollectionType = std::conditional_t; + using CollectionsHolder = std::vector::Type>; using NullableSet = HashSet>; using NullableSets = std::vector; @@ -269,7 +231,7 @@ private: BlockPtr update_field_loaded_block; std::vector> string_arenas; - std::vector no_attributes_containers; + std::vector::Type> no_attributes_containers; DictionaryHierarchicalParentToChildIndexPtr hierarchical_index; }; diff --git a/src/Dictionaries/HashedDictionaryCollectionTraits.h b/src/Dictionaries/HashedDictionaryCollectionTraits.h new file mode 100644 index 00000000000..20fc882a2fe --- /dev/null +++ b/src/Dictionaries/HashedDictionaryCollectionTraits.h @@ -0,0 +1,102 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/// sparse_hash_map/sparse_hash_set +template +concept IsGoogleSparseHashTable = std::is_same_v>>; + +template +concept IsStdMapCell = requires (V v) +{ + v->first; + v->second; +}; + +/// HashMap/HashMapWithSavedHash/HashSet/HashMapWithSavedHash/PackedHashMap and their Cells +template +concept IsBuiltinHashTable = ( + std::is_same_v, + typename C::grower_type>> || + std::is_same_v, + typename C::grower_type>> || + std::is_same_v, + typename C::grower_type>> || + std::is_same_v, + typename C::grower_type>> || + std::is_same_v, + typename C::grower_type>> +); + +template +concept IsBuiltinSetCell = requires (V v) +{ + v.getKey(); +}; + +template +concept IsBuiltinMapCell = requires (V v) +{ + v->getKey(); + v->getMapped(); +}; + +// NOLINTBEGIN(*) + +/// google::sparse_hash_map +template auto getSetKeyFromCell(const T & value) { return value; } +template auto getKeyFromCell(const T & value) requires (IsStdMapCell) { return value->first; } +template auto getValueFromCell(const T & value) requires (IsStdMapCell) { return value->second; } + +/// size() - returns table size, without empty and deleted +/// and since this is sparsehash, empty cells should not be significant, +/// and since items cannot be removed from the dictionary, deleted is also not important. +/// +/// NOTE: for google::sparse_hash_set value_type is Key, for sparse_hash_map +/// value_type is std::pair, and now we correctly takes into +/// account padding in structures, if any. +template auto getBufferSizeInBytes(const C & c) requires (IsGoogleSparseHashTable) { return c.size() * sizeof(typename C::value_type); } +/// bucket_count() - Returns table size, that includes empty and deleted +template auto getBufferSizeInCells(const C & c) requires (IsGoogleSparseHashTable) { return c.bucket_count(); } + +template auto resizeContainer(C & c, size_t size) requires (IsGoogleSparseHashTable) { return c.resize(size); } +template auto clearContainer(C & c) requires (IsGoogleSparseHashTable) { return c.clear(); } + +/// HashMap +template auto getSetKeyFromCell(const T & value) requires (IsBuiltinSetCell) { return value.getKey(); } +template auto getKeyFromCell(const T & value) requires (IsBuiltinMapCell) { return value->getKey(); } +template auto getValueFromCell(const T & value) requires (IsBuiltinMapCell) { return value->getMapped(); } + +template auto getBufferSizeInBytes(const C & c) requires (IsBuiltinHashTable) { return c.getBufferSizeInBytes(); } +template auto getBufferSizeInCells(const C & c) requires (IsBuiltinHashTable) { return c.getBufferSizeInCells(); } +template auto resizeContainer(C & c, size_t size) requires (IsBuiltinHashTable) { return c.reserve(size); } +template void clearContainer(C & c) requires (IsBuiltinHashTable) { return c.clearAndShrink(); } + +// NOLINTEND(*) + +} diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h new file mode 100644 index 00000000000..05bdfbbf00f --- /dev/null +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -0,0 +1,134 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/// HashMap with packed structure is better than google::sparse_hash_map if the +/// pair is small, for the sizeof(std::pair) == 16, RSS for hash +/// table with 1e9 elements will be: +/// +/// - google::sparse_hash_map : 26GiB +/// - HashMap : 35GiB +/// - PackedHashMap : 22GiB +/// - google::sparse_hash_map: 17GiB +/// +/// Also note here sizeof(std::pair<>) was used since google::sparse_hash_map +/// uses it to store , yes we can modify google::sparse_hash_map to work +/// with packed analog of std::pair, but the allocator overhead is still +/// significant, because of tons of reallocations (and those cannot be solved +/// with reserve() due to some internals of google::sparse_hash_map) and poor +/// jemalloc support of such pattern, which results in 33% fragmentation (in +/// comparison with glibc). +/// +/// Plus since google::sparse_hash_map cannot use packed structure, it will +/// have the same memory footprint for everything from UInt8 to UInt64 values +/// and so on. +/// +/// Returns true hen google::sparse_hash_map should be used, otherwise +/// PackedHashMap should be used instead. +template +constexpr bool useSparseHashForHashedDictionary() +{ + return sizeof(PackedPairNoInit) > 16; +} + +/// +/// Map (dictionary with attributes) +/// + +/// Type of the hash table for the dictionary. +template +struct HashedDictionaryMapType; + +/// Default implementation using builtin HashMap (for HASHED layout). +template +struct HashedDictionaryMapType +{ + using Type = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + HashMap>, + HashMapWithSavedHash>>; +}; + +/// Implementations for SPARSE_HASHED layout. +template +struct HashedDictionarySparseMapType; + +/// Implementation based on google::sparse_hash_map for SPARSE_HASHED. +template +struct HashedDictionarySparseMapType +{ + /// Here we use sparse_hash_map with DefaultHash<> for the following reasons: + /// + /// - DefaultHash<> is used for HashMap + /// - DefaultHash<> (from HashTable/Hash.h> works better then std::hash<> + /// in case of sequential set of keys, but with random access to this set, i.e. + /// + /// SELECT number FROM numbers(3000000) ORDER BY rand() + /// + /// And even though std::hash<> works better in some other cases, + /// DefaultHash<> is preferred since the difference for this particular + /// case is significant, i.e. it can be 10x+. + using Type = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + google::sparse_hash_map>, + google::sparse_hash_map>>; +}; + +/// Implementation based on PackedHashMap for SPARSE_HASHED. +template +struct HashedDictionarySparseMapType +{ + using Type = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + PackedHashMap>, + PackedHashMap>>; +}; +template +struct HashedDictionaryMapType + : public HashedDictionarySparseMapType< + dictionary_key_type, Key, Value, + /* use_sparse_hash= */ useSparseHashForHashedDictionary()> +{}; + +/// +/// Set (dictionary with attributes) +/// + +/// Type of the hash table for the dictionary. +template +struct HashedDictionarySetType; + +/// Default implementation using builtin HashMap (for HASHED layout). +template +struct HashedDictionarySetType +{ + using Type = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + HashSet>, + HashSetWithSavedHash>>; +}; + +/// Implementation for SPARSE_HASHED. +/// +/// NOTE: There is no implementation based on google::sparse_hash_set since +/// PackedHashMap is more optimal anyway (see comments for +/// useSparseHashForHashedDictionary()). +template +struct HashedDictionarySetType +{ + using Type = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + HashSet>, + HashSet>>; +}; + +} diff --git a/tests/performance/hashed_dictionary_sharded.xml b/tests/performance/hashed_dictionary_sharded.xml index 98c65fc49ea..d229ce02c49 100644 --- a/tests/performance/hashed_dictionary_sharded.xml +++ b/tests/performance/hashed_dictionary_sharded.xml @@ -22,7 +22,7 @@ CREATE TABLE simple_key_dictionary_source_table ( id UInt64, - value_int UInt64 + value_int UInt16 ) ENGINE = Memory From fb6f7631c20c4a1cc23b4743491804536d0b67ae Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 20:21:04 +0200 Subject: [PATCH 0398/2223] Add ability to pass grower for HashTable during creation Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashTable.h | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 439b96d6976..0e8510d1c8d 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -771,6 +771,14 @@ public: alloc(grower); } + explicit HashTable(const Grower & grower_) + : grower(grower_) + { + if (Cell::need_zero_value_storage) + this->zeroValue()->setZero(); + alloc(grower); + } + HashTable(size_t reserve_for_num_elements) /// NOLINT { if (Cell::need_zero_value_storage) From 8c6d691f52dc09958088f6b0b5f6820c4a70b47d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 21:06:18 +0200 Subject: [PATCH 0399/2223] Use HashTable constructor in HashSet Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashSet.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/HashTable/HashSet.h b/src/Common/HashTable/HashSet.h index 8f3761599ab..c25bfb14d9c 100644 --- a/src/Common/HashTable/HashSet.h +++ b/src/Common/HashTable/HashSet.h @@ -41,6 +41,8 @@ public: using Base = HashTable; using typename Base::LookupResult; + using Base::Base; + void merge(const Self & rhs) { if (!this->hasZero() && rhs.hasZero()) From 3698302ddb32b489aa7e3ad556318f8e0a39f0c0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 21:28:20 +0200 Subject: [PATCH 0400/2223] Accept float values for dictionary layouts configurations Signed-off-by: Azat Khuzhin --- src/Dictionaries/getDictionaryConfigurationFromAST.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 05065df5251..0b7352e9cbb 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -156,11 +156,11 @@ void buildLayoutConfiguration( const auto value_field = value_literal->value; - if (value_field.getType() != Field::Types::UInt64 && value_field.getType() != Field::Types::String) + if (value_field.getType() != Field::Types::UInt64 && value_field.getType() != Field::Types::Float64 && value_field.getType() != Field::Types::String) { throw DB::Exception( ErrorCodes::BAD_ARGUMENTS, - "Dictionary layout parameter value must be an UInt64 or String, got '{}' instead", + "Dictionary layout parameter value must be an UInt64, Float64 or String, got '{}' instead", value_field.getTypeName()); } From 2996b3860612b6990867c197d630fc50528ec5b8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 1 May 2023 20:34:47 +0200 Subject: [PATCH 0401/2223] Add ability to configure maximum load factor for the HASHED/SPARSE_HASHED layout As it turns out, HashMap/PackedHashMap works great even with max load factor of 0.99. By "great" I mean it least it works faster then google sparsehash, and not to mention it's friendliness to the memory allocator (it has zero fragmentation since it works with a continuious memory region, in comparison to the sparsehash that doing lots of realloc, which jemalloc does not like, due to it's slabs). Here is a table of different setups: settings | load (sec) | read (sec) | read (million rows/s) | bytes_allocated | RSS - | - | - | - | - | - HASHED upstream | - | - | - | - | 35GiB SPARSE_HASHED upstream | - | - | - | - | 26GiB - | - | - | - | - | - sparse_hash_map glibc hashbench | - | - | - | - | 17.5GiB sparse_hash_map packed allocator | 101.878 | 231.48 | 4.32 | - | 17.7GiB PackedHashMap 0.5 | 15.514 | 42.35 | 23.61 | 20GiB | 22GiB hashed 0.95 | 34.903 | 115.615 | 8.65 | 16GiB | 18.7GiB **PackedHashMap 0.95** | **93.6** | **19.883** | **10.68** | **10GiB** | **12.8GiB** PackedHashMap 0.99 | 26.113 | 83.6 | 11.96 | 10GiB | 12.3GiB As it shows, PackedHashMap with 0.95 max_load_factor, eats 2.6x less memory then SPARSE_HASHED in upstream, and it also 2x faster for read! v2: fix grower Signed-off-by: Azat Khuzhin --- docs/en/sql-reference/dictionaries/index.md | 8 ++ src/Dictionaries/HashedDictionary.cpp | 30 ++++- src/Dictionaries/HashedDictionary.h | 1 + .../HashedDictionaryCollectionType.h | 102 +++++++++++++++-- .../hashed_dictionary_load_factor.xml | 92 +++++++++++++++ ...hed_dictionary_load_factor_smoke.reference | 12 ++ ...40_hashed_dictionary_load_factor_smoke.sql | 107 ++++++++++++++++++ ...41_hashed_dictionary_load_factor.reference | 4 + ...02741_hashed_dictionary_load_factor.sql.j2 | 39 +++++++ 9 files changed, 384 insertions(+), 11 deletions(-) create mode 100644 tests/performance/hashed_dictionary_load_factor.xml create mode 100644 tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.reference create mode 100644 tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.sql create mode 100644 tests/queries/0_stateless/02741_hashed_dictionary_load_factor.reference create mode 100644 tests/queries/0_stateless/02741_hashed_dictionary_load_factor.sql.j2 diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 5801b7866cb..dfda1ff9c04 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -284,6 +284,13 @@ Configuration example: 10000 is good balance between memory and speed. Even for 10e10 elements and can handle all the load without starvation. --> 10000 + + 0.5 ``` @@ -327,6 +334,7 @@ Configuration example: 1 + ``` diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 048c378b917..fa9a846195d 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -630,6 +630,8 @@ void HashedDictionary::createAttributes() const auto size = dict_struct.attributes.size(); attributes.reserve(size); + HashTableGrowerWithMaxLoadFactor grower(configuration.max_load_factor); + for (const auto & dictionary_attribute : dict_struct.attributes) { auto type_call = [&, this](const auto & dictionary_attribute_type) @@ -639,8 +641,23 @@ void HashedDictionary::createAttributes() using ValueType = DictionaryValueType; auto is_nullable_sets = dictionary_attribute.is_nullable ? std::make_optional(configuration.shards) : std::optional{}; - Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_sets), CollectionsHolder(configuration.shards)}; - attributes.emplace_back(std::move(attribute)); + if constexpr (IsBuiltinHashTable::value_type>) + { + CollectionsHolder collections; + collections.reserve(configuration.shards); + for (size_t i = 0; i < configuration.shards; ++i) + collections.emplace_back(grower); + + Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_sets), std::move(collections)}; + attributes.emplace_back(std::move(attribute)); + } + else + { + Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_sets), CollectionsHolder(configuration.shards)}; + for (auto & container : std::get>(attribute.containers)) + container.max_load_factor(configuration.max_load_factor); + attributes.emplace_back(std::move(attribute)); + } }; callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call); @@ -648,7 +665,9 @@ void HashedDictionary::createAttributes() if (unlikely(attributes.size()) == 0) { - no_attributes_containers.resize(configuration.shards); + no_attributes_containers.reserve(configuration.shards); + for (size_t i = 0; i < configuration.shards; ++i) + no_attributes_containers.emplace_back(grower); } string_arenas.resize(configuration.shards); @@ -1136,9 +1155,14 @@ void registerDictionaryHashed(DictionaryFactory & factory) if (shard_load_queue_backlog <= 0) throw Exception(ErrorCodes::BAD_ARGUMENTS,"{}: SHARD_LOAD_QUEUE_BACKLOG parameter should be greater then zero", full_name); + float max_load_factor = static_cast(config.getDouble(config_prefix + dictionary_layout_prefix + ".max_load_factor", 0.5)); + if (max_load_factor < 0.5 || max_load_factor > 0.99) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: max_load_factor parameter should be within [0.5, 0.99], got {}", full_name, max_load_factor); + HashedDictionaryConfiguration configuration{ static_cast(shards), static_cast(shard_load_queue_backlog), + max_load_factor, require_nonempty, dict_lifetime, }; diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 08f5dfaa40d..bff22706435 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -25,6 +25,7 @@ struct HashedDictionaryConfiguration { const UInt64 shards; const UInt64 shard_load_queue_backlog; + const float max_load_factor; const bool require_nonempty; const DictionaryLifetime lifetime; }; diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index 05bdfbbf00f..1ab0ff8b0e3 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -40,6 +40,92 @@ constexpr bool useSparseHashForHashedDictionary() return sizeof(PackedPairNoInit) > 16; } +/// Grower with custom fill limit/load factor (instead of default 50%). +/// +/// It turns out that HashMap can outperform google::sparse_hash_map in case of +/// the structure size of not big, in terms of speed *and* memory. Even 99% of +/// max load factor was faster then google::sparse_hash_map in my simple tests +/// (1e9 UInt64 keys with UInt16 values, randomly distributed). +/// +/// And not to mention very high allocator memory fragmentation in +/// google::sparse_hash_map. +/// +/// Based on HashTableGrowerWithPrecalculation +class alignas(64) HashTableGrowerWithMaxLoadFactor +{ + static constexpr size_t initial_size_degree = 8; + UInt8 size_degree = initial_size_degree; + size_t precalculated_mask = (1ULL << initial_size_degree) - 1; + size_t precalculated_max_fill = 1ULL << (initial_size_degree - 1); + float max_load_factor = 0.5; + /// HashTableGrowerWithPrecalculation has 23, but to decrease memory usage + /// at least slightly 19 is used here. Also note, that for dictionaries it + /// is not that important since they are not that frequently loaded. + static constexpr size_t max_size_degree_quadratic = 19; + +public: + static constexpr auto initial_count = 1ULL << initial_size_degree; + + /// If collision resolution chains are contiguous, we can implement erase operation by moving the elements. + static constexpr auto performs_linear_probing_with_single_step = true; + + HashTableGrowerWithMaxLoadFactor() = default; + explicit HashTableGrowerWithMaxLoadFactor(float max_load_factor_) + : max_load_factor(max_load_factor_) + { + increaseSizeDegree(0); + } + + UInt8 sizeDegree() const { return size_degree; } + + void increaseSizeDegree(UInt8 delta) + { + size_degree += delta; + precalculated_mask = (1ULL << size_degree) - 1; + precalculated_max_fill = static_cast((1ULL << size_degree) * max_load_factor); + } + + /// The size of the hash table in the cells. + size_t bufSize() const { return 1ULL << size_degree; } + + /// From the hash value, get the cell number in the hash table. + size_t place(size_t x) const { return x & precalculated_mask; } + + /// The next cell in the collision resolution chain. + size_t next(size_t pos) const { return (pos + 1) & precalculated_mask; } + + /// Whether the hash table is sufficiently full. You need to increase the size of the hash table, or remove something unnecessary from it. + bool overflow(size_t elems) const { return elems > precalculated_max_fill; } + + /// Increase the size of the hash table. + void increaseSize() { increaseSizeDegree(size_degree >= max_size_degree_quadratic ? 1 : 2); } + + /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. + void set(size_t num_elems) + { + if (num_elems <= 1) + size_degree = initial_size_degree; + else if (initial_size_degree > static_cast(log2(num_elems - 1)) + 2) + size_degree = initial_size_degree; + else + { + /// Slightly more optimal than HashTableGrowerWithPrecalculation + /// and takes into account max_load_factor. + size_degree = static_cast(log2(num_elems - 1)) + 1; + if ((1ULL << size_degree) * max_load_factor < num_elems) + ++size_degree; + } + increaseSizeDegree(0); + } + + void setBufSize(size_t buf_size_) + { + size_degree = static_cast(log2(buf_size_ - 1) + 1); + increaseSizeDegree(0); + } +}; +static_assert(sizeof(HashTableGrowerWithMaxLoadFactor) == 64); + /// /// Map (dictionary with attributes) /// @@ -54,8 +140,8 @@ struct HashedDictionaryMapType>, - HashMapWithSavedHash>>; + HashMap, HashTableGrowerWithMaxLoadFactor>, + HashMapWithSavedHash, HashTableGrowerWithMaxLoadFactor>>; }; /// Implementations for SPARSE_HASHED layout. @@ -89,8 +175,8 @@ struct HashedDictionarySparseMapType>, - PackedHashMap>>; + PackedHashMap, HashTableGrowerWithMaxLoadFactor>, + PackedHashMap, HashTableGrowerWithMaxLoadFactor>>; }; template struct HashedDictionaryMapType @@ -113,8 +199,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet>, - HashSetWithSavedHash>>; + HashSet, HashTableGrowerWithMaxLoadFactor>, + HashSetWithSavedHash, HashTableGrowerWithMaxLoadFactor>>; }; /// Implementation for SPARSE_HASHED. @@ -127,8 +213,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet>, - HashSet>>; + HashSet, HashTableGrowerWithMaxLoadFactor>, + HashSet, HashTableGrowerWithMaxLoadFactor>>; }; } diff --git a/tests/performance/hashed_dictionary_load_factor.xml b/tests/performance/hashed_dictionary_load_factor.xml new file mode 100644 index 00000000000..dfc5c64b133 --- /dev/null +++ b/tests/performance/hashed_dictionary_load_factor.xml @@ -0,0 +1,92 @@ + + + + layout_suffix + + HASHED + SPARSE_HASHED + + + + + load_factor + + + 5 + 7 + 99 + + + + + + CREATE TABLE simple_key_dictionary_source_table + ( + id UInt64, + value_int UInt16 + ) ENGINE = Memory + + + + CREATE TABLE complex_key_dictionary_source_table + ( + id UInt64, + id_key String, + value_int UInt64 + ) ENGINE = Memory + + + + CREATE DICTIONARY IF NOT EXISTS simple_key_{layout_suffix}_dictionary_l0_{load_factor} + ( + id UInt64, + value_int UInt64 + ) + PRIMARY KEY id + SOURCE(CLICKHOUSE(TABLE 'simple_key_dictionary_source_table')) + LAYOUT({layout_suffix}(MAX_LOAD_FACTOR 0.{load_factor})) + LIFETIME(0) + + + + CREATE DICTIONARY IF NOT EXISTS complex_key_{layout_suffix}_dictionary_l0_{load_factor} + ( + id UInt64, + id_key String, + value_int UInt64 + ) + PRIMARY KEY id, id_key + SOURCE(CLICKHOUSE(TABLE 'complex_key_dictionary_source_table')) + LAYOUT(COMPLEX_KEY_{layout_suffix}(MAX_LOAD_FACTOR 0.{load_factor})) + LIFETIME(0) + + + INSERT INTO simple_key_dictionary_source_table SELECT number, number FROM numbers(3_000_000) + INSERT INTO complex_key_dictionary_source_table SELECT number, toString(number), number FROM numbers(2_000_000) + + SYSTEM RELOAD DICTIONARY simple_key_{layout_suffix}_dictionary_l0_{load_factor} + SYSTEM RELOAD DICTIONARY complex_key_{layout_suffix}_dictionary_l0_{load_factor} + + SYSTEM RELOAD DICTIONARY simple_key_{layout_suffix}_dictionary_l0_{load_factor} + SYSTEM RELOAD DICTIONARY complex_key_{layout_suffix}_dictionary_l0_{load_factor} + + + WITH rand64() % 3_000_000 as key + SELECT dictHas('default.simple_key_{layout_suffix}_dictionary_l0_{load_factor}', key) + FROM numbers(3_000_000) + FORMAT Null + + + + WITH (rand64() % 2_000_000, toString(rand64() % 2_000_000)) as key + SELECT dictHas('default.complex_key_{layout_suffix}_dictionary_l0_{load_factor}', key) + FROM numbers(2_000_000) + FORMAT Null + + + DROP DICTIONARY simple_key_{layout_suffix}_dictionary_l0_{load_factor} + DROP DICTIONARY complex_key_{layout_suffix}_dictionary_l0_{load_factor} + + DROP TABLE simple_key_dictionary_source_table + DROP TABLE complex_key_dictionary_source_table + diff --git a/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.reference b/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.reference new file mode 100644 index 00000000000..d43fa8e734c --- /dev/null +++ b/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.reference @@ -0,0 +1,12 @@ +CREATE DICTIONARY default.test_sparse_dictionary_load_factor\n(\n `key` UInt64,\n `value` UInt16\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(TABLE test_table))\nLIFETIME(MIN 0 MAX 0)\nLAYOUT(SPARSE_HASHED(MAX_LOAD_FACTOR 0.9)) +100000 +0 +CREATE DICTIONARY default.test_dictionary_load_factor\n(\n `key` UInt64,\n `value` UInt16\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(TABLE test_table))\nLIFETIME(MIN 0 MAX 0)\nLAYOUT(HASHED(MAX_LOAD_FACTOR 0.9)) +100000 +0 +CREATE DICTIONARY default.test_dictionary_load_factor_nullable\n(\n `key` UInt64,\n `value` Nullable(UInt16)\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(TABLE test_table_nullable))\nLIFETIME(MIN 0 MAX 0)\nLAYOUT(HASHED(MAX_LOAD_FACTOR 0.9)) +100000 +0 +CREATE DICTIONARY default.test_complex_dictionary_load_factor\n(\n `key_1` UInt64,\n `key_2` UInt64,\n `value` UInt16\n)\nPRIMARY KEY key_1, key_2\nSOURCE(CLICKHOUSE(TABLE test_table_complex))\nLIFETIME(MIN 0 MAX 0)\nLAYOUT(COMPLEX_KEY_HASHED(MAX_LOAD_FACTOR 0.9)) +100000 +0 diff --git a/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.sql b/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.sql new file mode 100644 index 00000000000..d4bb9a1b14a --- /dev/null +++ b/tests/queries/0_stateless/02740_hashed_dictionary_load_factor_smoke.sql @@ -0,0 +1,107 @@ +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + key UInt64, + value UInt16 +) ENGINE=Memory() AS SELECT number, number FROM numbers(1e5); + +DROP TABLE IF EXISTS test_table_nullable; +CREATE TABLE test_table_nullable +( + key UInt64, + value Nullable(UInt16) +) ENGINE=Memory() AS SELECT number, number % 2 == 0 ? NULL : number FROM numbers(1e5); + +DROP TABLE IF EXISTS test_table_string; +CREATE TABLE test_table_string +( + key String, + value UInt16 +) ENGINE=Memory() AS SELECT 'foo' || number::String, number FROM numbers(1e5); + +DROP TABLE IF EXISTS test_table_complex; +CREATE TABLE test_table_complex +( + key_1 UInt64, + key_2 UInt64, + value UInt16 +) ENGINE=Memory() AS SELECT number, number, number FROM numbers(1e5); + +DROP DICTIONARY IF EXISTS test_sparse_dictionary_load_factor; +CREATE DICTIONARY test_sparse_dictionary_load_factor +( + key UInt64, + value UInt16 +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table)) +LAYOUT(SPARSE_HASHED(MAX_LOAD_FACTOR 0.90)) +LIFETIME(0); +SHOW CREATE test_sparse_dictionary_load_factor; +SYSTEM RELOAD DICTIONARY test_sparse_dictionary_load_factor; +SELECT element_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'test_sparse_dictionary_load_factor'; +SELECT count() FROM test_table WHERE dictGet('test_sparse_dictionary_load_factor', 'value', key) != value; +DROP DICTIONARY test_sparse_dictionary_load_factor; + +DROP DICTIONARY IF EXISTS test_dictionary_load_factor; +CREATE DICTIONARY test_dictionary_load_factor +( + key UInt64, + value UInt16 +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table)) +LAYOUT(HASHED(MAX_LOAD_FACTOR 0.90)) +LIFETIME(0); +SHOW CREATE test_dictionary_load_factor; +SYSTEM RELOAD DICTIONARY test_dictionary_load_factor; +SELECT element_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'test_dictionary_load_factor'; +SELECT count() FROM test_table WHERE dictGet('test_dictionary_load_factor', 'value', key) != value; +DROP DICTIONARY test_dictionary_load_factor; + +DROP DICTIONARY IF EXISTS test_dictionary_load_factor_nullable; +CREATE DICTIONARY test_dictionary_load_factor_nullable +( + key UInt64, + value Nullable(UInt16) +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table_nullable)) +LAYOUT(HASHED(MAX_LOAD_FACTOR 0.90)) +LIFETIME(0); +SHOW CREATE test_dictionary_load_factor_nullable; +SYSTEM RELOAD DICTIONARY test_dictionary_load_factor_nullable; +SELECT element_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'test_dictionary_load_factor_nullable'; +SELECT count() FROM test_table_nullable WHERE dictGet('test_dictionary_load_factor_nullable', 'value', key) != value; +DROP DICTIONARY test_dictionary_load_factor_nullable; + +DROP DICTIONARY IF EXISTS test_complex_dictionary_load_factor; +CREATE DICTIONARY test_complex_dictionary_load_factor +( + key_1 UInt64, + key_2 UInt64, + value UInt16 +) PRIMARY KEY key_1, key_2 +SOURCE(CLICKHOUSE(TABLE test_table_complex)) +LAYOUT(COMPLEX_KEY_HASHED(MAX_LOAD_FACTOR 0.90)) +LIFETIME(0); +SYSTEM RELOAD DICTIONARY test_complex_dictionary_load_factor; +SHOW CREATE test_complex_dictionary_load_factor; +SELECT element_count FROM system.dictionaries WHERE database = currentDatabase() and name = 'test_complex_dictionary_load_factor'; +SELECT count() FROM test_table_complex WHERE dictGet('test_complex_dictionary_load_factor', 'value', (key_1, key_2)) != value; +DROP DICTIONARY test_complex_dictionary_load_factor; + +DROP DICTIONARY IF EXISTS test_dictionary_load_factor_string; +CREATE DICTIONARY test_dictionary_load_factor_string +( + key String, + value UInt16 +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table_string)) +LAYOUT(HASHED(MAX_LOAD_FACTOR 1)) +LIFETIME(0); +-- should because of MAX_LOAD_FACTOR is 1 (maximum allowed value is 0.99) +SYSTEM RELOAD DICTIONARY test_dictionary_load_factor_string; -- { serverError BAD_ARGUMENTS } +DROP DICTIONARY test_dictionary_load_factor_string; + +DROP TABLE test_table; +DROP TABLE test_table_nullable; +DROP TABLE test_table_string; +DROP TABLE test_table_complex; diff --git a/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.reference b/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.reference new file mode 100644 index 00000000000..abe891cbb9b --- /dev/null +++ b/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.reference @@ -0,0 +1,4 @@ +test_dictionary_hashed 1000000 0.4768 33558760 +test_dictionary_hashed_load_factor 1000000 0.9537 16781544 +test_dictionary_sparse_hashed 1000000 0.4768 20975848 +test_dictionary_sparse_hashed_load_factor 1000000 0.9537 10490088 diff --git a/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.sql.j2 b/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.sql.j2 new file mode 100644 index 00000000000..870acd54514 --- /dev/null +++ b/tests/queries/0_stateless/02741_hashed_dictionary_load_factor.sql.j2 @@ -0,0 +1,39 @@ +{# vi: ft=sql #} + +{% for layout in ["hashed", "sparse_hashed"] %} +DROP DICTIONARY IF EXISTS test_dictionary_{{layout}}; +DROP DICTIONARY IF EXISTS test_dictionary_{{layout}}_load_factor; +DROP TABLE IF EXISTS test_table; + +CREATE TABLE test_table +( + key UInt64, + value UInt16 +) ENGINE=Memory() AS SELECT number, number FROM numbers(1e6); + +CREATE DICTIONARY test_dictionary_{{layout}} +( + key UInt64, + value UInt16 +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table)) +LAYOUT({{layout}}()) +LIFETIME(0); + +CREATE DICTIONARY test_dictionary_{{layout}}_load_factor +( + key UInt64, + value UInt16 +) PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE test_table)) +LAYOUT({{layout}}(MAX_LOAD_FACTOR 0.98)) +LIFETIME(0); + +SYSTEM RELOAD DICTIONARY test_dictionary_{{layout}}; +SYSTEM RELOAD DICTIONARY test_dictionary_{{layout}}_load_factor; +SELECT name, element_count, round(load_factor, 4), bytes_allocated FROM system.dictionaries WHERE database = currentDatabase() ORDER BY name; + +DROP DICTIONARY IF EXISTS test_dictionary_{{layout}}; +DROP DICTIONARY IF EXISTS test_dictionary_{{layout}}_load_factor; +DROP TABLE test_table; +{% endfor %} From 7c8d8eeb5677a22e324c1c2dd36c4036d21bce0e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 2 May 2023 20:55:47 +0200 Subject: [PATCH 0402/2223] Use Cell::setMapped() over separate helper insertSetMapped() Signed-off-by: Azat Khuzhin --- src/Common/HashTable/FixedHashTable.h | 2 +- src/Common/HashTable/HashTable.h | 15 ++------------- src/Common/HashTable/TwoLevelHashTable.h | 2 +- 3 files changed, 4 insertions(+), 15 deletions(-) diff --git a/src/Common/HashTable/FixedHashTable.h b/src/Common/HashTable/FixedHashTable.h index 7df90fd98b9..49675aaafbc 100644 --- a/src/Common/HashTable/FixedHashTable.h +++ b/src/Common/HashTable/FixedHashTable.h @@ -358,7 +358,7 @@ public: std::pair res; emplace(Cell::getKey(x), res.first, res.second); if (res.second) - insertSetMapped(res.first->getMapped(), x); + res.first->setMapped(x); return res; } diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 0e8510d1c8d..a48cb8d49f2 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -117,7 +117,7 @@ inline bool bitEquals(T && a, T && b) * 3) Hash tables that store the key and do not have a "mapped" value, e.g. the normal HashTable. * GetKey returns the key, and GetMapped returns a zero void pointer. This simplifies generic * code that works with mapped values: it can overload on the return type of GetMapped(), and - * doesn't need other parameters. One example is insertSetMapped() function. + * doesn't need other parameters. One example is Cell::setMapped() function. * * 4) Hash tables that store both the key and the "mapped" value, e.g. HashMap. Both GetKey and * GetMapped are supported. @@ -216,17 +216,6 @@ struct HashTableCell }; -/** - * A helper function for HashTable::insert() to set the "mapped" value. - * Overloaded on the mapped type, does nothing if it's VoidMapped. - */ -template -void insertSetMapped(VoidMapped /* dest */, const ValueType & /* src */) {} - -template -void insertSetMapped(MappedType & dest, const ValueType & src) { dest = src.second; } - - /** Determines the size of the hash table, and when and how much it should be resized. * Has very small state (one UInt8) and useful for Set-s allocated in automatic memory (see uniqExact as an example). */ @@ -1046,7 +1035,7 @@ public: } if (res.second) - insertSetMapped(res.first->getMapped(), x); + res.first->setMapped(x); return res; } diff --git a/src/Common/HashTable/TwoLevelHashTable.h b/src/Common/HashTable/TwoLevelHashTable.h index bd4c4c366f2..a60655e4072 100644 --- a/src/Common/HashTable/TwoLevelHashTable.h +++ b/src/Common/HashTable/TwoLevelHashTable.h @@ -224,7 +224,7 @@ public: emplace(Cell::getKey(x), res.first, res.second, hash_value); if (res.second) - insertSetMapped(res.first->getMapped(), x); + res.first->setMapped(x); return res; } From 65dd87d0da27feea18a3a5ea8b94c24213209357 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 2 May 2023 17:01:20 +0200 Subject: [PATCH 0403/2223] Fix "reference binding to misaligned address" in PackedHashMap Use separate helpers that accept/return values, instead of reference, anyway PackedHashMap is developed for small structure. v0: fix for keys v2: fix for values v3: fix bitEquals v4: fix for iterating over HashMap Signed-off-by: Azat Khuzhin --- base/base/StringRef.h | 12 ++++++ src/Common/HashTable/PackedHashMap.h | 63 +++++++++++++++++++++++++++- 2 files changed, 74 insertions(+), 1 deletion(-) diff --git a/base/base/StringRef.h b/base/base/StringRef.h index a3e32ff5058..f428b7c747f 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -3,6 +3,7 @@ #include #include // for std::logic_error #include +#include #include #include #include @@ -326,5 +327,16 @@ namespace ZeroTraits inline void set(StringRef & x) { x.size = 0; } } +namespace PackedZeroTraits +{ + template class PackedPairNoInit> + inline bool check(const PackedPairNoInit p) + { return 0 == p.key.size; } + + template class PackedPairNoInit> + inline void set(PackedPairNoInit & p) + { p.key.size = 0; } +} + std::ostream & operator<<(std::ostream & os, const StringRef & str); diff --git a/src/Common/HashTable/PackedHashMap.h b/src/Common/HashTable/PackedHashMap.h index 6b52a04aa49..6432c299b9d 100644 --- a/src/Common/HashTable/PackedHashMap.h +++ b/src/Common/HashTable/PackedHashMap.h @@ -30,6 +30,67 @@ struct PackedPairNoInit } } __attribute__((packed)); +/// The difference with ZeroTraits is that PackedZeroTraits accepts PackedPairNoInit instead of Key. +namespace PackedZeroTraits +{ + template class PackedPairNoInit> + bool check(const PackedPairNoInit p) { return p.first == First{}; } + + template class PackedPairNoInit> + void set(PackedPairNoInit & p) { p.first = First{}; } +} + +/// setZero() should be overwritten to pass the pair instead of key, to avoid +/// "reference binding to misaligned address" errors from UBsan. +template +struct PackedHashMapCell : public HashMapCell> +{ + using Base = HashMapCell>; + using State = typename Base::State; + using value_type = typename Base::value_type; + using key_type = typename Base::key_type; + using Mapped = typename Base::Mapped; + + using Base::Base; + + void setZero() { PackedZeroTraits::set(this->value); } + + Key getKey() const { return this->value.first; } + static Key getKey(const value_type & value_) { return value_.first; } + + Mapped & getMapped() { return this->value.second; } + Mapped getMapped() const { return this->value.second; } + value_type getValue() const { return this->value; } + + bool keyEquals(const Key key_) const { return bitEqualsByValue(this->value.first, key_); } + bool keyEquals(const Key key_, size_t /*hash_*/) const { return bitEqualsByValue(this->value.first, key_); } + bool keyEquals(const Key key_, size_t /*hash_*/, const State & /*state*/) const { return bitEqualsByValue(this->value.first, key_); } + + bool isZero(const State & state) const { return isZero(this->value.first, state); } + static bool isZero(const Key key, const State & /*state*/) { return ZeroTraits::check(key); } + + static inline bool bitEqualsByValue(key_type a, key_type b) { return a == b; } + + template + auto get() const + { + if constexpr (I == 0) return this->value.first; + else if constexpr (I == 1) return this->value.second; + } +}; + +namespace std +{ + template + struct tuple_size> : std::integral_constant { }; + + template + struct tuple_element<0, PackedHashMapCell> { using type = Key; }; + + template + struct tuple_element<1, PackedHashMapCell> { using type = TMapped; }; +} + /// Packed HashMap - HashMap with structure without padding /// /// Sometimes padding in structure can be crucial, consider the following @@ -43,4 +104,4 @@ template < typename Hash = DefaultHash, typename Grower = HashTableGrower<>, typename Allocator = HashTableAllocator> -using PackedHashMap = HashMapTable>, Hash, Grower, Allocator>; +using PackedHashMap = HashMapTable, Hash, Grower, Allocator>; From fc19e79f50686b7ed94165ac7827b088611a5bf7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 18:14:40 +0200 Subject: [PATCH 0404/2223] Change coding style of declaring packed attribute in PackedHashMap Signed-off-by: Azat Khuzhin --- src/Common/HashTable/PackedHashMap.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/HashTable/PackedHashMap.h b/src/Common/HashTable/PackedHashMap.h index 6432c299b9d..0d25addb58e 100644 --- a/src/Common/HashTable/PackedHashMap.h +++ b/src/Common/HashTable/PackedHashMap.h @@ -9,7 +9,7 @@ /// NOTE: makePairNoInit() is omitted for PackedPairNoInit since it is not /// required for PackedHashMap (see mergeBlockWithPipe() for details) template -struct PackedPairNoInit +struct __attribute__((packed)) PackedPairNoInit { First first; Second second; @@ -28,7 +28,7 @@ struct PackedPairNoInit , second(std::forward(second_)) { } -} __attribute__((packed)); +}; /// The difference with ZeroTraits is that PackedZeroTraits accepts PackedPairNoInit instead of Key. namespace PackedZeroTraits From 422cbe08fe83ca83f8ce24c48a811a1a0202abdb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 12:46:26 +0200 Subject: [PATCH 0405/2223] Do not use PackedHashMap for non-POD for the purposes of layout In clang-16 the behaviour for POD types had been changed in [1], this does not allows us to use PackedHashMap for some types. [1]: https://github.com/llvm/llvm-project/commit/277123376ce08c98b07c154bf83e4092a5d4d3c6 Note, that I tried to come up with a more generic solution then enumeratic types, but failed. Though now I think that this is good, since this shows which types are not allowed for PackedHashMap Another option is to use -fclang-abi-compat=13.0 but I doubt it is a good idea. Signed-off-by: Azat Khuzhin --- .../HashedDictionaryCollectionType.h | 36 ++++++++++++++++++- src/Interpreters/examples/hash_map.cpp | 30 ---------------- 2 files changed, 35 insertions(+), 31 deletions(-) diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index 1ab0ff8b0e3..23a08601fff 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -11,6 +12,33 @@ namespace DB { +/// Return true if the type is POD [1] for the purpose of layout (this is not +/// the same as STL traits has). +/// +/// [1]: https://stackoverflow.com/questions/4178175/what-are-aggregates-and-pods-and-how-why-are-they-special/4178176#4178176 +/// +/// The behaviour had been change in clang-16, see this for more details: +/// - https://github.com/llvm/llvm-project/commit/a8b0c6fa28acced71db33e80bd0b51d00422035b +/// - https://github.com/llvm/llvm-project/commit/277123376ce08c98b07c154bf83e4092a5d4d3c6 +/// - https://github.com/llvm/llvm-project/issues/62422 +/// - https://github.com/llvm/llvm-project/issues/62353 +/// - https://github.com/llvm/llvm-project/issues/62358 +template +constexpr bool isPodLayout() +{ + if constexpr (std::is_same_v) + return false; + if constexpr (std::is_same_v) + return false; + if constexpr (std::is_same_v || std::is_same_v || std::is_same_v || std::is_same_v) + return false; + if constexpr (std::is_same_v) + return false; + if constexpr (std::is_same_v || std::is_same_v) + return false; + return true; +} + /// HashMap with packed structure is better than google::sparse_hash_map if the /// pair is small, for the sizeof(std::pair) == 16, RSS for hash /// table with 1e9 elements will be: @@ -37,7 +65,13 @@ namespace DB template constexpr bool useSparseHashForHashedDictionary() { - return sizeof(PackedPairNoInit) > 16; + if constexpr (!isPodLayout()) + return true; + if constexpr (!isPodLayout()) + return true; + /// NOTE: One should not use PackedPairNoInit here since this will + /// create instantion of this type, and it could be illformed. + return sizeof(V) > 8; } /// Grower with custom fill limit/load factor (instead of default 50%). diff --git a/src/Interpreters/examples/hash_map.cpp b/src/Interpreters/examples/hash_map.cpp index c0016bf8cd8..f9fc36454f6 100644 --- a/src/Interpreters/examples/hash_map.cpp +++ b/src/Interpreters/examples/hash_map.cpp @@ -279,35 +279,5 @@ int main(int argc, char ** argv) << std::endl; } - if (argc < 3 || std::stol(argv[2]) == 7) - { - Stopwatch watch; - - PackedHashMap map; - PackedHashMap::LookupResult it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - { - new (&it->getMapped()) Value; - std::swap(it->getMapped(), value); - INIT - } - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "PackedHashMap. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } - return 0; } From 7eba6def94e23d8049197d431daa15bfee4f85e4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:35:02 +0200 Subject: [PATCH 0406/2223] Add a comment for HashTableGrowerWithPrecalculation about load factor Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashTable.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index a48cb8d49f2..2ff789caad2 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -270,6 +270,7 @@ struct HashTableGrower /** Determines the size of the hash table, and when and how much it should be resized. * This structure is aligned to cache line boundary and also occupies it all. * Precalculates some values to speed up lookups and insertion into the HashTable (and thus has bigger memory footprint than HashTableGrower). + * This grower assume 0.5 load factor */ template class alignas(64) HashTableGrowerWithPrecalculation From 1ab130132cba6af42521ad35f3aaf97110e768be Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:37:59 +0200 Subject: [PATCH 0407/2223] Add more comments into HashedDictionaryCollectionType.h Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionaryCollectionType.h | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index 23a08601fff..cdf29a82b66 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -160,6 +160,17 @@ public: }; static_assert(sizeof(HashTableGrowerWithMaxLoadFactor) == 64); +/// Above goes various specialisations for the hash table that will be used for +/// HASHED/SPARSE_HASHED dictionary, it could use one of the following depends +/// on the layout of the dictionary and types of key/value (for more info see +/// comments in this file): +/// - HashMap +/// - HashSet +/// - HashMapWithSavedHash +/// - HashSetWithSavedHash +/// - PackedHashMap +/// - google::sparse_hash_map + /// /// Map (dictionary with attributes) /// From 6f351851adf5c6bcd269e473339e0f99dd760f0a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:41:47 +0200 Subject: [PATCH 0408/2223] Rename grower to HashTableGrowerWithPrecalculationAndMaxLoadFactor Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 2 +- .../HashedDictionaryCollectionType.h | 24 +++++++++---------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index fa9a846195d..3903317cff2 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -630,7 +630,7 @@ void HashedDictionary::createAttributes() const auto size = dict_struct.attributes.size(); attributes.reserve(size); - HashTableGrowerWithMaxLoadFactor grower(configuration.max_load_factor); + HashTableGrowerWithPrecalculationAndMaxLoadFactor grower(configuration.max_load_factor); for (const auto & dictionary_attribute : dict_struct.attributes) { diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index cdf29a82b66..a631813bdbc 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -85,7 +85,7 @@ constexpr bool useSparseHashForHashedDictionary() /// google::sparse_hash_map. /// /// Based on HashTableGrowerWithPrecalculation -class alignas(64) HashTableGrowerWithMaxLoadFactor +class alignas(64) HashTableGrowerWithPrecalculationAndMaxLoadFactor { static constexpr size_t initial_size_degree = 8; UInt8 size_degree = initial_size_degree; @@ -103,8 +103,8 @@ public: /// If collision resolution chains are contiguous, we can implement erase operation by moving the elements. static constexpr auto performs_linear_probing_with_single_step = true; - HashTableGrowerWithMaxLoadFactor() = default; - explicit HashTableGrowerWithMaxLoadFactor(float max_load_factor_) + HashTableGrowerWithPrecalculationAndMaxLoadFactor() = default; + explicit HashTableGrowerWithPrecalculationAndMaxLoadFactor(float max_load_factor_) : max_load_factor(max_load_factor_) { increaseSizeDegree(0); @@ -158,7 +158,7 @@ public: increaseSizeDegree(0); } }; -static_assert(sizeof(HashTableGrowerWithMaxLoadFactor) == 64); +static_assert(sizeof(HashTableGrowerWithPrecalculationAndMaxLoadFactor) == 64); /// Above goes various specialisations for the hash table that will be used for /// HASHED/SPARSE_HASHED dictionary, it could use one of the following depends @@ -185,8 +185,8 @@ struct HashedDictionaryMapType, HashTableGrowerWithMaxLoadFactor>, - HashMapWithSavedHash, HashTableGrowerWithMaxLoadFactor>>; + HashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, + HashMapWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; }; /// Implementations for SPARSE_HASHED layout. @@ -220,8 +220,8 @@ struct HashedDictionarySparseMapType, HashTableGrowerWithMaxLoadFactor>, - PackedHashMap, HashTableGrowerWithMaxLoadFactor>>; + PackedHashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, + PackedHashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; }; template struct HashedDictionaryMapType @@ -244,8 +244,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet, HashTableGrowerWithMaxLoadFactor>, - HashSetWithSavedHash, HashTableGrowerWithMaxLoadFactor>>; + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, + HashSetWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; }; /// Implementation for SPARSE_HASHED. @@ -258,8 +258,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet, HashTableGrowerWithMaxLoadFactor>, - HashSet, HashTableGrowerWithMaxLoadFactor>>; + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; }; } From 42eac6bfbc6ef80866a685b98c3ae89cb9e9e450 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:43:11 +0200 Subject: [PATCH 0409/2223] Wrap implementation helpers into HashedDictionaryImpl namespace Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 2 ++ src/Dictionaries/HashedDictionary.h | 4 ++-- src/Dictionaries/HashedDictionaryCollectionTraits.h | 5 +++++ src/Dictionaries/HashedDictionaryCollectionType.h | 5 +++++ 4 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 3903317cff2..1ddcfae99b4 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -32,6 +32,8 @@ namespace CurrentMetrics namespace DB { +using namespace HashedDictionaryImpl; + namespace ErrorCodes { extern const int BAD_ARGUMENTS; diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index bff22706435..30eecb9ab09 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -134,7 +134,7 @@ public: private: template - using CollectionsHolder = std::vector::Type>; + using CollectionsHolder = std::vector::Type>; using NullableSet = HashSet>; using NullableSets = std::vector; @@ -232,7 +232,7 @@ private: BlockPtr update_field_loaded_block; std::vector> string_arenas; - std::vector::Type> no_attributes_containers; + std::vector::Type> no_attributes_containers; DictionaryHierarchicalParentToChildIndexPtr hierarchical_index; }; diff --git a/src/Dictionaries/HashedDictionaryCollectionTraits.h b/src/Dictionaries/HashedDictionaryCollectionTraits.h index 20fc882a2fe..4989664890a 100644 --- a/src/Dictionaries/HashedDictionaryCollectionTraits.h +++ b/src/Dictionaries/HashedDictionaryCollectionTraits.h @@ -10,6 +10,9 @@ namespace DB { +namespace HashedDictionaryImpl +{ + /// sparse_hash_map/sparse_hash_set template concept IsGoogleSparseHashTable = std::is_same_v void clearContainer(C & c) requires (IsBuiltinHashTable // NOLINTEND(*) } + +} diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index a631813bdbc..d1772b99c03 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -12,6 +12,9 @@ namespace DB { +namespace HashedDictionaryImpl +{ + /// Return true if the type is POD [1] for the purpose of layout (this is not /// the same as STL traits has). /// @@ -263,3 +266,5 @@ struct HashedDictionarySetType }; } + +} From 634f168a74b8451236407e0e861078bbd72723d7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:54:20 +0200 Subject: [PATCH 0410/2223] Introduce max_size_degree for HashTableGrower{,WithPrecalculation} Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashTable.h | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 2ff789caad2..d4577d7814e 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -230,6 +230,8 @@ struct HashTableGrower /// If collision resolution chains are contiguous, we can implement erase operation by moving the elements. static constexpr auto performs_linear_probing_with_single_step = true; + static constexpr size_t max_size_degree = 23; + /// The size of the hash table in the cells. size_t bufSize() const { return 1ULL << size_degree; } @@ -248,7 +250,7 @@ struct HashTableGrower /// Increase the size of the hash table. void increaseSize() { - size_degree += size_degree >= 23 ? 1 : 2; + size_degree += size_degree >= max_size_degree ? 1 : 2; } /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. @@ -280,6 +282,7 @@ class alignas(64) HashTableGrowerWithPrecalculation UInt8 size_degree = initial_size_degree; size_t precalculated_mask = (1ULL << initial_size_degree) - 1; size_t precalculated_max_fill = 1ULL << (initial_size_degree - 1); + static constexpr size_t max_size_degree = 23; public: UInt8 sizeDegree() const { return size_degree; } @@ -309,7 +312,7 @@ public: bool overflow(size_t elems) const { return elems > precalculated_max_fill; } /// Increase the size of the hash table. - void increaseSize() { increaseSizeDegree(size_degree >= 23 ? 1 : 2); } + void increaseSize() { increaseSizeDegree(size_degree >= max_size_degree ? 1 : 2); } /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. void set(size_t num_elems) From 01bf041ccad7921b39013798a4b9261b1c37919e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:57:43 +0200 Subject: [PATCH 0411/2223] Rewrite HashTableGrower{,WithPrecalculation}::set w/o ternary operators Signed-off-by: Azat Khuzhin --- src/Common/HashTable/HashTable.h | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index d4577d7814e..ca3e88c93a2 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -256,11 +256,12 @@ struct HashTableGrower /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. void set(size_t num_elems) { - size_degree = num_elems <= 1 - ? initial_size_degree - : ((initial_size_degree > static_cast(log2(num_elems - 1)) + 2) - ? initial_size_degree - : (static_cast(log2(num_elems - 1)) + 2)); + if (num_elems <= 1) + size_degree = initial_size_degree; + else if (initial_size_degree > static_cast(log2(num_elems - 1)) + 2) + size_degree = initial_size_degree; + else + size_degree = static_cast(log2(num_elems - 1)) + 2; } void setBufSize(size_t buf_size_) @@ -317,11 +318,12 @@ public: /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. void set(size_t num_elems) { - size_degree = num_elems <= 1 - ? initial_size_degree - : ((initial_size_degree > static_cast(log2(num_elems - 1)) + 2) - ? initial_size_degree - : (static_cast(log2(num_elems - 1)) + 2)); + if (num_elems <= 1) + size_degree = initial_size_degree; + else if (initial_size_degree > static_cast(log2(num_elems - 1)) + 2) + size_degree = initial_size_degree; + else + size_degree = static_cast(log2(num_elems - 1)) + 2; increaseSizeDegree(0); } From c9cde110cdaebb27ddfaa8792988ac9b5e5162f2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 19:59:43 +0200 Subject: [PATCH 0412/2223] Add initial degree as parameter for HashTableGrowerWithPrecalculationAndMaxLoadFactor Signed-off-by: Azat Khuzhin --- .../HashedDictionaryCollectionType.h | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index d1772b99c03..745c1716964 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -88,9 +88,9 @@ constexpr bool useSparseHashForHashedDictionary() /// google::sparse_hash_map. /// /// Based on HashTableGrowerWithPrecalculation +template class alignas(64) HashTableGrowerWithPrecalculationAndMaxLoadFactor { - static constexpr size_t initial_size_degree = 8; UInt8 size_degree = initial_size_degree; size_t precalculated_mask = (1ULL << initial_size_degree) - 1; size_t precalculated_max_fill = 1ULL << (initial_size_degree - 1); @@ -161,7 +161,7 @@ public: increaseSizeDegree(0); } }; -static_assert(sizeof(HashTableGrowerWithPrecalculationAndMaxLoadFactor) == 64); +static_assert(sizeof(HashTableGrowerWithPrecalculationAndMaxLoadFactor<>) == 64); /// Above goes various specialisations for the hash table that will be used for /// HASHED/SPARSE_HASHED dictionary, it could use one of the following depends @@ -188,8 +188,8 @@ struct HashedDictionaryMapType, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, - HashMapWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; + HashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>, + HashMapWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>>; }; /// Implementations for SPARSE_HASHED layout. @@ -223,8 +223,8 @@ struct HashedDictionarySparseMapType, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, - PackedHashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; + PackedHashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>, + PackedHashMap, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>>; }; template struct HashedDictionaryMapType @@ -247,8 +247,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, - HashSetWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>, + HashSetWithSavedHash, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>>; }; /// Implementation for SPARSE_HASHED. @@ -261,8 +261,8 @@ struct HashedDictionarySetType { using Type = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, - HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>, - HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor>>; + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>, + HashSet, HashTableGrowerWithPrecalculationAndMaxLoadFactor<>>>; }; } From f8e7d2cb1f186e60d4537f0069969ae440ccb217 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 20:02:13 +0200 Subject: [PATCH 0413/2223] Remove part of the HashTableGrowerWithPrecalculationAndMaxLoadFactor comment Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionaryCollectionType.h | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Dictionaries/HashedDictionaryCollectionType.h b/src/Dictionaries/HashedDictionaryCollectionType.h index 745c1716964..0746d42bd1f 100644 --- a/src/Dictionaries/HashedDictionaryCollectionType.h +++ b/src/Dictionaries/HashedDictionaryCollectionType.h @@ -79,14 +79,6 @@ constexpr bool useSparseHashForHashedDictionary() /// Grower with custom fill limit/load factor (instead of default 50%). /// -/// It turns out that HashMap can outperform google::sparse_hash_map in case of -/// the structure size of not big, in terms of speed *and* memory. Even 99% of -/// max load factor was faster then google::sparse_hash_map in my simple tests -/// (1e9 UInt64 keys with UInt16 values, randomly distributed). -/// -/// And not to mention very high allocator memory fragmentation in -/// google::sparse_hash_map. -/// /// Based on HashTableGrowerWithPrecalculation template class alignas(64) HashTableGrowerWithPrecalculationAndMaxLoadFactor From 2b240d3721f66f1e3178be602837194e06d1bc44 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 20:10:06 +0200 Subject: [PATCH 0414/2223] Improve documentation for HASHED/SPARSE_HASHED/COMPLEX_KEY_HASHED/COMPLEX_KEY_SPARSE_HASHED Signed-off-by: Azat Khuzhin --- docs/en/sql-reference/dictionaries/index.md | 27 ++++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index dfda1ff9c04..712a17308ba 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -267,14 +267,16 @@ or LAYOUT(HASHED()) ``` -If `shards` greater then 1 (default is `1`) the dictionary will load data in parallel, useful if you have huge amount of elements in one dictionary. - Configuration example: ``` xml + 10 + 10000 + + + + ``` or ``` sql -LAYOUT(SPARSE_HASHED()) +LAYOUT(SPARSE_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000] [MAX_LOAD_FACTOR 0.5])) ``` It is also possible to use `shards` for this type of dictionary, and again it is more important for `sparse_hashed` then for `hashed`, since `sparse_hashed` is slower. @@ -332,7 +339,7 @@ Configuration example: ``` xml - 1 + @@ -342,7 +349,7 @@ Configuration example: or ``` sql -LAYOUT(COMPLEX_KEY_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000])) +LAYOUT(COMPLEX_KEY_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000] [MAX_LOAD_FACTOR 0.5])) ``` ### complex_key_sparse_hashed @@ -354,7 +361,9 @@ Configuration example: ``` xml - 1 + + + ``` @@ -362,7 +371,7 @@ Configuration example: or ``` sql -LAYOUT(COMPLEX_KEY_SPARSE_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000])) +LAYOUT(COMPLEX_KEY_SPARSE_HASHED([SHARDS 1] [SHARD_LOAD_QUEUE_BACKLOG 10000] [MAX_LOAD_FACTOR 0.5])) ``` ### hashed_array From e1e2a83a9e78b5e56476f38478895b8826e2b3dd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 15 May 2023 20:28:28 +0200 Subject: [PATCH 0415/2223] Print type of the structure that will be used for HASHED/SPARSE_HASHED Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 1ddcfae99b4..83d051df5e9 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -660,6 +660,11 @@ void HashedDictionary::createAttributes() container.max_load_factor(configuration.max_load_factor); attributes.emplace_back(std::move(attribute)); } + + if constexpr (IsBuiltinHashTable::value_type>) + LOG_TRACE(log, "Using builtin hash table for {} attribute", dictionary_attribute.name); + else + LOG_TRACE(log, "Using sparsehash for {} attribute", dictionary_attribute.name); }; callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call); From e37e8f83bb28dc752c0264cee52dcbb7c4f24352 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 16 May 2023 15:57:26 +0200 Subject: [PATCH 0416/2223] Fix flakiness of test_distributed_load_balancing MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit I saw the following in the logs for the failed test: 2023.05.16 07:12:12.894051 [ 262 ] {74575ac0-b296-4fdc-bc8e-3476a305e6ea} ConnectionPoolWithFailover: Connection failed at try №1, reason: Timeout exceeded while reading from socket (socket (172.16.3.2:9000), receive timeout 2000 ms) And I think that the culprit is the test_distributed_replica_max_ignored_errors for which it is normal, however not for others, and this should not affect other tests. So fix this by calling SYSTEM RELOAD CONFIG, which should reset error count. CI: https://s3.amazonaws.com/clickhouse-test-reports/49380/5abc1a1c68ee204c9024493be1d19835cf5630f7/integration_tests__release__[3_4].html Signed-off-by: Azat Khuzhin --- .../test_distributed_load_balancing/test.py | 26 ++++++++----------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index 1dba6a30bc4..271828f433e 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -29,23 +29,19 @@ nodes = len(cluster.instances) queries = nodes * 10 +# SYSTEM RELOAD CONFIG will reset some attributes of the nodes in cluster +# - error_count +# - last_used (round_robing) +# +# This is required to avoid interference results of one test to another +@pytest.fixture(scope="function", autouse=True) +def test_setup(): + for n in list(cluster.instances.values()): + n.query("SYSTEM RELOAD CONFIG") + + def bootstrap(): for n in list(cluster.instances.values()): - # At startup, server loads configuration files. - # - # However ConfigReloader does not know about already loaded files - # (files is empty()), hence it will always reload the configuration - # just after server starts (+ 2 seconds, reload timeout). - # - # And on configuration reload the clusters will be re-created, so some - # internal stuff will be reset: - # - error_count - # - last_used (round_robing) - # - # And if the reload will happen during round_robin test it will start - # querying from the beginning, so let's issue config reload just after - # start to avoid reload in the middle of the test execution. - n.query("SYSTEM RELOAD CONFIG") n.query("DROP TABLE IF EXISTS data") n.query("DROP TABLE IF EXISTS dist") n.query("CREATE TABLE data (key Int) Engine=Memory()") From dc353faf44c466cd89391a50076979d6f933e1d1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 16 May 2023 15:59:22 +0200 Subject: [PATCH 0417/2223] Simplify obtaining query shard in test_distributed_load_balancing Signed-off-by: Azat Khuzhin --- .../test_distributed_load_balancing/test.py | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index 271828f433e..e879f09ccc1 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -109,19 +109,14 @@ def get_node(query_node, table="dist", *args, **kwargs): rows = query_node.query( """ - SELECT c.host_name - FROM ( - SELECT _shard_num - FROM cluster(shards_cluster, system.query_log) - WHERE - initial_query_id = '{query_id}' AND - is_initial_query = 0 AND - type = 'QueryFinish' - ORDER BY event_date DESC, event_time DESC - LIMIT 1 - ) a - JOIN system.clusters c - ON a._shard_num = c.shard_num WHERE cluster = 'shards_cluster' + SELECT hostName() + FROM cluster(shards_cluster, system.query_log) + WHERE + initial_query_id = '{query_id}' AND + is_initial_query = 0 AND + type = 'QueryFinish' + ORDER BY event_date DESC, event_time DESC + LIMIT 1 """.format( query_id=query_id ) From 4efb2037a2f7cbf8a2fa28bdeb89ae9f618c6fbb Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 19 May 2023 06:22:21 +0000 Subject: [PATCH 0418/2223] Fix the premature amendment of the 00921 test --- .../0_stateless/00921_datetime64_compatibility_long.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference index 55bcf000d10..2f56230db37 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference +++ b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference @@ -49,7 +49,7 @@ Code: 43 "UInt8",11 ------------------------------------------ SELECT toUnixTimestamp(N) -"UInt32",1568650811 +"UInt32",1568592000 "UInt32",1568650811 "UInt32",1568650811 ------------------------------------------ From f47375d16c1c89c2ffea087617f1b4dd23734f00 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 May 2023 10:28:13 +0200 Subject: [PATCH 0419/2223] Support Tableau --- programs/server/config.xml | 8 ++++++-- src/IO/Resource/DynamicResourceManager.cpp | 4 ++-- .../queries/0_stateless/02737_sql_auto_is_null.reference | 1 + tests/queries/0_stateless/02737_sql_auto_is_null.sql | 2 ++ 4 files changed, 11 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02737_sql_auto_is_null.reference create mode 100644 tests/queries/0_stateless/02737_sql_auto_is_null.sql diff --git a/programs/server/config.xml b/programs/server/config.xml index 82dd697084c..c6ae47b3050 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -719,8 +719,12 @@ default - - + + SQL_ diff --git a/src/IO/Resource/DynamicResourceManager.cpp b/src/IO/Resource/DynamicResourceManager.cpp index 49e12984e33..df0de6575f4 100644 --- a/src/IO/Resource/DynamicResourceManager.cpp +++ b/src/IO/Resource/DynamicResourceManager.cpp @@ -98,9 +98,9 @@ DynamicResourceManager::State::Resource::~Resource() if (attached_to != nullptr) { ISchedulerNode * root = nodes.find("/")->second.ptr.get(); - attached_to->event_queue->enqueue([scheduler = attached_to, root] + attached_to->event_queue->enqueue([my_scheduler = attached_to, root] { - scheduler->removeChild(root); + my_scheduler->removeChild(root); }); } } diff --git a/tests/queries/0_stateless/02737_sql_auto_is_null.reference b/tests/queries/0_stateless/02737_sql_auto_is_null.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02737_sql_auto_is_null.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02737_sql_auto_is_null.sql b/tests/queries/0_stateless/02737_sql_auto_is_null.sql new file mode 100644 index 00000000000..22f1a9524ca --- /dev/null +++ b/tests/queries/0_stateless/02737_sql_auto_is_null.sql @@ -0,0 +1,2 @@ +SET SQL_AUTO_IS_NULL = 0; +SELECT getSetting('SQL_AUTO_IS_NULL'); From 4dbe5b83294b4a2c23e844de68de6530c653504c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 May 2023 11:13:28 +0200 Subject: [PATCH 0420/2223] Support them in tests --- tests/config/config.d/custom_settings_prefixes.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/custom_settings_prefixes.xml b/tests/config/config.d/custom_settings_prefixes.xml index ec66c1a930e..9baaa4e09f3 100644 --- a/tests/config/config.d/custom_settings_prefixes.xml +++ b/tests/config/config.d/custom_settings_prefixes.xml @@ -1,4 +1,4 @@ - custom_ + SQL_,custom_ From 6e468b29e894a85b954626e24fba86a2d0a8f4fb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 10:15:06 +0000 Subject: [PATCH 0421/2223] Check return value of ftruncate --- src/Coordination/Changelog.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index ddedae4fa0f..7275a080252 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -279,7 +279,16 @@ private: flush(); if (log_file_settings.max_size != 0) - ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); + { + int res = -1; + do + { + res = ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); + } while (res < 0 && errno == EINTR); + + if (res != 0) + LOG_WARNING(log, "Could not ftruncate file. Error: {}, errno: {}", errnoToString(), errno); + } if (log_file_settings.compress_logs) compressed_buffer.reset(); From d6df009842007f6497b4de6f941d4957a925f5c3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 May 2023 12:22:46 +0200 Subject: [PATCH 0422/2223] Fix --- src/Interpreters/Cache/FileCache.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 2d5744b630e..78670bb49e4 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -609,17 +609,19 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) if (releasable) { - removed_size += segment_metadata->size(); - --queue_size; - auto segment = segment_metadata->file_segment; if (segment->state() == FileSegment::State::DOWNLOADED) { const auto & key = segment->key(); + auto it = to_delete.find(key); if (it == to_delete.end()) it = to_delete.emplace(key, locked_key.getKeyMetadata()).first; it->second.add(segment_metadata); + + removed_size += segment_metadata->size(); + --queue_size; + return PriorityIterationResult::CONTINUE; } From f5506210d64d39b2bddecdc32a724f78b85d47af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 May 2023 12:43:55 +0200 Subject: [PATCH 0423/2223] Geo types are production ready --- docs/en/operations/settings/settings.md | 11 ----------- docs/en/sql-reference/data-types/index.md | 4 ++-- docs/ru/operations/settings/settings.md | 10 ---------- docs/ru/sql-reference/data-types/geo.md | 9 --------- src/Core/Settings.h | 5 +++-- src/IO/Resource/DynamicResourceManager.cpp | 4 ++-- src/Interpreters/InterpreterCreateQuery.cpp | 15 --------------- .../parseColumnsListForTableFunction.cpp | 12 ------------ .../parseColumnsListForTableFunction.h | 2 -- tests/queries/0_stateless/01291_geo_types.sql | 2 -- .../0_stateless/02513_validate_data_types.sql | 5 ----- .../queries/0_stateless/02524_fuzz_and_fuss_2.sql | 2 +- 12 files changed, 8 insertions(+), 73 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cddde2090f8..dfc433717b7 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3213,17 +3213,6 @@ Possible values: Default value: `0`. -## allow_experimental_geo_types {#allow-experimental-geo-types} - -Allows working with experimental [geo data types](../../sql-reference/data-types/geo.md). - -Possible values: - -- 0 — Working with geo data types is disabled. -- 1 — Working with geo data types is enabled. - -Default value: `0`. - ## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously} Adds a modifier `SYNC` to all `DROP` and `DETACH` queries. diff --git a/docs/en/sql-reference/data-types/index.md b/docs/en/sql-reference/data-types/index.md index 2ad8ac4bb23..88663968e50 100644 --- a/docs/en/sql-reference/data-types/index.md +++ b/docs/en/sql-reference/data-types/index.md @@ -29,5 +29,5 @@ ClickHouse data types include: - **Tuples**: A [`Tuple` of elements](./tuple.md), each having an individual type. - **Nullable**: [`Nullable`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column gettings its default value for the data type) - **IP addresses**: use [`IPv4`](./domains/ipv4.md) and [`IPv6`](./domains/ipv6.md) to efficiently store IP addresses -- **Geo types**: for[ geographical data](./geo.md), including `Point`, `Ring`, `Polygon` and `MultiPolygon` -- **Special data types**: including [`Expression`](./special-data-types/expression.md), [`Set`](./special-data-types/set.md), [`Nothing`](./special-data-types/nothing.md) and [`Interval`](./special-data-types/interval.md) \ No newline at end of file +- **Geo types**: for [geographical data](./geo.md), including `Point`, `Ring`, `Polygon` and `MultiPolygon` +- **Special data types**: including [`Expression`](./special-data-types/expression.md), [`Set`](./special-data-types/set.md), [`Nothing`](./special-data-types/nothing.md) and [`Interval`](./special-data-types/interval.md) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 066e71c25a5..fa3ea582c55 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3185,16 +3185,6 @@ SELECT * FROM test2; Значение по умолчанию: `0`. -## allow_experimental_geo_types {#allow-experimental-geo-types} - -Разрешает использование экспериментальных типов данных для работы с [географическими структурами](../../sql-reference/data-types/geo.md). - -Возможные значения: -- 0 — использование типов данных для работы с географическими структурами не поддерживается. -- 1 — использование типов данных для работы с географическими структурами поддерживается. - -Значение по умолчанию: `0`. - ## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously} Добавляет модификатор `SYNC` ко всем запросам `DROP` и `DETACH`. diff --git a/docs/ru/sql-reference/data-types/geo.md b/docs/ru/sql-reference/data-types/geo.md index 24d981195f5..52c7dd97da0 100644 --- a/docs/ru/sql-reference/data-types/geo.md +++ b/docs/ru/sql-reference/data-types/geo.md @@ -8,13 +8,8 @@ sidebar_label: Географические структуры ClickHouse поддерживает типы данных для отображения географических объектов — точек (местоположений), территорий и т.п. -:::danger "Предупреждение" - Сейчас использование типов данных для работы с географическими структурами является экспериментальной возможностью. Чтобы использовать эти типы данных, включите настройку `allow_experimental_geo_types = 1`. -::: - **См. также** - [Хранение географических структур данных](https://ru.wikipedia.org/wiki/GeoJSON). -- Настройка [allow_experimental_geo_types](../../operations/settings/settings.md#allow-experimental-geo-types). ## Point {#point-data-type} @@ -25,7 +20,6 @@ ClickHouse поддерживает типы данных для отображ Запрос: ```sql -SET allow_experimental_geo_types = 1; CREATE TABLE geo_point (p Point) ENGINE = Memory(); INSERT INTO geo_point VALUES((10, 10)); SELECT p, toTypeName(p) FROM geo_point; @@ -47,7 +41,6 @@ SELECT p, toTypeName(p) FROM geo_point; Запрос: ```sql -SET allow_experimental_geo_types = 1; CREATE TABLE geo_ring (r Ring) ENGINE = Memory(); INSERT INTO geo_ring VALUES([(0, 0), (10, 0), (10, 10), (0, 10)]); SELECT r, toTypeName(r) FROM geo_ring; @@ -69,7 +62,6 @@ SELECT r, toTypeName(r) FROM geo_ring; Запись в этой таблице описывает многоугольник с одной дырой: ```sql -SET allow_experimental_geo_types = 1; CREATE TABLE geo_polygon (pg Polygon) ENGINE = Memory(); INSERT INTO geo_polygon VALUES([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]]); SELECT pg, toTypeName(pg) FROM geo_polygon; @@ -92,7 +84,6 @@ SELECT pg, toTypeName(pg) FROM geo_polygon; Запись в этой таблице описывает элемент, состоящий из двух многоугольников — первый без дыр, а второй с одной дырой: ```sql -SET allow_experimental_geo_types = 1; CREATE TABLE geo_multipolygon (mpg MultiPolygon) ENGINE = Memory(); INSERT INTO geo_multipolygon VALUES([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]); SELECT mpg, toTypeName(mpg) FROM geo_multipolygon; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c65958b86b7..3e1400c8528 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -19,7 +19,7 @@ class IColumn; /** List of settings: type, name, default value, description, flags * - * This looks rather unconvenient. It is done that way to avoid repeating settings in different places. + * This looks rather inconvenient. It is done that way to avoid repeating settings in different places. * Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field, * but we are not going to do it, because settings is used everywhere as static struct fields. * @@ -541,7 +541,6 @@ class IColumn; M(Seconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \ M(Bool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \ M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ - M(Bool, allow_experimental_geo_types, true, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, cast_ipv4_ipv6_default_on_conversion_error, false, "CAST operator into IPv4, CAST operator into IPV6 type, toIPv4, toIPv6 functions will return default value instead of throwing exception on conversion error.", 0) \ @@ -768,6 +767,8 @@ class IColumn; MAKE_OBSOLETE(M, Bool, allow_experimental_bigint_types, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_window_functions, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_lightweight_delete, true) \ + MAKE_OBSOLETE(M, Bool, allow_experimental_geo_types, true) \ + \ MAKE_OBSOLETE(M, Milliseconds, async_insert_stale_timeout_ms, 0) \ MAKE_OBSOLETE(M, HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT) \ MAKE_OBSOLETE(M, Bool, database_replicated_ddl_output, true) \ diff --git a/src/IO/Resource/DynamicResourceManager.cpp b/src/IO/Resource/DynamicResourceManager.cpp index 49e12984e33..df0de6575f4 100644 --- a/src/IO/Resource/DynamicResourceManager.cpp +++ b/src/IO/Resource/DynamicResourceManager.cpp @@ -98,9 +98,9 @@ DynamicResourceManager::State::Resource::~Resource() if (attached_to != nullptr) { ISchedulerNode * root = nodes.find("/")->second.ptr.get(); - attached_to->event_queue->enqueue([scheduler = attached_to, root] + attached_to->event_queue->enqueue([my_scheduler = attached_to, root] { - scheduler->removeChild(root); + my_scheduler->removeChild(root); }); } } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a34bfa5a579..3e76ad23c30 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -848,21 +848,6 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat } } - if (!create.attach && !settings.allow_experimental_geo_types) - { - for (const auto & name_and_type_pair : properties.columns.getAllPhysical()) - { - const auto & type = name_and_type_pair.type->getName(); - if (type == "MultiPolygon" || type == "Polygon" || type == "Ring" || type == "Point") - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column '{}' which type is '{}' " - "because experimental geo types are not allowed. " - "Set setting allow_experimental_geo_types = 1 in order to allow it", - name_and_type_pair.name, type); - } - } - } - if (!create.attach && !settings.allow_experimental_object_type) { for (const auto & [name, type] : properties.columns.getAllPhysical()) diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index e7302b6324a..05d28bc53a8 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -35,18 +35,6 @@ void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings } } - if (!settings.allow_experimental_geo_types) - { - const auto & type_name = type->getName(); - if (type_name == "MultiPolygon" || type_name == "Polygon" || type_name == "Ring" || type_name == "Point") - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Cannot create column with type '{}' because experimental geo types are not allowed. Set setting " - "allow_experimental_geo_types = 1 in order to allow it", type_name); - } - } - if (!settings.allow_experimental_object_type) { if (type->hasDynamicSubcolumns()) diff --git a/src/Interpreters/parseColumnsListForTableFunction.h b/src/Interpreters/parseColumnsListForTableFunction.h index 14119ab55da..ef1bbe5498e 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.h +++ b/src/Interpreters/parseColumnsListForTableFunction.h @@ -16,14 +16,12 @@ struct DataTypeValidationSettings explicit DataTypeValidationSettings(const Settings & settings) : allow_suspicious_low_cardinality_types(settings.allow_suspicious_low_cardinality_types) - , allow_experimental_geo_types(settings.allow_experimental_geo_types) , allow_experimental_object_type(settings.allow_experimental_object_type) , allow_suspicious_fixed_string_types(settings.allow_suspicious_fixed_string_types) { } bool allow_suspicious_low_cardinality_types = true; - bool allow_experimental_geo_types = true; bool allow_experimental_object_type = true; bool allow_suspicious_fixed_string_types = true; }; diff --git a/tests/queries/0_stateless/01291_geo_types.sql b/tests/queries/0_stateless/01291_geo_types.sql index 6b686ddf520..4038c545679 100644 --- a/tests/queries/0_stateless/01291_geo_types.sql +++ b/tests/queries/0_stateless/01291_geo_types.sql @@ -1,7 +1,5 @@ DROP TABLE IF EXISTS geo; -SET allow_experimental_geo_types = 1; - CREATE TABLE geo (a Point, b Ring, c Polygon, d MultiPolygon) ENGINE=Memory(); INSERT INTO geo VALUES((0, 0), [(0, 0), (10, 0), (10, 10), (0, 10)], [[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]], [[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]); diff --git a/tests/queries/0_stateless/02513_validate_data_types.sql b/tests/queries/0_stateless/02513_validate_data_types.sql index 1235d00ba79..5eb91ac7879 100644 --- a/tests/queries/0_stateless/02513_validate_data_types.sql +++ b/tests/queries/0_stateless/02513_validate_data_types.sql @@ -5,10 +5,6 @@ select CAST('{"x" : 1}', 'JSON'); -- {serverError ILLEGAL_COLUMN} desc file(nonexist.json, JSONAsObject); -- {serverError ILLEGAL_COLUMN} desc file(nonexist.json, JSONEachRow, 'x JSON'); -- {serverError ILLEGAL_COLUMN} -set allow_experimental_geo_types=0; -select CAST([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]], 'Polygon'); -- {serverError ILLEGAL_COLUMN} -desc file(nonexist.json, JSONEachRow, 'pg Polygon'); -- {serverError ILLEGAL_COLUMN} - set allow_suspicious_low_cardinality_types=0; select CAST(1000000, 'LowCardinality(UInt64)'); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY} desc file(nonexist.json, JSONEachRow, 'lc LowCardinality(UInt64)'); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY} @@ -16,4 +12,3 @@ desc file(nonexist.json, JSONEachRow, 'lc LowCardinality(UInt64)'); -- {serverEr set allow_suspicious_fixed_string_types=0; select CAST('', 'FixedString(1000)'); -- {serverError ILLEGAL_COLUMN} desc file(nonexist.json, JSONEachRow, 'fs FixedString(1000)'); -- {serverError ILLEGAL_COLUMN} - diff --git a/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql b/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql index f0d8d5202db..d78e8d34232 100644 --- a/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql +++ b/tests/queries/0_stateless/02524_fuzz_and_fuss_2.sql @@ -9,6 +9,6 @@ ENGINE = Memory; INSERT INTO data_a_02187 SELECT * FROM system.one -SETTINGS max_block_size = '1', min_insert_block_size_rows = '65536', min_insert_block_size_bytes = '0', max_insert_threads = '0', max_threads = '3', receive_timeout = '10', receive_data_timeout_ms = '10000', connections_with_failover_max_tries = '0', extremes = '1', use_uncompressed_cache = '0', optimize_move_to_prewhere = '1', optimize_move_to_prewhere_if_final = '0', replication_alter_partitions_sync = '2', totals_mode = 'before_having', allow_suspicious_low_cardinality_types = '1', compile_expressions = '1', min_count_to_compile_expression = '0', group_by_two_level_threshold = '100', distributed_aggregation_memory_efficient = '0', distributed_group_by_no_merge = '1', optimize_distributed_group_by_sharding_key = '1', optimize_skip_unused_shards = '1', optimize_skip_unused_shards_rewrite_in = '1', force_optimize_skip_unused_shards = '2', optimize_skip_unused_shards_nesting = '1', force_optimize_skip_unused_shards_nesting = '2', merge_tree_min_rows_for_concurrent_read = '10000', force_primary_key = '1', network_compression_method = 'ZSTD', network_zstd_compression_level = '7', log_queries = '0', log_queries_min_type = 'QUERY_FINISH', distributed_product_mode = 'local', insert_quorum = '2', insert_quorum_timeout = '0', insert_quorum_parallel = '0', select_sequential_consistency = '1', join_use_nulls = '1', any_join_distinct_right_table_keys = '1', preferred_max_column_in_block_size_bytes = '32', insert_distributed_sync = '1', insert_allow_materialized_columns = '1', use_index_for_in_with_subqueries = '1', joined_subquery_requires_alias = '0', empty_result_for_aggregation_by_empty_set = '1', allow_suspicious_codecs = '1', query_profiler_real_time_period_ns = '0', query_profiler_cpu_time_period_ns = '0', opentelemetry_start_trace_probability = '1', max_rows_to_read = '1000000', read_overflow_mode = 'break', max_rows_to_group_by = '10', group_by_overflow_mode = 'any', max_rows_to_sort = '100', sort_overflow_mode = 'break', max_result_rows = '10', max_execution_time = '3', max_execution_speed = '1', max_bytes_in_join = '100', join_algorithm = 'partial_merge', max_memory_usage = '1099511627776', log_query_threads = '1', send_logs_level = 'fatal', enable_optimize_predicate_expression = '1', prefer_localhost_replica = '1', optimize_read_in_order = '1', optimize_aggregation_in_order = '1', read_in_order_two_level_merge_threshold = '1', allow_introspection_functions = '1', check_query_single_value_result = '1', allow_experimental_live_view = '1', default_table_engine = 'Memory', mutations_sync = '2', convert_query_to_cnf = '0', optimize_arithmetic_operations_in_aggregate_functions = '1', optimize_duplicate_order_by_and_distinct = '0', optimize_multiif_to_if = '0', optimize_monotonous_functions_in_order_by = '1', optimize_functions_to_subcolumns = '1', optimize_using_constraints = '1', optimize_substitute_columns = '1', optimize_append_index = '1', transform_null_in = '1', allow_experimental_geo_types = '1', data_type_default_nullable = '1', cast_keep_nullable = '1', cast_ipv4_ipv6_default_on_conversion_error = '0', system_events_show_zero_values = '1', enable_global_with_statement = '1', optimize_on_insert = '0', optimize_rewrite_sum_if_to_count_if = '1', distributed_ddl_output_mode = 'throw', union_default_mode = 'ALL', optimize_aggregators_of_group_by_keys = '1', optimize_group_by_function_keys = '1', short_circuit_function_evaluation = 'enable', async_insert = '1', enable_filesystem_cache = '0', allow_deprecated_database_ordinary = '1', allow_deprecated_syntax_for_merge_tree = '1', allow_experimental_nlp_functions = '1', allow_experimental_object_type = '1', allow_experimental_map_type = '1', optimize_use_projections = '1', input_format_null_as_default = '1', input_format_ipv4_default_on_conversion_error = '0', input_format_ipv6_default_on_conversion_error = '0', output_format_json_named_tuples_as_objects = '1', output_format_write_statistics = '0', output_format_pretty_row_numbers = '1'; +SETTINGS max_block_size = '1', min_insert_block_size_rows = '65536', min_insert_block_size_bytes = '0', max_insert_threads = '0', max_threads = '3', receive_timeout = '10', receive_data_timeout_ms = '10000', connections_with_failover_max_tries = '0', extremes = '1', use_uncompressed_cache = '0', optimize_move_to_prewhere = '1', optimize_move_to_prewhere_if_final = '0', replication_alter_partitions_sync = '2', totals_mode = 'before_having', allow_suspicious_low_cardinality_types = '1', compile_expressions = '1', min_count_to_compile_expression = '0', group_by_two_level_threshold = '100', distributed_aggregation_memory_efficient = '0', distributed_group_by_no_merge = '1', optimize_distributed_group_by_sharding_key = '1', optimize_skip_unused_shards = '1', optimize_skip_unused_shards_rewrite_in = '1', force_optimize_skip_unused_shards = '2', optimize_skip_unused_shards_nesting = '1', force_optimize_skip_unused_shards_nesting = '2', merge_tree_min_rows_for_concurrent_read = '10000', force_primary_key = '1', network_compression_method = 'ZSTD', network_zstd_compression_level = '7', log_queries = '0', log_queries_min_type = 'QUERY_FINISH', distributed_product_mode = 'local', insert_quorum = '2', insert_quorum_timeout = '0', insert_quorum_parallel = '0', select_sequential_consistency = '1', join_use_nulls = '1', any_join_distinct_right_table_keys = '1', preferred_max_column_in_block_size_bytes = '32', insert_distributed_sync = '1', insert_allow_materialized_columns = '1', use_index_for_in_with_subqueries = '1', joined_subquery_requires_alias = '0', empty_result_for_aggregation_by_empty_set = '1', allow_suspicious_codecs = '1', query_profiler_real_time_period_ns = '0', query_profiler_cpu_time_period_ns = '0', opentelemetry_start_trace_probability = '1', max_rows_to_read = '1000000', read_overflow_mode = 'break', max_rows_to_group_by = '10', group_by_overflow_mode = 'any', max_rows_to_sort = '100', sort_overflow_mode = 'break', max_result_rows = '10', max_execution_time = '3', max_execution_speed = '1', max_bytes_in_join = '100', join_algorithm = 'partial_merge', max_memory_usage = '1099511627776', log_query_threads = '1', send_logs_level = 'fatal', enable_optimize_predicate_expression = '1', prefer_localhost_replica = '1', optimize_read_in_order = '1', optimize_aggregation_in_order = '1', read_in_order_two_level_merge_threshold = '1', allow_introspection_functions = '1', check_query_single_value_result = '1', allow_experimental_live_view = '1', default_table_engine = 'Memory', mutations_sync = '2', convert_query_to_cnf = '0', optimize_arithmetic_operations_in_aggregate_functions = '1', optimize_duplicate_order_by_and_distinct = '0', optimize_multiif_to_if = '0', optimize_monotonous_functions_in_order_by = '1', optimize_functions_to_subcolumns = '1', optimize_using_constraints = '1', optimize_substitute_columns = '1', optimize_append_index = '1', transform_null_in = '1', data_type_default_nullable = '1', cast_keep_nullable = '1', cast_ipv4_ipv6_default_on_conversion_error = '0', system_events_show_zero_values = '1', enable_global_with_statement = '1', optimize_on_insert = '0', optimize_rewrite_sum_if_to_count_if = '1', distributed_ddl_output_mode = 'throw', union_default_mode = 'ALL', optimize_aggregators_of_group_by_keys = '1', optimize_group_by_function_keys = '1', short_circuit_function_evaluation = 'enable', async_insert = '1', enable_filesystem_cache = '0', allow_deprecated_database_ordinary = '1', allow_deprecated_syntax_for_merge_tree = '1', allow_experimental_nlp_functions = '1', allow_experimental_object_type = '1', allow_experimental_map_type = '1', optimize_use_projections = '1', input_format_null_as_default = '1', input_format_ipv4_default_on_conversion_error = '0', input_format_ipv6_default_on_conversion_error = '0', output_format_json_named_tuples_as_objects = '1', output_format_write_statistics = '0', output_format_pretty_row_numbers = '1'; DROP TABLE data_a_02187; From e46476dba22af226d1ac47355e0adeaa3759d2db Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 12:44:20 +0200 Subject: [PATCH 0424/2223] Update src/Coordination/Changelog.cpp Co-authored-by: alesapin --- src/Coordination/Changelog.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 7275a080252..e20fbf864eb 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -284,7 +284,8 @@ private: do { res = ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); - } while (res < 0 && errno == EINTR); + } + while (res < 0 && errno == EINTR); if (res != 0) LOG_WARNING(log, "Could not ftruncate file. Error: {}, errno: {}", errnoToString(), errno); From 9c3b17fa18ad14ece86ec665fc0dd5d8396f432c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 13:00:51 +0200 Subject: [PATCH 0425/2223] Remove whitespace --- src/Coordination/Changelog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index e20fbf864eb..894fd93cfa7 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -284,7 +284,7 @@ private: do { res = ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); - } + } while (res < 0 && errno == EINTR); if (res != 0) From 3121a57912752d70ac46402f46c695181571dea3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 May 2023 14:21:07 +0200 Subject: [PATCH 0426/2223] Add some assertions --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 47 +++++++++++-------- 1 file changed, 28 insertions(+), 19 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 00d23183f6a..b4ea30e54c8 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -708,14 +708,18 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() } else if (current_write_offset < file_offset_of_buffer_end) { + const auto path = file_segment.getPathInLocalCache(); + size_t file_size = 0; + if (fs::exists(path)) + file_size = fs::file_size(path); + throw Exception( ErrorCodes::LOGICAL_ERROR, - "Expected {} >= {} ({})", - current_write_offset, file_offset_of_buffer_end, getInfoForLog()); + "Invariant failed. Expected {} >= {} (size on fs: {}, {})", + current_write_offset, file_offset_of_buffer_end, file_size, getInfoForLog()); } } - - if (read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE) + else if (read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE) { /** * ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE means that on previous getImplementationBuffer() call @@ -884,25 +888,28 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (!result) { -#ifndef NDEBUG - if (read_type == ReadType::CACHED) + auto debug_check = [&]() { - size_t cache_file_size = getFileSizeFromReadBuffer(*implementation_buffer); - if (cache_file_size == 0) + if (read_type == ReadType::CACHED) { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Attempt to read from an empty cache file: {} (just before actual read)", - cache_file_size); + size_t cache_file_size = getFileSizeFromReadBuffer(*implementation_buffer); + if (cache_file_size == 0) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Attempt to read from an empty cache file: {} (just before actual read)", + cache_file_size); + } } - } - else - { - assert(file_offset_of_buffer_end == static_cast(implementation_buffer->getFileOffsetOfBufferEnd())); - } + else + { + chassert(file_offset_of_buffer_end == static_cast(implementation_buffer->getFileOffsetOfBufferEnd())); + } + chassert(!implementation_buffer->hasPendingData()); + return true; + }; - assert(!implementation_buffer->hasPendingData()); -#endif + chassert(debug_check()); Stopwatch watch(CLOCK_MONOTONIC); @@ -927,6 +934,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() { ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); + + chassert(file_offset_of_buffer_end + size <= file_segment.range().size()); } else { From 3107070e7699d14a6895dbdeb556b69986fc604b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 12:48:19 +0000 Subject: [PATCH 0427/2223] Avoid deadlock when starting table in attach thread --- .../ReplicatedMergeTreeRestartingThread.h | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 25 +++++--- .../test_replicated_table_attach/__init__.py | 0 .../configs/config.xml | 3 + .../test_replicated_table_attach/test.py | 58 +++++++++++++++++++ 5 files changed, 79 insertions(+), 10 deletions(-) create mode 100644 tests/integration/test_replicated_table_attach/__init__.py create mode 100644 tests/integration/test_replicated_table_attach/configs/config.xml create mode 100644 tests/integration/test_replicated_table_attach/test.py diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 8f5721e440d..5579fdf2033 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -29,6 +29,7 @@ public: void shutdown(bool part_of_full_shutdown); + void run(); private: StorageReplicatedMergeTree & storage; String log_name; @@ -43,8 +44,6 @@ private: UInt32 consecutive_check_failures = 0; /// How many consecutive checks have failed bool first_time = true; /// Activate replica for the first time. - void run(); - /// Restarts table if needed, returns false if it failed to restart replica. bool runImpl(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fcb7adbd69f..cdd43239bec 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4416,8 +4416,23 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) startBeingLeader(); - /// In this thread replica will be activated. - restarting_thread.start(); + if (from_attach_thread) + { + /// Try activating replica in current thread. + restarting_thread.run(); + } + else + { + /// Activate replica in a seperate thread. + restarting_thread.start(); + + /// Wait while restarting_thread finishing initialization. + /// NOTE It does not mean that replication is actually started after receiving this event. + /// It only means that an attempt to startup replication was made. + /// Table may be still in readonly mode if this attempt failed for any reason. + startup_event.wait(); + } + /// And this is just a callback session_expired_callback_handler = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [this]() { @@ -4425,12 +4440,6 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) restarting_thread.start(); }); - /// Wait while restarting_thread finishing initialization. - /// NOTE It does not mean that replication is actually started after receiving this event. - /// It only means that an attempt to startup replication was made. - /// Table may be still in readonly mode if this attempt failed for any reason. - startup_event.wait(); - startBackgroundMovesIfNeeded(); part_moves_between_shards_orchestrator.start(); diff --git a/tests/integration/test_replicated_table_attach/__init__.py b/tests/integration/test_replicated_table_attach/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_table_attach/configs/config.xml b/tests/integration/test_replicated_table_attach/configs/config.xml new file mode 100644 index 00000000000..b7b5d996072 --- /dev/null +++ b/tests/integration/test_replicated_table_attach/configs/config.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py new file mode 100644 index 00000000000..c09d67aab77 --- /dev/null +++ b/tests/integration/test_replicated_table_attach/test.py @@ -0,0 +1,58 @@ +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/config.xml"], + with_zookeeper=True, + stay_alive=True, +) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_startup_with_small_bg_pool(started_cluster): + node.query( + "CREATE TABLE replicated_table (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table', 'r1') ORDER BY k" + ) + + node.query("INSERT INTO replicated_table VALUES(20, 30)") + + def assert_values(): + assert node.query("SELECT * FROM replicated_table") == "20\t30\n" + + assert_values() + node.restart_clickhouse(stop_start_wait_sec=10) + assert_values() + + node.query("DROP TABLE replicated_table SYNC") + +def test_startup_with_small_bg_pool_partitioned(started_cluster): + node.query( + "CREATE TABLE replicated_table_partitioned (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table_partitioned', 'r1') ORDER BY k" + ) + + node.query("INSERT INTO replicated_table_partitioned VALUES(20, 30)") + + def assert_values(): + assert node.query("SELECT * FROM replicated_table_partitioned") == "20\t30\n" + + assert_values() + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node) + node.restart_clickhouse(stop_start_wait_sec=20) + assert_values() + + node.query("DROP TABLE replicated_table_partitioned SYNC") From f947d1cc0ae147bc830c8200daf70295dc9aec38 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 19 May 2023 13:00:26 +0000 Subject: [PATCH 0428/2223] Automatic style fix --- tests/integration/test_replicated_table_attach/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py index c09d67aab77..11365ea9163 100644 --- a/tests/integration/test_replicated_table_attach/test.py +++ b/tests/integration/test_replicated_table_attach/test.py @@ -13,6 +13,7 @@ node = cluster.add_instance( stay_alive=True, ) + @pytest.fixture(scope="module") def started_cluster(): try: @@ -39,6 +40,7 @@ def test_startup_with_small_bg_pool(started_cluster): node.query("DROP TABLE replicated_table SYNC") + def test_startup_with_small_bg_pool_partitioned(started_cluster): node.query( "CREATE TABLE replicated_table_partitioned (k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/replicated_table_partitioned', 'r1') ORDER BY k" From 55fc4adf055324cc9b359e57a8056025f744a317 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 19 May 2023 16:42:15 +0300 Subject: [PATCH 0429/2223] Update 02441_alter_delete_and_drop_column.sql --- tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql index b9b1b645e8e..9c4697362df 100644 --- a/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql +++ b/tests/queries/0_stateless/02441_alter_delete_and_drop_column.sql @@ -1,3 +1,4 @@ +-- Tags: no-replicated-database create table mut (n int, m int, k int) engine=ReplicatedMergeTree('/test/02441/{database}/mut', '1') order by n; set insert_keeper_fault_injection_probability=0; From acf71c5b9a976834202b856494f027421ee77422 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 15:48:31 +0200 Subject: [PATCH 0430/2223] Fix typo --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cdd43239bec..6373c164c40 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4423,7 +4423,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) } else { - /// Activate replica in a seperate thread. + /// Activate replica in a separate thread. restarting_thread.start(); /// Wait while restarting_thread finishing initialization. From 5237dd02451cb23d4101c0cecfe8bf133292103e Mon Sep 17 00:00:00 2001 From: mateng915 Date: Fri, 19 May 2023 22:06:43 +0800 Subject: [PATCH 0431/2223] New system table zookeeper connection (#45245) * Feature: Support new system table to show which zookeeper node be connected Description: ============ Currently we have no place to check which zk node be connected otherwise using lsof command. It not convenient Solution: ========= Implemented a new system table, system.zookeeper_host when CK Server has zk this table will show the zk node dir which connected by current CK server Noted: This table can support multi-zookeeper cluster scenario. * fixed review comments * added test case * update test cases * remove unused code * fixed review comments and removed unused code * updated test cases for print host, port and is_expired * modify the code comments * fixed CI Failed * fixed code style check failure * updated test cases by added Tags * update test reference * update test cases * added system.zookeeper_connection doc * Update docs/en/operations/system-tables/zookeeper_connection.md * Update docs/en/operations/system-tables/zookeeper_connection.md * Update docs/en/operations/system-tables/zookeeper_connection.md --------- Co-authored-by: Alexander Tokmakov --- .../system-tables/zookeeper_connection.md | 29 +++++++++++ src/Common/ZooKeeper/IKeeper.h | 2 + src/Common/ZooKeeper/TestKeeper.h | 3 ++ src/Common/ZooKeeper/ZooKeeper.cpp | 20 +++++++ src/Common/ZooKeeper/ZooKeeper.h | 8 +++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 ++ src/Common/ZooKeeper/ZooKeeperImpl.h | 3 ++ src/Interpreters/Context.cpp | 11 ++++ src/Interpreters/Context.h | 8 ++- .../StorageSystemZooKeeperConnection.cpp | 52 +++++++++++++++++++ .../System/StorageSystemZooKeeperConnection.h | 28 ++++++++++ src/Storages/System/attachSystemTables.cpp | 4 ++ ...2735_system_zookeeper_connection.reference | 1 + .../02735_system_zookeeper_connection.sql | 13 +++++ 14 files changed, 185 insertions(+), 1 deletion(-) create mode 100644 docs/en/operations/system-tables/zookeeper_connection.md create mode 100644 src/Storages/System/StorageSystemZooKeeperConnection.cpp create mode 100644 src/Storages/System/StorageSystemZooKeeperConnection.h create mode 100644 tests/queries/0_stateless/02735_system_zookeeper_connection.reference create mode 100644 tests/queries/0_stateless/02735_system_zookeeper_connection.sql diff --git a/docs/en/operations/system-tables/zookeeper_connection.md b/docs/en/operations/system-tables/zookeeper_connection.md new file mode 100644 index 00000000000..9438cda1808 --- /dev/null +++ b/docs/en/operations/system-tables/zookeeper_connection.md @@ -0,0 +1,29 @@ +--- +slug: /en/operations/system-tables/zookeeper_connection +--- +#zookeeper_connection + +This table does not exist if ZooKeeper is not configured. The 'system.zookeeper_connection' table shows current connections to ZooKeeper (including auxiliary ZooKeepers). Each row shows information about one connection. + +Columns: + +- `name` ([String](../../sql-reference/data-types/string.md)) — ZooKeeper cluster's name. +- `host` ([String](../../sql-reference/data-types/string.md)) — The hostname/IP of the ZooKeeper node that ClickHouse connected to. +- `port` ([String](../../sql-reference/data-types/string.md)) — The port of the ZooKeeper node that ClickHouse connected to. +- `index` ([UInt8](../../sql-reference/data-types/int-uint.md)) — The index of the ZooKeeper node that ClickHouse connected to. The index is from ZooKeeper config. +- `connected_time` ([String](../../sql-reference/data-types/string.md)) — When the connection was established +- `is_expired` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Is the current connection expired. +- `keeper_api_version` ([String](../../sql-reference/data-types/string.md)) — Keeper API version. +- `client_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Session id of the connection. + +Example: + +``` sql +SELECT * FROM system.zookeeper_connection; +``` + +``` text +┌─name──────────────┬─host─────────┬─port─┬─index─┬──────connected_time─┬─is_expired─┬─keeper_api_version─┬──────────client_id─┐ +│ default_zookeeper │ 127.0.0.1 │ 2181 │ 0 │ 2023-05-19 14:30:16 │ 0 │ 0 │ 216349144108826660 │ +└───────────────────┴──────────────┴──────┴───────┴─────────────────────┴────────────┴────────────────────┴────────────────────┘ +``` diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index b09f096d761..86f9a388644 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -466,6 +466,8 @@ public: /// Useful to check owner of ephemeral node. virtual int64_t getSessionID() const = 0; + virtual String getConnectedAddress() const = 0; + /// If the method will throw an exception, callbacks won't be called. /// /// After the method is executed successfully, you must wait for callbacks diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 27405d8d571..11e56daf6b4 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -39,6 +39,7 @@ public: bool isExpired() const override { return expired; } int64_t getSessionID() const override { return 0; } + String getConnectedAddress() const override { return connected_zk_address; } void create( @@ -126,6 +127,8 @@ private: zkutil::ZooKeeperArgs args; + String connected_zk_address; + std::mutex push_request_mutex; std::atomic expired{false}; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 09047b5b232..c423e4fd498 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -111,6 +111,26 @@ void ZooKeeper::init(ZooKeeperArgs args_) LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(args.hosts, ",")); else LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(args.hosts, ","), args.chroot); + + String address = impl->getConnectedAddress(); + + size_t colon_pos = address.find(':'); + connected_zk_host = address.substr(0, colon_pos); + connected_zk_port = address.substr(colon_pos + 1); + + connected_zk_index = 0; + + if (args.hosts.size() > 1) + { + for (size_t i = 0; i < args.hosts.size(); i++) + { + if (args.hosts[i] == address) + { + connected_zk_index = i; + break; + } + } + } } else if (args.implementation == "testkeeper") { diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index ca6a44c4cbc..9b85938c726 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -523,6 +523,10 @@ public: void setServerCompletelyStarted(); + String getConnectedZooKeeperHost() const { return connected_zk_host; } + String getConnectedZooKeeperPort() const { return connected_zk_port; } + size_t getConnectedZooKeeperIndex() const { return connected_zk_index; } + private: void init(ZooKeeperArgs args_); @@ -586,6 +590,10 @@ private: ZooKeeperArgs args; + String connected_zk_host; + String connected_zk_port; + size_t connected_zk_index; + std::mutex mutex; Poco::Logger * log = nullptr; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 6c79fc4f178..34be8aa1332 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -433,6 +433,8 @@ void ZooKeeper::connect( } connected = true; + connected_zk_address = node.address.toString(); + break; } catch (...) @@ -448,6 +450,8 @@ void ZooKeeper::connect( if (!connected) { WriteBufferFromOwnString message; + connected_zk_address = ""; + message << "All connection tries failed while connecting to ZooKeeper. nodes: "; bool first = true; for (const auto & node : nodes) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index c0c57d3f719..6715607ca88 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -125,6 +125,8 @@ public: /// Useful to check owner of ephemeral node. int64_t getSessionID() const override { return session_id; } + String getConnectedAddress() const override { return connected_zk_address; } + void executeGenericRequest( const ZooKeeperRequestPtr & request, ResponseCallback callback); @@ -201,6 +203,7 @@ public: private: ACLs default_acls; + String connected_zk_address; zkutil::ZooKeeperArgs args; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0ef9ea53ee8..d9f450191bc 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2778,6 +2778,17 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const return zookeeper->second; } + +std::map Context::getAuxiliaryZooKeepers() const +{ + std::lock_guard lock(shared->auxiliary_zookeepers_mutex); + + if (!shared->auxiliary_zookeepers.empty()) + return shared->auxiliary_zookeepers; + else + return std::map(); +} + #if USE_ROCKSDB MergeTreeMetadataCachePtr Context::getMergeTreeMetadataCache() const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 87843a458e8..15f2ff625ef 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -32,7 +32,11 @@ namespace Poco::Net { class IPAddress; } -namespace zkutil { class ZooKeeper; } +namespace zkutil +{ + class ZooKeeper; + using ZooKeeperPtr = std::shared_ptr; +} struct OvercommitTracker; @@ -827,6 +831,8 @@ public: std::shared_ptr getZooKeeper() const; /// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry. std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; + /// return Auxiliary Zookeeper map + std::map getAuxiliaryZooKeepers() const; /// Try to connect to Keeper using get(Auxiliary)ZooKeeper. Useful for /// internal Keeper start (check connection to some other node). Return true diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp new file mode 100644 index 00000000000..f249097654e --- /dev/null +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() +{ + return { + {"name", std::make_shared()}, + {"host", std::make_shared()}, + {"port", std::make_shared()}, + {"index", std::make_shared()}, + {"connected_time", std::make_shared()}, + {"is_expired", std::make_shared()}, + {"keeper_api_version", std::make_shared()}, + {"client_id", std::make_shared()} + }; +} + +void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, ContextPtr context, + const SelectQueryInfo &) const +{ + res_columns[0]->insert("default_zookeeper"); + res_columns[1]->insert(context->getZooKeeper()->getConnectedZooKeeperHost()); + res_columns[2]->insert(context->getZooKeeper()->getConnectedZooKeeperPort()); + res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); + res_columns[4]->insert(context->getZooKeeperSessionUptime()); + res_columns[5]->insert(context->getZooKeeper()->expired()); + res_columns[6]->insert(context->getZooKeeper()->getApiVersion()); + res_columns[7]->insert(context->getZooKeeper()->getClientID()); + + for (const auto & elem : context->getAuxiliaryZooKeepers()) + { + res_columns[0]->insert(elem.first); + res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); + res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); + res_columns[2]->insert(elem.second->getConnectedZooKeeperPort()); + res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); + res_columns[4]->insert(elem.second->getSessionUptime()); + res_columns[5]->insert(elem.second->expired()); + res_columns[6]->insert(elem.second->getApiVersion()); + res_columns[7]->insert(elem.second->getClientID()); + } + +} + +} diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.h b/src/Storages/System/StorageSystemZooKeeperConnection.h new file mode 100644 index 00000000000..dd4c293c112 --- /dev/null +++ b/src/Storages/System/StorageSystemZooKeeperConnection.h @@ -0,0 +1,28 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/** Implements `zookeeper_connection` system table, which allows you to get information about the connected zookeeper info. + */ +class StorageSystemZooKeeperConnection final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemZooKeeperConnection"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} + diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index d6982ba30d5..424c74662ec 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -80,6 +80,7 @@ #include #include #include +#include #ifdef OS_LINUX #include @@ -186,7 +187,10 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "named_collections"); if (has_zookeeper) + { attach(context, system_database, "zookeeper"); + attach(context, system_database, "zookeeper_connection"); + } if (context->getConfigRef().getInt("allow_experimental_transactions", 0)) attach(context, system_database, "transactions"); diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference new file mode 100644 index 00000000000..c9cc8adede8 --- /dev/null +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference @@ -0,0 +1 @@ +[ :1]:9181 0 diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql new file mode 100644 index 00000000000..10f12177b2e --- /dev/null +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql @@ -0,0 +1,13 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS test_zk_connection_table; + +CREATE TABLE test_zk_connection_table ( + key UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/{database}/02731_zk_connection/{shard}', '{replica}') +ORDER BY tuple(); + +select host, port, is_expired from system.zookeeper_connection where name='default_zookeeper'; + +DROP TABLE IF EXISTS test_zk_connection_table; From 0eab528f9f411deb2ba3eafa74c46acc8610b2a2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 May 2023 16:23:56 +0200 Subject: [PATCH 0432/2223] Move common code --- programs/keeper/Keeper.cpp | 103 +---------------- programs/server/Server.cpp | 104 +----------------- .../assertProcessUserMatchesDataOwner.cpp | 66 +++++++++++ .../assertProcessUserMatchesDataOwner.h | 10 ++ src/Common/makeSocketAddress.cpp | 36 ++++++ src/Common/makeSocketAddress.h | 8 ++ src/Server/waitServersToFinish.cpp | 36 ++++++ src/Server/waitServersToFinish.h | 9 ++ 8 files changed, 173 insertions(+), 199 deletions(-) create mode 100644 src/Common/assertProcessUserMatchesDataOwner.cpp create mode 100644 src/Common/assertProcessUserMatchesDataOwner.h create mode 100644 src/Common/makeSocketAddress.cpp create mode 100644 src/Common/makeSocketAddress.h create mode 100644 src/Server/waitServersToFinish.cpp create mode 100644 src/Server/waitServersToFinish.h diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 3d1773260f5..8976653f4e6 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -11,6 +11,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -80,87 +83,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace -{ - -size_t waitServersToFinish(std::vector & servers, size_t seconds_to_wait) -{ - const size_t sleep_max_ms = 1000 * seconds_to_wait; - const size_t sleep_one_ms = 100; - size_t sleep_current_ms = 0; - size_t current_connections = 0; - for (;;) - { - current_connections = 0; - - for (auto & server : servers) - { - server.stop(); - current_connections += server.currentConnections(); - } - - if (!current_connections) - break; - - sleep_current_ms += sleep_one_ms; - if (sleep_current_ms < sleep_max_ms) - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_one_ms)); - else - break; - } - return current_connections; -} - -Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log) -{ - Poco::Net::SocketAddress socket_address; - try - { - socket_address = Poco::Net::SocketAddress(host, port); - } - catch (const Poco::Net::DNSException & e) - { - const auto code = e.code(); - if (code == EAI_FAMILY -#if defined(EAI_ADDRFAMILY) - || code == EAI_ADDRFAMILY -#endif - ) - { - LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. " - "If it is an IPv6 address and your host has disabled IPv6, then consider to " - "specify IPv4 address to listen in element of configuration " - "file. Example: 0.0.0.0", - host, e.code(), e.message()); - } - - throw; - } - return socket_address; -} - -std::string getUserName(uid_t user_id) -{ - /// Try to convert user id into user name. - auto buffer_size = sysconf(_SC_GETPW_R_SIZE_MAX); - if (buffer_size <= 0) - buffer_size = 1024; - std::string buffer; - buffer.reserve(buffer_size); - - struct passwd passwd_entry; - struct passwd * result = nullptr; - const auto error = getpwuid_r(user_id, &passwd_entry, buffer.data(), buffer_size, &result); - - if (error) - throwFromErrno("Failed to find user name for " + toString(user_id), ErrorCodes::FAILED_TO_GETPWUID, error); - else if (result) - return result->pw_name; - return toString(user_id); -} - -} - Poco::Net::SocketAddress Keeper::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const { auto address = makeSocketAddress(host, port, &logger()); @@ -364,24 +286,7 @@ try std::filesystem::create_directories(path); /// Check that the process user id matches the owner of the data. - const auto effective_user_id = geteuid(); - struct stat statbuf; - if (stat(path.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid) - { - const auto effective_user = getUserName(effective_user_id); - const auto data_owner = getUserName(statbuf.st_uid); - std::string message = "Effective user of the process (" + effective_user + - ") does not match the owner of the data (" + data_owner + ")."; - if (effective_user_id == 0) - { - message += " Run under 'sudo -u " + data_owner + "'."; - throw Exception::createDeprecated(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA); - } - else - { - LOG_WARNING(log, fmt::runtime(message)); - } - } + assertProcessUserMatchesDataOwner(path, [&](const std::string & message){ LOG_WARNING(log, fmt::runtime(message)); }); DB::ServerUUID::load(path + "/uuid", log); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 632f3f3a02d..56938977970 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -39,6 +39,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -200,40 +203,6 @@ int mainEntryClickHouseServer(int argc, char ** argv) } } - -namespace -{ - -size_t waitServersToFinish(std::vector & servers, size_t seconds_to_wait) -{ - const size_t sleep_max_ms = 1000 * seconds_to_wait; - const size_t sleep_one_ms = 100; - size_t sleep_current_ms = 0; - size_t current_connections = 0; - for (;;) - { - current_connections = 0; - - for (auto & server : servers) - { - server.stop(); - current_connections += server.currentConnections(); - } - - if (!current_connections) - break; - - sleep_current_ms += sleep_one_ms; - if (sleep_current_ms < sleep_max_ms) - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_one_ms)); - else - break; - } - return current_connections; -} - -} - namespace DB { @@ -261,54 +230,6 @@ static std::string getCanonicalPath(std::string && path) return std::move(path); } -static std::string getUserName(uid_t user_id) -{ - /// Try to convert user id into user name. - auto buffer_size = sysconf(_SC_GETPW_R_SIZE_MAX); - if (buffer_size <= 0) - buffer_size = 1024; - std::string buffer; - buffer.reserve(buffer_size); - - struct passwd passwd_entry; - struct passwd * result = nullptr; - const auto error = getpwuid_r(user_id, &passwd_entry, buffer.data(), buffer_size, &result); - - if (error) - throwFromErrno("Failed to find user name for " + toString(user_id), ErrorCodes::FAILED_TO_GETPWUID, error); - else if (result) - return result->pw_name; - return toString(user_id); -} - -Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log) -{ - Poco::Net::SocketAddress socket_address; - try - { - socket_address = Poco::Net::SocketAddress(host, port); - } - catch (const Poco::Net::DNSException & e) - { - const auto code = e.code(); - if (code == EAI_FAMILY -#if defined(EAI_ADDRFAMILY) - || code == EAI_ADDRFAMILY -#endif - ) - { - LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. " - "If it is an IPv6 address and your host has disabled IPv6, then consider to " - "specify IPv4 address to listen in element of configuration " - "file. Example: 0.0.0.0", - host, e.code(), e.message()); - } - - throw; - } - return socket_address; -} - Poco::Net::SocketAddress Server::socketBindListen( const Poco::Util::AbstractConfiguration & config, Poco::Net::ServerSocket & socket, @@ -959,24 +880,7 @@ try std::string default_database = server_settings.default_database.toString(); /// Check that the process user id matches the owner of the data. - const auto effective_user_id = geteuid(); - struct stat statbuf; - if (stat(path_str.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid) - { - const auto effective_user = getUserName(effective_user_id); - const auto data_owner = getUserName(statbuf.st_uid); - std::string message = "Effective user of the process (" + effective_user + - ") does not match the owner of the data (" + data_owner + ")."; - if (effective_user_id == 0) - { - message += " Run under 'sudo -u " + data_owner + "'."; - throw Exception::createDeprecated(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA); - } - else - { - global_context->addWarningMessage(message); - } - } + assertProcessUserMatchesDataOwner(path_str, [&](const std::string & message){ global_context->addWarningMessage(message); }); global_context->setPath(path_str); diff --git a/src/Common/assertProcessUserMatchesDataOwner.cpp b/src/Common/assertProcessUserMatchesDataOwner.cpp new file mode 100644 index 00000000000..f2557a4aaaf --- /dev/null +++ b/src/Common/assertProcessUserMatchesDataOwner.cpp @@ -0,0 +1,66 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FAILED_TO_GETPWUID; + extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA; +} + +namespace +{ + std::string getUserName(uid_t user_id) + { + /// Try to convert user id into user name. + auto buffer_size = sysconf(_SC_GETPW_R_SIZE_MAX); + if (buffer_size <= 0) + buffer_size = 1024; + std::string buffer; + buffer.reserve(buffer_size); + + struct passwd passwd_entry; + struct passwd * result = nullptr; + const auto error = getpwuid_r(user_id, &passwd_entry, buffer.data(), buffer_size, &result); + + if (error) + throwFromErrno("Failed to find user name for " + std::to_string(user_id), ErrorCodes::FAILED_TO_GETPWUID, error); + else if (result) + return result->pw_name; + return std::to_string(user_id); + } +} + +void assertProcessUserMatchesDataOwner(const std::string & path, std::function on_warning) +{ + /// Check that the process user id matches the owner of the data. + const auto effective_user_id = geteuid(); + struct stat statbuf; + if (stat(path.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid) + { + const auto effective_user = getUserName(effective_user_id); + const auto data_owner = getUserName(statbuf.st_uid); + std::string message = fmt::format( + "Effective user of the process ({}) does not match the owner of the data ({}).", + effective_user, data_owner); + + if (effective_user_id == 0) + { + message += fmt::format(" Run under 'sudo -u {}'.", data_owner); + throw Exception(ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA, "{}", message); + } + else + { + on_warning(message); + } + } +} + +} diff --git a/src/Common/assertProcessUserMatchesDataOwner.h b/src/Common/assertProcessUserMatchesDataOwner.h new file mode 100644 index 00000000000..b31d795da71 --- /dev/null +++ b/src/Common/assertProcessUserMatchesDataOwner.h @@ -0,0 +1,10 @@ +#pragma once +#include + +namespace DB +{ + +void assertProcessUserMatchesDataOwner( + const std::string & path, std::function on_warning); + +} diff --git a/src/Common/makeSocketAddress.cpp b/src/Common/makeSocketAddress.cpp new file mode 100644 index 00000000000..b5df6a4ef03 --- /dev/null +++ b/src/Common/makeSocketAddress.cpp @@ -0,0 +1,36 @@ +#include +#include +#include + +namespace DB +{ + +Poco::Net::SocketAddress makeSocketAddress(const std::string & host, uint16_t port, Poco::Logger * log) +{ + Poco::Net::SocketAddress socket_address; + try + { + socket_address = Poco::Net::SocketAddress(host, port); + } + catch (const Poco::Net::DNSException & e) + { + const auto code = e.code(); + if (code == EAI_FAMILY +#if defined(EAI_ADDRFAMILY) + || code == EAI_ADDRFAMILY +#endif + ) + { + LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. " + "If it is an IPv6 address and your host has disabled IPv6, then consider to " + "specify IPv4 address to listen in element of configuration " + "file. Example: 0.0.0.0", + host, e.code(), e.message()); + } + + throw; + } + return socket_address; +} + +} diff --git a/src/Common/makeSocketAddress.h b/src/Common/makeSocketAddress.h new file mode 100644 index 00000000000..0e68a4ac077 --- /dev/null +++ b/src/Common/makeSocketAddress.h @@ -0,0 +1,8 @@ +#include + +namespace Poco { class Logger; } + +namespace DB +{ +Poco::Net::SocketAddress makeSocketAddress(const std::string & host, uint16_t port, Poco::Logger * log); +} diff --git a/src/Server/waitServersToFinish.cpp b/src/Server/waitServersToFinish.cpp new file mode 100644 index 00000000000..f2e36fae86c --- /dev/null +++ b/src/Server/waitServersToFinish.cpp @@ -0,0 +1,36 @@ +#include +#include +#include + +namespace DB +{ + +size_t waitServersToFinish(std::vector & servers, size_t seconds_to_wait) +{ + const size_t sleep_max_ms = 1000 * seconds_to_wait; + const size_t sleep_one_ms = 100; + size_t sleep_current_ms = 0; + size_t current_connections = 0; + for (;;) + { + current_connections = 0; + + for (auto & server : servers) + { + server.stop(); + current_connections += server.currentConnections(); + } + + if (!current_connections) + break; + + sleep_current_ms += sleep_one_ms; + if (sleep_current_ms < sleep_max_ms) + sleepForMilliseconds(sleep_one_ms); + else + break; + } + return current_connections; +} + +} diff --git a/src/Server/waitServersToFinish.h b/src/Server/waitServersToFinish.h new file mode 100644 index 00000000000..1f496d703c2 --- /dev/null +++ b/src/Server/waitServersToFinish.h @@ -0,0 +1,9 @@ +#pragma once + +namespace DB +{ +class ProtocolServerAdapter; + +size_t waitServersToFinish(std::vector & servers, size_t seconds_to_wait); + +} From b29edc473786a6c3cd5dcc37a53fa36c32283782 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 May 2023 16:38:14 +0200 Subject: [PATCH 0433/2223] Add method --- src/IO/S3/Client.cpp | 16 ++++++++++++---- src/IO/S3/Client.h | 9 +++++++-- 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 93ef30a927e..668b1a3959d 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -133,11 +133,12 @@ ProviderType deduceProviderType(const std::string & url) Client::Client( size_t max_redirects_, ServerSideEncryptionKMSConfig sse_kms_config_, - const std::shared_ptr & credentials_provider, + const std::shared_ptr & credentials_provider_, const Aws::Client::ClientConfiguration & client_configuration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, bool use_virtual_addressing) - : Aws::S3::S3Client(credentials_provider, client_configuration, std::move(sign_payloads), use_virtual_addressing) + : Aws::S3::S3Client(credentials_provider_, client_configuration, std::move(sign_payloads), use_virtual_addressing) + , credentials_provider(credentials_provider_) , max_redirects(max_redirects_) , sse_kms_config(std::move(sse_kms_config_)) , log(&Poco::Logger::get("S3Client")) @@ -177,6 +178,7 @@ Client::Client( Client::Client(const Client & other) : Aws::S3::S3Client(other) , initial_endpoint(other.initial_endpoint) + , credentials_provider(other.credentials_provider) , explicit_region(other.explicit_region) , detect_region(other.detect_region) , provider_type(other.provider_type) @@ -188,6 +190,11 @@ Client::Client(const Client & other) ClientCacheRegistry::instance().registerClient(cache); } +Aws::Auth::AWSCredentials Client::getCredentials() const +{ + return credentials_provider->GetAWSCredentials(); +} + bool Client::checkIfWrongRegionDefined(const std::string & bucket, const Aws::S3::S3Error & error, std::string & region) const { if (detect_region) @@ -711,7 +718,8 @@ std::unique_ptr ClientFactory::create( // NOLINT const String & server_side_encryption_customer_key_base64, ServerSideEncryptionKMSConfig sse_kms_config, HTTPHeaderEntries headers, - CredentialsConfiguration credentials_configuration) + CredentialsConfiguration credentials_configuration, + const String & session_token) { PocoHTTPClientConfiguration client_configuration = cfg_; client_configuration.updateSchemeAndRegion(); @@ -735,7 +743,7 @@ std::unique_ptr ClientFactory::create( // NOLINT // These will be added after request signing client_configuration.extra_headers = std::move(headers); - Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key); + Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key, session_token); auto credentials_provider = std::make_shared( client_configuration, std::move(credentials), diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 36edb443681..e1b99c893a6 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -138,6 +138,9 @@ public: /// Returns the initial endpoint. const String & getInitialEndpoint() const { return initial_endpoint; } + const String & getRegion() const { return explicit_region; } + + Aws::Auth::AWSCredentials getCredentials() const; /// Decorator for RetryStrategy needed for this client to work correctly. /// We want to manually handle permanent moves (status code 301) because: @@ -207,7 +210,7 @@ private: Client(size_t max_redirects_, ServerSideEncryptionKMSConfig sse_kms_config_, - const std::shared_ptr& credentials_provider, + const std::shared_ptr & credentials_provider_, const Aws::Client::ClientConfiguration& client_configuration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, bool use_virtual_addressing); @@ -247,6 +250,7 @@ private: void insertRegionOverride(const std::string & bucket, const std::string & region) const; String initial_endpoint; + std::shared_ptr credentials_provider; std::string explicit_region; mutable bool detect_region = true; @@ -282,7 +286,8 @@ public: const String & server_side_encryption_customer_key_base64, ServerSideEncryptionKMSConfig sse_kms_config, HTTPHeaderEntries headers, - CredentialsConfiguration credentials_configuration); + CredentialsConfiguration credentials_configuration, + const String & session_token = ""); PocoHTTPClientConfiguration createClientConfiguration( const String & force_region, From 10a382bd12de3b31c54655587e28cd06a1123360 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 19 May 2023 10:49:43 -0400 Subject: [PATCH 0434/2223] add ALTER and DROP --- docs/en/operations/named-collections.md | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 1866315529e..8ad403e448e 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -57,8 +57,7 @@ In the above example the `passowrd_sha256_hex` value is the hexadecimal represen ### XML example -```xml -$ cat /etc/clickhouse-server/config.d/named_collections.xml +```xml title='/etc/clickhouse-server/config.d/named_collections.xml' @@ -70,6 +69,27 @@ $ cat /etc/clickhouse-server/config.d/named_collections.xml ``` +## Modifying named collections + +Named collections that are created with DDL queries can be altered or dropped with DDL. Named collections created with XML files can be managed by editing or deleting the corresponding XML. + +### Alter a DDL named collection + +Change or add the keys `key1` and `key3` of the collection `collection2`: +```sql +ALTER NAMED COLLECTION collection2 SET key1=4, key3='value3' +``` + +Remove the key `key2` from `collection2`: +```sql +ALTER NAMED COLLECTION collection2 DELETE key2 +``` + +### Drop the DDL named collection `collection2`: +```sql +DROP NAMED COLLECTION collection2 +``` + ## Named collections for accessing S3 The description of parameters see [s3 Table Function](../sql-reference/table-functions/s3.md). @@ -389,3 +409,4 @@ SELECT dictGet('dict', 'b', 1); │ a │ └─────────────────────────┘ ``` + From ff63a73cf535fd9b18b20e6986748f8f5fc93a9f Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 19 May 2023 11:03:53 -0400 Subject: [PATCH 0435/2223] review --- docs/en/operations/named-collections.md | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 8ad403e448e..fc2d7c3540c 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -71,7 +71,7 @@ In the above example the `passowrd_sha256_hex` value is the hexadecimal represen ## Modifying named collections -Named collections that are created with DDL queries can be altered or dropped with DDL. Named collections created with XML files can be managed by editing or deleting the corresponding XML. +Named collections that are created with DDL queries can be altered or dropped with DDL. Named collections created with XML files can be managed by editing or deleting the corresponding XML. ### Alter a DDL named collection @@ -85,6 +85,11 @@ Remove the key `key2` from `collection2`: ALTER NAMED COLLECTION collection2 DELETE key2 ``` +Change or add the key `key1` and delete the key `key3` of the collection `collection2`: +```sql +ALTER NAMED COLLECTION collection2 SET key1=4, DELETE key3 +``` + ### Drop the DDL named collection `collection2`: ```sql DROP NAMED COLLECTION collection2 From 4af818746461a59f73c3e36de291eefdbabf8e95 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 19 May 2023 15:06:02 +0000 Subject: [PATCH 0436/2223] Activate restarting thread in both cases --- .../MergeTree/ReplicatedMergeTreeRestartingThread.h | 8 +++++++- src/Storages/StorageReplicatedMergeTree.cpp | 6 +++--- .../test_replicated_table_attach/configs/config.xml | 3 +++ tests/integration/test_replicated_table_attach/test.py | 3 +++ 4 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 5579fdf2033..9e99baab4c3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -23,7 +23,13 @@ class ReplicatedMergeTreeRestartingThread public: explicit ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_); - void start() { task->activateAndSchedule(); } + void start(bool schedule = true) + { + if (schedule) + task->activateAndSchedule(); + else + task->activate(); + } void wakeup() { task->schedule(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6373c164c40..eccdded9a6f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4416,6 +4416,9 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) startBeingLeader(); + /// Activate replica in a separate thread if we are not calling from attach thread + restarting_thread.start(/*schedule=*/!from_attach_thread); + if (from_attach_thread) { /// Try activating replica in current thread. @@ -4423,9 +4426,6 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) } else { - /// Activate replica in a separate thread. - restarting_thread.start(); - /// Wait while restarting_thread finishing initialization. /// NOTE It does not mean that replication is actually started after receiving this event. /// It only means that an attempt to startup replication was made. diff --git a/tests/integration/test_replicated_table_attach/configs/config.xml b/tests/integration/test_replicated_table_attach/configs/config.xml index b7b5d996072..fea3eab4126 100644 --- a/tests/integration/test_replicated_table_attach/configs/config.xml +++ b/tests/integration/test_replicated_table_attach/configs/config.xml @@ -1,3 +1,6 @@ 1 + + 5 + diff --git a/tests/integration/test_replicated_table_attach/test.py b/tests/integration/test_replicated_table_attach/test.py index 11365ea9163..2d209ddaf79 100644 --- a/tests/integration/test_replicated_table_attach/test.py +++ b/tests/integration/test_replicated_table_attach/test.py @@ -57,4 +57,7 @@ def test_startup_with_small_bg_pool_partitioned(started_cluster): node.restart_clickhouse(stop_start_wait_sec=20) assert_values() + # check that we activate it in the end + node.query_with_retry("INSERT INTO replicated_table_partitioned VALUES(20, 30)") + node.query("DROP TABLE replicated_table_partitioned SYNC") From 90872c2671f468cef899834d8fb15af6e5896960 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 19 May 2023 17:22:37 +0200 Subject: [PATCH 0437/2223] Update query_log.md Fix links to data types --- docs/en/operations/system-tables/query_log.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 42247e6fba2..1bcecfeb161 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -97,8 +97,8 @@ Columns: - `forwarded_for` ([String](../../sql-reference/data-types/string.md)) — HTTP header `X-Forwarded-For` passed in the HTTP query. - `quota_key` ([String](../../sql-reference/data-types/string.md)) — The `quota key` specified in the [quotas](../../operations/quotas.md) setting (see `keyed`). - `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse revision. -- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/array.md)) — ProfileEvents that measure different metrics. The description of them could be found in the table [system.events](../../operations/system-tables/events.md#system_tables-events) -- `Settings` ([Map(String, String)](../../sql-reference/data-types/array.md)) — Settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1. +- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/map.md)) — ProfileEvents that measure different metrics. The description of them could be found in the table [system.events](../../operations/system-tables/events.md#system_tables-events) +- `Settings` ([Map(String, String)](../../sql-reference/data-types/map.md)) — Settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1. - `log_comment` ([String](../../sql-reference/data-types/string.md)) — Log comment. It can be set to arbitrary string no longer than [max_query_size](../../operations/settings/settings.md#settings-max_query_size). An empty string if it is not defined. - `thread_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Thread ids that are participating in query execution. - `used_aggregate_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `aggregate functions`, which were used during query execution. From 3e42ee7f2baf73707fdcabcd4f4afeeafbbbc14e Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 May 2023 17:22:57 +0200 Subject: [PATCH 0438/2223] Get rid of finalize callback in object storages --- src/Backups/BackupIO_S3.cpp | 1 + src/Coordination/KeeperSnapshotManagerS3.cpp | 1 + src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- src/Disks/IO/WriteBufferFromAzureBlobStorage.h | 7 +++++-- ...oteFS.cpp => WriteBufferWithFinalizeCallback.cpp} | 10 +++++----- ...mRemoteFS.h => WriteBufferWithFinalizeCallback.h} | 12 ++++++------ .../AzureBlobStorage/AzureObjectStorage.cpp | 5 +---- .../AzureBlobStorage/AzureObjectStorage.h | 2 -- .../ObjectStorages/Cached/CachedObjectStorage.cpp | 3 +-- .../ObjectStorages/Cached/CachedObjectStorage.h | 1 - .../ObjectStorages/DiskObjectStorageTransaction.cpp | 9 ++++++--- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp | 6 +----- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h | 1 - src/Disks/ObjectStorages/IObjectStorage.h | 3 --- .../ObjectStorages/Local/LocalObjectStorage.cpp | 6 +----- src/Disks/ObjectStorages/Local/LocalObjectStorage.h | 1 - src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 10 +++------- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 1 - src/Disks/ObjectStorages/Web/WebObjectStorage.cpp | 2 -- src/Disks/ObjectStorages/Web/WebObjectStorage.h | 1 - src/IO/WriteBufferFromS3.cpp | 4 +++- src/IO/WriteBufferFromS3.h | 8 +++++--- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 3 ++- src/Storages/HDFS/WriteBufferFromHDFS.h | 9 +++++---- src/Storages/StorageS3.cpp | 1 + 25 files changed, 48 insertions(+), 61 deletions(-) rename src/Disks/IO/{WriteIndirectBufferFromRemoteFS.cpp => WriteBufferWithFinalizeCallback.cpp} (64%) rename src/Disks/IO/{WriteIndirectBufferFromRemoteFS.h => WriteBufferWithFinalizeCallback.h} (59%) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 40ecde71173..f1fd276e34b 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -255,6 +255,7 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, + DBMS_DEFAULT_BUFFER_SIZE, request_settings, std::nullopt, threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3"), diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index ffb3a0bc18b..1afe0b352c5 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -149,6 +149,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa s3_client->client, s3_client->uri.bucket, key, + DBMS_DEFAULT_BUFFER_SIZE, request_settings_1 }; }; diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 44185f74f60..b5d296bd865 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -26,7 +26,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( size_t max_single_part_upload_size_, size_t buf_size_, const WriteSettings & write_settings_) - : BufferWithOwnMemory(buf_size_, nullptr, 0) + : WriteBufferFromFileBase(buf_size_, nullptr, 0) , log(&Poco::Logger::get("WriteBufferFromAzureBlobStorage")) , max_single_part_upload_size(max_single_part_upload_size_) , blob_path(blob_path_) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 27492809384..f1be81922e1 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -6,7 +6,7 @@ #include -#include +#include #include #include #include @@ -21,7 +21,7 @@ class Logger; namespace DB { -class WriteBufferFromAzureBlobStorage : public BufferWithOwnMemory +class WriteBufferFromAzureBlobStorage : public WriteBufferFromFileBase { public: using AzureClientPtr = std::shared_ptr; @@ -37,6 +37,9 @@ public: void nextImpl() override; + std::string getFileName() const override { return blob_path; } + void sync() override { next(); } + private: void finalizeImpl() override; void execWithRetry(std::function func, size_t num_tries, size_t cost = 0); diff --git a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp similarity index 64% rename from src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp rename to src/Disks/IO/WriteBufferWithFinalizeCallback.cpp index a909e8f109c..49e230b9dc3 100644 --- a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp @@ -1,11 +1,11 @@ -#include "WriteIndirectBufferFromRemoteFS.h" +#include "WriteBufferWithFinalizeCallback.h" namespace DB { -WriteIndirectBufferFromRemoteFS::WriteIndirectBufferFromRemoteFS( +WriteBufferWithFinalizeCallback::WriteBufferWithFinalizeCallback( std::unique_ptr impl_, - CreateMetadataCallback && create_callback_, + FinalizeCallback && create_callback_, const String & remote_path_) : WriteBufferFromFileDecorator(std::move(impl_)) , create_metadata_callback(std::move(create_callback_)) @@ -14,7 +14,7 @@ WriteIndirectBufferFromRemoteFS::WriteIndirectBufferFromRemoteFS( } -WriteIndirectBufferFromRemoteFS::~WriteIndirectBufferFromRemoteFS() +WriteBufferWithFinalizeCallback::~WriteBufferWithFinalizeCallback() { try { @@ -26,7 +26,7 @@ WriteIndirectBufferFromRemoteFS::~WriteIndirectBufferFromRemoteFS() } } -void WriteIndirectBufferFromRemoteFS::finalizeImpl() +void WriteBufferWithFinalizeCallback::finalizeImpl() { WriteBufferFromFileDecorator::finalizeImpl(); if (create_metadata_callback) diff --git a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.h b/src/Disks/IO/WriteBufferWithFinalizeCallback.h similarity index 59% rename from src/Disks/IO/WriteIndirectBufferFromRemoteFS.h rename to src/Disks/IO/WriteBufferWithFinalizeCallback.h index f06ebe89589..73c1b8d25d4 100644 --- a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.h @@ -8,25 +8,25 @@ namespace DB { -using CreateMetadataCallback = std::function; +using FinalizeCallback = std::function; /// Stores data in S3/HDFS and adds the object path and object size to metadata file on local FS. -class WriteIndirectBufferFromRemoteFS final : public WriteBufferFromFileDecorator +class WriteBufferWithFinalizeCallback final : public WriteBufferFromFileDecorator { public: - WriteIndirectBufferFromRemoteFS( + WriteBufferWithFinalizeCallback( std::unique_ptr impl_, - CreateMetadataCallback && create_callback_, + FinalizeCallback && create_callback_, const String & remote_path_); - ~WriteIndirectBufferFromRemoteFS() override; + ~WriteBufferWithFinalizeCallback() override; String getFileName() const override { return remote_path; } private: void finalizeImpl() override; - CreateMetadataCallback create_metadata_callback; + FinalizeCallback create_metadata_callback; String remote_path; }; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 9db5d13a7f8..9c45c5f806e 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -129,7 +129,6 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO const StoredObject & object, WriteMode mode, std::optional, - FinalizeCallback && finalize_callback, size_t buf_size, const WriteSettings & write_settings) { @@ -138,14 +137,12 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO LOG_TEST(log, "Writing file: {}", object.remote_path); - auto buffer = std::make_unique( + return std::make_unique( client.get(), object.remote_path, settings.get()->max_single_part_upload_size, buf_size, patchSettings(write_settings)); - - return std::make_unique(std::move(buffer), std::move(finalize_callback), object.remote_path); } void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 648016fb732..276c992e4d7 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -83,7 +82,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index a26969c6175..acf9430e85c 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -97,13 +97,12 @@ std::unique_ptr CachedObjectStorage::writeObject( /// N const StoredObject & object, WriteMode mode, // Cached doesn't support append, only rewrite std::optional attributes, - FinalizeCallback && finalize_callback, size_t buf_size, const WriteSettings & write_settings) { /// Add cache relating settings to WriteSettings. auto modified_write_settings = IObjectStorage::patchSettings(write_settings); - auto implementation_buffer = object_storage->writeObject(object, mode, attributes, std::move(finalize_callback), buf_size, modified_write_settings); + auto implementation_buffer = object_storage->writeObject(object, mode, attributes, buf_size, modified_write_settings); bool cache_on_write = modified_write_settings.enable_filesystem_cache_on_write_operations && FileCacheFactory::instance().getByName(cache_config_name).settings.cache_on_write_operations diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 276f6669bfa..f8e346e1aed 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -43,7 +43,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 2c22df64d90..257a6fdf2ea 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -658,14 +659,16 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile operations_to_execute.emplace_back(std::move(write_operation)); - /// We always use mode Rewrite because we simulate append using metadata and different files - return object_storage.writeObject( + auto impl = object_storage.writeObject( object, + /// We always use mode Rewrite because we simulate append using metadata and different files WriteMode::Rewrite, object_attributes, - std::move(create_metadata_callback), buf_size, settings); + + return std::make_unique( + std::move(impl), std::move(create_metadata_callback), object.remote_path); } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index e50e410823d..e746668dd40 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include @@ -83,7 +82,6 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL const StoredObject & object, WriteMode mode, std::optional attributes, - FinalizeCallback && finalize_callback, size_t buf_size, const WriteSettings & write_settings) { @@ -93,11 +91,9 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL "HDFS API doesn't support custom attributes/metadata for stored objects"); /// Single O_WRONLY in libhdfs adds O_TRUNC - auto hdfs_buffer = std::make_unique( + return std::make_unique( object.remote_path, config, settings->replication, patchSettings(write_settings), buf_size, mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND); - - return std::make_unique(std::move(hdfs_buffer), std::move(finalize_callback), object.remote_path); } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 4064a5c5b7f..d4eac33b33e 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -81,7 +81,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index f4f1b063ade..f129bd22758 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -48,8 +48,6 @@ struct ObjectMetadata std::optional attributes; }; -using FinalizeCallback = std::function; - /// Base class for all object storages which implement some subset of ordinary filesystem operations. /// /// Examples of object storages are S3, Azure Blob Storage, HDFS. @@ -119,7 +117,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) = 0; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 8cf0b27a517..e1e90df591b 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -124,7 +123,6 @@ std::unique_ptr LocalObjectStorage::writeObject( /// NO const StoredObject & object, WriteMode mode, std::optional /* attributes */, - FinalizeCallback && finalize_callback, size_t buf_size, const WriteSettings & /* write_settings */) { @@ -132,9 +130,7 @@ std::unique_ptr LocalObjectStorage::writeObject( /// NO throw Exception(ErrorCodes::BAD_ARGUMENTS, "LocalObjectStorage doesn't support append to files"); LOG_TEST(log, "Write object: {}", object.remote_path); - auto impl = std::make_unique(object.remote_path, buf_size); - return std::make_unique( - std::move(impl), std::move(finalize_callback), object.remote_path); + return std::make_unique(object.remote_path, buf_size); } void LocalObjectStorage::removeObject(const StoredObject & object) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h index 2d31ccff398..630320ab7f9 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h @@ -41,7 +41,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 79b3d3a2b8b..a82174ff1b1 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -160,8 +159,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN const StoredObject & object, WriteMode mode, // S3 doesn't support append, only rewrite std::optional attributes, - FinalizeCallback && finalize_callback, - size_t buf_size [[maybe_unused]], + size_t buf_size, const WriteSettings & write_settings) { WriteSettings disk_write_settings = IObjectStorage::patchSettings(write_settings); @@ -174,17 +172,15 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN if (write_settings.s3_allow_parallel_part_upload) scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); - auto s3_buffer = std::make_unique( + return std::make_unique( client.get(), bucket, object.remote_path, + buf_size, settings_ptr->request_settings, attributes, std::move(scheduler), disk_write_settings); - - return std::make_unique( - std::move(s3_buffer), std::move(finalize_callback), object.remote_path); } void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 766c1aa033d..70ed899586e 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -97,7 +97,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index c57b75f4038..4ff18383713 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -9,7 +9,6 @@ #include #include -#include #include #include #include @@ -211,7 +210,6 @@ std::unique_ptr WebObjectStorage::writeObject( /// NOLI const StoredObject & /* object */, WriteMode /* mode */, std::optional /* attributes */, - FinalizeCallback && /* finalize_callback */, size_t /* buf_size */, const WriteSettings & /* write_settings */) { diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index f55e853b4fa..e85b7224892 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -51,7 +51,6 @@ public: const StoredObject & object, WriteMode mode, std::optional attributes = {}, - FinalizeCallback && finalize_callback = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 01ab8ff7cbb..c7a3b131d09 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -79,11 +79,13 @@ WriteBufferFromS3::WriteBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, const String & key_, + size_t buf_size_, const S3Settings::RequestSettings & request_settings_, std::optional> object_metadata_, ThreadPoolCallbackRunner schedule_, const WriteSettings & write_settings_) - : bucket(bucket_) + : WriteBufferFromFileBase(buf_size_, nullptr, 0) + , bucket(bucket_) , key(key_) , request_settings(request_settings_) , upload_settings(request_settings.getUploadSettings()) diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index ac6c430606f..f4200b0a646 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -5,7 +5,7 @@ #if USE_AWS_S3 #include -#include +#include #include #include #include @@ -24,13 +24,14 @@ namespace DB * Data is divided on chunks with size greater than 'minimum_upload_part_size'. Last chunk can be less than this threshold. * Each chunk is written as a part to S3. */ -class WriteBufferFromS3 final : public BufferWithOwnMemory +class WriteBufferFromS3 final : public WriteBufferFromFileBase { public: WriteBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, const String & key_, + size_t buf_size_, const S3Settings::RequestSettings & request_settings_, std::optional> object_metadata_ = std::nullopt, ThreadPoolCallbackRunner schedule_ = {}, @@ -39,8 +40,9 @@ public: ~WriteBufferFromS3() override; void nextImpl() override; void preFinalize() override; + std::string getFileName() const override { return key; } + void sync() override { next(); } -public: class IBufferAllocationPolicy { public: diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 293f5d31836..fad0447d2cf 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -107,8 +107,9 @@ WriteBufferFromHDFS::WriteBufferFromHDFS( const WriteSettings & write_settings_, size_t buf_size_, int flags_) - : BufferWithOwnMemory(buf_size_) + : WriteBufferFromFileBase(buf_size_, nullptr, 0) , impl(std::make_unique(hdfs_name_, config_, replication_, write_settings_, flags_)) + , filename(hdfs_name_) { } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.h b/src/Storages/HDFS/WriteBufferFromHDFS.h index ec54348c174..71e6e55addc 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/HDFS/WriteBufferFromHDFS.h @@ -5,7 +5,7 @@ #if USE_HDFS #include #include -#include +#include #include #include #include @@ -17,7 +17,7 @@ namespace DB /** Accepts HDFS path to file and opens it. * Closes file by himself (thus "owns" a file descriptor). */ -class WriteBufferFromHDFS final : public BufferWithOwnMemory +class WriteBufferFromHDFS final : public WriteBufferFromFileBase { public: @@ -29,19 +29,20 @@ public: size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags = O_WRONLY); - WriteBufferFromHDFS(WriteBufferFromHDFS &&) = default; - ~WriteBufferFromHDFS() override; void nextImpl() override; void sync() override; + std::string getFileName() const override { return filename; } + private: void finalizeImpl() override; struct WriteBufferFromHDFSImpl; std::unique_ptr impl; + const std::string filename; }; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a4d9dc9f2e3..4723693ac3a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -775,6 +775,7 @@ public: configuration_.client, bucket, key, + DBMS_DEFAULT_BUFFER_SIZE, configuration_.request_settings, std::nullopt, threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelWrite"), From 791bb6cd4c40e5ca9cf22e303a5daffcab729a62 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 May 2023 17:35:01 +0200 Subject: [PATCH 0439/2223] Fix style check --- programs/keeper/Keeper.cpp | 2 -- programs/server/Server.cpp | 2 -- src/Common/makeSocketAddress.h | 3 +++ 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 8976653f4e6..7633465c5e5 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -78,8 +78,6 @@ namespace ErrorCodes extern const int NO_ELEMENTS_IN_CONFIG; extern const int SUPPORT_IS_DISABLED; extern const int NETWORK_ERROR; - extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA; - extern const int FAILED_TO_GETPWUID; extern const int LOGICAL_ERROR; } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 56938977970..03ada89e86f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -213,8 +213,6 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int INVALID_CONFIG_PARAMETER; - extern const int FAILED_TO_GETPWUID; - extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA; extern const int NETWORK_ERROR; extern const int CORRUPTED_DATA; } diff --git a/src/Common/makeSocketAddress.h b/src/Common/makeSocketAddress.h index 0e68a4ac077..9c7d10a0471 100644 --- a/src/Common/makeSocketAddress.h +++ b/src/Common/makeSocketAddress.h @@ -1,8 +1,11 @@ +#pragma once #include namespace Poco { class Logger; } namespace DB { + Poco::Net::SocketAddress makeSocketAddress(const std::string & host, uint16_t port, Poco::Logger * log); + } From c85c3afa1f50307d9a92d24559fe9628fe8cee37 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Sun, 7 May 2023 12:18:52 +0000 Subject: [PATCH 0440/2223] Added option to rename files, loaded via TableFunctionFile, after success processing --- src/Client/ClientBase.cpp | 1 + src/Common/FileRenamer.cpp | 99 ++++++++++++++++++++++++ src/Common/FileRenamer.h | 39 ++++++++++ src/Core/Settings.h | 2 + src/Storages/StorageFile.cpp | 64 +++++++++++++++ src/Storages/StorageFile.h | 10 ++- src/TableFunctions/TableFunctionFile.cpp | 1 + 7 files changed, 214 insertions(+), 2 deletions(-) create mode 100644 src/Common/FileRenamer.cpp create mode 100644 src/Common/FileRenamer.h diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 77a93a25e9b..571637c6005 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1361,6 +1361,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des columns_description_for_query, ConstraintsDescription{}, String{}, + {}, }; StoragePtr storage = std::make_shared(in_file, global_context->getUserFilesPath(), args); storage->startup(); diff --git a/src/Common/FileRenamer.cpp b/src/Common/FileRenamer.cpp new file mode 100644 index 00000000000..7a19c50a0d1 --- /dev/null +++ b/src/Common/FileRenamer.cpp @@ -0,0 +1,99 @@ +#include + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace fs = std::filesystem; + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +FileRenamer::FileRenamer() = default; + +FileRenamer::FileRenamer(const String & renaming_rule) + : rule(renaming_rule) +{ + FileRenamer::validateRenamingRule(rule, true); +} + +String FileRenamer::generateNewFilename(const String & filename) const +{ + // Split filename and extension + String file_base = fs::path(filename).stem(); + String file_ext = fs::path(filename).extension(); + + // Get current timestamp in microseconds + String timestamp; + if (rule.find("%t") != String::npos) + { + auto now = std::chrono::system_clock::now(); + std::stringstream ss; + ss << timeInMicroseconds(now); + timestamp = ss.str(); + } + + // Define placeholders and their corresponding values + std::map placeholders = + { + {"%f", file_base}, + {"%e", file_ext}, + {"%t", timestamp}, + {"%%", "%"} + }; + + // Replace placeholders with their actual values + String new_name = rule; + for (const auto & [placeholder, value] : placeholders) + boost::replace_all(new_name, placeholder, value); + + return new_name; +} + +bool FileRenamer::isEmpty() const +{ + return rule.empty(); +} + +bool FileRenamer::validateRenamingRule(const String & rule, bool throw_on_error) +{ + // Check if the rule contains invalid placeholders + re2::RE2 invalid_placeholder_pattern("^([^%]|%[fet%])*$"); + if (!re2::RE2::FullMatch(rule, invalid_placeholder_pattern)) + { + if (throw_on_error) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid renaming rule: Allowed placeholders only %f, %e, %t, and %%"); + return false; + } + + // Replace valid placeholders with empty strings and count remaining percentage signs. + String replaced_rule = rule; + boost::replace_all(replaced_rule, "%f", ""); + boost::replace_all(replaced_rule, "%e", ""); + boost::replace_all(replaced_rule, "%t", ""); + if (std::count(replaced_rule.begin(), replaced_rule.end(), '%') % 2) + { + if (throw_on_error) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid renaming rule: Odd number of consecutive percentage signs"); + return false; + } + + return true; +} + + +} // DB diff --git a/src/Common/FileRenamer.h b/src/Common/FileRenamer.h new file mode 100644 index 00000000000..c062978d6f6 --- /dev/null +++ b/src/Common/FileRenamer.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/** + * The FileRenamer class provides functionality for renaming files based on given pattern with placeholders + * The supported placeholders are: + * %f - Original filename without extension ("sample") + * %e - Original file extension with dot (".csv") + * %t - Timestamp (in microseconds) + * %% - Percentage sign ("%") + * + * Example: + * Pattern - "processed_%f_%t%e" + * Original filename - "sample.csv" + * New filename - "processed_sample_1683405960646224.csv" + */ +class FileRenamer +{ +public: + FileRenamer(); + + FileRenamer(const String & renaming_rule); + + String generateNewFilename(const String & filename) const; + + bool isEmpty() const; + + static bool validateRenamingRule(const String & rule, bool throw_on_error = false); + +private: + String rule; +}; + +} // DB diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 69546011770..d0ce641efb5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -713,6 +713,8 @@ class IColumn; M(String, workload, "default", "Name of workload to be used to access resources", 0) \ M(Milliseconds, storage_system_stack_trace_pipe_read_timeout_ms, 100, "Maximum time to read from a pipe for receiving information from the threads when querying the `system.stack_trace` table. This setting is used for testing purposes and not meant to be changed by users.", 0) \ \ + M(String, rename_files_after_processing, "", "Rename successfully processed files according to the specified pattern; Pattern can include the following placeholders: `%f` (original filename without extension), `%e` (file extension with dot), `%t` (current timestamp in µs), and `%%` (% sign)", 0) \ + \ M(Bool, parallelize_output_from_storages, true, "Parallelize output for reading step from storage. It allows parallelizing query processing right after reading from storage if possible", 0) \ M(String, insert_deduplication_token, "", "If not empty, used for duplicate detection instead of data digest", 0) \ M(String, ann_index_select_query_params, "", "Parameters passed to ANN indexes in SELECT queries, the format is 'param1=x, param2=y, ...'", 0) \ diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 425fe6bee31..2ea8da1a873 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -40,6 +40,7 @@ #include #include #include +#include #include #include @@ -76,6 +77,7 @@ namespace ErrorCodes extern const int UNKNOWN_IDENTIFIER; extern const int INCORRECT_FILE_NAME; extern const int FILE_DOESNT_EXIST; + extern const int FILE_ALREADY_EXISTS; extern const int TIMEOUT_EXCEEDED; extern const int INCOMPATIBLE_COLUMNS; extern const int CANNOT_STAT; @@ -460,6 +462,8 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us else path_for_partitioned_write = table_path_; + file_renamer = FileRenamer(args.rename_after_processing); + setStorageMetadata(args); } @@ -593,9 +597,68 @@ public: shared_lock = std::shared_lock(storage->rwlock, getLockTimeout(context)); if (!shared_lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); + storage->readers_counter.fetch_add(1, std::memory_order_release); } } + + /** + * If specified option --rename_files_after_processing and files created by TableFunctionFile + * Last reader will rename files according to specified patten if desctuctor of reader was called without uncaught exceptions + */ + void beforeDestroy() + { + if (storage->file_renamer.isEmpty()) + return; + + int32_t cnt = storage->readers_counter.fetch_sub(1, std::memory_order_acq_rel); + + if (std::uncaught_exceptions() == 0 && cnt == 1 && !storage->was_renamed) + { + shared_lock.unlock(); + auto exclusive_lock = std::unique_lock{storage->rwlock, getLockTimeout(context)}; + + if (!exclusive_lock) + return; + if (storage->readers_counter.load(std::memory_order_acquire) != 0 || storage->was_renamed) + return; + + for (auto & file_path_ref : storage->paths) { + try + { + auto file_path = fs::path(file_path_ref); + String new_filename = storage->file_renamer.generateNewFilename(file_path.filename().string()); + file_path.replace_filename(new_filename); + + // Normalize new path + file_path = file_path.lexically_normal(); + + // Checking access rights + checkCreationIsAllowed(context, context->getUserFilesPath(), file_path, true); + + // Checking an existing of new file + if (fs::exists(file_path)) + throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "File {} already exists", file_path.string()); + + fs::rename(fs::path(file_path_ref), file_path); + file_path_ref = file_path.string(); + storage->was_renamed = true; + } + catch (const std::exception & e) + { + // Cannot throw exception from destructor, will write only error + LOG_ERROR(&Poco::Logger::get("~StorageFileSource"), "Failed to rename file {}: {}", file_path_ref, e.what()); + continue; + } + } + } + } + + ~StorageFileSource() override + { + beforeDestroy(); + } + String getName() const override { return storage->getName(); @@ -1217,6 +1280,7 @@ void registerStorageFile(StorageFactory & factory) factory_args.columns, factory_args.constraints, factory_args.comment, + {}, }; ASTs & engine_args_ast = factory_args.engine_args; diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 53ce7eeaaf6..0513864fd0f 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -2,12 +2,11 @@ #include #include - +#include #include #include - namespace DB { @@ -23,6 +22,8 @@ public: const ColumnsDescription & columns; const ConstraintsDescription & constraints; const String & comment; + + const std::string rename_after_processing; }; /// From file descriptor @@ -139,6 +140,11 @@ private: std::unique_ptr read_buffer_from_fd; std::unique_ptr peekable_read_buffer_from_fd; std::atomic has_peekable_read_buffer_from_fd = false; + + // Counts the number of readers + std::atomic readers_counter = 0; + FileRenamer file_renamer; + bool was_renamed = false; }; } diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index ff64bb3dc67..0e49f26db40 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -75,6 +75,7 @@ StoragePtr TableFunctionFile::getStorage(const String & source, columns, ConstraintsDescription{}, String{}, + global_context->getSettingsRef().rename_files_after_processing, }; if (fd >= 0) return std::make_shared(fd, args); From 7b3964ff7a63ce6a4027f0dfb8f06c019239ac1c Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Sun, 7 May 2023 14:59:40 +0000 Subject: [PATCH 0441/2223] Added test --- .../02732_rename_after_processing.reference | 17 ++++ .../02732_rename_after_processing.sh | 77 +++++++++++++++++++ 2 files changed, 94 insertions(+) create mode 100644 tests/queries/0_stateless/02732_rename_after_processing.reference create mode 100755 tests/queries/0_stateless/02732_rename_after_processing.sh diff --git a/tests/queries/0_stateless/02732_rename_after_processing.reference b/tests/queries/0_stateless/02732_rename_after_processing.reference new file mode 100644 index 00000000000..26e152f6b10 --- /dev/null +++ b/tests/queries/0_stateless/02732_rename_after_processing.reference @@ -0,0 +1,17 @@ +4 +processed_tmp1.csv +OK +10 10 +processed_tmp2.csv +OK +8 +processed_tmp3_1.csv +processed_tmp3_2.csv +OK +OK +4 +OK +OK +tmp5.csv +OK +tmp5.csv diff --git a/tests/queries/0_stateless/02732_rename_after_processing.sh b/tests/queries/0_stateless/02732_rename_after_processing.sh new file mode 100755 index 00000000000..05fbfb716ec --- /dev/null +++ b/tests/queries/0_stateless/02732_rename_after_processing.sh @@ -0,0 +1,77 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# see 01658_read_file_to_stringcolumn.sh +CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +# Prepare data +mkdir -p ${CLICKHOUSE_USER_FILES_PATH} +echo '"id","str","int","text"' > ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv +echo '1,"abc",123,"abacaba"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv +echo '2,"def",456,"bacabaa"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv +echo '3,"story",78912,"acabaab"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv +echo '4,"history",21321321,"cabaaba"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv + +cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp1.csv +cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp2.csv +cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp3_1.csv +cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp3_2.csv +cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp4.csv +cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp5.csv + +### Checking that renaming works + +# simple select +${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f%e" -q "SELECT COUNT(*) FROM file('tmp1.csv')" +ls ${CLICKHOUSE_USER_FILES_PATH} | grep "processed_tmp1.csv" +if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp1.csv" ]; then + echo "OK" +fi + +# select with multiple file() calls +${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f%e" --multiline -q """ +SELECT + sum(a.id) as aid, + sum(b.id) as bid +FROM file('tmp2.csv') AS a +INNER JOIN file('tmp2.csv') AS b +ON a.text = b.text +""" +ls ${CLICKHOUSE_USER_FILES_PATH} | grep "processed_tmp2.csv" +if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp2.csv" ]; then + echo "OK" +fi + +# rename multiple files +${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f%e" -q "SELECT COUNT(*) FROM file('tmp3*.csv')" +ls ${CLICKHOUSE_USER_FILES_PATH} | grep "processed_tmp3_1.csv" +ls ${CLICKHOUSE_USER_FILES_PATH} | grep "processed_tmp3_2.csv" +if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp3_1.csv" ]; then + echo "OK" +fi +if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp3_2.csv" ]; then + echo "OK" +fi + +# check timestamp placeholder +${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f_%t.csv" -q "SELECT COUNT(*) FROM file('tmp4.csv')" +ls ${CLICKHOUSE_USER_FILES_PATH} | grep -E "^processed_tmp4_[0-9]+\.csv$" > /dev/null && echo "OK" + +### Checking errors + +# cannot overwrite an existing file +${CLICKHOUSE_CLIENT} --rename-files-after-processing="tmp.csv" -q "SELECT COUNT(*) FROM file('tmp5.csv')" \ + 2>&1| grep "already exists" > /dev/null && echo "OK" +ls ${CLICKHOUSE_USER_FILES_PATH} | grep "tmp5.csv" + +# сannot move file from user_files +${CLICKHOUSE_CLIENT} --rename-files-after-processing="../%f%e" -q "SELECT COUNT(*) FROM file('tmp5.csv')" \ + 2>&1| grep "is not inside" > /dev/null && echo "OK" +ls ${CLICKHOUSE_USER_FILES_PATH} | grep "tmp5.csv" + +# Clean +rm -rd $CLICKHOUSE_USER_FILES_PATH \ No newline at end of file From 2b68a6a22a84329f639af953e1fdaad9a6e21584 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Sun, 7 May 2023 17:43:34 +0000 Subject: [PATCH 0442/2223] Fix style --- src/Common/FileRenamer.cpp | 5 +-- src/Storages/StorageFile.cpp | 5 ++- .../02732_rename_after_processing.reference | 8 ++-- .../02732_rename_after_processing.sh | 43 +++++++++++++------ 4 files changed, 39 insertions(+), 22 deletions(-) diff --git a/src/Common/FileRenamer.cpp b/src/Common/FileRenamer.cpp index 7a19c50a0d1..3473d543c00 100644 --- a/src/Common/FileRenamer.cpp +++ b/src/Common/FileRenamer.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include @@ -42,9 +41,7 @@ String FileRenamer::generateNewFilename(const String & filename) const if (rule.find("%t") != String::npos) { auto now = std::chrono::system_clock::now(); - std::stringstream ss; - ss << timeInMicroseconds(now); - timestamp = ss.str(); + timestamp = std::to_string(timeInMicroseconds(now)); } // Define placeholders and their corresponding values diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 2ea8da1a873..06af0a00953 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -604,7 +604,7 @@ public: /** * If specified option --rename_files_after_processing and files created by TableFunctionFile - * Last reader will rename files according to specified patten if desctuctor of reader was called without uncaught exceptions + * Last reader will rename files according to specified pattern if desctuctor of reader was called without uncaught exceptions */ void beforeDestroy() { @@ -623,7 +623,8 @@ public: if (storage->readers_counter.load(std::memory_order_acquire) != 0 || storage->was_renamed) return; - for (auto & file_path_ref : storage->paths) { + for (auto & file_path_ref : storage->paths) + { try { auto file_path = fs::path(file_path_ref); diff --git a/tests/queries/0_stateless/02732_rename_after_processing.reference b/tests/queries/0_stateless/02732_rename_after_processing.reference index 26e152f6b10..2f6ccfc1c5e 100644 --- a/tests/queries/0_stateless/02732_rename_after_processing.reference +++ b/tests/queries/0_stateless/02732_rename_after_processing.reference @@ -1,14 +1,14 @@ 4 processed_tmp1.csv -OK +!tmp1.csv 10 10 processed_tmp2.csv -OK +!tmp2.csv 8 processed_tmp3_1.csv processed_tmp3_2.csv -OK -OK +!tmp3_1.csv +!tmp3_2.csv 4 OK OK diff --git a/tests/queries/0_stateless/02732_rename_after_processing.sh b/tests/queries/0_stateless/02732_rename_after_processing.sh index 05fbfb716ec..93bad2eac7d 100755 --- a/tests/queries/0_stateless/02732_rename_after_processing.sh +++ b/tests/queries/0_stateless/02732_rename_after_processing.sh @@ -27,9 +27,11 @@ cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp5.csv # simple select ${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f%e" -q "SELECT COUNT(*) FROM file('tmp1.csv')" -ls ${CLICKHOUSE_USER_FILES_PATH} | grep "processed_tmp1.csv" +if [ -e "${CLICKHOUSE_USER_FILES_PATH}/processed_tmp1.csv" ]; then + echo "processed_tmp1.csv" +fi if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp1.csv" ]; then - echo "OK" + echo "!tmp1.csv" fi # select with multiple file() calls @@ -41,37 +43,54 @@ FROM file('tmp2.csv') AS a INNER JOIN file('tmp2.csv') AS b ON a.text = b.text """ -ls ${CLICKHOUSE_USER_FILES_PATH} | grep "processed_tmp2.csv" +if [ -e "${CLICKHOUSE_USER_FILES_PATH}/processed_tmp2.csv" ]; then + echo "processed_tmp2.csv" +fi if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp2.csv" ]; then - echo "OK" + echo "!tmp2.csv" fi # rename multiple files ${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f%e" -q "SELECT COUNT(*) FROM file('tmp3*.csv')" -ls ${CLICKHOUSE_USER_FILES_PATH} | grep "processed_tmp3_1.csv" -ls ${CLICKHOUSE_USER_FILES_PATH} | grep "processed_tmp3_2.csv" +if [ -e "${CLICKHOUSE_USER_FILES_PATH}/processed_tmp3_1.csv" ]; then + echo "processed_tmp3_1.csv" +fi +if [ -e "${CLICKHOUSE_USER_FILES_PATH}/processed_tmp3_2.csv" ]; then + echo "processed_tmp3_2.csv" +fi if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp3_1.csv" ]; then - echo "OK" + echo "!tmp3_1.csv" fi if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp3_2.csv" ]; then - echo "OK" + echo "!tmp3_2.csv" fi # check timestamp placeholder ${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f_%t.csv" -q "SELECT COUNT(*) FROM file('tmp4.csv')" -ls ${CLICKHOUSE_USER_FILES_PATH} | grep -E "^processed_tmp4_[0-9]+\.csv$" > /dev/null && echo "OK" +# ls ${CLICKHOUSE_USER_FILES_PATH} | grep -E "^processed_tmp4_[0-9]+\.csv$" > /dev/null && echo "OK" +rg="processed_tmp4_[0-9]+\.csv" +for x in "${CLICKHOUSE_USER_FILES_PATH}"/processed*; do + if [[ $x =~ $rg ]]; then + echo "OK" + break + fi; +done ### Checking errors # cannot overwrite an existing file ${CLICKHOUSE_CLIENT} --rename-files-after-processing="tmp.csv" -q "SELECT COUNT(*) FROM file('tmp5.csv')" \ 2>&1| grep "already exists" > /dev/null && echo "OK" -ls ${CLICKHOUSE_USER_FILES_PATH} | grep "tmp5.csv" +if [ -e "${CLICKHOUSE_USER_FILES_PATH}/tmp5.csv" ]; then + echo "tmp5.csv" +fi # сannot move file from user_files ${CLICKHOUSE_CLIENT} --rename-files-after-processing="../%f%e" -q "SELECT COUNT(*) FROM file('tmp5.csv')" \ 2>&1| grep "is not inside" > /dev/null && echo "OK" -ls ${CLICKHOUSE_USER_FILES_PATH} | grep "tmp5.csv" +if [ -e "${CLICKHOUSE_USER_FILES_PATH}/tmp5.csv" ]; then + echo "tmp5.csv" +fi # Clean -rm -rd $CLICKHOUSE_USER_FILES_PATH \ No newline at end of file +rm -rd $CLICKHOUSE_USER_FILES_PATH From eb7b48aab20172b79b1573c05c8a4baa02fe0804 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Tue, 16 May 2023 18:48:39 +0000 Subject: [PATCH 0443/2223] Fix test issues --- .../02732_rename_after_processing.reference | 4 + .../02732_rename_after_processing.sh | 85 +++++++++++-------- 2 files changed, 55 insertions(+), 34 deletions(-) diff --git a/tests/queries/0_stateless/02732_rename_after_processing.reference b/tests/queries/0_stateless/02732_rename_after_processing.reference index 2f6ccfc1c5e..39cdb677e09 100644 --- a/tests/queries/0_stateless/02732_rename_after_processing.reference +++ b/tests/queries/0_stateless/02732_rename_after_processing.reference @@ -15,3 +15,7 @@ OK tmp5.csv OK tmp5.csv +OK +tmp5.csv +OK +tmp5.csv diff --git a/tests/queries/0_stateless/02732_rename_after_processing.sh b/tests/queries/0_stateless/02732_rename_after_processing.sh index 93bad2eac7d..dbf2427d2dc 100755 --- a/tests/queries/0_stateless/02732_rename_after_processing.sh +++ b/tests/queries/0_stateless/02732_rename_after_processing.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -9,28 +8,30 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') # Prepare data -mkdir -p ${CLICKHOUSE_USER_FILES_PATH} -echo '"id","str","int","text"' > ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv -echo '1,"abc",123,"abacaba"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv -echo '2,"def",456,"bacabaa"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv -echo '3,"story",78912,"acabaab"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv -echo '4,"history",21321321,"cabaaba"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv +unique_name=${CLICKHOUSE_TEST_UNIQUE_NAME} +tmp_dir=${CLICKHOUSE_USER_FILES_PATH}/${unique_name} +mkdir -p $tmp_dir +echo '"id","str","int","text"' > ${tmp_dir}/tmp.csv +echo '1,"abc",123,"abacaba"' >> ${tmp_dir}/tmp.csv +echo '2,"def",456,"bacabaa"' >> ${tmp_dir}/tmp.csv +echo '3,"story",78912,"acabaab"' >> ${tmp_dir}/tmp.csv +echo '4,"history",21321321,"cabaaba"' >> ${tmp_dir}/tmp.csv -cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp1.csv -cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp2.csv -cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp3_1.csv -cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp3_2.csv -cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp4.csv -cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp5.csv +cp ${tmp_dir}/tmp.csv ${tmp_dir}/tmp1.csv +cp ${tmp_dir}/tmp.csv ${tmp_dir}/tmp2.csv +cp ${tmp_dir}/tmp.csv ${tmp_dir}/tmp3_1.csv +cp ${tmp_dir}/tmp.csv ${tmp_dir}/tmp3_2.csv +cp ${tmp_dir}/tmp.csv ${tmp_dir}/tmp4.csv +cp ${tmp_dir}/tmp.csv ${tmp_dir}/tmp5.csv ### Checking that renaming works # simple select -${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f%e" -q "SELECT COUNT(*) FROM file('tmp1.csv')" -if [ -e "${CLICKHOUSE_USER_FILES_PATH}/processed_tmp1.csv" ]; then +${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f%e" -q "SELECT COUNT(*) FROM file('${unique_name}/tmp1.csv')" +if [ -e "${tmp_dir}/processed_tmp1.csv" ]; then echo "processed_tmp1.csv" fi -if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp1.csv" ]; then +if [ ! -e "${tmp_dir}/tmp1.csv" ]; then echo "!tmp1.csv" fi @@ -39,37 +40,37 @@ ${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f%e" --multilin SELECT sum(a.id) as aid, sum(b.id) as bid -FROM file('tmp2.csv') AS a -INNER JOIN file('tmp2.csv') AS b +FROM file('${unique_name}/tmp2.csv') AS a +INNER JOIN file('${unique_name}/tmp2.csv') AS b ON a.text = b.text """ -if [ -e "${CLICKHOUSE_USER_FILES_PATH}/processed_tmp2.csv" ]; then +if [ -e "${tmp_dir}/processed_tmp2.csv" ]; then echo "processed_tmp2.csv" fi -if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp2.csv" ]; then +if [ ! -e "${tmp_dir}/tmp2.csv" ]; then echo "!tmp2.csv" fi # rename multiple files -${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f%e" -q "SELECT COUNT(*) FROM file('tmp3*.csv')" -if [ -e "${CLICKHOUSE_USER_FILES_PATH}/processed_tmp3_1.csv" ]; then +${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f%e" -q "SELECT COUNT(*) FROM file('${unique_name}/tmp3*.csv')" +if [ -e "${tmp_dir}/processed_tmp3_1.csv" ]; then echo "processed_tmp3_1.csv" fi -if [ -e "${CLICKHOUSE_USER_FILES_PATH}/processed_tmp3_2.csv" ]; then +if [ -e "${tmp_dir}/processed_tmp3_2.csv" ]; then echo "processed_tmp3_2.csv" fi -if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp3_1.csv" ]; then +if [ ! -e "${tmp_dir}/tmp3_1.csv" ]; then echo "!tmp3_1.csv" fi -if [ ! -e "${CLICKHOUSE_USER_FILES_PATH}/tmp3_2.csv" ]; then +if [ ! -e "${tmp_dir}/tmp3_2.csv" ]; then echo "!tmp3_2.csv" fi # check timestamp placeholder -${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f_%t.csv" -q "SELECT COUNT(*) FROM file('tmp4.csv')" -# ls ${CLICKHOUSE_USER_FILES_PATH} | grep -E "^processed_tmp4_[0-9]+\.csv$" > /dev/null && echo "OK" +${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f_%t.csv" -q "SELECT COUNT(*) FROM file('${unique_name}/tmp4.csv')" +# ls ${tmp_dir} | grep -E "^processed_tmp4_[0-9]+\.csv$" > /dev/null && echo "OK" rg="processed_tmp4_[0-9]+\.csv" -for x in "${CLICKHOUSE_USER_FILES_PATH}"/processed*; do +for x in "${tmp_dir}"/processed*; do if [[ $x =~ $rg ]]; then echo "OK" break @@ -79,18 +80,34 @@ done ### Checking errors # cannot overwrite an existing file -${CLICKHOUSE_CLIENT} --rename-files-after-processing="tmp.csv" -q "SELECT COUNT(*) FROM file('tmp5.csv')" \ +${CLICKHOUSE_CLIENT} --rename-files-after-processing="tmp.csv" -q "SELECT COUNT(*) FROM file('${unique_name}/tmp5.csv')" \ 2>&1| grep "already exists" > /dev/null && echo "OK" -if [ -e "${CLICKHOUSE_USER_FILES_PATH}/tmp5.csv" ]; then +if [ -e "${tmp_dir}/tmp5.csv" ]; then echo "tmp5.csv" fi -# сannot move file from user_files -${CLICKHOUSE_CLIENT} --rename-files-after-processing="../%f%e" -q "SELECT COUNT(*) FROM file('tmp5.csv')" \ +# сannot move file outside user_files +${CLICKHOUSE_CLIENT} --rename-files-after-processing="../../%f%e" -q "SELECT COUNT(*) FROM file('${unique_name}/tmp5.csv')" \ 2>&1| grep "is not inside" > /dev/null && echo "OK" -if [ -e "${CLICKHOUSE_USER_FILES_PATH}/tmp5.csv" ]; then +if [ -e "${tmp_dir}/tmp5.csv" ]; then + echo "tmp5.csv" +fi + +# check invalid placeholders + +# unknown type of placeholder (%k) +${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f_%k" -q "SELECT COUNT(*) FROM file('${unique_name}/tmp5.csv')" \ + 2>&1| grep "Allowed placeholders only" > /dev/null && echo "OK" +if [ -e "${tmp_dir}/tmp5.csv" ]; then + echo "tmp5.csv" +fi + +# dd number of consecutive percentage signs after replace valid placeholders +${CLICKHOUSE_CLIENT} --rename-files-after-processing="processed_%f_%%%%e" -q "SELECT COUNT(*) FROM file('${unique_name}/tmp5.csv')" \ + 2>&1| grep "Odd number of consecutive percentage signs" > /dev/null && echo "OK" +if [ -e "${tmp_dir}/tmp5.csv" ]; then echo "tmp5.csv" fi # Clean -rm -rd $CLICKHOUSE_USER_FILES_PATH +rm -rd $tmp_dir From 5e33dd5d5c9eace011d4c729e3c760c017955de9 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Wed, 17 May 2023 19:14:46 +0000 Subject: [PATCH 0444/2223] Added chmod for tmp dir and files in test --- tests/queries/0_stateless/02732_rename_after_processing.sh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/02732_rename_after_processing.sh b/tests/queries/0_stateless/02732_rename_after_processing.sh index dbf2427d2dc..4c87070cd35 100755 --- a/tests/queries/0_stateless/02732_rename_after_processing.sh +++ b/tests/queries/0_stateless/02732_rename_after_processing.sh @@ -11,12 +11,18 @@ CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from unique_name=${CLICKHOUSE_TEST_UNIQUE_NAME} tmp_dir=${CLICKHOUSE_USER_FILES_PATH}/${unique_name} mkdir -p $tmp_dir +rm -rf ${tmp_dir}/* + +chmod 777 ${tmp_dir} + echo '"id","str","int","text"' > ${tmp_dir}/tmp.csv echo '1,"abc",123,"abacaba"' >> ${tmp_dir}/tmp.csv echo '2,"def",456,"bacabaa"' >> ${tmp_dir}/tmp.csv echo '3,"story",78912,"acabaab"' >> ${tmp_dir}/tmp.csv echo '4,"history",21321321,"cabaaba"' >> ${tmp_dir}/tmp.csv +chmod 777 ${tmp_dir}/tmp.csv + cp ${tmp_dir}/tmp.csv ${tmp_dir}/tmp1.csv cp ${tmp_dir}/tmp.csv ${tmp_dir}/tmp2.csv cp ${tmp_dir}/tmp.csv ${tmp_dir}/tmp3_1.csv From 4550a705f97497607a627f5cca8f265b650abdc1 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Wed, 17 May 2023 19:49:17 +0000 Subject: [PATCH 0445/2223] Fix style --- tests/queries/0_stateless/02732_rename_after_processing.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02732_rename_after_processing.sh b/tests/queries/0_stateless/02732_rename_after_processing.sh index 4c87070cd35..c4f80d3462b 100755 --- a/tests/queries/0_stateless/02732_rename_after_processing.sh +++ b/tests/queries/0_stateless/02732_rename_after_processing.sh @@ -11,7 +11,7 @@ CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from unique_name=${CLICKHOUSE_TEST_UNIQUE_NAME} tmp_dir=${CLICKHOUSE_USER_FILES_PATH}/${unique_name} mkdir -p $tmp_dir -rm -rf ${tmp_dir}/* +rm -rf ${tmp_dir:?}/* chmod 777 ${tmp_dir} From 900d50633d9c541eff5ac05bdef19cd84e3d6c16 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Wed, 17 May 2023 20:15:26 +0000 Subject: [PATCH 0446/2223] retrigger checks From 5368355c659ef2f502ea9787593700c1ae03067d Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Fri, 19 May 2023 16:05:51 +0000 Subject: [PATCH 0447/2223] Marked test as broken due to fail of the new analyzer --- tests/broken_tests.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index e61c1316e17..cef8f68b210 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -137,3 +137,4 @@ 01600_parts_types_metrics_long 01287_max_execution_speed 02703_row_policy_for_database +02732_rename_after_processing From 4260e86b1a1dd05ba9ca65cb6cba3c61a636bb06 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 19 May 2023 13:00:18 -0400 Subject: [PATCH 0448/2223] fix typo --- docs/en/operations/named-collections.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index fc2d7c3540c..7db8f2b642b 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -50,7 +50,7 @@ To manage named collections with DDL a user must have the `named_control_collect ``` :::tip -In the above example the `passowrd_sha256_hex` value is the hexadecimal representation of the SHA256 hash of the password. This configuration for the user `default` has the attribute `replace=true` as in the default configuration has a plain text `password` set, and it is not possible to have both plaintext and sha256 hex passwords set for a user. +In the above example the `passowrd_sha256_hex` value is the hexadecimal representation of the SHA256 hash of the password. This configuration for the user `default` has the attribute `replace=true` as in the default configuration has a plain text `password` set, and it is not possible to have both plain text and sha256 hex passwords set for a user. ::: ## Storing named collections in configuration files From 35e77f8e2a8aaecc1d3328863ab38670908fab72 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 19 May 2023 17:53:22 +0000 Subject: [PATCH 0449/2223] fix comment --- src/Interpreters/Cluster.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index b696b539013..8c30dbe230f 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -487,8 +487,8 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown element in config: {}", replica_key); } - addShard(settings, std::move(replica_addresses), false, current_shard_num, - std::move(insert_paths), /* treat_local_as_remote */ weight, internal_replication); + addShard(settings, std::move(replica_addresses), /* treat_local_as_remote = */ false, current_shard_num, + std::move(insert_paths), weight, internal_replication); } else throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown element in config: {}", key); From 086888b2856b52edcbca2ff9c1d949ed1c48142a Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 19 May 2023 17:54:29 +0000 Subject: [PATCH 0450/2223] fix ConnectionPoolWithFailover::getPriority --- src/Client/ConnectionPoolWithFailover.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index e0c46804c60..129bc10bc27 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -73,9 +73,9 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts Int64 ConnectionPoolWithFailover::getPriority() const { - return (*std::max_element(nested_pools.begin(), nested_pools.end(), [](const auto &a, const auto &b) + return (*std::max_element(nested_pools.begin(), nested_pools.end(), [](const auto & a, const auto & b) { - return a->getPriority() - b->getPriority(); + return a->getPriority() < b->getPriority(); }))->getPriority(); } From d69c35fcdd4b183f94acffbf8fc6e73988d39ac1 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 19 May 2023 17:57:00 +0000 Subject: [PATCH 0451/2223] fix PoolWithFailover `error_count` integer overflow --- src/Client/ConnectionPool.h | 1 - src/Common/PoolWithFailoverBase.h | 8 ++++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index 9b1f7dc5b9d..aacd0a063c7 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -135,7 +135,6 @@ private: Protocol::Compression compression; /// Whether to compress data when interacting with the server. Protocol::Secure secure; /// Whether to encrypt data when interacting with the server. Int64 priority; /// priority from - }; /** diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 0e8fbb4e6d1..928d8b74a3f 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -101,7 +101,7 @@ public: struct ShuffledPool { NestedPool * pool{}; - const PoolState * state{}; + const PoolState * state{}; // WARNING: valid only during initial ordering, dangling size_t index = 0; size_t error_count = 0; size_t slowdown_count = 0; @@ -115,7 +115,6 @@ public: /// this functor. The pools with lower result value will be tried first. using GetPriorityFunc = std::function; - /// Returns at least min_entries and at most max_entries connections (at most one connection per nested pool). /// The method will throw if it is unable to get min_entries alive connections or /// if fallback_to_stale_replicas is false and it is unable to get min_entries connections to up-to-date replicas. @@ -175,10 +174,11 @@ PoolWithFailoverBase::getShuffledPools( } /// Sort the pools into order in which they will be tried (based on respective PoolStates). + /// Note that `error_count` and `slowdown_count` are used for ordering, but set to zero in the resulting ShuffledPool std::vector shuffled_pools; shuffled_pools.reserve(nested_pools.size()); for (size_t i = 0; i < nested_pools.size(); ++i) - shuffled_pools.push_back(ShuffledPool{nested_pools[i].get(), &pool_states[i], i, 0}); + shuffled_pools.push_back(ShuffledPool{nested_pools[i].get(), &pool_states[i], i, /* error_count = */ 0, /* slowdown_count = */ 0}); ::sort( shuffled_pools.begin(), shuffled_pools.end(), [](const ShuffledPool & lhs, const ShuffledPool & rhs) @@ -371,7 +371,7 @@ PoolWithFailoverBase::updatePoolStates(size_t max_ignored_errors) /// distributed_replica_max_ignored_errors for (auto & state : result) - state.error_count = std::max(0, state.error_count - max_ignored_errors); + state.error_count = state.error_count > max_ignored_errors ? state.error_count - max_ignored_errors : 0; return result; } From 052d8aca71e0639769620dae96ae060c1b5e6e70 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 19 May 2023 18:13:29 +0000 Subject: [PATCH 0452/2223] limit `max_tries` value by `max_error_cap` to avoid unlimited number of retries --- src/Common/PoolWithFailoverBase.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 928d8b74a3f..646e10d6443 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -227,6 +227,10 @@ PoolWithFailoverBase::getMany( { std::vector shuffled_pools = getShuffledPools(max_ignored_errors, get_priority); + /// Limit `max_tries` value by `max_error_cap` to avoid unlimited number of retries + if (max_tries > max_error_cap) + max_tries = max_error_cap; + /// We will try to get a connection from each pool until a connection is produced or max_tries is reached. std::vector try_results(shuffled_pools.size()); size_t entries_count = 0; From 50919e0cc611f09fa7444eeb14845a2b451305d6 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 19 May 2023 14:24:50 -0400 Subject: [PATCH 0453/2223] Update src/Core/Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1941c20baf..610c7135a75 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -156,7 +156,7 @@ class IColumn; M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ \ M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \ - M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table (up to `max_parallel_replicas`). It will work for any kind of MergeTree table.", 0) \ + M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ \ From 6fd5d8e8bab77a5265b6beb3f20b101960c2ab9c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 19 May 2023 02:07:30 +0000 Subject: [PATCH 0454/2223] Add setting output_format_parquet_compliant_nested_types to produce more compatible Parquet files --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 5 +++-- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/ParquetBlockOutputFormat.cpp | 11 +++++++++-- 5 files changed, 15 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 69546011770..682a7569f3a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -913,6 +913,7 @@ class IColumn; M(Bool, output_format_parquet_fixed_string_as_fixed_byte_array, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary for FixedString columns.", 0) \ M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \ M(ParquetCompression, output_format_parquet_compression_method, "lz4", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \ + M(Bool, output_format_parquet_compliant_nested_types, true, "In parquet file schema, use name 'element' instead of 'item' for list elements. This is a historical artifact of Arrow library implementation. Generally increases compatibility, except perhaps with some old versions of Arrow.", 0) \ M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index fb2adff2e88..388cad54791 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,8 +80,9 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reade to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}}}, + {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, + {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, + {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 082ccb85970..dd6252b96f1 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -122,6 +122,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.output_fixed_string_as_fixed_byte_array = settings.output_format_parquet_fixed_string_as_fixed_byte_array; format_settings.parquet.max_block_size = settings.input_format_parquet_max_block_size; format_settings.parquet.output_compression_method = settings.output_format_parquet_compression_method; + format_settings.parquet.output_compliant_nested_types = settings.output_format_parquet_compliant_nested_types; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index b9a55ffaa83..c88af650671 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -220,6 +220,7 @@ struct FormatSettings UInt64 max_block_size = 8192; ParquetVersion output_version; ParquetCompression output_compression_method = ParquetCompression::SNAPPY; + bool output_compliant_nested_types = true; } parquet; struct Pretty diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 0a78fd2725a..e39271efa7c 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -155,12 +155,19 @@ void ParquetBlockOutputFormat::write(Chunk chunk, size_t row_group_size) parquet::WriterProperties::Builder builder; builder.version(getParquetVersion(format_settings)); builder.compression(getParquetCompression(format_settings.parquet.output_compression_method)); - auto props = builder.build(); + + parquet::ArrowWriterProperties::Builder writer_props_builder; + if (format_settings.parquet.output_compliant_nested_types) + writer_props_builder.enable_compliant_nested_types(); + else + writer_props_builder.disable_compliant_nested_types(); + auto result = parquet::arrow::FileWriter::Open( *arrow_table->schema(), arrow::default_memory_pool(), sink, - props); + builder.build(), + writer_props_builder.build()); if (!result.ok()) throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while opening a table: {}", result.status().ToString()); file_writer = std::move(result.ValueOrDie()); From 1ded6d64097dccf0585e814f2c3d522d59912b73 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 19 May 2023 16:53:26 -0400 Subject: [PATCH 0455/2223] Update grant.md --- docs/en/sql-reference/statements/grant.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index b6d6f285f3d..e6073f3523a 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -9,7 +9,7 @@ sidebar_label: GRANT - Grants [privileges](#grant-privileges) to ClickHouse user accounts or roles. - Assigns roles to user accounts or to the other roles. -To revoke privileges, use the [REVOKE](../../sql-reference/statements/revoke.md) statement. Also you can list granted privileges with the [SHOW GRANTS](../../sql-reference/statements/show.md#show-grants-statement) statement. +To revoke privileges, use the [REVOKE](../../sql-reference/statements/revoke.md) statement. Also you can list granted privileges with the [SHOW GRANTS](../../sql-reference/statements/show.md#show-grants) statement. ## Granting Privilege Syntax From c93836b9620f2bd424d5f6132404a455c94a39dd Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 19 May 2023 22:26:53 +0000 Subject: [PATCH 0456/2223] fix --- src/Storages/MergeTree/MergeTreeReadPool.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 1de5361a7db..4e95a210bae 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -56,6 +56,13 @@ MergeTreeReadPool::MergeTreeReadPool( , backoff_settings{context_->getSettingsRef()} , backoff_state{threads_} { + /// parts don't contain duplicate MergeTreeDataPart's. + const auto per_part_sum_marks = fillPerPartInfo( + parts_ranges, storage_snapshot, is_part_on_remote_disk, + predict_block_size_bytes, + column_names, virtual_column_names, prewhere_info, + actions_settings, reader_settings, per_part_params); + if (std::ranges::count(is_part_on_remote_disk, true)) { const auto & settings = context_->getSettingsRef(); @@ -82,13 +89,6 @@ MergeTreeReadPool::MergeTreeReadPool( } } - /// parts don't contain duplicate MergeTreeDataPart's. - const auto per_part_sum_marks = fillPerPartInfo( - parts_ranges, storage_snapshot, is_part_on_remote_disk, - predict_block_size_bytes, - column_names, virtual_column_names, prewhere_info, - actions_settings, reader_settings, per_part_params); - fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges); } From f459ac5517bcac49c7e6d583fca827504d2b0aa1 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 20 May 2023 01:38:35 +0200 Subject: [PATCH 0457/2223] resolve ambiguity by adding a section to docs --- docs/en/operations/settings/settings.md | 20 ++++++++++++++++++++ docs/ru/operations/settings/settings.md | 20 ++++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cf9209e182f..9cca1ee5ec3 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4077,6 +4077,26 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric 1999-12-13 07:23:23.123 ``` +:::warning +The way this setting affects parsing of Date or DateTime types may seem non-obvious, see example and explanation below: +::: + +```sql +CREATE TABLE test_tz (`d` DateTime('UTC')) ENGINE = Memory AS SELECT toDateTime('2000-01-01 00:00:00', 'UTC'); + +SELECT *, timezone() FROM test_tz WHERE d = toDateTime('2000-01-01 00:00:00') SETTINGS session_timezone = 'Asia/Novosibirsk' +0 rows in set. + +SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS session_timezone = 'Asia/Novosibirsk' +┌───────────────────d─┬─timezone()───────┐ +│ 2000-01-01 00:00:00 │ Asia/Novosibirsk │ +└─────────────────────┴──────────────────┘ +``` + +This happens due to different parsing pipelines: + - `toDateTime('2000-01-01 00:00:00')` creates a new DateTime in a usual way, and thus `session_timezone` setting from query context is applied. + - `2000-01-01 00:00:00` is parsed to a DateTime inheriting type of `d` column, including DateTime's time zone, and `session_timezone` has no impact on this value. + Possible values: - Any timezone name from `system.time_zones`, e.g. `Europe/Berlin`, `UTC` or `Zulu` diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 98486847fd9..607082054cc 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4105,6 +4105,26 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric Значение по умолчанию: `''`. +:::warning +То, как этот параметр влияет на парсинг значений типа Date или DateTime, может показаться неочевидным. Пример и пояснение см. ниже: +::: + +```sql +CREATE TABLE test_tz (`d` DateTime('UTC')) ENGINE = Memory AS SELECT toDateTime('2000-01-01 00:00:00', 'UTC'); + +SELECT *, timezone() FROM test_tz WHERE d = toDateTime('2000-01-01 00:00:00') SETTINGS session_timezone = 'Asia/Novosibirsk' +0 rows in set. + +SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS session_timezone = 'Asia/Novosibirsk' +┌───────────────────d─┬─timezone()───────┐ +│ 2000-01-01 00:00:00 │ Asia/Novosibirsk │ +└─────────────────────┴──────────────────┘ +``` + +Это происходит из-за различного происхождения значения, используемого для сравнения: +- `toDateTime('2000-01-01 00:00:00')` создаёт значение типа `DateTime` как и в любом другом случае, в том числе применяет параметр `session_timezone` из контекста запроса, +- `2000-01-01 00:00:00` парсится в `DateTime` того же типа, что и колонка `d` (в том числе с той же `timezone`), и параметр `session_timezone` в данном случае не учитывается. + **Смотрите также** - [timezone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone) \ No newline at end of file From 6e80537ab61b95e5d6aa85a50532efc336ed4cdf Mon Sep 17 00:00:00 2001 From: LiuYangkuan Date: Sat, 20 May 2023 12:41:48 +0800 Subject: [PATCH 0458/2223] support passing fqdn to register cluster node in keeper --- src/Interpreters/ClusterDiscovery.cpp | 10 +++++++++- src/Interpreters/ClusterDiscovery.h | 13 ++++++++++--- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 02046dec066..ce59dfcae45 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -125,10 +125,12 @@ ClusterDiscovery::ClusterDiscovery( ClusterInfo( /* name_= */ key, /* zk_root_= */ config.getString(prefix + ".path"), + /* host_name= */ config.getString(prefix + ".my_hostname", getFQDNOrHostName()), /* port= */ context->getTCPPort(), /* secure= */ config.getBool(prefix + ".secure", false), /* shard_id= */ config.getUInt(prefix + ".shard", 0), - /* observer_mode= */ ConfigHelper::getBool(config, prefix + ".observer") + /* observer_mode= */ ConfigHelper::getBool(config, prefix + ".observer"), + /* invisible_mode= */ ConfigHelper::getBool(config, prefix + ".invisible") ) ); } @@ -294,6 +296,12 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) return false; } + if (cluster_info.current_cluster_is_invisible) + { + LOG_DEBUG(log, "cluster '{}' is invisible!", cluster_info.name); + return true; + } + if (!needUpdate(node_uuids, nodes_info)) { LOG_DEBUG(log, "No update required for cluster '{}'", cluster_info.name); diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 8c976ca1b7f..140e3691c03 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -78,16 +77,24 @@ private: /// Current node may not belong to cluster, to be just an observer. bool current_node_is_observer = false; + /// For internal management need. + /// Is it designed that when deploying multiple compute groups, + /// they are mutually invisible to each other. + bool current_cluster_is_invisible = false; + explicit ClusterInfo(const String & name_, const String & zk_root_, + const String & host_name, UInt16 port, bool secure, size_t shard_id, - bool observer_mode) + bool observer_mode, + bool invisible) : name(name_) , zk_root(zk_root_) - , current_node(getFQDNOrHostName() + ":" + toString(port), secure, shard_id) + , current_node(host_name + ":" + toString(port), secure, shard_id) , current_node_is_observer(observer_mode) + , current_cluster_is_invisible(invisible) { } }; From 7189481fad990824fddeee045c97e66d2cd4985c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 May 2023 09:00:28 +0200 Subject: [PATCH 0459/2223] Preserve backward incompatibility for renamed settings by using aliases - optimize_use_projections/allow_experimental_projection_optimization - enable_lightweight_delete/allow_experimental_lightweight_delete Signed-off-by: Azat Khuzhin --- src/Core/Settings.h | 6 ++---- .../0_stateless/02319_lightweight_delete_on_merge_tree.sql | 2 +- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 21dc267749b..b9e728a3ca4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -507,7 +507,7 @@ class IColumn; M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ - M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) \ + M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ @@ -557,7 +557,7 @@ class IColumn; M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ - M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) \ + M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \ @@ -764,7 +764,6 @@ class IColumn; MAKE_OBSOLETE(M, Bool, allow_experimental_database_atomic, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_bigint_types, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_window_functions, true) \ - MAKE_OBSOLETE(M, Bool, allow_experimental_lightweight_delete, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_geo_types, true) \ \ MAKE_OBSOLETE(M, Milliseconds, async_insert_stale_timeout_ms, 0) \ @@ -777,7 +776,6 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ - MAKE_OBSOLETE(M, Bool, allow_experimental_projection_optimization, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_query_cache, true) \ /* moved to config.xml: see also src/Core/ServerSettings.h */ \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_buffer_flush_schedule_pool_size, 16) \ diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index 9413c664293..050b8e37722 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -110,7 +110,7 @@ DROP TABLE t_proj; CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTree order by id settings min_bytes_for_wide_part=0; SET allow_experimental_lightweight_delete = false; -DELETE FROM merge_table_standard_delete WHERE id = 10; -- allow_experimental_lightweight_delete=false is now ignored +DELETE FROM merge_table_standard_delete WHERE id = 10; -- { serverError SUPPORT_IS_DISABLED } SET enable_lightweight_delete = false; DELETE FROM merge_table_standard_delete WHERE id = 10; -- { serverError SUPPORT_IS_DISABLED } DROP TABLE merge_table_standard_delete; From a0480daef393be5616b51e4a2d0e6516c81b050e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 20 May 2023 12:43:24 +0200 Subject: [PATCH 0460/2223] Update waitServersToFinish.h --- src/Server/waitServersToFinish.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Server/waitServersToFinish.h b/src/Server/waitServersToFinish.h index 1f496d703c2..5e90790cefb 100644 --- a/src/Server/waitServersToFinish.h +++ b/src/Server/waitServersToFinish.h @@ -1,4 +1,5 @@ #pragma once +#include namespace DB { From 52c5fd5cb948ad54a9b0bafe35ad0197148584fd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 May 2023 13:10:28 +0200 Subject: [PATCH 0461/2223] Rewrite OpenTelemetry context serialization for DDL without IO/Operators.h This is required to switch to escaped versions. Signed-off-by: Azat Khuzhin --- src/Common/OpenTelemetryTraceContext.cpp | 35 ++++++++++++------------ 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 40d06e71456..f1b7cc8c430 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -5,7 +5,8 @@ #include #include #include -#include +#include +#include #include @@ -249,26 +250,26 @@ String TracingContext::composeTraceparentHeader() const void TracingContext::deserialize(ReadBuffer & buf) { - buf >> this->trace_id - >> "\n" - >> this->span_id - >> "\n" - >> this->tracestate - >> "\n" - >> this->trace_flags - >> "\n"; + readUUIDText(trace_id, buf); + assertChar('\n', buf); + readIntText(span_id, buf); + assertChar('\n', buf); + readString(tracestate, buf); + assertChar('\n', buf); + readIntText(trace_flags, buf); + assertChar('\n', buf); } void TracingContext::serialize(WriteBuffer & buf) const { - buf << this->trace_id - << "\n" - << this->span_id - << "\n" - << this->tracestate - << "\n" - << this->trace_flags - << "\n"; + writeUUIDText(trace_id, buf); + writeChar('\n', buf); + writeIntText(span_id, buf); + writeChar('\n', buf); + writeString(tracestate, buf); + writeChar('\n', buf); + writeIntText(trace_flags, buf); + writeChar('\n', buf); } const TracingContextOnThread & CurrentContext() From cd5413a12df5b808e579936210d5b273b1aa8225 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 May 2023 13:10:51 +0200 Subject: [PATCH 0462/2223] Refactor test 02423_ddl_for_opentelemetry to use clickhouse-client This will allow to add new line into tracestate to check the format breakage Signed-off-by: Azat Khuzhin --- .../02423_ddl_for_opentelemetry.sh | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/02423_ddl_for_opentelemetry.sh b/tests/queries/0_stateless/02423_ddl_for_opentelemetry.sh index b2a5ebdd4ad..93a8c1db342 100755 --- a/tests/queries/0_stateless/02423_ddl_for_opentelemetry.sh +++ b/tests/queries/0_stateless/02423_ddl_for_opentelemetry.sh @@ -15,13 +15,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # $3 - Query Settings function execute_query() { - # Some queries are supposed to fail, use -f to suppress error messages - echo $2 | ${CLICKHOUSE_CURL_COMMAND} -q -s --max-time 180 \ - -X POST \ - -H "traceparent: 00-$1-5150000000000515-01" \ - -H "tracestate: a\nb cd" \ - "${CLICKHOUSE_URL}&${3}" \ - --data @- + local trace_id=$1 && shift + local ddl_version=$1 && shift + local opts=( + --opentelemetry-traceparent "00-$trace_id-5150000000000515-01" + --opentelemetry-tracestate "a\nb cd" + --distributed_ddl_output_mode "none" + --distributed_ddl_entry_format_version "$ddl_version" + ) + ${CLICKHOUSE_CLIENT} "${opts[@]}" "$@" } # This function takes following argument: @@ -82,9 +84,9 @@ for ddl_version in 3 4; do echo "===ddl_format_version ${ddl_version}====" trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(generateUUIDv4()))"); - execute_query $trace_id "CREATE TABLE ${CLICKHOUSE_DATABASE}.ddl_test_for_opentelemetry ON CLUSTER ${cluster_name} (id UInt64) Engine=MergeTree ORDER BY id" "distributed_ddl_output_mode=none&distributed_ddl_entry_format_version=${ddl_version}" + execute_query $trace_id $ddl_version -q "CREATE TABLE ${CLICKHOUSE_DATABASE}.ddl_test_for_opentelemetry ON CLUSTER ${cluster_name} (id UInt64) Engine=MergeTree ORDER BY id" - check_span 1 $trace_id "HTTPHandler" + check_span 1 $trace_id "TCPHandler" if [ $cluster_name = "test_shard_localhost" ]; then check_span 1 $trace_id "%executeDDLQueryOnCluster%" "attribute['clickhouse.cluster']='${cluster_name}'" @@ -106,7 +108,7 @@ for ddl_version in 3 4; do check_span $expected $trace_id "%DDLWorker::processTask%" # For queries that tracing are enabled(format version is 4 or Replicated database engine), there should be two 'query' spans, - # one is for the HTTPHandler, the other is for the DDL executing in DDLWorker. + # one is for the TCPHandler, the other is for the DDL executing in DDLWorker. # # For other format, there should be only one 'query' span if [ $cluster_name = "test_shard_localhost" ]; then @@ -134,9 +136,9 @@ done echo "===exception====" trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(generateUUIDv4()))"); -execute_query $trace_id "DROP TABLE ${CLICKHOUSE_DATABASE}.ddl_test_for_opentelemetry_non_exist ON CLUSTER ${cluster_name}" "distributed_ddl_output_mode=none&distributed_ddl_entry_format_version=4" 2>&1| grep -Fv "UNKNOWN_TABLE" +execute_query $trace_id 4 -q "DROP TABLE ${CLICKHOUSE_DATABASE}.ddl_test_for_opentelemetry_non_exist ON CLUSTER ${cluster_name}" 2>&1 | grep 'DB::Exception ' | grep -Fv "UNKNOWN_TABLE" -check_span 1 $trace_id "HTTPHandler" +check_span 1 $trace_id "TCPHandler" if [ $cluster_name = "test_shard_localhost" ]; then expected=1 @@ -148,7 +150,7 @@ check_span $expected $trace_id "%executeDDLQueryOnCluster%" "attribute['clickhou check_span $expected $trace_id "%DDLWorker::processTask%" "kind = 'CONSUMER'" if [ $cluster_name = "test_shard_localhost" ]; then - # There should be two 'query' spans, one is for the HTTPHandler, the other is for the DDL executing in DDLWorker. + # There should be two 'query' spans, one is for the TCPHandler, the other is for the DDL executing in DDLWorker. # Both of these two spans contain exception expected=2 else From 82054d40a56645e2386452bd4828805d5b4c3a23 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 May 2023 13:12:14 +0200 Subject: [PATCH 0463/2223] Add proper escaping for DDL OpenTelemetry context serialization Before you was able to break the format by using "\n" or "\t", that will simply lead to DDL hang, because DDLWorker will simply log the error and do nothing more: DDLWorker: Cannot parse DDL task query-0000000056: Incorrect task format. Will try to send error status: Code: 27. DB::ParsingException: Cannot parse input: expected '\n' before: 'bar\n1\n'. (CANNOT_PARSE_INPUT_ASSERTION_FAILED) (version 23.5.1.1) Fix this by adding proper escaping. Signed-off-by: Azat Khuzhin --- src/Common/OpenTelemetryTraceContext.cpp | 4 ++-- tests/queries/0_stateless/02423_ddl_for_opentelemetry.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index f1b7cc8c430..9edbc462ceb 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -254,7 +254,7 @@ void TracingContext::deserialize(ReadBuffer & buf) assertChar('\n', buf); readIntText(span_id, buf); assertChar('\n', buf); - readString(tracestate, buf); + readEscapedString(tracestate, buf); assertChar('\n', buf); readIntText(trace_flags, buf); assertChar('\n', buf); @@ -266,7 +266,7 @@ void TracingContext::serialize(WriteBuffer & buf) const writeChar('\n', buf); writeIntText(span_id, buf); writeChar('\n', buf); - writeString(tracestate, buf); + writeEscapedString(tracestate, buf); writeChar('\n', buf); writeIntText(trace_flags, buf); writeChar('\n', buf); diff --git a/tests/queries/0_stateless/02423_ddl_for_opentelemetry.sh b/tests/queries/0_stateless/02423_ddl_for_opentelemetry.sh index 93a8c1db342..7136698d5b7 100755 --- a/tests/queries/0_stateless/02423_ddl_for_opentelemetry.sh +++ b/tests/queries/0_stateless/02423_ddl_for_opentelemetry.sh @@ -19,7 +19,7 @@ function execute_query() local ddl_version=$1 && shift local opts=( --opentelemetry-traceparent "00-$trace_id-5150000000000515-01" - --opentelemetry-tracestate "a\nb cd" + --opentelemetry-tracestate $'a\nb cd' --distributed_ddl_output_mode "none" --distributed_ddl_entry_format_version "$ddl_version" ) From 8924c17575567deaf3ae005b5edc31292417c037 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 May 2023 13:31:27 +0200 Subject: [PATCH 0464/2223] Fix build --- src/IO/S3/tests/gtest_aws_s3_client.cpp | 1 + src/IO/tests/gtest_writebuffer_s3.cpp | 1 + 2 files changed, 2 insertions(+) diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 1e5c643d5e5..a9b5fa03f30 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -92,6 +92,7 @@ void doWriteRequest(std::shared_ptr client, const DB::S3:: client, uri.bucket, uri.key, + DBMS_DEFAULT_BUFFER_SIZE, request_settings ); diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index c0bd6742ea3..bc16af7f779 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -529,6 +529,7 @@ public: client, bucket, file_name, + DBMS_DEFAULT_BUFFER_SIZE, request_settings, std::nullopt, getAsyncPolicy().getScheduler()); From a9e834aea4bdea39748dd521306f534d9fe59c35 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 May 2023 16:04:15 +0200 Subject: [PATCH 0465/2223] Fix 02735_system_zookeeper_connection for DatabaseReplicated Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02735_system_zookeeper_connection.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql index 10f12177b2e..b409913d183 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-replicated-database DROP TABLE IF EXISTS test_zk_connection_table; From af5793b26e0ee8fc02201f8d0439b5a15a019e7f Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 20 May 2023 16:38:45 +0200 Subject: [PATCH 0466/2223] fix incode docs --- src/Functions/serverConstants.cpp | 28 +++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 1460fc16265..9cd43be50c4 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -160,31 +160,33 @@ REGISTER_FUNCTION(TcpPort) REGISTER_FUNCTION(Timezone) { - factory.registerFunction({ - R"( + factory.registerFunction( + FunctionDocumentation{ + .description=R"( Returns the default timezone for current session. Used as default timezone for parsing DateTime|DateTime64 without explicitly specified timezone. Can be changed with SET timezone = 'New/Tz' [example:timezone] -)", - Documentation::Examples{{"timezone", "SELECT timezone();"}}, - Documentation::Categories{"Constant", "Miscellaneous"} - }); - factory.registerAlias("timeZone", "timezone"); + )", + .examples{{"timezone", "SELECT timezone();", ""}}, + .categories{"Constant", "Miscellaneous"} +}); +factory.registerAlias("timeZone", "timezone"); } REGISTER_FUNCTION(ServerTimezone) { - factory.registerFunction({ - R"( + factory.registerFunction( + FunctionDocumentation{ + .description=R"( Returns the timezone name in which server operates. [example:serverTimezone] -)", - Documentation::Examples{{"serverTimezone", "SELECT serverTimezone();"}}, - Documentation::Categories{"Constant", "Miscellaneous"} - }); + )", + .examples{{"serverTimezone", "SELECT serverTimezone();", ""}}, + .categories{"Constant", "Miscellaneous"} +}); factory.registerAlias("serverTimeZone", "serverTimezone"); factory.registerAlias("servertimezone", "serverTimezone"); } From 2444e93ad73d81ef45ca0f36e2a9478c0483d719 Mon Sep 17 00:00:00 2001 From: Yusuke Tanaka Date: Sun, 21 May 2023 14:42:17 +0900 Subject: [PATCH 0467/2223] Fix broken link to async_insert_deduplicate in doc --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e150060befe..791186dd9c1 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1714,7 +1714,7 @@ Default value: `100000`. ### async_insert_max_query_number {#async-insert-max-query-number} -The maximum number of insert queries per block before being inserted. This setting takes effect only if [async_insert_deduplicate](#settings-async-insert-deduplicate) is enabled. +The maximum number of insert queries per block before being inserted. This setting takes effect only if [async_insert_deduplicate](#async-insert-deduplicate) is enabled. Possible values: @@ -1745,7 +1745,7 @@ Possible values: Default value: `0`. -### async_insert_deduplicate {#settings-async-insert-deduplicate} +### async_insert_deduplicate {#async-insert-deduplicate} Enables or disables insert deduplication of `ASYNC INSERT` (for Replicated\* tables). From c56e6a8b807f44419a923846c85a51c7a568b9fc Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 12:15:06 +0000 Subject: [PATCH 0468/2223] Add more profile events for distributconnections --- src/Client/ConnectionEstablisher.cpp | 5 +++++ src/Common/ProfileEvents.cpp | 2 ++ 2 files changed, 7 insertions(+) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 4be1475939a..98051a50eb3 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -4,6 +4,8 @@ namespace ProfileEvents { + extern const Event DistributedConnectionTries; + extern const Event DistributedConnectionUsable; extern const Event DistributedConnectionMissingTable; extern const Event DistributedConnectionStaleReplica; } @@ -35,6 +37,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: SCOPE_EXIT(is_finished = true); try { + ProfileEvents::increment(ProfileEvents::DistributedConnectionTries); result.entry = pool->get(*timeouts, settings, /* force_connected = */ false); AsyncCallbackSetter async_setter(&*result.entry, std::move(async_callback)); @@ -45,6 +48,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) { result.entry->forceConnected(*timeouts); + ProfileEvents::increment(ProfileEvents::DistributedConnectionUsable); result.is_usable = true; result.is_up_to_date = true; return; @@ -65,6 +69,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: return; } + ProfileEvents::increment(ProfileEvents::DistributedConnectionUsable); result.is_usable = true; UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 82b1f289b9e..ba2a3b2afc1 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -131,6 +131,8 @@ M(ZooKeeperBytesSent, "Number of bytes send over network while communicating with ZooKeeper.") \ M(ZooKeeperBytesReceived, "Number of bytes received over network while communicating with ZooKeeper.") \ \ + M(DistributedConnectionTries, "Total count of distributed connection attempts.") \ + M(DistributedConnectionUsable, "Total count of successful distributed connections to a usable server (with required table, but maybe stale).") \ M(DistributedConnectionFailTry, "Total count when distributed connection fails with retry.") \ M(DistributedConnectionMissingTable, "Number of times we rejected a replica from a distributed query, because it did not contain a table needed for the query.") \ M(DistributedConnectionStaleReplica, "Number of times we rejected a replica from a distributed query, because some table needed for a query had replication lag higher than the configured threshold.") \ From 44b1754ccfe364fa5d62d623e620e31ea1c10251 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 12:43:47 +0000 Subject: [PATCH 0469/2223] more profile events --- src/Common/PoolBase.h | 8 ++++++++ src/Common/ProfileEvents.cpp | 2 ++ 2 files changed, 10 insertions(+) diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index ba19c3e2150..8cabb472d8f 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -7,7 +7,13 @@ #include #include +#include +#include +namespace ProfileEvents +{ + extern const Event ConnectionPoolIsFullMicroseconds; +} namespace DB { @@ -144,6 +150,7 @@ public: return Entry(*items.back()); } + Stopwatch blocked; if (timeout < 0) { LOG_INFO(log, "No free connections in pool. Waiting indefinitely."); @@ -155,6 +162,7 @@ public: LOG_INFO(log, "No free connections in pool. Waiting {} ms.", timeout_ms.count()); available.wait_for(lock, timeout_ms); } + ProfileEvents::increment(ProfileEvents::ConnectionPoolIsFullMicroseconds, blocked.elapsedMicroseconds()); } } diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index ba2a3b2afc1..dd96b724793 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -503,6 +503,8 @@ The server successfully detected this situation and will download merged part fr M(MergeTreeReadTaskRequestsSentElapsedMicroseconds, "Time spent in callbacks requested from the remote server back to the initiator server to choose the read task (for MergeTree tables). Measured on the remote server side.") \ M(MergeTreeAllRangesAnnouncementsSentElapsedMicroseconds, "Time spent in sending the announcement from the remote server to the initiator server about the set of data parts (for MergeTree tables). Measured on the remote server side.") \ \ + M(ConnectionPoolIsFullMicroseconds, "Total time spent waiting for a slot in connection pool.") \ + \ M(LogTest, "Number of log messages with level Test") \ M(LogTrace, "Number of log messages with level Trace") \ M(LogDebug, "Number of log messages with level Debug") \ From 0a3d986e42fc452e79b188ea047a8bbc3c132052 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 21 May 2023 20:53:33 +0800 Subject: [PATCH 0470/2223] Fix reporting projection broken part --- src/Storages/MergeTree/DataPartsExchange.cpp | 7 +-- src/Storages/MergeTree/MergeTreeData.cpp | 10 +++- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../__init__.py | 0 .../configs/testkeeper.xml | 6 ++ .../test.py | 60 +++++++++++++++++++ 6 files changed, 76 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_projection_report_broken_part/__init__.py create mode 100644 tests/integration/test_projection_report_broken_part/configs/testkeeper.xml create mode 100644 tests/integration/test_projection_report_broken_part/test.py diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 4f2ad823c3a..f2e35e2dcd2 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -130,12 +130,7 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write auto report_broken_part = [&]() { - if (part && part->isProjectionPart()) - { - auto parent_part = part->getParentPart()->shared_from_this(); - data.reportBrokenPart(parent_part); - } - else if (part) + if (part) data.reportBrokenPart(part); else LOG_TRACE(log, "Part {} was not found, do not report it as broken", part_name); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cf94e41a992..b21f44baeb5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7420,8 +7420,14 @@ Strings MergeTreeData::getDataPaths() const } -void MergeTreeData::reportBrokenPart(MergeTreeData::DataPartPtr & data_part) const +void MergeTreeData::reportBrokenPart(MergeTreeData::DataPartPtr data_part) const { + if (!data_part) + return; + + if (data_part->isProjectionPart()) + data_part = data_part->getParentPart()->shared_from_this(); + if (data_part->getDataPartStorage().isBroken()) { auto parts = getDataPartsForInternalUsage(); @@ -7433,7 +7439,7 @@ void MergeTreeData::reportBrokenPart(MergeTreeData::DataPartPtr & data_part) con broken_part_callback(part->name); } } - else if (data_part && data_part->getState() == MergeTreeDataPartState::Active) + else if (data_part->getState() == MergeTreeDataPartState::Active) broken_part_callback(data_part->name); else LOG_DEBUG(log, "Will not check potentially broken part {} because it's not active", data_part->getNameWithState()); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 04b008b623c..5488ce72631 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -718,7 +718,7 @@ public: /// Should be called if part data is suspected to be corrupted. /// Has the ability to check all other parts /// which reside on the same disk of the suspicious part. - void reportBrokenPart(MergeTreeData::DataPartPtr & data_part) const; + void reportBrokenPart(MergeTreeData::DataPartPtr data_part) const; /// TODO (alesap) Duplicate method required for compatibility. /// Must be removed. diff --git a/tests/integration/test_projection_report_broken_part/__init__.py b/tests/integration/test_projection_report_broken_part/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_projection_report_broken_part/configs/testkeeper.xml b/tests/integration/test_projection_report_broken_part/configs/testkeeper.xml new file mode 100644 index 00000000000..617371b13fa --- /dev/null +++ b/tests/integration/test_projection_report_broken_part/configs/testkeeper.xml @@ -0,0 +1,6 @@ + + + + testkeeper + + diff --git a/tests/integration/test_projection_report_broken_part/test.py b/tests/integration/test_projection_report_broken_part/test.py new file mode 100644 index 00000000000..768b6ccae4b --- /dev/null +++ b/tests/integration/test_projection_report_broken_part/test.py @@ -0,0 +1,60 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +# pylint: disable=line-too-long + +import pytest +import time + +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=[ + "configs/testkeeper.xml", + ], +) + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_projection_broken_part(): + node.query( + """ + create table test_projection_broken_parts_1 (a int, b int, projection ab (select a, sum(b) group by a)) + engine = ReplicatedMergeTree('/clickhouse-tables/test_projection_broken_parts', 'r1') + order by a settings index_granularity = 1; + + create table test_projection_broken_parts_2 (a int, b int, projection ab (select a, sum(b) group by a)) + engine ReplicatedMergeTree('/clickhouse-tables/test_projection_broken_parts', 'r2') + order by a settings index_granularity = 1; + + insert into test_projection_broken_parts_1 values (1, 1), (1, 2), (1, 3); + + system sync replica test_projection_broken_parts_2; + """ + ) + + # break projection part + node.exec_in_container( + [ + "bash", + "-c", + "rm /var/lib/clickhouse/data/default/test_projection_broken_parts_1/all_0_0_0/ab.proj/data.bin", + ] + ) + + expected_error = "No such file or directory" + assert expected_error in node.query_and_get_error( + "select sum(b) from test_projection_broken_parts_1 group by a" + ) + + time.sleep(2) + + assert int(node.query("select sum(b) from test_projection_broken_parts_1 group by a")) == 6 From 312f751503a12fc9612f071f89a48266dfb42c37 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 21 May 2023 13:08:55 +0000 Subject: [PATCH 0471/2223] Uppercase remaining SQL keywords --- docs/en/sql-reference/dictionaries/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index eb45247e74a..4e9bb2936db 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2206,7 +2206,7 @@ Regular expression tree dictionaries are a special type of dictionary which repr Regular expression tree dictionaries are defined in ClickHouse open-source using the YAMLRegExpTree source which is provided the path to a YAML file containing the regular expression tree. ```sql -create dictionary regexp_dict +CREATE DICTIONARY regexp_dict ( regexp String, name String, @@ -2322,7 +2322,7 @@ clickhouse client \ You can see how to [Insert Local Files](https://clickhouse.com/docs/en/integrations/data-ingestion/insert-local-files) for more details. After we initialize the source table, we can create a RegexpTree by table source: ``` sql -create dictionary regexp_dict +CREATE DICTIONARY regexp_dict ( regexp String, name String, From 9d9d4e3d62ea042952bf808143898529dd86822d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 21 May 2023 13:33:03 +0000 Subject: [PATCH 0472/2223] Some fixups --- docs/en/sql-reference/dictionaries/index.md | 37 ++++++++++----------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 4e9bb2936db..522fe132a66 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2201,7 +2201,7 @@ Result: Regular expression tree dictionaries are a special type of dictionary which represent the mapping from key to attributes using a tree of regular expressions. There are some use cases, e.g. parsing of (user agent)[https://en.wikipedia.org/wiki/User_agent] strings, which can be expressed elegantly with regexp tree dictionaries. -### Use Regular Expression Tree Dictionary in ClickHouse Open-Source Environment +### Use Regular Expression Tree Dictionary in ClickHouse Open-Source Regular expression tree dictionaries are defined in ClickHouse open-source using the YAMLRegExpTree source which is provided the path to a YAML file containing the regular expression tree. @@ -2238,15 +2238,14 @@ The dictionary source `YAMLRegExpTree` represents the structure of a regexp tree version: '10' ``` -This config consists of a list of Regular Expression Tree nodes. Each node has following structure: +This config consists of a list of regular expression tree nodes. Each node has the following structure: -- **regexp** means the regular expression of this node. -- **user defined attributes** is a list of dictionary attributes defined in the dictionary structure. In this case, we have two attributes: `name` and `version`. The first node has both attributes. The second node only has `name` attribute, because the `version` is defined in the children nodes. - - The value of an attribute could contain a **back reference** which refers to a capture group of the matched regular expression. Reference number ranges from 1 to 9 and writes as `$1` or `\1`. During the query execution, the back reference in the value will be replaced by the matched capture group. -- **children nodes** is the children of a regexp tree node, which has their own attributes and children nodes. String matching preceeds in a depth-first fasion. If a string matches any regexp node in the top layer, the dictionary checks if the string matches the children nodes of it. If it matches, we assign the attributes of the matching nodes. If two or more nodes define the same attribute, children nodes have more priority. - - the name of **children nodes** in YAML files can be arbitrary. +- **regexp**: the regular expression of the node. +- **attributes**: a list of user-defined dictionary attributes. In this example, there are two attributes: `name` and `version`. The first node defines both attributes. The second node only defines attribute `name`. Attribute `version` is provided by the child nodes of the second node. + - The value of an attribute may contain **back references**, referring to capture groups of the matched regular expression. In the example, the value of attribute `version` in the first node consists of a back-reference `\1` to capture group `(\d+[\.\d]*)` in the regular expression. Back-reference numbers range from 1 to 9 and are written as `$1` or `\1` (for number 1). The back reference is replaced by the matched capture group during query execution. +- **child nodes**: a list of children of a regexp tree node, each of which has its own attributes and (potentially) children nodes. String matching proceeds in a depth-first fashion. If a string matches a regexp node, the dictionary checks if it also matches the nodes' child nodes. If that is the case, the attributes of the deepest matching node are assigned. Attributes of a child node overwrite equally named attributes of parent nodes. The name of child nodes in YAML files can be arbitrary, e.g. `versions` in above example. -Due to the specialty of regexp tree dictionary, we only allow functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. +Regexp tree dictionaries only allow access using functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. Example: @@ -2262,16 +2261,16 @@ Result: └─────────────────────────────────────────────────────────────────┘ ``` -In this case, we match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the top layer's second node, so the dictionary continues to look into the children nodes and find it matches `3[12]/tclwebkit`. As a result, the value of `name` is `Android` defined in the first layer and the value of `version` is `12` defined the child node. +In this case, we first match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the top layer's second node. The dictionary then continues to look into the child nodes and find that the string also matches `3[12]/tclwebkit`. As a result, the value of attribute `name` is `Android` (defined in the first layer) and the value of `version` is `12` (defined the child node). -With a powerful YAML configure file, we can use RepexpTree dictionary as a UA parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) +With a powerful YAML configure file, we can use a regexp tree dictionaries as a user agent string parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) ### Use Regular Expression Tree Dictionary in ClickHouse Cloud -We have shown how Regular Expression Tree work in the local environment, but we cannot use `YAMLRegExpTree` in the cloud. If we have a local YAML file, we can use this file to create Regular Expression Tree Dictionary in the local environment, then dump this dictionary to a csv file by the `dictionary` table function and [INTO OUTFILE](../statements/select/into-outfile.md) clause. +Above used `YAMLRegExpTree` source works in ClickHouse Open Source but not in ClickHouse Cloud. To use regexp tree dictionaries in ClickHouse could, first create a regexp tree dictionary from a YAML file locally in ClickHouse Open Source, then dump this dictionary into a CSV file using the `dictionary` table function and the [INTO OUTFILE](../statements/select/into-outfile.md) clause. ```sql -select * from dictionary(regexp_dict) into outfile('regexp_dict.csv') +SELECT * FROM dictionary(regexp_dict) INTO OUTFILE('regexp_dict.csv') ``` The content of csv file is: @@ -2285,15 +2284,15 @@ The content of csv file is: 6,2,"3[12]/tclwebkit","['version']","['10']" ``` -The schema of dumped file is always +The schema of dumped file is: -- `id UInt64` represents the id of the RegexpTree node. -- `parent_id UInt64` represents the id of the parent of a node. -- `regexp String` represents the regular expression string. -- `keys Array(String)` represents the names of user defined attributes. -- `values Array(String)` represents the values of user defined attributes. +- `id UInt64`: the id of the RegexpTree node. +- `parent_id UInt64`: the id of the parent of a node. +- `regexp String`: the regular expression string. +- `keys Array(String)`: the names of user-defined attributes. +- `values Array(String)`: the values of user-defined attributes. -On the cloud, we can create a table `regexp_dictionary_source_table` with the above table structure. +To create the dictionary in ClickHouse Cloud, first create a table `regexp_dictionary_source_table` with below table structure: ```sql CREATE TABLE regexp_dictionary_source_table From 491cf8b6e199757b35cef5273ea8c3cea76879b9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 21 May 2023 13:43:05 +0000 Subject: [PATCH 0473/2223] Fix minor mistakes --- docs/en/sql-reference/dictionaries/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 522fe132a66..f593cbe9052 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2261,7 +2261,7 @@ Result: └─────────────────────────────────────────────────────────────────┘ ``` -In this case, we first match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the top layer's second node. The dictionary then continues to look into the child nodes and find that the string also matches `3[12]/tclwebkit`. As a result, the value of attribute `name` is `Android` (defined in the first layer) and the value of `version` is `12` (defined the child node). +In this case, we first match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*)` in the top layer's second node. The dictionary then continues to look into the child nodes and finds that the string also matches `3[12]/tclwebkit`. As a result, the value of attribute `name` is `Android` (defined in the first layer) and the value of attribute `version` is `12` (defined the child node). With a powerful YAML configure file, we can use a regexp tree dictionaries as a user agent string parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) From 128b8e5889691b3a55129dbb3adbec22fa509a4e Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 14:28:16 +0000 Subject: [PATCH 0474/2223] fix tests + add test for dynamic pools --- src/Common/AsyncLoader.cpp | 11 ++++- src/Common/AsyncLoader.h | 21 ++++++--- src/Common/tests/gtest_async_loader.cpp | 60 ++++++++++++++++++++++++- 3 files changed, 81 insertions(+), 11 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index ccb4944b4c1..a359c5b4196 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -42,6 +42,11 @@ std::exception_ptr LoadJob::exception() const return load_exception; } +size_t LoadJob::execution_pool() const +{ + return execution_pool_id; +} + size_t LoadJob::pool() const { return pool_id; @@ -113,8 +118,9 @@ void LoadJob::enqueued() enqueue_time = std::chrono::system_clock::now(); } -void LoadJob::execute(const LoadJobPtr & self) +void LoadJob::execute(size_t pool, const LoadJobPtr & self) { + execution_pool_id = pool; start_time = std::chrono::system_clock::now(); func(self); } @@ -657,6 +663,7 @@ void AsyncLoader::worker(Pool & pool) { DENY_ALLOCATIONS_IN_SCOPE; + size_t pool_id = &pool - &*pools.begin(); LoadJobPtr job; std::exception_ptr exception_from_job; while (true) @@ -691,7 +698,7 @@ void AsyncLoader::worker(Pool & pool) try { - job->execute(job); + job->execute(pool_id, job); exception_from_job = {}; } catch (...) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index d52cd03ef32..7c6c934d03e 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -57,7 +57,13 @@ public: LoadStatus status() const; std::exception_ptr exception() const; - // Returns current pool of the job. May differ from initial pool. + // Returns pool in which the job is executing (was executed). May differ from initial pool and from current pool. + // Value is only valid (and constant) after execution started. + size_t execution_pool() const; + + // Returns current pool of the job. May differ from initial and execution pool. + // This value is intended for creating new jobs during this job execution. + // Value may change during job execution by `prioritize()`. size_t pool() const; // Sync wait for a pending job to be finished: OK, FAILED or CANCELED status. @@ -90,8 +96,9 @@ private: void scheduled(); void enqueued(); - void execute(const LoadJobPtr & self); + void execute(size_t pool, const LoadJobPtr & self); + std::atomic execution_pool_id; std::atomic pool_id; std::function func; @@ -185,7 +192,7 @@ inline void scheduleLoad(const LoadTaskPtrs & tasks) } template -inline void scheduleLoad(Args && ... args) +inline void scheduleLoadAll(Args && ... args) { (scheduleLoad(std::forward(args)), ...); } @@ -208,16 +215,16 @@ inline void waitLoad(const LoadTaskPtrs & tasks) } template -inline void waitLoad(Args && ... args) +inline void waitLoadAll(Args && ... args) { (waitLoad(std::forward(args)), ...); } template -inline void scheduleAndWaitLoad(Args && ... args) +inline void scheduleAndWaitLoadAll(Args && ... args) { - scheduleLoad(std::forward(args)...); - waitLoad(std::forward(args)...); + scheduleLoadAll(std::forward(args)...); + waitLoadAll(std::forward(args)...); } inline LoadJobSet getGoals(const LoadTaskPtrs & tasks) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 763f7d7972c..eabddfbebeb 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -566,7 +566,7 @@ TEST(AsyncLoader, TestOverload) AsyncLoaderTest t(3); t.loader.start(); - size_t max_threads = t.loader.getMaxThreads(); + size_t max_threads = t.loader.getMaxThreads(/* pool = */ 0); std::atomic executing{0}; for (int concurrency = 4; concurrency <= 8; concurrency++) @@ -794,8 +794,64 @@ TEST(AsyncLoader, SetMaxThreads) syncs[idx]->arrive_and_wait(); // (A) sync_index++; if (sync_index < syncs.size()) - t.loader.setMaxThreads(max_threads_values[sync_index]); + t.loader.setMaxThreads(/* pool = */ 0, max_threads_values[sync_index]); syncs[idx]->arrive_and_wait(); // (B) this sync point is required to allow `executing` value to go back down to zero after we change number of workers } t.loader.wait(); } + +TEST(AsyncLoader, DynamicPools) +{ + const size_t max_threads[] { 2, 10 }; + const int jobs_in_chain = 16; + AsyncLoaderTest t({ + {.max_threads = max_threads[0], .priority = 0}, + {.max_threads = max_threads[1], .priority = 1}, + }); + + t.loader.start(); + + std::atomic executing[2] { 0, 0 }; // Number of currently executing jobs per pool + + for (int concurrency = 1; concurrency <= 12; concurrency++) + { + std::atomic boosted{false}; // Visible concurrency was increased + std::atomic left{concurrency * jobs_in_chain / 2}; // Number of jobs to start before `prioritize()` call + + LoadJobSet jobs_to_prioritize; + + auto job_func = [&] (const LoadJobPtr & self) + { + auto pool_id = self->execution_pool(); + executing[pool_id]++; + if (executing[pool_id] > max_threads[0]) + boosted = true; + ASSERT_LE(executing[pool_id], max_threads[pool_id]); + + // Dynamic prioritization + if (--left == 0) + { + for (const auto & job : jobs_to_prioritize) + t.loader.prioritize(job, 1); + } + + t.randomSleepUs(100, 200, 100); + + ASSERT_LE(executing[pool_id], max_threads[pool_id]); + executing[pool_id]--; + }; + + std::vector tasks; + tasks.reserve(concurrency); + for (int i = 0; i < concurrency; i++) + tasks.push_back(makeLoadTask(t.loader, t.chainJobSet(jobs_in_chain, job_func))); + jobs_to_prioritize = getGoals(tasks); // All jobs + scheduleAndWaitLoadAll(tasks); + + ASSERT_EQ(executing[0], 0); + ASSERT_EQ(executing[1], 0); + ASSERT_EQ(boosted, concurrency > 2); + boosted = false; + } + +} From 24320f8f93f56aa9a7088c4daf80a066facdc5b6 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Sun, 21 May 2023 15:58:29 +0000 Subject: [PATCH 0475/2223] fixed bad pattern in tests --- .../0_stateless/02722_database_filesystem.sh | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index 80f97af693e..7466141d3e3 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -9,19 +8,21 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') # Prepare data -mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/tmp/ -echo '"id","str","int","text"' > ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv -echo '1,"abc",123,"abacaba"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv -echo '2,"def",456,"bacabaa"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv -echo '3,"story",78912,"acabaab"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv -echo '4,"history",21321321,"cabaaba"' >> ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv +unique_name=${CLICKHOUSE_TEST_UNIQUE_NAME} +user_files_tmp_dir=${CLICKHOUSE_USER_FILES_PATH}/${unique_name} +mkdir -p ${user_files_tmp_dir}/tmp/ +echo '"id","str","int","text"' > ${user_files_tmp_dir}/tmp.csv +echo '1,"abc",123,"abacaba"' >> ${user_files_tmp_dir}/tmp.csv +echo '2,"def",456,"bacabaa"' >> ${user_files_tmp_dir}/tmp.csv +echo '3,"story",78912,"acabaab"' >> ${user_files_tmp_dir}/tmp.csv +echo '4,"history",21321321,"cabaaba"' >> ${user_files_tmp_dir}/tmp.csv tmp_dir=${CLICKHOUSE_TEST_UNIQUE_NAME} [[ -d $tmp_dir ]] && rm -rd $tmp_dir mkdir $tmp_dir -cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${tmp_dir}/tmp.csv -cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp/tmp.csv -cp ${CLICKHOUSE_USER_FILES_PATH}/tmp.csv ${CLICKHOUSE_USER_FILES_PATH}/tmp.myext +cp ${user_files_tmp_dir}/tmp.csv ${tmp_dir}/tmp.csv +cp ${user_files_tmp_dir}/tmp.csv ${user_files_tmp_dir}/tmp/tmp.csv +cp ${user_files_tmp_dir}/tmp.csv ${user_files_tmp_dir}/tmp.myext ################# echo "Test 1: create filesystem database and check implicit calls" @@ -31,8 +32,8 @@ CREATE DATABASE test1 ENGINE = Filesystem; """ echo $? ${CLICKHOUSE_CLIENT} --query "SHOW DATABASES" | grep "test1" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp.csv\`;" -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp/tmp.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.csv\`;" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp/tmp.csv\`;" ${CLICKHOUSE_LOCAL} -q "SELECT COUNT(*) FROM \"${tmp_dir}/tmp.csv\"" ################# @@ -62,9 +63,9 @@ CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 107" > /dev/null && echo "OK" # BAD_ARGUMENTS: Cannot determine the file format by it's extension -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp.myext\`;" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.myext\`;" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" # Clean ${CLICKHOUSE_CLIENT} --query "DROP DATABASE test1;" rm -rd $tmp_dir -rm -rd $CLICKHOUSE_USER_FILES_PATH +rm -rd $user_files_tmp_dir From 8a40a48a4902800306d6772bee920b980481d9b4 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 May 2023 00:44:17 +0800 Subject: [PATCH 0476/2223] Fix python format --- .../integration/test_projection_report_broken_part/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_projection_report_broken_part/test.py b/tests/integration/test_projection_report_broken_part/test.py index 768b6ccae4b..f376adf4f1a 100644 --- a/tests/integration/test_projection_report_broken_part/test.py +++ b/tests/integration/test_projection_report_broken_part/test.py @@ -16,6 +16,7 @@ node = cluster.add_instance( ], ) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: @@ -24,6 +25,7 @@ def start_cluster(): finally: cluster.shutdown() + def test_projection_broken_part(): node.query( """ @@ -57,4 +59,7 @@ def test_projection_broken_part(): time.sleep(2) - assert int(node.query("select sum(b) from test_projection_broken_parts_1 group by a")) == 6 + assert ( + int(node.query("select sum(b) from test_projection_broken_parts_1 group by a")) + == 6 + ) From c5765e71f921bb156543973833220a4ae7395cdb Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 17:02:56 +0000 Subject: [PATCH 0477/2223] requeue jobs w/o allocations and spawn workers during prioritization --- src/Common/AsyncLoader.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index a359c5b4196..0f75f17465b 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -579,17 +579,18 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::un // Update priority and push job forward through ready queue if needed UInt64 ready_seqno = info->second.ready_seqno; - if (ready_seqno) - old_pool.ready_queue.erase(ready_seqno); - job->pool_id.store(new_pool_id); // Set user-facing pool and priority (may affect executing jobs) + + // Requeue job into the new pool queue without allocations if (ready_seqno) { - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - new_pool.ready_queue.emplace(ready_seqno, job); - }); + new_pool.ready_queue.insert(old_pool.ready_queue.extract(ready_seqno)); + if (canSpawnWorker(new_pool, lock)) + spawn(new_pool, lock); } + // Set user-facing pool and priority (may affect executing jobs) + job->pool_id.store(new_pool_id); + // Recurse into dependencies for (const auto & dep : job->dependencies) prioritize(dep, new_pool_id, lock); From cd11c258646d509c12242c38c0c246cf0650b57a Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 17:04:00 +0000 Subject: [PATCH 0478/2223] fix test + more testing for dynamic prioritization --- src/Common/tests/gtest_async_loader.cpp | 91 +++++++++++++++++++++++-- 1 file changed, 87 insertions(+), 4 deletions(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index eabddfbebeb..975cb6a4573 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -629,6 +629,46 @@ TEST(AsyncLoader, StaticPriorities) ASSERT_EQ(schedule, "A9E9D9F9G9H9C4B3"); } +TEST(AsyncLoader, SimplePrioritization) +{ + AsyncLoaderTest t({ + {.max_threads = 1, .priority = 0}, + {.max_threads = 1, .priority = 1}, + {.max_threads = 1, .priority = 2}, + }); + + t.loader.start(); + + std::atomic executed{0}; // Number of previously executed jobs (to test execution order) + LoadJobPtr job_to_prioritize; + + auto job_func_A_booster = [&] (const LoadJobPtr &) + { + ASSERT_EQ(executed++, 0); + t.loader.prioritize(job_to_prioritize, 2); + }; + + auto job_func_B_tester = [&] (const LoadJobPtr &) + { + ASSERT_EQ(executed++, 2); + }; + + auto job_func_C_boosted = [&] (const LoadJobPtr &) + { + ASSERT_EQ(executed++, 1); + }; + + std::vector jobs; + jobs.push_back(makeLoadJob({}, 1, "A", job_func_A_booster)); // 0 + jobs.push_back(makeLoadJob({jobs[0]}, 1, "tester", job_func_B_tester)); // 1 + jobs.push_back(makeLoadJob({}, 0, "C", job_func_C_boosted)); // 2 + auto task = makeLoadTask(t.loader, { jobs.begin(), jobs.end() }); + + job_to_prioritize = jobs[2]; // C + + scheduleAndWaitLoadAll(task); +} + TEST(AsyncLoader, DynamicPriorities) { AsyncLoaderTest t({ @@ -646,15 +686,53 @@ TEST(AsyncLoader, DynamicPriorities) for (bool prioritize : {false, true}) { + // Although all pools have max_threads=1, workers from different pools can run simultaneously just after `prioritize()` call + std::barrier sync(2); + bool wait_sync = prioritize; + std::mutex schedule_mutex; std::string schedule; LoadJobPtr job_to_prioritize; + // Order of execution of jobs D and E after prioritization is undefined, because it depend on `ready_seqno` + // (Which depends on initial `schedule()` order, which in turn depend on `std::unordered_map` order) + // So we have to obtain `ready_seqno` to be sure. + UInt64 ready_seqno_D = 0; + UInt64 ready_seqno_E = 0; + auto job_func = [&] (const LoadJobPtr & self) { + { + std::unique_lock lock{schedule_mutex}; + schedule += fmt::format("{}{}", self->name, self->execution_pool()); + } + if (prioritize && self->name == "C") - t.loader.prioritize(job_to_prioritize, 9); // dynamic prioritization - schedule += fmt::format("{}{}", self->name, self->pool()); + { + for (const auto & state : t.loader.getJobStates()) + { + if (state.job->name == "D") + ready_seqno_D = state.ready_seqno; + if (state.job->name == "E") + ready_seqno_E = state.ready_seqno; + } + + // Jobs D and E should be enqueued at the moment + ASSERT_LT(0, ready_seqno_D); + ASSERT_LT(0, ready_seqno_E); + + // Dynamic prioritization G0 -> G9 + // Note that it will spawn concurrent worker in higher priority pool + t.loader.prioritize(job_to_prioritize, 9); + + sync.arrive_and_wait(); // (A) wait for higher priority worker (B) to test they can be concurrent + } + + if (wait_sync && (self->name == "D" || self->name == "E")) + { + wait_sync = false; + sync.arrive_and_wait(); // (B) + } }; // Job DAG with initial priorities. During execution of C4, job G0 priority is increased to G9, postponing B3 job executing. @@ -676,14 +754,19 @@ TEST(AsyncLoader, DynamicPriorities) jobs.push_back(makeLoadJob({ jobs[6] }, 0, "H", job_func)); // 7 auto task = t.schedule({ jobs.begin(), jobs.end() }); - job_to_prioritize = jobs[6]; + job_to_prioritize = jobs[6]; // G t.loader.start(); t.loader.wait(); t.loader.stop(); if (prioritize) - ASSERT_EQ(schedule, "A4C4E9D9F9G9B3H0"); + { + if (ready_seqno_D < ready_seqno_E) + ASSERT_EQ(schedule, "A4C4D9E9F9G9B3H0"); + else + ASSERT_EQ(schedule, "A4C4E9D9F9G9B3H0"); + } else ASSERT_EQ(schedule, "A4C4B3E2D1F0G0H0"); } From 372601d6dfd9fb07a97882c1209f49e5c305adf4 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 17:04:49 +0000 Subject: [PATCH 0479/2223] fix --- src/Common/tests/gtest_async_loader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 975cb6a4573..2e4c42d6826 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -660,7 +660,7 @@ TEST(AsyncLoader, SimplePrioritization) std::vector jobs; jobs.push_back(makeLoadJob({}, 1, "A", job_func_A_booster)); // 0 - jobs.push_back(makeLoadJob({jobs[0]}, 1, "tester", job_func_B_tester)); // 1 + jobs.push_back(makeLoadJob({jobs[0]}, 1, "B", job_func_B_tester)); // 1 jobs.push_back(makeLoadJob({}, 0, "C", job_func_C_boosted)); // 2 auto task = makeLoadTask(t.loader, { jobs.begin(), jobs.end() }); From 0586a274321f4f8bb852ef1cd6893ed587941d19 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 18 May 2023 21:58:18 +0200 Subject: [PATCH 0480/2223] Charge only server memory for dictionaries Right now the memory will be counted for query/user for dictionary, but only if it load by user (via SYSTEM RELOAD QUERY or via dictGet()), but it could be also loaded in backgrounad (due to lifetime, or update_field, so it is like Buffer, only server memory should be charged. v2: mark test as long Signed-off-by: Azat Khuzhin Co-authored-by: Sergei Trifonov --- src/Dictionaries/HashedDictionary.cpp | 7 ++++ src/Interpreters/ExternalLoader.cpp | 4 +++ .../02760_dictionaries_memory.reference | 0 .../02760_dictionaries_memory.sql.j2 | 32 +++++++++++++++++++ 4 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/02760_dictionaries_memory.reference create mode 100644 tests/queries/0_stateless/02760_dictionaries_memory.sql.j2 diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 83d051df5e9..b528c0faa4c 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -68,6 +69,9 @@ public: shards_queues[shard].emplace(backlog); pool.scheduleOrThrowOnError([this, shard, thread_group = CurrentThread::getGroup()] { + /// Do not account memory that was occupied by the dictionaries for the query/user context. + MemoryTrackerBlockerInThread memory_blocker; + if (thread_group) CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("HashedDictLoad"); @@ -226,6 +230,9 @@ HashedDictionary::~HashedDictionary() pool.trySchedule([&container, thread_group = CurrentThread::getGroup()] { + /// Do not account memory that was occupied by the dictionaries for the query/user context. + MemoryTrackerBlockerInThread memory_blocker; + if (thread_group) CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("HashedDictDtor"); diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 42258b3b912..5dee750889c 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -978,6 +979,9 @@ private: if (thread_group) CurrentThread::attachToGroup(thread_group); + /// Do not account memory that was occupied by the dictionaries for the query/user context. + MemoryTrackerBlockerInThread memory_blocker; + LOG_TRACE(log, "Start loading object '{}'", name); try { diff --git a/tests/queries/0_stateless/02760_dictionaries_memory.reference b/tests/queries/0_stateless/02760_dictionaries_memory.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02760_dictionaries_memory.sql.j2 b/tests/queries/0_stateless/02760_dictionaries_memory.sql.j2 new file mode 100644 index 00000000000..ea979506e07 --- /dev/null +++ b/tests/queries/0_stateless/02760_dictionaries_memory.sql.j2 @@ -0,0 +1,32 @@ +-- Tags: long +-- Tag long: in parallel runs could exceed 60 seconds +{# vim: ft=sql #} + +SET max_memory_usage=0; +DROP DICTIONARY IF EXISTS dict; +DROP TABLE IF EXISTS dict_data; + +CREATE TABLE dict_data (key UInt64, value UInt64) Engine=Memory(); +INSERT INTO dict_data SELECT number, number%10 FROM numbers(3_000_000); + +SET max_memory_usage='4Mi'; +{% for layout in [ + 'FLAT(INITIAL_ARRAY_SIZE 3_000_000 MAX_ARRAY_SIZE 3_000_000)', + 'HASHED()', + 'HASHED_ARRAY()', + 'SPARSE_HASHED()', + 'SPARSE_HASHED(SHARDS 2 /* shards are special, they use threads */)', +] %} +CREATE DICTIONARY dict (key UInt64, value UInt64) PRIMARY KEY key SOURCE(CLICKHOUSE(TABLE dict_data)) LIFETIME(0) LAYOUT({{layout}}); +SYSTEM RELOAD DICTIONARY dict; +-- assert that dictionary in memory takes more than 20MB, that way for each +-- shard we will have 10MB, that way we ensure that the allocations will be +-- definitely correct for the memory tracker to hit the MEMORY_LIMIT_EXCEEDED +-- error. +SELECT throwIf(bytes_allocated < 20e6, 'Memory constraints does not matched for layout {{layout}}') FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict' FORMAT Null; +DROP DICTIONARY dict; + +CREATE DICTIONARY dict (key UInt64, value UInt64) PRIMARY KEY key SOURCE(CLICKHOUSE(TABLE dict_data)) LIFETIME(0) LAYOUT({{layout}}); +SELECT dictGet('dict', 'value', 1::UInt64) FORMAT Null; +DROP DICTIONARY dict; +{% endfor %} From b6cc504717dacf968dd8bff0d9f76e824fd3a246 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 May 2023 09:22:00 +0200 Subject: [PATCH 0481/2223] Remove Common/OpenTelemetryTraceContext.h from Context.h Signed-off-by: Azat Khuzhin --- src/Interpreters/Context.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 15f2ff625ef..1be662e0958 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include From 66cf16410daf872b241c5b9f65e2e95eac9f85a5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 19 May 2023 10:37:22 +0200 Subject: [PATCH 0482/2223] Preserve initial_query_id for ON CLUSTER queries v2: add proper escaping v3: set distributed_ddl_output_mode=none for test to fix replicated database build Signed-off-by: Azat Khuzhin --- src/Interpreters/DDLTask.cpp | 16 +++++++++++++ src/Interpreters/DDLTask.h | 4 +++- src/Interpreters/DDLWorker.cpp | 2 ++ src/Interpreters/executeDDLQueryOnCluster.cpp | 1 + .../02761_ddl_initial_query_id.reference | 5 ++++ .../0_stateless/02761_ddl_initial_query_id.sh | 24 +++++++++++++++++++ 6 files changed, 51 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02761_ddl_initial_query_id.reference create mode 100755 tests/queries/0_stateless/02761_ddl_initial_query_id.sh diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 799f1b0b4f4..b24856a6146 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -104,6 +104,14 @@ String DDLLogEntry::toString() const if (version >= OPENTELEMETRY_ENABLED_VERSION) wb << "tracing: " << this->tracing_context; + /// NOTE: OPENTELEMETRY_ENABLED_VERSION has new line in TracingContext::serialize(), so no need to add one more + + if (version >= PRESERVE_INITIAL_QUERY_ID_VERSION) + { + writeString("initial_query_id: ", wb); + writeEscapedString(initial_query_id, wb); + writeChar('\n', wb); + } return wb.str(); } @@ -150,6 +158,14 @@ void DDLLogEntry::parse(const String & data) rb >> "tracing: " >> this->tracing_context; } + if (version >= PRESERVE_INITIAL_QUERY_ID_VERSION) + { + checkString("initial_query_id: ", rb); + readEscapedString(initial_query_id, rb); + checkChar('\n', rb); + } + + assertEOF(rb); if (!host_id_strings.empty()) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 2043de6701e..e92b1f9a885 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -71,10 +71,11 @@ struct DDLLogEntry static constexpr const UInt64 SETTINGS_IN_ZK_VERSION = 2; static constexpr const UInt64 NORMALIZE_CREATE_ON_INITIATOR_VERSION = 3; static constexpr const UInt64 OPENTELEMETRY_ENABLED_VERSION = 4; + static constexpr const UInt64 PRESERVE_INITIAL_QUERY_ID_VERSION = 5; /// Add new version here /// Remember to update the value below once new version is added - static constexpr const UInt64 DDL_ENTRY_FORMAT_MAX_VERSION = 4; + static constexpr const UInt64 DDL_ENTRY_FORMAT_MAX_VERSION = 5; UInt64 version = 1; String query; @@ -82,6 +83,7 @@ struct DDLLogEntry String initiator; // optional std::optional settings; OpenTelemetry::TracingContext tracing_context; + String initial_query_id; void setSettingsIfRequired(ContextPtr context); String toString() const; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 7c021bd82e6..81c78000ac3 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -476,6 +476,8 @@ bool DDLWorker::tryExecuteQuery(DDLTaskBase & task, const ZooKeeperPtr & zookeep query_context->setSetting("implicit_transaction", Field{0}); } + query_context->getClientInfo().initial_query_id = task.entry.initial_query_id; + if (!task.is_initial_query) query_scope.emplace(query_context); executeQuery(istr, ostr, !task.is_initial_query, query_context, {}); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index e4d880c1e0c..04c1d545207 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -184,6 +184,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, entry.initiator = ddl_worker.getCommonHostID(); entry.setSettingsIfRequired(context); entry.tracing_context = OpenTelemetry::CurrentContext(); + entry.initial_query_id = context->getClientInfo().initial_query_id; String node_path = ddl_worker.enqueueQuery(entry); return getDistributedDDLStatus(node_path, entry, context, /* hosts_to_wait */ nullptr); diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.reference b/tests/queries/0_stateless/02761_ddl_initial_query_id.reference new file mode 100644 index 00000000000..5c6f448eed5 --- /dev/null +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.reference @@ -0,0 +1,5 @@ +default distributed_ddl_entry_format_version +DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost +distributed_ddl_entry_format_version=PRESERVE_INITIAL_QUERY_ID_VERSION +DROP TABLE IF EXISTS default.foo +DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.sh b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh new file mode 100755 index 00000000000..e9a315b812b --- /dev/null +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "default distributed_ddl_entry_format_version" +query_id="$(random_str 10)" +$CLICKHOUSE_CLIENT --query_id "$query_id" --distributed_ddl_output_mode=none -q "DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost" +$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" +$CLICKHOUSE_CLIENT -q "SELECT query FROM system.query_log WHERE initial_query_id = '$query_id' AND type != 'QueryStart'" + +echo "distributed_ddl_entry_format_version=PRESERVE_INITIAL_QUERY_ID_VERSION" +PRESERVE_INITIAL_QUERY_ID_VERSION=5 +query_id="$(random_str 10)" +# Check that serialization will not be broken with new lines in initial_query_id +query_id+=$'\nfoo' +$CLICKHOUSE_CLIENT --distributed_ddl_entry_format_version=$PRESERVE_INITIAL_QUERY_ID_VERSION --query_id "$query_id" --distributed_ddl_output_mode=none -q "DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost" +$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" +# - normalizeQuery() is required to strip out DDL comment +# - replace() is required to avoid non deterministic behaviour of +# normalizeQuery() that replaces the identifier with "?" only if it has more +# then two numbers. +$CLICKHOUSE_CLIENT -q "SELECT normalizeQuery(replace(query, currentDatabase(), 'default')) FROM system.query_log WHERE initial_query_id = '$query_id' AND type != 'QueryStart' ORDER BY event_time_microseconds" From ef06bb8f1409961d789d071703dfe1d1b4aef871 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 21 May 2023 23:07:03 +0200 Subject: [PATCH 0483/2223] Fix crashing in case of Replicated database without arguments Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- .../0_stateless/02762_replicated_database_no_args.reference | 0 .../queries/0_stateless/02762_replicated_database_no_args.sql | 4 ++++ 3 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02762_replicated_database_no_args.reference create mode 100644 tests/queries/0_stateless/02762_replicated_database_no_args.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3e76ad23c30..ab9e1fb04d6 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -227,7 +227,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) metadata_path = metadata_path / "metadata" / database_name_escaped; } - if (create.storage->engine->name == "Replicated" && !internal && !create.attach) + if (create.storage->engine->name == "Replicated" && !internal && !create.attach && create.storage->engine->arguments) { /// Fill in default parameters if (create.storage->engine->arguments->children.size() == 1) diff --git a/tests/queries/0_stateless/02762_replicated_database_no_args.reference b/tests/queries/0_stateless/02762_replicated_database_no_args.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02762_replicated_database_no_args.sql b/tests/queries/0_stateless/02762_replicated_database_no_args.sql new file mode 100644 index 00000000000..1409a059b02 --- /dev/null +++ b/tests/queries/0_stateless/02762_replicated_database_no_args.sql @@ -0,0 +1,4 @@ +-- Tags: no-parallel + +set allow_experimental_database_replicated=1; +create database replicated_db_no_args engine=Replicated; -- { serverError BAD_ARGUMENTS } From f5dc07d052e60bc6ebe6d27a743722ae1b6b8e27 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 21 May 2023 22:01:28 +0000 Subject: [PATCH 0484/2223] tryReserve() cleanup simplify removing eviction candidates --- src/Interpreters/Cache/FileCache.cpp | 23 ++++++++++------------- src/Interpreters/Cache/Metadata.h | 2 +- 2 files changed, 11 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 9ab7943e263..55d7177fd4c 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -573,27 +573,25 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) else queue_size += 1; - class EvictionCandidates final : public std::vector + struct EvictionCandidates { - public: explicit EvictionCandidates(KeyMetadataPtr key_metadata_) : key_metadata(key_metadata_) {} - KeyMetadata & getMetadata() { return *key_metadata; } - void add(FileSegmentMetadataPtr candidate) { candidate->removal_candidate = true; - push_back(candidate); + candidates.push_back(candidate); } ~EvictionCandidates() { - for (const auto & candidate : *this) + // todo: it looks redundant, - why is it needed? + for (const auto & candidate : candidates) candidate->removal_candidate = false; } - private: KeyMetadataPtr key_metadata; + std::vector candidates; }; std::unordered_map to_delete; @@ -681,23 +679,22 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) for (auto & [current_key, deletion_info] : to_delete) { - auto locked_key = deletion_info.getMetadata().tryLock(); + auto locked_key = deletion_info.key_metadata->tryLock(); if (!locked_key) continue; /// key could become invalid after we released the key lock above, just skip it. - for (auto it = deletion_info.begin(); it != deletion_info.end();) + for (const auto & candidate : deletion_info.candidates) { - chassert((*it)->releasable()); + chassert(candidate->releasable()); - auto segment = (*it)->file_segment; + auto segment = candidate->file_segment; locked_key->removeFileSegment(segment->offset(), segment->lock()); segment->getQueueIterator()->remove(cache_lock); if (query_context) query_context->remove(current_key, segment->offset(), cache_lock); - - it = deletion_info.erase(it); } + deletion_info.candidates.clear(); } /// queue_iteratir is std::nullopt here if no space has been reserved yet, a file_segment_metadata diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 586c7e5c2a8..2e015b07ed0 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -24,7 +24,7 @@ struct FileSegmentMetadata : private boost::noncopyable bool valid() const { return !removal_candidate.load(); } - Priority::Iterator getQueueIterator() { return file_segment->getQueueIterator(); } + Priority::Iterator getQueueIterator() const { return file_segment->getQueueIterator(); } FileSegmentPtr file_segment; std::atomic removal_candidate{false}; From 8c816a5c4a97044b1d3d902145ef4eefecd7beb8 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 22 May 2023 01:00:40 +0200 Subject: [PATCH 0485/2223] update --- src/Common/DateLUT.h | 37 ++++++++++++++++--------------- src/Core/Settings.h | 2 +- src/Functions/serverConstants.cpp | 4 +--- src/IO/ReadHelpers.h | 4 ++-- 4 files changed, 23 insertions(+), 24 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 59b280240ea..23698331afe 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -17,30 +17,24 @@ class DateLUT : private boost::noncopyable { public: - /// Return singleton DateLUTImpl instance for server's (native) time zone. - static ALWAYS_INLINE const DateLUTImpl & serverTimezoneInstance() - { - const auto & date_lut = getInstance(); - return *date_lut.default_impl.load(std::memory_order_acquire); - } - - /// Return singleton DateLUTImpl instance for timezone set by `timezone` setting for current session is used. - /// If it is not set, server's timezone (the one which server has) is being used. + /// Return singleton DateLUTImpl instance for session timezone. + /// The session timezone is configured by a session setting. + /// If not set (empty string), it is the server timezone. static ALWAYS_INLINE const DateLUTImpl & instance() { const auto & date_lut = getInstance(); if (DB::CurrentThread::isInitialized()) { - std::string effective_time_zone; - const auto query_context = DB::CurrentThread::get().getQueryContext(); + std::string context_timezone; + const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext(); if (query_context) { - effective_time_zone = extractTimezoneFromContext(query_context); + context_timezone = extractTimezoneFromContext(query_context); - if (!effective_time_zone.empty()) - return date_lut.getImplementation(effective_time_zone); + if (!context_timezone.empty()) + return date_lut.getImplementation(context_timezone); } /// Timezone is passed in query_context, but on CH-Client we have no query context, @@ -48,10 +42,10 @@ public: const auto global_context = DB::CurrentThread::get().getGlobalContext(); if (global_context) { - effective_time_zone = extractTimezoneFromContext(global_context); + context_timezone = extractTimezoneFromContext(global_context); - if (!effective_time_zone.empty()) - return date_lut.getImplementation(effective_time_zone); + if (!context_timezone.empty()) + return date_lut.getImplementation(context_timezone); } } @@ -67,6 +61,13 @@ public: return date_lut.getImplementation(time_zone); } + // Return singleton DateLUTImpl for the server time zone. + static ALWAYS_INLINE const DateLUTImpl & serverTimezoneInstance() + { + const auto & date_lut = getInstance(); + return *date_lut.default_impl.load(std::memory_order_acquire); + } + static void setDefaultTimezone(const std::string & time_zone) { auto & date_lut = getInstance(); @@ -80,7 +81,7 @@ protected: private: static DateLUT & getInstance(); - static std::string extractTimezoneFromContext(const DB::ContextPtr query_context); + static std::string extractTimezoneFromContext(DB::ContextPtr query_context); const DateLUTImpl & getImplementation(const std::string & time_zone) const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 90063f8efd4..2d766e8e18f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -747,7 +747,7 @@ class IColumn; M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ - M(Timezone, session_timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone in current session.", 0) \ + M(Timezone, session_timezone, "", "The default timezone for the current session. The server default timezone, if empty.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 9cd43be50c4..d3e1e6e10fe 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -60,7 +60,7 @@ namespace }; - /// Returns default timezone for current session. + /// Returns timezone for current session. class FunctionTimezone : public FunctionConstantBase { public: @@ -187,8 +187,6 @@ Returns the timezone name in which server operates. .examples{{"serverTimezone", "SELECT serverTimezone();", ""}}, .categories{"Constant", "Miscellaneous"} }); - factory.registerAlias("serverTimeZone", "serverTimezone"); - factory.registerAlias("servertimezone", "serverTimezone"); } REGISTER_FUNCTION(Uptime) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 3bd9275322e..cbe18e11c9a 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -706,7 +706,7 @@ inline void convertToDayNum(DayNum & date, ExtendedDayNum & from) } template -inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) +inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut) { static constexpr bool throw_exception = std::is_same_v; @@ -723,7 +723,7 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLU } template -inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) +inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut) { static constexpr bool throw_exception = std::is_same_v; From 39ff030a6e9a646a90f0616c63b9b8064588436e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 4 May 2023 10:17:04 +0800 Subject: [PATCH 0486/2223] grace hash join supports right/full join --- src/Interpreters/GraceHashJoin.cpp | 79 +- src/Interpreters/GraceHashJoin.h | 3 +- src/Interpreters/JoinUtils.cpp | 2 +- .../Transforms/JoiningTransform.cpp | 87 +- src/Processors/Transforms/JoiningTransform.h | 6 +- .../02724_grace_hash_join.reference | 1804 +++++++++++++++++ .../0_stateless/02724_grace_hash_join.sql | 44 + 7 files changed, 1965 insertions(+), 60 deletions(-) create mode 100644 tests/queries/0_stateless/02724_grace_hash_join.reference create mode 100644 tests/queries/0_stateless/02724_grace_hash_join.sql diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 7795061072c..32801887f2b 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -302,10 +302,12 @@ void GraceHashJoin::initBuckets() current_bucket->startJoining(); } -bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) +bool GraceHashJoin::isSupported(const std::shared_ptr & table_join [[maybe_unused]]) { + bool is_asof = (table_join->strictness() == JoinStrictness::Asof); - return !is_asof && isInnerOrLeft(table_join->kind()) && table_join->oneDisjunct(); + auto kind = table_join->kind(); + return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct(); } GraceHashJoin::~GraceHashJoin() = default; @@ -325,7 +327,6 @@ bool GraceHashJoin::hasMemoryOverflow(size_t total_rows, size_t total_bytes) con /// One row can't be split, avoid loop if (total_rows < 2) return false; - bool has_overflow = !table_join->sizeLimits().softCheck(total_rows, total_bytes); if (has_overflow) @@ -471,23 +472,33 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block &, const Block &, UInt64) const +IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks( + const Block & left_sample_block_ [[maybe_unused]], const Block & result_sample_block_ [[maybe_unused]], UInt64 max_block_size_ [[maybe_unused]]) const { - /// We do no support returning non joined blocks here. - /// TODO: They _should_ be reported by getDelayedBlocks instead - return nullptr; + return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream { public: - explicit DelayedBlocks(size_t current_bucket_, Buckets buckets_, InMemoryJoinPtr hash_join_, const Names & left_key_names_, const Names & right_key_names_) + explicit DelayedBlocks( + size_t current_bucket_, + Buckets buckets_, + InMemoryJoinPtr hash_join_, + const Names & left_key_names_, + const Names & right_key_names_, + const Block & left_sample_block_, + const Block & result_sample_block_, + size_t max_block_size_) : current_bucket(current_bucket_) , buckets(std::move(buckets_)) , hash_join(std::move(hash_join_)) , left_reader(buckets[current_bucket]->getLeftTableReader()) , left_key_names(left_key_names_) , right_key_names(right_key_names_) + , left_sample_block(left_sample_block_) + , result_sample_block(result_sample_block_) + , max_block_size(max_block_size_) { } @@ -499,10 +510,27 @@ public: do { - block = left_reader.read(); - if (!block) + if (!is_left_reader_finished) { - return {}; + block = left_reader.read(); + if (!block) + { + is_left_reader_finished = true; + non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); + } + } + if (is_left_reader_finished) + { + if (non_joined_blocks) + { + block = non_joined_blocks->next(); + return block; + } + else + { + // left/inner join. + return {}; + } } Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets); @@ -544,6 +572,11 @@ public: Names left_key_names; Names right_key_names; + Block left_sample_block; + Block result_sample_block; + bool is_left_reader_finished = false; + IBlocksStreamPtr non_joined_blocks; + size_t max_block_size; }; IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() @@ -555,16 +588,6 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; - if (hash_join) - { - auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); - for (auto & block : right_blocks) - { - Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, block, buckets.size()); - flushBlocksToBuckets(blocks, buckets, bucket_idx); - } - } - hash_join = makeInMemoryJoin(); for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) @@ -589,8 +612,18 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() LOG_TRACE(log, "Loaded bucket {} with {}(/{}) rows", bucket_idx, hash_join->getTotalRowCount(), num_rows); - - return std::make_unique(current_bucket->idx, buckets, hash_join, left_key_names, right_key_names); + auto result_sample_block = left_sample_block; + ExtraBlockPtr tmp; + hash_join->joinBlock(result_sample_block, tmp); + return std::make_unique( + current_bucket->idx, + buckets, + hash_join, + left_key_names, + right_key_names, + left_sample_block, + result_sample_block, + max_block_size); } LOG_TRACE(log, "Finished loading all {} buckets", buckets.size()); diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index b8d83f4cad0..eb39ee09208 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -13,7 +13,6 @@ namespace DB { - class TableJoin; class HashJoin; @@ -79,7 +78,7 @@ public: bool supportTotals() const override { return false; } IBlocksStreamPtr - getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; + getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size) const override; /// Open iterator over joined blocks. /// Must be called after all @joinBlock calls. diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index a05b58e14a1..59ee86ffa18 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -19,6 +19,7 @@ #include + namespace DB { @@ -864,7 +865,6 @@ Block NotJoinedBlocks::nextImpl() size_t rows_added = filler->fillColumns(columns_right); if (rows_added == 0) return {}; - addLeftColumns(result_block, rows_added); addRightColumns(result_block, columns_right); } diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index bba8ec6fa16..4789e9c514a 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -350,6 +350,11 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (!task) { + if (input.isFinished()) + { + output.finish(); + return Status::Finished; + } if (!input.hasData()) { input.setNeeded(); @@ -362,10 +367,14 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() output.pushException(data.exception); return Status::Finished; } - - if (!data.chunk.hasChunkInfo()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); - task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); + if (data.chunk.hasChunkInfo()) + task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); + else + { + // Try to get one task from DelayedJoinedBlocksTransform again. + task = nullptr; + return Status::NeedData; + } } else { @@ -386,7 +395,6 @@ void DelayedJoinedBlocksWorkerTransform::work() { if (!task) return; - Block block = task->delayed_blocks->next(); if (!block) @@ -408,15 +416,19 @@ DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, J void DelayedJoinedBlocksTransform::work() { - if (finished) + if (all_buckets_finished) return; - delayed_blocks = join->getDelayedBlocks(); - finished = finished || delayed_blocks == nullptr; + if (!delayed_blocks) + { + all_buckets_finished = true; + } } IProcessor::Status DelayedJoinedBlocksTransform::prepare() { + + bool should_finished = false; for (auto & output : outputs) { if (output.isFinished()) @@ -424,41 +436,56 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() /// If at least one output is finished, then we have read all data from buckets. /// Some workers can still be busy with joining the last chunk of data in memory, /// but after that they also will finish when they will try to get next chunk. - finished = true; - continue; + should_finished = true; + break; } - if (!output.canPush()) - return Status::PortFull; } - - if (finished) + if (should_finished) { for (auto & output : outputs) { - if (output.isFinished()) - continue; - Chunk chunk; - chunk.setChunkInfo(std::make_shared()); - output.push(std::move(chunk)); - output.finish(); + if (!output.isFinished()) + { + output.finish(); + } } - return Status::Finished; } - if (delayed_blocks) + // No pending buckets. + if (all_buckets_finished && !delayed_blocks) { for (auto & output : outputs) - { - Chunk chunk; - chunk.setChunkInfo(std::make_shared(delayed_blocks)); - output.push(std::move(chunk)); - } - delayed_blocks = nullptr; - return Status::PortFull; + output.finish(); + return Status::Finished; } - return Status::Ready; + // Try to read next bucket. + if (!delayed_blocks) + return Status::Ready; + + // Put the ready task to the first available output port. + // Put empty tasks to other output ports. these ports will do nothing but require data again. + // Each bucket will handled in one thread, avoid lock contention in left file reader. + for (auto & output : outputs) + { + if (output.canPush()) + { + if (delayed_blocks) + { + Chunk chunk; + chunk.setChunkInfo(std::make_shared(delayed_blocks)); + output.push(std::move(chunk)); + delayed_blocks = nullptr; + } + else + { + Chunk chunk; + output.push(std::move(chunk)); + } + } + } + return Status::PortFull; } } diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index e7edff40c56..f189f53c004 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -125,7 +125,6 @@ public: using DelayedBlocksTaskPtr = std::shared_ptr; - /// Reads delayed joined blocks from Join class DelayedJoinedBlocksTransform : public IProcessor { @@ -139,9 +138,8 @@ public: private: JoinPtr join; - - IBlocksStreamPtr delayed_blocks = nullptr; - bool finished = false; + IBlocksStreamPtr delayed_blocks; + bool all_buckets_finished = false; }; class DelayedJoinedBlocksWorkerTransform : public IProcessor diff --git a/tests/queries/0_stateless/02724_grace_hash_join.reference b/tests/queries/0_stateless/02724_grace_hash_join.reference new file mode 100644 index 00000000000..296ef623f04 --- /dev/null +++ b/tests/queries/0_stateless/02724_grace_hash_join.reference @@ -0,0 +1,1804 @@ +inner join 1 +51 11 21 +52 12 22 +53 13 23 +54 14 24 +55 15 25 +56 16 26 +57 17 27 +58 18 28 +59 19 29 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 0 20 +81 1 21 +82 2 22 +83 3 23 +84 4 24 +85 5 25 +86 6 26 +87 7 27 +88 8 28 +89 9 29 +90 10 0 +91 11 1 +92 12 2 +93 13 3 +94 14 4 +95 15 5 +96 16 6 +97 17 7 +98 18 8 +99 19 9 +100 0 10 +101 1 11 +102 2 12 +103 3 13 +104 4 14 +105 5 15 +106 6 16 +107 7 17 +108 8 18 +109 9 19 +110 10 20 +111 11 21 +112 12 22 +113 13 23 +114 14 24 +115 15 25 +116 16 26 +117 17 27 +118 18 28 +119 19 29 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 0 20 +141 1 21 +142 2 22 +143 3 23 +144 4 24 +145 5 25 +146 6 26 +147 7 27 +148 8 28 +149 9 29 +150 10 0 +151 11 1 +152 12 2 +153 13 3 +154 14 4 +155 15 5 +156 16 6 +157 17 7 +158 18 8 +159 19 9 +160 0 10 +161 1 11 +162 2 12 +163 3 13 +164 4 14 +165 5 15 +166 6 16 +167 7 17 +168 8 18 +169 9 19 +170 10 20 +171 11 21 +172 12 22 +173 13 23 +174 14 24 +175 15 25 +176 16 26 +177 17 27 +178 18 28 +179 19 29 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +inner join 2 +51 21 11 +52 22 12 +53 23 13 +54 24 14 +55 25 15 +56 26 16 +57 27 17 +58 28 18 +59 29 19 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 20 0 +81 21 1 +82 22 2 +83 23 3 +84 24 4 +85 25 5 +86 26 6 +87 27 7 +88 28 8 +89 29 9 +90 0 10 +91 1 11 +92 2 12 +93 3 13 +94 4 14 +95 5 15 +96 6 16 +97 7 17 +98 8 18 +99 9 19 +100 10 0 +101 11 1 +102 12 2 +103 13 3 +104 14 4 +105 15 5 +106 16 6 +107 17 7 +108 18 8 +109 19 9 +110 20 10 +111 21 11 +112 22 12 +113 23 13 +114 24 14 +115 25 15 +116 26 16 +117 27 17 +118 28 18 +119 29 19 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 20 0 +141 21 1 +142 22 2 +143 23 3 +144 24 4 +145 25 5 +146 26 6 +147 27 7 +148 28 8 +149 29 9 +150 0 10 +151 1 11 +152 2 12 +153 3 13 +154 4 14 +155 5 15 +156 6 16 +157 7 17 +158 8 18 +159 9 19 +160 10 0 +161 11 1 +162 12 2 +163 13 3 +164 14 4 +165 15 5 +166 16 6 +167 17 7 +168 18 8 +169 19 9 +170 20 10 +171 21 11 +172 22 12 +173 23 13 +174 24 14 +175 25 15 +176 26 16 +177 27 17 +178 28 18 +179 29 19 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +left join 1 +51 11 21 +52 12 22 +53 13 23 +54 14 24 +55 15 25 +56 16 26 +57 17 27 +58 18 28 +59 19 29 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 0 20 +81 1 21 +82 2 22 +83 3 23 +84 4 24 +85 5 25 +86 6 26 +87 7 27 +88 8 28 +89 9 29 +90 10 0 +91 11 1 +92 12 2 +93 13 3 +94 14 4 +95 15 5 +96 16 6 +97 17 7 +98 18 8 +99 19 9 +100 0 10 +101 1 11 +102 2 12 +103 3 13 +104 4 14 +105 5 15 +106 6 16 +107 7 17 +108 8 18 +109 9 19 +110 10 20 +111 11 21 +112 12 22 +113 13 23 +114 14 24 +115 15 25 +116 16 26 +117 17 27 +118 18 28 +119 19 29 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 0 20 +141 1 21 +142 2 22 +143 3 23 +144 4 24 +145 5 25 +146 6 26 +147 7 27 +148 8 28 +149 9 29 +150 10 0 +151 11 1 +152 12 2 +153 13 3 +154 14 4 +155 15 5 +156 16 6 +157 17 7 +158 18 8 +159 19 9 +160 0 10 +161 1 11 +162 2 12 +163 3 13 +164 4 14 +165 5 15 +166 6 16 +167 7 17 +168 8 18 +169 9 19 +170 10 20 +171 11 21 +172 12 22 +173 13 23 +174 14 24 +175 15 25 +176 16 26 +177 17 27 +178 18 28 +179 19 29 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +left join 2 +0 0 \N +1 1 \N +2 2 \N +3 3 \N +4 4 \N +5 5 \N +6 6 \N +7 7 \N +8 8 \N +9 9 \N +10 10 \N +11 11 \N +12 12 \N +13 13 \N +14 14 \N +15 15 \N +16 16 \N +17 17 \N +18 18 \N +19 19 \N +20 20 \N +21 21 \N +22 22 \N +23 23 \N +24 24 \N +25 25 \N +26 26 \N +27 27 \N +28 28 \N +29 29 \N +30 0 \N +31 1 \N +32 2 \N +33 3 \N +34 4 \N +35 5 \N +36 6 \N +37 7 \N +38 8 \N +39 9 \N +40 10 \N +41 11 \N +42 12 \N +43 13 \N +44 14 \N +45 15 \N +46 16 \N +47 17 \N +48 18 \N +49 19 \N +50 20 \N +51 21 11 +52 22 12 +53 23 13 +54 24 14 +55 25 15 +56 26 16 +57 27 17 +58 28 18 +59 29 19 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 20 0 +81 21 1 +82 22 2 +83 23 3 +84 24 4 +85 25 5 +86 26 6 +87 27 7 +88 28 8 +89 29 9 +90 0 10 +91 1 11 +92 2 12 +93 3 13 +94 4 14 +95 5 15 +96 6 16 +97 7 17 +98 8 18 +99 9 19 +100 10 0 +101 11 1 +102 12 2 +103 13 3 +104 14 4 +105 15 5 +106 16 6 +107 17 7 +108 18 8 +109 19 9 +110 20 10 +111 21 11 +112 22 12 +113 23 13 +114 24 14 +115 25 15 +116 26 16 +117 27 17 +118 28 18 +119 29 19 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 20 0 +141 21 1 +142 22 2 +143 23 3 +144 24 4 +145 25 5 +146 26 6 +147 27 7 +148 28 8 +149 29 9 +150 0 10 +151 1 11 +152 2 12 +153 3 13 +154 4 14 +155 5 15 +156 6 16 +157 7 17 +158 8 18 +159 9 19 +160 10 0 +161 11 1 +162 12 2 +163 13 3 +164 14 4 +165 15 5 +166 16 6 +167 17 7 +168 18 8 +169 19 9 +170 20 10 +171 21 11 +172 22 12 +173 23 13 +174 24 14 +175 25 15 +176 26 16 +177 27 17 +178 28 18 +179 29 19 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +200 20 \N +201 21 \N +202 22 \N +203 23 \N +204 24 \N +205 25 \N +206 26 \N +207 27 \N +208 28 \N +209 29 \N +210 0 \N +211 1 \N +212 2 \N +213 3 \N +214 4 \N +215 5 \N +216 6 \N +217 7 \N +218 8 \N +219 9 \N +220 10 \N +221 11 \N +222 12 \N +223 13 \N +224 14 \N +225 15 \N +226 16 \N +227 17 \N +228 18 \N +229 19 \N +230 20 \N +231 21 \N +232 22 \N +233 23 \N +234 24 \N +235 25 \N +236 26 \N +237 27 \N +238 28 \N +239 29 \N +240 0 \N +241 1 \N +242 2 \N +243 3 \N +244 4 \N +245 5 \N +246 6 \N +247 7 \N +248 8 \N +249 9 \N +250 10 \N +251 11 \N +252 12 \N +253 13 \N +254 14 \N +255 15 \N +256 16 \N +257 17 \N +258 18 \N +259 19 \N +260 20 \N +261 21 \N +262 22 \N +263 23 \N +264 24 \N +265 25 \N +266 26 \N +267 27 \N +268 28 \N +269 29 \N +270 0 \N +271 1 \N +272 2 \N +273 3 \N +274 4 \N +275 5 \N +276 6 \N +277 7 \N +278 8 \N +279 9 \N +280 10 \N +281 11 \N +282 12 \N +283 13 \N +284 14 \N +285 15 \N +286 16 \N +287 17 \N +288 18 \N +289 19 \N +290 20 \N +291 21 \N +292 22 \N +293 23 \N +294 24 \N +295 25 \N +296 26 \N +297 27 \N +298 28 \N +299 29 \N +right join 1 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 29 +0 \N 29 +0 \N 29 +0 \N 29 +0 \N 29 +51 11 21 +52 12 22 +53 13 23 +54 14 24 +55 15 25 +56 16 26 +57 17 27 +58 18 28 +59 19 29 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 0 20 +81 1 21 +82 2 22 +83 3 23 +84 4 24 +85 5 25 +86 6 26 +87 7 27 +88 8 28 +89 9 29 +90 10 0 +91 11 1 +92 12 2 +93 13 3 +94 14 4 +95 15 5 +96 16 6 +97 17 7 +98 18 8 +99 19 9 +100 0 10 +101 1 11 +102 2 12 +103 3 13 +104 4 14 +105 5 15 +106 6 16 +107 7 17 +108 8 18 +109 9 19 +110 10 20 +111 11 21 +112 12 22 +113 13 23 +114 14 24 +115 15 25 +116 16 26 +117 17 27 +118 18 28 +119 19 29 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 0 20 +141 1 21 +142 2 22 +143 3 23 +144 4 24 +145 5 25 +146 6 26 +147 7 27 +148 8 28 +149 9 29 +150 10 0 +151 11 1 +152 12 2 +153 13 3 +154 14 4 +155 15 5 +156 16 6 +157 17 7 +158 18 8 +159 19 9 +160 0 10 +161 1 11 +162 2 12 +163 3 13 +164 4 14 +165 5 15 +166 6 16 +167 7 17 +168 8 18 +169 9 19 +170 10 20 +171 11 21 +172 12 22 +173 13 23 +174 14 24 +175 15 25 +176 16 26 +177 17 27 +178 18 28 +179 19 29 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +right join 2 +51 21 11 +52 22 12 +53 23 13 +54 24 14 +55 25 15 +56 26 16 +57 27 17 +58 28 18 +59 29 19 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 20 0 +81 21 1 +82 22 2 +83 23 3 +84 24 4 +85 25 5 +86 26 6 +87 27 7 +88 28 8 +89 29 9 +90 0 10 +91 1 11 +92 2 12 +93 3 13 +94 4 14 +95 5 15 +96 6 16 +97 7 17 +98 8 18 +99 9 19 +100 10 0 +101 11 1 +102 12 2 +103 13 3 +104 14 4 +105 15 5 +106 16 6 +107 17 7 +108 18 8 +109 19 9 +110 20 10 +111 21 11 +112 22 12 +113 23 13 +114 24 14 +115 25 15 +116 26 16 +117 27 17 +118 28 18 +119 29 19 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 20 0 +141 21 1 +142 22 2 +143 23 3 +144 24 4 +145 25 5 +146 26 6 +147 27 7 +148 28 8 +149 29 9 +150 0 10 +151 1 11 +152 2 12 +153 3 13 +154 4 14 +155 5 15 +156 6 16 +157 7 17 +158 8 18 +159 9 19 +160 10 0 +161 11 1 +162 12 2 +163 13 3 +164 14 4 +165 15 5 +166 16 6 +167 17 7 +168 18 8 +169 19 9 +170 20 10 +171 21 11 +172 22 12 +173 23 13 +174 24 14 +175 25 15 +176 26 16 +177 27 17 +178 28 18 +179 29 19 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +full join 1 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 0 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 1 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 2 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 3 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 4 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 5 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 6 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 7 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 8 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 9 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 10 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 11 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 12 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 13 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 14 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 15 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 16 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 17 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 18 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 19 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 20 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 21 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 22 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 23 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 24 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 25 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 26 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 27 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 28 +0 \N 29 +0 \N 29 +0 \N 29 +0 \N 29 +0 \N 29 +51 11 21 +52 12 22 +53 13 23 +54 14 24 +55 15 25 +56 16 26 +57 17 27 +58 18 28 +59 19 29 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 0 20 +81 1 21 +82 2 22 +83 3 23 +84 4 24 +85 5 25 +86 6 26 +87 7 27 +88 8 28 +89 9 29 +90 10 0 +91 11 1 +92 12 2 +93 13 3 +94 14 4 +95 15 5 +96 16 6 +97 17 7 +98 18 8 +99 19 9 +100 0 10 +101 1 11 +102 2 12 +103 3 13 +104 4 14 +105 5 15 +106 6 16 +107 7 17 +108 8 18 +109 9 19 +110 10 20 +111 11 21 +112 12 22 +113 13 23 +114 14 24 +115 15 25 +116 16 26 +117 17 27 +118 18 28 +119 19 29 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 0 20 +141 1 21 +142 2 22 +143 3 23 +144 4 24 +145 5 25 +146 6 26 +147 7 27 +148 8 28 +149 9 29 +150 10 0 +151 11 1 +152 12 2 +153 13 3 +154 14 4 +155 15 5 +156 16 6 +157 17 7 +158 18 8 +159 19 9 +160 0 10 +161 1 11 +162 2 12 +163 3 13 +164 4 14 +165 5 15 +166 6 16 +167 7 17 +168 8 18 +169 9 19 +170 10 20 +171 11 21 +172 12 22 +173 13 23 +174 14 24 +175 15 25 +176 16 26 +177 17 27 +178 18 28 +179 19 29 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +full join 2 +0 0 \N +1 1 \N +2 2 \N +3 3 \N +4 4 \N +5 5 \N +6 6 \N +7 7 \N +8 8 \N +9 9 \N +10 10 \N +11 11 \N +12 12 \N +13 13 \N +14 14 \N +15 15 \N +16 16 \N +17 17 \N +18 18 \N +19 19 \N +20 20 \N +21 21 \N +22 22 \N +23 23 \N +24 24 \N +25 25 \N +26 26 \N +27 27 \N +28 28 \N +29 29 \N +30 0 \N +31 1 \N +32 2 \N +33 3 \N +34 4 \N +35 5 \N +36 6 \N +37 7 \N +38 8 \N +39 9 \N +40 10 \N +41 11 \N +42 12 \N +43 13 \N +44 14 \N +45 15 \N +46 16 \N +47 17 \N +48 18 \N +49 19 \N +50 20 \N +51 21 11 +52 22 12 +53 23 13 +54 24 14 +55 25 15 +56 26 16 +57 27 17 +58 28 18 +59 29 19 +60 0 0 +61 1 1 +62 2 2 +63 3 3 +64 4 4 +65 5 5 +66 6 6 +67 7 7 +68 8 8 +69 9 9 +70 10 10 +71 11 11 +72 12 12 +73 13 13 +74 14 14 +75 15 15 +76 16 16 +77 17 17 +78 18 18 +79 19 19 +80 20 0 +81 21 1 +82 22 2 +83 23 3 +84 24 4 +85 25 5 +86 26 6 +87 27 7 +88 28 8 +89 29 9 +90 0 10 +91 1 11 +92 2 12 +93 3 13 +94 4 14 +95 5 15 +96 6 16 +97 7 17 +98 8 18 +99 9 19 +100 10 0 +101 11 1 +102 12 2 +103 13 3 +104 14 4 +105 15 5 +106 16 6 +107 17 7 +108 18 8 +109 19 9 +110 20 10 +111 21 11 +112 22 12 +113 23 13 +114 24 14 +115 25 15 +116 26 16 +117 27 17 +118 28 18 +119 29 19 +120 0 0 +121 1 1 +122 2 2 +123 3 3 +124 4 4 +125 5 5 +126 6 6 +127 7 7 +128 8 8 +129 9 9 +130 10 10 +131 11 11 +132 12 12 +133 13 13 +134 14 14 +135 15 15 +136 16 16 +137 17 17 +138 18 18 +139 19 19 +140 20 0 +141 21 1 +142 22 2 +143 23 3 +144 24 4 +145 25 5 +146 26 6 +147 27 7 +148 28 8 +149 29 9 +150 0 10 +151 1 11 +152 2 12 +153 3 13 +154 4 14 +155 5 15 +156 6 16 +157 7 17 +158 8 18 +159 9 19 +160 10 0 +161 11 1 +162 12 2 +163 13 3 +164 14 4 +165 15 5 +166 16 6 +167 17 7 +168 18 8 +169 19 9 +170 20 10 +171 21 11 +172 22 12 +173 23 13 +174 24 14 +175 25 15 +176 26 16 +177 27 17 +178 28 18 +179 29 19 +180 0 0 +181 1 1 +182 2 2 +183 3 3 +184 4 4 +185 5 5 +186 6 6 +187 7 7 +188 8 8 +189 9 9 +190 10 10 +191 11 11 +192 12 12 +193 13 13 +194 14 14 +195 15 15 +196 16 16 +197 17 17 +198 18 18 +199 19 19 +200 20 \N +201 21 \N +202 22 \N +203 23 \N +204 24 \N +205 25 \N +206 26 \N +207 27 \N +208 28 \N +209 29 \N +210 0 \N +211 1 \N +212 2 \N +213 3 \N +214 4 \N +215 5 \N +216 6 \N +217 7 \N +218 8 \N +219 9 \N +220 10 \N +221 11 \N +222 12 \N +223 13 \N +224 14 \N +225 15 \N +226 16 \N +227 17 \N +228 18 \N +229 19 \N +230 20 \N +231 21 \N +232 22 \N +233 23 \N +234 24 \N +235 25 \N +236 26 \N +237 27 \N +238 28 \N +239 29 \N +240 0 \N +241 1 \N +242 2 \N +243 3 \N +244 4 \N +245 5 \N +246 6 \N +247 7 \N +248 8 \N +249 9 \N +250 10 \N +251 11 \N +252 12 \N +253 13 \N +254 14 \N +255 15 \N +256 16 \N +257 17 \N +258 18 \N +259 19 \N +260 20 \N +261 21 \N +262 22 \N +263 23 \N +264 24 \N +265 25 \N +266 26 \N +267 27 \N +268 28 \N +269 29 \N +270 0 \N +271 1 \N +272 2 \N +273 3 \N +274 4 \N +275 5 \N +276 6 \N +277 7 \N +278 8 \N +279 9 \N +280 10 \N +281 11 \N +282 12 \N +283 13 \N +284 14 \N +285 15 \N +286 16 \N +287 17 \N +288 18 \N +289 19 \N +290 20 \N +291 21 \N +292 22 \N +293 23 \N +294 24 \N +295 25 \N +296 26 \N +297 27 \N +298 28 \N +299 29 \N diff --git a/tests/queries/0_stateless/02724_grace_hash_join.sql b/tests/queries/0_stateless/02724_grace_hash_join.sql new file mode 100644 index 00000000000..370f6058bef --- /dev/null +++ b/tests/queries/0_stateless/02724_grace_hash_join.sql @@ -0,0 +1,44 @@ +set join_algorithm = 'grace_hash'; +set max_rows_in_join = 100; + +drop table if exists test_t1; +drop table if exists test_t2; + +CREATE TABLE test_t1 +( + `x` UInt64, + `y` Nullable(UInt64) +) +ENGINE = Memory; + +CREATE TABLE test_t2 +( + `x` UInt64, + `y` Nullable(UInt64) +) +ENGINE = Memory; + +insert into test_t1 select number as x, number % 20 as y from numbers(200) where number > 50 order by y; +insert into test_t2 select number as x , number % 30 as y from numbers(300) order by y; + +select 'inner join 1'; +select l.x, l.y, r.y from test_t1 as l inner join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; +select 'inner join 2'; +select l.x, l.y, r.y from test_t2 as l inner join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; +select 'left join 1'; +select l.x, l.y, r.y from test_t1 as l left join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; +select 'left join 2'; +select l.x, l.y, r.y from test_t2 as l left join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; +select 'right join 1'; +select l.x, l.y, r.y from test_t1 as l right join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; +select 'right join 2'; +select l.x, l.y, r.y from test_t2 as l right join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; +select 'full join 1'; +select l.x, l.y, r.y from test_t1 as l full join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; +select 'full join 2'; +select l.x, l.y, r.y from test_t2 as l full join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; + + + + + From 39db0f84d95ade3ab88c0c6bb8cd24b4ac2a58a0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 4 May 2023 11:27:20 +0800 Subject: [PATCH 0487/2223] add comment --- src/Interpreters/GraceHashJoin.cpp | 11 ++++++++--- src/Interpreters/JoinUtils.cpp | 2 +- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 32801887f2b..95bb76b8373 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -302,7 +302,7 @@ void GraceHashJoin::initBuckets() current_bucket->startJoining(); } -bool GraceHashJoin::isSupported(const std::shared_ptr & table_join [[maybe_unused]]) +bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) { bool is_asof = (table_join->strictness() == JoinStrictness::Asof); @@ -472,8 +472,10 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks( - const Block & left_sample_block_ [[maybe_unused]], const Block & result_sample_block_ [[maybe_unused]], UInt64 max_block_size_ [[maybe_unused]]) const +// This is only be called for bucket[0] at present. other buckets non-joined blocks are generated in +// DelayedBlocks. +IBlocksStreamPtr +GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); } @@ -510,6 +512,8 @@ public: do { + // When left reader finish, return non-joined blocks. + // empty block means the end of this stream. if (!is_left_reader_finished) { block = left_reader.read(); @@ -521,6 +525,7 @@ public: } if (is_left_reader_finished) { + // full/right join, non_joined_blocks != nullptr if (non_joined_blocks) { block = non_joined_blocks->next(); diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index 59ee86ffa18..a05b58e14a1 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -19,7 +19,6 @@ #include - namespace DB { @@ -865,6 +864,7 @@ Block NotJoinedBlocks::nextImpl() size_t rows_added = filler->fillColumns(columns_right); if (rows_added == 0) return {}; + addLeftColumns(result_block, rows_added); addRightColumns(result_block, columns_right); } From 7772fed161b96cda58b837e589413b5fec01d49d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 5 May 2023 15:09:39 +0800 Subject: [PATCH 0488/2223] update 1. fixed the memoery overflow problem when handle all delayed buckets parallely 2. resue exists tests --- src/Interpreters/GraceHashJoin.cpp | 87 +- src/Interpreters/GraceHashJoin.h | 3 + .../Transforms/JoiningTransform.cpp | 90 +- src/Processors/Transforms/JoiningTransform.h | 6 +- .../01721_join_implicit_cast_long.sql.j2 | 1 - .../0_stateless/02273_full_sort_join.sql.j2 | 2 - .../02274_full_sort_join_nodistinct.sql.j2 | 1 - .../02275_full_sort_join_long.reference | 24 +- .../02275_full_sort_join_long.sql.j2 | 9 +- .../02724_grace_hash_join.reference | 1804 ----------------- .../0_stateless/02724_grace_hash_join.sql | 44 - 11 files changed, 128 insertions(+), 1943 deletions(-) delete mode 100644 tests/queries/0_stateless/02724_grace_hash_join.reference delete mode 100644 tests/queries/0_stateless/02724_grace_hash_join.sql diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 95bb76b8373..407663ac310 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -12,6 +12,7 @@ #include #include #include +#include "GraceHashJoin.h" #include #include @@ -472,12 +473,67 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -// This is only be called for bucket[0] at present. other buckets non-joined blocks are generated in +// Add lock for reading non-joined blocks, it's thread safe. +class GraceHashJoin::NonJoinedBlocksStream : public IBlocksStream +{ +public: + explicit NonJoinedBlocksStream( + InMemoryJoinPtr hash_join_, + const Block & left_sample_block_, + const Block & result_sample_block_, + size_t max_block_size_) + : hash_join(hash_join_) + , left_sample_block(left_sample_block_) + , result_sample_block(result_sample_block_) + , max_block_size(max_block_size_) + { + } + + Block nextImpl() override + { + // initialize non_joined_blocks lazily. + if (!has_init_non_joined_blocks) [[unlikely]] + { + std::lock_guard lock(mutex); + if (!has_init_non_joined_blocks) + { + has_init_non_joined_blocks = true; + non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); + } + } + // For right/full join, non_joined_blocks is not null. + // For inner/left join, non_joined_blocks is null, and return a empty block directly. + std::lock_guard lock(mutex); + if (non_joined_blocks) + return non_joined_blocks->next(); + return {}; + } + +private: + InMemoryJoinPtr hash_join; + Block left_sample_block; + Block result_sample_block; + size_t max_block_size; + bool has_init_non_joined_blocks = false; + IBlocksStreamPtr non_joined_blocks = nullptr; + std::mutex mutex; +}; + +// This is only be called for bucket[0] at present. other buckets' non-joined blocks are generated in // DelayedBlocks. IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); + if (!has_initialized_non_joined_blocks) [[unlikely]] + { + std::lock_guard lock(hash_join_mutex); + if (!has_initialized_non_joined_blocks) + { + has_initialized_non_joined_blocks = true; + non_joined_blocks = std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); + } + } + return non_joined_blocks; } class GraceHashJoin::DelayedBlocks : public IBlocksStream @@ -498,12 +554,14 @@ public: , left_reader(buckets[current_bucket]->getLeftTableReader()) , left_key_names(left_key_names_) , right_key_names(right_key_names_) - , left_sample_block(left_sample_block_) - , result_sample_block(result_sample_block_) - , max_block_size(max_block_size_) + , non_joined_blocks_iter(hash_join, left_sample_block_, result_sample_block_, max_block_size_) { } + // One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransforms, need locks for + // - reading from left_reader. left_reader.read() has had a lock inside. + // - reading non-joined blocks from hash_join. Since iterate on non-joined blocks will has state + // changed inside. Block nextImpl() override { Block block; @@ -520,24 +578,14 @@ public: if (!block) { is_left_reader_finished = true; - non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); } } if (is_left_reader_finished) { - // full/right join, non_joined_blocks != nullptr - if (non_joined_blocks) - { - block = non_joined_blocks->next(); - return block; - } - else - { - // left/inner join. - return {}; - } + return non_joined_blocks_iter.next(); } + // block comes from left_reader, need to join with right table to get the result. Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets); block = std::move(blocks[current_idx]); @@ -577,11 +625,8 @@ public: Names left_key_names; Names right_key_names; - Block left_sample_block; - Block result_sample_block; bool is_left_reader_finished = false; - IBlocksStreamPtr non_joined_blocks; - size_t max_block_size; + NonJoinedBlocksStream non_joined_blocks_iter; }; IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index eb39ee09208..ec0e5ac05aa 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -43,6 +43,7 @@ class HashJoin; class GraceHashJoin final : public IJoin { class FileBucket; + class NonJoinedBlocksStream; class DelayedBlocks; using InMemoryJoin = HashJoin; @@ -144,6 +145,8 @@ private: InMemoryJoinPtr hash_join; Block hash_join_sample_block; mutable std::mutex hash_join_mutex; + mutable bool has_initialized_non_joined_blocks = false; + mutable std::shared_ptr non_joined_blocks = nullptr; }; } diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 4789e9c514a..b638a5582c5 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -350,11 +350,6 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (!task) { - if (input.isFinished()) - { - output.finish(); - return Status::Finished; - } if (!input.hasData()) { input.setNeeded(); @@ -367,14 +362,10 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() output.pushException(data.exception); return Status::Finished; } - if (data.chunk.hasChunkInfo()) - task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); - else - { - // Try to get one task from DelayedJoinedBlocksTransform again. - task = nullptr; - return Status::NeedData; - } + + if (!data.chunk.hasChunkInfo()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); + task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); } else { @@ -395,6 +386,7 @@ void DelayedJoinedBlocksWorkerTransform::work() { if (!task) return; + Block block = task->delayed_blocks->next(); if (!block) @@ -416,19 +408,15 @@ DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, J void DelayedJoinedBlocksTransform::work() { - if (all_buckets_finished) + if (finished) return; + delayed_blocks = join->getDelayedBlocks(); - if (!delayed_blocks) - { - all_buckets_finished = true; - } + finished = finished || delayed_blocks == nullptr; } IProcessor::Status DelayedJoinedBlocksTransform::prepare() { - - bool should_finished = false; for (auto & output : outputs) { if (output.isFinished()) @@ -436,56 +424,44 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() /// If at least one output is finished, then we have read all data from buckets. /// Some workers can still be busy with joining the last chunk of data in memory, /// but after that they also will finish when they will try to get next chunk. - should_finished = true; - break; + finished = true; + continue; } - } - if (should_finished) - { - for (auto & output : outputs) - { - if (!output.isFinished()) - { - output.finish(); - } - } - return Status::Finished; + if (!output.canPush()) + return Status::PortFull; } - // No pending buckets. - if (all_buckets_finished && !delayed_blocks) + if (finished) { + // Since have memory limit, cannot handle all buckets parallelly by different + // DelayedJoinedBlocksWorkerTransform. So send the same task to all outputs. + // Wait for all DelayedJoinedBlocksWorkerTransform be idle before getting next bucket. for (auto & output : outputs) + { + if (output.isFinished()) + continue; + Chunk chunk; + chunk.setChunkInfo(std::make_shared()); + output.push(std::move(chunk)); output.finish(); + } + return Status::Finished; } - // Try to read next bucket. - if (!delayed_blocks) - return Status::Ready; - - // Put the ready task to the first available output port. - // Put empty tasks to other output ports. these ports will do nothing but require data again. - // Each bucket will handled in one thread, avoid lock contention in left file reader. - for (auto & output : outputs) + if (delayed_blocks) { - if (output.canPush()) + for (auto & output : outputs) { - if (delayed_blocks) - { - Chunk chunk; - chunk.setChunkInfo(std::make_shared(delayed_blocks)); - output.push(std::move(chunk)); - delayed_blocks = nullptr; - } - else - { - Chunk chunk; - output.push(std::move(chunk)); - } + Chunk chunk; + chunk.setChunkInfo(std::make_shared(delayed_blocks)); + output.push(std::move(chunk)); } + delayed_blocks = nullptr; + return Status::PortFull; } - return Status::PortFull; + + return Status::Ready; } } diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index f189f53c004..e7edff40c56 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -125,6 +125,7 @@ public: using DelayedBlocksTaskPtr = std::shared_ptr; + /// Reads delayed joined blocks from Join class DelayedJoinedBlocksTransform : public IProcessor { @@ -138,8 +139,9 @@ public: private: JoinPtr join; - IBlocksStreamPtr delayed_blocks; - bool all_buckets_finished = false; + + IBlocksStreamPtr delayed_blocks = nullptr; + bool finished = false; }; class DelayedJoinedBlocksWorkerTransform : public IProcessor diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 index f5321939f28..38f71f4c5ec 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 @@ -10,7 +10,6 @@ INSERT INTO t1 SELECT number as a, 100 + number as b FROM system.numbers LIMIT 1 INSERT INTO t2 SELECT number - 5 as a, 200 + number - 5 as b FROM system.numbers LIMIT 1, 10; {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 8b739330364..b371d9544da 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -26,9 +26,7 @@ INSERT INTO t2 'val' || toString(number) as s FROM numbers_mt({{ table_size - 3 }}); - {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index 613da65421e..6bf634ca1f5 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -16,7 +16,6 @@ INSERT INTO t2 VALUES (1, 'val11'), (1, 'val12'), (2, 'val22'), (2, 'val23'), (2 INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (NULL, 'val28'), (3, 'val3'); {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.reference b/tests/queries/0_stateless/02275_full_sort_join_long.reference index 9ec06aea3e6..73482358d12 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.reference +++ b/tests/queries/0_stateless/02275_full_sort_join_long.reference @@ -41,16 +41,34 @@ ALL INNER ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 98cc46c9cb4..0b28fd67050 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -22,11 +22,6 @@ INSERT INTO t2 FROM numbers_mt({{ rtable_size }}) ; -{% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } -SELECT 'skipped'; -{% endif -%} -{% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} @@ -40,7 +35,6 @@ SET join_algorithm = '{{ join_algorithm }}'; SET max_block_size = {{ block_size }}; -{% if not (kind == 'ANY' and join_algorithm == 'grace_hash') -%} SELECT '{{ kind }} INNER'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 @@ -58,9 +52,8 @@ SELECT '{{ kind }} RIGHT'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 {{ kind }} RIGHT JOIN t2 ON t1.key == t2.key -; {{ is_implemented(join_algorithm) }} +; -{% endif -%} {% endfor -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02724_grace_hash_join.reference b/tests/queries/0_stateless/02724_grace_hash_join.reference deleted file mode 100644 index 296ef623f04..00000000000 --- a/tests/queries/0_stateless/02724_grace_hash_join.reference +++ /dev/null @@ -1,1804 +0,0 @@ -inner join 1 -51 11 21 -52 12 22 -53 13 23 -54 14 24 -55 15 25 -56 16 26 -57 17 27 -58 18 28 -59 19 29 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 0 20 -81 1 21 -82 2 22 -83 3 23 -84 4 24 -85 5 25 -86 6 26 -87 7 27 -88 8 28 -89 9 29 -90 10 0 -91 11 1 -92 12 2 -93 13 3 -94 14 4 -95 15 5 -96 16 6 -97 17 7 -98 18 8 -99 19 9 -100 0 10 -101 1 11 -102 2 12 -103 3 13 -104 4 14 -105 5 15 -106 6 16 -107 7 17 -108 8 18 -109 9 19 -110 10 20 -111 11 21 -112 12 22 -113 13 23 -114 14 24 -115 15 25 -116 16 26 -117 17 27 -118 18 28 -119 19 29 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 0 20 -141 1 21 -142 2 22 -143 3 23 -144 4 24 -145 5 25 -146 6 26 -147 7 27 -148 8 28 -149 9 29 -150 10 0 -151 11 1 -152 12 2 -153 13 3 -154 14 4 -155 15 5 -156 16 6 -157 17 7 -158 18 8 -159 19 9 -160 0 10 -161 1 11 -162 2 12 -163 3 13 -164 4 14 -165 5 15 -166 6 16 -167 7 17 -168 8 18 -169 9 19 -170 10 20 -171 11 21 -172 12 22 -173 13 23 -174 14 24 -175 15 25 -176 16 26 -177 17 27 -178 18 28 -179 19 29 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -inner join 2 -51 21 11 -52 22 12 -53 23 13 -54 24 14 -55 25 15 -56 26 16 -57 27 17 -58 28 18 -59 29 19 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 20 0 -81 21 1 -82 22 2 -83 23 3 -84 24 4 -85 25 5 -86 26 6 -87 27 7 -88 28 8 -89 29 9 -90 0 10 -91 1 11 -92 2 12 -93 3 13 -94 4 14 -95 5 15 -96 6 16 -97 7 17 -98 8 18 -99 9 19 -100 10 0 -101 11 1 -102 12 2 -103 13 3 -104 14 4 -105 15 5 -106 16 6 -107 17 7 -108 18 8 -109 19 9 -110 20 10 -111 21 11 -112 22 12 -113 23 13 -114 24 14 -115 25 15 -116 26 16 -117 27 17 -118 28 18 -119 29 19 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 20 0 -141 21 1 -142 22 2 -143 23 3 -144 24 4 -145 25 5 -146 26 6 -147 27 7 -148 28 8 -149 29 9 -150 0 10 -151 1 11 -152 2 12 -153 3 13 -154 4 14 -155 5 15 -156 6 16 -157 7 17 -158 8 18 -159 9 19 -160 10 0 -161 11 1 -162 12 2 -163 13 3 -164 14 4 -165 15 5 -166 16 6 -167 17 7 -168 18 8 -169 19 9 -170 20 10 -171 21 11 -172 22 12 -173 23 13 -174 24 14 -175 25 15 -176 26 16 -177 27 17 -178 28 18 -179 29 19 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -left join 1 -51 11 21 -52 12 22 -53 13 23 -54 14 24 -55 15 25 -56 16 26 -57 17 27 -58 18 28 -59 19 29 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 0 20 -81 1 21 -82 2 22 -83 3 23 -84 4 24 -85 5 25 -86 6 26 -87 7 27 -88 8 28 -89 9 29 -90 10 0 -91 11 1 -92 12 2 -93 13 3 -94 14 4 -95 15 5 -96 16 6 -97 17 7 -98 18 8 -99 19 9 -100 0 10 -101 1 11 -102 2 12 -103 3 13 -104 4 14 -105 5 15 -106 6 16 -107 7 17 -108 8 18 -109 9 19 -110 10 20 -111 11 21 -112 12 22 -113 13 23 -114 14 24 -115 15 25 -116 16 26 -117 17 27 -118 18 28 -119 19 29 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 0 20 -141 1 21 -142 2 22 -143 3 23 -144 4 24 -145 5 25 -146 6 26 -147 7 27 -148 8 28 -149 9 29 -150 10 0 -151 11 1 -152 12 2 -153 13 3 -154 14 4 -155 15 5 -156 16 6 -157 17 7 -158 18 8 -159 19 9 -160 0 10 -161 1 11 -162 2 12 -163 3 13 -164 4 14 -165 5 15 -166 6 16 -167 7 17 -168 8 18 -169 9 19 -170 10 20 -171 11 21 -172 12 22 -173 13 23 -174 14 24 -175 15 25 -176 16 26 -177 17 27 -178 18 28 -179 19 29 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -left join 2 -0 0 \N -1 1 \N -2 2 \N -3 3 \N -4 4 \N -5 5 \N -6 6 \N -7 7 \N -8 8 \N -9 9 \N -10 10 \N -11 11 \N -12 12 \N -13 13 \N -14 14 \N -15 15 \N -16 16 \N -17 17 \N -18 18 \N -19 19 \N -20 20 \N -21 21 \N -22 22 \N -23 23 \N -24 24 \N -25 25 \N -26 26 \N -27 27 \N -28 28 \N -29 29 \N -30 0 \N -31 1 \N -32 2 \N -33 3 \N -34 4 \N -35 5 \N -36 6 \N -37 7 \N -38 8 \N -39 9 \N -40 10 \N -41 11 \N -42 12 \N -43 13 \N -44 14 \N -45 15 \N -46 16 \N -47 17 \N -48 18 \N -49 19 \N -50 20 \N -51 21 11 -52 22 12 -53 23 13 -54 24 14 -55 25 15 -56 26 16 -57 27 17 -58 28 18 -59 29 19 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 20 0 -81 21 1 -82 22 2 -83 23 3 -84 24 4 -85 25 5 -86 26 6 -87 27 7 -88 28 8 -89 29 9 -90 0 10 -91 1 11 -92 2 12 -93 3 13 -94 4 14 -95 5 15 -96 6 16 -97 7 17 -98 8 18 -99 9 19 -100 10 0 -101 11 1 -102 12 2 -103 13 3 -104 14 4 -105 15 5 -106 16 6 -107 17 7 -108 18 8 -109 19 9 -110 20 10 -111 21 11 -112 22 12 -113 23 13 -114 24 14 -115 25 15 -116 26 16 -117 27 17 -118 28 18 -119 29 19 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 20 0 -141 21 1 -142 22 2 -143 23 3 -144 24 4 -145 25 5 -146 26 6 -147 27 7 -148 28 8 -149 29 9 -150 0 10 -151 1 11 -152 2 12 -153 3 13 -154 4 14 -155 5 15 -156 6 16 -157 7 17 -158 8 18 -159 9 19 -160 10 0 -161 11 1 -162 12 2 -163 13 3 -164 14 4 -165 15 5 -166 16 6 -167 17 7 -168 18 8 -169 19 9 -170 20 10 -171 21 11 -172 22 12 -173 23 13 -174 24 14 -175 25 15 -176 26 16 -177 27 17 -178 28 18 -179 29 19 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -200 20 \N -201 21 \N -202 22 \N -203 23 \N -204 24 \N -205 25 \N -206 26 \N -207 27 \N -208 28 \N -209 29 \N -210 0 \N -211 1 \N -212 2 \N -213 3 \N -214 4 \N -215 5 \N -216 6 \N -217 7 \N -218 8 \N -219 9 \N -220 10 \N -221 11 \N -222 12 \N -223 13 \N -224 14 \N -225 15 \N -226 16 \N -227 17 \N -228 18 \N -229 19 \N -230 20 \N -231 21 \N -232 22 \N -233 23 \N -234 24 \N -235 25 \N -236 26 \N -237 27 \N -238 28 \N -239 29 \N -240 0 \N -241 1 \N -242 2 \N -243 3 \N -244 4 \N -245 5 \N -246 6 \N -247 7 \N -248 8 \N -249 9 \N -250 10 \N -251 11 \N -252 12 \N -253 13 \N -254 14 \N -255 15 \N -256 16 \N -257 17 \N -258 18 \N -259 19 \N -260 20 \N -261 21 \N -262 22 \N -263 23 \N -264 24 \N -265 25 \N -266 26 \N -267 27 \N -268 28 \N -269 29 \N -270 0 \N -271 1 \N -272 2 \N -273 3 \N -274 4 \N -275 5 \N -276 6 \N -277 7 \N -278 8 \N -279 9 \N -280 10 \N -281 11 \N -282 12 \N -283 13 \N -284 14 \N -285 15 \N -286 16 \N -287 17 \N -288 18 \N -289 19 \N -290 20 \N -291 21 \N -292 22 \N -293 23 \N -294 24 \N -295 25 \N -296 26 \N -297 27 \N -298 28 \N -299 29 \N -right join 1 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 29 -0 \N 29 -0 \N 29 -0 \N 29 -0 \N 29 -51 11 21 -52 12 22 -53 13 23 -54 14 24 -55 15 25 -56 16 26 -57 17 27 -58 18 28 -59 19 29 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 0 20 -81 1 21 -82 2 22 -83 3 23 -84 4 24 -85 5 25 -86 6 26 -87 7 27 -88 8 28 -89 9 29 -90 10 0 -91 11 1 -92 12 2 -93 13 3 -94 14 4 -95 15 5 -96 16 6 -97 17 7 -98 18 8 -99 19 9 -100 0 10 -101 1 11 -102 2 12 -103 3 13 -104 4 14 -105 5 15 -106 6 16 -107 7 17 -108 8 18 -109 9 19 -110 10 20 -111 11 21 -112 12 22 -113 13 23 -114 14 24 -115 15 25 -116 16 26 -117 17 27 -118 18 28 -119 19 29 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 0 20 -141 1 21 -142 2 22 -143 3 23 -144 4 24 -145 5 25 -146 6 26 -147 7 27 -148 8 28 -149 9 29 -150 10 0 -151 11 1 -152 12 2 -153 13 3 -154 14 4 -155 15 5 -156 16 6 -157 17 7 -158 18 8 -159 19 9 -160 0 10 -161 1 11 -162 2 12 -163 3 13 -164 4 14 -165 5 15 -166 6 16 -167 7 17 -168 8 18 -169 9 19 -170 10 20 -171 11 21 -172 12 22 -173 13 23 -174 14 24 -175 15 25 -176 16 26 -177 17 27 -178 18 28 -179 19 29 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -right join 2 -51 21 11 -52 22 12 -53 23 13 -54 24 14 -55 25 15 -56 26 16 -57 27 17 -58 28 18 -59 29 19 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 20 0 -81 21 1 -82 22 2 -83 23 3 -84 24 4 -85 25 5 -86 26 6 -87 27 7 -88 28 8 -89 29 9 -90 0 10 -91 1 11 -92 2 12 -93 3 13 -94 4 14 -95 5 15 -96 6 16 -97 7 17 -98 8 18 -99 9 19 -100 10 0 -101 11 1 -102 12 2 -103 13 3 -104 14 4 -105 15 5 -106 16 6 -107 17 7 -108 18 8 -109 19 9 -110 20 10 -111 21 11 -112 22 12 -113 23 13 -114 24 14 -115 25 15 -116 26 16 -117 27 17 -118 28 18 -119 29 19 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 20 0 -141 21 1 -142 22 2 -143 23 3 -144 24 4 -145 25 5 -146 26 6 -147 27 7 -148 28 8 -149 29 9 -150 0 10 -151 1 11 -152 2 12 -153 3 13 -154 4 14 -155 5 15 -156 6 16 -157 7 17 -158 8 18 -159 9 19 -160 10 0 -161 11 1 -162 12 2 -163 13 3 -164 14 4 -165 15 5 -166 16 6 -167 17 7 -168 18 8 -169 19 9 -170 20 10 -171 21 11 -172 22 12 -173 23 13 -174 24 14 -175 25 15 -176 26 16 -177 27 17 -178 28 18 -179 29 19 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -full join 1 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 0 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 1 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 2 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 3 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 4 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 5 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 6 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 7 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 8 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 9 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 10 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 11 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 12 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 13 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 14 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 15 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 16 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 17 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 18 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 19 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 20 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 21 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 22 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 23 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 24 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 25 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 26 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 27 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 28 -0 \N 29 -0 \N 29 -0 \N 29 -0 \N 29 -0 \N 29 -51 11 21 -52 12 22 -53 13 23 -54 14 24 -55 15 25 -56 16 26 -57 17 27 -58 18 28 -59 19 29 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 0 20 -81 1 21 -82 2 22 -83 3 23 -84 4 24 -85 5 25 -86 6 26 -87 7 27 -88 8 28 -89 9 29 -90 10 0 -91 11 1 -92 12 2 -93 13 3 -94 14 4 -95 15 5 -96 16 6 -97 17 7 -98 18 8 -99 19 9 -100 0 10 -101 1 11 -102 2 12 -103 3 13 -104 4 14 -105 5 15 -106 6 16 -107 7 17 -108 8 18 -109 9 19 -110 10 20 -111 11 21 -112 12 22 -113 13 23 -114 14 24 -115 15 25 -116 16 26 -117 17 27 -118 18 28 -119 19 29 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 0 20 -141 1 21 -142 2 22 -143 3 23 -144 4 24 -145 5 25 -146 6 26 -147 7 27 -148 8 28 -149 9 29 -150 10 0 -151 11 1 -152 12 2 -153 13 3 -154 14 4 -155 15 5 -156 16 6 -157 17 7 -158 18 8 -159 19 9 -160 0 10 -161 1 11 -162 2 12 -163 3 13 -164 4 14 -165 5 15 -166 6 16 -167 7 17 -168 8 18 -169 9 19 -170 10 20 -171 11 21 -172 12 22 -173 13 23 -174 14 24 -175 15 25 -176 16 26 -177 17 27 -178 18 28 -179 19 29 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -full join 2 -0 0 \N -1 1 \N -2 2 \N -3 3 \N -4 4 \N -5 5 \N -6 6 \N -7 7 \N -8 8 \N -9 9 \N -10 10 \N -11 11 \N -12 12 \N -13 13 \N -14 14 \N -15 15 \N -16 16 \N -17 17 \N -18 18 \N -19 19 \N -20 20 \N -21 21 \N -22 22 \N -23 23 \N -24 24 \N -25 25 \N -26 26 \N -27 27 \N -28 28 \N -29 29 \N -30 0 \N -31 1 \N -32 2 \N -33 3 \N -34 4 \N -35 5 \N -36 6 \N -37 7 \N -38 8 \N -39 9 \N -40 10 \N -41 11 \N -42 12 \N -43 13 \N -44 14 \N -45 15 \N -46 16 \N -47 17 \N -48 18 \N -49 19 \N -50 20 \N -51 21 11 -52 22 12 -53 23 13 -54 24 14 -55 25 15 -56 26 16 -57 27 17 -58 28 18 -59 29 19 -60 0 0 -61 1 1 -62 2 2 -63 3 3 -64 4 4 -65 5 5 -66 6 6 -67 7 7 -68 8 8 -69 9 9 -70 10 10 -71 11 11 -72 12 12 -73 13 13 -74 14 14 -75 15 15 -76 16 16 -77 17 17 -78 18 18 -79 19 19 -80 20 0 -81 21 1 -82 22 2 -83 23 3 -84 24 4 -85 25 5 -86 26 6 -87 27 7 -88 28 8 -89 29 9 -90 0 10 -91 1 11 -92 2 12 -93 3 13 -94 4 14 -95 5 15 -96 6 16 -97 7 17 -98 8 18 -99 9 19 -100 10 0 -101 11 1 -102 12 2 -103 13 3 -104 14 4 -105 15 5 -106 16 6 -107 17 7 -108 18 8 -109 19 9 -110 20 10 -111 21 11 -112 22 12 -113 23 13 -114 24 14 -115 25 15 -116 26 16 -117 27 17 -118 28 18 -119 29 19 -120 0 0 -121 1 1 -122 2 2 -123 3 3 -124 4 4 -125 5 5 -126 6 6 -127 7 7 -128 8 8 -129 9 9 -130 10 10 -131 11 11 -132 12 12 -133 13 13 -134 14 14 -135 15 15 -136 16 16 -137 17 17 -138 18 18 -139 19 19 -140 20 0 -141 21 1 -142 22 2 -143 23 3 -144 24 4 -145 25 5 -146 26 6 -147 27 7 -148 28 8 -149 29 9 -150 0 10 -151 1 11 -152 2 12 -153 3 13 -154 4 14 -155 5 15 -156 6 16 -157 7 17 -158 8 18 -159 9 19 -160 10 0 -161 11 1 -162 12 2 -163 13 3 -164 14 4 -165 15 5 -166 16 6 -167 17 7 -168 18 8 -169 19 9 -170 20 10 -171 21 11 -172 22 12 -173 23 13 -174 24 14 -175 25 15 -176 26 16 -177 27 17 -178 28 18 -179 29 19 -180 0 0 -181 1 1 -182 2 2 -183 3 3 -184 4 4 -185 5 5 -186 6 6 -187 7 7 -188 8 8 -189 9 9 -190 10 10 -191 11 11 -192 12 12 -193 13 13 -194 14 14 -195 15 15 -196 16 16 -197 17 17 -198 18 18 -199 19 19 -200 20 \N -201 21 \N -202 22 \N -203 23 \N -204 24 \N -205 25 \N -206 26 \N -207 27 \N -208 28 \N -209 29 \N -210 0 \N -211 1 \N -212 2 \N -213 3 \N -214 4 \N -215 5 \N -216 6 \N -217 7 \N -218 8 \N -219 9 \N -220 10 \N -221 11 \N -222 12 \N -223 13 \N -224 14 \N -225 15 \N -226 16 \N -227 17 \N -228 18 \N -229 19 \N -230 20 \N -231 21 \N -232 22 \N -233 23 \N -234 24 \N -235 25 \N -236 26 \N -237 27 \N -238 28 \N -239 29 \N -240 0 \N -241 1 \N -242 2 \N -243 3 \N -244 4 \N -245 5 \N -246 6 \N -247 7 \N -248 8 \N -249 9 \N -250 10 \N -251 11 \N -252 12 \N -253 13 \N -254 14 \N -255 15 \N -256 16 \N -257 17 \N -258 18 \N -259 19 \N -260 20 \N -261 21 \N -262 22 \N -263 23 \N -264 24 \N -265 25 \N -266 26 \N -267 27 \N -268 28 \N -269 29 \N -270 0 \N -271 1 \N -272 2 \N -273 3 \N -274 4 \N -275 5 \N -276 6 \N -277 7 \N -278 8 \N -279 9 \N -280 10 \N -281 11 \N -282 12 \N -283 13 \N -284 14 \N -285 15 \N -286 16 \N -287 17 \N -288 18 \N -289 19 \N -290 20 \N -291 21 \N -292 22 \N -293 23 \N -294 24 \N -295 25 \N -296 26 \N -297 27 \N -298 28 \N -299 29 \N diff --git a/tests/queries/0_stateless/02724_grace_hash_join.sql b/tests/queries/0_stateless/02724_grace_hash_join.sql deleted file mode 100644 index 370f6058bef..00000000000 --- a/tests/queries/0_stateless/02724_grace_hash_join.sql +++ /dev/null @@ -1,44 +0,0 @@ -set join_algorithm = 'grace_hash'; -set max_rows_in_join = 100; - -drop table if exists test_t1; -drop table if exists test_t2; - -CREATE TABLE test_t1 -( - `x` UInt64, - `y` Nullable(UInt64) -) -ENGINE = Memory; - -CREATE TABLE test_t2 -( - `x` UInt64, - `y` Nullable(UInt64) -) -ENGINE = Memory; - -insert into test_t1 select number as x, number % 20 as y from numbers(200) where number > 50 order by y; -insert into test_t2 select number as x , number % 30 as y from numbers(300) order by y; - -select 'inner join 1'; -select l.x, l.y, r.y from test_t1 as l inner join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; -select 'inner join 2'; -select l.x, l.y, r.y from test_t2 as l inner join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; -select 'left join 1'; -select l.x, l.y, r.y from test_t1 as l left join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; -select 'left join 2'; -select l.x, l.y, r.y from test_t2 as l left join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; -select 'right join 1'; -select l.x, l.y, r.y from test_t1 as l right join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; -select 'right join 2'; -select l.x, l.y, r.y from test_t2 as l right join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; -select 'full join 1'; -select l.x, l.y, r.y from test_t1 as l full join test_t2 as r on l.x = r.x order by l.x, l.y, r.y; -select 'full join 2'; -select l.x, l.y, r.y from test_t2 as l full join test_t1 as r on l.x = r.x order by l.x, l.y, r.y; - - - - - From d89beb1bf7ddfa782eb7420aa5e148de5237ffc6 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sat, 6 May 2023 09:05:23 +0800 Subject: [PATCH 0489/2223] update tests --- src/Interpreters/GraceHashJoin.cpp | 15 ++----- src/Interpreters/GraceHashJoin.h | 2 - ...01721_join_implicit_cast_long.reference.j2 | 42 ------------------- .../02273_full_sort_join.reference.j2 | 16 ------- ...274_full_sort_join_nodistinct.reference.j2 | 32 -------------- 5 files changed, 3 insertions(+), 104 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 407663ac310..ee369d4b195 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -524,16 +524,7 @@ private: IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - if (!has_initialized_non_joined_blocks) [[unlikely]] - { - std::lock_guard lock(hash_join_mutex); - if (!has_initialized_non_joined_blocks) - { - has_initialized_non_joined_blocks = true; - non_joined_blocks = std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); - } - } - return non_joined_blocks; + return std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream @@ -580,7 +571,7 @@ public: is_left_reader_finished = true; } } - if (is_left_reader_finished) + if (is_left_reader_finished || !block) { return non_joined_blocks_iter.next(); } @@ -625,7 +616,7 @@ public: Names left_key_names; Names right_key_names; - bool is_left_reader_finished = false; + std::atomic is_left_reader_finished = false; NonJoinedBlocksStream non_joined_blocks_iter; }; diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index ec0e5ac05aa..bc5f894bc8c 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -145,8 +145,6 @@ private: InMemoryJoinPtr hash_join; Block hash_join_sample_block; mutable std::mutex hash_join_mutex; - mutable bool has_initialized_non_joined_blocks = false; - mutable std::shared_ptr non_joined_blocks = nullptr; }; } diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 index e9f32087439..d40fc7ec864 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 @@ -1,7 +1,6 @@ {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} === {{ join_algorithm }} === = full = -{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -17,7 +16,6 @@ 8 108 \N 9 109 \N 10 110 \N -{% endif -%} = left = 1 101 201 2 102 202 @@ -30,7 +28,6 @@ 9 109 \N 10 110 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -41,7 +38,6 @@ 3 103 203 4 104 204 5 105 205 -{% endif -%} = inner = 1 101 201 2 102 202 @@ -49,7 +45,6 @@ 4 104 204 5 105 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -65,7 +60,6 @@ 8 8 0 9 9 0 10 10 0 -{% endif -%} = left = 1 1 1 2 2 2 @@ -78,7 +72,6 @@ 9 9 0 10 10 0 = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -89,7 +82,6 @@ 3 3 3 4 4 4 5 5 5 -{% endif -%} = inner = 1 1 1 2 2 2 @@ -98,7 +90,6 @@ 5 5 5 = join on = = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -114,7 +105,6 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -127,7 +117,6 @@ 9 109 0 \N 10 110 0 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -138,7 +127,6 @@ 3 103 3 203 4 104 4 204 5 105 5 205 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -146,7 +134,6 @@ 4 104 4 204 5 105 5 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -162,7 +149,6 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -175,7 +161,6 @@ 9 109 0 \N 10 110 0 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -186,7 +171,6 @@ 3 103 3 203 4 104 4 204 5 105 5 205 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -196,7 +180,6 @@ = agg = 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -205,13 +188,11 @@ 1 55 1055 0 0 -10 0 990 1 55 15 1055 1015 -{% endif -%} = types = 1 1 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -219,11 +200,8 @@ 1 1 1 -{% endif -%} -{% if join_algorithm not in ['full_sorting_merge'] -%} === join use nulls === = full = -{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -239,7 +217,6 @@ 8 108 \N 9 109 \N 10 110 \N -{% endif -%} = left = 1 101 201 2 102 202 @@ -252,7 +229,6 @@ 9 109 \N 10 110 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -263,7 +239,6 @@ 3 103 203 4 104 204 5 105 205 -{% endif -%} = inner = 1 101 201 2 102 202 @@ -271,7 +246,6 @@ 4 104 204 5 105 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -287,7 +261,6 @@ \N \N -2 \N \N -1 \N \N 0 -{% endif -%} = left = 1 1 1 2 2 2 @@ -300,7 +273,6 @@ 9 9 \N 10 10 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -311,7 +283,6 @@ \N \N -2 \N \N -1 \N \N 0 -{% endif -%} = inner = 1 1 1 2 2 2 @@ -320,7 +291,6 @@ 5 5 5 = join on = = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -336,7 +306,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -349,7 +318,6 @@ 9 109 \N \N 10 110 \N \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -360,7 +328,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -368,7 +335,6 @@ 4 104 4 204 5 105 5 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -384,7 +350,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -397,7 +362,6 @@ 9 109 \N \N 10 110 \N \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -408,7 +372,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -418,7 +381,6 @@ = agg = 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -427,13 +389,11 @@ 1 55 1055 1 55 15 1055 1015 \N \N -10 \N 990 -{% endif -%} = types = 1 1 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -441,6 +401,4 @@ 1 1 1 -{% endif -%} -{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 index 98bfd9d9b2b..7b994c2d11f 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 @@ -50,7 +50,6 @@ ALL LEFT | bs = {{ block_size }} 14 14 val9 0 14 14 val9 0 ALL RIGHT | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -64,7 +63,6 @@ ALL RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ALL INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -85,7 +83,6 @@ ALL LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | bs = {{ block_size }} | copmosite key -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -99,7 +96,6 @@ ALL RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 -{% endif -%} ANY INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 @@ -137,7 +133,6 @@ ANY LEFT | bs = {{ block_size }} 13 13 val13 0 14 14 val9 0 ANY RIGHT | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -150,7 +145,6 @@ ANY RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ANY INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | bs = {{ block_size }} | copmosite key @@ -170,7 +164,6 @@ ANY LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | bs = {{ block_size }} | copmosite key -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -183,7 +176,6 @@ ANY RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 1 \N 1 val3 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 -{% endif -%} {% endfor -%} ALL INNER | join_use_nulls = 1 4 4 0 0 @@ -219,7 +211,6 @@ ALL LEFT | join_use_nulls = 1 14 14 val9 0 14 14 val9 0 ALL RIGHT | join_use_nulls = 1 -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -233,7 +224,6 @@ ALL RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ALL INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -254,7 +244,6 @@ ALL LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | join_use_nulls = 1 | copmosite key -{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 @@ -268,7 +257,6 @@ ALL RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 -{% endif -%} ANY INNER | join_use_nulls = 1 4 4 0 0 5 5 0 0 @@ -296,7 +284,6 @@ ANY LEFT | join_use_nulls = 1 13 13 val13 0 14 14 val9 0 ANY RIGHT | join_use_nulls = 1 -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -309,7 +296,6 @@ ANY RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ANY INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | join_use_nulls = 1 | copmosite key @@ -329,7 +315,6 @@ ANY LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | join_use_nulls = 1 | copmosite key -{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 \N \N \N 1 1 1 \N val7 @@ -342,5 +327,4 @@ ANY RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 -{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 2cc6c6e85d6..6b5b5bfd298 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -108,7 +108,6 @@ t1 ALL LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -161,7 +160,6 @@ t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val28 2 2 5 val28 3 3 4 val3 -{% endif -%} t1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 2 2 5 5 @@ -177,7 +175,6 @@ t1 ANY LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -188,9 +185,7 @@ t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val27 2 2 5 val28 3 3 4 val3 -{% endif -%} t1 ALL FULL JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 2 2 5 5 @@ -243,9 +238,7 @@ t1 ALL FULL JOIN t2 | bs = {{ block_size }} 2 2 5 5 2 2 5 5 3 3 4 4 -{% endif -%} t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 5 5 @@ -298,7 +291,6 @@ t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} 2 5 5 2 5 5 3 4 4 -{% endif -%} t1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -315,7 +307,6 @@ t1 ALL LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -326,7 +317,6 @@ t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 -{% endif -%} t1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -341,7 +331,6 @@ t1 ANY LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -352,9 +341,7 @@ t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 -{% endif -%} t1 ALL FULL JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 5 0 \N 0 5 0 \N 0 5 @@ -372,9 +359,7 @@ t1 ALL FULL JOIN tn2 | bs = {{ block_size }} 2 \N 5 0 2 \N 5 0 3 3 4 4 -{% endif -%} t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 5 0 @@ -392,7 +377,6 @@ t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} \N 0 5 \N 0 5 \N 0 5 -{% endif -%} tn1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -409,7 +393,6 @@ tn1 ALL LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -420,7 +403,6 @@ tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -{% endif -%} tn1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -435,7 +417,6 @@ tn1 ANY LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -446,9 +427,7 @@ tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -{% endif -%} tn1 ALL FULL JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -466,9 +445,7 @@ tn1 ALL FULL JOIN t2 | bs = {{ block_size }} \N 2 0 5 \N 2 0 5 \N 2 0 5 -{% endif -%} tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 0 5 @@ -486,7 +463,6 @@ tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} \N 5 0 \N 5 0 \N 5 0 -{% endif -%} tn1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -503,7 +479,6 @@ tn1 ALL LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -514,7 +489,6 @@ tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 -{% endif -%} tn1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -529,7 +503,6 @@ tn1 ANY LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -540,9 +513,7 @@ tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 -{% endif -%} tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -560,9 +531,7 @@ tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} \N \N 5 0 \N \N 5 0 \N \N 5 0 -{% endif -%} tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 3 4 4 @@ -580,6 +549,5 @@ tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} \N 5 0 \N 5 0 \N 5 0 -{% endif -%} {% endfor -%} {% endfor -%} From 5c44e6a562f23a388b9895088320286b1bf7b598 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sat, 6 May 2023 16:26:40 +0800 Subject: [PATCH 0490/2223] triger ci --- src/Interpreters/GraceHashJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index ee369d4b195..bdfd63602ec 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -501,7 +501,7 @@ public: non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); } } - // For right/full join, non_joined_blocks is not null. + // For right/full join, non_joined_blocks is not null // For inner/left join, non_joined_blocks is null, and return a empty block directly. std::lock_guard lock(mutex); if (non_joined_blocks) From 89dd538beac9f1b690cf772764939318fc6cc47a Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 8 May 2023 09:29:29 +0800 Subject: [PATCH 0491/2223] update --- src/Interpreters/GraceHashJoin.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index bdfd63602ec..7d924bb3a02 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -629,6 +629,16 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; + if (hash_join) + { + auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); + for (auto & block : right_blocks) + { + Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, block, buckets.size()); + flushBlocksToBuckets(blocks, buckets, bucket_idx); + } + } + hash_join = makeInMemoryJoin(); for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) From 8efec9bccab4097574985030fa5aa112bc357242 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 8 May 2023 09:59:56 +0800 Subject: [PATCH 0492/2223] add locks for getNonJoinedBlocks --- src/Interpreters/GraceHashJoin.cpp | 13 ++++++++++++- src/Interpreters/GraceHashJoin.h | 2 ++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 7d924bb3a02..0e839e538fd 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -524,7 +525,17 @@ private: IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - return std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); + std::lock_guard lock(hash_join_mutex); + if (!non_joined_block_stream) + { + if (last_hash_join_for_non_joined && last_hash_join_for_non_joined != hash_join) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "getNonJoinedBlocks should be called once"); + } + non_joined_block_stream = std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); + last_hash_join_for_non_joined = hash_join; + } + return non_joined_block_stream; } class GraceHashJoin::DelayedBlocks : public IBlocksStream diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index bc5f894bc8c..0ff3bc3f609 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -145,6 +145,8 @@ private: InMemoryJoinPtr hash_join; Block hash_join_sample_block; mutable std::mutex hash_join_mutex; + mutable IBlocksStreamPtr non_joined_block_stream = nullptr; + mutable InMemoryJoinPtr last_hash_join_for_non_joined = nullptr; }; } From d5efc0e6884b39ac77f31e1acff6a8eb370261ce Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 8 May 2023 11:35:48 +0800 Subject: [PATCH 0493/2223] update --- src/Interpreters/GraceHashJoin.cpp | 15 +++------------ src/Interpreters/GraceHashJoin.h | 2 -- 2 files changed, 3 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 0e839e538fd..06a4cfaa76a 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -522,20 +522,11 @@ private: // This is only be called for bucket[0] at present. other buckets' non-joined blocks are generated in // DelayedBlocks. +// There is a finished counter in JoiningTransform, only the last JoiningTransform could call this function. IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - std::lock_guard lock(hash_join_mutex); - if (!non_joined_block_stream) - { - if (last_hash_join_for_non_joined && last_hash_join_for_non_joined != hash_join) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "getNonJoinedBlocks should be called once"); - } - non_joined_block_stream = std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); - last_hash_join_for_non_joined = hash_join; - } - return non_joined_block_stream; + return std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream @@ -582,7 +573,7 @@ public: is_left_reader_finished = true; } } - if (is_left_reader_finished || !block) + if (is_left_reader_finished && !block) { return non_joined_blocks_iter.next(); } diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index 0ff3bc3f609..bc5f894bc8c 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -145,8 +145,6 @@ private: InMemoryJoinPtr hash_join; Block hash_join_sample_block; mutable std::mutex hash_join_mutex; - mutable IBlocksStreamPtr non_joined_block_stream = nullptr; - mutable InMemoryJoinPtr last_hash_join_for_non_joined = nullptr; }; } From 80af345ea6b8fa3e10ce84578fca2aaf76808ee1 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 8 May 2023 14:09:41 +0800 Subject: [PATCH 0494/2223] update --- src/Interpreters/GraceHashJoin.cpp | 3 ++- .../0_stateless/01721_join_implicit_cast_long.reference.j2 | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 06a4cfaa76a..55ad0971a8f 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -571,9 +571,10 @@ public: if (!block) { is_left_reader_finished = true; + return non_joined_blocks_iter.next(); } } - if (is_left_reader_finished && !block) + else { return non_joined_blocks_iter.next(); } diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 index d40fc7ec864..ae43aa7195c 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 @@ -200,6 +200,7 @@ 1 1 1 +{% if join_algorithm not in ['full_sorting_merge'] -%} === join use nulls === = full = -4 \N 196 @@ -401,4 +402,5 @@ 1 1 1 +{% endif -%} {% endfor -%} From 29ade233979aab464dcc4d8141b2a51166a7bcb5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 8 May 2023 15:18:32 +0800 Subject: [PATCH 0495/2223] fixed: return invalid mismatch rows on full/right join --- src/Interpreters/GraceHashJoin.cpp | 80 ++++++++++-------------------- src/Interpreters/GraceHashJoin.h | 1 - 2 files changed, 25 insertions(+), 56 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 55ad0971a8f..2e622a4c18e 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -13,7 +12,6 @@ #include #include #include -#include "GraceHashJoin.h" #include #include @@ -474,59 +472,13 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -// Add lock for reading non-joined blocks, it's thread safe. -class GraceHashJoin::NonJoinedBlocksStream : public IBlocksStream -{ -public: - explicit NonJoinedBlocksStream( - InMemoryJoinPtr hash_join_, - const Block & left_sample_block_, - const Block & result_sample_block_, - size_t max_block_size_) - : hash_join(hash_join_) - , left_sample_block(left_sample_block_) - , result_sample_block(result_sample_block_) - , max_block_size(max_block_size_) - { - } - - Block nextImpl() override - { - // initialize non_joined_blocks lazily. - if (!has_init_non_joined_blocks) [[unlikely]] - { - std::lock_guard lock(mutex); - if (!has_init_non_joined_blocks) - { - has_init_non_joined_blocks = true; - non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); - } - } - // For right/full join, non_joined_blocks is not null - // For inner/left join, non_joined_blocks is null, and return a empty block directly. - std::lock_guard lock(mutex); - if (non_joined_blocks) - return non_joined_blocks->next(); - return {}; - } - -private: - InMemoryJoinPtr hash_join; - Block left_sample_block; - Block result_sample_block; - size_t max_block_size; - bool has_init_non_joined_blocks = false; - IBlocksStreamPtr non_joined_blocks = nullptr; - std::mutex mutex; -}; - // This is only be called for bucket[0] at present. other buckets' non-joined blocks are generated in // DelayedBlocks. // There is a finished counter in JoiningTransform, only the last JoiningTransform could call this function. IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - return std::make_shared(hash_join, left_sample_block_, result_sample_block_, max_block_size_); + return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream @@ -547,7 +499,9 @@ public: , left_reader(buckets[current_bucket]->getLeftTableReader()) , left_key_names(left_key_names_) , right_key_names(right_key_names_) - , non_joined_blocks_iter(hash_join, left_sample_block_, result_sample_block_, max_block_size_) + , left_sample_block(left_sample_block_) + , result_sample_block(result_sample_block_) + , max_block_size(max_block_size_) { } @@ -557,6 +511,12 @@ public: // changed inside. Block nextImpl() override { + // there is data race case wihthout this lock: + // 1. thread 1 read the last block from left_reader, but not finish the join + // 2. thread 2 try to read from non-joined blocks. Since thread 1 has not finished, + // the used flags in the hash_join is incomplete, then thread 2 return invalid mismatch rows. + std::lock_guard lock(non_joined_blocks_mutex); + Block block; size_t num_buckets = buckets.size(); size_t current_idx = buckets[current_bucket]->idx; @@ -571,12 +531,18 @@ public: if (!block) { is_left_reader_finished = true; - return non_joined_blocks_iter.next(); + non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); } } - else + + if (is_left_reader_finished) { - return non_joined_blocks_iter.next(); + if (non_joined_blocks) + { + return non_joined_blocks->next(); + } + else + return {}; } // block comes from left_reader, need to join with right table to get the result. @@ -619,8 +585,12 @@ public: Names left_key_names; Names right_key_names; - std::atomic is_left_reader_finished = false; - NonJoinedBlocksStream non_joined_blocks_iter; + Block left_sample_block; + Block result_sample_block; + size_t max_block_size = 0; + bool is_left_reader_finished = false; + IBlocksStreamPtr non_joined_blocks = nullptr; + std::mutex non_joined_blocks_mutex; }; IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index bc5f894bc8c..eb39ee09208 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -43,7 +43,6 @@ class HashJoin; class GraceHashJoin final : public IJoin { class FileBucket; - class NonJoinedBlocksStream; class DelayedBlocks; using InMemoryJoin = HashJoin; From 603c024eb0615392277297273d62e9b790237416 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 9 May 2023 11:17:09 +0800 Subject: [PATCH 0496/2223] ensure only the last processor could access non-joined blocks --- src/Interpreters/GraceHashJoin.cpp | 67 +++--------------- .../Transforms/JoiningTransform.cpp | 69 +++++++++++++++++-- src/Processors/Transforms/JoiningTransform.h | 24 ++++++- src/QueryPipeline/QueryPipelineBuilder.cpp | 2 +- ...274_full_sort_join_nodistinct.reference.j2 | 20 ++++++ 5 files changed, 115 insertions(+), 67 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 2e622a4c18e..ad4457f924e 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -472,9 +472,9 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -// This is only be called for bucket[0] at present. other buckets' non-joined blocks are generated in -// DelayedBlocks. -// There is a finished counter in JoiningTransform, only the last JoiningTransform could call this function. +// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform, +// only the last processor could call this function to ensure all used flags have been inserted. +// To support delayed stream mode, need to keep the hash join before next getDelayedBlocks call. IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { @@ -489,60 +489,30 @@ public: Buckets buckets_, InMemoryJoinPtr hash_join_, const Names & left_key_names_, - const Names & right_key_names_, - const Block & left_sample_block_, - const Block & result_sample_block_, - size_t max_block_size_) + const Names & right_key_names_) : current_bucket(current_bucket_) , buckets(std::move(buckets_)) , hash_join(std::move(hash_join_)) , left_reader(buckets[current_bucket]->getLeftTableReader()) , left_key_names(left_key_names_) , right_key_names(right_key_names_) - , left_sample_block(left_sample_block_) - , result_sample_block(result_sample_block_) - , max_block_size(max_block_size_) { } - // One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransforms, need locks for - // - reading from left_reader. left_reader.read() has had a lock inside. - // - reading non-joined blocks from hash_join. Since iterate on non-joined blocks will has state - // changed inside. Block nextImpl() override { - // there is data race case wihthout this lock: - // 1. thread 1 read the last block from left_reader, but not finish the join - // 2. thread 2 try to read from non-joined blocks. Since thread 1 has not finished, - // the used flags in the hash_join is incomplete, then thread 2 return invalid mismatch rows. - std::lock_guard lock(non_joined_blocks_mutex); - Block block; size_t num_buckets = buckets.size(); size_t current_idx = buckets[current_bucket]->idx; do { - // When left reader finish, return non-joined blocks. - // empty block means the end of this stream. - if (!is_left_reader_finished) + // One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransform. + // There is a lock inside left_reader.read(). + block = left_reader.read(); + if (!block) { - block = left_reader.read(); - if (!block) - { - is_left_reader_finished = true; - non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); - } - } - - if (is_left_reader_finished) - { - if (non_joined_blocks) - { - return non_joined_blocks->next(); - } - else - return {}; + return {}; } // block comes from left_reader, need to join with right table to get the result. @@ -585,12 +555,6 @@ public: Names left_key_names; Names right_key_names; - Block left_sample_block; - Block result_sample_block; - size_t max_block_size = 0; - bool is_left_reader_finished = false; - IBlocksStreamPtr non_joined_blocks = nullptr; - std::mutex non_joined_blocks_mutex; }; IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() @@ -636,18 +600,7 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() LOG_TRACE(log, "Loaded bucket {} with {}(/{}) rows", bucket_idx, hash_join->getTotalRowCount(), num_rows); - auto result_sample_block = left_sample_block; - ExtraBlockPtr tmp; - hash_join->joinBlock(result_sample_block, tmp); - return std::make_unique( - current_bucket->idx, - buckets, - hash_join, - left_key_names, - right_key_names, - left_sample_block, - result_sample_block, - max_block_size); + return std::make_unique(current_bucket->idx, buckets, hash_join, left_key_names, right_key_names); } LOG_TRACE(log, "Finished loading all {} buckets", buckets.size()); diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index b638a5582c5..d80d6b8e387 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -311,8 +311,16 @@ void FillingRightJoinSideTransform::work() } -DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform(Block output_header) - : IProcessor(InputPorts{Block()}, OutputPorts{output_header}) +DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform( + Block left_header_, + Block output_header_, + size_t max_block_size_, + JoinPtr join_) + : IProcessor(InputPorts{Block()}, OutputPorts{output_header_}) + , left_header(left_header_) + , output_header(output_header_) + , max_block_size(max_block_size_) + , join(join_) { } @@ -366,6 +374,7 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (!data.chunk.hasChunkInfo()) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); + } else { @@ -386,12 +395,24 @@ void DelayedJoinedBlocksWorkerTransform::work() { if (!task) return; + Block block; + if (!left_delayed_stream_finished) + { + block = task->delayed_blocks->next(); - Block block = task->delayed_blocks->next(); - + if (!block) + { + left_delayed_stream_finished = true; + block = nextNonJoinedBlock(); + } + } + else + { + block = nextNonJoinedBlock(); + } if (!block) { - task.reset(); + resetTask(); return; } @@ -400,6 +421,38 @@ void DelayedJoinedBlocksWorkerTransform::work() output_chunk.setColumns(block.getColumns(), rows); } +void DelayedJoinedBlocksWorkerTransform::resetTask() +{ + task.reset(); + left_delayed_stream_finished = false; + setup_non_joined_stream = false; + non_joined_delay_stream = nullptr; +} + +Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() +{ + if (!setup_non_joined_stream) + { + setup_non_joined_stream = true; + // Before read from non-joined stream, all blocks in left file reader must have been joined. + // For example, in HashJoin, it may return invalid mismatch rows from non-joined stream before + // the all blocks in left file reader have been finished, since the used flags are incomplete. + // To make only one processor could read from non-joined stream seems be a easy way. + if (task && task->left_delayed_stream_finish_counter->isLast()) + { + if (!non_joined_delay_stream) + { + non_joined_delay_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); + } + } + } + if (non_joined_delay_stream) + { + return non_joined_delay_stream->next(); + } + return {}; +} + DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, JoinPtr join_) : IProcessor(InputPorts{}, OutputPorts(num_streams, Block())) , join(std::move(join_)) @@ -451,10 +504,14 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (delayed_blocks) { + // This counter is used to ensure that only the last DelayedJoinedBlocksWorkerTransform + // could read non-joined blocks from the join. + auto left_delayed_stream_finished_counter = std::make_shared(outputs.size()); for (auto & output : outputs) { Chunk chunk; - chunk.setChunkInfo(std::make_shared(delayed_blocks)); + auto task = std::make_shared(delayed_blocks, left_delayed_stream_finished_counter); + chunk.setChunkInfo(task); output.push(std::move(chunk)); } delayed_blocks = nullptr; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index e7edff40c56..e9bd79bf623 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -116,9 +116,14 @@ class DelayedBlocksTask : public ChunkInfo public: explicit DelayedBlocksTask() : finished(true) {} - explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_) : delayed_blocks(std::move(delayed_blocks_)) {} + explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) + : delayed_blocks(std::move(delayed_blocks_)) + , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) + { + } IBlocksStreamPtr delayed_blocks = nullptr; + JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter = nullptr; bool finished = false; }; @@ -147,7 +152,11 @@ private: class DelayedJoinedBlocksWorkerTransform : public IProcessor { public: - explicit DelayedJoinedBlocksWorkerTransform(Block output_header); + explicit DelayedJoinedBlocksWorkerTransform( + Block left_header_, + Block output_header_, + size_t max_block_size_, + JoinPtr join_); String getName() const override { return "DelayedJoinedBlocksWorkerTransform"; } @@ -155,10 +164,19 @@ public: void work() override; private: + Block left_header; + Block output_header; + size_t max_block_size; + JoinPtr join; DelayedBlocksTaskPtr task; Chunk output_chunk; - bool finished = false; + bool left_delayed_stream_finished = false; + bool setup_non_joined_stream = false; + IBlocksStreamPtr non_joined_delay_stream = nullptr; + + void resetTask(); + Block nextNonJoinedBlock(); }; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index a4edf107b2f..764997e7b7e 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -491,7 +491,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (delayed_root) { // Process delayed joined blocks when all JoiningTransform are finished. - auto delayed = std::make_shared(joined_header); + auto delayed = std::make_shared(left_header, joined_header, max_block_size, join); if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output"); diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 6b5b5bfd298..2dc7fe33423 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -360,6 +360,7 @@ t1 ALL FULL JOIN tn2 | bs = {{ block_size }} 2 \N 5 0 3 3 4 4 t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} +{% if join_algorithm == 'full_sorting_merge' -%} 1 4 5 1 4 5 2 5 0 @@ -377,6 +378,25 @@ t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} \N 0 5 \N 0 5 \N 0 5 +{% else -%} +0 0 5 +0 0 5 +0 0 5 +0 0 5 +0 0 5 +0 0 5 +0 0 5 +1 4 5 +1 4 5 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +2 5 0 +3 4 4 +{% endif -%} tn1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 From 258239927109d808276f475d02c37a245c84f944 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 9 May 2023 17:42:07 +0800 Subject: [PATCH 0497/2223] fixed: a in-memory bucket contains rows of other buckets --- src/Interpreters/GraceHashJoin.cpp | 22 ++++++++++--------- .../Transforms/JoiningTransform.cpp | 11 +++++----- src/Processors/Transforms/JoiningTransform.h | 2 +- 3 files changed, 18 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index ad4457f924e..129d58a5436 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -566,16 +566,6 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; - if (hash_join) - { - auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); - for (auto & block : right_blocks) - { - Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, block, buckets.size()); - flushBlocksToBuckets(blocks, buckets, bucket_idx); - } - } - hash_join = makeInMemoryJoin(); for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) @@ -639,6 +629,18 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) if (!hash_join) hash_join = makeInMemoryJoin(); + // buckets size has been changed in other threads. Need to scatter current_block again. + // rehash could only happen under hash_join_mutex's scope. + auto current_buckets = getCurrentBuckets(); + if (buckets_snapshot.size() != current_buckets.size()) + { + LOG_TRACE(log, "mismatch buckets size. previous:{}, current:{}", buckets_snapshot.size(), getCurrentBuckets().size()); + Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, current_block, current_buckets.size()); + flushBlocksToBuckets(blocks, current_buckets, bucket_index); + current_block = std::move(blocks[bucket_index]); + if (!current_block.rows()) + return; + } hash_join->addJoinedBlock(current_block, /* check_limits = */ false); if (!hasMemoryOverflow(hash_join)) diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index d80d6b8e387..d4ea601209f 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -189,7 +189,6 @@ void JoiningTransform::transform(Chunk & chunk) } else block = readExecute(chunk); - auto num_rows = block.rows(); chunk.setColumns(block.getColumns(), num_rows); } @@ -426,7 +425,7 @@ void DelayedJoinedBlocksWorkerTransform::resetTask() task.reset(); left_delayed_stream_finished = false; setup_non_joined_stream = false; - non_joined_delay_stream = nullptr; + non_joined_delayed_stream = nullptr; } Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() @@ -440,15 +439,15 @@ Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() // To make only one processor could read from non-joined stream seems be a easy way. if (task && task->left_delayed_stream_finish_counter->isLast()) { - if (!non_joined_delay_stream) + if (!non_joined_delayed_stream) { - non_joined_delay_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); + non_joined_delayed_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); } } } - if (non_joined_delay_stream) + if (non_joined_delayed_stream) { - return non_joined_delay_stream->next(); + return non_joined_delayed_stream->next(); } return {}; } diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index e9bd79bf623..17222f0e514 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -173,7 +173,7 @@ private: bool left_delayed_stream_finished = false; bool setup_non_joined_stream = false; - IBlocksStreamPtr non_joined_delay_stream = nullptr; + IBlocksStreamPtr non_joined_delayed_stream = nullptr; void resetTask(); Block nextNonJoinedBlock(); From 983514f9461862b6d999455863f3178dd09ab2a1 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 10 May 2023 16:00:41 +0800 Subject: [PATCH 0498/2223] update tests for using clause --- ...274_full_sort_join_nodistinct.reference.j2 | 22 +++---------------- .../02274_full_sort_join_nodistinct.sql.j2 | 3 ++- 2 files changed, 5 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 2dc7fe33423..2b8ca580f50 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -359,8 +359,8 @@ t1 ALL FULL JOIN tn2 | bs = {{ block_size }} 2 \N 5 0 2 \N 5 0 3 3 4 4 +{% if join_algorithm != 'grace_hash' -%} t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} -{% if join_algorithm == 'full_sorting_merge' -%} 1 4 5 1 4 5 2 5 0 @@ -378,24 +378,6 @@ t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} \N 0 5 \N 0 5 \N 0 5 -{% else -%} -0 0 5 -0 0 5 -0 0 5 -0 0 5 -0 0 5 -0 0 5 -0 0 5 -1 4 5 -1 4 5 -2 5 0 -2 5 0 -2 5 0 -2 5 0 -2 5 0 -2 5 0 -2 5 0 -3 4 4 {% endif -%} tn1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 @@ -551,6 +533,7 @@ tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} \N \N 5 0 \N \N 5 0 \N \N 5 0 +{% if join_algorithm != 'grace_hash' -%} tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} 1 4 5 1 4 5 @@ -569,5 +552,6 @@ tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} \N 5 0 \N 5 0 \N 5 0 +{% endif -%} {% endfor -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index 6bf634ca1f5..a1f7b133cf6 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -46,9 +46,10 @@ SELECT t1.key, t2.key, length(t1.s), t2.s FROM {{ t1 }} AS t1 {{ kind }} RIGHT J SELECT '{{ t1 }} ALL FULL JOIN {{ t2 }} | bs = {{ block_size }}'; SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} FULL JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} +{% if join_algorithm == 'full_sorting_merge' or t2 != 'tn2' -%} SELECT '{{ t1 }} ALL FULL JOIN USING {{ t2 }} | bs = {{ block_size }}'; SELECT key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 ALL FULL JOIN {{ t2 }} AS t2 USING (key) ORDER BY key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} - +{% endif -%} {% endfor -%} {% endfor -%} SET max_bytes_in_join = 0; From 35d83887057e245dc038e918913be45fa3ca66ae Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 12 May 2023 09:38:21 +0800 Subject: [PATCH 0499/2223] update --- src/Interpreters/GraceHashJoin.cpp | 13 +++++++++---- src/Processors/Transforms/JoiningTransform.cpp | 2 +- src/Processors/Transforms/JoiningTransform.h | 1 + 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 129d58a5436..4a4c69ff473 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -471,10 +471,15 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } - -// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform, -// only the last processor could call this function to ensure all used flags have been inserted. -// To support delayed stream mode, need to keep the hash join before next getDelayedBlocks call. +/// Each bucket are handled by the following steps +/// 1. build hash_join by the right side blocks. +/// 2. join left side with the hash_join, +/// 3. read right non-joined blocks from hash_join. +/// buckets are handled one by one, each hash_join will not be release before the right non-joined blocks are emitted. +/// +/// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform, +/// only one processor could take the non-joined blocks from right stream, and ensure all rows from +/// left stream have been emitted before this. IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index d4ea601209f..256ef66a27d 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -504,7 +504,7 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (delayed_blocks) { // This counter is used to ensure that only the last DelayedJoinedBlocksWorkerTransform - // could read non-joined blocks from the join. + // could read right non-joined blocks from the join. auto left_delayed_stream_finished_counter = std::make_shared(outputs.size()); for (auto & output : outputs) { diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 17222f0e514..3577906b26a 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -171,6 +171,7 @@ private: DelayedBlocksTaskPtr task; Chunk output_chunk; + /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left bool left_delayed_stream_finished = false; bool setup_non_joined_stream = false; IBlocksStreamPtr non_joined_delayed_stream = nullptr; From cc93f9ab52669dda67dce644241c4f7e88b94216 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 17 May 2023 09:06:33 +0800 Subject: [PATCH 0500/2223] update tests for flaky test --- docs/en/operations/settings/settings.md | 2 ++ tests/queries/0_stateless/02273_full_sort_join.reference.j2 | 2 +- tests/queries/0_stateless/02273_full_sort_join.sql.j2 | 2 +- .../0_stateless/02274_full_sort_join_nodistinct.reference.j2 | 2 +- .../queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 | 2 +- 5 files changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 791186dd9c1..2239084a429 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -452,6 +452,8 @@ Possible values: The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which don’t belong to the current bucket are flushed and reassigned. + Supports `INNER/LEFT/RIGHT/FULL ALL/ANY JOIN`. + - hash [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 index 7b994c2d11f..0af4158e971 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 @@ -1,7 +1,7 @@ {% set table_size = 15 -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} -- {{ join_algorithm }} -- -{% for block_size in range(1, table_size + 1) -%} +{% for block_size in range(1, table_size + 1, 4) -%} ALL INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index b371d9544da..6500306356c 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -36,7 +36,7 @@ SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}10K{% else %}0{% SELECT '-- {{ join_algorithm }} --'; SET join_algorithm = '{{ join_algorithm }}'; -{% for block_size in range(1, table_size + 1) -%} +{% for block_size in range(1, table_size + 1, 4) -%} {% for kind in ['ALL', 'ANY'] -%} SET max_block_size = {{ block_size }}; diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 2b8ca580f50..df968e86e8d 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -1,6 +1,6 @@ {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} --- {{ join_algorithm }} --- -{% for block_size in range(1, 11) -%} +{% for block_size in range(1, 11, 4) -%} t1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index a1f7b133cf6..f8eb4b1a53e 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -26,7 +26,7 @@ SET join_algorithm = '{{ join_algorithm }}'; SELECT '--- {{ join_algorithm }} ---'; -{% for block_size in range(1, 11) -%} +{% for block_size in range(1, 11, 4) -%} SET max_block_size = {{ block_size }}; {% for t1, t2 in [('t1', 't2'), ('t1', 'tn2'), ('tn1', 't2'), ('tn1', 'tn2')] -%} From c30658a9ed4dd8f144e9da3b5696c7921f5aa5cf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 22 May 2023 08:57:03 +0200 Subject: [PATCH 0501/2223] Fix hashed/sparse_hashed dictionaries max_load_factor upper range Previously due to comparison of floats with doubles, it was incorrectly works for the upper range: (lldb) p (float)0.99 > (float)0.99 (bool) $0 = false (lldb) p (float)0.99 > (double)0.99 (bool) $1 = true This should also fix performance tests errors on CI: clickhouse_driver.errors.ServerException: Code: 36. DB::Exception: default.simple_key_HASHED_dictionary_l0_99: max_load_factor parameter should be within [0.5, 0.99], got 0.99. Stack trace: Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 83d051df5e9..b3d8ca8f5ce 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -1163,7 +1163,7 @@ void registerDictionaryHashed(DictionaryFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS,"{}: SHARD_LOAD_QUEUE_BACKLOG parameter should be greater then zero", full_name); float max_load_factor = static_cast(config.getDouble(config_prefix + dictionary_layout_prefix + ".max_load_factor", 0.5)); - if (max_load_factor < 0.5 || max_load_factor > 0.99) + if (max_load_factor < 0.5f || max_load_factor > 0.99f) throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: max_load_factor parameter should be within [0.5, 0.99], got {}", full_name, max_load_factor); HashedDictionaryConfiguration configuration{ From 855f0abcf9dbda9d8483c712bf17100f913a8aae Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 22 May 2023 10:20:45 +0200 Subject: [PATCH 0502/2223] Removed unneeded try-except --- .../test_ssl_cert_authentication/test.py | 25 ++++++++----------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index c601bcb1db8..e8597dc98ea 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -164,20 +164,17 @@ def get_ssl_context(cert_name): def execute_query_https( query, user, enable_ssl_auth=True, cert_name=None, password=None ): - try: - url = f"https://{instance.ip_address}:{HTTPS_PORT}/?query={query}" - headers = {"X-ClickHouse-User": user} - if enable_ssl_auth: - headers["X-ClickHouse-SSL-Certificate-Auth"] = "on" - if password: - headers["X-ClickHouse-Key"] = password - http_client = urllib3.PoolManager(ssl_context=get_ssl_context(cert_name)) - response = http_client.request("GET", url, headers=headers) - if response.status != 200: - raise Exception(response.status) - return response.data.decode("utf-8") - except: - raise + url = f"https://{instance.ip_address}:{HTTPS_PORT}/?query={query}" + headers = {"X-ClickHouse-User":user} + if enable_ssl_auth: + headers["X-ClickHouse-SSL-Certificate-Auth"] = "on" + if password: + headers["X-ClickHouse-Key"] = password + http_client=urllib3.PoolManager(ssl_context=get_ssl_context(cert_name)) + response = http_client.request('GET', url, headers=headers) + if response.status != 200: + raise Exception(response.status) + return response.data.decode("utf-8") def test_https(): From a7b641134a3fabc5e14f8ace6a6fe444d52370db Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Mon, 22 May 2023 08:28:11 +0000 Subject: [PATCH 0503/2223] Follow the reviewer's recommendations regarding the documentation --- .../functions/date-time-functions.md | 18 ++++++++++-------- .../functions/date-time-functions.md | 18 ++++++++++-------- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 342f64620b7..668864c49af 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -357,9 +357,9 @@ Alias: `SECOND`. ## toUnixTimestamp -Converts a string, date or date with time to a `UInt32` number -- [Unix Timestamp](https://en.wikipedia.org/wiki/Unix_time). +Converts a string, a date or a date with time to the [Unix Timestamp](https://en.wikipedia.org/wiki/Unix_time) in `UInt32` representation. -The string argument is converted according to the timezone (optional second argument, server timezone is used by default). +If the function is called with a string, it accepts an optional timezone argument. **Syntax** @@ -382,6 +382,7 @@ SELECT toUnixTimestamp(dt_str) AS from_str, toUnixTimestamp(dt_str, 'Asia/Tokyo') AS from_str_tokyo, toUnixTimestamp(toDateTime(dt_str)) AS from_datetime, + toUnixTimestamp(toDateTime64(dt_str, 0)) AS from_datetime64, toUnixTimestamp(toDate(dt_str)) AS from_date, toUnixTimestamp(toDate32(dt_str)) AS from_date32 FORMAT Vertical; @@ -392,12 +393,13 @@ Result: ``` text Row 1: ────── -dt_str: 2017-11-05 08:07:47 -from_str: 1509869267 -from_str_tokyo: 1509836867 -from_datetime: 1509869267 -from_date: 1509840000 -from_date32: 1509840000 +dt_str: 2017-11-05 08:07:47 +from_str: 1509869267 +from_str_tokyo: 1509836867 +from_datetime: 1509869267 +from_datetime64: 1509869267 +from_date: 1509840000 +from_date32: 1509840000 ``` :::note diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index f0a8109786f..5111441c0b9 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -235,8 +235,8 @@ SELECT toDateTime('2021-04-21 10:20:30', 'Europe/Moscow') AS Time, toTypeName(Ti ## toUnixTimestamp {#to-unix-timestamp} -Переводит строку, дату или дату-с-временем в число типа `UInt32` -- [Unix Timestamp](https://en.wikipedia.org/wiki/Unix_time). -Строковый аргумент конвертируется в дату и время в соответствии с часовым поясом (необязательный второй аргумент, часовой пояс сервера используется по умолчанию). +Переводит строку, дату или дату-с-временем в [Unix Timestamp](https://en.wikipedia.org/wiki/Unix_time), имеющий тип `UInt32`. +Строка может сопровождаться вторым (необязательным) аргументом, указывающим часовой пояс. **Синтаксис** @@ -261,6 +261,7 @@ SELECT toUnixTimestamp(dt_str) AS from_str, toUnixTimestamp(dt_str, 'Asia/Tokyo') AS from_str_tokyo, toUnixTimestamp(toDateTime(dt_str)) AS from_datetime, + toUnixTimestamp(toDateTime64(dt_str, 0)) AS from_datetime64, toUnixTimestamp(toDate(dt_str)) AS from_date, toUnixTimestamp(toDate32(dt_str)) AS from_date32 FORMAT Vertical; @@ -271,12 +272,13 @@ FORMAT Vertical; ``` text Row 1: ────── -dt_str: 2017-11-05 08:07:47 -from_str: 1509869267 -from_str_tokyo: 1509836867 -from_datetime: 1509869267 -from_date: 1509840000 -from_date32: 1509840000 +dt_str: 2017-11-05 08:07:47 +from_str: 1509869267 +from_str_tokyo: 1509836867 +from_datetime: 1509869267 +from_datetime64: 1509869267 +from_date: 1509840000 +from_date32: 1509840000 ``` :::note From a0b384bc4885001fbb65119d3f0d4d15be218123 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 22 May 2023 08:30:04 +0000 Subject: [PATCH 0504/2223] Automatic style fix --- tests/integration/test_ssl_cert_authentication/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index e8597dc98ea..b05a6acc16b 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -165,13 +165,13 @@ def execute_query_https( query, user, enable_ssl_auth=True, cert_name=None, password=None ): url = f"https://{instance.ip_address}:{HTTPS_PORT}/?query={query}" - headers = {"X-ClickHouse-User":user} + headers = {"X-ClickHouse-User": user} if enable_ssl_auth: headers["X-ClickHouse-SSL-Certificate-Auth"] = "on" if password: headers["X-ClickHouse-Key"] = password - http_client=urllib3.PoolManager(ssl_context=get_ssl_context(cert_name)) - response = http_client.request('GET', url, headers=headers) + http_client = urllib3.PoolManager(ssl_context=get_ssl_context(cert_name)) + response = http_client.request("GET", url, headers=headers) if response.status != 200: raise Exception(response.status) return response.data.decode("utf-8") From a257ff6cf3ba1124e861452e4b3b52ada3ea2d5c Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 22 May 2023 10:41:22 +0200 Subject: [PATCH 0505/2223] address comment --- src/Common/OptimizedRegularExpression.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 92b3ad32ecd..f2fe922ef19 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -68,7 +68,7 @@ const char * analyzeImpl( bool has_alternative_on_depth_0 = false; bool has_case_insensitive_flag = false; - /// Substring with a position. + /// Substring with is_prefix. using Substring = std::pair; using Substrings = std::vector; From 183f90e45a7601e5ad4af63b24dabfc506a637ae Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 22 May 2023 02:02:09 +0000 Subject: [PATCH 0506/2223] Update MongoDB protocol --- .../Foundation/include/Poco/BinaryReader.h | 3 + .../Foundation/include/Poco/BinaryWriter.h | 5 + base/poco/Foundation/src/BinaryReader.cpp | 25 ++ base/poco/Foundation/src/BinaryWriter.cpp | 11 +- base/poco/MongoDB/CMakeLists.txt | 1 + .../poco/MongoDB/include/Poco/MongoDB/Array.h | 36 +- .../MongoDB/include/Poco/MongoDB/Binary.h | 2 +- .../MongoDB/include/Poco/MongoDB/Connection.h | 22 +- .../MongoDB/include/Poco/MongoDB/Cursor.h | 3 + .../MongoDB/include/Poco/MongoDB/Database.h | 83 +++- .../MongoDB/include/Poco/MongoDB/Document.h | 27 +- .../MongoDB/include/Poco/MongoDB/Element.h | 6 +- .../include/Poco/MongoDB/JavaScriptCode.h | 2 +- .../include/Poco/MongoDB/MessageHeader.h | 11 +- .../MongoDB/include/Poco/MongoDB/MongoDB.h | 12 + .../MongoDB/include/Poco/MongoDB/ObjectId.h | 2 +- .../include/Poco/MongoDB/OpMsgCursor.h | 96 ++++ .../include/Poco/MongoDB/OpMsgMessage.h | 163 +++++++ .../Poco/MongoDB/PoolableConnectionFactory.h | 16 + .../include/Poco/MongoDB/RegularExpression.h | 2 +- .../include/Poco/MongoDB/ResponseMessage.h | 3 + base/poco/MongoDB/src/Array.cpp | 4 +- base/poco/MongoDB/src/Connection.cpp | 26 ++ base/poco/MongoDB/src/Cursor.cpp | 6 + base/poco/MongoDB/src/Database.cpp | 48 +- base/poco/MongoDB/src/DeleteRequest.cpp | 4 +- base/poco/MongoDB/src/Document.cpp | 14 +- base/poco/MongoDB/src/Element.cpp | 2 +- base/poco/MongoDB/src/GetMoreRequest.cpp | 2 +- base/poco/MongoDB/src/InsertRequest.cpp | 2 +- base/poco/MongoDB/src/KillCursorsRequest.cpp | 2 +- base/poco/MongoDB/src/Message.cpp | 2 +- base/poco/MongoDB/src/MessageHeader.cpp | 12 +- base/poco/MongoDB/src/ObjectId.cpp | 2 +- base/poco/MongoDB/src/OpMsgCursor.cpp | 187 ++++++++ base/poco/MongoDB/src/OpMsgMessage.cpp | 412 ++++++++++++++++++ base/poco/MongoDB/src/QueryRequest.cpp | 6 +- base/poco/MongoDB/src/RegularExpression.cpp | 4 +- base/poco/MongoDB/src/ReplicaSet.cpp | 6 +- base/poco/MongoDB/src/RequestMessage.cpp | 4 +- base/poco/MongoDB/src/ResponseMessage.cpp | 20 +- base/poco/MongoDB/src/UpdateRequest.cpp | 2 +- .../runner/compose/docker_compose_mongo.yml | 2 +- .../compose/docker_compose_mongo_secure.yml | 2 +- src/Dictionaries/MongoDBDictionarySource.cpp | 15 +- src/Dictionaries/MongoDBDictionarySource.h | 1 - src/Processors/Sources/MongoDBSource.cpp | 76 +++- src/Processors/Sources/MongoDBSource.h | 32 +- src/Storages/StorageMongoDB.cpp | 34 +- .../integration/test_storage_mongodb/test.py | 42 +- 50 files changed, 1399 insertions(+), 103 deletions(-) create mode 100644 base/poco/MongoDB/include/Poco/MongoDB/OpMsgCursor.h create mode 100644 base/poco/MongoDB/include/Poco/MongoDB/OpMsgMessage.h create mode 100644 base/poco/MongoDB/src/OpMsgCursor.cpp create mode 100644 base/poco/MongoDB/src/OpMsgMessage.cpp diff --git a/base/poco/Foundation/include/Poco/BinaryReader.h b/base/poco/Foundation/include/Poco/BinaryReader.h index 4042b507a2f..2b9bca29944 100644 --- a/base/poco/Foundation/include/Poco/BinaryReader.h +++ b/base/poco/Foundation/include/Poco/BinaryReader.h @@ -117,6 +117,9 @@ public: void readRaw(char * buffer, std::streamsize length); /// Reads length bytes of raw data into buffer. + void readCString(std::string& value); + /// Reads zero-terminated C-string into value. + void readBOM(); /// Reads a byte-order mark from the stream and configures /// the reader for the encountered byte order. diff --git a/base/poco/Foundation/include/Poco/BinaryWriter.h b/base/poco/Foundation/include/Poco/BinaryWriter.h index aa280d4ccab..a35d76d84bc 100644 --- a/base/poco/Foundation/include/Poco/BinaryWriter.h +++ b/base/poco/Foundation/include/Poco/BinaryWriter.h @@ -56,6 +56,8 @@ public: LITTLE_ENDIAN_BYTE_ORDER = 3 /// little-endian byte-order }; + static const std::streamsize DEFAULT_MAX_CSTR_LENGTH { 1024 }; + BinaryWriter(std::ostream & ostr, StreamByteOrder byteOrder = NATIVE_BYTE_ORDER); /// Creates the BinaryWriter. @@ -131,6 +133,9 @@ public: void writeRaw(const char * buffer, std::streamsize length); /// Writes length raw bytes from the given buffer to the stream. + void writeCString(const char* cString, std::streamsize maxLength = DEFAULT_MAX_CSTR_LENGTH); + /// Writes zero-terminated C-string. + void writeBOM(); /// Writes a byte-order mark to the stream. A byte order mark is /// a 16-bit integer with a value of 0xFEFF, written in host byte-order. diff --git a/base/poco/Foundation/src/BinaryReader.cpp b/base/poco/Foundation/src/BinaryReader.cpp index f2961e03966..37ec2bc9040 100644 --- a/base/poco/Foundation/src/BinaryReader.cpp +++ b/base/poco/Foundation/src/BinaryReader.cpp @@ -274,6 +274,31 @@ void BinaryReader::readRaw(char* buffer, std::streamsize length) } +void BinaryReader::readCString(std::string& value) +{ + value.clear(); + if (!_istr.good()) + { + return; + } + value.reserve(256); + while (true) + { + char c; + _istr.get(c); + if (!_istr.good()) + { + break; + } + if (c == '\0') + { + break; + } + value += c; + } +} + + void BinaryReader::readBOM() { UInt16 bom; diff --git a/base/poco/Foundation/src/BinaryWriter.cpp b/base/poco/Foundation/src/BinaryWriter.cpp index 6db5ab7cb90..c3fcabc4374 100644 --- a/base/poco/Foundation/src/BinaryWriter.cpp +++ b/base/poco/Foundation/src/BinaryWriter.cpp @@ -271,7 +271,7 @@ BinaryWriter& BinaryWriter::operator << (const std::string& value) BinaryWriter& BinaryWriter::operator << (const char* value) { poco_check_ptr (value); - + if (_pTextConverter) { std::string converted; @@ -332,6 +332,15 @@ void BinaryWriter::writeRaw(const char* buffer, std::streamsize length) } +void BinaryWriter::writeCString(const char* cString, std::streamsize maxLength) +{ + const std::size_t len = ::strnlen(cString, maxLength); + writeRaw(cString, len); + static const char zero = '\0'; + _ostr.write(&zero, sizeof(zero)); +} + + void BinaryWriter::writeBOM() { UInt16 value = 0xFEFF; diff --git a/base/poco/MongoDB/CMakeLists.txt b/base/poco/MongoDB/CMakeLists.txt index 8f5c6be2cae..bb6f90ed8f5 100644 --- a/base/poco/MongoDB/CMakeLists.txt +++ b/base/poco/MongoDB/CMakeLists.txt @@ -13,3 +13,4 @@ target_compile_options (_poco_mongodb target_include_directories (_poco_mongodb SYSTEM PUBLIC "include") target_link_libraries (_poco_mongodb PUBLIC Poco::Net) + diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Array.h b/base/poco/MongoDB/include/Poco/MongoDB/Array.h index 4ed9cdd87ee..8a30c785b2d 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Array.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Array.h @@ -33,7 +33,7 @@ namespace MongoDB /// This class represents a BSON Array. { public: - typedef SharedPtr Ptr; + using Ptr = SharedPtr; Array(); /// Creates an empty Array. @@ -41,8 +41,31 @@ namespace MongoDB virtual ~Array(); /// Destroys the Array. + // Document template functions available for backward compatibility + using Document::add; + using Document::get; + template - T get(int pos) const + Document & add(T value) + /// Creates an element with the name from the current pos and value and + /// adds it to the array document. + /// + /// The active document is returned to allow chaining of the add methods. + { + return Document::add(Poco::NumberFormatter::format(size()), value); + } + + Document & add(const char * value) + /// Creates an element with a name from the current pos and value and + /// adds it to the array document. + /// + /// The active document is returned to allow chaining of the add methods. + { + return Document::add(Poco::NumberFormatter::format(size()), value); + } + + template + T get(std::size_t pos) const /// Returns the element at the given index and tries to convert /// it to the template type. If the element is not found, a /// Poco::NotFoundException will be thrown. If the element cannot be @@ -52,7 +75,7 @@ namespace MongoDB } template - T get(int pos, const T & deflt) const + T get(std::size_t pos, const T & deflt) const /// Returns the element at the given index and tries to convert /// it to the template type. If the element is not found, or /// has the wrong type, the deflt argument will be returned. @@ -60,12 +83,12 @@ namespace MongoDB return Document::get(Poco::NumberFormatter::format(pos), deflt); } - Element::Ptr get(int pos) const; + Element::Ptr get(std::size_t pos) const; /// Returns the element at the given index. /// An empty element will be returned if the element is not found. template - bool isType(int pos) const + bool isType(std::size_t pos) const /// Returns true if the type of the element equals the TypeId of ElementTrait, /// otherwise false. { @@ -74,6 +97,9 @@ namespace MongoDB std::string toString(int indent = 0) const; /// Returns a string representation of the Array. + + private: + friend void BSONReader::read(Array::Ptr & to); }; diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Binary.h b/base/poco/MongoDB/include/Poco/MongoDB/Binary.h index 1005cb000f5..aad8736e8b6 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Binary.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Binary.h @@ -40,7 +40,7 @@ namespace MongoDB /// A Binary stores its data in a Poco::Buffer. { public: - typedef SharedPtr Ptr; + using Ptr = SharedPtr; Binary(); /// Creates an empty Binary with subtype 0. diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h index dcb813b75bc..cf679d530aa 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h @@ -18,6 +18,7 @@ #define MongoDB_Connection_INCLUDED +#include "Poco/MongoDB/OpMsgMessage.h" #include "Poco/MongoDB/RequestMessage.h" #include "Poco/MongoDB/ResponseMessage.h" #include "Poco/Mutex.h" @@ -39,7 +40,7 @@ namespace MongoDB /// for more information on the wire protocol. { public: - typedef Poco::SharedPtr Ptr; + using Ptr = Poco::SharedPtr; class MongoDB_API SocketFactory { @@ -90,7 +91,7 @@ namespace MongoDB Poco::Net::SocketAddress address() const; /// Returns the address of the MongoDB server. - + const std::string & uri() const; /// Returns the uri on which the connection was made. @@ -145,6 +146,21 @@ namespace MongoDB /// Use this when a response is expected: only a "query" or "getmore" /// request will return a response. + void sendRequest(OpMsgMessage & request, OpMsgMessage & response); + /// Sends a request to the MongoDB server and receives the response + /// using newer wire protocol with OP_MSG. + + void sendRequest(OpMsgMessage & request); + /// Sends an unacknowledged request to the MongoDB server using newer + /// wire protocol with OP_MSG. + /// No response is sent by the server. + + void readResponse(OpMsgMessage & response); + /// Reads additional response data when previous message's flag moreToCome + /// indicates that server will send more data. + /// NOTE: See comments in OpMsgCursor code. + + protected: void connect(); @@ -164,7 +180,7 @@ namespace MongoDB } inline const std::string & Connection::uri() const { - return _uri; + return _uri; } diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Cursor.h b/base/poco/MongoDB/include/Poco/MongoDB/Cursor.h index 4aed9fe64fb..8849d737a62 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Cursor.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Cursor.h @@ -40,6 +40,9 @@ namespace MongoDB Cursor(const std::string & fullCollectionName, QueryRequest::Flags flags = QueryRequest::QUERY_DEFAULT); /// Creates a Cursor for the given database and collection ("database.collection"), using the specified flags. + Cursor(const Document & aggregationResponse); + /// Creates a Cursor for the given aggregation query response. + virtual ~Cursor(); /// Destroys the Cursor. diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Database.h b/base/poco/MongoDB/include/Poco/MongoDB/Database.h index 62aea632b08..3334a673df6 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Database.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Database.h @@ -26,6 +26,8 @@ #include "Poco/MongoDB/QueryRequest.h" #include "Poco/MongoDB/UpdateRequest.h" +#include "Poco/MongoDB/OpMsgCursor.h" +#include "Poco/MongoDB/OpMsgMessage.h" namespace Poco { @@ -45,6 +47,9 @@ namespace MongoDB virtual ~Database(); /// Destroys the Database. + const std::string & name() const; + /// Database name + bool authenticate( Connection & connection, const std::string & username, @@ -62,34 +67,49 @@ namespace MongoDB /// May throw a Poco::ProtocolException if authentication fails for a reason other than /// invalid credentials. + Document::Ptr queryBuildInfo(Connection & connection) const; + /// Queries server build info (all wire protocols) + + Document::Ptr queryServerHello(Connection & connection) const; + /// Queries hello response from server (all wire protocols) + Int64 count(Connection & connection, const std::string & collectionName) const; - /// Sends a count request for the given collection to MongoDB. + /// Sends a count request for the given collection to MongoDB. (old wire protocol) /// /// If the command fails, -1 is returned. Poco::SharedPtr createCommand() const; - /// Creates a QueryRequest for a command. + /// Creates a QueryRequest for a command. (old wire protocol) Poco::SharedPtr createCountRequest(const std::string & collectionName) const; /// Creates a QueryRequest to count the given collection. - /// The collectionname must not contain the database name. + /// The collectionname must not contain the database name. (old wire protocol) Poco::SharedPtr createDeleteRequest(const std::string & collectionName) const; /// Creates a DeleteRequest to delete documents in the given collection. - /// The collectionname must not contain the database name. + /// The collectionname must not contain the database name. (old wire protocol) Poco::SharedPtr createInsertRequest(const std::string & collectionName) const; /// Creates an InsertRequest to insert new documents in the given collection. - /// The collectionname must not contain the database name. + /// The collectionname must not contain the database name. (old wire protocol) Poco::SharedPtr createQueryRequest(const std::string & collectionName) const; - /// Creates a QueryRequest. + /// Creates a QueryRequest. (old wire protocol) /// The collectionname must not contain the database name. Poco::SharedPtr createUpdateRequest(const std::string & collectionName) const; - /// Creates an UpdateRequest. + /// Creates an UpdateRequest. (old wire protocol) /// The collectionname must not contain the database name. + Poco::SharedPtr createOpMsgMessage(const std::string & collectionName) const; + /// Creates OpMsgMessage. (new wire protocol) + + Poco::SharedPtr createOpMsgMessage() const; + /// Creates OpMsgMessage for database commands that do not require collection as an argument. (new wire protocol) + + Poco::SharedPtr createOpMsgCursor(const std::string & collectionName) const; + /// Creates OpMsgCursor. (new wire protocol) + Poco::MongoDB::Document::Ptr ensureIndex( Connection & connection, const std::string & collection, @@ -100,14 +120,16 @@ namespace MongoDB int version = 0, int ttl = 0); /// Creates an index. The document returned is the result of a getLastError call. - /// For more info look at the ensureIndex information on the MongoDB website. + /// For more info look at the ensureIndex information on the MongoDB website. (old wire protocol) Document::Ptr getLastErrorDoc(Connection & connection) const; /// Sends the getLastError command to the database and returns the error document. + /// (old wire protocol) std::string getLastError(Connection & connection) const; /// Sends the getLastError command to the database and returns the err element /// from the error document. When err is null, an empty string is returned. + /// (old wire protocol) static const std::string AUTH_MONGODB_CR; /// Default authentication mechanism prior to MongoDB 3.0. @@ -115,6 +137,27 @@ namespace MongoDB static const std::string AUTH_SCRAM_SHA1; /// Default authentication mechanism for MongoDB 3.0. + enum WireVersion + /// Wire version as reported by the command hello. + /// See details in MongoDB github, repository specifications. + /// @see queryServerHello + { + VER_26 = 1, + VER_26_2 = 2, + VER_30 = 3, + VER_32 = 4, + VER_34 = 5, + VER_36 = 6, ///< First wire version that supports OP_MSG + VER_40 = 7, + VER_42 = 8, + VER_44 = 9, + VER_50 = 13, + VER_51 = 14, ///< First wire version that supports only OP_MSG + VER_52 = 15, + VER_53 = 16, + VER_60 = 17 + }; + protected: bool authCR(Connection & connection, const std::string & username, const std::string & password); bool authSCRAM(Connection & connection, const std::string & username, const std::string & password); @@ -127,6 +170,12 @@ namespace MongoDB // // inlines // + inline const std::string & Database::name() const + { + return _dbname; + } + + inline Poco::SharedPtr Database::createCommand() const { Poco::SharedPtr cmd = createQueryRequest("$cmd"); @@ -158,6 +207,24 @@ namespace MongoDB return new Poco::MongoDB::UpdateRequest(_dbname + '.' + collectionName); } + // -- New wire protocol commands + + inline Poco::SharedPtr Database::createOpMsgMessage(const std::string & collectionName) const + { + return new Poco::MongoDB::OpMsgMessage(_dbname, collectionName); + } + + inline Poco::SharedPtr Database::createOpMsgMessage() const + { + // Collection name for database commands is not needed. + return createOpMsgMessage(""); + } + + inline Poco::SharedPtr Database::createOpMsgCursor(const std::string & collectionName) const + { + return new Poco::MongoDB::OpMsgCursor(_dbname, collectionName); + } + } } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Document.h b/base/poco/MongoDB/include/Poco/MongoDB/Document.h index 12889663827..9e1df349e20 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Document.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Document.h @@ -31,6 +31,7 @@ namespace Poco namespace MongoDB { + class Array; class ElementFindByName { @@ -48,8 +49,8 @@ namespace MongoDB /// Represents a MongoDB (BSON) document. { public: - typedef SharedPtr Ptr; - typedef std::vector Vector; + using Ptr = SharedPtr; + using Vector = std::vector; Document(); /// Creates an empty Document. @@ -86,6 +87,10 @@ namespace MongoDB /// Unlike the other add methods, this method returns /// a reference to the new document. + Array & addNewArray(const std::string & name); + /// Create a new array and add it to this document. + /// Method returns a reference to the new array. + void clear(); /// Removes all elements from the document. @@ -95,7 +100,7 @@ namespace MongoDB bool empty() const; /// Returns true if the document doesn't contain any documents. - bool exists(const std::string & name); + bool exists(const std::string & name) const; /// Returns true if the document has an element with the given name. template @@ -158,6 +163,9 @@ namespace MongoDB /// return an Int64. When the element is not found, a /// Poco::NotFoundException will be thrown. + bool remove(const std::string & name); + /// Removes an element from the document. + template bool isType(const std::string & name) const /// Returns true when the type of the element equals the TypeId of ElementTrait. @@ -227,12 +235,23 @@ namespace MongoDB } - inline bool Document::exists(const std::string & name) + inline bool Document::exists(const std::string & name) const { return std::find_if(_elements.begin(), _elements.end(), ElementFindByName(name)) != _elements.end(); } + inline bool Document::remove(const std::string & name) + { + auto it = std::find_if(_elements.begin(), _elements.end(), ElementFindByName(name)); + if (it == _elements.end()) + return false; + + _elements.erase(it); + return true; + } + + inline std::size_t Document::size() const { return _elements.size(); diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Element.h b/base/poco/MongoDB/include/Poco/MongoDB/Element.h index b5592bd0e0b..26525d7d02b 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Element.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Element.h @@ -45,7 +45,7 @@ namespace MongoDB /// Represents an Element of a Document or an Array. { public: - typedef Poco::SharedPtr Ptr; + using Ptr = Poco::SharedPtr; explicit Element(const std::string & name); /// Creates the Element with the given name. @@ -80,7 +80,7 @@ namespace MongoDB } - typedef std::list ElementSet; + using ElementSet = std::list; template @@ -266,7 +266,7 @@ namespace MongoDB } - typedef Nullable NullValue; + using NullValue = Nullable; // BSON Null Value diff --git a/base/poco/MongoDB/include/Poco/MongoDB/JavaScriptCode.h b/base/poco/MongoDB/include/Poco/MongoDB/JavaScriptCode.h index df1edc16817..c0f584b7c19 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/JavaScriptCode.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/JavaScriptCode.h @@ -35,7 +35,7 @@ namespace MongoDB /// Represents JavaScript type in BSON. { public: - typedef SharedPtr Ptr; + using Ptr = SharedPtr; JavaScriptCode(); /// Creates an empty JavaScriptCode object. diff --git a/base/poco/MongoDB/include/Poco/MongoDB/MessageHeader.h b/base/poco/MongoDB/include/Poco/MongoDB/MessageHeader.h index 2b88e30fc74..98f45e876c1 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/MessageHeader.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/MessageHeader.h @@ -28,6 +28,9 @@ namespace MongoDB { + class Message; // Required to disambiguate friend declaration in MessageHeader. + + class MongoDB_API MessageHeader /// Represents the message header which is always prepended to a /// MongoDB request or response message. @@ -37,14 +40,18 @@ namespace MongoDB enum OpCode { + // Opcodes deprecated in MongoDB 5.0 OP_REPLY = 1, - OP_MSG = 1000, OP_UPDATE = 2001, OP_INSERT = 2002, OP_QUERY = 2004, OP_GET_MORE = 2005, OP_DELETE = 2006, - OP_KILL_CURSORS = 2007 + OP_KILL_CURSORS = 2007, + + /// Opcodes supported in MongoDB 5.1 and later + OP_COMPRESSED = 2012, + OP_MSG = 2013 }; explicit MessageHeader(OpCode); diff --git a/base/poco/MongoDB/include/Poco/MongoDB/MongoDB.h b/base/poco/MongoDB/include/Poco/MongoDB/MongoDB.h index 253f1f8ab27..de246ddc9dd 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/MongoDB.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/MongoDB.h @@ -33,6 +33,13 @@ // +#if defined(_WIN32) && defined(POCO_DLL) +# if defined(MongoDB_EXPORTS) +# define MongoDB_API __declspec(dllexport) +# else +# define MongoDB_API __declspec(dllimport) +# endif +#endif #if !defined(MongoDB_API) @@ -47,6 +54,11 @@ // // Automatically link MongoDB library. // +#if defined(_MSC_VER) +# if !defined(POCO_NO_AUTOMATIC_LIBS) && !defined(MongoDB_EXPORTS) +# pragma comment(lib, "PocoMongoDB" POCO_LIB_SUFFIX) +# endif +#endif #endif // MongoDBMongoDB_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/ObjectId.h b/base/poco/MongoDB/include/Poco/MongoDB/ObjectId.h index 76bb190db48..8a335320ea0 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/ObjectId.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/ObjectId.h @@ -44,7 +44,7 @@ namespace MongoDB /// as its value. { public: - typedef SharedPtr Ptr; + using Ptr = SharedPtr; explicit ObjectId(const std::string & id); /// Creates an ObjectId from a string. diff --git a/base/poco/MongoDB/include/Poco/MongoDB/OpMsgCursor.h b/base/poco/MongoDB/include/Poco/MongoDB/OpMsgCursor.h new file mode 100644 index 00000000000..a465a71bb1c --- /dev/null +++ b/base/poco/MongoDB/include/Poco/MongoDB/OpMsgCursor.h @@ -0,0 +1,96 @@ +// +// OpMsgCursor.h +// +// Library: MongoDB +// Package: MongoDB +// Module: OpMsgCursor +// +// Definition of the OpMsgCursor class. +// +// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. +// and Contributors. +// +// SPDX-License-Identifier: BSL-1.0 +// + + +#ifndef MongoDB_OpMsgCursor_INCLUDED +#define MongoDB_OpMsgCursor_INCLUDED + + +#include "Poco/MongoDB/Connection.h" +#include "Poco/MongoDB/MongoDB.h" +#include "Poco/MongoDB/OpMsgMessage.h" + +namespace Poco +{ +namespace MongoDB +{ + + + class MongoDB_API OpMsgCursor : public Document + /// OpMsgCursor is an helper class for querying multiple documents using OpMsgMessage. + { + public: + OpMsgCursor(const std::string & dbname, const std::string & collectionName); + /// Creates a OpMsgCursor for the given database and collection. + + virtual ~OpMsgCursor(); + /// Destroys the OpMsgCursor. + + void setEmptyFirstBatch(bool empty); + /// Empty first batch is used to get error response faster with little server processing + + bool emptyFirstBatch() const; + + void setBatchSize(Int32 batchSize); + /// Set non-default batch size + + Int32 batchSize() const; + /// Current batch size (zero or negative number indicates default batch size) + + Int64 cursorID() const; + + OpMsgMessage & next(Connection & connection); + /// Tries to get the next documents. As long as response message has a + /// cursor ID next can be called to retrieve the next bunch of documents. + /// + /// The cursor must be killed (see kill()) when not all documents are needed. + + OpMsgMessage & query(); + /// Returns the associated query. + + void kill(Connection & connection); + /// Kills the cursor and reset it so that it can be reused. + + private: + OpMsgMessage _query; + OpMsgMessage _response; + + bool _emptyFirstBatch{false}; + Int32 _batchSize{-1}; + /// Batch size used in the cursor. Zero or negative value means that default shall be used. + + Int64 _cursorID{0}; + }; + + + // + // inlines + // + inline OpMsgMessage & OpMsgCursor::query() + { + return _query; + } + + inline Int64 OpMsgCursor::cursorID() const + { + return _cursorID; + } + + +} +} // namespace Poco::MongoDB + + +#endif // MongoDB_OpMsgCursor_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/OpMsgMessage.h b/base/poco/MongoDB/include/Poco/MongoDB/OpMsgMessage.h new file mode 100644 index 00000000000..699c7fc4e12 --- /dev/null +++ b/base/poco/MongoDB/include/Poco/MongoDB/OpMsgMessage.h @@ -0,0 +1,163 @@ +// +// OpMsgMessage.h +// +// Library: MongoDB +// Package: MongoDB +// Module: OpMsgMessage +// +// Definition of the OpMsgMessage class. +// +// Copyright (c) 2022, Applied Informatics Software Engineering GmbH. +// and Contributors. +// +// SPDX-License-Identifier: BSL-1.0 +// + + +#ifndef MongoDB_OpMsgMessage_INCLUDED +#define MongoDB_OpMsgMessage_INCLUDED + + +#include "Poco/MongoDB/Document.h" +#include "Poco/MongoDB/Message.h" +#include "Poco/MongoDB/MongoDB.h" + +#include + +namespace Poco +{ +namespace MongoDB +{ + + + class MongoDB_API OpMsgMessage : public Message + /// This class represents a request/response (OP_MSG) to send requests and receive responses to/from MongoDB. + { + public: + // Constants for most often used MongoDB commands that can be sent using OP_MSG + // For complete list see: https://www.mongodb.com/docs/manual/reference/command/ + + // Query and write + static const std::string CMD_INSERT; + static const std::string CMD_DELETE; + static const std::string CMD_UPDATE; + static const std::string CMD_FIND; + static const std::string CMD_FIND_AND_MODIFY; + static const std::string CMD_GET_MORE; + + // Aggregation + static const std::string CMD_AGGREGATE; + static const std::string CMD_COUNT; + static const std::string CMD_DISTINCT; + static const std::string CMD_MAP_REDUCE; + + // Replication and administration + static const std::string CMD_HELLO; + static const std::string CMD_REPL_SET_GET_STATUS; + static const std::string CMD_REPL_SET_GET_CONFIG; + + static const std::string CMD_CREATE; + static const std::string CMD_CREATE_INDEXES; + static const std::string CMD_DROP; + static const std::string CMD_DROP_DATABASE; + static const std::string CMD_KILL_CURSORS; + static const std::string CMD_LIST_DATABASES; + static const std::string CMD_LIST_INDEXES; + + // Diagnostic + static const std::string CMD_BUILD_INFO; + static const std::string CMD_COLL_STATS; + static const std::string CMD_DB_STATS; + static const std::string CMD_HOST_INFO; + + + enum Flags : UInt32 + { + MSG_FLAGS_DEFAULT = 0, + + MSG_CHECKSUM_PRESENT = (1 << 0), + + MSG_MORE_TO_COME = (1 << 1), + /// Sender will send another message and is not prepared for overlapping messages + + MSG_EXHAUST_ALLOWED = (1 << 16) + /// Client is prepared for multiple replies (using the moreToCome bit) to this request + }; + + OpMsgMessage(); + /// Creates an OpMsgMessage for response. + + OpMsgMessage(const std::string & databaseName, const std::string & collectionName, UInt32 flags = MSG_FLAGS_DEFAULT); + /// Creates an OpMsgMessage for requests. + + virtual ~OpMsgMessage(); + + const std::string & databaseName() const; + + const std::string & collectionName() const; + + void setCommandName(const std::string & command); + /// Sets the command name and clears the command document + + void setCursor(Poco::Int64 cursorID, Poco::Int32 batchSize = -1); + /// Sets the command "getMore" for the cursor id with batch size (if it is not negative). + + const std::string & commandName() const; + /// Current command name. + + void setAcknowledgedRequest(bool ack); + /// Set false to create request that does not return response. + /// It has effect only for commands that write or delete documents. + /// Default is true (request returns acknowledge response). + + bool acknowledgedRequest() const; + + UInt32 flags() const; + + Document & body(); + /// Access to body document. + /// Additional query arguments shall be added after setting the command name. + + const Document & body() const; + + Document::Vector & documents(); + /// Documents prepared for request or retrieved in response. + + const Document::Vector & documents() const; + /// Documents prepared for request or retrieved in response. + + bool responseOk() const; + /// Reads "ok" status from the response message. + + void clear(); + /// Clears the message. + + void send(std::ostream & ostr); + /// Writes the request to stream. + + void read(std::istream & istr); + /// Reads the response from the stream. + + private: + enum PayloadType : UInt8 + { + PAYLOAD_TYPE_0 = 0, + PAYLOAD_TYPE_1 = 1 + }; + + std::string _databaseName; + std::string _collectionName; + UInt32 _flags{MSG_FLAGS_DEFAULT}; + std::string _commandName; + bool _acknowledged{true}; + + Document _body; + Document::Vector _documents; + }; + + +} +} // namespace Poco::MongoDB + + +#endif // MongoDB_OpMsgMessage_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/PoolableConnectionFactory.h b/base/poco/MongoDB/include/Poco/MongoDB/PoolableConnectionFactory.h index 9d35c728e5e..53f4a5127ef 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/PoolableConnectionFactory.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/PoolableConnectionFactory.h @@ -94,7 +94,23 @@ namespace MongoDB operator Connection::Ptr() { return _connection; } +#if defined(POCO_ENABLE_CPP11) + // Disable copy to prevent unwanted release of resources: C++11 way + PooledConnection(const PooledConnection &) = delete; + PooledConnection & operator=(const PooledConnection &) = delete; + + // Enable move semantics + PooledConnection(PooledConnection && other) = default; + PooledConnection & operator=(PooledConnection &&) = default; +#endif + private: +#if !defined(POCO_ENABLE_CPP11) + // Disable copy to prevent unwanted release of resources: pre C++11 way + PooledConnection(const PooledConnection &); + PooledConnection & operator=(const PooledConnection &); +#endif + Poco::ObjectPool & _pool; Connection::Ptr _connection; }; diff --git a/base/poco/MongoDB/include/Poco/MongoDB/RegularExpression.h b/base/poco/MongoDB/include/Poco/MongoDB/RegularExpression.h index b9a8694d321..244b8c14163 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/RegularExpression.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/RegularExpression.h @@ -33,7 +33,7 @@ namespace MongoDB /// Represents a regular expression in BSON format. { public: - typedef SharedPtr Ptr; + using Ptr = SharedPtr; RegularExpression(); /// Creates an empty RegularExpression. diff --git a/base/poco/MongoDB/include/Poco/MongoDB/ResponseMessage.h b/base/poco/MongoDB/include/Poco/MongoDB/ResponseMessage.h index 132859cc75f..9cb92cb16c4 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/ResponseMessage.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/ResponseMessage.h @@ -38,6 +38,9 @@ namespace MongoDB ResponseMessage(); /// Creates an empty ResponseMessage. + ResponseMessage(const Int64 & cursorID); + /// Creates an ResponseMessage for existing cursor ID. + virtual ~ResponseMessage(); /// Destroys the ResponseMessage. diff --git a/base/poco/MongoDB/src/Array.cpp b/base/poco/MongoDB/src/Array.cpp index c6d96d1371d..6fff0994d82 100644 --- a/base/poco/MongoDB/src/Array.cpp +++ b/base/poco/MongoDB/src/Array.cpp @@ -20,7 +20,7 @@ namespace Poco { namespace MongoDB { -Array::Array(): +Array::Array(): Document() { } @@ -31,7 +31,7 @@ Array::~Array() } -Element::Ptr Array::get(int pos) const +Element::Ptr Array::get(std::size_t pos) const { std::string name = Poco::NumberFormatter::format(pos); return Document::get(name); diff --git a/base/poco/MongoDB/src/Connection.cpp b/base/poco/MongoDB/src/Connection.cpp index 38c31d2250a..fa20887054b 100644 --- a/base/poco/MongoDB/src/Connection.cpp +++ b/base/poco/MongoDB/src/Connection.cpp @@ -319,4 +319,30 @@ void Connection::sendRequest(RequestMessage& request, ResponseMessage& response) } +void Connection::sendRequest(OpMsgMessage& request, OpMsgMessage& response) +{ + Poco::Net::SocketOutputStream sos(_socket); + request.send(sos); + + response.clear(); + readResponse(response); +} + + +void Connection::sendRequest(OpMsgMessage& request) +{ + request.setAcknowledgedRequest(false); + Poco::Net::SocketOutputStream sos(_socket); + request.send(sos); +} + + +void Connection::readResponse(OpMsgMessage& response) +{ + Poco::Net::SocketInputStream sis(_socket); + response.read(sis); +} + + + } } // Poco::MongoDB diff --git a/base/poco/MongoDB/src/Cursor.cpp b/base/poco/MongoDB/src/Cursor.cpp index 69031e0ab65..ef7a4ca961d 100644 --- a/base/poco/MongoDB/src/Cursor.cpp +++ b/base/poco/MongoDB/src/Cursor.cpp @@ -33,6 +33,12 @@ Cursor::Cursor(const std::string& fullCollectionName, QueryRequest::Flags flags) } +Cursor::Cursor(const Document& aggregationResponse) : + _query(aggregationResponse.get("cursor")->get("ns")), + _response(aggregationResponse.get("cursor")->get("id")) +{ +} + Cursor::~Cursor() { try diff --git a/base/poco/MongoDB/src/Database.cpp b/base/poco/MongoDB/src/Database.cpp index 2b31523bdc4..1a0d3cfe559 100644 --- a/base/poco/MongoDB/src/Database.cpp +++ b/base/poco/MongoDB/src/Database.cpp @@ -334,6 +334,50 @@ bool Database::authSCRAM(Connection& connection, const std::string& username, co } +Document::Ptr Database::queryBuildInfo(Connection& connection) const +{ + // build info can be issued on "config" system database + Poco::SharedPtr request = createCommand(); + request->selector().add("buildInfo", 1); + + Poco::MongoDB::ResponseMessage response; + connection.sendRequest(*request, response); + + Document::Ptr buildInfo; + if ( response.documents().size() > 0 ) + { + buildInfo = response.documents()[0]; + } + else + { + throw Poco::ProtocolException("Didn't get a response from the buildinfo command"); + } + return buildInfo; +} + + +Document::Ptr Database::queryServerHello(Connection& connection) const +{ + // hello can be issued on "config" system database + Poco::SharedPtr request = createCommand(); + request->selector().add("hello", 1); + + Poco::MongoDB::ResponseMessage response; + connection.sendRequest(*request, response); + + Document::Ptr hello; + if ( response.documents().size() > 0 ) + { + hello = response.documents()[0]; + } + else + { + throw Poco::ProtocolException("Didn't get a response from the hello command"); + } + return hello; +} + + Int64 Database::count(Connection& connection, const std::string& collectionName) const { Poco::SharedPtr countRequest = createCountRequest(collectionName); @@ -390,7 +434,7 @@ Document::Ptr Database::getLastErrorDoc(Connection& connection) const { Document::Ptr errorDoc; - Poco::SharedPtr request = createQueryRequest("$cmd"); + Poco::SharedPtr request = createCommand(); request->setNumberToReturn(1); request->selector().add("getLastError", 1); @@ -420,7 +464,7 @@ std::string Database::getLastError(Connection& connection) const Poco::SharedPtr Database::createCountRequest(const std::string& collectionName) const { - Poco::SharedPtr request = createQueryRequest("$cmd"); + Poco::SharedPtr request = createCommand(); request->setNumberToReturn(1); request->selector().add("count", collectionName); return request; diff --git a/base/poco/MongoDB/src/DeleteRequest.cpp b/base/poco/MongoDB/src/DeleteRequest.cpp index 67a88c33302..ba75beb55fb 100644 --- a/base/poco/MongoDB/src/DeleteRequest.cpp +++ b/base/poco/MongoDB/src/DeleteRequest.cpp @@ -20,8 +20,8 @@ namespace MongoDB { DeleteRequest::DeleteRequest(const std::string& collectionName, DeleteRequest::Flags flags): - RequestMessage(MessageHeader::OP_DELETE), - _flags(flags), + RequestMessage(MessageHeader::OP_DELETE), + _flags(flags), _fullCollectionName(collectionName), _selector() { diff --git a/base/poco/MongoDB/src/Document.cpp b/base/poco/MongoDB/src/Document.cpp index 114fc993891..f7c5c9c5dc6 100644 --- a/base/poco/MongoDB/src/Document.cpp +++ b/base/poco/MongoDB/src/Document.cpp @@ -35,6 +35,14 @@ Document::~Document() } +Array& Document::addNewArray(const std::string& name) +{ + Array::Ptr newArray = new Array(); + add(name, newArray); + return *newArray; +} + + Element::Ptr Document::get(const std::string& name) const { Element::Ptr element; @@ -84,7 +92,7 @@ void Document::read(BinaryReader& reader) while (type != '\0') { Element::Ptr element; - + std::string name = BSONReader(reader).readCString(); switch (type) @@ -198,7 +206,7 @@ void Document::write(BinaryWriter& writer) else { std::stringstream sstream; - Poco::BinaryWriter tempWriter(sstream); + Poco::BinaryWriter tempWriter(sstream, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); for (ElementSet::iterator it = _elements.begin(); it != _elements.end(); ++it) { tempWriter << static_cast((*it)->type()); @@ -207,7 +215,7 @@ void Document::write(BinaryWriter& writer) element->write(tempWriter); } tempWriter.flush(); - + Poco::Int32 len = static_cast(5 + sstream.tellp()); /* 5 = sizeof(len) + 0-byte */ writer << len; writer.writeRaw(sstream.str()); diff --git a/base/poco/MongoDB/src/Element.cpp b/base/poco/MongoDB/src/Element.cpp index 89629e0503e..f91ce264493 100644 --- a/base/poco/MongoDB/src/Element.cpp +++ b/base/poco/MongoDB/src/Element.cpp @@ -24,7 +24,7 @@ Element::Element(const std::string& name) : _name(name) } -Element::~Element() +Element::~Element() { } diff --git a/base/poco/MongoDB/src/GetMoreRequest.cpp b/base/poco/MongoDB/src/GetMoreRequest.cpp index f8a6b73c6ad..2c1f6909eb7 100644 --- a/base/poco/MongoDB/src/GetMoreRequest.cpp +++ b/base/poco/MongoDB/src/GetMoreRequest.cpp @@ -21,7 +21,7 @@ namespace MongoDB { GetMoreRequest::GetMoreRequest(const std::string& collectionName, Int64 cursorID): - RequestMessage(MessageHeader::OP_GET_MORE), + RequestMessage(MessageHeader::OP_GET_MORE), _fullCollectionName(collectionName), _numberToReturn(100), _cursorID(cursorID) diff --git a/base/poco/MongoDB/src/InsertRequest.cpp b/base/poco/MongoDB/src/InsertRequest.cpp index ec8dc9cf94a..65be5654b3e 100644 --- a/base/poco/MongoDB/src/InsertRequest.cpp +++ b/base/poco/MongoDB/src/InsertRequest.cpp @@ -20,7 +20,7 @@ namespace MongoDB { InsertRequest::InsertRequest(const std::string& collectionName, Flags flags): - RequestMessage(MessageHeader::OP_INSERT), + RequestMessage(MessageHeader::OP_INSERT), _flags(flags), _fullCollectionName(collectionName) { diff --git a/base/poco/MongoDB/src/KillCursorsRequest.cpp b/base/poco/MongoDB/src/KillCursorsRequest.cpp index 6baa0e0be8f..448002aa16a 100644 --- a/base/poco/MongoDB/src/KillCursorsRequest.cpp +++ b/base/poco/MongoDB/src/KillCursorsRequest.cpp @@ -37,7 +37,7 @@ void KillCursorsRequest::buildRequest(BinaryWriter& writer) for (std::vector::iterator it = _cursors.begin(); it != _cursors.end(); ++it) { writer << *it; - } + } } diff --git a/base/poco/MongoDB/src/Message.cpp b/base/poco/MongoDB/src/Message.cpp index c29d282d15a..7b1cb23bab6 100644 --- a/base/poco/MongoDB/src/Message.cpp +++ b/base/poco/MongoDB/src/Message.cpp @@ -19,7 +19,7 @@ namespace Poco { namespace MongoDB { -Message::Message(MessageHeader::OpCode opcode): +Message::Message(MessageHeader::OpCode opcode): _header(opcode) { } diff --git a/base/poco/MongoDB/src/MessageHeader.cpp b/base/poco/MongoDB/src/MessageHeader.cpp index 222121243db..b472bcec465 100644 --- a/base/poco/MongoDB/src/MessageHeader.cpp +++ b/base/poco/MongoDB/src/MessageHeader.cpp @@ -20,10 +20,10 @@ namespace Poco { namespace MongoDB { -MessageHeader::MessageHeader(OpCode opCode): - _messageLength(0), - _requestID(0), - _responseTo(0), +MessageHeader::MessageHeader(OpCode opCode): + _messageLength(0), + _requestID(0), + _responseTo(0), _opCode(opCode) { } @@ -42,7 +42,7 @@ void MessageHeader::read(BinaryReader& reader) Int32 opCode; reader >> opCode; - _opCode = (OpCode) opCode; + _opCode = static_cast(opCode); if (!reader.good()) { @@ -56,7 +56,7 @@ void MessageHeader::write(BinaryWriter& writer) writer << _messageLength; writer << _requestID; writer << _responseTo; - writer << (Int32) _opCode; + writer << static_cast(_opCode); } diff --git a/base/poco/MongoDB/src/ObjectId.cpp b/base/poco/MongoDB/src/ObjectId.cpp index 3065a2ffc30..0125c246c2d 100644 --- a/base/poco/MongoDB/src/ObjectId.cpp +++ b/base/poco/MongoDB/src/ObjectId.cpp @@ -32,7 +32,7 @@ ObjectId::ObjectId(const std::string& id) poco_assert_dbg(id.size() == 24); const char* p = id.c_str(); - for (std::size_t i = 0; i < 12; ++i) + for (std::size_t i = 0; i < 12; ++i) { _id[i] = fromHex(p); p += 2; diff --git a/base/poco/MongoDB/src/OpMsgCursor.cpp b/base/poco/MongoDB/src/OpMsgCursor.cpp new file mode 100644 index 00000000000..bc95851ae33 --- /dev/null +++ b/base/poco/MongoDB/src/OpMsgCursor.cpp @@ -0,0 +1,187 @@ +// +// OpMsgCursor.cpp +// +// Library: MongoDB +// Package: MongoDB +// Module: OpMsgCursor +// +// Copyright (c) 2022, Applied Informatics Software Engineering GmbH. +// and Contributors. +// +// SPDX-License-Identifier: BSL-1.0 +// + + +#include "Poco/MongoDB/OpMsgCursor.h" +#include "Poco/MongoDB/Array.h" + +// +// NOTE: +// +// MongoDB specification indicates that the flag MSG_EXHAUST_ALLOWED shall be +// used in the request when the receiver is ready to receive multiple messages +// without sending additional requests in between. Sender (MongoDB) indicates +// that more messages follow with flag MSG_MORE_TO_COME. +// +// It seems that this does not work properly. MSG_MORE_TO_COME is set and reading +// next messages sometimes works, however often the data is missing in response +// or the message header contains wrong message length and reading blocks. +// Opcode in the header is correct. +// +// Using MSG_EXHAUST_ALLOWED is therefore currently disabled. +// +// It seems that related JIRA ticket is: +// +// https://jira.mongodb.org/browse/SERVER-57297 +// +// https://github.com/mongodb/specifications/blob/master/source/message/OP_MSG.rst +// + +#define MONGODB_EXHAUST_ALLOWED_WORKS false + +namespace Poco { +namespace MongoDB { + + +static const std::string keyCursor {"cursor"}; +static const std::string keyFirstBatch {"firstBatch"}; +static const std::string keyNextBatch {"nextBatch"}; + +static Poco::Int64 cursorIdFromResponse(const MongoDB::Document& doc); + + +OpMsgCursor::OpMsgCursor(const std::string& db, const std::string& collection): +#if MONGODB_EXHAUST_ALLOWED_WORKS + _query(db, collection, OpMsgMessage::MSG_EXHAUST_ALLOWED) +#else + _query(db, collection) +#endif +{ +} + +OpMsgCursor::~OpMsgCursor() +{ + try + { + poco_assert_dbg(_cursorID == 0); + } + catch (...) + { + } +} + + +void OpMsgCursor::setEmptyFirstBatch(bool empty) +{ + _emptyFirstBatch = empty; +} + + +bool OpMsgCursor::emptyFirstBatch() const +{ + return _emptyFirstBatch; +} + + +void OpMsgCursor::setBatchSize(Int32 batchSize) +{ + _batchSize = batchSize; +} + + +Int32 OpMsgCursor::batchSize() const +{ + return _batchSize; +} + + +OpMsgMessage& OpMsgCursor::next(Connection& connection) +{ + if (_cursorID == 0) + { + _response.clear(); + + if (_emptyFirstBatch || _batchSize > 0) + { + Int32 bsize = _emptyFirstBatch ? 0 : _batchSize; + if (_query.commandName() == OpMsgMessage::CMD_FIND) + { + _query.body().add("batchSize", bsize); + } + else if (_query.commandName() == OpMsgMessage::CMD_AGGREGATE) + { + auto& cursorDoc = _query.body().addNewDocument("cursor"); + cursorDoc.add("batchSize", bsize); + } + } + + connection.sendRequest(_query, _response); + + const auto& rdoc = _response.body(); + _cursorID = cursorIdFromResponse(rdoc); + } + else + { +#if MONGODB_EXHAUST_ALLOWED_WORKS + std::cout << "Response flags: " << _response.flags() << std::endl; + if (_response.flags() & OpMsgMessage::MSG_MORE_TO_COME) + { + std::cout << "More to come. Reading more response: " << std::endl; + _response.clear(); + connection.readResponse(_response); + } + else +#endif + { + _response.clear(); + _query.setCursor(_cursorID, _batchSize); + connection.sendRequest(_query, _response); + } + } + + const auto& rdoc = _response.body(); + _cursorID = cursorIdFromResponse(rdoc); + + return _response; +} + + +void OpMsgCursor::kill(Connection& connection) +{ + _response.clear(); + if (_cursorID != 0) + { + _query.setCommandName(OpMsgMessage::CMD_KILL_CURSORS); + + MongoDB::Array::Ptr cursors = new MongoDB::Array(); + cursors->add(_cursorID); + _query.body().add("cursors", cursors); + + connection.sendRequest(_query, _response); + + const auto killed = _response.body().get("cursorsKilled", nullptr); + if (!killed || killed->size() != 1 || killed->get(0, -1) != _cursorID) + { + throw Poco::ProtocolException("Cursor not killed as expected: " + std::to_string(_cursorID)); + } + + _cursorID = 0; + _query.clear(); + _response.clear(); + } +} + + +Poco::Int64 cursorIdFromResponse(const MongoDB::Document& doc) +{ + Poco::Int64 id {0}; + auto cursorDoc = doc.get(keyCursor, nullptr); + if(cursorDoc) + { + id = cursorDoc->get("id", 0); + } + return id; +} + + +} } // Namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/OpMsgMessage.cpp b/base/poco/MongoDB/src/OpMsgMessage.cpp new file mode 100644 index 00000000000..2b55772ca59 --- /dev/null +++ b/base/poco/MongoDB/src/OpMsgMessage.cpp @@ -0,0 +1,412 @@ +// +// OpMsgMessage.cpp +// +// Library: MongoDB +// Package: MongoDB +// Module: OpMsgMessage +// +// Copyright (c) 2022, Applied Informatics Software Engineering GmbH. +// and Contributors. +// +// SPDX-License-Identifier: BSL-1.0 +// + +#include "Poco/MongoDB/OpMsgMessage.h" +#include "Poco/MongoDB/MessageHeader.h" +#include "Poco/MongoDB/Array.h" +#include "Poco/StreamCopier.h" +#include "Poco/Logger.h" + +#define POCO_MONGODB_DUMP false + +namespace Poco { +namespace MongoDB { + +// Query and write +const std::string OpMsgMessage::CMD_INSERT { "insert" }; +const std::string OpMsgMessage::CMD_DELETE { "delete" }; +const std::string OpMsgMessage::CMD_UPDATE { "update" }; +const std::string OpMsgMessage::CMD_FIND { "find" }; +const std::string OpMsgMessage::CMD_FIND_AND_MODIFY { "findAndModify" }; +const std::string OpMsgMessage::CMD_GET_MORE { "getMore" }; + +// Aggregation +const std::string OpMsgMessage::CMD_AGGREGATE { "aggregate" }; +const std::string OpMsgMessage::CMD_COUNT { "count" }; +const std::string OpMsgMessage::CMD_DISTINCT { "distinct" }; +const std::string OpMsgMessage::CMD_MAP_REDUCE { "mapReduce" }; + +// Replication and administration +const std::string OpMsgMessage::CMD_HELLO { "hello" }; +const std::string OpMsgMessage::CMD_REPL_SET_GET_STATUS { "replSetGetStatus" }; +const std::string OpMsgMessage::CMD_REPL_SET_GET_CONFIG { "replSetGetConfig" }; + +const std::string OpMsgMessage::CMD_CREATE { "create" }; +const std::string OpMsgMessage::CMD_CREATE_INDEXES { "createIndexes" }; +const std::string OpMsgMessage::CMD_DROP { "drop" }; +const std::string OpMsgMessage::CMD_DROP_DATABASE { "dropDatabase" }; +const std::string OpMsgMessage::CMD_KILL_CURSORS { "killCursors" }; +const std::string OpMsgMessage::CMD_LIST_DATABASES { "listDatabases" }; +const std::string OpMsgMessage::CMD_LIST_INDEXES { "listIndexes" }; + +// Diagnostic +const std::string OpMsgMessage::CMD_BUILD_INFO { "buildInfo" }; +const std::string OpMsgMessage::CMD_COLL_STATS { "collStats" }; +const std::string OpMsgMessage::CMD_DB_STATS { "dbStats" }; +const std::string OpMsgMessage::CMD_HOST_INFO { "hostInfo" }; + + +static const std::string& commandIdentifier(const std::string& command); + /// Commands have different names for the payload that is sent in a separate section + + +static const std::string keyCursor {"cursor"}; +static const std::string keyFirstBatch {"firstBatch"}; +static const std::string keyNextBatch {"nextBatch"}; + + +OpMsgMessage::OpMsgMessage() : + Message(MessageHeader::OP_MSG) +{ +} + + +OpMsgMessage::OpMsgMessage(const std::string& databaseName, const std::string& collectionName, UInt32 flags) : + Message(MessageHeader::OP_MSG), + _databaseName(databaseName), + _collectionName(collectionName), + _flags(flags) +{ +} + + +OpMsgMessage::~OpMsgMessage() +{ +} + +const std::string& OpMsgMessage::databaseName() const +{ + return _databaseName; +} + + +const std::string& OpMsgMessage::collectionName() const +{ + return _collectionName; +} + + +void OpMsgMessage::setCommandName(const std::string& command) +{ + _commandName = command; + _body.clear(); + + // IMPORTANT: Command name must be first + if (_collectionName.empty()) + { + // Collection is not specified. It is assumed that this particular command does + // not need it. + _body.add(_commandName, Int32(1)); + } + else + { + _body.add(_commandName, _collectionName); + } + _body.add("$db", _databaseName); +} + + +void OpMsgMessage::setCursor(Poco::Int64 cursorID, Poco::Int32 batchSize) +{ + _commandName = OpMsgMessage::CMD_GET_MORE; + _body.clear(); + + // IMPORTANT: Command name must be first + _body.add(_commandName, cursorID); + _body.add("$db", _databaseName); + _body.add("collection", _collectionName); + if (batchSize > 0) + { + _body.add("batchSize", batchSize); + } +} + + +const std::string& OpMsgMessage::commandName() const +{ + return _commandName; +} + + +void OpMsgMessage::setAcknowledgedRequest(bool ack) +{ + const auto& id = commandIdentifier(_commandName); + if (id.empty()) + return; + + _acknowledged = ack; + + auto writeConcern = _body.get("writeConcern", nullptr); + if (writeConcern) + writeConcern->remove("w"); + + if (ack) + { + _flags = _flags & (~MSG_MORE_TO_COME); + } + else + { + _flags = _flags | MSG_MORE_TO_COME; + if (!writeConcern) + _body.addNewDocument("writeConcern").add("w", 0); + else + writeConcern->add("w", 0); + } + +} + + +bool OpMsgMessage::acknowledgedRequest() const +{ + return _acknowledged; +} + + +UInt32 OpMsgMessage::flags() const +{ + return _flags; +} + + +Document& OpMsgMessage::body() +{ + return _body; +} + + +const Document& OpMsgMessage::body() const +{ + return _body; +} + + +Document::Vector& OpMsgMessage::documents() +{ + return _documents; +} + + +const Document::Vector& OpMsgMessage::documents() const +{ + return _documents; +} + + +bool OpMsgMessage::responseOk() const +{ + Poco::Int64 ok {false}; + if (_body.exists("ok")) + { + ok = _body.getInteger("ok"); + } + return (ok != 0); +} + + +void OpMsgMessage::clear() +{ + _flags = MSG_FLAGS_DEFAULT; + _commandName.clear(); + _body.clear(); + _documents.clear(); +} + + +void OpMsgMessage::send(std::ostream& ostr) +{ + BinaryWriter socketWriter(ostr, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); + + // Serialise the body + std::stringstream ss; + BinaryWriter writer(ss, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); + writer << _flags; + + writer << PAYLOAD_TYPE_0; + _body.write(writer); + + if (!_documents.empty()) + { + // Serialise attached documents + + std::stringstream ssdoc; + BinaryWriter wdoc(ssdoc, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); + for (auto& doc: _documents) + { + doc->write(wdoc); + } + wdoc.flush(); + + const std::string& identifier = commandIdentifier(_commandName); + const Poco::Int32 size = static_cast(sizeof(size) + identifier.size() + 1 + ssdoc.tellp()); + writer << PAYLOAD_TYPE_1; + writer << size; + writer.writeCString(identifier.c_str()); + StreamCopier::copyStream(ssdoc, ss); + } + writer.flush(); + +#if POCO_MONGODB_DUMP + const std::string section = ss.str(); + std::string dump; + Logger::formatDump(dump, section.data(), section.length()); + std::cout << dump << std::endl; +#endif + + messageLength(static_cast(ss.tellp())); + + _header.write(socketWriter); + StreamCopier::copyStream(ss, ostr); + + ostr.flush(); +} + + +void OpMsgMessage::read(std::istream& istr) +{ + std::string message; + { + BinaryReader reader(istr, BinaryReader::LITTLE_ENDIAN_BYTE_ORDER); + _header.read(reader); + + poco_assert_dbg(_header.opCode() == _header.OP_MSG); + + const std::streamsize remainingSize {_header.getMessageLength() - _header.MSG_HEADER_SIZE }; + message.reserve(remainingSize); + +#if POCO_MONGODB_DUMP + std::cout + << "Message hdr: " << _header.getMessageLength() << " " << remainingSize << " " + << _header.opCode() << " " << _header.getRequestID() << " " << _header.responseTo() + << std::endl; +#endif + + reader.readRaw(remainingSize, message); + +#if POCO_MONGODB_DUMP + std::string dump; + Logger::formatDump(dump, message.data(), message.length()); + std::cout << dump << std::endl; +#endif + } + // Read complete message and then interpret it. + + std::istringstream msgss(message); + BinaryReader reader(msgss, BinaryReader::LITTLE_ENDIAN_BYTE_ORDER); + + Poco::UInt8 payloadType {0xFF}; + + reader >> _flags; + reader >> payloadType; + poco_assert_dbg(payloadType == PAYLOAD_TYPE_0); + + _body.read(reader); + + // Read next sections from the buffer + while (msgss.good()) + { + // NOTE: Not tested yet with database, because it returns everything in the body. + // Does MongoDB ever return documents as Payload type 1? + reader >> payloadType; + if (!msgss.good()) + { + break; + } + poco_assert_dbg(payloadType == PAYLOAD_TYPE_1); +#if POCO_MONGODB_DUMP + std::cout << "section payload: " << payloadType << std::endl; +#endif + + Poco::Int32 sectionSize {0}; + reader >> sectionSize; + poco_assert_dbg(sectionSize > 0); + +#if POCO_MONGODB_DUMP + std::cout << "section size: " << sectionSize << std::endl; +#endif + std::streamoff offset = sectionSize - sizeof(sectionSize); + std::streampos endOfSection = msgss.tellg() + offset; + + std::string identifier; + reader.readCString(identifier); +#if POCO_MONGODB_DUMP + std::cout << "section identifier: " << identifier << std::endl; +#endif + + // Loop to read documents from this section. + while (msgss.tellg() < endOfSection) + { +#if POCO_MONGODB_DUMP + std::cout << "section doc: " << msgss.tellg() << " " << endOfSection << std::endl; +#endif + Document::Ptr doc = new Document(); + doc->read(reader); + _documents.push_back(doc); + if (msgss.tellg() < 0) + { + break; + } + } + } + + // Extract documents from the cursor batch if they are there. + MongoDB::Array::Ptr batch; + auto curDoc = _body.get(keyCursor, nullptr); + if (curDoc) + { + batch = curDoc->get(keyFirstBatch, nullptr); + if (!batch) + { + batch = curDoc->get(keyNextBatch, nullptr); + } + } + if (batch) + { + for(std::size_t i = 0; i < batch->size(); i++) + { + const auto& d = batch->get(i, nullptr); + if (d) + { + _documents.push_back(d); + } + } + } + +} + +const std::string& commandIdentifier(const std::string& command) +{ + // Names of identifiers for commands that send bulk documents in the request + // The identifier is set in the section type 1. + static std::map identifiers { + { OpMsgMessage::CMD_INSERT, "documents" }, + { OpMsgMessage::CMD_DELETE, "deletes" }, + { OpMsgMessage::CMD_UPDATE, "updates" }, + + // Not sure if create index can send document section + { OpMsgMessage::CMD_CREATE_INDEXES, "indexes" } + }; + + const auto i = identifiers.find(command); + if (i != identifiers.end()) + { + return i->second; + } + + // This likely means that documents are incorrectly set for a command + // that does not send list of documents in section type 1. + static const std::string emptyIdentifier; + return emptyIdentifier; +} + + +} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/QueryRequest.cpp b/base/poco/MongoDB/src/QueryRequest.cpp index 7044335ba30..6d7d23a8456 100644 --- a/base/poco/MongoDB/src/QueryRequest.cpp +++ b/base/poco/MongoDB/src/QueryRequest.cpp @@ -20,10 +20,10 @@ namespace MongoDB { QueryRequest::QueryRequest(const std::string& collectionName, QueryRequest::Flags flags): - RequestMessage(MessageHeader::OP_QUERY), - _flags(flags), + RequestMessage(MessageHeader::OP_QUERY), + _flags(flags), _fullCollectionName(collectionName), - _numberToSkip(0), + _numberToSkip(0), _numberToReturn(100), _selector(), _returnFieldSelector() diff --git a/base/poco/MongoDB/src/RegularExpression.cpp b/base/poco/MongoDB/src/RegularExpression.cpp index e95e7da82e1..5f7eb6bb51b 100644 --- a/base/poco/MongoDB/src/RegularExpression.cpp +++ b/base/poco/MongoDB/src/RegularExpression.cpp @@ -25,8 +25,8 @@ RegularExpression::RegularExpression() } -RegularExpression::RegularExpression(const std::string& pattern, const std::string& options): - _pattern(pattern), +RegularExpression::RegularExpression(const std::string& pattern, const std::string& options): + _pattern(pattern), _options(options) { } diff --git a/base/poco/MongoDB/src/ReplicaSet.cpp b/base/poco/MongoDB/src/ReplicaSet.cpp index b56fea49311..fce2f2bdada 100644 --- a/base/poco/MongoDB/src/ReplicaSet.cpp +++ b/base/poco/MongoDB/src/ReplicaSet.cpp @@ -21,7 +21,7 @@ namespace Poco { namespace MongoDB { -ReplicaSet::ReplicaSet(const std::vector &addresses): +ReplicaSet::ReplicaSet(const std::vector &addresses): _addresses(addresses) { } @@ -81,8 +81,8 @@ Connection::Ptr ReplicaSet::isMaster(const Net::SocketAddress& address) { conn = 0; } - - return 0; + + return 0; } diff --git a/base/poco/MongoDB/src/RequestMessage.cpp b/base/poco/MongoDB/src/RequestMessage.cpp index 6391d966198..999ed8a6ba1 100644 --- a/base/poco/MongoDB/src/RequestMessage.cpp +++ b/base/poco/MongoDB/src/RequestMessage.cpp @@ -21,7 +21,7 @@ namespace Poco { namespace MongoDB { -RequestMessage::RequestMessage(MessageHeader::OpCode opcode): +RequestMessage::RequestMessage(MessageHeader::OpCode opcode): Message(opcode) { } @@ -35,7 +35,7 @@ RequestMessage::~RequestMessage() void RequestMessage::send(std::ostream& ostr) { std::stringstream ss; - BinaryWriter requestWriter(ss); + BinaryWriter requestWriter(ss, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); buildRequest(requestWriter); requestWriter.flush(); diff --git a/base/poco/MongoDB/src/ResponseMessage.cpp b/base/poco/MongoDB/src/ResponseMessage.cpp index 3254ace63e6..e8216767494 100644 --- a/base/poco/MongoDB/src/ResponseMessage.cpp +++ b/base/poco/MongoDB/src/ResponseMessage.cpp @@ -21,10 +21,20 @@ namespace MongoDB { ResponseMessage::ResponseMessage(): - Message(MessageHeader::OP_REPLY), - _responseFlags(0), - _cursorID(0), - _startingFrom(0), + Message(MessageHeader::OP_REPLY), + _responseFlags(0), + _cursorID(0), + _startingFrom(0), + _numberReturned(0) +{ +} + + +ResponseMessage::ResponseMessage(const Int64& cursorID): + Message(MessageHeader::OP_REPLY), + _responseFlags(0), + _cursorID(cursorID), + _startingFrom(0), _numberReturned(0) { } @@ -50,7 +60,7 @@ void ResponseMessage::read(std::istream& istr) clear(); BinaryReader reader(istr, BinaryReader::LITTLE_ENDIAN_BYTE_ORDER); - + _header.read(reader); reader >> _responseFlags; diff --git a/base/poco/MongoDB/src/UpdateRequest.cpp b/base/poco/MongoDB/src/UpdateRequest.cpp index 2af4621ff64..7477fc752d5 100644 --- a/base/poco/MongoDB/src/UpdateRequest.cpp +++ b/base/poco/MongoDB/src/UpdateRequest.cpp @@ -20,7 +20,7 @@ namespace MongoDB { UpdateRequest::UpdateRequest(const std::string& collectionName, UpdateRequest::Flags flags): - RequestMessage(MessageHeader::OP_UPDATE), + RequestMessage(MessageHeader::OP_UPDATE), _flags(flags), _fullCollectionName(collectionName), _selector(), diff --git a/docker/test/integration/runner/compose/docker_compose_mongo.yml b/docker/test/integration/runner/compose/docker_compose_mongo.yml index 9a6eae6ca8c..60361e9e98d 100644 --- a/docker/test/integration/runner/compose/docker_compose_mongo.yml +++ b/docker/test/integration/runner/compose/docker_compose_mongo.yml @@ -1,7 +1,7 @@ version: '2.3' services: mongo1: - image: mongo:5.0 + image: mongo:5.1 restart: always environment: MONGO_INITDB_ROOT_USERNAME: root diff --git a/docker/test/integration/runner/compose/docker_compose_mongo_secure.yml b/docker/test/integration/runner/compose/docker_compose_mongo_secure.yml index 193e5d26568..f5b0ffed130 100644 --- a/docker/test/integration/runner/compose/docker_compose_mongo_secure.yml +++ b/docker/test/integration/runner/compose/docker_compose_mongo_secure.yml @@ -1,7 +1,7 @@ version: '2.3' services: mongo1: - image: mongo:3.6 + image: mongo:3.5 restart: always environment: MONGO_INITDB_ROOT_USERNAME: root diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index b7e342f3c80..46910fa9f6a 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -170,7 +170,7 @@ MongoDBDictionarySource::~MongoDBDictionarySource() = default; QueryPipeline MongoDBDictionarySource::loadAll() { - return QueryPipeline(std::make_shared(connection, createCursor(db, collection, sample_block), sample_block, max_block_size)); + return QueryPipeline(std::make_shared(connection, db, collection, Poco::MongoDB::Document{}, sample_block, max_block_size)); } QueryPipeline MongoDBDictionarySource::loadIds(const std::vector & ids) @@ -178,7 +178,7 @@ QueryPipeline MongoDBDictionarySource::loadIds(const std::vector & ids) if (!dict_struct.id) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is required for selective loading"); - auto cursor = createCursor(db, collection, sample_block); + Poco::MongoDB::Document query; /** NOTE: While building array, Poco::MongoDB requires passing of different unused element names, along with values. * In general, Poco::MongoDB is quite inefficient and bulky. @@ -188,9 +188,9 @@ QueryPipeline MongoDBDictionarySource::loadIds(const std::vector & ids) for (const UInt64 id : ids) ids_array->add(DB::toString(id), static_cast(id)); - cursor->query().selector().addNewDocument(dict_struct.id->name).add("$in", ids_array); + query.addNewDocument(dict_struct.id->name).add("$in", ids_array); - return QueryPipeline(std::make_shared(connection, std::move(cursor), sample_block, max_block_size)); + return QueryPipeline(std::make_shared(connection, db, collection, query, sample_block, max_block_size)); } @@ -199,8 +199,7 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con if (!dict_struct.key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is required for selective loading"); - auto cursor = createCursor(db, collection, sample_block); - + Poco::MongoDB::Document query; Poco::MongoDB::Array::Ptr keys_array(new Poco::MongoDB::Array); for (const auto row_idx : requested_rows) @@ -254,9 +253,9 @@ QueryPipeline MongoDBDictionarySource::loadKeys(const Columns & key_columns, con } /// If more than one key we should use $or - cursor->query().selector().add("$or", keys_array); + query.add("$or", keys_array); - return QueryPipeline(std::make_shared(connection, std::move(cursor), sample_block, max_block_size)); + return QueryPipeline(std::make_shared(connection, db, collection, query, sample_block, max_block_size)); } std::string MongoDBDictionarySource::toString() const diff --git a/src/Dictionaries/MongoDBDictionarySource.h b/src/Dictionaries/MongoDBDictionarySource.h index fefcb1bff9f..6d93bc6c090 100644 --- a/src/Dictionaries/MongoDBDictionarySource.h +++ b/src/Dictionaries/MongoDBDictionarySource.h @@ -16,7 +16,6 @@ namespace Util namespace MongoDB { class Connection; - class Cursor; } } diff --git a/src/Processors/Sources/MongoDBSource.cpp b/src/Processors/Sources/MongoDBSource.cpp index 279a842143f..94b9cb7ad64 100644 --- a/src/Processors/Sources/MongoDBSource.cpp +++ b/src/Processors/Sources/MongoDBSource.cpp @@ -3,10 +3,12 @@ #include #include +#include +#include #include #include +#include #include -#include #include #include @@ -365,27 +367,79 @@ namespace } -std::unique_ptr createCursor(const std::string & database, const std::string & collection, const Block & sample_block_to_select) +bool isMongoDBWireProtocolOld(Poco::MongoDB::Connection & connection_) { - auto cursor = std::make_unique(database, collection); + Poco::MongoDB::Database db("config"); + Poco::MongoDB::Document::Ptr doc = db.queryServerHello(connection_); + auto _wireVersion = doc->getInteger("maxWireVersion"); + return _wireVersion < Poco::MongoDB::Database::WireVersion::VER_36; +} + + +MongoDBCursor::MongoDBCursor( + const std::string & database, + const std::string & collection, + const Block & sample_block_to_select, + const Poco::MongoDB::Document & query, + Poco::MongoDB::Connection & connection) + : is_wire_protocol_old(isMongoDBWireProtocolOld(connection)) +{ + Poco::MongoDB::Document projection; /// Looks like selecting _id column is implicit by default. if (!sample_block_to_select.has("_id")) - cursor->query().returnFieldSelector().add("_id", 0); + projection.add("_id", 0); for (const auto & column : sample_block_to_select) - cursor->query().returnFieldSelector().add(column.name, 1); - return cursor; + projection.add(column.name, 1); + + if (is_wire_protocol_old) + { + old_cursor = std::make_unique(database, collection); + old_cursor->query().selector() = query; + old_cursor->query().returnFieldSelector() = projection; + } + else + { + new_cursor = std::make_unique(database, collection); + new_cursor->query().setCommandName(Poco::MongoDB::OpMsgMessage::CMD_FIND); + new_cursor->query().body().addNewDocument("filter") = query; + new_cursor->query().body().addNewDocument("projection") = projection; + } } +Poco::MongoDB::Document::Vector MongoDBCursor::nextDocuments(Poco::MongoDB::Connection & connection) +{ + if (is_wire_protocol_old) + { + auto response = old_cursor->next(connection); + cursorID_ = response.cursorID(); + return std::move(response.documents()); + } + else + { + auto response = new_cursor->next(connection); + cursorID_ = new_cursor->cursorID(); + return std::move(response.documents()); + } +} + +Int64 MongoDBCursor::cursorID() +{ + return cursorID_; +} + + MongoDBSource::MongoDBSource( std::shared_ptr & connection_, - std::unique_ptr cursor_, + const String & database_name_, + const String & collection_name_, + const Poco::MongoDB::Document & query_, const Block & sample_block, UInt64 max_block_size_) : ISource(sample_block.cloneEmpty()) , connection(connection_) - , cursor{std::move(cursor_)} + , cursor(database_name_, collection_name_, sample_block, query_, *connection_) , max_block_size{max_block_size_} { description.init(sample_block); @@ -412,9 +466,9 @@ Chunk MongoDBSource::generate() size_t num_rows = 0; while (num_rows < max_block_size) { - Poco::MongoDB::ResponseMessage & response = cursor->next(*connection); + auto documents = cursor.nextDocuments(*connection); - for (auto & document : response.documents()) + for (auto & document : documents) { if (document->exists("ok") && document->exists("$err") && document->exists("code") && document->getInteger("ok") == 0) @@ -458,7 +512,7 @@ Chunk MongoDBSource::generate() } } - if (response.cursorID() == 0) + if (cursor.cursorID() == 0) { all_read = true; break; diff --git a/src/Processors/Sources/MongoDBSource.h b/src/Processors/Sources/MongoDBSource.h index d4681d2c05f..f816ccfd1c9 100644 --- a/src/Processors/Sources/MongoDBSource.h +++ b/src/Processors/Sources/MongoDBSource.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -14,7 +15,9 @@ namespace Poco namespace MongoDB { class Connection; + class Document; class Cursor; + class OpMsgCursor; } } @@ -30,7 +33,28 @@ struct MongoDBArrayInfo void authenticate(Poco::MongoDB::Connection & connection, const std::string & database, const std::string & user, const std::string & password); -std::unique_ptr createCursor(const std::string & database, const std::string & collection, const Block & sample_block_to_select); +bool isMongoDBWireProtocolOld(Poco::MongoDB::Connection & connection_); + +class MongoDBCursor +{ +public: + MongoDBCursor( + const std::string & database, + const std::string & collection, + const Block & sample_block_to_select, + const Poco::MongoDB::Document & query, + Poco::MongoDB::Connection & connection); + + Poco::MongoDB::Document::Vector nextDocuments(Poco::MongoDB::Connection & connection); + + Int64 cursorID(); + +private: + const bool is_wire_protocol_old; + std::unique_ptr old_cursor; + std::unique_ptr new_cursor; + Int64 cursorID_ = 0; +}; /// Converts MongoDB Cursor to a stream of Blocks class MongoDBSource final : public ISource @@ -38,7 +62,9 @@ class MongoDBSource final : public ISource public: MongoDBSource( std::shared_ptr & connection_, - std::unique_ptr cursor_, + const String & database_name_, + const String & collection_name_, + const Poco::MongoDB::Document & query_, const Block & sample_block, UInt64 max_block_size_); @@ -50,7 +76,7 @@ private: Chunk generate() override; std::shared_ptr connection; - std::unique_ptr cursor; + MongoDBCursor cursor; const UInt64 max_block_size; ExternalResultDescription description; bool all_read = false; diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 63b8c2d00a1..2a1d7e80c07 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -99,6 +99,7 @@ public: , db_name(db_name_) , metadata_snapshot{metadata_snapshot_} , connection(connection_) + , is_wire_protocol_old(isMongoDBWireProtocolOld(*connection_)) { } @@ -107,7 +108,7 @@ public: void consume(Chunk chunk) override { Poco::MongoDB::Database db(db_name); - Poco::MongoDB::Document::Ptr index = new Poco::MongoDB::Document(); + Poco::MongoDB::Document::Vector documents; auto block = getHeader().cloneWithColumns(chunk.detachColumns()); @@ -118,20 +119,35 @@ public: const auto data_types = block.getDataTypes(); const auto data_names = block.getNames(); - std::vector row(num_cols); + documents.reserve(num_rows); + for (const auto i : collections::range(0, num_rows)) { + Poco::MongoDB::Document::Ptr document = new Poco::MongoDB::Document(); + for (const auto j : collections::range(0, num_cols)) { WriteBufferFromOwnString ostr; data_types[j]->getDefaultSerialization()->serializeText(*columns[j], i, ostr, FormatSettings{}); - row[j] = ostr.str(); - index->add(data_names[j], row[j]); + document->add(data_names[j], ostr.str()); } + + documents.push_back(std::move(document)); + } + + if (is_wire_protocol_old) + { + Poco::SharedPtr insert_request = db.createInsertRequest(collection_name); + insert_request->documents() = std::move(documents); + connection->sendRequest(*insert_request); + } + else + { + Poco::SharedPtr insert_request = db.createOpMsgMessage(collection_name); + insert_request->setCommandName(Poco::MongoDB::OpMsgMessage::CMD_INSERT); + insert_request->documents() = std::move(documents); + connection->sendRequest(*insert_request); } - Poco::SharedPtr insert_request = db.createInsertRequest(collection_name); - insert_request->documents().push_back(index); - connection->sendRequest(*insert_request); } private: @@ -139,6 +155,8 @@ private: String db_name; StorageMetadataPtr metadata_snapshot; std::shared_ptr connection; + + const bool is_wire_protocol_old; }; @@ -162,7 +180,7 @@ Pipe StorageMongoDB::read( sample_block.insert({ column_data.type, column_data.name }); } - return Pipe(std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size)); + return Pipe(std::make_shared(connection, database_name, collection_name, Poco::MongoDB::Document{}, sample_block, max_block_size)); } SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */) diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 6ba5520704d..e6e77c64515 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -71,6 +71,39 @@ def test_simple_select(started_cluster): simple_mongo_table.drop() +def test_simple_select_from_view(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + simple_mongo_table_view = db.create_collection( + "simple_table_view", viewOn="simple_table" + ) + + node = started_cluster.instances["node"] + node.query( + "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo2:27017', 'test', 'simple_table_view', 'root', 'clickhouse')" + ) + + assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" + assert ( + node.query("SELECT sum(key) FROM simple_mongo_table") + == str(sum(range(0, 100))) + "\n" + ) + + assert ( + node.query("SELECT data from simple_mongo_table where key = 42") + == hex(42 * 42) + "\n" + ) + node.query("DROP TABLE simple_mongo_table") + simple_mongo_table_view.drop() + simple_mongo_table.drop() + + @pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) def test_arrays(started_cluster): mongo_connection = get_mongo_connection(started_cluster) @@ -411,13 +444,16 @@ def test_simple_insert_select(started_cluster): node.query( "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" ) - node.query("INSERT INTO simple_mongo_table SELECT 1, 'kek'") + node.query( + "INSERT INTO simple_mongo_table SELECT number, 'kek' || toString(number) FROM numbers(10)" + ) assert ( - node.query("SELECT data from simple_mongo_table where key = 1").strip() == "kek" + node.query("SELECT data from simple_mongo_table where key = 7").strip() + == "kek7" ) node.query("INSERT INTO simple_mongo_table(key) SELECT 12") - assert int(node.query("SELECT count() from simple_mongo_table")) == 2 + assert int(node.query("SELECT count() from simple_mongo_table")) == 11 assert ( node.query("SELECT data from simple_mongo_table where key = 12").strip() == "" ) From 98aace14ae943b5308a5b444c9a7c8ec0dd2f903 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Mon, 22 May 2023 09:23:23 +0000 Subject: [PATCH 0507/2223] Add DATE_SECONDS_PER_DAY macro definition to replace the numeric literal 86400 --- src/Common/DateLUTImpl.h | 2 ++ src/Functions/FunctionsConversion.h | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index b40b4d7c65b..551375d1f5c 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -10,6 +10,8 @@ #include +#define DATE_SECONDS_PER_DAY 86400 /// Number of seconds in a day, 60 * 60 * 24 + #define DATE_LUT_MIN_YEAR 1900 /// 1900 since majority of financial organizations consider 1900 as an initial year. #define DATE_LUT_MAX_YEAR 2299 /// Last supported year (complete) #define DATE_LUT_YEARS (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 2e21932d0e2..940585d6d57 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -300,7 +300,7 @@ struct ConvertImpl if constexpr (std::is_same_v && std::is_same_v) vec_to[i] = static_cast(static_cast(vec_from[i])); else if constexpr (std::is_same_v && (std::is_same_v || std::is_same_v)) - vec_to[i] = static_cast(vec_from[i] * 86400); + vec_to[i] = static_cast(vec_from[i] * DATE_SECONDS_PER_DAY); else vec_to[i] = static_cast(vec_from[i]); } From 8bc4a3b2c03129d08911ecfbd3f6630c19012ea7 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 12 May 2023 14:40:17 +0800 Subject: [PATCH 0508/2223] try to reserve hash table size --- src/Interpreters/GraceHashJoin.cpp | 16 +++++++++++---- src/Interpreters/GraceHashJoin.h | 3 ++- src/Interpreters/HashJoin.cpp | 8 +++++--- src/Interpreters/HashJoin.h | 32 ++++++++++++++++++++++++++++-- 4 files changed, 49 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 4a4c69ff473..4ddf147126c 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -571,7 +571,14 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; - hash_join = makeInMemoryJoin(); + size_t prev_keys_num = 0; + if (hash_join) + { + // Use previous hash_join's keys number to estimate next hash_join's size is reasonable. + prev_keys_num = hash_join->getTotalRowCount(); + } + + hash_join = makeInMemoryJoin(prev_keys_num); for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) { @@ -604,9 +611,9 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() return nullptr; } -GraceHashJoin::InMemoryJoinPtr GraceHashJoin::makeInMemoryJoin() +GraceHashJoin::InMemoryJoinPtr GraceHashJoin::makeInMemoryJoin(size_t reserve_num) { - return std::make_unique(table_join, right_sample_block, any_take_last_row); + return std::make_unique(table_join, right_sample_block, any_take_last_row, reserve_num); } Block GraceHashJoin::prepareRightBlock(const Block & block) @@ -646,6 +653,7 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) if (!current_block.rows()) return; } + auto prev_keys_num = hash_join->getTotalRowCount(); hash_join->addJoinedBlock(current_block, /* check_limits = */ false); if (!hasMemoryOverflow(hash_join)) @@ -674,7 +682,7 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) current_block = concatenateBlocks(current_blocks); } - hash_join = makeInMemoryJoin(); + hash_join = makeInMemoryJoin(prev_keys_num); if (current_block.rows() > 0) hash_join->addJoinedBlock(current_block, /* check_limits = */ false); diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index eb39ee09208..ec611f373ed 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -90,7 +90,8 @@ public: private: void initBuckets(); /// Create empty join for in-memory processing. - InMemoryJoinPtr makeInMemoryJoin(); + /// reserve_num for reserving space in hash table. + InMemoryJoinPtr makeInMemoryJoin(size_t reserve_num = 0); /// Add right table block to the @join. Calls @rehash on overflow. void addJoinedBlockImpl(Block block); diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 0af33a8bd20..146b57049a6 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -217,7 +217,7 @@ static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nulla JoinCommon::removeColumnNullability(column); } -HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_sample_block_, bool any_take_last_row_) +HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_sample_block_, bool any_take_last_row_, size_t reserve_num) : table_join(table_join_) , kind(table_join->kind()) , strictness(table_join->strictness()) @@ -302,7 +302,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s } for (auto & maps : data->maps) - dataMapInit(maps); + dataMapInit(maps, reserve_num); } HashJoin::Type HashJoin::chooseMethod(JoinKind kind, const ColumnRawPtrs & key_columns, Sizes & key_sizes) @@ -454,13 +454,15 @@ struct KeyGetterForType using Type = typename KeyGetterForTypeImpl::Type; }; -void HashJoin::dataMapInit(MapsVariant & map) +void HashJoin::dataMapInit(MapsVariant & map, size_t reserve_num) { if (kind == JoinKind::Cross) return; joinDispatchInit(kind, strictness, map); joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.create(data->type); }); + if (reserve_num) + joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); }); } bool HashJoin::empty() const diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 50eda4482bd..9c88f10cb75 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -146,7 +146,7 @@ public: class HashJoin : public IJoin { public: - HashJoin(std::shared_ptr table_join_, const Block & right_sample_block, bool any_take_last_row_ = false); + HashJoin(std::shared_ptr table_join_, const Block & right_sample_block, bool any_take_last_row_ = false, size_t reserve_num = 0); ~HashJoin() override; @@ -217,6 +217,16 @@ public: M(keys256) \ M(hashed) + /// Only for maps using hash table. + #define APPLY_FOR_HASH_JOIN_VARIANTS(M) \ + M(key32) \ + M(key64) \ + M(key_string) \ + M(key_fixed_string) \ + M(keys128) \ + M(keys256) \ + M(hashed) + /// Used for reading from StorageJoin and applying joinGet function #define APPLY_FOR_JOIN_VARIANTS_LIMITED(M) \ @@ -266,6 +276,24 @@ public: } } + void reserve(Type which, size_t num) + { + switch (which) + { + case Type::EMPTY: break; + case Type::CROSS: break; + case Type::key8: break; + case Type::key16: break; + + #define M(NAME) \ + case Type::NAME: NAME->reserve(num); break; + APPLY_FOR_HASH_JOIN_VARIANTS(M) + #undef M + } + + + } + size_t getTotalRowCount(Type which) const { switch (which) @@ -409,7 +437,7 @@ private: /// If set HashJoin instance is not available for modification (addJoinedBlock) TableLockHolder storage_join_lock = nullptr; - void dataMapInit(MapsVariant &); + void dataMapInit(MapsVariant &, size_t); void initRightBlockStructure(Block & saved_block_sample); From 826aa8021aaacd9bfd854da759cc4444db48a7ab Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 16 May 2023 16:38:45 +0800 Subject: [PATCH 0509/2223] fixed: unnecessary hash table allocation --- src/Interpreters/GraceHashJoin.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 4ddf147126c..97fccfb34ed 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -572,14 +572,13 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; size_t prev_keys_num = 0; - if (hash_join) + // If there is only one bucket, don't take this check. + if (hash_join && buckets.size() > 1) { // Use previous hash_join's keys number to estimate next hash_join's size is reasonable. prev_keys_num = hash_join->getTotalRowCount(); } - hash_join = makeInMemoryJoin(prev_keys_num); - for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) { current_bucket = buckets[bucket_idx].get(); @@ -592,6 +591,7 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() continue; } + hash_join = makeInMemoryJoin(prev_keys_num); auto right_reader = current_bucket->startJoining(); size_t num_rows = 0; /// count rows that were written and rehashed while (Block block = right_reader.read()) From 02b04fd9bf57d7393041db92535ae94095c2b0f5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 16 May 2023 18:23:38 +0800 Subject: [PATCH 0510/2223] reuse previous hash table's space directly --- src/Interpreters/GraceHashJoin.cpp | 8 +++----- src/Interpreters/HashJoin.cpp | 25 +++++++++++++++++++++++++ src/Interpreters/HashJoin.h | 23 +++++++++++++++++++++++ 3 files changed, 51 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 97fccfb34ed..e09cf725fef 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -571,7 +571,6 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; - size_t prev_keys_num = 0; // If there is only one bucket, don't take this check. if (hash_join && buckets.size() > 1) { @@ -591,7 +590,7 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() continue; } - hash_join = makeInMemoryJoin(prev_keys_num); + hash_join->clear(); auto right_reader = current_bucket->startJoining(); size_t num_rows = 0; /// count rows that were written and rehashed while (Block block = right_reader.read()) @@ -661,8 +660,7 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) current_block = {}; - auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); - hash_join = nullptr; + const auto & right_blocks = hash_join->getJoinedBlocks(); buckets_snapshot = rehashBuckets(buckets_snapshot.size() * 2); @@ -682,7 +680,7 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) current_block = concatenateBlocks(current_blocks); } - hash_join = makeInMemoryJoin(prev_keys_num); + hash_join->clear(); if (current_block.rows() > 0) hash_join->addJoinedBlock(current_block, /* check_limits = */ false); diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 146b57049a6..3b663d990f0 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -27,6 +27,7 @@ #include #include #include +#include "joinDispatch.h" namespace DB { @@ -2123,10 +2124,34 @@ BlocksList HashJoin::releaseJoinedBlocks(bool restructure) return restored_blocks; } +const BlocksList & HashJoin::getJoinedBlocks() const +{ + return data->blocks; +} + const ColumnWithTypeAndName & HashJoin::rightAsofKeyColumn() const { /// It should be nullable when right side is nullable return savedBlockSample().getByName(table_join->getOnlyClause().key_names_right.back()); } +void HashJoin::clear() +{ + used_flags.clear(); + data->clear(kind, strictness); +} + +void HashJoin::RightTableData::clear(JoinKind kind_, JoinStrictness strictness_) +{ + blocks.clear(); + blocks_nullmaps.clear(); + blocks_allocated_size = 0; + blocks_nullmaps_allocated_size = 0; + for (auto & map : maps) + { + joinDispatch(kind_, strictness_, map, [&](auto, auto, auto & map_) { map_.clear(type); }); + } + +} + } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 9c88f10cb75..43ef4345c1e 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -71,6 +71,8 @@ public: template bool setUsedOnce(const T & f); + + void clear() { flags.clear(); } }; } @@ -276,6 +278,20 @@ public: } } + void clear(Type which) + { + switch (which) + { + case Type::EMPTY: break; + case Type::CROSS: break; + + #define M(NAME) \ + case Type::NAME: NAME->clear(); break; + APPLY_FOR_JOIN_VARIANTS(M) + #undef M + } + } + void reserve(Type which, size_t num) { switch (which) @@ -367,6 +383,8 @@ public: size_t blocks_allocated_size = 0; size_t blocks_nullmaps_allocated_size = 0; + + void clear(JoinKind kind, JoinStrictness strictness); }; using RightTableDataPtr = std::shared_ptr; @@ -382,6 +400,8 @@ public: RightTableDataPtr getJoinedData() const { return data; } BlocksList releaseJoinedBlocks(bool restructure = false); + // Get joined blocks without releasing them + const BlocksList & getJoinedBlocks() const; /// Modify right block (update structure according to sample block) to save it in block list static Block prepareRightBlock(const Block & block, const Block & saved_block_sample_); @@ -394,6 +414,9 @@ public: void debugKeys() const; + // make a clear for reuse. + void clear(); + private: template friend class NotJoinedHash; From 4d24b645f07db9c5772f2d91caf10d69f928ed2d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 17 May 2023 12:26:19 +0800 Subject: [PATCH 0511/2223] fixed: alloca new hash table when bytes is oveflow --- src/Interpreters/GraceHashJoin.cpp | 27 ++++++++++++++++--- .../02275_full_sort_join_long.sql.j2 | 2 +- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index e09cf725fef..c921398b5d2 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -589,8 +589,10 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() bucket_idx); continue; } - - hash_join->clear(); + if (!hash_join) + hash_join = makeInMemoryJoin(); + else + hash_join->clear(); auto right_reader = current_bucket->startJoining(); size_t num_rows = 0; /// count rows that were written and rehashed while (Block block = right_reader.read()) @@ -654,8 +656,10 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) } auto prev_keys_num = hash_join->getTotalRowCount(); hash_join->addJoinedBlock(current_block, /* check_limits = */ false); + size_t current_hash_keys_num = hash_join->getTotalRowCount(); + size_t current_hash_bytes = hash_join->getTotalByteCount(); - if (!hasMemoryOverflow(hash_join)) + if (!hasMemoryOverflow(current_hash_keys_num, current_hash_bytes)) return; current_block = {}; @@ -680,7 +684,22 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) current_block = concatenateBlocks(current_blocks); } - hash_join->clear(); + // If it's a overflow caused by hash table buffer bytes, need to reallocate a new hash table. + if (hasMemoryOverflow(2, current_hash_bytes)) + { + if (hasMemoryOverflow(prev_hash_keys_num, prev_hash_bytes)) + { + hash_join = makeInMemoryJoin(prev_hash_keys_num / 2); + } + else + { + hash_join = makeInMemoryJoin(prev_hash_keys_num); + } + } + else + { + hash_join->clear(); + } if (current_block.rows() > 0) hash_join->addJoinedBlock(current_block, /* check_limits = */ false); diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 0b28fd67050..8652ab746b4 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -25,7 +25,7 @@ INSERT INTO t2 {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} -SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}1M{% else %}0{% endif %}'; +SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}10M{% else %}0{% endif %}'; SELECT '-- {{ join_algorithm }} --'; SET join_algorithm = '{{ join_algorithm }}'; From f33f1e4840e82f16f88c3e9e700854889a66170d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 17 May 2023 14:41:33 +0800 Subject: [PATCH 0512/2223] roll back --- src/Interpreters/GraceHashJoin.cpp | 30 ++++--------------- src/Interpreters/HashJoin.cpp | 25 ---------------- src/Interpreters/HashJoin.h | 23 -------------- .../02275_full_sort_join_long.sql.j2 | 2 +- 4 files changed, 7 insertions(+), 73 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index c921398b5d2..f54ee9d85c7 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -571,6 +571,7 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; + size_t prev_keys_num = 0; // If there is only one bucket, don't take this check. if (hash_join && buckets.size() > 1) { @@ -589,10 +590,8 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() bucket_idx); continue; } - if (!hash_join) - hash_join = makeInMemoryJoin(); - else - hash_join->clear(); + + hash_join = makeInMemoryJoin(prev_keys_num); auto right_reader = current_bucket->startJoining(); size_t num_rows = 0; /// count rows that were written and rehashed while (Block block = right_reader.read()) @@ -656,15 +655,13 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) } auto prev_keys_num = hash_join->getTotalRowCount(); hash_join->addJoinedBlock(current_block, /* check_limits = */ false); - size_t current_hash_keys_num = hash_join->getTotalRowCount(); - size_t current_hash_bytes = hash_join->getTotalByteCount(); - if (!hasMemoryOverflow(current_hash_keys_num, current_hash_bytes)) + if (!hasMemoryOverflow(hash_join)) return; current_block = {}; - const auto & right_blocks = hash_join->getJoinedBlocks(); + auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); buckets_snapshot = rehashBuckets(buckets_snapshot.size() * 2); @@ -684,22 +681,7 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) current_block = concatenateBlocks(current_blocks); } - // If it's a overflow caused by hash table buffer bytes, need to reallocate a new hash table. - if (hasMemoryOverflow(2, current_hash_bytes)) - { - if (hasMemoryOverflow(prev_hash_keys_num, prev_hash_bytes)) - { - hash_join = makeInMemoryJoin(prev_hash_keys_num / 2); - } - else - { - hash_join = makeInMemoryJoin(prev_hash_keys_num); - } - } - else - { - hash_join->clear(); - } + hash_join = makeInMemoryJoin(prev_keys_num); if (current_block.rows() > 0) hash_join->addJoinedBlock(current_block, /* check_limits = */ false); diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 3b663d990f0..146b57049a6 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -27,7 +27,6 @@ #include #include #include -#include "joinDispatch.h" namespace DB { @@ -2124,34 +2123,10 @@ BlocksList HashJoin::releaseJoinedBlocks(bool restructure) return restored_blocks; } -const BlocksList & HashJoin::getJoinedBlocks() const -{ - return data->blocks; -} - const ColumnWithTypeAndName & HashJoin::rightAsofKeyColumn() const { /// It should be nullable when right side is nullable return savedBlockSample().getByName(table_join->getOnlyClause().key_names_right.back()); } -void HashJoin::clear() -{ - used_flags.clear(); - data->clear(kind, strictness); -} - -void HashJoin::RightTableData::clear(JoinKind kind_, JoinStrictness strictness_) -{ - blocks.clear(); - blocks_nullmaps.clear(); - blocks_allocated_size = 0; - blocks_nullmaps_allocated_size = 0; - for (auto & map : maps) - { - joinDispatch(kind_, strictness_, map, [&](auto, auto, auto & map_) { map_.clear(type); }); - } - -} - } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 43ef4345c1e..9c88f10cb75 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -71,8 +71,6 @@ public: template bool setUsedOnce(const T & f); - - void clear() { flags.clear(); } }; } @@ -278,20 +276,6 @@ public: } } - void clear(Type which) - { - switch (which) - { - case Type::EMPTY: break; - case Type::CROSS: break; - - #define M(NAME) \ - case Type::NAME: NAME->clear(); break; - APPLY_FOR_JOIN_VARIANTS(M) - #undef M - } - } - void reserve(Type which, size_t num) { switch (which) @@ -383,8 +367,6 @@ public: size_t blocks_allocated_size = 0; size_t blocks_nullmaps_allocated_size = 0; - - void clear(JoinKind kind, JoinStrictness strictness); }; using RightTableDataPtr = std::shared_ptr; @@ -400,8 +382,6 @@ public: RightTableDataPtr getJoinedData() const { return data; } BlocksList releaseJoinedBlocks(bool restructure = false); - // Get joined blocks without releasing them - const BlocksList & getJoinedBlocks() const; /// Modify right block (update structure according to sample block) to save it in block list static Block prepareRightBlock(const Block & block, const Block & saved_block_sample_); @@ -414,9 +394,6 @@ public: void debugKeys() const; - // make a clear for reuse. - void clear(); - private: template friend class NotJoinedHash; diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 8652ab746b4..0b28fd67050 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -25,7 +25,7 @@ INSERT INTO t2 {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} -SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}10M{% else %}0{% endif %}'; +SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}1M{% else %}0{% endif %}'; SELECT '-- {{ join_algorithm }} --'; SET join_algorithm = '{{ join_algorithm }}'; From 0573d79ff9feb2f6bbe625528f5fb4dc2f2e2ca8 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 22 May 2023 14:42:43 +0800 Subject: [PATCH 0513/2223] update --- src/Interpreters/HashJoin.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 9c88f10cb75..58e47432d41 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -290,8 +290,6 @@ public: APPLY_FOR_HASH_JOIN_VARIANTS(M) #undef M } - - } size_t getTotalRowCount(Type which) const From 85893b1a0b9f3ba1fe52c2e24c57b1aac37d19dc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 May 2023 09:49:16 +0000 Subject: [PATCH 0514/2223] Clarify dropping removal_candidate flag with comment --- src/Interpreters/Cache/FileCache.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 55d7177fd4c..728ecc7b5ab 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -585,7 +585,8 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) ~EvictionCandidates() { - // todo: it looks redundant, - why is it needed? + /// If failed to reserve space, we don't delete the candidates but drop the flag instead + /// so the segments can used again for (const auto & candidate : candidates) candidate->removal_candidate = false; } From 804e5e12ba79c1d6ffac53de206492ac8e25bed5 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 21 May 2023 20:38:26 +0300 Subject: [PATCH 0515/2223] JIT compilation not equals NaN fix --- src/Functions/FunctionsComparison.h | 2 +- .../25337_jit_compare_functions_nan.reference | 7 ++++++ .../25337_jit_compare_functions_nan.sql | 25 +++++++++++++++++++ 3 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/25337_jit_compare_functions_nan.reference create mode 100644 tests/queries/0_stateless/25337_jit_compare_functions_nan.sql diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 2b9298bd7e8..a9c8bbaa183 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -591,7 +591,7 @@ template <> struct CompileOp { static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool /*is_signed*/) { - return x->getType()->isIntegerTy() ? b.CreateICmpNE(x, y) : b.CreateFCmpONE(x, y); + return x->getType()->isIntegerTy() ? b.CreateICmpNE(x, y) : b.CreateFCmpUNE(x, y); } }; diff --git a/tests/queries/0_stateless/25337_jit_compare_functions_nan.reference b/tests/queries/0_stateless/25337_jit_compare_functions_nan.reference new file mode 100644 index 00000000000..0fc42c1b712 --- /dev/null +++ b/tests/queries/0_stateless/25337_jit_compare_functions_nan.reference @@ -0,0 +1,7 @@ +-- +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/25337_jit_compare_functions_nan.sql b/tests/queries/0_stateless/25337_jit_compare_functions_nan.sql new file mode 100644 index 00000000000..61d165139d6 --- /dev/null +++ b/tests/queries/0_stateless/25337_jit_compare_functions_nan.sql @@ -0,0 +1,25 @@ +SET compile_expressions = 1; +SET min_count_to_compile_expression = 0; + +DROP TABLE IF EXISTS test_table_1; +DROP TABLE IF EXISTS test_table_2; + +CREATE TABLE test_table_1 (id UInt32) ENGINE = MergeTree ORDER BY (id); +create table test_table_2 (id UInt32) ENGINE = MergeTree ORDER BY (id); +INSERT INTO test_table_1 VALUES (2); +INSERT INTO test_table_2 VALUES (2); + +select t1.id, t2.id FROM test_table_1 AS t1 RIGHT JOIN test_table_2 AS t2 ON (t1.id = t2.id) +WHERE (acos(t2.id) <> atan(t1.id)) and (not (acos(t2.id) <> atan(t1.id))); + +DROP TABLE test_table_1; +DROP TABLE test_table_2; + +SELECT '--'; + +SELECT (acos(a) <> atan(b)) and (not (acos(a) <> atan(b))) r FROM (SELECT 2 a, 2 b); +SELECT (acos(a) <> atan(b)) and (not (acos(a) <> atan(b))) r FROM (SELECT 2 a, 2 b); +SELECT (acos(a) <> atan(b)) and (not (acos(a) <> atan(b))) r FROM (SELECT 2 a, 2 b); +SELECT (acos(a) <> atan(b)) and (not (acos(a) <> atan(b))) r FROM (SELECT 2 a, 2 b); +SELECT (acos(a) <> atan(b)) and (not (acos(a) <> atan(b))) r FROM (SELECT 2 a, 2 b); +SELECT (acos(a) <> atan(b)) and (not (acos(a) <> atan(b))) r FROM (SELECT 2 a, 2 b); From 01c01bf235886abff07473550feba6d8cfd4c24a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 22 May 2023 13:15:29 +0300 Subject: [PATCH 0516/2223] Fixed tests --- ...ns_nan.reference => 02763_jit_compare_functions_nan.reference} | 0 ...pare_functions_nan.sql => 02763_jit_compare_functions_nan.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{25337_jit_compare_functions_nan.reference => 02763_jit_compare_functions_nan.reference} (100%) rename tests/queries/0_stateless/{25337_jit_compare_functions_nan.sql => 02763_jit_compare_functions_nan.sql} (100%) diff --git a/tests/queries/0_stateless/25337_jit_compare_functions_nan.reference b/tests/queries/0_stateless/02763_jit_compare_functions_nan.reference similarity index 100% rename from tests/queries/0_stateless/25337_jit_compare_functions_nan.reference rename to tests/queries/0_stateless/02763_jit_compare_functions_nan.reference diff --git a/tests/queries/0_stateless/25337_jit_compare_functions_nan.sql b/tests/queries/0_stateless/02763_jit_compare_functions_nan.sql similarity index 100% rename from tests/queries/0_stateless/25337_jit_compare_functions_nan.sql rename to tests/queries/0_stateless/02763_jit_compare_functions_nan.sql From 5c48f9634705abd5b4f3d93a34e4e4a4af2b69d7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 May 2023 12:35:57 +0200 Subject: [PATCH 0517/2223] Fix build --- programs/keeper/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index e5d56023f7b..1f1138f49eb 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -69,6 +69,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/ProtocolServerAdapter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusMetricsWriter.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/waitServersToFinish.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPRequestHandlerFactoryMain.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/ReadHeaders.cpp From 2d0ebba67f013325c6ff5bc0267e906310f2ea64 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 May 2023 12:58:56 +0200 Subject: [PATCH 0518/2223] Better --- src/Interpreters/Cache/FileCache.cpp | 25 ++++++++++--------------- 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 78670bb49e4..65fc489f5ad 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -563,17 +563,9 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) file_segment.key(), file_segment.offset()); } - size_t queue_size = main_priority->getElementsCount(cache_lock); - chassert(queue_size <= main_priority->getElementsLimit()); - /// A file_segment_metadata acquires a LRUQueue iterator on first successful space reservation attempt. auto queue_iterator = file_segment.getQueueIterator(); - if (queue_iterator) - chassert(file_segment.getReservedSize() > 0); - else - queue_size += 1; - - size_t removed_size = 0; + chassert(!queue_iterator || file_segment.getReservedSize() > 0); class EvictionCandidates final : public std::vector { @@ -599,6 +591,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) }; std::unordered_map to_delete; + size_t freeable_space = 0, freeable_count = 0; auto iterate_func = [&](LockedKey & locked_key, FileSegmentMetadataPtr segment_metadata) { @@ -619,8 +612,8 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) it = to_delete.emplace(key, locked_key.getKeyMetadata()).first; it->second.add(segment_metadata); - removed_size += segment_metadata->size(); - --queue_size; + freeable_space += segment_metadata->size(); + freeable_count += 1; return PriorityIterationResult::CONTINUE; } @@ -636,7 +629,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) { auto is_query_priority_overflow = [&] { - const size_t new_size = query_priority->getSize(cache_lock) + size - removed_size; + const size_t new_size = query_priority->getSize(cache_lock) + size - freeable_space; return new_size > query_priority->getSizeLimit(); }; @@ -656,9 +649,11 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) auto is_main_priority_overflow = [&] { /// max_size == 0 means unlimited cache size, - /// max_element_size means unlimited number of cache elements. - return (main_priority->getSizeLimit() != 0 && main_priority->getSize(cache_lock) + size - removed_size > main_priority->getSizeLimit()) - || (main_priority->getElementsLimit() != 0 && queue_size > main_priority->getElementsLimit()); + /// max_element_size == 0 means unlimited number of cache elements. + return (main_priority->getSizeLimit() != 0 + && (main_priority->getSize(cache_lock) + size - freeable_space > main_priority->getSizeLimit())) + || (main_priority->getElementsLimit() != 0 + && (main_priority->getElementsCount(cache_lock) + 1 - freeable_count > main_priority->getElementsLimit())); }; main_priority->iterate( From 9ea0575ff8c7ac04d46ec93d012debf01eda55c5 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 May 2023 11:24:29 +0000 Subject: [PATCH 0519/2223] Update: rest tests which output is differ with enabled analyzer --- .../01655_plan_optimizations.reference | 55 ++++++++++++++++- .../0_stateless/01655_plan_optimizations.sh | 60 ++++++++++++++++--- 2 files changed, 107 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 48d99647b43..9796d2e4f82 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -1,5 +1,4 @@ Too many optimizations applied to query plan -Too many optimizations applied to query plan > sipHash should be calculated after filtration FUNCTION sipHash64 Filter column: equals @@ -27,6 +26,11 @@ COLUMN Const(UInt8) -> notEquals(y, 0) Aggregating Filter Filter +> (analyzer) filter should be pushed down after aggregating, column after aggregation is const +COLUMN Const(UInt8) -> notEquals(y_1, 0_UInt8) +Aggregating +Filter +Filter 0 1 1 1 2 1 2 3 1 @@ -42,6 +46,11 @@ Filter column ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4)) Aggregating Filter column: notEquals(y, 0) +> (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased +Filter column +ALIAS notEquals(s_0, 4_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 4_UInt8)) +Aggregating +Filter column: notEquals(y_1, 0_UInt8) 0 1 1 2 2 3 @@ -56,6 +65,11 @@ Filter column FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2 Aggregating Filter column: notEquals(y, 0) +> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted +Filter column +FUNCTION and(minus(s_0, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 4_UInt8)) UInt8 : 2 +Aggregating +Filter column: notEquals(y_1, 0_UInt8) 0 1 1 2 2 3 @@ -70,6 +84,11 @@ Filter column FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) +> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed +Filter column +FUNCTION and(minus(s_0, 8_UInt8) :: 0, minus(s_0, 4_UInt8) :: 2) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 8_UInt8), minus(s_0, 4_UInt8)) +Aggregating +Filter column: notEquals(y_1, 0_UInt8) 0 1 1 2 2 3 @@ -83,6 +102,11 @@ Filter column ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) Aggregating Filter column: and(notEquals(y, 0), minus(y, 4)) +> (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased +Filter column +ALIAS notEquals(s_0, 8_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 8_UInt8), minus(y_1, 4_UInt8)) +Aggregating +Filter column: and(notEquals(y_1, 0_UInt8), minus(y_1, 4_UInt8)) 0 1 1 2 2 3 @@ -95,11 +119,19 @@ Filter column: and(notEquals(y, 0), minus(y, 4)) Filter column: and(notEquals(y, 2), notEquals(x, 0)) ARRAY JOIN x Filter column: notEquals(y, 2) +> (analyzer) filter is split, one part is filtered before ARRAY JOIN +Filter column: and(notEquals(y_1, 2_UInt8), notEquals(x_0, 0_UInt8)) +ARRAY JOIN x_0 +Filter column: notEquals(y_1, 2_UInt8) 1 3 > filter is pushed down before Distinct Distinct Distinct Filter column: notEquals(y, 2) +> (analyzer) filter is pushed down before Distinct +Distinct +Distinct +Filter column: notEquals(y_1, 2_UInt8) 0 0 0 1 1 0 @@ -108,12 +140,20 @@ Filter column: notEquals(y, 2) Sorting Sorting Filter column: and(notEquals(x, 0), notEquals(y, 0)) +> (analyzer) filter is pushed down before sorting steps +Sorting +Sorting +Filter column: and(notEquals(x_0, 0_UInt8), notEquals(y_1, 0_UInt8)) 1 2 1 1 > filter is pushed down before TOTALS HAVING and aggregating TotalsHaving Aggregating Filter column: notEquals(y, 2) +> (analyzer) filter is pushed down before TOTALS HAVING and aggregating +TotalsHaving +Aggregating +Filter column: notEquals(y_0, 2_UInt8) 0 12 1 15 3 10 @@ -129,12 +169,18 @@ Filter Join Filter column: notEquals(number, 1) Join +> (analyzer) one condition of filter is pushed down before LEFT JOIN +Join +Filter column: notEquals(l.number_0, 1_UInt8) 0 0 3 3 > one condition of filter is pushed down before INNER JOIN Join Filter column: notEquals(number, 1) Join +> (analyzer) one condition of filter is pushed down before INNER JOIN +Join +Filter column: notEquals(l.number_0, 1_UInt8) 3 3 > filter is pushed down before UNION Union @@ -149,5 +195,12 @@ FUNCTION sipHash64 Sorting Expression (Before ORDER BY) FUNCTION plus +> (analyzer) function calculation should be done after sorting and limit (if possible) +> Expression should be divided into two subexpressions and only one of them should be moved after Sorting +Expression ((Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)) [lifted up part])) +FUNCTION sipHash64 +Sorting +Expression ((Before ORDER BY + (Projection + Change column names to column identifiers))) +FUNCTION plus > this query should be executed without throwing an exception 0 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 7c299f9cc26..d68c2c8b414 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -124,11 +124,17 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> filter is split, one part is filtered before ARRAY JOIN" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " explain actions = 1 select x, y from ( select range(number) as x, number + 1 as y from numbers(3) ) array join x where y != 2 and x != 0" | grep -o "Filter column: and(notEquals(y, 2), notEquals(x, 0))\|ARRAY JOIN x\|Filter column: notEquals(y, 2)" +echo "> (analyzer) filter is split, one part is filtered before ARRAY JOIN" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " + explain actions = 1 select x, y from ( + select range(number) as x, number + 1 as y from numbers(3) + ) array join x where y != 2 and x != 0" | + grep -o "Filter column: and(notEquals(y_1, 2_UInt8), notEquals(x_0, 0_UInt8))\|ARRAY JOIN x_0\|Filter column: notEquals(y_1, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select x, y from ( select range(number) as x, number + 1 as y from numbers(3) @@ -148,12 +154,19 @@ $CLICKHOUSE_CLIENT -q " # settings enable_optimize_predicate_expression=0" echo "> filter is pushed down before Distinct" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " explain actions = 1 select x, y from ( select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) ) where y != 2 settings enable_optimize_predicate_expression=0" | grep -o "Distinct\|Filter column: notEquals(y, 2)" +echo "> (analyzer) filter is pushed down before Distinct" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " + explain actions = 1 select x, y from ( + select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) + ) where y != 2 + settings enable_optimize_predicate_expression=0" | + grep -o "Distinct\|Filter column: notEquals(y_1, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select x, y from ( select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) @@ -161,12 +174,19 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> filter is pushed down before sorting steps" -$CLICKHOUSE_CLIENT --convert_query_to_cnf=0 -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " explain actions = 1 select x, y from ( select number % 2 as x, number % 3 as y from numbers(6) order by y desc ) where x != 0 and y != 0 settings enable_optimize_predicate_expression = 0" | grep -o "Sorting\|Filter column: and(notEquals(x, 0), notEquals(y, 0))" +echo "> (analyzer) filter is pushed down before sorting steps" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " + explain actions = 1 select x, y from ( + select number % 2 as x, number % 3 as y from numbers(6) order by y desc + ) where x != 0 and y != 0 + settings enable_optimize_predicate_expression = 0" | + grep -o "Sorting\|Filter column: and(notEquals(x_0, 0_UInt8), notEquals(y_1, 0_UInt8))" $CLICKHOUSE_CLIENT -q " select x, y from ( select number % 2 as x, number % 3 as y from numbers(6) order by y desc @@ -174,12 +194,19 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression = 0" echo "> filter is pushed down before TOTALS HAVING and aggregating" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " explain actions = 1 select * from ( select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals ) where y != 2 settings enable_optimize_predicate_expression=0" | grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(y, 2)" +echo "> (analyzer) filter is pushed down before TOTALS HAVING and aggregating" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " + explain actions = 1 select * from ( + select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals + ) where y != 2 + settings enable_optimize_predicate_expression=0" | + grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(y_0, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select * from ( select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals @@ -197,24 +224,38 @@ $CLICKHOUSE_CLIENT -q " ) where number != 2 settings enable_optimize_predicate_expression=0" echo "> one condition of filter is pushed down before LEFT JOIN" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " explain actions = 1 select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | grep -o "Join\|Filter column: notEquals(number, 1)" +echo "> (analyzer) one condition of filter is pushed down before LEFT JOIN" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " + explain actions = 1 + select number as a, r.b from numbers(4) as l any left join ( + select number + 2 as b from numbers(3) + ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | + grep -o "Join\|Filter column: notEquals(l.number_0, 1_UInt8)" $CLICKHOUSE_CLIENT -q " select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | sort echo "> one condition of filter is pushed down before INNER JOIN" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " explain actions = 1 select number as a, r.b from numbers(4) as l any inner join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | grep -o "Join\|Filter column: notEquals(number, 1)" +echo "> (analyzer) one condition of filter is pushed down before INNER JOIN" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " + explain actions = 1 + select number as a, r.b from numbers(4) as l any inner join ( + select number + 2 as b from numbers(3) + ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | + grep -o "Join\|Filter column: notEquals(l.number_0, 1_UInt8)" $CLICKHOUSE_CLIENT -q " select number as a, r.b from numbers(4) as l any inner join ( select number + 2 as b from numbers(3) @@ -233,7 +274,12 @@ $CLICKHOUSE_CLIENT -q " echo "> function calculation should be done after sorting and limit (if possible)" echo "> Expression should be divided into two subexpressions and only one of them should be moved after Sorting" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " + explain actions = 1 select number as n, sipHash64(n) from numbers(100) order by number + 1 limit 5" | + sed 's/^ *//g' | grep -o "^ *\(Expression (.*Before ORDER BY.*)\|Sorting\|FUNCTION \w\+\)" +echo "> (analyzer) function calculation should be done after sorting and limit (if possible)" +echo "> Expression should be divided into two subexpressions and only one of them should be moved after Sorting" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select number as n, sipHash64(n) from numbers(100) order by number + 1 limit 5" | sed 's/^ *//g' | grep -o "^ *\(Expression (.*Before ORDER BY.*)\|Sorting\|FUNCTION \w\+\)" echo "> this query should be executed without throwing an exception" From d27b88538d745f10892838fb9b6bd4b9f194ffd6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Mon, 22 May 2023 13:41:50 +0200 Subject: [PATCH 0520/2223] Fix grammar Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Interpreters/Cache/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 728ecc7b5ab..3af521026af 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -586,7 +586,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) ~EvictionCandidates() { /// If failed to reserve space, we don't delete the candidates but drop the flag instead - /// so the segments can used again + /// so the segments can be used again for (const auto & candidate : candidates) candidate->removal_candidate = false; } From bde2cf96b135235908b71cc0bf071a175321dc4d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 May 2023 12:24:16 +0000 Subject: [PATCH 0521/2223] Better --- programs/keeper/CMakeLists.txt | 7 +- programs/keeper/Keeper.cpp | 39 +++++--- programs/keeper/Keeper.h | 7 -- src/Coordination/Changelog.cpp | 55 +++++++---- src/Coordination/Changelog.h | 10 +- .../KeeperAsynchronousMetrics.cpp | 6 +- src/Coordination/KeeperAsynchronousMetrics.h | 6 +- src/Coordination/KeeperContext.cpp | 91 +++++++++++++++---- src/Coordination/KeeperContext.h | 16 +++- src/Coordination/KeeperLogStore.cpp | 9 +- src/Coordination/KeeperLogStore.h | 6 +- src/Coordination/KeeperServer.cpp | 21 +---- src/Coordination/KeeperSnapshotManager.cpp | 38 +++----- src/Coordination/KeeperSnapshotManager.h | 15 +-- src/Coordination/KeeperStateMachine.cpp | 2 - src/Coordination/KeeperStateMachine.h | 1 - src/Coordination/KeeperStateManager.cpp | 32 ++++--- src/Coordination/KeeperStateManager.h | 15 +-- .../{Context => Standalone}/Context.cpp | 58 ++++++++++++ .../{Context => Standalone}/Context.h | 9 ++ .../{Context => Standalone}/Settings.cpp | 0 .../ThreadStatusExt.cpp | 0 src/Coordination/TinyContext.cpp | 87 ------------------ src/Coordination/TinyContext.h | 36 -------- src/Interpreters/Context.h | 2 +- 25 files changed, 282 insertions(+), 286 deletions(-) rename src/Coordination/{Context => Standalone}/Context.cpp (80%) rename src/Coordination/{Context => Standalone}/Context.h (88%) rename src/Coordination/{Context => Standalone}/Settings.cpp (100%) rename src/Coordination/{Context => Standalone}/ThreadStatusExt.cpp (100%) delete mode 100644 src/Coordination/TinyContext.cpp delete mode 100644 src/Coordination/TinyContext.h diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index c0c0a6dd1b0..1b5b9e6a7b2 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -50,7 +50,6 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStorage.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperAsynchronousMetrics.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/TinyContext.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/pathUtils.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SessionExpiryQueue.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SummingStateMachine.cpp @@ -157,9 +156,9 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/GraphiteWriter.cpp ${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/Context.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/Settings.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/ThreadStatusExt.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Standalone/Context.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Standalone/Settings.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Standalone/ThreadStatusExt.cpp Keeper.cpp clickhouse-keeper.cpp diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 996c4678450..d5cf61daa6e 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -44,6 +44,8 @@ #include +#include + int mainEntryClickHouseKeeper(int argc, char ** argv) { @@ -280,9 +282,12 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options) BaseDaemon::defineOptions(options); } -struct Keeper::KeeperHTTPContext : public IHTTPContext +namespace { - explicit KeeperHTTPContext(TinyContextPtr context_) + +struct KeeperHTTPContext : public IHTTPContext +{ + explicit KeeperHTTPContext(ContextPtr context_) : context(std::move(context_)) {} @@ -326,12 +331,14 @@ struct Keeper::KeeperHTTPContext : public IHTTPContext return {context->getConfigRef().getInt64("keeper_server.http_send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0}; } - TinyContextPtr context; + ContextPtr context; }; -HTTPContextPtr Keeper::httpContext() +HTTPContextPtr httpContext() { - return std::make_shared(tiny_context); + return std::make_shared(Context::getGlobalContextInstance()); +} + } int Keeper::main(const std::vector & /*args*/) @@ -419,12 +426,14 @@ try global_context->setPath(path); global_context->setRemoteHostFilter(config()); + if (config().has("macros")) + global_context->setMacros(std::make_unique(config(), "macros", log)); + registerDisks(/*global_skip_access_check=*/false); - tiny_context = std::make_shared(); /// This object will periodically calculate some metrics. KeeperAsynchronousMetrics async_metrics( - tiny_context, + global_context, config().getUInt("asynchronous_metrics_update_period_s", 1), [&]() -> std::vector { @@ -449,12 +458,12 @@ try } /// Initialize keeper RAFT. Do nothing if no keeper_server in config. - tiny_context->initializeKeeperDispatcher(/* start_async = */ true); - FourLetterCommandFactory::registerCommands(*tiny_context->getKeeperDispatcher()); + global_context->initializeKeeperDispatcher(/* start_async = */ true); + FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); - auto config_getter = [this] () -> const Poco::Util::AbstractConfiguration & + auto config_getter = [&] () -> const Poco::Util::AbstractConfiguration & { - return tiny_context->getConfigRef(); + return global_context->getConfigRef(); }; auto tcp_receive_timeout = config().getInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC); @@ -476,7 +485,7 @@ try "Keeper (tcp): " + address.toString(), std::make_unique( new KeeperTCPHandlerFactory( - config_getter, tiny_context->getKeeperDispatcher(), + config_getter, global_context->getKeeperDispatcher(), tcp_receive_timeout, tcp_send_timeout, false), server_pool, socket)); }); @@ -494,7 +503,7 @@ try "Keeper with secure protocol (tcp_secure): " + address.toString(), std::make_unique( new KeeperTCPHandlerFactory( - config_getter, tiny_context->getKeeperDispatcher(), + config_getter, global_context->getKeeperDispatcher(), tcp_receive_timeout, tcp_send_timeout, true), server_pool, socket)); #else UNUSED(port); @@ -546,7 +555,7 @@ try [&](ConfigurationPtr config, bool /* initial_loading */) { if (config->has("keeper_server")) - tiny_context->updateKeeperConfiguration(*config); + global_context->updateKeeperConfiguration(*config); }, /* already_loaded = */ false); /// Reload it right now (initial loading) @@ -577,7 +586,7 @@ try else LOG_INFO(log, "Closed connections to Keeper."); - tiny_context->shutdownKeeperDispatcher(); + global_context->shutdownKeeperDispatcher(); /// Wait server pool to avoid use-after-free of destroyed context in the handlers server_pool.joinAll(); diff --git a/programs/keeper/Keeper.h b/programs/keeper/Keeper.h index 8a7724acb85..f889ffa595b 100644 --- a/programs/keeper/Keeper.h +++ b/programs/keeper/Keeper.h @@ -1,9 +1,7 @@ #pragma once #include -#include #include -#include namespace Poco { @@ -68,11 +66,6 @@ protected: std::string getDefaultConfigFileName() const override; private: - TinyContextPtr tiny_context; - - struct KeeperHTTPContext; - HTTPContextPtr httpContext(); - Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const; using CreateServerFunc = std::function; diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 1fda760cab0..852a21c1c45 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -85,17 +86,19 @@ class ChangelogWriter public: ChangelogWriter( std::map & existing_changelogs_, - DiskPtr disk_, + KeeperContextPtr keeper_context_, LogFileSettings log_file_settings_) : existing_changelogs(existing_changelogs_) , log_file_settings(log_file_settings_) - , disk(disk_) + , keeper_context(std::move(keeper_context_)) , log(&Poco::Logger::get("Changelog")) { } void setFile(ChangelogFileDescriptionPtr file_description, WriteMode mode) { + auto disk = getDisk(); + try { if (mode == WriteMode::Append && file_description->expectedEntriesCountInLog() != log_file_settings.rotate_interval) @@ -146,7 +149,7 @@ public: /// There is bug when compressed_buffer has value, file_buf's ownership transfer to compressed_buffer bool isFileSet() const { - return compressed_buffer.get() != nullptr || file_buf.get() != nullptr; + return compressed_buffer != nullptr || file_buf != nullptr; } bool appendRecord(ChangelogRecord && record) @@ -276,7 +279,7 @@ private: const auto * file_buffer = tryGetFileBuffer(); - if (log_file_settings.max_size != 0) + if (log_file_settings.max_size != 0 && isLocalDisk()) { int res = -1; do @@ -354,11 +357,12 @@ private: { initial_file_size = 0; prealloc_done = true; - LOG_WARNING(log, "Could not preallocate space on disk {} using fallocate", disk->getName()); + LOG_WARNING(log, "Could not preallocate space on disk {} using fallocate", getDisk()->getName()); return; } #ifdef OS_LINUX + if (isLocalDisk()) { int res = -1; do @@ -383,6 +387,16 @@ private: prealloc_done = true; } + DiskPtr getDisk() const + { + return keeper_context->getLogDisk(); + } + + bool isLocalDisk() const + { + return dynamic_cast(getDisk().get()) != nullptr; + } + std::map & existing_changelogs; ChangelogFileDescriptionPtr current_file_description{nullptr}; @@ -396,7 +410,7 @@ private: LogFileSettings log_file_settings; - DiskPtr disk; + KeeperContextPtr keeper_context; Poco::Logger * const log; }; @@ -533,18 +547,20 @@ private: }; Changelog::Changelog( - DiskPtr disk_, Poco::Logger * log_, - LogFileSettings log_file_settings) - : disk(disk_) - , changelogs_detached_dir("detached") + LogFileSettings log_file_settings, + KeeperContextPtr keeper_context_) + : changelogs_detached_dir("detached") , rotate_interval(log_file_settings.rotate_interval) , log(log_) , write_operations(std::numeric_limits::max()) , append_completion_queue(std::numeric_limits::max()) + , keeper_context(std::move(keeper_context_)) { /// Load all files in changelog directory + auto disk = getDisk(); + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { if (it->name() == changelogs_detached_dir) @@ -564,7 +580,7 @@ Changelog::Changelog( append_completion_thread = ThreadFromGlobalPool([this] { appendCompletionThread(); }); current_writer = std::make_unique( - existing_changelogs, disk, log_file_settings); + existing_changelogs, keeper_context, log_file_settings); } void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) @@ -636,7 +652,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin break; } - ChangelogReader reader(disk, changelog_description.path); + ChangelogReader reader(getDisk(), changelog_description.path); last_log_read_result = reader.readChangelog(logs, start_to_read_from, log); last_log_read_result->log_start_index = changelog_description.from_log_index; @@ -703,7 +719,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin if (last_log_read_result->last_read_index == 0 || last_log_read_result->error) /// If it's broken log then remove it { LOG_INFO(log, "Removing chagelog {} because it's empty or read finished with error", description->path); - disk->removeFile(description->path); + getDisk()->removeFile(description->path); existing_changelogs.erase(last_log_read_result->log_start_index); std::erase_if(logs, [last_log_read_result](const auto & item) { return item.first >= last_log_read_result->log_start_index; }); } @@ -748,8 +764,15 @@ std::string getCurrentTimestampFolder() } +DiskPtr Changelog::getDisk() const +{ + return keeper_context->getLogDisk(); +} + void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end) { + auto disk = getDisk(); + const auto timestamp_folder = (fs::path(changelogs_detached_dir) / getCurrentTimestampFolder()).generic_string(); for (auto itr = begin; itr != end;) @@ -920,7 +943,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) auto to_remove_itr = existing_changelogs.upper_bound(index); for (auto itr = to_remove_itr; itr != existing_changelogs.end();) { - disk->removeFile(itr->second->path); + getDisk()->removeFile(itr->second->path); itr = existing_changelogs.erase(itr); } } @@ -974,7 +997,7 @@ void Changelog::compact(uint64_t up_to_log_index) { try { - disk->removeFile(itr->second->path); + getDisk()->removeFile(itr->second->path); LOG_INFO(log, "Removed changelog {} because of compaction.", itr->second->path); } catch (Exception & e) @@ -1179,7 +1202,7 @@ void Changelog::cleanLogThread() { try { - disk->removeFile(path); + getDisk()->removeFile(path); LOG_INFO(log, "Removed changelog {} because of compaction.", path); } catch (Exception & e) diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 7204c80a356..6f0c4e45605 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB { @@ -87,9 +88,9 @@ class Changelog { public: Changelog( - DiskPtr disk_, Poco::Logger * log_, - LogFileSettings log_file_settings); + LogFileSettings log_file_settings, + KeeperContextPtr keeper_context_); Changelog(Changelog &&) = delete; @@ -152,6 +153,8 @@ private: /// Pack log_entry into changelog record static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry); + DiskPtr getDisk() const; + /// Currently existing changelogs std::map existing_changelogs; @@ -169,7 +172,6 @@ private: /// Clean useless log files in a background thread void cleanLogThread(); - DiskPtr disk; const String changelogs_detached_dir; const uint64_t rotate_interval; Poco::Logger * log; @@ -223,6 +225,8 @@ private: nuraft::wptr raft_server; + KeeperContextPtr keeper_context; + bool initialized = false; }; diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 2d523a26dcc..1427130b184 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -108,8 +108,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM } KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( - TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), tiny_context(std::move(tiny_context_)) + ContextPtr context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_)) { } @@ -117,7 +117,7 @@ void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values { #if USE_NURAFT { - auto keeper_dispatcher = tiny_context->tryGetKeeperDispatcher(); + auto keeper_dispatcher = context->tryGetKeeperDispatcher(); if (keeper_dispatcher) updateKeeperInformation(*keeper_dispatcher, new_values); } diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index 8fa27336bc5..14092c11c15 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -13,10 +13,10 @@ class KeeperAsynchronousMetrics : public AsynchronousMetrics { public: KeeperAsynchronousMetrics( - TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + ContextPtr context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); private: - TinyContextPtr tiny_context; + ContextPtr context; void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override; }; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 9e504f5aa07..ff3f96f199b 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -20,10 +20,10 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance()); - log_storage_path = getLogsPathFromConfig(config); - snapshot_storage_path = getSnapshotsPathFromConfig(config); + log_storage = getLogsPathFromConfig(config); + snapshot_storage = getSnapshotsPathFromConfig(config); - state_file_path = getStateFilePathFromConfig(config); + state_file_storage = getStatePathFromConfig(config); } KeeperContext::Phase KeeperContext::getServerState() const @@ -51,54 +51,109 @@ void KeeperContext::setDigestEnabled(bool digest_enabled_) digest_enabled = digest_enabled_; } +DiskPtr KeeperContext::getDisk(const Storage & storage) const +{ + if (const auto * storage_disk = std::get_if(&storage)) + return *storage_disk; + + const auto & disk_name = std::get(storage); + + return disk_selector->get(disk_name); +} + +DiskPtr KeeperContext::getLogDisk() const +{ + return getDisk(log_storage); +} + +DiskPtr KeeperContext::getSnapshotsDisk() const +{ + return getDisk(snapshot_storage); +} + +DiskPtr KeeperContext::getStateFileDisk() const +{ + return getDisk(state_file_storage); +} + KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { + const auto create_local_disk = [](const auto & path) + { + if (!fs::exists(path)) + fs::create_directories(path); + + return std::make_shared("LogDisk", path, 0); + }; + /// the most specialized path if (config.has("keeper_server.log_storage_path")) - return std::make_shared("LogDisk", config.getString("keeper_server.log_storage_path"), 0); + return create_local_disk(config.getString("keeper_server.log_storage_path")); if (config.has("keeper_server.log_storage_disk")) return config.getString("keeper_server.log_storage_disk"); if (config.has("keeper_server.storage_path")) - return std::make_shared("LogDisk", std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs", 0); + return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs"); if (standalone_keeper) - return std::make_shared("LogDisk", std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs", 0); + return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs"); else - return std::make_shared("LogDisk", std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs", 0); + return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs"); } -std::string KeeperContext::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) +KeeperContext::Storage KeeperContext::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { + const auto create_local_disk = [](const auto & path) + { + if (!fs::exists(path)) + fs::create_directories(path); + + return std::make_shared("SnapshotDisk", path, 0); + }; + /// the most specialized path if (config.has("keeper_server.snapshot_storage_path")) - return config.getString("keeper_server.snapshot_storage_path"); + return create_local_disk(config.getString("keeper_server.snapshot_storage_path")); + + if (config.has("keeper_server.snapshot_storage_disk")) + return config.getString("keeper_server.snapshot_storage_disk"); if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots"; + return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots"); if (standalone_keeper) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots"; + return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots"); else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots"; + return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots"); } -std::string KeeperContext::getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config) +KeeperContext::Storage KeeperContext::getStatePathFromConfig(const Poco::Util::AbstractConfiguration & config) const { + const auto create_local_disk = [](const auto & path) + { + if (!fs::exists(path)) + fs::create_directories(path); + + return std::make_shared("SnapshotDisk", path, 0); + }; + + if (config.has("keeper_server.state_storage_disk")) + return config.getString("keeper_server.state_storage_disk"); + if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "state"; + return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")}); if (config.has("keeper_server.snapshot_storage_path")) - return std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path() / "state"; + return create_local_disk(std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path()); if (config.has("keeper_server.log_storage_path")) - return std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path() / "state"; + return create_local_disk(std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path()); if (standalone_keeper) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "state"; + return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)}); else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/state"; + return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination"); } } diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 2a215f9d58f..1fc01f12bba 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -31,13 +31,19 @@ public: bool digestEnabled() const; void setDigestEnabled(bool digest_enabled_); + + DiskPtr getLogDisk() const; + DiskPtr getSnapshotsDisk() const; + DiskPtr getStateFileDisk() const; private: /// local disk defined using path or disk name using Storage = std::variant; Storage getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const; - std::string getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config); - std::string getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config); + Storage getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const; + Storage getStatePathFromConfig(const Poco::Util::AbstractConfiguration & config) const; + + DiskPtr getDisk(const Storage & storage) const; Phase server_state{Phase::INIT}; @@ -46,9 +52,9 @@ private: std::shared_ptr disk_selector; - Storage log_storage_path; - Storage snapshot_storage_path; - Storage state_file_path; + Storage log_storage; + Storage snapshot_storage; + Storage state_file_storage; bool standalone_keeper; }; diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index 7c4f76e0180..a9153475dbc 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -6,9 +6,9 @@ namespace DB { -KeeperLogStore::KeeperLogStore(DiskPtr disk_, LogFileSettings log_file_settings) +KeeperLogStore::KeeperLogStore(LogFileSettings log_file_settings, KeeperContextPtr keeper_context) : log(&Poco::Logger::get("KeeperLogStore")) - , changelog(disk_, log, log_file_settings) + , changelog(log, log_file_settings, keeper_context) { if (log_file_settings.force_sync) LOG_INFO(log, "force_sync enabled"); @@ -16,11 +16,6 @@ KeeperLogStore::KeeperLogStore(DiskPtr disk_, LogFileSettings log_file_settings) LOG_INFO(log, "force_sync disabled"); } -KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, LogFileSettings log_file_settings) - : KeeperLogStore(std::make_shared("Keeper-logs", changelogs_path, 0), log_file_settings) -{ -} - uint64_t KeeperLogStore::start_index() const { std::lock_guard lock(changelog_lock); diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 2902de129e7..6e71d8c55cf 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB @@ -13,10 +14,7 @@ namespace DB class KeeperLogStore : public nuraft::log_store { public: - KeeperLogStore(DiskPtr disk_, LogFileSettings log_file_settings); - - /// For gtest - KeeperLogStore(const std::string & changelogs_path, LogFileSettings log_file_settings); + KeeperLogStore(LogFileSettings log_file_settings, KeeperContextPtr keeper_context); /// Read log storage from filesystem starting from last_commited_log_index void init(uint64_t last_commited_log_index, uint64_t logs_to_keep); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index d63593436f4..4cacf566df6 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -122,39 +122,22 @@ KeeperServer::KeeperServer( keeper_context->initialize(config); - //if (!fs::exists(keeper_context->snapshot_storage_path)) - // fs::create_directories(keeper_context->snapshot_storage_path); - auto snapshots_disk = std::make_shared("Keeper-snapshots", "", 0); - state_machine = nuraft::cs_new( responses_queue_, snapshots_queue_, - snapshots_disk, coordination_settings, keeper_context, config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr, commit_callback, checkAndGetSuperdigest(configuration_and_settings_->super_digest)); - //auto state_path = fs::path(keeper_context->state_file_path).parent_path().generic_string(); - //auto state_file_name = fs::path(configuration_and_settings_->state_file_path).filename().generic_string(); - - //if (!fs::exists(state_path)) - // fs::create_directories(state_path); - auto state_disk = std::make_shared("Keeper-state", "", 0); - - //if (!fs::exists(configuration_and_settings_->log_storage_path)) - // fs::create_directories(configuration_and_settings_->log_storage_path); - auto logs_disk = std::make_shared("Keeper-logs", "", 0); - state_manager = nuraft::cs_new( server_id, "keeper_server", - logs_disk, - state_disk, "state", config, - coordination_settings); + coordination_settings, + keeper_context); } /** diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index e1c0c034cff..d47ea475c42 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -508,36 +508,18 @@ KeeperStorageSnapshot::~KeeperStorageSnapshot() } KeeperSnapshotManager::KeeperSnapshotManager( - const std::string & snapshots_path_, size_t snapshots_to_keep_, const KeeperContextPtr & keeper_context_, bool compress_snapshots_zstd_, const std::string & superdigest_, size_t storage_tick_time_) - : KeeperSnapshotManager( - std::make_shared("Keeper-snapshots", snapshots_path_, 0), - snapshots_to_keep_, - keeper_context_, - compress_snapshots_zstd_, - superdigest_, - storage_tick_time_) -{ -} - -KeeperSnapshotManager::KeeperSnapshotManager( - DiskPtr disk_, - size_t snapshots_to_keep_, - const KeeperContextPtr & keeper_context_, - bool compress_snapshots_zstd_, - const std::string & superdigest_, - size_t storage_tick_time_) - : disk(disk_) - , snapshots_to_keep(snapshots_to_keep_) + : snapshots_to_keep(snapshots_to_keep_) , compress_snapshots_zstd(compress_snapshots_zstd_) , superdigest(superdigest_) , storage_tick_time(storage_tick_time_) , keeper_context(keeper_context_) { + auto disk = getDisk(); for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { const auto & name = it->name(); @@ -566,6 +548,7 @@ std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; + auto disk = getDisk(); auto plain_buf = disk->writeFile(tmp_snapshot_file_name); copyData(reader, *plain_buf); plain_buf->sync(); @@ -589,7 +572,7 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff } catch (const DB::Exception &) { - disk->removeFile(latest_itr->second); + getDisk()->removeFile(latest_itr->second); existing_snapshots.erase(latest_itr->first); tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -602,7 +585,7 @@ nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFrom { const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx); WriteBufferFromNuraftBuffer writer; - auto reader = disk->readFile(snapshot_path); + auto reader = getDisk()->readFile(snapshot_path); copyData(*reader, writer); return writer.getBuffer(); } @@ -664,6 +647,11 @@ SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() return deserializeSnapshotFromBuffer(buffer); } +DiskPtr KeeperSnapshotManager::getDisk() const +{ + return keeper_context->getSnapshotsDisk(); +} + void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() { while (existing_snapshots.size() > snapshots_to_keep) @@ -675,7 +663,7 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) auto itr = existing_snapshots.find(log_idx); if (itr == existing_snapshots.end()) throw Exception(ErrorCodes::UNKNOWN_SNAPSHOT, "Unknown snapshot with log index {}", log_idx); - disk->removeFile(itr->second); + getDisk()->removeFile(itr->second); existing_snapshots.erase(itr); } @@ -685,7 +673,7 @@ std::pair KeeperSnapshotManager::serializeSnapshot auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - auto writer = disk->writeFile(tmp_snapshot_file_name); + auto writer = getDisk()->writeFile(tmp_snapshot_file_name); std::unique_ptr compressed_writer; if (compress_snapshots_zstd) compressed_writer = wrapWriteBufferWithCompressionMethod(std::move(writer), CompressionMethod::Zstd, 3); @@ -700,7 +688,7 @@ std::pair KeeperSnapshotManager::serializeSnapshot try { - disk->moveFile(tmp_snapshot_file_name, snapshot_file_name); + getDisk()->moveFile(tmp_snapshot_file_name, snapshot_file_name); } catch (fs::filesystem_error & e) { diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 9babad9ed98..7b1129018d8 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -99,16 +99,6 @@ class KeeperSnapshotManager { public: KeeperSnapshotManager( - DiskPtr disk_, - size_t snapshots_to_keep_, - const KeeperContextPtr & keeper_context_, - bool compress_snapshots_zstd_ = true, - const std::string & superdigest_ = "", - size_t storage_tick_time_ = 500); - - /// For gtest - KeeperSnapshotManager( - const std::string & snapshots_path_, size_t snapshots_to_keep_, const KeeperContextPtr & keeper_context_, bool compress_snapshots_zstd_ = true, @@ -157,7 +147,7 @@ public: try { - if (disk->exists(path)) + if (getDisk()->exists(path)) return path; } catch (...) @@ -170,11 +160,12 @@ public: private: void removeOutdatedSnapshotsIfNeeded(); + DiskPtr getDisk() const; + /// Checks first 4 buffer bytes to became sure that snapshot compressed with /// ZSTD codec. static bool isZstdCompressed(nuraft::ptr buffer); - DiskPtr disk; /// How many snapshots to keep before remove const size_t snapshots_to_keep; /// All existing snapshots in our path (log_index -> path) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index a7c845e5017..69c15db51da 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -43,7 +43,6 @@ namespace KeeperStateMachine::KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - DiskPtr disk_, const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, @@ -52,7 +51,6 @@ KeeperStateMachine::KeeperStateMachine( : commit_callback(commit_callback_) , coordination_settings(coordination_settings_) , snapshot_manager( - disk_, coordination_settings->snapshots_to_keep, keeper_context_, coordination_settings->compress_snapshots_with_zstd_format, diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 1fe4db4de68..afe11150f36 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -26,7 +26,6 @@ public: KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - DiskPtr disk_, const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index bcd5ce89e49..352e67e7c43 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -212,12 +212,14 @@ KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfigur return result; } -KeeperStateManager::KeeperStateManager( - int server_id_, const std::string & host, int port, const std::string & logs_path, const std::string & state_file_path) +KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, KeeperContextPtr keeper_context_) : my_server_id(server_id_) , secure(false) - , log_store(nuraft::cs_new(std::make_shared("Keeper-logs", logs_path, 0), LogFileSettings{.force_sync =false, .compress_logs = false, .rotate_interval = 5000})) - , server_state_file_name(fs::path(state_file_path).filename().generic_string()) + , log_store(nuraft::cs_new( + LogFileSettings{.force_sync = false, .compress_logs = false, .rotate_interval = 5000}, + keeper_context_)) + , server_state_file_name("state") + , keeper_context(keeper_context_) , logger(&Poco::Logger::get("KeeperStateManager")) { auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); @@ -230,17 +232,15 @@ KeeperStateManager::KeeperStateManager( KeeperStateManager::KeeperStateManager( int my_server_id_, const std::string & config_prefix_, - DiskPtr log_disk_, - DiskPtr state_disk_, const std::string & server_state_file_name_, const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings) + const CoordinationSettingsPtr & coordination_settings, + KeeperContextPtr keeper_context_) : my_server_id(my_server_id_) , secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false)) , config_prefix(config_prefix_) , configuration_wrapper(parseServersConfiguration(config, false)) , log_store(nuraft::cs_new( - log_disk_, LogFileSettings { .force_sync = coordination_settings->force_sync, @@ -248,9 +248,10 @@ KeeperStateManager::KeeperStateManager( .rotate_interval = coordination_settings->rotate_log_storage_interval, .max_size = coordination_settings->max_log_file_size, .overallocate_size = coordination_settings->log_file_overallocate_size - })) - , disk(state_disk_) + }, + keeper_context_)) , server_state_file_name(server_state_file_name_) + , keeper_context(keeper_context_) , logger(&Poco::Logger::get("KeeperStateManager")) { } @@ -299,6 +300,11 @@ const String & KeeperStateManager::getOldServerStatePath() return old_path; } +DiskPtr KeeperStateManager::getStateFileDisk() const +{ + return keeper_context->getStateFileDisk(); +} + namespace { enum ServerStateVersion : uint8_t @@ -314,6 +320,8 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state) { const auto & old_path = getOldServerStatePath(); + auto disk = getStateFileDisk(); + if (disk->exists(server_state_file_name)) disk->moveFile(server_state_file_name, old_path); @@ -338,7 +346,9 @@ nuraft::ptr KeeperStateManager::read_state() { const auto & old_path = getOldServerStatePath(); - const auto try_read_file = [this](const auto & path) -> nuraft::ptr + auto disk = getStateFileDisk(); + + const auto try_read_file = [&](const auto & path) -> nuraft::ptr { try { diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index d8369100d1c..f24f0c2b1e5 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -39,19 +39,17 @@ public: KeeperStateManager( int server_id_, const std::string & config_prefix_, - DiskPtr logs_disk_, - DiskPtr state_disk_, - const std::string & state_file_path, + const std::string & server_state_file_name_, const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings); + const CoordinationSettingsPtr & coordination_settings, + KeeperContextPtr keeper_context_); /// Constructor for tests KeeperStateManager( int server_id_, const std::string & host, int port, - const std::string & logs_path, - const std::string & state_file_path); + KeeperContextPtr keeper_context_); void loadLogStore(uint64_t last_commited_index, uint64_t logs_to_keep); @@ -114,6 +112,8 @@ public: private: const String & getOldServerStatePath(); + DiskPtr getStateFileDisk() const; + /// Wrapper struct for Keeper cluster config. We parse this /// info from XML files. struct KeeperConfigurationWrapper @@ -137,9 +137,10 @@ private: nuraft::ptr log_store; - DiskPtr disk; const String server_state_file_name; + KeeperContextPtr keeper_context; + Poco::Logger * logger; public: diff --git a/src/Coordination/Context/Context.cpp b/src/Coordination/Standalone/Context.cpp similarity index 80% rename from src/Coordination/Context/Context.cpp rename to src/Coordination/Standalone/Context.cpp index aeb4e405938..667fabf4d66 100644 --- a/src/Coordination/Context/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -38,6 +38,9 @@ struct ContextSharedPart : boost::noncopyable /// For access of most of shared objects. Recursive mutex. mutable std::recursive_mutex mutex; + mutable std::mutex keeper_dispatcher_mutex; + mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); + ServerSettings server_settings; String path; /// Path to the data directory, with a slash at the end. @@ -129,6 +132,11 @@ MultiVersion::Version Context::getMacros() const return shared->macros.get(); } +void Context::setMacros(std::unique_ptr && macros) +{ + shared->macros.set(std::move(macros)); +} + BackgroundSchedulePool & Context::getSchedulePool() const { auto lock = getLock(); @@ -256,4 +264,54 @@ ReadSettings Context::getReadSettings() const return ReadSettings{}; } +void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) const +{ + const auto & config_ref = getConfigRef(); + + std::lock_guard lock(shared->keeper_dispatcher_mutex); + + if (shared->keeper_dispatcher) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); + + if (config_ref.has("keeper_server")) + { + shared->keeper_dispatcher = std::make_shared(); + shared->keeper_dispatcher->initialize(config_ref, true, start_async, getMacros()); + } +} + +std::shared_ptr Context::getKeeperDispatcher() const +{ + std::lock_guard lock(shared->keeper_dispatcher_mutex); + if (!shared->keeper_dispatcher) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests"); + + return shared->keeper_dispatcher; +} + +std::shared_ptr Context::tryGetKeeperDispatcher() const +{ + std::lock_guard lock(shared->keeper_dispatcher_mutex); + return shared->keeper_dispatcher; +} + +void Context::shutdownKeeperDispatcher() const +{ + std::lock_guard lock(shared->keeper_dispatcher_mutex); + if (shared->keeper_dispatcher) + { + shared->keeper_dispatcher->shutdown(); + shared->keeper_dispatcher.reset(); + } +} + +void Context::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::AbstractConfiguration & config_) +{ + std::lock_guard lock(shared->keeper_dispatcher_mutex); + if (!shared->keeper_dispatcher) + return; + + shared->keeper_dispatcher->updateConfiguration(getConfigRef(), getMacros()); +} + } diff --git a/src/Coordination/Context/Context.h b/src/Coordination/Standalone/Context.h similarity index 88% rename from src/Coordination/Context/Context.h rename to src/Coordination/Standalone/Context.h index 683209d942e..c2eee981aaa 100644 --- a/src/Coordination/Context/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -2,6 +2,8 @@ #include +#include + #include #include @@ -85,6 +87,7 @@ public: void setPath(const String & path); MultiVersion::Version getMacros() const; + void setMacros(std::unique_ptr && macros); BackgroundSchedulePool & getSchedulePool() const; @@ -106,6 +109,12 @@ public: ThrottlerPtr getLocalWriteThrottler() const; ReadSettings getReadSettings() const; + + std::shared_ptr getKeeperDispatcher() const; + std::shared_ptr tryGetKeeperDispatcher() const; + void initializeKeeperDispatcher(bool start_async) const; + void shutdownKeeperDispatcher() const; + void updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config); }; } diff --git a/src/Coordination/Context/Settings.cpp b/src/Coordination/Standalone/Settings.cpp similarity index 100% rename from src/Coordination/Context/Settings.cpp rename to src/Coordination/Standalone/Settings.cpp diff --git a/src/Coordination/Context/ThreadStatusExt.cpp b/src/Coordination/Standalone/ThreadStatusExt.cpp similarity index 100% rename from src/Coordination/Context/ThreadStatusExt.cpp rename to src/Coordination/Standalone/ThreadStatusExt.cpp diff --git a/src/Coordination/TinyContext.cpp b/src/Coordination/TinyContext.cpp deleted file mode 100644 index 47b0a48dcda..00000000000 --- a/src/Coordination/TinyContext.cpp +++ /dev/null @@ -1,87 +0,0 @@ -#include - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -void TinyContext::setConfig(const ConfigurationPtr & config_) -{ - std::lock_guard lock(keeper_dispatcher_mutex); - config = config_; -} - -const Poco::Util::AbstractConfiguration & TinyContext::getConfigRef() const -{ - std::lock_guard lock(keeper_dispatcher_mutex); - return config ? *config : Poco::Util::Application::instance().config(); -} - - -void TinyContext::initializeKeeperDispatcher([[maybe_unused]] bool start_async) const -{ - const auto & config_ref = getConfigRef(); - - std::lock_guard lock(keeper_dispatcher_mutex); - - if (keeper_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); - - if (config_ref.has("keeper_server")) - { - keeper_dispatcher = std::make_shared(); - - MultiVersion::Version macros; - - if (config_ref.has("macros")) - macros = std::make_unique(config_ref, "macros", &Poco::Logger::get("TinyContext")); - keeper_dispatcher->initialize(config_ref, true, start_async, macros); - } -} - -std::shared_ptr TinyContext::getKeeperDispatcher() const -{ - std::lock_guard lock(keeper_dispatcher_mutex); - if (!keeper_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests"); - - return keeper_dispatcher; -} - -std::shared_ptr TinyContext::tryGetKeeperDispatcher() const -{ - std::lock_guard lock(keeper_dispatcher_mutex); - return keeper_dispatcher; -} - -void TinyContext::shutdownKeeperDispatcher() const -{ - std::lock_guard lock(keeper_dispatcher_mutex); - if (keeper_dispatcher) - { - keeper_dispatcher->shutdown(); - keeper_dispatcher.reset(); - } -} - -void TinyContext::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::AbstractConfiguration & config_) -{ - std::lock_guard lock(keeper_dispatcher_mutex); - if (!keeper_dispatcher) - return; - - MultiVersion::Version macros; - - if (config_.has("macros")) - macros = std::make_unique(config_, "macros", &Poco::Logger::get("TinyContext")); - - keeper_dispatcher->updateConfiguration(config_, macros); -} - -} diff --git a/src/Coordination/TinyContext.h b/src/Coordination/TinyContext.h deleted file mode 100644 index b966d445004..00000000000 --- a/src/Coordination/TinyContext.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once -#include -#include - -#include -#include - -namespace DB -{ - -class KeeperDispatcher; - -class TinyContext : public std::enable_shared_from_this -{ -public: - std::shared_ptr getKeeperDispatcher() const; - std::shared_ptr tryGetKeeperDispatcher() const; - void initializeKeeperDispatcher(bool start_async) const; - void shutdownKeeperDispatcher() const; - void updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config); - - using ConfigurationPtr = Poco::AutoPtr; - - void setConfig(const ConfigurationPtr & config); - const Poco::Util::AbstractConfiguration & getConfigRef() const; - -private: - mutable std::mutex keeper_dispatcher_mutex; - mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); - - ConfigurationPtr config TSA_GUARDED_BY(keeper_dispatcher_mutex); -}; - -using TinyContextPtr = std::shared_ptr; - -} diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b2aee8d9ec0..0ec39f18757 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1237,6 +1237,6 @@ struct HTTPContext : public IHTTPContext #else -#include +#include #endif From ef09ed711738c934bfc82fb2b2090ebf37fc1ebb Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 19 May 2023 17:47:15 +0000 Subject: [PATCH 0522/2223] Fix assert in SpanHolder::finish() with fibers attempt 2 --- src/Common/AsyncTaskExecutor.cpp | 26 ++--- src/Common/AsyncTaskExecutor.h | 43 ------- src/Common/Fiber.h | 139 ++++++++++++++++++++++- src/Common/OpenTelemetryTraceContext.cpp | 43 ++++--- 4 files changed, 167 insertions(+), 84 deletions(-) diff --git a/src/Common/AsyncTaskExecutor.cpp b/src/Common/AsyncTaskExecutor.cpp index 68af535b22a..757901093eb 100644 --- a/src/Common/AsyncTaskExecutor.cpp +++ b/src/Common/AsyncTaskExecutor.cpp @@ -3,18 +3,11 @@ namespace DB { -thread_local FiberInfo current_fiber_info; - AsyncTaskExecutor::AsyncTaskExecutor(std::unique_ptr task_) : task(std::move(task_)) { createFiber(); } -FiberInfo AsyncTaskExecutor::getCurrentFiberInfo() -{ - return current_fiber_info; -} - void AsyncTaskExecutor::resume() { if (routine_is_finished) @@ -38,10 +31,7 @@ void AsyncTaskExecutor::resume() void AsyncTaskExecutor::resumeUnlocked() { - auto parent_fiber_info = current_fiber_info; - current_fiber_info = FiberInfo{&fiber, &parent_fiber_info}; - fiber = std::move(fiber).resume(); - current_fiber_info = parent_fiber_info; + fiber.resume(); } void AsyncTaskExecutor::cancel() @@ -69,27 +59,27 @@ struct AsyncTaskExecutor::Routine struct AsyncCallback { AsyncTaskExecutor & executor; - Fiber & fiber; + Fiber::Impl & fiber_impl; void operator()(int fd, Poco::Timespan timeout, AsyncEventTimeoutType type, const std::string & desc, uint32_t events) { executor.processAsyncEvent(fd, timeout, type, desc, events); - fiber = std::move(fiber).resume(); + fiber_impl = std::move(fiber_impl).resume(); executor.clearAsyncEvent(); } }; struct ResumeCallback { - Fiber & fiber; + Fiber::Impl & fiber_impl; void operator()() { - fiber = std::move(fiber).resume(); + fiber_impl = std::move(fiber_impl).resume(); } }; - Fiber operator()(Fiber && sink) + Fiber::Impl operator()(Fiber::Impl && sink) { auto async_callback = AsyncCallback{executor, sink}; auto suspend_callback = ResumeCallback{sink}; @@ -116,12 +106,12 @@ struct AsyncTaskExecutor::Routine void AsyncTaskExecutor::createFiber() { - fiber = boost::context::fiber(std::allocator_arg_t(), fiber_stack, Routine{*this}); + fiber = Fiber(fiber_stack, Routine{*this}); } void AsyncTaskExecutor::destroyFiber() { - boost::context::fiber to_destroy = std::move(fiber); + Fiber to_destroy = std::move(fiber); } String getSocketTimeoutExceededMessageByTimeoutType(AsyncEventTimeoutType type, Poco::Timespan timeout, const String & socket_description) diff --git a/src/Common/AsyncTaskExecutor.h b/src/Common/AsyncTaskExecutor.h index 55dc2913c13..b71a5820a4e 100644 --- a/src/Common/AsyncTaskExecutor.h +++ b/src/Common/AsyncTaskExecutor.h @@ -80,7 +80,6 @@ public: }; #endif - static FiberInfo getCurrentFiberInfo(); protected: /// Method that is called in resume() before actual fiber resuming. /// If it returns false, resume() will return immediately without actual fiber resuming. @@ -124,48 +123,6 @@ private: std::unique_ptr task; }; -/// Simple implementation for fiber local variable. -template -struct FiberLocal -{ -public: - FiberLocal() - { - /// Initialize main instance for this thread. Instances for fibers will inherit it, - /// (it's needed because main instance could be changed before creating fibers - /// and changes should be visible in fibers). - data[nullptr] = T(); - } - - T & operator*() - { - return get(); - } - - T * operator->() - { - return &get(); - } - -private: - T & get() - { - return getInstanceForFiber(AsyncTaskExecutor::getCurrentFiberInfo()); - } - - T & getInstanceForFiber(FiberInfo info) - { - auto it = data.find(info.fiber); - /// If it's the first request, we need to initialize instance for the fiber - /// using instance from parent fiber or main thread that created fiber. - if (it == data.end()) - it = data.insert({info.fiber, getInstanceForFiber(*info.parent_fiber_info)}).first; - return it->second; - } - - std::unordered_map data; -}; - String getSocketTimeoutExceededMessageByTimeoutType(AsyncEventTimeoutType type, Poco::Timespan timeout, const String & socket_description); } diff --git a/src/Common/Fiber.h b/src/Common/Fiber.h index 11549c6cd93..f8b0c6202f5 100644 --- a/src/Common/Fiber.h +++ b/src/Common/Fiber.h @@ -4,4 +4,141 @@ #include #include -using Fiber = boost::context::fiber; +/// Class wrapper for boost::context::fiber. +/// It tracks current executing fiber for thread and +/// supports storing fiber-specific data +/// that will be destroyed on fiber destructor. +class Fiber +{ +public: + using CleanUpFn = std::function; + using Impl = boost::context::fiber; + + template< typename StackAlloc, typename Fn> + Fiber(StackAlloc && salloc, Fn && fn) : impl(std::allocator_arg_t(), std::forward(salloc), std::forward(fn)) + { + } + + Fiber() = default; + + Fiber(Fiber && other) = default; + Fiber & operator=(Fiber && other) = default; + + Fiber(const Fiber &) = delete; + Fiber & operator =(const Fiber &) = delete; + + ~Fiber() + { + for (auto & [_, data] : local_data) + data.cleanup_fn(data.ptr); + } + + explicit operator bool() const + { + return impl.operator bool(); + } + + void resume() + { + /// Update information about current executing fiber. + auto & current_fiber_info = getCurrentFiberInfo(); + auto parent_fiber_info = current_fiber_info; + current_fiber_info = FiberInfo{this, &parent_fiber_info}; + impl = std::move(impl).resume(); + /// Restore current fiber info. + current_fiber_info = parent_fiber_info; + } + + /// Set pointer to fiber-specific data, it will be stored in hash-map + /// using provided key and cleaned on fiber destructor using provided + /// cleanup function. + void setLocalData(void * key, void * ptr, CleanUpFn cleanup_fn) + { + local_data[key] = FiberLocalData{ptr, cleanup_fn}; + } + + /// Get pointer to fiber-specific data by key. + /// If no data was stored by this key, return nullptr. + void * getLocalData(void * key) + { + return local_data[key].ptr; + } + + struct FiberInfo + { + Fiber * fiber = nullptr; + FiberInfo * parent_info = nullptr; + }; + + static FiberInfo & getCurrentFiberInfo() + { + thread_local static FiberInfo current_fiber_info; + return current_fiber_info; + } + +private: + struct FiberLocalData + { + void * ptr; + CleanUpFn cleanup_fn; + }; + + Impl impl; + std::unordered_map local_data; +}; + +/// Implementation for fiber local variable. +/// If we are not in fiber, it returns thread local data. +/// If we are in fiber, it returns fiber local data. +/// Fiber local data is destroyed in Fiber destructor. +/// On first request, fiber local data is copied from parent +/// fiber data or from current thread data if there is no parent fiber. +template +class FiberLocal +{ +public: + T & operator*() + { + return get(); + } + + T * operator->() + { + return &get(); + } + +private: + friend Fiber; + + T & get() + { + return getInstanceForFiber(Fiber::getCurrentFiberInfo()); + } + + T & getInstanceForFiber(const Fiber::FiberInfo & fiber_info) + { + /// If it's not a fiber, return thread local instance. + if (!fiber_info.fiber) + return getThreadLocalInstance(); + + T * ptr = static_cast(fiber_info.fiber->getLocalData(this)); + /// If it's the first request, we need to initialize instance for the fiber + /// using instance from parent fiber or main thread that executes this fiber. + if (!ptr) + { + auto parent_instance = getInstanceForFiber(*fiber_info.parent_info); + /// Crete new object and store pointer inside Fiber, so it will be destroyed in Fiber destructor. + ptr = new T(parent_instance); + fiber_info.fiber->setLocalData(this, ptr, [](void * to_delete){ delete static_cast(to_delete); }); + } + + return *ptr; + } + + T & getThreadLocalInstance() + { + static thread_local T thread_local_instance; + return thread_local_instance; + } +}; + diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 40d06e71456..515c8f346df 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -14,9 +14,8 @@ namespace DB namespace OpenTelemetry { -///// This code can be executed inside several fibers in one thread, -///// we should use fiber local tracing context. -thread_local FiberLocal current_fiber_trace_context; +/// This code can be executed inside fibers, we should use fiber local tracing context. +FiberLocal current_trace_context; bool Span::addAttribute(std::string_view name, UInt64 value) noexcept { @@ -108,7 +107,7 @@ bool Span::addAttributeImpl(std::string_view name, std::string_view value) noexc SpanHolder::SpanHolder(std::string_view _operation_name, SpanKind _kind) { - if (!current_fiber_trace_context->isTraceEnabled()) + if (!current_trace_context->isTraceEnabled()) { return; } @@ -116,8 +115,8 @@ SpanHolder::SpanHolder(std::string_view _operation_name, SpanKind _kind) /// Use try-catch to make sure the ctor is exception safe. try { - this->trace_id = current_fiber_trace_context->trace_id; - this->parent_span_id = current_fiber_trace_context->span_id; + this->trace_id = current_trace_context->trace_id; + this->parent_span_id = current_trace_context->span_id; this->span_id = thread_local_rng(); // create a new id for this span this->operation_name = _operation_name; this->kind = _kind; @@ -136,7 +135,7 @@ SpanHolder::SpanHolder(std::string_view _operation_name, SpanKind _kind) } /// Set current span as parent of other spans created later on this thread. - current_fiber_trace_context->span_id = this->span_id; + current_trace_context->span_id = this->span_id; } void SpanHolder::finish() noexcept @@ -145,12 +144,12 @@ void SpanHolder::finish() noexcept return; // First of all, restore old value of current span. - assert(current_fiber_trace_context->span_id == span_id); - current_fiber_trace_context->span_id = parent_span_id; + assert(current_trace_context->span_id == span_id); + current_trace_context->span_id = parent_span_id; try { - auto log = current_fiber_trace_context->span_log.lock(); + auto log = current_trace_context->span_log.lock(); /// The log might be disabled, check it before use if (log) @@ -273,7 +272,7 @@ void TracingContext::serialize(WriteBuffer & buf) const const TracingContextOnThread & CurrentContext() { - return *current_fiber_trace_context; + return *current_trace_context; } void TracingContextOnThread::reset() noexcept @@ -295,7 +294,7 @@ TracingContextHolder::TracingContextHolder( /// If any exception is raised during the construction, the tracing is not enabled on current thread. try { - if (current_fiber_trace_context->isTraceEnabled()) + if (current_trace_context->isTraceEnabled()) { /// /// This is not the normal case, @@ -308,15 +307,15 @@ TracingContextHolder::TracingContextHolder( /// So this branch ensures this class can be instantiated multiple times on one same thread safely. /// this->is_context_owner = false; - this->root_span.trace_id = current_fiber_trace_context->trace_id; - this->root_span.parent_span_id = current_fiber_trace_context->span_id; + this->root_span.trace_id = current_trace_context->trace_id; + this->root_span.parent_span_id = current_trace_context->span_id; this->root_span.span_id = thread_local_rng(); this->root_span.operation_name = _operation_name; this->root_span.start_time_us = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); /// Set the root span as parent of other spans created on current thread - current_fiber_trace_context->span_id = this->root_span.span_id; + current_trace_context->span_id = this->root_span.span_id; return; } @@ -360,10 +359,10 @@ TracingContextHolder::TracingContextHolder( } /// Set up trace context on current thread only when the root span is successfully initialized. - *current_fiber_trace_context = _parent_trace_context; - current_fiber_trace_context->span_id = this->root_span.span_id; - current_fiber_trace_context->trace_flags = TRACE_FLAG_SAMPLED; - current_fiber_trace_context->span_log = _span_log; + *current_trace_context = _parent_trace_context; + current_trace_context->span_id = this->root_span.span_id; + current_trace_context->trace_flags = TRACE_FLAG_SAMPLED; + current_trace_context->span_log = _span_log; } TracingContextHolder::~TracingContextHolder() @@ -375,7 +374,7 @@ TracingContextHolder::~TracingContextHolder() try { - auto shared_span_log = current_fiber_trace_context->span_log.lock(); + auto shared_span_log = current_trace_context->span_log.lock(); if (shared_span_log) { try @@ -406,11 +405,11 @@ TracingContextHolder::~TracingContextHolder() if (this->is_context_owner) { /// Clear the context on current thread - current_fiber_trace_context->reset(); + current_trace_context->reset(); } else { - current_fiber_trace_context->span_id = this->root_span.parent_span_id; + current_trace_context->span_id = this->root_span.parent_span_id; } } From 6bfbbc94bf72e657b64de4c50f8a6c1cd3c087a0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 May 2023 14:59:35 +0200 Subject: [PATCH 0523/2223] A little better --- src/Interpreters/Cache/FileCache.cpp | 32 +++++++++++++++++----------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 85648d9d96a..183ae686908 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -634,13 +634,16 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) return new_size > query_priority->getSizeLimit(); }; - query_priority->iterate( - [&](LockedKey & locked_key, FileSegmentMetadataPtr segment_metadata) - { return is_query_priority_overflow() ? iterate_func(locked_key, segment_metadata) : PriorityIterationResult::BREAK; }, - cache_lock); - if (is_query_priority_overflow()) - return false; + { + query_priority->iterate( + [&](LockedKey & locked_key, FileSegmentMetadataPtr segment_metadata) + { return is_query_priority_overflow() ? iterate_func(locked_key, segment_metadata) : PriorityIterationResult::BREAK; }, + cache_lock); + + if (is_query_priority_overflow()) + return false; + } LOG_TEST( log, "Query limits satisfied (while reserving for {}:{})", @@ -654,7 +657,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) const bool is_overflow = (main_priority->getSizeLimit() != 0 && (main_priority->getSize(cache_lock) + size - freeable_space > main_priority->getSizeLimit())) || (main_priority->getElementsLimit() != 0 - && (main_priority->getElementsCount(cache_lock) + 1 - freeable_count > main_priority->getElementsLimit())); + && freeable_count == 0 && main_priority->getElementsCount(cache_lock) == main_priority->getElementsLimit()); LOG_TEST( log, "Overflow: {}, size: {}, ready to remove: {}, current cache size: {}/{}, elements: {}/{}, while reserving for {}:{}", @@ -666,13 +669,16 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) return is_overflow; }; - main_priority->iterate( - [&](LockedKey & locked_key, FileSegmentMetadataPtr segment_metadata) - { return is_main_priority_overflow() ? iterate_func(locked_key, segment_metadata) : PriorityIterationResult::BREAK; }, - cache_lock); - if (is_main_priority_overflow()) - return false; + { + main_priority->iterate( + [&](LockedKey & locked_key, FileSegmentMetadataPtr segment_metadata) + { return is_main_priority_overflow() ? iterate_func(locked_key, segment_metadata) : PriorityIterationResult::BREAK; }, + cache_lock); + + if (is_main_priority_overflow()) + return false; + } if (!file_segment.getKeyMetadata()->createBaseDirectory()) return false; From 821b64b4207aaf1ccf35d9ab8e2d4faa9ec0a32b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 22 May 2023 15:18:29 +0200 Subject: [PATCH 0524/2223] apply review suggestions --- src/Parsers/IAST_fwd.h | 2 +- src/Storages/Kafka/KafkaConsumer.cpp | 6 +----- src/Storages/MergeTree/MergeTreeData.cpp | 3 +-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 3 +-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 15 +++++++-------- src/Storages/MergeTree/ReplicatedMergeTreeQueue.h | 8 ++++---- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../02439_merge_selecting_partitions.sql | 6 +++++- 8 files changed, 21 insertions(+), 24 deletions(-) diff --git a/src/Parsers/IAST_fwd.h b/src/Parsers/IAST_fwd.h index 53d41d42d65..27116939a23 100644 --- a/src/Parsers/IAST_fwd.h +++ b/src/Parsers/IAST_fwd.h @@ -29,7 +29,7 @@ template inline typename DB::ASTs::size_type erase_if(DB::ASTs & asts, Predicate pred) { auto old_size = asts.size(); - asts.erase(std::remove_if(asts.begin(), asts.end(), pred), asts.end()); + std::erase_if(asts, pred); return old_size - asts.size(); } diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 4a14e1292fa..b2e6129c61c 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -489,7 +489,7 @@ size_t KafkaConsumer::filterMessageErrors() { assert(current == messages.begin()); - auto new_end = std::remove_if(messages.begin(), messages.end(), [this](auto & message) + size_t skipped = std::erase_if(messages, [this](auto & message) { if (auto error = message.get_error()) { @@ -500,12 +500,8 @@ size_t KafkaConsumer::filterMessageErrors() return false; }); - size_t skipped = std::distance(new_end, messages.end()); if (skipped) - { LOG_ERROR(log, "There were {} messages with an error", skipped); - messages.erase(new_end, messages.end()); - } return skipped; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b21f44baeb5..440c91e3082 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5820,11 +5820,10 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const ActiveDataPartSet active_parts(format_version); auto detached_parts = getDetachedParts(); - auto new_end_it = std::remove_if(detached_parts.begin(), detached_parts.end(), [&partition_id](const DetachedPartInfo & part_info) + std::erase_if(detached_parts, [&partition_id](const DetachedPartInfo & part_info) { return !part_info.valid_name || !part_info.prefix.empty() || part_info.partition_id != partition_id; }); - detached_parts.resize(std::distance(detached_parts.begin(), new_end_it)); for (const auto & part_info : detached_parts) { diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 029ee528276..29a1574b66e 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -251,11 +251,10 @@ MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::getDataPartsToSelectM if (!partitions_hint) return res; - auto new_end_it = std::remove_if(res.begin(), res.end(), [partitions_hint](const auto & part) + std::erase_if(res, [partitions_hint](const auto & part) { return !partitions_hint->contains(part->info.partition_id); }); - res.erase(new_end_it, res.end()); return res; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 115e1833af5..5eddd0b5370 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -160,14 +160,13 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) Strings children = zookeeper->getChildren(queue_path); - auto to_remove_it = std::remove_if( - children.begin(), children.end(), [&](const String & path) + size_t removed_entries = std::erase_if(children, + [&](const String & path) { return already_loaded_paths.count(path); }); - LOG_DEBUG(log, "Having {} queue entries to load, {} entries already loaded.", (to_remove_it - children.begin()), (children.end() - to_remove_it)); - children.erase(to_remove_it, children.end()); + LOG_DEBUG(log, "Having {} queue entries to load, {} entries already loaded.", children.size(), removed_entries); ::sort(children.begin(), children.end()); @@ -2104,7 +2103,7 @@ ReplicatedMergeTreeQueue::QueueLocks ReplicatedMergeTreeQueue::lockQueue() return QueueLocks(state_mutex, pull_logs_to_queue_mutex, update_mutations_mutex); } -TrivialMergePredicate::TrivialMergePredicate(ReplicatedMergeTreeQueue & queue_) +LocalMergePredicate::LocalMergePredicate(ReplicatedMergeTreeQueue & queue_) : queue(queue_) { } @@ -2190,7 +2189,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( inprogress_quorum_part.clear(); } -bool TrivialMergePredicate::operator()( +bool LocalMergePredicate::operator()( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, const MergeTreeTransaction *, @@ -2316,7 +2315,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( return nested_pred.canMergeTwoParts(left, right, out_reason); } -bool TrivialMergePredicate::canMergeTwoParts( +bool LocalMergePredicate::canMergeTwoParts( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, String * out_reason) const @@ -2404,7 +2403,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( return nested_pred.canMergeSinglePart(part, out_reason); } -bool TrivialMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const +bool LocalMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const { std::lock_guard lock(queue.state_mutex); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 2e7e3e3c96b..79572e13963 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -32,7 +32,7 @@ class ReplicatedMergeTreeQueue { private: friend class CurrentlyExecuting; - friend class TrivialMergePredicate; + friend class LocalMergePredicate; friend class ReplicatedMergeTreeMergePredicate; friend class MergeFromLogEntryTask; friend class ReplicatedMergeMutateTaskBase; @@ -493,10 +493,10 @@ public: /// Lightweight version of ReplicatedMergeTreeMergePredicate that do not make any ZooKeeper requests, /// but may return false-positive results. Checks only a subset of required conditions. -class TrivialMergePredicate +class LocalMergePredicate { public: - TrivialMergePredicate(ReplicatedMergeTreeQueue & queue_); + LocalMergePredicate(ReplicatedMergeTreeQueue & queue_); bool operator()(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, @@ -561,7 +561,7 @@ public: String getCoveringVirtualPart(const String & part_name) const; private: - TrivialMergePredicate nested_pred; + LocalMergePredicate nested_pred; const ReplicatedMergeTreeQueue & queue; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c43dab25ecc..ea9ffee4939 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3284,7 +3284,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() PartitionIdsHint partitions_to_merge_in; if (can_assign_merge) { - auto lightweight_merge_pred = TrivialMergePredicate(queue); + auto lightweight_merge_pred = LocalMergePredicate(queue); partitions_to_merge_in = merger_mutator.getPartitionsThatMayBeMerged( max_source_parts_size_for_merge, lightweight_merge_pred, merge_with_ttl_allowed, NO_TRANSACTION_PTR); if (partitions_to_merge_in.empty()) diff --git a/tests/queries/0_stateless/02439_merge_selecting_partitions.sql b/tests/queries/0_stateless/02439_merge_selecting_partitions.sql index de57e35a5a7..88ce2834d6b 100644 --- a/tests/queries/0_stateless/02439_merge_selecting_partitions.sql +++ b/tests/queries/0_stateless/02439_merge_selecting_partitions.sql @@ -1,4 +1,6 @@ +drop table if exists rmt; + create table rmt (n int, m int) engine=ReplicatedMergeTree('/test/02439/{shard}/{database}', '{replica}') partition by n order by n; insert into rmt select number, number from numbers(50); insert into rmt values (1, 2); @@ -21,4 +23,6 @@ system flush logs; -- it should not list unneeded partitions where we cannot merge anything select distinct path from system.zookeeper_log where path like '/test/02439/s1/' || currentDatabase() || '/block_numbers/%' - and op_num in ('List', 'SimpleList', 'FilteredList') and path not like '%/block_numbers/1' + and op_num in ('List', 'SimpleList', 'FilteredList') and path not like '%/block_numbers/1'; + +drop table rmt; From ea59d2ec5d0c2f3e29a03b3dcc9600cb881adbea Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 May 2023 14:06:46 +0000 Subject: [PATCH 0525/2223] Allow custom cleanup function --- src/Common/Fiber.h | 50 +++++++++++++++++++++++++++++++++++++++++----- 1 file changed, 45 insertions(+), 5 deletions(-) diff --git a/src/Common/Fiber.h b/src/Common/Fiber.h index f8b0c6202f5..69858c127a7 100644 --- a/src/Common/Fiber.h +++ b/src/Common/Fiber.h @@ -11,7 +11,12 @@ class Fiber { public: - using CleanUpFn = std::function; + struct CleanUpFn + { + virtual void operator()(void *) = 0; + virtual ~CleanUpFn() = default; + }; + using Impl = boost::context::fiber; template< typename StackAlloc, typename Fn> @@ -30,7 +35,7 @@ public: ~Fiber() { for (auto & [_, data] : local_data) - data.cleanup_fn(data.ptr); + (*data.cleanup_fn)(data.ptr); } explicit operator bool() const @@ -52,7 +57,7 @@ public: /// Set pointer to fiber-specific data, it will be stored in hash-map /// using provided key and cleaned on fiber destructor using provided /// cleanup function. - void setLocalData(void * key, void * ptr, CleanUpFn cleanup_fn) + void setLocalData(void * key, void * ptr, CleanUpFn * cleanup_fn) { local_data[key] = FiberLocalData{ptr, cleanup_fn}; } @@ -80,7 +85,7 @@ private: struct FiberLocalData { void * ptr; - CleanUpFn cleanup_fn; + CleanUpFn * cleanup_fn; }; Impl impl; @@ -93,10 +98,43 @@ private: /// Fiber local data is destroyed in Fiber destructor. /// On first request, fiber local data is copied from parent /// fiber data or from current thread data if there is no parent fiber. +/// Implementation is similar to boost::fiber::fiber_specific_ptr +/// (we cannot use it because we don't use boost::fiber API. template class FiberLocal { public: + struct DefaultCleanUpFn : public Fiber::CleanUpFn + { + void operator()(void * data) override + { + delete static_cast(data); + } + }; + + struct CustomCleanUpFn : public Fiber::CleanUpFn + { + explicit CustomCleanUpFn(void (*fn_)(T*)) : fn(fn_) + { + } + + void operator()(void * data) override + { + if (likely(fn != nullptr)) + fn(static_cast(data)); + } + + void (*fn)(T*); + }; + + FiberLocal() : cleanup_fn(std::make_unique()) + { + } + + explicit FiberLocal(void (*fn)(T*)) : cleanup_fn(std::make_unique(fn)) + { + } + T & operator*() { return get(); @@ -129,7 +167,7 @@ private: auto parent_instance = getInstanceForFiber(*fiber_info.parent_info); /// Crete new object and store pointer inside Fiber, so it will be destroyed in Fiber destructor. ptr = new T(parent_instance); - fiber_info.fiber->setLocalData(this, ptr, [](void * to_delete){ delete static_cast(to_delete); }); + fiber_info.fiber->setLocalData(this, ptr, cleanup_fn.get()); } return *ptr; @@ -140,5 +178,7 @@ private: static thread_local T thread_local_instance; return thread_local_instance; } + + std::unique_ptr cleanup_fn; }; From 48ad2896c7eebff9f5813778d050b9482dd97b8a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 May 2023 14:13:37 +0000 Subject: [PATCH 0526/2223] Remove segments from candidates as soon as handle them --- src/Interpreters/Cache/FileCache.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 3af521026af..f45eb5ff24f 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -528,7 +528,7 @@ KeyMetadata::iterator FileCache::addFileSegment( } } -bool FileCache::tryReserve(FileSegment & file_segment, size_t size) +bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) { assertInitialized(); auto cache_lock = cache_guard.lock(); @@ -684,18 +684,22 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size) if (!locked_key) continue; /// key could become invalid after we released the key lock above, just skip it. - for (const auto & candidate : deletion_info.candidates) + /// delete from vector in reverse order just for efficiency + auto & candidates = deletion_info.candidates; + while (!candidates.empty()) { + auto & candidate = candidates.back(); chassert(candidate->releasable()); - auto segment = candidate->file_segment; + const auto * segment = candidate->file_segment.get(); locked_key->removeFileSegment(segment->offset(), segment->lock()); segment->getQueueIterator()->remove(cache_lock); if (query_context) query_context->remove(current_key, segment->offset(), cache_lock); + + candidates.pop_back(); } - deletion_info.candidates.clear(); } /// queue_iteratir is std::nullopt here if no space has been reserved yet, a file_segment_metadata From df436b2cd4a7f93bffc8e0295f5cbb3b1ac7b3c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 06:59:08 +0000 Subject: [PATCH 0527/2223] Spark compatibility: Add new function space() --- .../functions/string-functions.md | 42 ++++- src/Functions/repeat.cpp | 41 ++--- src/Functions/space.cpp | 166 ++++++++++++++++++ .../02752_space_function.reference | 51 ++++++ .../0_stateless/02752_space_function.sql | 50 ++++++ 5 files changed, 328 insertions(+), 22 deletions(-) create mode 100644 src/Functions/space.cpp create mode 100644 tests/queries/0_stateless/02752_space_function.reference create mode 100644 tests/queries/0_stateless/02752_space_function.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 920a1f972d8..8662d08431c 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -323,11 +323,11 @@ Alias: `REPEAT` **Arguments** - `s` — The string to repeat. [String](../../sql-reference/data-types/string.md). -- `n` — The number of times to repeat the string. [UInt or Int](../../sql-reference/data-types/int-uint.md). +- `n` — The number of times to repeat the string. [UInt* or Int*](../../sql-reference/data-types/int-uint.md). **Returned value** -The single string containing string `s` repeated `n` times. If `n` \< 1, the function returns empty string. +A string containing string `s` repeated `n` times. If `n` <= 0, the function returns the empty string. Type: `String`. @@ -345,6 +345,44 @@ Result: └────────────────────────────────┘ ``` +## space + +Concatenates a space (` `) as many times with itself as specified. + +**Syntax** + +``` sql +space(n) +``` + +Alias: `SPACE`. + +**Arguments** + +- `n` — The number of times to repeat the space. [UInt* or Int*](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +The string containing string ` ` repeated `n` times. If `n` <= 0, the function returns the empty string. + +Type: `String`. + +**Example** + +Query: + +``` sql +SELECT space(3); +``` + +Result: + +``` text +┌─space(3) ────┐ +│ │ +└──────────────┘ +``` + ## reverse Reverses the sequence of bytes in a string. diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 0c323c39969..88eaebf4f03 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -186,36 +186,37 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isString(arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[0]->getName(), getName()); - if (!isInteger(arguments[1])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[1]->getName(), getName()); - return arguments[0]; + FunctionArgumentDescriptors args{ + {"s", &isString, nullptr, "String"}, + {"n", &isInteger, nullptr, "Integer"}, + }; + + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(); } bool useDefaultImplementationForConstants() const override { return true; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override { - const auto & strcolumn = arguments[0].column; - const auto & numcolumn = arguments[1].column; + const auto & col_str = arguments[0].column; + const auto & col_num = arguments[1].column; ColumnPtr res; - if (const ColumnString * col = checkAndGetColumn(strcolumn.get())) + if (const ColumnString * col = checkAndGetColumn(col_str.get())) { - if (const ColumnConst * scale_column_num = checkAndGetColumn(numcolumn.get())) + if (const ColumnConst * col_num_const = checkAndGetColumn(col_num.get())) { auto col_res = ColumnString::create(); castType(arguments[1].type.get(), [&](const auto & type) { using DataType = std::decay_t; using T = typename DataType::FieldType; - T repeat_time = scale_column_num->getValue(); - RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), repeat_time); + T times = col_num_const->getValue(); + RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), times); return true; }); return col_res; @@ -224,9 +225,9 @@ public: { using DataType = std::decay_t; using T = typename DataType::FieldType; - const ColumnVector * colnum = checkAndGetColumn>(numcolumn.get()); + const ColumnVector * column = checkAndGetColumn>(col_num.get()); auto col_res = ColumnString::create(); - RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), colnum->getData()); + RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), column->getData()); res = std::move(col_res); return true; })) @@ -234,7 +235,7 @@ public: return res; } } - else if (const ColumnConst * col_const = checkAndGetColumn(strcolumn.get())) + else if (const ColumnConst * col_const = checkAndGetColumn(col_str.get())) { /// Note that const-const case is handled by useDefaultImplementationForConstants. @@ -244,9 +245,9 @@ public: { using DataType = std::decay_t; using T = typename DataType::FieldType; - const ColumnVector * colnum = checkAndGetColumn>(numcolumn.get()); + const ColumnVector * column = checkAndGetColumn>(col_num.get()); auto col_res = ColumnString::create(); - RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), colnum->getData()); + RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), column->getData()); res = std::move(col_res); return true; })) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp new file mode 100644 index 00000000000..afdc01b717d --- /dev/null +++ b/src/Functions/space.cpp @@ -0,0 +1,166 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + +/// Prints whitespace n-times. Actually, space() could also be pushed down to repeat(). Chose a standalone-implementation because +/// we can do memset() whereas repeat() does memcpy(). +class FunctionSpace : public IFunction +{ +private: + static constexpr auto space = ' '; + +public: + static constexpr auto name = "space"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"n", &isInteger, nullptr, "Integer"} + }; + + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(); + } + + + template + bool executeConstant(ColumnPtr col_times, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const + { + const ColumnConst * col_times_const = checkAndGetColumn(col_times.get()); + + const ColumnPtr & col_times_const_internal = col_times_const->getDataColumnPtr(); + if (!checkAndGetColumn(col_times_const_internal.get())) + return false; + + using T = typename DataType::FieldType; + T times = col_times_const->getValue(); + + if (times < 1) + times = 0; + + res_offsets.resize(col_times->size()); + res_chars.resize(col_times->size() * (times + 1)); + + size_t pos = 0; + + for (size_t i = 0; i < col_times->size(); ++i) + { + memset(res_chars.begin() + pos, space, times); + pos += times; + + *(res_chars.begin() + pos) = '\n'; + pos += 1; + + res_offsets[i] = pos; + } + + return true; + } + + + template + bool executeVector(ColumnPtr col_times_, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const + { + auto * col_times = checkAndGetColumn(col_times_.get()); + if (!col_times) + return false; + + res_offsets.resize(col_times->size()); + res_chars.reserve(col_times->size() * 10); /// heuristic + + const PaddedPODArray & times_data = col_times->getData(); + + size_t pos = 0; + + for (size_t i = 0; i < col_times->size(); ++i) + { + typename DataType::FieldType times = times_data[i]; + + if (times < 1) + times = 0; + + if (res_chars.size() + times + 1 >= res_chars.capacity()) + res_chars.reserve(2 * res_chars.capacity()); + + memset(res_chars.begin() + pos, space, times); + pos += times; + + *(res_chars.begin() + pos) = '\n'; + pos += 1; + + res_offsets[i] = pos; + } + + res_chars.resize(pos); + + return true; + } + + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const auto & col_num = arguments[0].column; + + auto col_res = ColumnString::create(); + + ColumnString::Offsets & res_offsets = col_res->getOffsets(); + ColumnString::Chars & res_chars = col_res->getChars(); + + if (const ColumnConst * col_num_const = checkAndGetColumn(col_num.get())) + { + if ((executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars))) + return col_res; + } + else + { + if ((executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars))) + return col_res; + } + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); + } +}; +} + +REGISTER_FUNCTION(Space) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/tests/queries/0_stateless/02752_space_function.reference b/tests/queries/0_stateless/02752_space_function.reference new file mode 100644 index 00000000000..b0adccb415b --- /dev/null +++ b/tests/queries/0_stateless/02752_space_function.reference @@ -0,0 +1,51 @@ +const times, uint + + + + +const times, int + + + + +negative const times + + + + +negative tests +\N +non-const times, uint + 3 + 2 + 1 + 0 + 12 + 10 + 4 + 5 + 4 + 21 + 9 + 7 + 56 + 20 + 5 + 7 +non-const times, int + 3 + 2 + 1 + 0 + 12 + 10 + 4 + 5 + 0 + 0 + 0 + 0 + 56 + 20 + 5 + 7 diff --git a/tests/queries/0_stateless/02752_space_function.sql b/tests/queries/0_stateless/02752_space_function.sql new file mode 100644 index 00000000000..8f3605e6e6f --- /dev/null +++ b/tests/queries/0_stateless/02752_space_function.sql @@ -0,0 +1,50 @@ +SELECT 'const times, uint'; +SELECT space(3::UInt8); +SELECT space(3::UInt16); +SELECT space(3::UInt32); +SELECT space(3::UInt64); +SELECT 'const times, int'; +SELECT space(3::Int8); +SELECT space(3::Int16); +SELECT space(3::Int32); +SELECT space(3::Int64); + +SELECT 'negative const times'; +SELECT space(-3::Int8); +SELECT space(-3::Int16); +SELECT space(-3::Int32); +SELECT space(-3::Int64); + +SELECT 'negative tests'; +SELECT space('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(['abc']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(('abc')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(NULL); + +DROP TABLE IF EXISTS defaults; +CREATE TABLE defaults +( + u8 UInt8, + u16 UInt16, + u32 UInt32, + u64 UInt64, + i8 Int8, + i16 Int16, + i32 Int32, + i64 Int64 +) ENGINE = Memory(); + +INSERT INTO defaults values (3, 12, 4, 56, 3, 12, -4, 56) (2, 10, 21, 20, 2, 10, -21, 20) (1, 4, 9, 5, 1, 4, -9, 5) (0, 5, 7, 7, 0, 5, -7, 7); + +SELECT 'non-const times, uint'; +SELECT space(u8), length(space(u8)) FROM defaults; +SELECT space(u16), length(space(u16)) FROM defaults; +SELECT space(u32), length(space(u32)) from defaults; +SELECT space(u64), length(space(u64)) FROM defaults; +SELECT 'non-const times, int'; +SELECT space(i8), length(space(i8)) FROM defaults; +SELECT space(i16), length(space(i16)) FROM defaults; +SELECT space(i32), length(space(i32)) from defaults; +SELECT space(i64), length(space(i64)) FROM defaults; + +DROP TABLE defaults; From 90f4b1777832a8e6c3a343671be091c727f3e065 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 22 May 2023 15:45:18 +0000 Subject: [PATCH 0528/2223] Fix build & test --- .../test/integration/runner/compose/docker_compose_mongo.yml | 4 ++-- src/Processors/Sources/MongoDBSource.cpp | 2 +- src/Processors/Sources/MongoDBSource.h | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_mongo.yml b/docker/test/integration/runner/compose/docker_compose_mongo.yml index 60361e9e98d..8cdcbc421e8 100644 --- a/docker/test/integration/runner/compose/docker_compose_mongo.yml +++ b/docker/test/integration/runner/compose/docker_compose_mongo.yml @@ -1,7 +1,7 @@ version: '2.3' services: mongo1: - image: mongo:5.1 + image: mongo:6.0 restart: always environment: MONGO_INITDB_ROOT_USERNAME: root @@ -11,7 +11,7 @@ services: command: --profile=2 --verbose mongo2: - image: mongo:5.0 + image: mongo:6.0 restart: always ports: - ${MONGO_NO_CRED_EXTERNAL_PORT:-27017}:${MONGO_NO_CRED_INTERNAL_PORT:-27017} diff --git a/src/Processors/Sources/MongoDBSource.cpp b/src/Processors/Sources/MongoDBSource.cpp index 94b9cb7ad64..74dfa13158c 100644 --- a/src/Processors/Sources/MongoDBSource.cpp +++ b/src/Processors/Sources/MongoDBSource.cpp @@ -424,7 +424,7 @@ Poco::MongoDB::Document::Vector MongoDBCursor::nextDocuments(Poco::MongoDB::Conn } } -Int64 MongoDBCursor::cursorID() +Int64 MongoDBCursor::cursorID() const { return cursorID_; } diff --git a/src/Processors/Sources/MongoDBSource.h b/src/Processors/Sources/MongoDBSource.h index f816ccfd1c9..2bc5481e20b 100644 --- a/src/Processors/Sources/MongoDBSource.h +++ b/src/Processors/Sources/MongoDBSource.h @@ -47,7 +47,7 @@ public: Poco::MongoDB::Document::Vector nextDocuments(Poco::MongoDB::Connection & connection); - Int64 cursorID(); + Int64 cursorID() const; private: const bool is_wire_protocol_old; From ebff47ad3a1a574df33206790e857f02095d3716 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Mon, 22 May 2023 12:14:23 -0400 Subject: [PATCH 0529/2223] Update nlp-functions.md --- .../sql-reference/functions/nlp-functions.md | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/docs/en/sql-reference/functions/nlp-functions.md b/docs/en/sql-reference/functions/nlp-functions.md index 337fb19b244..904950fce30 100644 --- a/docs/en/sql-reference/functions/nlp-functions.md +++ b/docs/en/sql-reference/functions/nlp-functions.md @@ -38,6 +38,41 @@ Result: │ ['I','think','it','is','a','bless','in','disguis'] │ └────────────────────────────────────────────────────┘ ``` +### Supported languages for stem() + +:::note +The stem() function uses the [Snowball stemming](https://snowballstem.org/) library, see the Snowball website for updated languages etc. +::: + +- Arabic +- Armenian +- Basque +- Catalan +- Danish +- Dutch +- English +- Finnish +- French +- German +- Greek +- Hindi +- Hungarian +- Indonesian +- Irish +- Italian +- Lithuanian +- Nepali +- Norwegian +- Porter +- Portuguese +- Romanian +- Russian +- Serbian +- Spanish +- Swedish +- Tamil +- Turkish +- Yiddish ## lemmatize From fc10ba871f86433b70a2b354e499fd99187a294f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 May 2023 16:36:34 +0000 Subject: [PATCH 0530/2223] Analyzer: Do not execute table functions multiple times --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Interpreters/Context.cpp | 25 +++++++++++++++++++++++ src/Interpreters/Context.h | 3 +++ 3 files changed, 29 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 7ab0261850b..aa915e48d35 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6355,7 +6355,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, auto table_function_ast = table_function_node_typed.toAST(); table_function_ptr->parseArguments(table_function_ast, scope_context); - auto table_function_storage = table_function_ptr->execute(table_function_ast, scope_context, table_function_ptr->getName()); + auto table_function_storage = scope_context->getQueryContext()->executeTableFunction(table_function_ast, table_function_ptr); table_function_node_typed.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d9f450191bc..62573fb18a7 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1623,6 +1623,31 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const return res; } +StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const TableFunctionPtr & table_function_ptr) +{ + auto hash = table_expression->getTreeHash(); + String key = toString(hash.first) + '_' + toString(hash.second); + StoragePtr & res = table_function_results[key]; + + if (!res) + { + res = table_function_ptr->execute(table_expression, shared_from_this(), table_function_ptr->getName()); + + /// Since ITableFunction::parseArguments() may change table_expression, i.e.: + /// + /// remote('127.1', system.one) -> remote('127.1', 'system.one'), + /// + auto new_hash = table_expression->getTreeHash(); + if (hash != new_hash) + { + key = toString(new_hash.first) + '_' + toString(new_hash.second); + table_function_results[key] = res; + } + } + + return res; +} + void Context::addViewSource(const StoragePtr & storage) { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 1be662e0958..3862984bb6f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -108,6 +108,7 @@ class StorageS3Settings; class IDatabase; class DDLWorker; class ITableFunction; +using TableFunctionPtr = std::shared_ptr; class Block; class ActionLocksManager; using ActionLocksManagerPtr = std::shared_ptr; @@ -650,6 +651,8 @@ public: /// For table functions s3/file/url/hdfs/input we can use structure from /// insertion table depending on select expression. StoragePtr executeTableFunction(const ASTPtr & table_expression, const ASTSelectQuery * select_query_hint = nullptr); + /// Overload for the new analyzer. Structure inference is performed in QueryAnalysisPass. + StoragePtr executeTableFunction(const ASTPtr & table_expression, const TableFunctionPtr & table_function_ptr); void addViewSource(const StoragePtr & storage); StoragePtr getViewSource() const; From 0ad041e7c31956c4e7b9fd92d098700c35c6441b Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 May 2023 16:42:14 +0000 Subject: [PATCH 0531/2223] Remove redundant code --- src/Interpreters/Context.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 62573fb18a7..5ec12d7e0ea 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1632,17 +1632,6 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const if (!res) { res = table_function_ptr->execute(table_expression, shared_from_this(), table_function_ptr->getName()); - - /// Since ITableFunction::parseArguments() may change table_expression, i.e.: - /// - /// remote('127.1', system.one) -> remote('127.1', 'system.one'), - /// - auto new_hash = table_expression->getTreeHash(); - if (hash != new_hash) - { - key = toString(new_hash.first) + '_' + toString(new_hash.second); - table_function_results[key] = res; - } } return res; From eb7b5a5bb7a355ca2e97bc07aee3a37442f846ee Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 22 May 2023 13:02:39 -0400 Subject: [PATCH 0532/2223] add more info to NLP docs --- .../sql-reference/functions/nlp-functions.md | 62 ++++++++++--------- 1 file changed, 34 insertions(+), 28 deletions(-) diff --git a/docs/en/sql-reference/functions/nlp-functions.md b/docs/en/sql-reference/functions/nlp-functions.md index 904950fce30..f10415783a5 100644 --- a/docs/en/sql-reference/functions/nlp-functions.md +++ b/docs/en/sql-reference/functions/nlp-functions.md @@ -12,18 +12,18 @@ This is an experimental feature that is currently in development and is not read Performs stemming on a given word. -**Syntax** +### Syntax ``` sql stem('language', word) ``` -**Arguments** +### Arguments -- `language` — Language which rules will be applied. Must be in lowercase. [String](../../sql-reference/data-types/string.md#string). +- `language` — Language which rules will be applied. Use the two letter [ISO 639-1 code](https://en.wikipedia.org/wiki/List_of_ISO_639-1_codes). - `word` — word that needs to be stemmed. Must be in lowercase. [String](../../sql-reference/data-types/string.md#string). -**Examples** +### Examples Query: @@ -78,18 +78,18 @@ The stem() function uses the [Snowball stemming](https://snowballstem.org/) libr Performs lemmatization on a given word. Needs dictionaries to operate, which can be obtained [here](https://github.com/vpodpecan/lemmagen3/tree/master/src/lemmagen3/models). -**Syntax** +### Syntax ``` sql lemmatize('language', word) ``` -**Arguments** +### Arguments - `language` — Language which rules will be applied. [String](../../sql-reference/data-types/string.md#string). - `word` — Word that needs to be lemmatized. Must be lowercase. [String](../../sql-reference/data-types/string.md#string). -**Examples** +### Examples Query: @@ -105,12 +105,18 @@ Result: └─────────────────────┘ ``` -Configuration: +### Configuration + +This configuration specifies that the dictionary `en.bin` should be used for lemmatization of English (`en`) words. The `.bin` files can be downloaded from +[here](https://github.com/vpodpecan/lemmagen3/tree/master/src/lemmagen3/models). + ``` xml + en en.bin + ``` @@ -123,18 +129,18 @@ With the `plain` extension type we need to provide a path to a simple text file, With the `wordnet` extension type we need to provide a path to a directory with WordNet thesaurus in it. Thesaurus must contain a WordNet sense index. -**Syntax** +### Syntax ``` sql synonyms('extension_name', word) ``` -**Arguments** +### Arguments - `extension_name` — Name of the extension in which search will be performed. [String](../../sql-reference/data-types/string.md#string). - `word` — Word that will be searched in extension. [String](../../sql-reference/data-types/string.md#string). -**Examples** +### Examples Query: @@ -150,7 +156,7 @@ Result: └──────────────────────────────────────────┘ ``` -Configuration: +### Configuration ``` xml @@ -172,17 +178,17 @@ Detects the language of the UTF8-encoded input string. The function uses the [CL The `detectLanguage` function works best when providing over 200 characters in the input string. -**Syntax** +### Syntax ``` sql detectLanguage('text_to_be_analyzed') ``` -**Arguments** +### Arguments - `text_to_be_analyzed` — A collection (or sentences) of strings to analyze. [String](../../sql-reference/data-types/string.md#string). -**Returned value** +### Returned value - The 2-letter ISO code of the detected language @@ -191,7 +197,7 @@ Other possible results: - `un` = unknown, can not detect any language. - `other` = the detected language does not have 2 letter code. -**Examples** +### Examples Query: @@ -210,22 +216,22 @@ fr Similar to the `detectLanguage` function, but `detectLanguageMixed` returns a `Map` of 2-letter language codes that are mapped to the percentage of the certain language in the text. -**Syntax** +### Syntax ``` sql detectLanguageMixed('text_to_be_analyzed') ``` -**Arguments** +### Arguments - `text_to_be_analyzed` — A collection (or sentences) of strings to analyze. [String](../../sql-reference/data-types/string.md#string). -**Returned value** +### Returned value - `Map(String, Float32)`: The keys are 2-letter ISO codes and the values are a percentage of text found for that language -**Examples** +### Examples Query: @@ -246,17 +252,17 @@ Result: Similar to the `detectLanguage` function, except the `detectLanguageUnknown` function works with non-UTF8-encoded strings. Prefer this version when your character set is UTF-16 or UTF-32. -**Syntax** +### Syntax ``` sql detectLanguageUnknown('text_to_be_analyzed') ``` -**Arguments** +### Arguments - `text_to_be_analyzed` — A collection (or sentences) of strings to analyze. [String](../../sql-reference/data-types/string.md#string). -**Returned value** +### Returned value - The 2-letter ISO code of the detected language @@ -265,7 +271,7 @@ Other possible results: - `un` = unknown, can not detect any language. - `other` = the detected language does not have 2 letter code. -**Examples** +### Examples Query: @@ -286,21 +292,21 @@ Result: The `detectCharset` function detects the character set of the non-UTF8-encoded input string. -**Syntax** +### Syntax ``` sql detectCharset('text_to_be_analyzed') ``` -**Arguments** +### Arguments - `text_to_be_analyzed` — A collection (or sentences) of strings to analyze. [String](../../sql-reference/data-types/string.md#string). -**Returned value** +### Returned value - A `String` containing the code of the detected character set -**Examples** +### Examples Query: From b8305503d89783b6700ae2c43f69b96798181b03 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 22 May 2023 19:07:18 +0200 Subject: [PATCH 0533/2223] more flexible cleanup thread scheduling --- base/base/interpolate.h | 5 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 + src/Storages/MergeTree/MergeTreeData.cpp | 26 ++- src/Storages/MergeTree/MergeTreeData.h | 6 + src/Storages/MergeTree/MergeTreeSettings.h | 4 +- .../ReplicatedMergeTreeCleanupThread.cpp | 171 +++++++++++++++--- .../ReplicatedMergeTreeCleanupThread.h | 28 ++- .../MergeTree/SimpleMergeSelector.cpp | 8 +- src/Storages/StorageReplicatedMergeTree.cpp | 9 +- src/Storages/StorageReplicatedMergeTree.h | 4 +- tests/config/config.d/merge_tree.xml | 2 + .../test.py | 3 +- .../test_broken_part_during_merge/test.py | 2 +- .../test.py | 6 +- tests/integration/test_drop_replica/test.py | 15 +- tests/integration/test_jbod_balancer/test.py | 1 + tests/integration/test_jbod_ha/test.py | 1 + tests/integration/test_lost_part/test.py | 8 +- tests/integration/test_multiple_disks/test.py | 6 +- .../test_old_parts_finally_removed/test.py | 5 +- .../test_parts_delete_zookeeper/test.py | 2 +- .../integration/test_recovery_replica/test.py | 2 +- tests/integration/test_storage_nats/test.py | 3 +- .../integration/test_storage_rabbitmq/test.py | 6 +- tests/integration/test_system_metrics/test.py | 4 +- tests/integration/test_ttl_replicated/test.py | 3 +- ..._replace_partition_from_table_zookeeper.sh | 8 +- .../00652_replicated_mutations_zookeeper.sh | 6 +- ...ated_minimalistic_part_header_zookeeper.sh | 6 +- ...0953_zookeeper_suetin_deduplication_bug.sh | 2 +- .../00988_parallel_parts_removal.sql | 4 +- ...tem_parts_race_condition_zookeeper_long.sh | 10 +- ...tem_parts_race_condition_drop_zookeeper.sh | 3 +- ...034_move_partition_from_table_zookeeper.sh | 6 +- ...ent_move_partition_from_table_zookeeper.sh | 3 +- ...076_parallel_alter_replicated_zookeeper.sh | 3 +- ...9_parallel_alter_detach_table_zookeeper.sh | 5 +- .../01103_optimize_drop_race_zookeeper.sh | 4 +- .../0_stateless/01158_zookeeper_log_long.sql | 2 +- ...nactive_replica_cleanup_nodes_zookeeper.sh | 6 +- ...e_condition_rename_clear_zookeeper_long.sh | 4 +- .../01509_parallel_quorum_and_merge_long.sh | 3 +- ...nt_ttl_and_normal_merges_zookeeper_long.sh | 3 +- .../0_stateless/02067_lost_part_s3.sql | 12 +- .../02370_lost_part_intersecting_merges.sh | 2 +- .../02396_system_parts_race_condition_rm.sh | 8 +- ...397_system_parts_race_condition_drop_rm.sh | 4 +- .../02432_s3_parallel_parts_cleanup.sql | 4 +- .../02448_clone_replica_lost_part.sql | 6 +- ..._projection_and_mutation_work_together.sql | 6 +- .../02515_cleanup_async_insert_block_ids.sh | 2 +- 52 files changed, 353 insertions(+), 112 deletions(-) diff --git a/base/base/interpolate.h b/base/base/interpolate.h index 1d4fc0b6257..4c27f70c95b 100644 --- a/base/base/interpolate.h +++ b/base/base/interpolate.h @@ -11,3 +11,8 @@ constexpr double interpolateExponential(double min, double max, double ratio) assert(min > 0 && ratio >= 0 && ratio <= 1); return min * std::pow(max / min, ratio); } + +constexpr double interpolateLinear(double min, double max, double ratio) +{ + return std::lerp(min, max, ratio); +} diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index d27b03fff44..3d2b6ecc540 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -211,9 +211,9 @@ void IMergeTreeDataPart::MinMaxIndex::appendFiles(const MergeTreeData & data, St } -static void incrementStateMetric(MergeTreeDataPartState state) +void IMergeTreeDataPart::incrementStateMetric(MergeTreeDataPartState state_) const { - switch (state) + switch (state_) { case MergeTreeDataPartState::Temporary: CurrentMetrics::add(CurrentMetrics::PartsTemporary); @@ -227,6 +227,7 @@ static void incrementStateMetric(MergeTreeDataPartState state) CurrentMetrics::add(CurrentMetrics::PartsCommitted); return; case MergeTreeDataPartState::Outdated: + storage.total_outdated_parts_count.fetch_add(1, std::memory_order_relaxed); CurrentMetrics::add(CurrentMetrics::PartsOutdated); return; case MergeTreeDataPartState::Deleting: @@ -238,9 +239,9 @@ static void incrementStateMetric(MergeTreeDataPartState state) } } -static void decrementStateMetric(MergeTreeDataPartState state) +void IMergeTreeDataPart::decrementStateMetric(MergeTreeDataPartState state_) const { - switch (state) + switch (state_) { case MergeTreeDataPartState::Temporary: CurrentMetrics::sub(CurrentMetrics::PartsTemporary); @@ -254,6 +255,7 @@ static void decrementStateMetric(MergeTreeDataPartState state) CurrentMetrics::sub(CurrentMetrics::PartsCommitted); return; case MergeTreeDataPartState::Outdated: + storage.total_outdated_parts_count.fetch_sub(1, std::memory_order_relaxed); CurrentMetrics::sub(CurrentMetrics::PartsOutdated); return; case MergeTreeDataPartState::Deleting: diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 388d96314c0..ecc1523b6c0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -623,6 +623,9 @@ private: /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; + void incrementStateMetric(MergeTreeDataPartState state) const; + void decrementStateMetric(MergeTreeDataPartState state) const; + mutable MergeTreeDataPartState state{MergeTreeDataPartState::Temporary}; /// This ugly flag is needed for debug assertions only diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b21f44baeb5..5cfc4c577dc 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -81,6 +81,7 @@ #include #include +#include #include #include @@ -4311,6 +4312,29 @@ size_t MergeTreeData::getActivePartsCount() const } +size_t MergeTreeData::getOutdatedPartsCount() const +{ + return total_outdated_parts_count.load(std::memory_order_relaxed); +} + +size_t MergeTreeData::getNumberOfOutdatedPartsWithExpiredRemovalTime() const +{ + size_t res = 0; + + auto time_now = time(nullptr); + + auto parts_lock = lockParts(); + auto outdated_parts_range = getDataPartsStateRange(DataPartState::Outdated); + for (const auto & part : outdated_parts_range) + { + auto part_remove_time = part->remove_time.load(std::memory_order_relaxed); + if (part_remove_time <= time_now && time_now - part_remove_time >= getSettings()->old_parts_lifetime.totalSeconds() && part.unique()) + ++res; + } + + return res; +} + std::pair MergeTreeData::getMaxPartsCountAndSizeForPartitionWithState(DataPartState state) const { auto lock = lockParts(); @@ -4519,7 +4543,7 @@ void MergeTreeData::delayMutationOrThrowIfNeeded(Poco::Event * until, const Cont size_t allowed_mutations_over_threshold = num_mutations_to_throw - num_mutations_to_delay; double delay_factor = std::min(static_cast(mutations_over_threshold) / allowed_mutations_over_threshold, 1.0); - size_t delay_milliseconds = static_cast(std::lerp(settings->min_delay_to_mutate_ms, settings->max_delay_to_mutate_ms, delay_factor)); + size_t delay_milliseconds = static_cast(interpolateLinear(settings->min_delay_to_mutate_ms, settings->max_delay_to_mutate_ms, delay_factor)); ProfileEvents::increment(ProfileEvents::DelayedMutations); ProfileEvents::increment(ProfileEvents::DelayedMutationsMilliseconds, delay_milliseconds); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5488ce72631..4a71c24e6d3 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -532,6 +532,10 @@ public: size_t getActivePartsCount() const; + size_t getOutdatedPartsCount() const; + + size_t getNumberOfOutdatedPartsWithExpiredRemovalTime() const; + /// Returns a pair with: max number of parts in partition across partitions; sum size of parts inside that partition. /// (if there are multiple partitions with max number of parts, the sum size of parts is returned for arbitrary of them) std::pair getMaxPartsCountAndSizeForPartitionWithState(DataPartState state) const; @@ -1491,6 +1495,8 @@ private: std::atomic total_active_size_rows = 0; std::atomic total_active_size_parts = 0; + mutable std::atomic total_outdated_parts_count = 0; + // Record all query ids which access the table. It's guarded by `query_id_set_mutex` and is always mutable. mutable std::set query_id_set TSA_GUARDED_BY(query_id_set_mutex); mutable std::mutex query_id_set_mutex; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c9e81ce9103..78d703e795c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -120,8 +120,10 @@ struct Settings; \ /** Check delay of replicas settings. */ \ M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ - M(UInt64, cleanup_delay_period, 30, "Period to clean old queue logs, blocks hashes and parts.", 0) \ + M(UInt64, cleanup_delay_period, 30, "Minimum period to clean old queue logs, blocks hashes and parts.", 0) \ + M(UInt64, max_cleanup_delay_period, 300, "Maximum period to clean old queue logs, blocks hashes and parts.", 0) \ M(UInt64, cleanup_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to cleanup_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables.", 0) \ + M(UInt64, cleanup_thread_preferred_points_per_iteration, 150, "Preferred batch size for background cleanup (points are abstract but 1 point is approximately equivalent to 1 inserted block).", 0) \ M(UInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.", 0) \ M(UInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.", 0) \ M(UInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.", 0) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 0409cadc1e9..35a860ebb42 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -25,19 +25,22 @@ ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplic : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeCleanupThread)") , log(&Poco::Logger::get(log_name)) + , sleep_ms(storage.getSettings()->cleanup_delay_period * 1000) { task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ run(); }); } void ReplicatedMergeTreeCleanupThread::run() { - auto storage_settings = storage.getSettings(); - const auto sleep_ms = storage_settings->cleanup_delay_period * 1000 - + std::uniform_int_distribution(0, storage_settings->cleanup_delay_period_random_add * 1000)(rng); + SCOPE_EXIT({ is_running.store(false, std::memory_order_relaxed); }); + is_running.store(true, std::memory_order_relaxed); + auto storage_settings = storage.getSettings(); + + Float32 cleanup_points = 0; try { - iterate(); + cleanup_points = iterate(); } catch (const Coordination::Exception & e) { @@ -51,39 +54,144 @@ void ReplicatedMergeTreeCleanupThread::run() tryLogCurrentException(log, __PRETTY_FUNCTION__); } + UInt64 prev_timestamp = prev_cleanup_timestamp_ms.load(std::memory_order_relaxed); + UInt64 now_ms = clock_gettime_ns_adjusted(prev_timestamp * 1'000'000) / 1'000'000; + + /// Do not adjust sleep_ms on the first run after starting the server + if (prev_timestamp && storage_settings->cleanup_thread_preferred_points_per_iteration) + { + /// We don't want to run the task too often when the table was barely changed and there's almost nothing to cleanup. + /// But we cannot simply sleep max_cleanup_delay_period (300s) when nothing was cleaned up and cleanup_delay_period (30s) + /// when we removed something, because inserting one part per 30s will lead to running cleanup each 30s just to remove one part. + /// So we need some interpolation based on preferred batch size. + auto expected_cleanup_points = storage_settings->cleanup_thread_preferred_points_per_iteration; + + /// How long should we sleep to remove cleanup_thread_preferred_points_per_iteration on the next iteration? + Float32 ratio = cleanup_points / expected_cleanup_points; + if (ratio == 0) + sleep_ms = storage_settings->max_cleanup_delay_period * 1000; + else + sleep_ms = static_cast(sleep_ms / ratio); + + if (sleep_ms < storage_settings->cleanup_delay_period * 1000) + sleep_ms = storage_settings->cleanup_delay_period * 1000; + if (storage_settings->max_cleanup_delay_period * 1000 < sleep_ms) + sleep_ms = storage_settings->max_cleanup_delay_period * 1000; + + UInt64 interval_ms = now_ms - prev_timestamp; + LOG_TRACE(log, "Scheduling next cleanup after {}ms (points: {}, interval: {}ms, ratio: {}, points per minute: {})", + sleep_ms, cleanup_points, interval_ms, ratio, cleanup_points / interval_ms * 60'000); + } + prev_cleanup_timestamp_ms.store(now_ms, std::memory_order_relaxed); + + sleep_ms += std::uniform_int_distribution(0, storage_settings->cleanup_delay_period_random_add * 1000)(rng); task->scheduleAfter(sleep_ms); } - -void ReplicatedMergeTreeCleanupThread::iterate() +void ReplicatedMergeTreeCleanupThread::wakeupEarlierIfNeeded() { - storage.clearOldPartsAndRemoveFromZK(); + /// It may happen that the tables was idle for a long time, but then a user started to aggressively insert (or mutate) data. + /// In this case, sleep_ms was set to the highest possible value, the task is not going to wake up soon, + /// but the number of objects to clean up is growing. We need to wakeup the task earlier. + auto storage_settings = storage.getSettings(); + if (!storage_settings->cleanup_thread_preferred_points_per_iteration) + return; + + /// The number of other objects (logs, blocks, etc) is usually correlated with the number of Outdated parts. + /// Do not wake up unless we have too many. + size_t number_of_outdated_objects = storage.getOutdatedPartsCount(); + if (number_of_outdated_objects < storage_settings->cleanup_thread_preferred_points_per_iteration * 2) + return; + + /// A race condition is possible here, but it's okay + if (is_running.load(std::memory_order_relaxed)) + return; + + /// Do not re-check all parts too often (avoid constantly calling getNumberOfOutdatedPartsWithExpiredRemovalTime()) + if (!wakeup_check_timer.compareAndRestart(storage_settings->cleanup_delay_period / 4)) + return; + + UInt64 prev_run_timestamp_ms = prev_cleanup_timestamp_ms.load(std::memory_order_relaxed); + UInt64 now_ms = clock_gettime_ns_adjusted(prev_run_timestamp_ms * 1'000'000) / 1'000'000; + if (!prev_run_timestamp_ms || now_ms <= prev_run_timestamp_ms) + return; + + /// Don't run it more often than cleanup_delay_period + UInt64 seconds_passed = (now_ms - prev_run_timestamp_ms) / 1000; + if (seconds_passed < storage_settings->cleanup_delay_period) + return; + + /// Do not count parts that cannot be removed anyway. Do not wake up unless we have too many. + number_of_outdated_objects = storage.getNumberOfOutdatedPartsWithExpiredRemovalTime(); + if (number_of_outdated_objects < storage_settings->cleanup_thread_preferred_points_per_iteration * 2) + return; + + LOG_TRACE(log, "Waking up cleanup thread because there are {} outdated objects and previous cleanup finished {}s ago", + number_of_outdated_objects, seconds_passed); + + wakeup(); +} + + +Float32 ReplicatedMergeTreeCleanupThread::iterate() +{ + size_t cleaned_logs = 0; + Float32 cleaned_blocks = 0; + size_t cleaned_other = 0; + size_t cleaned_part_like = 0; + size_t cleaned_parts = storage.clearOldPartsAndRemoveFromZK(); + + auto storage_settings = storage.getSettings(); { auto lock = storage.lockForShare(RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations); /// Both use relative_data_path which changes during rename, so we /// do it under share lock - storage.clearOldWriteAheadLogs(); - storage.clearOldTemporaryDirectories(storage.getSettings()->temporary_directories_lifetime.totalSeconds()); + cleaned_other += storage.clearOldWriteAheadLogs(); + cleaned_part_like += storage.clearOldTemporaryDirectories(storage.getSettings()->temporary_directories_lifetime.totalSeconds()); if (storage.getSettings()->merge_tree_enable_clear_old_broken_detached) - storage.clearOldBrokenPartsFromDetachedDirectory(); + cleaned_part_like += storage.clearOldBrokenPartsFromDetachedDirectory(); } /// This is loose condition: no problem if we actually had lost leadership at this moment /// and two replicas will try to do cleanup simultaneously. if (storage.is_leader) { - clearOldLogs(); - auto storage_settings = storage.getSettings(); - clearOldBlocks("blocks", storage_settings->replicated_deduplication_window_seconds, storage_settings->replicated_deduplication_window, cached_block_stats_for_sync_inserts); - clearOldBlocks("async_blocks", storage_settings->replicated_deduplication_window_seconds_for_async_inserts, storage_settings->replicated_deduplication_window_for_async_inserts, cached_block_stats_for_async_inserts); - clearOldMutations(); - storage.clearEmptyParts(); + cleaned_logs = clearOldLogs(); + size_t normal_blocks = clearOldBlocks("blocks", storage_settings->replicated_deduplication_window_seconds, + storage_settings->replicated_deduplication_window, cached_block_stats_for_sync_inserts); + + size_t async_blocks = clearOldBlocks("async_blocks", + storage_settings->replicated_deduplication_window_seconds_for_async_inserts, + storage_settings->replicated_deduplication_window_for_async_inserts, + cached_block_stats_for_async_inserts); + + /// Many async blocks are transformed into one ordinary block + Float32 async_blocks_per_block = static_cast(storage_settings->replicated_deduplication_window) / + (storage_settings->replicated_deduplication_window_for_async_inserts + 1); + cleaned_blocks = (normal_blocks + async_blocks * async_blocks_per_block) / 2; + + cleaned_other += clearOldMutations(); + cleaned_part_like += storage.clearEmptyParts(); } + + /// We need to measure the number of removed objects somehow (for better scheduling), + /// but just summing the number of removed async blocks, logs, and empty parts does not make any sense. + /// So we are trying to (approximately) measure the number of inserted blocks/parts, so we will be able to compare apples to apples. + + /// Each inserted block produces 3 objects that have to be cleaned up: one block, one log entry and one part. + /// A few new parts get merged together producing one log entry and one part. + + /// Other objects (like mutations and WALs) are much more rare than Outdated parts (because mutations usually produce + /// many Outdated parts, and WALs usually contain many parts too). We count then as one part for simplicity. + + constexpr Float32 parts_number_amplification = 1.3f; /// Assuming we merge 4-5 parts each time + Float32 cleaned_inserted_parts = (cleaned_blocks + (cleaned_logs + cleaned_parts) / parts_number_amplification) / 3; + return cleaned_inserted_parts + cleaned_part_like + cleaned_other; } -void ReplicatedMergeTreeCleanupThread::clearOldLogs() +size_t ReplicatedMergeTreeCleanupThread::clearOldLogs() { auto zookeeper = storage.getZooKeeper(); auto storage_settings = storage.getSettings(); @@ -102,7 +210,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs() size_t min_replicated_logs_to_keep = static_cast(storage_settings->min_replicated_logs_to_keep * ratio); if (static_cast(children_count) < min_replicated_logs_to_keep) - return; + return 0; Strings replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas", &stat); @@ -114,7 +222,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs() Strings entries = zookeeper->getChildren(storage.zookeeper_path + "/log"); if (entries.empty()) - return; + return 0; ::sort(entries.begin(), entries.end()); @@ -227,7 +335,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs() entries.erase(std::lower_bound(entries.begin(), entries.end(), "log-" + padIndex(min_saved_log_pointer)), entries.end()); if (entries.empty()) - return; + return 0; markLostReplicas( host_versions_lost_replicas, @@ -268,6 +376,8 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs() if (i != 0) LOG_DEBUG(log, "Removed {} old log entries: {} - {}", i, entries[0], entries[i - 1]); + + return i; } @@ -323,7 +433,7 @@ struct ReplicatedMergeTreeCleanupThread::NodeWithStat } }; -void ReplicatedMergeTreeCleanupThread::clearOldBlocks(const String & blocks_dir_name, UInt64 window_seconds, UInt64 window_size, NodeCTimeAndVersionCache & cached_block_stats) +size_t ReplicatedMergeTreeCleanupThread::clearOldBlocks(const String & blocks_dir_name, UInt64 window_seconds, UInt64 window_size, NodeCTimeAndVersionCache & cached_block_stats) { auto zookeeper = storage.getZooKeeper(); @@ -331,7 +441,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks(const String & blocks_dir_ getBlocksSortedByTime(blocks_dir_name, *zookeeper, timed_blocks, cached_block_stats); if (timed_blocks.empty()) - return; + return 0; /// Use ZooKeeper's first node (last according to time) timestamp as "current" time. Int64 current_time = timed_blocks.front().ctime; @@ -350,7 +460,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks(const String & blocks_dir_ auto num_nodes_to_delete = timed_blocks.end() - first_outdated_block; if (!num_nodes_to_delete) - return; + return 0; auto last_outdated_block = timed_blocks.end() - 1; LOG_TRACE(log, "Will clear {} old blocks from {} (ctime {}) to {} (ctime {})", num_nodes_to_delete, @@ -388,6 +498,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks(const String & blocks_dir_ } LOG_TRACE(log, "Cleared {} old blocks from ZooKeeper", num_nodes_to_delete); + return num_nodes_to_delete; } @@ -456,17 +567,17 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(const String & bloc } -void ReplicatedMergeTreeCleanupThread::clearOldMutations() +size_t ReplicatedMergeTreeCleanupThread::clearOldMutations() { auto storage_settings = storage.getSettings(); if (!storage_settings->finished_mutations_to_keep) - return; + return 0; if (storage.queue.countFinishedMutations() <= storage_settings->finished_mutations_to_keep) { /// Not strictly necessary, but helps to avoid unnecessary ZooKeeper requests. /// If even this replica hasn't finished enough mutations yet, then we don't need to clean anything. - return; + return 0; } auto zookeeper = storage.getZooKeeper(); @@ -481,7 +592,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldMutations() // No Need to check return value to delete mutations. zookeeper->tryGet(storage.zookeeper_path + "/replicas/" + replica + "/mutation_pointer", pointer); if (pointer.empty()) - return; /// One replica hasn't done anything yet so we can't delete any mutations. + return 0; /// One replica hasn't done anything yet so we can't delete any mutations. min_pointer = std::min(parse(pointer), min_pointer); } @@ -492,11 +603,11 @@ void ReplicatedMergeTreeCleanupThread::clearOldMutations() entries.erase(std::upper_bound(entries.begin(), entries.end(), padIndex(min_pointer)), entries.end()); /// Do not remove last `storage_settings->finished_mutations_to_keep` entries. if (entries.size() <= storage_settings->finished_mutations_to_keep) - return; + return 0; entries.erase(entries.end() - storage_settings->finished_mutations_to_keep, entries.end()); if (entries.empty()) - return; + return 0; Coordination::Requests ops; size_t batch_start_i = 0; @@ -526,6 +637,8 @@ void ReplicatedMergeTreeCleanupThread::clearOldMutations() ops.clear(); } } + + return entries.size(); } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h index 76b9ee4a575..57de7944970 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -31,6 +32,8 @@ public: void stop() { task->deactivate(); } + void wakeupEarlierIfNeeded(); + private: StorageReplicatedMergeTree & storage; String log_name; @@ -38,11 +41,20 @@ private: BackgroundSchedulePool::TaskHolder task; pcg64 rng{randomSeed()}; - void run(); - void iterate(); + UInt64 sleep_ms; - /// Remove old records from ZooKeeper. - void clearOldLogs(); + std::atomic prev_cleanup_timestamp_ms = 0; + std::atomic is_running = false; + + AtomicStopwatch wakeup_check_timer; + + void run(); + + /// Returns a number this is directly proportional to the number of cleaned up blocks + Float32 iterate(); + + /// Remove old records from ZooKeeper. Returns the number of removed logs + size_t clearOldLogs(); /// The replica is marked as "lost" if it is inactive and its log pointer /// is far behind and we are not going to keep logs for it. @@ -52,11 +64,11 @@ private: size_t replicas_count, const zkutil::ZooKeeperPtr & zookeeper); using NodeCTimeAndVersionCache = std::map>; - /// Remove old block hashes from ZooKeeper. This is done by the leader replica. - void clearOldBlocks(const String & blocks_dir_name, UInt64 window_seconds, UInt64 window_size, NodeCTimeAndVersionCache & cached_block_stats); + /// Remove old block hashes from ZooKeeper. This is done by the leader replica. Returns the number of removed blocks + size_t clearOldBlocks(const String & blocks_dir_name, UInt64 window_seconds, UInt64 window_size, NodeCTimeAndVersionCache & cached_block_stats); - /// Remove old mutations that are done from ZooKeeper. This is done by the leader replica. - void clearOldMutations(); + /// Remove old mutations that are done from ZooKeeper. This is done by the leader replica. Returns the number of removed mutations + size_t clearOldMutations(); NodeCTimeAndVersionCache cached_block_stats_for_sync_inserts; NodeCTimeAndVersionCache cached_block_stats_for_async_inserts; diff --git a/src/Storages/MergeTree/SimpleMergeSelector.cpp b/src/Storages/MergeTree/SimpleMergeSelector.cpp index af3373fd175..7e7539f71d5 100644 --- a/src/Storages/MergeTree/SimpleMergeSelector.cpp +++ b/src/Storages/MergeTree/SimpleMergeSelector.cpp @@ -28,7 +28,7 @@ struct Estimator { double difference = std::abs(log2(static_cast(sum_size) / size_prev_at_left)); if (difference < settings.heuristic_to_align_parts_max_absolute_difference_in_powers_of_two) - current_score *= std::lerp(settings.heuristic_to_align_parts_max_score_adjustment, 1, + current_score *= interpolateLinear(settings.heuristic_to_align_parts_max_score_adjustment, 1, difference / settings.heuristic_to_align_parts_max_absolute_difference_in_powers_of_two); } @@ -115,8 +115,8 @@ bool allow( // std::cerr << "size_normalized: " << size_normalized << "\n"; /// Calculate boundaries for age - double min_age_to_lower_base = std::lerp(settings.min_age_to_lower_base_at_min_size, settings.min_age_to_lower_base_at_max_size, size_normalized); - double max_age_to_lower_base = std::lerp(settings.max_age_to_lower_base_at_min_size, settings.max_age_to_lower_base_at_max_size, size_normalized); + double min_age_to_lower_base = interpolateLinear(settings.min_age_to_lower_base_at_min_size, settings.min_age_to_lower_base_at_max_size, size_normalized); + double max_age_to_lower_base = interpolateLinear(settings.max_age_to_lower_base_at_min_size, settings.max_age_to_lower_base_at_max_size, size_normalized); // std::cerr << "min_age_to_lower_base: " << min_age_to_lower_base << "\n"; // std::cerr << "max_age_to_lower_base: " << max_age_to_lower_base << "\n"; @@ -137,7 +137,7 @@ bool allow( // std::cerr << "combined_ratio: " << combined_ratio << "\n"; - double lowered_base = std::lerp(settings.base, 2.0, combined_ratio); + double lowered_base = interpolateLinear(settings.base, 2.0, combined_ratio); // std::cerr << "------- lowered_base: " << lowered_base << "\n"; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d9c8f09ccf1..2b948e1fd60 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3147,6 +3147,8 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel bool StorageReplicatedMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) { + cleanup_thread.wakeupEarlierIfNeeded(); + /// If replication queue is stopped exit immediately as we successfully executed the task if (queue.actions_blocker.isCancelled()) return false; @@ -6589,7 +6591,7 @@ bool StorageReplicatedMergeTree::hasLightweightDeletedMask() const return has_lightweight_delete_parts.load(std::memory_order_relaxed); } -void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() +size_t StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() { auto table_lock = lockForShare( RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -6598,8 +6600,9 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() /// Now these parts are in Deleting state. If we fail to remove some of them we must roll them back to Outdated state. /// Otherwise they will not be deleted. DataPartsVector parts = grabOldParts(); + size_t total_parts_to_remove = parts.size(); if (parts.empty()) - return; + return total_parts_to_remove; DataPartsVector parts_to_delete_only_from_filesystem; // Only duplicates DataPartsVector parts_to_delete_completely; // All parts except duplicates @@ -6707,6 +6710,8 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() /// Otherwise nobody will try to remove them again (see grabOldParts). delete_parts_from_fs_and_rollback_in_case_of_error(parts_to_remove_from_filesystem, "old"); } + + return total_parts_to_remove; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 29b6a4d6817..01b86dd1425 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -342,8 +342,8 @@ public: private: std::atomic_bool are_restoring_replica {false}; - /// Delete old parts from disk and from ZooKeeper. - void clearOldPartsAndRemoveFromZK(); + /// Delete old parts from disk and from ZooKeeper. Returns the number of removed parts + size_t clearOldPartsAndRemoveFromZK(); template friend class ReplicatedMergeTreeSinkImpl; diff --git a/tests/config/config.d/merge_tree.xml b/tests/config/config.d/merge_tree.xml index 43bdb6aa07b..5521e5ba515 100644 --- a/tests/config/config.d/merge_tree.xml +++ b/tests/config/config.d/merge_tree.xml @@ -1,5 +1,7 @@ 8 + 60 + 10 diff --git a/tests/integration/test_broken_detached_part_clean_up/test.py b/tests/integration/test_broken_detached_part_clean_up/test.py index 5b18fa34494..9a70ebe0d48 100644 --- a/tests/integration/test_broken_detached_part_clean_up/test.py +++ b/tests/integration/test_broken_detached_part_clean_up/test.py @@ -141,7 +141,8 @@ def test_remove_broken_detached_part_replicated_merge_tree(started_cluster): merge_tree_enable_clear_old_broken_detached=1, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds=5, cleanup_delay_period=1, - cleanup_delay_period_random_add=0; + cleanup_delay_period_random_add=0, + cleanup_thread_preferred_points_per_iteration=0; """ ) diff --git a/tests/integration/test_broken_part_during_merge/test.py b/tests/integration/test_broken_part_during_merge/test.py index f4110844466..26962236869 100644 --- a/tests/integration/test_broken_part_during_merge/test.py +++ b/tests/integration/test_broken_part_during_merge/test.py @@ -25,7 +25,7 @@ def test_merge_and_part_corruption(started_cluster): """ CREATE TABLE replicated_mt(date Date, id UInt32, value Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{replica}') ORDER BY id - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1; + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0; """.format( replica=node1.name ) diff --git a/tests/integration/test_consistant_parts_after_move_partition/test.py b/tests/integration/test_consistant_parts_after_move_partition/test.py index 63a51472773..91fa884c093 100644 --- a/tests/integration/test_consistant_parts_after_move_partition/test.py +++ b/tests/integration/test_consistant_parts_after_move_partition/test.py @@ -14,11 +14,13 @@ def initialize_database(nodes, shard): CREATE TABLE `{database}`.src (p UInt64, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/tables/test_consistent_shard1{shard}/replicated', '{replica}') ORDER BY d PARTITION BY p - SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0; + SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, + cleanup_delay_period=0, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0; CREATE TABLE `{database}`.dest (p UInt64, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/tables/test_consistent_shard2{shard}/replicated', '{replica}') ORDER BY d PARTITION BY p - SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0; + SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, + cleanup_delay_period=0, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0; """.format( shard=shard, replica=node.name, database=CLICKHOUSE_DATABASE ) diff --git a/tests/integration/test_drop_replica/test.py b/tests/integration/test_drop_replica/test.py index e87edb0a578..0941e664982 100644 --- a/tests/integration/test_drop_replica/test.py +++ b/tests/integration/test_drop_replica/test.py @@ -11,7 +11,8 @@ def fill_nodes(nodes, shard): CREATE DATABASE test; CREATE TABLE test.test_table(date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0; + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) + SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0; """.format( shard=shard, replica=node.name ) @@ -22,7 +23,8 @@ def fill_nodes(nodes, shard): CREATE DATABASE test1; CREATE TABLE test1.test_table(date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test1/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0; + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test1/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) + SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0; """.format( shard=shard, replica=node.name ) @@ -33,7 +35,8 @@ def fill_nodes(nodes, shard): CREATE DATABASE test2; CREATE TABLE test2.test_table(date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test2/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0; + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test2/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) + SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0; """.format( shard=shard, replica=node.name ) @@ -44,7 +47,8 @@ def fill_nodes(nodes, shard): CREATE DATABASE test3; CREATE TABLE test3.test_table(date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test3/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0; + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test3/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) + SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0; """.format( shard=shard, replica=node.name ) @@ -55,7 +59,8 @@ def fill_nodes(nodes, shard): CREATE DATABASE test4; CREATE TABLE test4.test_table(date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test4/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0; + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test4/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) + SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0; """.format( shard=shard, replica=node.name ) diff --git a/tests/integration/test_jbod_balancer/test.py b/tests/integration/test_jbod_balancer/test.py index df34a075d5a..4797eec5381 100644 --- a/tests/integration/test_jbod_balancer/test.py +++ b/tests/integration/test_jbod_balancer/test.py @@ -134,6 +134,7 @@ def test_replicated_balanced_merge_fetch(start_cluster): old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 2, + cleanup_thread_preferred_points_per_iteration=0, min_bytes_to_rebalance_partition_over_jbod = 1024, max_bytes_to_merge_at_max_space_in_pool = 4096 """.format( diff --git a/tests/integration/test_jbod_ha/test.py b/tests/integration/test_jbod_ha/test.py index 5cbb5989ff3..033d751912a 100644 --- a/tests/integration/test_jbod_ha/test.py +++ b/tests/integration/test_jbod_ha/test.py @@ -58,6 +58,7 @@ def test_jbod_ha(start_cluster): old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 2, + cleanup_thread_preferred_points_per_iteration=0, max_bytes_to_merge_at_max_space_in_pool = 4096 """.format( i diff --git a/tests/integration/test_lost_part/test.py b/tests/integration/test_lost_part/test.py index dd4c2105d55..44cd19fd1fb 100644 --- a/tests/integration/test_lost_part/test.py +++ b/tests/integration/test_lost_part/test.py @@ -42,7 +42,7 @@ def test_lost_part_same_replica(start_cluster): for node in [node1, node2]: node.query( f"CREATE TABLE mt0 (id UInt64, date Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/t', '{node.name}') ORDER BY tuple() PARTITION BY date " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1" + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0" ) node1.query("SYSTEM STOP MERGES mt0") @@ -109,7 +109,7 @@ def test_lost_part_other_replica(start_cluster): for node in [node1, node2]: node.query( f"CREATE TABLE mt1 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t1', '{node.name}') ORDER BY tuple() " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1" + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0" ) node1.query("SYSTEM STOP MERGES mt1") @@ -178,7 +178,7 @@ def test_lost_part_mutation(start_cluster): for node in [node1, node2]: node.query( f"CREATE TABLE mt2 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t2', '{node.name}') ORDER BY tuple() " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1" + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0" ) node1.query("SYSTEM STOP MERGES mt2") @@ -241,7 +241,7 @@ def test_lost_last_part(start_cluster): for node in [node1, node2]: node.query( f"CREATE TABLE mt3 (id UInt64, p String) ENGINE ReplicatedMergeTree('/clickhouse/tables/t3', '{node.name}') " - "ORDER BY tuple() PARTITION BY p SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1" + "ORDER BY tuple() PARTITION BY p SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0" ) node1.query("SYSTEM STOP MERGES mt3") diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 0e51df017b2..54e7f6dd8ee 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -1528,7 +1528,8 @@ def test_simple_replication_and_moves(start_cluster): s1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_table_for_moves', '{}') ORDER BY tuple() - SETTINGS storage_policy='moving_jbod_with_external', old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=2 + SETTINGS storage_policy='moving_jbod_with_external', old_parts_lifetime=1, + cleanup_delay_period=1, cleanup_delay_period_random_add=2, cleanup_thread_preferred_points_per_iteration=0 """.format( i + 1 ) @@ -1609,7 +1610,8 @@ def test_download_appropriate_disk(start_cluster): s1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_table_for_download', '{}') ORDER BY tuple() - SETTINGS storage_policy='moving_jbod_with_external', old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=2 + SETTINGS storage_policy='moving_jbod_with_external', old_parts_lifetime=1, + cleanup_delay_period=1, cleanup_delay_period_random_add=2, cleanup_thread_preferred_points_per_iteration=0 """.format( i + 1 ) diff --git a/tests/integration/test_old_parts_finally_removed/test.py b/tests/integration/test_old_parts_finally_removed/test.py index 5347d433419..cbd701588d5 100644 --- a/tests/integration/test_old_parts_finally_removed/test.py +++ b/tests/integration/test_old_parts_finally_removed/test.py @@ -27,7 +27,8 @@ def started_cluster(): def test_part_finally_removed(started_cluster): node1.query( - "CREATE TABLE drop_outdated_part (Key UInt64) ENGINE = ReplicatedMergeTree('/table/d', '1') ORDER BY tuple() SETTINGS old_parts_lifetime=10, cleanup_delay_period=10, cleanup_delay_period_random_add=1" + "CREATE TABLE drop_outdated_part (Key UInt64) ENGINE = ReplicatedMergeTree('/table/d', '1') ORDER BY tuple() " + "SETTINGS old_parts_lifetime=10, cleanup_delay_period=10, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0" ) node1.query("INSERT INTO drop_outdated_part VALUES (1)") @@ -44,7 +45,7 @@ def test_part_finally_removed(started_cluster): ) node1.query( - "ALTER TABLE drop_outdated_part MODIFY SETTING old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1" + "ALTER TABLE drop_outdated_part MODIFY SETTING old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0" ) for i in range(60): diff --git a/tests/integration/test_parts_delete_zookeeper/test.py b/tests/integration/test_parts_delete_zookeeper/test.py index a78aefa4595..9fd07e7b65d 100644 --- a/tests/integration/test_parts_delete_zookeeper/test.py +++ b/tests/integration/test_parts_delete_zookeeper/test.py @@ -21,7 +21,7 @@ def start_cluster(): CREATE DATABASE test; CREATE TABLE test_table(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/replicated', 'node1') - ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS old_parts_lifetime=4, cleanup_delay_period=1; + ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS old_parts_lifetime=4, cleanup_delay_period=1, cleanup_thread_preferred_points_per_iteration=0; """ ) diff --git a/tests/integration/test_recovery_replica/test.py b/tests/integration/test_recovery_replica/test.py index 0a63da4db22..582e018f5d2 100644 --- a/tests/integration/test_recovery_replica/test.py +++ b/tests/integration/test_recovery_replica/test.py @@ -4,7 +4,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry -SETTINGS = "SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0" +SETTINGS = "SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0" def fill_nodes(nodes): diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py index 1d7e046864b..4d7e4cf813d 100644 --- a/tests/integration/test_storage_nats/test.py +++ b/tests/integration/test_storage_nats/test.py @@ -931,7 +931,8 @@ def test_nats_overloaded_insert(nats_cluster): CREATE TABLE test.view_overload (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key - SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; + SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3, + cleanup_thread_preferred_points_per_iteration=0; CREATE MATERIALIZED VIEW test.consumer_overload TO test.view_overload AS SELECT * FROM test.nats_consume; """ diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 4e1e28373e3..b4dcf86e0ba 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -642,7 +642,8 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): CREATE TABLE test.view (key UInt64, value UInt64, channel_id String) ENGINE = MergeTree ORDER BY key - SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; + SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3, + cleanup_thread_preferred_points_per_iteration=0; CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT *, _channel_id AS channel_id FROM test.rabbitmq; """ @@ -1116,7 +1117,8 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): CREATE TABLE test.destination(key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key - SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3; + SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3, + cleanup_thread_preferred_points_per_iteration=0; """ ) diff --git a/tests/integration/test_system_metrics/test.py b/tests/integration/test_system_metrics/test.py index 9ebe198a109..338622b824e 100644 --- a/tests/integration/test_system_metrics/test.py +++ b/tests/integration/test_system_metrics/test.py @@ -13,7 +13,9 @@ def fill_nodes(nodes, shard): CREATE DATABASE test; CREATE TABLE test.test_table(date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0; + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) + SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, + cleanup_delay_period=0, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0; """.format( shard=shard, replica=node.name ) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index a3e7d6e4b8b..4ea4472b812 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -422,7 +422,8 @@ def test_ttl_empty_parts(started_cluster): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_empty_parts', '{replica}') ORDER BY id SETTINGS max_bytes_to_merge_at_min_space_in_pool = 1, max_bytes_to_merge_at_max_space_in_pool = 1, - cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, old_parts_lifetime = 1 + cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime = 1 """.format( replica=node.name diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index a0a3416e406..399511db701 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -36,8 +36,12 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r1;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r2;" $CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE dst_r1 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst_1', '1') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE dst_r2 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst_1', '2') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE dst_r1 (p UInt64, k String, d UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst_1', '1') PARTITION BY p ORDER BY k +SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE dst_r2 (p UInt64, k String, d UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst_1', '2') PARTITION BY p ORDER BY k +SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0;" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (0, '0', 1);" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" diff --git a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh index 1f5bcbdc0d0..d8b1bdec328 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh @@ -56,11 +56,13 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r2 SYNC" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations_cleaner', 'r1') ORDER BY x SETTINGS \ finished_mutations_to_keep = 2, cleanup_delay_period = 1, - cleanup_delay_period_random_add = 0" + cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations_cleaner', 'r2') ORDER BY x SETTINGS \ finished_mutations_to_keep = 2, cleanup_delay_period = 1, - cleanup_delay_period_random_add = 0" + cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0" # Insert some data ${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4), (5)" diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index 5fc3fa460e6..bab2304cec2 100755 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -20,13 +20,15 @@ CREATE TABLE part_header_r1(x UInt32, y UInt32) SETTINGS use_minimalistic_part_header_in_zookeeper = 0, old_parts_lifetime = 1, cleanup_delay_period = 0, - cleanup_delay_period_random_add = 0; + cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0; CREATE TABLE part_header_r2(x UInt32, y UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/{shard}', '2{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 1, old_parts_lifetime = 1, cleanup_delay_period = 0, - cleanup_delay_period_random_add = 0; + cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0; SELECT '*** Test fetches ***'; INSERT INTO part_header_r1 VALUES (1, 1); diff --git a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh index c713c7c4926..ad0146b9d99 100755 --- a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh +++ b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh @@ -22,7 +22,7 @@ CREATE TABLE elog ( ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/{shard}', '{replica}') PARTITION BY date ORDER BY (engine_id) -SETTINGS replicated_deduplication_window = 2, cleanup_delay_period=4, cleanup_delay_period_random_add=0;" +SETTINGS replicated_deduplication_window = 2, cleanup_delay_period=4, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0;" $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 1, 'hello')" $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')" diff --git a/tests/queries/0_stateless/00988_parallel_parts_removal.sql b/tests/queries/0_stateless/00988_parallel_parts_removal.sql index bff9bbe6d8d..5bd31ba1baa 100644 --- a/tests/queries/0_stateless/00988_parallel_parts_removal.sql +++ b/tests/queries/0_stateless/00988_parallel_parts_removal.sql @@ -1,6 +1,8 @@ DROP TABLE IF EXISTS mt; -CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS max_part_removal_threads = 16, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, old_parts_lifetime = 1, parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; +CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x + SETTINGS max_part_removal_threads = 16, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime = 1, parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; SYSTEM STOP MERGES mt; diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index 5b1c50262bf..e0b7ab29924 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -13,8 +13,14 @@ $CLICKHOUSE_CLIENT -n -q " DROP TABLE IF EXISTS alter_table0; DROP TABLE IF EXISTS alter_table1; - CREATE TABLE alter_table0 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, replicated_max_mutations_in_one_entry = $(($RANDOM / 50 + 100)); - CREATE TABLE alter_table1 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, replicated_max_mutations_in_one_entry = $(($RANDOM / 50 + 200)); + CREATE TABLE alter_table0 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r1') ORDER BY a PARTITION BY b % 10 + SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0, replicated_max_mutations_in_one_entry = $(($RANDOM / 50 + 100)); + CREATE TABLE alter_table1 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r2') ORDER BY a PARTITION BY b % 10 + SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0, replicated_max_mutations_in_one_entry = $(($RANDOM / 50 + 200)); " function thread1() diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh index f4f38ad9c83..811681794a5 100755 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh @@ -58,7 +58,8 @@ function thread6() $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS alter_table_$REPLICA; CREATE TABLE alter_table_$REPLICA (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r_$REPLICA') ORDER BY a PARTITION BY b % 10 - SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, replicated_max_mutations_in_one_entry = $(($RANDOM / 50));"; + SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0, replicated_max_mutations_in_one_entry = $(($RANDOM / 50));"; sleep 0.$RANDOM; done } diff --git a/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh index 5e9e69d999d..e0a84323dbd 100755 --- a/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh @@ -28,7 +28,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst;" $CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/src1', '1') PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE dst (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst1', '1') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE dst (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst1', '1') PARTITION BY p ORDER BY k +SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0;" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (0, '0', 1);" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" @@ -58,7 +59,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE dst;" $CLICKHOUSE_CLIENT --query="SELECT 'MOVE incompatible schema missing column';" $CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/src2', '1') PARTITION BY p ORDER BY (d, p);" -$CLICKHOUSE_CLIENT --query="CREATE TABLE dst (p UInt64, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst2', '1') PARTITION BY p ORDER BY (d, p) SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE dst (p UInt64, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst2', '1') PARTITION BY p ORDER BY (d, p) +SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0;" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (0, '0', 1);" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" diff --git a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh index 8ef03be02b6..06a460f3600 100755 --- a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh @@ -11,7 +11,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst;" $CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/src', '1') PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE dst (p UInt64, k String) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst', '1') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE dst (p UInt64, k String) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/dst', '1') PARTITION BY p ORDER BY k +SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0;" function thread1() { diff --git a/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh b/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh index 7f53bf2a627..5f69427c0cd 100755 --- a/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh @@ -31,7 +31,8 @@ for i in $(seq $REPLICAS); do max_replicated_merges_in_queue = 1000, temporary_directories_lifetime = 10, cleanup_delay_period = 3, - cleanup_delay_period_random_add = 0" + cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0" done $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_mutate_mt_1 SELECT number, number + 10, toString(number) from numbers(10)" diff --git a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh index aec27792603..e508b77a0c2 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh @@ -12,7 +12,10 @@ for i in $(seq $REPLICAS); do done for i in $(seq $REPLICAS); do - $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_detach_$i (key UInt64, value1 UInt8, value2 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_detach', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000,temporary_directories_lifetime=10,cleanup_delay_period=3,cleanup_delay_period_random_add=0" + $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_detach_$i (key UInt64, value1 UInt8, value2 UInt8) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_detach', '$i') ORDER BY key + SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000, + temporary_directories_lifetime=10,cleanup_delay_period=3,cleanup_delay_period_random_add=0,cleanup_thread_preferred_points_per_iteration=0" done $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_detach_1 SELECT number, number + 10, number from numbers(10)" diff --git a/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh b/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh index 95f8dfc0377..3461283b5ea 100755 --- a/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh +++ b/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh @@ -27,7 +27,9 @@ function thread3() { while true; do $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS concurrent_optimize_table; - CREATE TABLE concurrent_optimize_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_optimize_table', '1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0;"; + CREATE TABLE concurrent_optimize_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_optimize_table', '1') ORDER BY a PARTITION BY b % 10 + SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, cleanup_thread_preferred_points_per_iteration=0;"; sleep 0.$RANDOM; sleep 0.$RANDOM; sleep 0.$RANDOM; diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.sql b/tests/queries/0_stateless/01158_zookeeper_log_long.sql index 45771494af6..9b5ae7ad7c6 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.sql +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.sql @@ -6,7 +6,7 @@ SET insert_keeper_fault_injection_probability=0; -- disable fault injection; par drop table if exists rmt sync; -- cleanup code will perform extra Exists -- (so the .reference will not match) -create table rmt (n int) engine=ReplicatedMergeTree('/test/01158/{database}/rmt', '1') order by n settings cleanup_delay_period=86400, replicated_can_become_leader=0; +create table rmt (n int) engine=ReplicatedMergeTree('/test/01158/{database}/rmt', '1') order by n settings cleanup_delay_period=86400, max_cleanup_delay_period=86400, replicated_can_become_leader=0; system sync replica rmt; insert into rmt values (1); insert into rmt values (1); diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index 411705e0469..2d761df998e 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -13,8 +13,10 @@ SCALE=5000 $CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS r1; DROP TABLE IF EXISTS r2; - CREATE TABLE r1 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}', '1{replica}') ORDER BY x SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; - CREATE TABLE r2 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}', '2{replica}') ORDER BY x SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; + CREATE TABLE r1 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}', '1{replica}') ORDER BY x + SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; + CREATE TABLE r2 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}', '2{replica}') ORDER BY x + SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; DETACH TABLE r2; " diff --git a/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper_long.sh b/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper_long.sh index 80318ba67fb..c3c87eeaf8b 100755 --- a/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper_long.sh +++ b/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper_long.sh @@ -8,7 +8,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames0" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames50" -$CLICKHOUSE_CLIENT --query "CREATE TABLE table_for_renames0 (value UInt64, data String) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_rename', '1') ORDER BY tuple() SETTINGS cleanup_delay_period = 1, cleanup_delay_period_random_add = 0" +$CLICKHOUSE_CLIENT --query "CREATE TABLE table_for_renames0 (value UInt64, data String) +ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_rename', '1') ORDER BY tuple() +SETTINGS cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, cleanup_thread_preferred_points_per_iteration=0" $CLICKHOUSE_CLIENT --query "INSERT INTO table_for_renames0 SELECT number, toString(number) FROM numbers(1000)" diff --git a/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh b/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh index 445706e35bf..bf88ad0e0b2 100755 --- a/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh +++ b/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh @@ -13,7 +13,8 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1 SYNC" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2 SYNC" -$CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q1 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_q', 'r1') ORDER BY tuple() SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q1 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_q', 'r1') ORDER BY tuple() +SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, cleanup_thread_preferred_points_per_iteration=0" $CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q2 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_q', 'r2') ORDER BY tuple() SETTINGS always_fetch_merged_part = 1" diff --git a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh index a3682a3a74b..5e1600a0673 100755 --- a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh +++ b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh @@ -24,7 +24,8 @@ for i in $(seq 1 $NUM_REPLICAS); do ENGINE ReplicatedMergeTree('/test/01921_concurrent_ttl_and_normal_merges/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/ttl_table', '$i') ORDER BY tuple() TTL key + INTERVAL 1 SECOND - SETTINGS merge_with_ttl_timeout=1, max_replicated_merges_with_ttl_in_queue=100, max_number_of_merges_with_ttl_in_pool=100, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" + SETTINGS merge_with_ttl_timeout=1, max_replicated_merges_with_ttl_in_queue=100, max_number_of_merges_with_ttl_in_pool=100, + cleanup_delay_period=1, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0;" done function optimize_thread diff --git a/tests/queries/0_stateless/02067_lost_part_s3.sql b/tests/queries/0_stateless/02067_lost_part_s3.sql index 12afdcd4421..7df15ab33c4 100644 --- a/tests/queries/0_stateless/02067_lost_part_s3.sql +++ b/tests/queries/0_stateless/02067_lost_part_s3.sql @@ -4,11 +4,17 @@ DROP TABLE IF EXISTS partslost_0; DROP TABLE IF EXISTS partslost_1; DROP TABLE IF EXISTS partslost_2; -CREATE TABLE partslost_0 (x String) ENGINE=ReplicatedMergeTree('/clickhouse/table/{database}_02067_lost/partslost', '0') ORDER BY tuple() SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 1; +CREATE TABLE partslost_0 (x String) ENGINE=ReplicatedMergeTree('/clickhouse/table/{database}_02067_lost/partslost', '0') ORDER BY tuple() + SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, old_parts_lifetime = 1, + cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0; -CREATE TABLE partslost_1 (x String) ENGINE=ReplicatedMergeTree('/clickhouse/table/{database}_02067_lost/partslost', '1') ORDER BY tuple() SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 1; +CREATE TABLE partslost_1 (x String) ENGINE=ReplicatedMergeTree('/clickhouse/table/{database}_02067_lost/partslost', '1') ORDER BY tuple() + SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, old_parts_lifetime = 1, + cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0; -CREATE TABLE partslost_2 (x String) ENGINE=ReplicatedMergeTree('/clickhouse/table/{database}_02067_lost/partslost', '2') ORDER BY tuple() SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 1; +CREATE TABLE partslost_2 (x String) ENGINE=ReplicatedMergeTree('/clickhouse/table/{database}_02067_lost/partslost', '2') ORDER BY tuple() + SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, old_parts_lifetime = 1, + cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0; INSERT INTO partslost_0 SELECT toString(number) AS x from system.numbers LIMIT 10000; diff --git a/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh index bc297cbb963..e34163d0502 100755 --- a/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh +++ b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT -q "drop table if exists rmt1 sync;" $CLICKHOUSE_CLIENT -q "drop table if exists rmt2 sync;" $CLICKHOUSE_CLIENT -q "create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '1') order by n - settings cleanup_delay_period=0, cleanup_delay_period_random_add=0, old_parts_lifetime=0" + settings cleanup_delay_period=0, cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime=0" $CLICKHOUSE_CLIENT -q "create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '2') order by n" $CLICKHOUSE_CLIENT -q "system stop replicated sends rmt2" diff --git a/tests/queries/0_stateless/02396_system_parts_race_condition_rm.sh b/tests/queries/0_stateless/02396_system_parts_race_condition_rm.sh index 5df1a9ba095..e31a091ff45 100755 --- a/tests/queries/0_stateless/02396_system_parts_race_condition_rm.sh +++ b/tests/queries/0_stateless/02396_system_parts_race_condition_rm.sh @@ -15,8 +15,12 @@ $CLICKHOUSE_CLIENT -n -q " DROP TABLE IF EXISTS alter_table0; DROP TABLE IF EXISTS alter_table1; - CREATE TABLE alter_table0 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0; - CREATE TABLE alter_table1 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0 + CREATE TABLE alter_table0 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r1') ORDER BY a PARTITION BY b % 10 + SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, cleanup_thread_preferred_points_per_iteration=0; + CREATE TABLE alter_table1 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r2') ORDER BY a PARTITION BY b % 10 + SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, cleanup_thread_preferred_points_per_iteration=0 " function thread1() diff --git a/tests/queries/0_stateless/02397_system_parts_race_condition_drop_rm.sh b/tests/queries/0_stateless/02397_system_parts_race_condition_drop_rm.sh index 548179b94c9..39e513f6be4 100755 --- a/tests/queries/0_stateless/02397_system_parts_race_condition_drop_rm.sh +++ b/tests/queries/0_stateless/02397_system_parts_race_condition_drop_rm.sh @@ -58,7 +58,9 @@ function thread6() while true; do REPLICA=$(($RANDOM % 10)) $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS alter_table_$REPLICA; - CREATE TABLE alter_table_$REPLICA (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r_$REPLICA') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0;"; + CREATE TABLE alter_table_$REPLICA (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r_$REPLICA') ORDER BY a PARTITION BY b % 10 + SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, cleanup_thread_preferred_points_per_iteration=0;"; sleep 0.$RANDOM; done } diff --git a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql index 88fb2cdf9b1..bab4bf7881c 100644 --- a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql +++ b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql @@ -8,7 +8,7 @@ drop table if exists rmt2; -- Disable compact parts, because we need hardlinks in mutations. create table rmt (n int, m int, k int) engine=ReplicatedMergeTree('/test/02432/{database}', '1') order by tuple() settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1, - max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, + max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0, max_replicated_merges_in_queue=0, max_replicated_mutations_in_queue=0, min_bytes_for_wide_part=0, min_rows_for_wide_part=0; insert into rmt(n, m) values (1, 42); @@ -38,7 +38,7 @@ select count(), sum(n), sum(m) from rmt; -- New table can assign merges/mutations and can remove old parts create table rmt2 (n int, m int, k String) engine=ReplicatedMergeTree('/test/02432/{database}', '2') order by tuple() settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1, - max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, + max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0, min_bytes_for_wide_part=0, min_rows_for_wide_part=0, max_replicated_merges_in_queue=1, old_parts_lifetime=0; diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index 4befe952a14..44303a1c532 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -5,9 +5,11 @@ SET insert_keeper_fault_injection_probability=0; -- disable fault injection; par drop table if exists rmt1; drop table if exists rmt2; create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '1') order by tuple() - settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, old_parts_lifetime=0, max_parts_to_merge_at_once=4; + settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, + cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime=0, max_parts_to_merge_at_once=4; create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '2') order by tuple() - settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, old_parts_lifetime=0, max_parts_to_merge_at_once=4; + settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, + cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime=0, max_parts_to_merge_at_once=4; -- insert part only on one replica system stop replicated sends rmt1; diff --git a/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql index 98427874160..b4504a55643 100644 --- a/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql +++ b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql @@ -24,7 +24,8 @@ CREATE TABLE wikistat1 ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/02494_zero_copy_and_projection', '1') ORDER BY (path, time) -SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, allow_remote_fs_zero_copy_replication=1, min_bytes_for_wide_part=0; +SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0, allow_remote_fs_zero_copy_replication=1, min_bytes_for_wide_part=0; CREATE TABLE wikistat2 ( @@ -49,7 +50,8 @@ CREATE TABLE wikistat2 ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/02494_zero_copy_and_projection', '2') ORDER BY (path, time) -SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, allow_remote_fs_zero_copy_replication=1, min_bytes_for_wide_part=0; +SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, + cleanup_thread_preferred_points_per_iteration=0, allow_remote_fs_zero_copy_replication=1, min_bytes_for_wide_part=0; INSERT INTO wikistat1 SELECT toDateTime('2020-10-01 00:00:00'), 'hello', 'world', '/data/path', 10 from numbers(100); diff --git a/tests/queries/0_stateless/02515_cleanup_async_insert_block_ids.sh b/tests/queries/0_stateless/02515_cleanup_async_insert_block_ids.sh index 458a5e95faa..bc6e7eeb214 100755 --- a/tests/queries/0_stateless/02515_cleanup_async_insert_block_ids.sh +++ b/tests/queries/0_stateless/02515_cleanup_async_insert_block_ids.sh @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -n --query " CREATE TABLE t_async_insert_cleanup ( KeyID UInt32 ) Engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/t_async_insert_cleanup', '{replica}') - ORDER BY (KeyID) SETTINGS cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, replicated_deduplication_window_for_async_inserts=10 + ORDER BY (KeyID) SETTINGS cleanup_delay_period = 1, cleanup_delay_period_random_add = 1, cleanup_thread_preferred_points_per_iteration=0, replicated_deduplication_window_for_async_inserts=10 " for i in {1..100}; do From c2b1e8ca0d00301cd6987eb20ced83e3ead7a168 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 22 May 2023 19:30:28 +0200 Subject: [PATCH 0534/2223] fix --- src/Parsers/IAST_fwd.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/IAST_fwd.h b/src/Parsers/IAST_fwd.h index 27116939a23..53d41d42d65 100644 --- a/src/Parsers/IAST_fwd.h +++ b/src/Parsers/IAST_fwd.h @@ -29,7 +29,7 @@ template inline typename DB::ASTs::size_type erase_if(DB::ASTs & asts, Predicate pred) { auto old_size = asts.size(); - std::erase_if(asts, pred); + asts.erase(std::remove_if(asts.begin(), asts.end(), pred), asts.end()); return old_size - asts.size(); } From 5b768ebd975ff9cf3567aba290f61be7458b0592 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 22 May 2023 19:32:32 +0200 Subject: [PATCH 0535/2223] update default settings for Replicated database --- src/Databases/DatabaseReplicatedSettings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseReplicatedSettings.h b/src/Databases/DatabaseReplicatedSettings.h index 9d52a82faf4..66c1a860b73 100644 --- a/src/Databases/DatabaseReplicatedSettings.h +++ b/src/Databases/DatabaseReplicatedSettings.h @@ -8,8 +8,8 @@ namespace DB class ASTStorage; #define LIST_OF_DATABASE_REPLICATED_SETTINGS(M, ALIAS) \ - M(Float, max_broken_tables_ratio, 0.5, "Do not recover replica automatically if the ratio of staled tables to all tables is greater", 0) \ - M(UInt64, max_replication_lag_to_enqueue, 10, "Replica will throw exception on attempt to execute query if its replication lag greater", 0) \ + M(Float, max_broken_tables_ratio, 1, "Do not recover replica automatically if the ratio of staled tables to all tables is greater", 0) \ + M(UInt64, max_replication_lag_to_enqueue, 50, "Replica will throw exception on attempt to execute query if its replication lag greater", 0) \ M(UInt64, wait_entry_commited_timeout_sec, 3600, "Replicas will try to cancel query if timeout exceed, but initiator host has not executed it yet", 0) \ M(String, collection_name, "", "A name of a collection defined in server's config where all info for cluster authentication is defined", 0) \ M(Bool, check_consistency, true, "Check consistency of local metadata and metadata in Keeper, do replica recovery on inconsistency", 0) \ From c4d862a16fa9d66c7c1fc3e67a615c15b9e37a60 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 May 2023 19:51:58 +0200 Subject: [PATCH 0536/2223] Make async reader work with any impl --- src/Backups/BackupEntryFromAppendOnlyFile.cpp | 1 + src/Backups/BackupEntryFromImmutableFile.cpp | 2 + src/Backups/BackupIO_Default.cpp | 2 +- src/Backups/BackupImpl.cpp | 2 +- src/Coordination/KeeperStateManager.cpp | 1 + src/Disks/IDisk.h | 2 + ....cpp => AsynchronousBoundedReadBuffer.cpp} | 127 ++++++------------ src/Disks/IO/AsynchronousBoundedReadBuffer.h | 96 +++++++++++++ ...ynchronousReadIndirectBufferFromRemoteFS.h | 111 --------------- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 61 +++------ src/Disks/IO/ReadBufferFromRemoteFSGather.h | 56 +++----- .../IO/ReadIndirectBufferFromRemoteFS.cpp | 2 +- src/Disks/IO/ReadIndirectBufferFromRemoteFS.h | 2 - .../AzureBlobStorage/AzureObjectStorage.cpp | 5 +- .../AzureBlobStorage/AzureObjectStorage.h | 1 - ...ObjectStorageRemoteMetadataRestoreHelper.h | 1 + .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 1 - src/Disks/ObjectStorages/IObjectStorage.cpp | 1 + src/Disks/ObjectStorages/IObjectStorage.h | 4 +- .../Local/LocalObjectStorage.cpp | 7 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 6 +- src/Disks/ObjectStorages/StoredObject.cpp | 14 ++ src/Disks/ObjectStorages/StoredObject.h | 2 + .../ObjectStorages/Web/WebObjectStorage.cpp | 5 +- src/IO/AsyncReadCounters.h | 1 + src/IO/ReadBufferFromFileDecorator.h | 2 - src/IO/SeekableReadBuffer.h | 2 - .../FilesystemReadPrefetchesLog.h | 2 + .../MergeTree/MergeTreeDeduplicationLog.cpp | 1 + src/Storages/StorageS3.cpp | 6 +- src/Storages/StorageSet.cpp | 1 + 31 files changed, 230 insertions(+), 297 deletions(-) rename src/Disks/IO/{AsynchronousReadIndirectBufferFromRemoteFS.cpp => AsynchronousBoundedReadBuffer.cpp} (72%) create mode 100644 src/Disks/IO/AsynchronousBoundedReadBuffer.h delete mode 100644 src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h create mode 100644 src/Disks/ObjectStorages/StoredObject.cpp diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index 1d73ab52820..5303d9abffd 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index cc635dd8541..d066db0c6ae 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include namespace DB diff --git a/src/Backups/BackupIO_Default.cpp b/src/Backups/BackupIO_Default.cpp index f7ba061cf3a..b36cb22498d 100644 --- a/src/Backups/BackupIO_Default.cpp +++ b/src/Backups/BackupIO_Default.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 7fcb42ec378..306236534b6 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 70687ba471c..8736fb7d4e3 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 6bbd7c26bec..5d75f3b70e5 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -20,6 +21,7 @@ #include #include #include +#include namespace fs = std::filesystem; diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp similarity index 72% rename from src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp rename to src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index 24b7042e459..611fd00a023 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -1,4 +1,4 @@ -#include "AsynchronousReadIndirectBufferFromRemoteFS.h" +#include "AsynchronousBoundedReadBuffer.h" #include #include @@ -43,105 +43,77 @@ namespace ErrorCodes } -AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS( +AsynchronousBoundedReadBuffer::AsynchronousBoundedReadBuffer( + ImplPtr impl_, IAsynchronousReader & reader_, const ReadSettings & settings_, - std::shared_ptr impl_, - std::shared_ptr async_read_counters_, - std::shared_ptr prefetches_log_) + AsyncReadCountersPtr async_read_counters_, + FilesystemReadPrefetchesLogPtr prefetches_log_) : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0) + , impl(std::move(impl_)) , read_settings(settings_) , reader(reader_) - , base_priority(settings_.priority) - , impl(impl_) , prefetch_buffer(settings_.prefetch_buffer_size) - , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr - ? CurrentThread::getQueryId() : "") + , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") , current_reader_id(getRandomASCIIString(8)) -#ifndef NDEBUG - , log(&Poco::Logger::get("AsynchronousBufferFromRemoteFS")) -#else - , log(&Poco::Logger::get("AsyncBuffer(" + impl->getFileName() + ")")) -#endif + , log(&Poco::Logger::get("AsynchronousBoundedReadBuffer")) , async_read_counters(async_read_counters_) , prefetches_log(prefetches_log_) { ProfileEvents::increment(ProfileEvents::RemoteFSBuffers); } -String AsynchronousReadIndirectBufferFromRemoteFS::getFileName() const +bool AsynchronousBoundedReadBuffer::hasPendingDataToRead() { - return impl->getFileName(); -} - - -String AsynchronousReadIndirectBufferFromRemoteFS::getInfoForLog() -{ - return impl->getInfoForLog(); -} - -size_t AsynchronousReadIndirectBufferFromRemoteFS::getFileSize() -{ - return impl->getFileSize(); -} - -bool AsynchronousReadIndirectBufferFromRemoteFS::hasPendingDataToRead() -{ - /** - * Note: read_until_position here can be std::nullopt only for non-MergeTree tables. - * For mergeTree tables it must be guaranteed that setReadUntilPosition() or - * setReadUntilEnd() is called before any read or prefetch. - * setReadUntilEnd() always sets read_until_position to file size. - * setReadUntilPosition(pos) always has pos > 0, because if - * right_offset_in_compressed_file is 0, then setReadUntilEnd() is used. - */ if (read_until_position) { - /// Everything is already read. - if (file_offset_of_buffer_end == *read_until_position) + if (file_offset_of_buffer_end == *read_until_position) /// Everything is already read. return false; if (file_offset_of_buffer_end > *read_until_position) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {}, info: {})", - file_offset_of_buffer_end, *read_until_position, impl->getInfoForLog()); + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Read beyond last offset ({} > {}, info: {})", + file_offset_of_buffer_end, *read_until_position, impl->getInfoForLog()); + } } return true; } - -std::future AsynchronousReadIndirectBufferFromRemoteFS::asyncReadInto(char * data, size_t size, int64_t priority) +std::future +AsynchronousBoundedReadBuffer::asyncReadInto(char * data, size_t size, int64_t priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(*impl, async_read_counters); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = base_priority + priority; + request.priority = read_settings.priority + priority; request.ignore = bytes_to_ignore; return reader.submit(request); } - -void AsynchronousReadIndirectBufferFromRemoteFS::prefetch(int64_t priority) +void AsynchronousBoundedReadBuffer::prefetch(int64_t priority) { if (prefetch_future.valid()) return; - /// Check boundary, which was set in readUntilPosition(). if (!hasPendingDataToRead()) return; - last_prefetch_info.submit_time = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + last_prefetch_info.submit_time = std::chrono::duration_cast( + std::chrono::system_clock::now().time_since_epoch()).count(); last_prefetch_info.priority = priority; - /// Prefetch even in case hasPendingData() == true. - chassert(prefetch_buffer.size() == read_settings.prefetch_buffer_size || prefetch_buffer.size() == read_settings.remote_fs_buffer_size); + chassert(prefetch_buffer.size() == read_settings.prefetch_buffer_size + || prefetch_buffer.size() == read_settings.remote_fs_buffer_size); prefetch_future = asyncReadInto(prefetch_buffer.data(), prefetch_buffer.size(), priority); ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); } -void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t position) +void AsynchronousBoundedReadBuffer::setReadUntilPosition(size_t position) { if (!read_until_position || position != *read_until_position) { @@ -157,21 +129,16 @@ void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t pos } } - -void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilEnd() +void AsynchronousBoundedReadBuffer::appendToPrefetchLog( + FilesystemPrefetchState state, + int64_t size, + const std::unique_ptr & execution_watch) { - setReadUntilPosition(impl->getFileSize()); -} - - -void AsynchronousReadIndirectBufferFromRemoteFS::appendToPrefetchLog(FilesystemPrefetchState state, int64_t size, const std::unique_ptr & execution_watch) -{ - const auto & object = impl->getCurrentObject(); FilesystemReadPrefetchesLogElement elem { .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), .query_id = query_id, - .path = object.local_path, + .path = impl->getFileName(), .offset = file_offset_of_buffer_end, .size = size, .prefetch_submit_time = last_prefetch_info.submit_time, @@ -187,7 +154,7 @@ void AsynchronousReadIndirectBufferFromRemoteFS::appendToPrefetchLog(FilesystemP } -bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() +bool AsynchronousBoundedReadBuffer::nextImpl() { if (!hasPendingDataToRead()) return false; @@ -245,14 +212,14 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() /// In case of multiple files for the same file in clickhouse (i.e. log family) /// file_offset_of_buffer_end will not match getImplementationBufferOffset() /// so we use [impl->getImplementationBufferOffset(), impl->getFileSize()] - chassert(file_offset_of_buffer_end >= impl->getImplementationBufferOffset()); + chassert(file_offset_of_buffer_end >= impl->getFileOffsetOfBufferEnd()); chassert(file_offset_of_buffer_end <= impl->getFileSize()); return bytes_read; } -off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence) +off_t AsynchronousBoundedReadBuffer::seek(off_t offset, int whence) { ProfileEvents::increment(ProfileEvents::RemoteFSSeeks); @@ -268,7 +235,7 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence) } else { - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "ReadBufferFromFileDescriptor::seek expects SEEK_SET or SEEK_CUR as whence"); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Expected SEEK_SET or SEEK_CUR as whence"); } /// Position is unchanged. @@ -322,9 +289,8 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence) if (read_until_position && new_pos > *read_until_position) { ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset); - impl->reset(); - file_offset_of_buffer_end = new_pos = *read_until_position; /// read_until_position is a non-included boundary. + impl->seek(file_offset_of_buffer_end, SEEK_SET); return new_pos; } @@ -332,8 +298,7 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence) * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. * Note: we read in range [file_offset_of_buffer_end, read_until_position). */ - if (impl->initialized() - && read_until_position && new_pos < *read_until_position + if (read_until_position && new_pos < *read_until_position && new_pos > file_offset_of_buffer_end && new_pos < file_offset_of_buffer_end + read_settings.remote_read_min_bytes_for_seek) { @@ -342,31 +307,21 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset, int whence) } else { - if (impl->initialized()) - { - ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset); - impl->reset(); - } + ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset); file_offset_of_buffer_end = new_pos; + impl->seek(file_offset_of_buffer_end, SEEK_SET); } return new_pos; } -off_t AsynchronousReadIndirectBufferFromRemoteFS::getPosition() -{ - return file_offset_of_buffer_end - available() + bytes_to_ignore; -} - - -void AsynchronousReadIndirectBufferFromRemoteFS::finalize() +void AsynchronousBoundedReadBuffer::finalize() { resetPrefetch(FilesystemPrefetchState::UNNEEDED); } - -AsynchronousReadIndirectBufferFromRemoteFS::~AsynchronousReadIndirectBufferFromRemoteFS() +AsynchronousBoundedReadBuffer::~AsynchronousBoundedReadBuffer() { try { @@ -378,7 +333,7 @@ AsynchronousReadIndirectBufferFromRemoteFS::~AsynchronousReadIndirectBufferFromR } } -void AsynchronousReadIndirectBufferFromRemoteFS::resetPrefetch(FilesystemPrefetchState state) +void AsynchronousBoundedReadBuffer::resetPrefetch(FilesystemPrefetchState state) { if (!prefetch_future.valid()) return; diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.h b/src/Disks/IO/AsynchronousBoundedReadBuffer.h new file mode 100644 index 00000000000..45256cdfac2 --- /dev/null +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.h @@ -0,0 +1,96 @@ +#pragma once + +#include "config.h" +#include +#include +#include +#include +#include + +namespace Poco { class Logger; } + +namespace DB +{ + +struct AsyncReadCounters; +using AsyncReadCountersPtr = std::shared_ptr; +class ReadBufferFromRemoteFSGather; + +class AsynchronousBoundedReadBuffer : public ReadBufferFromFileBase +{ +public: + using Impl = ReadBufferFromFileBase; + using ImplPtr = std::unique_ptr; + + explicit AsynchronousBoundedReadBuffer( + ImplPtr impl_, + IAsynchronousReader & reader_, + const ReadSettings & settings_, + AsyncReadCountersPtr async_read_counters_ = nullptr, + FilesystemReadPrefetchesLogPtr prefetches_log_ = nullptr); + + ~AsynchronousBoundedReadBuffer() override; + + String getFileName() const override { return impl->getFileName(); } + + size_t getFileSize() override { return impl->getFileSize(); } + + String getInfoForLog() override { return impl->getInfoForLog(); } + + off_t seek(off_t offset_, int whence) override; + + void prefetch(int64_t priority) override; + + void setReadUntilPosition(size_t position) override; /// [..., position). + + void setReadUntilEnd() override { return setReadUntilPosition(getFileSize()); } + + off_t getPosition() override { return file_offset_of_buffer_end - available() + bytes_to_ignore; } + +private: + const ImplPtr impl; + const ReadSettings read_settings; + IAsynchronousReader & reader; + + size_t file_offset_of_buffer_end = 0; + std::optional read_until_position; + /// If nonzero then working_buffer is empty. + /// If a prefetch is in flight, the prefetch task has been instructed to ignore this many bytes. + size_t bytes_to_ignore = 0; + + Memory<> prefetch_buffer; + std::future prefetch_future; + + const std::string query_id; + const std::string current_reader_id; + + Poco::Logger * log; + + AsyncReadCountersPtr async_read_counters; + FilesystemReadPrefetchesLogPtr prefetches_log; + + struct LastPrefetchInfo + { + UInt64 submit_time = 0; + size_t priority = 0; + }; + LastPrefetchInfo last_prefetch_info; + + bool nextImpl() override; + + void finalize(); + + bool hasPendingDataToRead(); + + void appendToPrefetchLog( + FilesystemPrefetchState state, + int64_t size, + const std::unique_ptr & execution_watch); + + std::future asyncReadInto(char * data, size_t size, int64_t priority); + + void resetPrefetch(FilesystemPrefetchState state); + +}; + +} diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h deleted file mode 100644 index e8fb3fe248b..00000000000 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ /dev/null @@ -1,111 +0,0 @@ -#pragma once - -#include "config.h" -#include -#include -#include -#include -#include - -namespace Poco { class Logger; } - -namespace DB -{ - -struct AsyncReadCounters; -class ReadBufferFromRemoteFSGather; - -/** - * Reads data from S3/HDFS/Web using stored paths in metadata. -* This class is an asynchronous version of ReadIndirectBufferFromRemoteFS. -* -* Buffers chain for diskS3: -* AsynchronousIndirectReadBufferFromRemoteFS -> ReadBufferFromRemoteFS -> -* -> ReadBufferFromS3 -> ReadBufferFromIStream. -* -* Buffers chain for diskWeb: -* AsynchronousIndirectReadBufferFromRemoteFS -> ReadBufferFromRemoteFS -> -* -> ReadIndirectBufferFromWebServer -> ReadBufferFromHTTP -> ReadBufferFromIStream. -* -* We pass either `memory` or `prefetch_buffer` through all this chain and return it back. -*/ -class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase -{ -public: - explicit AsynchronousReadIndirectBufferFromRemoteFS( - IAsynchronousReader & reader_, const ReadSettings & settings_, - std::shared_ptr impl_, - std::shared_ptr async_read_counters_, - std::shared_ptr prefetches_log_); - - ~AsynchronousReadIndirectBufferFromRemoteFS() override; - - off_t seek(off_t offset_, int whence) override; - - off_t getPosition() override; - - String getFileName() const override; - - void prefetch(int64_t priority) override; - - void setReadUntilPosition(size_t position) override; /// [..., position). - - void setReadUntilEnd() override; - - String getInfoForLog() override; - - size_t getFileSize() override; - - bool isIntegratedWithFilesystemCache() const override { return true; } - -private: - bool nextImpl() override; - - void finalize(); - - bool hasPendingDataToRead(); - - void appendToPrefetchLog(FilesystemPrefetchState state, int64_t size, const std::unique_ptr & execution_watch); - - std::future asyncReadInto(char * data, size_t size, int64_t priority); - - void resetPrefetch(FilesystemPrefetchState state); - - ReadSettings read_settings; - - IAsynchronousReader & reader; - - int64_t base_priority; - - std::shared_ptr impl; - - std::future prefetch_future; - - size_t file_offset_of_buffer_end = 0; - - Memory<> prefetch_buffer; - - std::string query_id; - - std::string current_reader_id; - - /// If nonzero then working_buffer is empty. - /// If a prefetch is in flight, the prefetch task has been instructed to ignore this many bytes. - size_t bytes_to_ignore = 0; - - std::optional read_until_position; - - Poco::Logger * log; - - std::shared_ptr async_read_counters; - std::shared_ptr prefetches_log; - - struct LastPrefetchInfo - { - UInt64 submit_time = 0; - size_t priority = 0; - }; - LastPrefetchInfo last_prefetch_info; -}; - -} diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 68b5a9c9d96..8db93e3ed6e 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -12,22 +12,24 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_SEEK_THROUGH_FILE; +} ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( ReadBufferCreator && read_buffer_creator_, const StoredObjects & blobs_to_read_, const ReadSettings & settings_, std::shared_ptr cache_log_) - : ReadBuffer(nullptr, 0) - , read_buffer_creator(std::move(read_buffer_creator_)) - , blobs_to_read(blobs_to_read_) + : ReadBufferFromFileBase(0, nullptr, 0) , settings(settings_) + , blobs_to_read(blobs_to_read_) + , read_buffer_creator(std::move(read_buffer_creator_)) + , cache_log(settings.enable_filesystem_cache_log ? cache_log_ : nullptr) , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") , log(&Poco::Logger::get("ReadBufferFromRemoteFSGather")) { - if (cache_log_ && settings.enable_filesystem_cache_log) - cache_log = cache_log_; - if (!blobs_to_read.empty()) current_object = blobs_to_read.front(); @@ -38,9 +40,9 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object) { - if (current_buf != nullptr && !with_cache) + if (current_buf && !with_cache) { - appendFilesystemCacheLog(); + appendUncachedReadInfo(); } current_object = object; @@ -70,7 +72,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c return current_read_buffer_creator(); } -void ReadBufferFromRemoteFSGather::appendFilesystemCacheLog() +void ReadBufferFromRemoteFSGather::appendUncachedReadInfo() { if (!cache_log || current_object.remote_path.empty()) return; @@ -218,44 +220,23 @@ bool ReadBufferFromRemoteFSGather::readImpl() return result; } -size_t ReadBufferFromRemoteFSGather::getFileOffsetOfBufferEnd() const -{ - return file_offset_of_buffer_end; -} - void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t position) { - if (position != read_until_position) - { - read_until_position = position; - reset(); - } -} + if (position == read_until_position) + return; -void ReadBufferFromRemoteFSGather::reset() -{ + read_until_position = position; current_buf.reset(); } -String ReadBufferFromRemoteFSGather::getFileName() const +off_t ReadBufferFromRemoteFSGather::seek(off_t offset, int whence) { - return current_object.remote_path; -} + if (whence != SEEK_SET) + throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only seeking with SEEK_SET is allowed"); -size_t ReadBufferFromRemoteFSGather::getFileSize() const -{ - size_t size = 0; - for (const auto & object : blobs_to_read) - size += object.bytes_size; - return size; -} - -String ReadBufferFromRemoteFSGather::getInfoForLog() -{ - if (!current_buf) - return ""; - - return current_buf->getInfoForLog(); + file_offset_of_buffer_end = offset; + current_buf.reset(); + return file_offset_of_buffer_end; } size_t ReadBufferFromRemoteFSGather::getImplementationBufferOffset() const @@ -269,7 +250,7 @@ size_t ReadBufferFromRemoteFSGather::getImplementationBufferOffset() const ReadBufferFromRemoteFSGather::~ReadBufferFromRemoteFSGather() { if (!with_cache) - appendFilesystemCacheLog(); + appendUncachedReadInfo(); } } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 8c55f747e5b..08e5b97d018 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -10,12 +10,13 @@ namespace Poco { class Logger; } namespace DB { +class FilesystemCacheLog; /** * Remote disk might need to split one clickhouse file into multiple files in remote fs. * This class works like a proxy to allow transition from one file into multiple. */ -class ReadBufferFromRemoteFSGather final : public ReadBuffer +class ReadBufferFromRemoteFSGather final : public ReadBufferFromFileBase { friend class ReadIndirectBufferFromRemoteFS; @@ -30,25 +31,25 @@ public: ~ReadBufferFromRemoteFSGather() override; - String getFileName() const; + String getFileName() const override { return current_object.remote_path; } - void reset(); + String getInfoForLog() override { return current_buf ? current_buf->getInfoForLog() : ""; } void setReadUntilPosition(size_t position) override; IAsynchronousReader::Result readInto(char * data, size_t size, size_t offset, size_t ignore) override; - size_t getFileSize() const; + size_t getFileSize() override { return getTotalSize(blobs_to_read); } - size_t getFileOffsetOfBufferEnd() const; + size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; } bool initialized() const { return current_buf != nullptr; } - String getInfoForLog(); - size_t getImplementationBufferOffset() const; - const StoredObject & getCurrentObject() const { return current_object; } + off_t seek(off_t offset, int whence) override; + + off_t getPosition() override { return file_offset_of_buffer_end - available() + bytes_to_ignore; } private: SeekableReadBufferPtr createImplementationBuffer(const StoredObject & object); @@ -61,40 +62,25 @@ private: bool moveToNextBuffer(); - void appendFilesystemCacheLog(); - - ReadBufferCreator read_buffer_creator; - - StoredObjects blobs_to_read; - - ReadSettings settings; - - size_t read_until_position = 0; - - StoredObject current_object; + void appendUncachedReadInfo(); + const ReadSettings settings; + const StoredObjects blobs_to_read; + const ReadBufferCreator read_buffer_creator; + const std::shared_ptr cache_log; + const String query_id; bool with_cache; - String query_id; - - Poco::Logger * log; - - SeekableReadBufferPtr current_buf; - - size_t current_buf_idx = 0; - + size_t read_until_position = 0; size_t file_offset_of_buffer_end = 0; - - /** - * File: |___________________| - * Buffer: |~~~~~~~| - * file_offset_of_buffer_end: ^ - */ size_t bytes_to_ignore = 0; - size_t total_bytes_read_from_current_file = 0; - std::shared_ptr cache_log; + StoredObject current_object; + size_t current_buf_idx = 0; + SeekableReadBufferPtr current_buf; + + Poco::Logger * log; }; } diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp index 8a33a6ce9a1..04521011599 100644 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp @@ -82,8 +82,8 @@ off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) else throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET or SEEK_CUR modes are allowed."); - impl->reset(); resetWorkingBuffer(); + impl->seek(file_offset_of_buffer_end, SEEK_SET); file_offset_of_buffer_end = impl->file_offset_of_buffer_end; return impl->file_offset_of_buffer_end; diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h index 0424c2e56d3..19647b1fa39 100644 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h @@ -31,8 +31,6 @@ public: void setReadUntilEnd() override; - bool isIntegratedWithFilesystemCache() const override { return true; } - size_t getFileSize() override; private: diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 9db5d13a7f8..997002a1e4f 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -112,8 +113,8 @@ std::unique_ptr AzureObjectStorage::readObjects( /// NOL if (disk_read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - reader, disk_read_settings, std::move(reader_impl), + return std::make_unique( + std::move(reader_impl), reader, disk_read_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 648016fb732..9616fd84770 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -5,7 +5,6 @@ #include #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h index 19011a04722..cb8d9b8a5af 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index e50e410823d..122414e3082 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -7,7 +7,6 @@ #include #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 52e8b1a465d..9d3e09bd4a7 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index f4f1b063ade..08969943353 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -12,12 +12,14 @@ #include #include #include +#include -#include #include #include #include #include +#include +#include namespace DB diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 8cf0b27a517..f3da39d6866 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -64,12 +65,12 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL global_context->getFilesystemCacheLog()); /// We use `remove_fs_method` (not `local_fs_method`) because we are about to use - /// AsynchronousReadIndirectBufferFromRemoteFS which works by the remote_fs_* settings. + /// AsynchronousBoundedReadBuffer which works by the remote_fs_* settings. if (modified_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - reader, modified_settings, std::move(impl), + return std::make_unique( + std::move(impl), reader, modified_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 79b3d3a2b8b..89159fc2d5a 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include #include @@ -128,8 +128,8 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - reader, disk_read_settings, std::move(s3_impl), + return std::make_unique( + std::move(s3_impl), reader, disk_read_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); } diff --git a/src/Disks/ObjectStorages/StoredObject.cpp b/src/Disks/ObjectStorages/StoredObject.cpp new file mode 100644 index 00000000000..6a363c64107 --- /dev/null +++ b/src/Disks/ObjectStorages/StoredObject.cpp @@ -0,0 +1,14 @@ +#include + +namespace DB +{ + +size_t getTotalSize(const StoredObjects & objects) +{ + size_t size = 0; + for (const auto & object : objects) + size += object.bytes_size; + return size; +} + +} diff --git a/src/Disks/ObjectStorages/StoredObject.h b/src/Disks/ObjectStorages/StoredObject.h index 94c9fd0946d..8afbb116a83 100644 --- a/src/Disks/ObjectStorages/StoredObject.h +++ b/src/Disks/ObjectStorages/StoredObject.h @@ -29,4 +29,6 @@ struct StoredObject using StoredObjects = std::vector; +size_t getTotalSize(const StoredObjects & objects); + } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index c57b75f4038..4aa0ac5708c 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include #include @@ -190,8 +191,8 @@ std::unique_ptr WebObjectStorage::readObject( /// NOLINT if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - reader, read_settings, std::move(web_impl), + return std::make_unique( + std::move(web_impl), reader, read_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); } diff --git a/src/IO/AsyncReadCounters.h b/src/IO/AsyncReadCounters.h index 1d941b4fc47..1f84b2a214b 100644 --- a/src/IO/AsyncReadCounters.h +++ b/src/IO/AsyncReadCounters.h @@ -27,5 +27,6 @@ struct AsyncReadCounters void dumpToMapColumn(IColumn * column) const; }; +using AsyncReadCountersPtr = std::shared_ptr; } diff --git a/src/IO/ReadBufferFromFileDecorator.h b/src/IO/ReadBufferFromFileDecorator.h index 1d035e8d74b..6e62c7f741b 100644 --- a/src/IO/ReadBufferFromFileDecorator.h +++ b/src/IO/ReadBufferFromFileDecorator.h @@ -27,8 +27,6 @@ public: ReadBuffer & getWrappedReadBuffer() { return *impl; } - bool isIntegratedWithFilesystemCache() const override { return impl->isIntegratedWithFilesystemCache(); } - size_t getFileSize() override; protected: diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index b055aa57975..736ab5bbc71 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -49,8 +49,6 @@ public: /// If true, setReadUntilPosition() guarantees that eof will be reported at the given position. virtual bool supportsRightBoundedReads() const { return false; } - virtual bool isIntegratedWithFilesystemCache() const { return false; } - /// Returns true if seek() actually works, false if seek() will always throw (or make subsequent /// nextImpl() calls throw). /// diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.h b/src/Interpreters/FilesystemReadPrefetchesLog.h index a7672c49d91..7052cf2769d 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.h +++ b/src/Interpreters/FilesystemReadPrefetchesLog.h @@ -45,4 +45,6 @@ public: using SystemLog::SystemLog; }; +using FilesystemReadPrefetchesLogPtr = std::shared_ptr; + } diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 503b4aac51d..09a04f13fc7 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a4d9dc9f2e3..b8df218ad8a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -31,7 +31,7 @@ #include #include -#include +#include #include #include @@ -676,8 +676,8 @@ std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size; auto & pool_reader = context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - auto async_reader = std::make_unique( - pool_reader, modified_settings, std::move(s3_impl), + auto async_reader = std::make_unique( + std::move(s3_impl), pool_reader, modified_settings, context->getAsyncReadCounters(), context->getFilesystemReadPrefetchesLog()); async_reader->setReadUntilEnd(); diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 32721a0020b..00b5dbfc5e3 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include From 17b639c612e5939083c2d1463f63e80686d049e0 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 May 2023 18:22:05 +0000 Subject: [PATCH 0537/2223] Make better --- src/Client/ConnectionEstablisher.cpp | 7 +- src/Client/ConnectionEstablisher.h | 2 +- src/Client/PacketReceiver.cpp | 2 +- src/Client/PacketReceiver.h | 2 +- src/Common/AsyncTaskExecutor.cpp | 20 +- src/Common/AsyncTaskExecutor.h | 4 +- src/Common/Fiber.h | 187 +++++++----------- src/Common/OpenTelemetryTraceContext.cpp | 2 +- .../RemoteQueryExecutorReadContext.cpp | 2 +- .../RemoteQueryExecutorReadContext.h | 2 +- .../01455_opentelemetry_distributed.reference | 2 +- 11 files changed, 90 insertions(+), 142 deletions(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 98051a50eb3..a2c51cc0382 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -4,8 +4,6 @@ namespace ProfileEvents { - extern const Event DistributedConnectionTries; - extern const Event DistributedConnectionUsable; extern const Event DistributedConnectionMissingTable; extern const Event DistributedConnectionStaleReplica; } @@ -37,7 +35,6 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: SCOPE_EXIT(is_finished = true); try { - ProfileEvents::increment(ProfileEvents::DistributedConnectionTries); result.entry = pool->get(*timeouts, settings, /* force_connected = */ false); AsyncCallbackSetter async_setter(&*result.entry, std::move(async_callback)); @@ -48,7 +45,6 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) { result.entry->forceConnected(*timeouts); - ProfileEvents::increment(ProfileEvents::DistributedConnectionUsable); result.is_usable = true; result.is_up_to_date = true; return; @@ -69,7 +65,6 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: return; } - ProfileEvents::increment(ProfileEvents::DistributedConnectionUsable); result.is_usable = true; UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0; @@ -121,7 +116,7 @@ ConnectionEstablisherAsync::ConnectionEstablisherAsync( epoll.add(timeout_descriptor.getDescriptor()); } -void ConnectionEstablisherAsync::Task::run(AsyncCallback async_callback, ResumeCallback) +void ConnectionEstablisherAsync::Task::run(AsyncCallback async_callback, SuspendCallback) { connection_establisher_async.reset(); connection_establisher_async.connection_establisher.setAsyncCallback(async_callback); diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index 5993c9e066f..5b58563dc01 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -91,7 +91,7 @@ private: ConnectionEstablisherAsync & connection_establisher_async; - void run(AsyncCallback async_callback, ResumeCallback suspend_callback) override; + void run(AsyncCallback async_callback, SuspendCallback suspend_callback) override; }; void cancelAfter() override; diff --git a/src/Client/PacketReceiver.cpp b/src/Client/PacketReceiver.cpp index 7c6dbb19212..9934a69bce1 100644 --- a/src/Client/PacketReceiver.cpp +++ b/src/Client/PacketReceiver.cpp @@ -57,7 +57,7 @@ bool PacketReceiver::checkTimeout() return true; } -void PacketReceiver::Task::run(AsyncCallback async_callback, ResumeCallback suspend_callback) +void PacketReceiver::Task::run(AsyncCallback async_callback, SuspendCallback suspend_callback) { while (true) { diff --git a/src/Client/PacketReceiver.h b/src/Client/PacketReceiver.h index 99e5f7c2f10..deedf5cccdc 100644 --- a/src/Client/PacketReceiver.h +++ b/src/Client/PacketReceiver.h @@ -57,7 +57,7 @@ private: PacketReceiver & receiver; - void run(AsyncCallback async_callback, ResumeCallback suspend_callback) override; + void run(AsyncCallback async_callback, SuspendCallback suspend_callback) override; }; /// When epoll file descriptor is ready, check if it's an expired timeout. diff --git a/src/Common/AsyncTaskExecutor.cpp b/src/Common/AsyncTaskExecutor.cpp index 757901093eb..b824a0a5b31 100644 --- a/src/Common/AsyncTaskExecutor.cpp +++ b/src/Common/AsyncTaskExecutor.cpp @@ -59,30 +59,19 @@ struct AsyncTaskExecutor::Routine struct AsyncCallback { AsyncTaskExecutor & executor; - Fiber::Impl & fiber_impl; + SuspendCallback suspend_callback; void operator()(int fd, Poco::Timespan timeout, AsyncEventTimeoutType type, const std::string & desc, uint32_t events) { executor.processAsyncEvent(fd, timeout, type, desc, events); - fiber_impl = std::move(fiber_impl).resume(); + suspend_callback(); executor.clearAsyncEvent(); } }; - struct ResumeCallback + void operator()(SuspendCallback suspend_callback) { - Fiber::Impl & fiber_impl; - - void operator()() - { - fiber_impl = std::move(fiber_impl).resume(); - } - }; - - Fiber::Impl operator()(Fiber::Impl && sink) - { - auto async_callback = AsyncCallback{executor, sink}; - auto suspend_callback = ResumeCallback{sink}; + auto async_callback = AsyncCallback{executor, suspend_callback}; try { executor.task->run(async_callback, suspend_callback); @@ -100,7 +89,6 @@ struct AsyncTaskExecutor::Routine } executor.routine_is_finished = true; - return std::move(sink); } }; diff --git a/src/Common/AsyncTaskExecutor.h b/src/Common/AsyncTaskExecutor.h index b71a5820a4e..9cc6cdb9c35 100644 --- a/src/Common/AsyncTaskExecutor.h +++ b/src/Common/AsyncTaskExecutor.h @@ -22,7 +22,7 @@ enum class AsyncEventTimeoutType }; using AsyncCallback = std::function; -using ResumeCallback = std::function; +using SuspendCallback = std::function; struct FiberInfo { @@ -38,7 +38,7 @@ struct FiberInfo struct AsyncTask { public: - virtual void run(AsyncCallback async_callback, ResumeCallback suspend_callback) = 0; + virtual void run(AsyncCallback async_callback, SuspendCallback suspend_callback) = 0; virtual ~AsyncTask() = default; }; diff --git a/src/Common/Fiber.h b/src/Common/Fiber.h index 69858c127a7..f48ace149f4 100644 --- a/src/Common/Fiber.h +++ b/src/Common/Fiber.h @@ -3,6 +3,7 @@ /// BOOST_USE_ASAN, BOOST_USE_TSAN and BOOST_USE_UCONTEXT should be correctly defined for sanitizers. #include #include +#include /// Class wrapper for boost::context::fiber. /// It tracks current executing fiber for thread and @@ -10,17 +11,14 @@ /// that will be destroyed on fiber destructor. class Fiber { -public: - struct CleanUpFn - { - virtual void operator()(void *) = 0; - virtual ~CleanUpFn() = default; - }; - +private: using Impl = boost::context::fiber; + using FiberPtr = Fiber *; + template friend class FiberLocal; +public: template< typename StackAlloc, typename Fn> - Fiber(StackAlloc && salloc, Fn && fn) : impl(std::allocator_arg_t(), std::forward(salloc), std::forward(fn)) + Fiber(StackAlloc && salloc, Fn && fn) : impl(std::allocator_arg_t(), std::forward(salloc), RoutineImpl(std::forward(fn))) { } @@ -32,12 +30,6 @@ public: Fiber(const Fiber &) = delete; Fiber & operator =(const Fiber &) = delete; - ~Fiber() - { - for (auto & [_, data] : local_data) - (*data.cleanup_fn)(data.ptr); - } - explicit operator bool() const { return impl.operator bool(); @@ -46,95 +38,82 @@ public: void resume() { /// Update information about current executing fiber. - auto & current_fiber_info = getCurrentFiberInfo(); - auto parent_fiber_info = current_fiber_info; - current_fiber_info = FiberInfo{this, &parent_fiber_info}; + FiberPtr & current_fiber = getCurrentFiber(); + FiberPtr parent_fiber = current_fiber; + current_fiber = this; impl = std::move(impl).resume(); - /// Restore current fiber info. - current_fiber_info = parent_fiber_info; - } - - /// Set pointer to fiber-specific data, it will be stored in hash-map - /// using provided key and cleaned on fiber destructor using provided - /// cleanup function. - void setLocalData(void * key, void * ptr, CleanUpFn * cleanup_fn) - { - local_data[key] = FiberLocalData{ptr, cleanup_fn}; - } - - /// Get pointer to fiber-specific data by key. - /// If no data was stored by this key, return nullptr. - void * getLocalData(void * key) - { - return local_data[key].ptr; - } - - struct FiberInfo - { - Fiber * fiber = nullptr; - FiberInfo * parent_info = nullptr; - }; - - static FiberInfo & getCurrentFiberInfo() - { - thread_local static FiberInfo current_fiber_info; - return current_fiber_info; + /// Restore parent fiber. + current_fiber = parent_fiber; } private: - struct FiberLocalData + template + struct RoutineImpl { - void * ptr; - CleanUpFn * cleanup_fn; + struct SuspendCallback + { + Impl & impl; + + void operator()() + { + impl = std::move(impl).resume(); + } + }; + + explicit RoutineImpl(Fn && fn_) : fn(std::move(fn_)) + { + } + + Impl operator()(Impl && sink) + { + SuspendCallback suspend_callback{sink}; + fn(suspend_callback); + return std::move(sink); + } + + Fn fn; }; + static FiberPtr & getCurrentFiber() + { + thread_local static FiberPtr current_fiber; + return current_fiber; + } + + /// Special wrapper to store data in uniquer_ptr. + struct DataWrapper + { + virtual ~DataWrapper() = default; + }; + + using DataPtr = std::unique_ptr; + + /// Get reference to fiber-specific data by key + /// (the pointer to the structure that uses this data). + DataPtr & getLocalData(void * key) + { + return local_data[key]; + } + + Impl && release() + { + return std::move(impl); + } + Impl impl; - std::unordered_map local_data; + std::map local_data; }; /// Implementation for fiber local variable. -/// If we are not in fiber, it returns thread local data. -/// If we are in fiber, it returns fiber local data. +/// If we are in fiber, it returns fiber local data, +/// otherwise it returns it's single field. /// Fiber local data is destroyed in Fiber destructor. -/// On first request, fiber local data is copied from parent -/// fiber data or from current thread data if there is no parent fiber. /// Implementation is similar to boost::fiber::fiber_specific_ptr /// (we cannot use it because we don't use boost::fiber API. template class FiberLocal { public: - struct DefaultCleanUpFn : public Fiber::CleanUpFn - { - void operator()(void * data) override - { - delete static_cast(data); - } - }; - - struct CustomCleanUpFn : public Fiber::CleanUpFn - { - explicit CustomCleanUpFn(void (*fn_)(T*)) : fn(fn_) - { - } - - void operator()(void * data) override - { - if (likely(fn != nullptr)) - fn(static_cast(data)); - } - - void (*fn)(T*); - }; - - FiberLocal() : cleanup_fn(std::make_unique()) - { - } - - explicit FiberLocal(void (*fn)(T*)) : cleanup_fn(std::make_unique(fn)) - { - } - T & operator*() { return get(); @@ -146,39 +125,25 @@ public: } private: - friend Fiber; + struct DataWrapperImpl : public Fiber::DataWrapper + { + T impl; + }; T & get() { - return getInstanceForFiber(Fiber::getCurrentFiberInfo()); - } + Fiber * current_fiber = Fiber::getCurrentFiber(); + if (!current_fiber) + return main_instance; - T & getInstanceForFiber(const Fiber::FiberInfo & fiber_info) - { - /// If it's not a fiber, return thread local instance. - if (!fiber_info.fiber) - return getThreadLocalInstance(); - - T * ptr = static_cast(fiber_info.fiber->getLocalData(this)); - /// If it's the first request, we need to initialize instance for the fiber - /// using instance from parent fiber or main thread that executes this fiber. + Fiber::DataPtr & ptr = current_fiber->getLocalData(this); + /// Initialize instance on first request. if (!ptr) - { - auto parent_instance = getInstanceForFiber(*fiber_info.parent_info); - /// Crete new object and store pointer inside Fiber, so it will be destroyed in Fiber destructor. - ptr = new T(parent_instance); - fiber_info.fiber->setLocalData(this, ptr, cleanup_fn.get()); - } + ptr = std::make_unique(); - return *ptr; + return dynamic_cast(ptr.get())->impl; } - T & getThreadLocalInstance() - { - static thread_local T thread_local_instance; - return thread_local_instance; - } - - std::unique_ptr cleanup_fn; + T main_instance; }; diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 515c8f346df..d121fbf67b7 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -15,7 +15,7 @@ namespace OpenTelemetry { /// This code can be executed inside fibers, we should use fiber local tracing context. -FiberLocal current_trace_context; +thread_local FiberLocal current_trace_context; bool Span::addAttribute(std::string_view name, UInt64 value) noexcept { diff --git a/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp b/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp index 0b23b4836ce..06d4a8cef87 100644 --- a/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp @@ -34,7 +34,7 @@ bool RemoteQueryExecutorReadContext::checkBeforeTaskResume() } -void RemoteQueryExecutorReadContext::Task::run(AsyncCallback async_callback, ResumeCallback suspend_callback) +void RemoteQueryExecutorReadContext::Task::run(AsyncCallback async_callback, SuspendCallback suspend_callback) { read_context.executor.sendQueryUnlocked(ClientInfo::QueryKind::SECONDARY_QUERY, async_callback); read_context.is_query_sent = true; diff --git a/src/QueryPipeline/RemoteQueryExecutorReadContext.h b/src/QueryPipeline/RemoteQueryExecutorReadContext.h index 4d1d6c17606..adfc0c5eacf 100644 --- a/src/QueryPipeline/RemoteQueryExecutorReadContext.h +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.h @@ -58,7 +58,7 @@ private: RemoteQueryExecutorReadContext & read_context; - void run(AsyncCallback async_callback, ResumeCallback suspend_callback) override; + void run(AsyncCallback async_callback, SuspendCallback suspend_callback) override; }; std::atomic_bool is_in_progress = false; diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference index 0729dd7b881..a6d43856aec 100644 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference @@ -6,7 +6,7 @@ {"query":"SELECT 1 AS `1` FROM `system`.`one`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} {"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} {"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"} -{"initial query spans with proper parent":"1"} +{"initial query spans with proper parent":"2"} {"unique non-empty tracestate values":"1"} ===native=== {"query":"select * from url('http:\/\/127.0.0.2:8123\/?query=select%201%20format%20Null', CSV, 'a int')","status":"QueryFinish","tracestate":"another custom state","sorted_by_start_time":1} From 2541ad69d5403985c3ef916a7995087048fabed1 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 May 2023 18:23:39 +0000 Subject: [PATCH 0538/2223] Fix bad conflicts resolving --- src/Client/ConnectionEstablisher.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index a2c51cc0382..897fb5fde73 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -4,6 +4,8 @@ namespace ProfileEvents { + extern const Event DistributedConnectionTries; + extern const Event DistributedConnectionUsable; extern const Event DistributedConnectionMissingTable; extern const Event DistributedConnectionStaleReplica; } @@ -35,6 +37,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: SCOPE_EXIT(is_finished = true); try { + ProfileEvents::increment(ProfileEvents::DistributedConnectionTries); result.entry = pool->get(*timeouts, settings, /* force_connected = */ false); AsyncCallbackSetter async_setter(&*result.entry, std::move(async_callback)); @@ -45,6 +48,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) { result.entry->forceConnected(*timeouts); + ProfileEvents::increment(ProfileEvents::DistributedConnectionUsable); result.is_usable = true; result.is_up_to_date = true; return; @@ -65,6 +69,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: return; } + ProfileEvents::increment(ProfileEvents::DistributedConnectionUsable); result.is_usable = true; UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0; From d5cfcdfae10c3706ad2e33d0e3a1f52e6c59ece0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 19:09:11 +0000 Subject: [PATCH 0539/2223] String terminator: \n --> \0 --- src/Functions/space.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index afdc01b717d..dd30e548f30 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -70,7 +70,7 @@ public: memset(res_chars.begin() + pos, space, times); pos += times; - *(res_chars.begin() + pos) = '\n'; + *(res_chars.begin() + pos) = '\0'; pos += 1; res_offsets[i] = pos; @@ -107,7 +107,7 @@ public: memset(res_chars.begin() + pos, space, times); pos += times; - *(res_chars.begin() + pos) = '\n'; + *(res_chars.begin() + pos) = '\0'; pos += 1; res_offsets[i] = pos; From d76498dca06962873d9fa654d9cda55937135a82 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 19:19:08 +0000 Subject: [PATCH 0540/2223] reserve() --> resize() --- src/Functions/space.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index dd30e548f30..70e4d3e6794 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -88,7 +88,7 @@ public: return false; res_offsets.resize(col_times->size()); - res_chars.reserve(col_times->size() * 10); /// heuristic + res_chars.resize(col_times->size() * 10); /// heuristic const PaddedPODArray & times_data = col_times->getData(); @@ -102,7 +102,7 @@ public: times = 0; if (res_chars.size() + times + 1 >= res_chars.capacity()) - res_chars.reserve(2 * res_chars.capacity()); + res_chars.resize(2 * res_chars.capacity()); memset(res_chars.begin() + pos, space, times); pos += times; From bf19765c9bd69d78bf8d6113563e0a1f48339e54 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 May 2023 19:34:19 +0000 Subject: [PATCH 0541/2223] Fix possible use-of-uninitialized-value --- src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 6f73ede5d4d..1c2efe3a41d 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -44,7 +44,7 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat( format_settings_, std::make_unique(*buf_, ignore_spaces_, format_settings_), format_settings_.custom.try_detect_header) - , buf(std::move(buf_)) + , buf(std::move(buf_)), ignore_spaces(ignore_spaces_) { /// In case of CustomSeparatedWithNames(AndTypes) formats and enabled setting input_format_with_names_use_header we don't know /// the exact number of columns in data (because it can contain unknown columns). So, if field_delimiter and row_after_delimiter are From cee6c3914fef9913b0b249fcec359a692f413a32 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 22 May 2023 21:36:55 +0200 Subject: [PATCH 0542/2223] Fix build --- src/IO/WriteBufferFromPocoSocket.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index df34e8003cb..c316ff17e2a 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -110,7 +110,7 @@ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() #ifndef NDEBUG if (!finalized) { - LOG_ERROR(log, "WriteBufferFromPocoSocket is not finalized in destructor. It's a bug"); + LOG_ERROR(&Poco::Logger::get("WriteBufferFromPocoSocket"), "WriteBufferFromPocoSocket is not finalized in destructor. It's a bug"); std::terminate(); } #else From 646eeb63a4cc720b05ff9de48364be32a6936d94 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 22 May 2023 19:46:05 +0000 Subject: [PATCH 0543/2223] Fix build --- src/Functions/FunctionGenerateRandomStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 16dac4f5112..f85b2596530 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -424,7 +424,7 @@ String FunctionGenerateRandomStructure::generateRandomStructure(size_t seed, con REGISTER_FUNCTION(GenerateRandomStructure) { - factory.registerFunction( + factory.registerFunction(FunctionDocumentation { .description=R"( Generates a random table structure. From 1f0250f73080e9fbab1e79cce0271a3cbf30bf29 Mon Sep 17 00:00:00 2001 From: Timur Solodovnikov Date: Mon, 22 May 2023 13:00:38 -0700 Subject: [PATCH 0544/2223] Updating url table functions docs --- docs/en/sql-reference/table-functions/url.md | 5 +++-- docs/ru/sql-reference/table-functions/url.md | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 014dc3ae853..cbb4e0ac91e 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -13,7 +13,7 @@ sidebar_label: url **Syntax** ``` sql -url(URL [,format] [,structure]) +url(URL [,format] [,structure] [,headers]) ``` **Parameters** @@ -21,6 +21,7 @@ url(URL [,format] [,structure]) - `URL` — HTTP or HTTPS server address, which can accept `GET` or `POST` requests (for `SELECT` or `INSERT` queries correspondingly). Type: [String](../../sql-reference/data-types/string.md). - `format` — [Format](../../interfaces/formats.md#formats) of the data. Type: [String](../../sql-reference/data-types/string.md). - `structure` — Table structure in `'UserID UInt64, Name String'` format. Determines column names and types. Type: [String](../../sql-reference/data-types/string.md). +- `headers` - Headers in `'headers('key1'='value1', 'key2'='value2')'` format. You can set headers for HTTP call. **Returned value** @@ -31,7 +32,7 @@ A table with the specified format and structure and with data from the defined ` Getting the first 3 lines of a table that contains columns of `String` and [UInt32](../../sql-reference/data-types/int-uint.md) type from HTTP-server which answers in [CSV](../../interfaces/formats.md#csv) format. ``` sql -SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3; +SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32', headers('Accept'='text/csv; charset=utf-8')) LIMIT 3; ``` Inserting data from a `URL` into a table: diff --git a/docs/ru/sql-reference/table-functions/url.md b/docs/ru/sql-reference/table-functions/url.md index e5d9faeec00..0c0819c8a67 100644 --- a/docs/ru/sql-reference/table-functions/url.md +++ b/docs/ru/sql-reference/table-functions/url.md @@ -21,6 +21,7 @@ url(URL [,format] [,structure]) - `URL` — HTTP или HTTPS-адрес сервера, который может принимать запросы `GET` или `POST` (для запросов `SELECT` или `INSERT` соответственно). Тип: [String](../../sql-reference/data-types/string.md). - `format` — [формат](../../interfaces/formats.md#formats) данных. Тип: [String](../../sql-reference/data-types/string.md). - `structure` — структура таблицы в формате `'UserID UInt64, Name String'`. Определяет имена и типы столбцов. Тип: [String](../../sql-reference/data-types/string.md). +- `headers` - HTTP-заголовки в формате `'headers('key1'='value1', 'key2'='value2')'`. Определяет заголовки для HTTP вызова. **Возвращаемое значение** @@ -31,7 +32,7 @@ url(URL [,format] [,structure]) Получение с HTTP-сервера первых 3 строк таблицы с данными в формате [CSV](../../interfaces/formats.md#csv), содержащей столбцы типа [String](../../sql-reference/data-types/string.md) и [UInt32](../../sql-reference/data-types/int-uint.md). ``` sql -SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3; +SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32', headers('Accept'='text/csv; charset=utf-8')) LIMIT 3; ``` Вставка данных в таблицу: From 786bbee62cec5ec55750eaace43bebae683d6197 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 20:03:29 +0000 Subject: [PATCH 0545/2223] Extend tests --- .../02752_space_function.reference | 69 ++++++++++++++----- .../0_stateless/02752_space_function.sql | 47 ++++++++----- 2 files changed, 82 insertions(+), 34 deletions(-) diff --git a/tests/queries/0_stateless/02752_space_function.reference b/tests/queries/0_stateless/02752_space_function.reference index b0adccb415b..d265a843ba9 100644 --- a/tests/queries/0_stateless/02752_space_function.reference +++ b/tests/queries/0_stateless/02752_space_function.reference @@ -1,21 +1,56 @@ -const times, uint - - - - -const times, int - - - - -negative const times - - - - +const, uint + 3 + 3 + 3 + 3 +const, int + 3 + 3 + 3 + 3 +const, int, negative + 0 + 0 + 0 + 0 negative tests +null \N -non-const times, uint +const, uint, multiple + + + + + + + + + + + + + + + + +const int, multiple + + + + + + + + + + + + + + + + +non-const, uint 3 2 1 @@ -32,7 +67,7 @@ non-const times, uint 20 5 7 -non-const times, int +non-const, int 3 2 1 diff --git a/tests/queries/0_stateless/02752_space_function.sql b/tests/queries/0_stateless/02752_space_function.sql index 8f3605e6e6f..d621af13096 100644 --- a/tests/queries/0_stateless/02752_space_function.sql +++ b/tests/queries/0_stateless/02752_space_function.sql @@ -1,24 +1,26 @@ -SELECT 'const times, uint'; -SELECT space(3::UInt8); -SELECT space(3::UInt16); -SELECT space(3::UInt32); -SELECT space(3::UInt64); -SELECT 'const times, int'; -SELECT space(3::Int8); -SELECT space(3::Int16); -SELECT space(3::Int32); -SELECT space(3::Int64); +SELECT 'const, uint'; +SELECT space(3::UInt8), length(space(3::UInt8)); +SELECT space(3::UInt16), length(space(3::UInt16)); +SELECT space(3::UInt32), length(space(3::UInt32)); +SELECT space(3::UInt64), length(space(3::UInt64)); +SELECT 'const, int'; +SELECT space(3::Int8), length(space(3::Int8)); +SELECT space(3::Int16), length(space(3::Int16)); +SELECT space(3::Int32), length(space(3::Int32)); +SELECT space(3::Int64), length(space(3::Int64)); -SELECT 'negative const times'; -SELECT space(-3::Int8); -SELECT space(-3::Int16); -SELECT space(-3::Int32); -SELECT space(-3::Int64); +SELECT 'const, int, negative'; +SELECT space(-3::Int8), length(space(-3::Int8)); +SELECT space(-3::Int16), length(space(-3::Int16)); +SELECT space(-3::Int32), length(space(-3::Int32)); +SELECT space(-3::Int64), length(space(-3::Int64)); SELECT 'negative tests'; SELECT space('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(['abc']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(('abc')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT 'null'; SELECT space(NULL); DROP TABLE IF EXISTS defaults; @@ -36,12 +38,23 @@ CREATE TABLE defaults INSERT INTO defaults values (3, 12, 4, 56, 3, 12, -4, 56) (2, 10, 21, 20, 2, 10, -21, 20) (1, 4, 9, 5, 1, 4, -9, 5) (0, 5, 7, 7, 0, 5, -7, 7); -SELECT 'non-const times, uint'; +SELECT 'const, uint, multiple'; +SELECT space(30::UInt8) FROM defaults; +SELECT space(30::UInt16) FROM defaults; +SELECT space(30::UInt32) FROM defaults; +SELECT space(30::UInt64) FROM defaults; +SELECT 'const int, multiple'; +SELECT space(30::Int8) FROM defaults; +SELECT space(30::Int16) FROM defaults; +SELECT space(30::Int32) FROM defaults; +SELECT space(30::Int64) FROM defaults; + +SELECT 'non-const, uint'; SELECT space(u8), length(space(u8)) FROM defaults; SELECT space(u16), length(space(u16)) FROM defaults; SELECT space(u32), length(space(u32)) from defaults; SELECT space(u64), length(space(u64)) FROM defaults; -SELECT 'non-const times, int'; +SELECT 'non-const, int'; SELECT space(i8), length(space(i8)) FROM defaults; SELECT space(i16), length(space(i16)) FROM defaults; SELECT space(i32), length(space(i32)) from defaults; From ce6054590fa3b294783b73edd4f5d589b32fc4a7 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 22 May 2023 22:49:09 +0200 Subject: [PATCH 0546/2223] Fix bad merge --- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index f776de2f7bc..05c0c8f3961 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include From d103e9adeca4aef846a8bb0d04a024e13a4e96ec Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 22 May 2023 23:36:39 +0200 Subject: [PATCH 0547/2223] Update github.com/distribution/distribution --- programs/diagnostics/go.mod | 1 + programs/diagnostics/go.sum | 2 ++ 2 files changed, 3 insertions(+) diff --git a/programs/diagnostics/go.mod b/programs/diagnostics/go.mod index 58487fced80..34c6b0037ae 100644 --- a/programs/diagnostics/go.mod +++ b/programs/diagnostics/go.mod @@ -33,6 +33,7 @@ require ( github.com/cenkalti/backoff/v4 v4.2.0 // indirect github.com/containerd/containerd v1.6.17 // indirect github.com/davecgh/go-spew v1.1.1 // indirect + github.com/distribution/distribution v2.8.2+incompatible // indirect github.com/docker/distribution v2.8.1+incompatible // indirect github.com/docker/docker v23.0.0+incompatible // indirect github.com/docker/go-units v0.5.0 // indirect diff --git a/programs/diagnostics/go.sum b/programs/diagnostics/go.sum index 71c3cbcd2d6..a95dfb4fd2b 100644 --- a/programs/diagnostics/go.sum +++ b/programs/diagnostics/go.sum @@ -126,6 +126,8 @@ github.com/cyphar/filepath-securejoin v0.2.3/go.mod h1:aPGpWjXOXUn2NCNjFvBE6aRxG github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/distribution/distribution v2.8.2+incompatible h1:k9+4DKdOG+quPFZXT/mUsiQrGu9vYCp+dXpuPkuqhk8= +github.com/distribution/distribution v2.8.2+incompatible/go.mod h1:EgLm2NgWtdKgzF9NpMzUKgzmR7AMmb0VQi2B+ZzDRjc= github.com/docker/distribution v2.8.1+incompatible h1:Q50tZOPR6T/hjNsyc9g8/syEs6bk8XXApsHjKukMl68= github.com/docker/distribution v2.8.1+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= github.com/docker/docker v23.0.0+incompatible h1:L6c28tNyqZ4/ub9AZC9d5QUuunoHHfEH4/Ue+h/E5nE= From 3a29f275e00f3a0199b9e64ca66833e61e2545c3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 May 2023 21:50:12 +0000 Subject: [PATCH 0548/2223] Fix: do not generate suffix on new chunk if didn't reach current range end --- src/Processors/Transforms/FillingTransform.cpp | 2 +- ...02730_with_fill_by_sorting_prefix.reference | 18 +++++++++--------- .../02730_with_fill_by_sorting_prefix.sql | 16 ++++++++-------- 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 052a5a1c183..4d28c079dd0 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -786,7 +786,7 @@ void FillingTransform::transform(Chunk & chunk) }); /// generate suffix for the previous range - if (!last_range_sort_prefix.empty()) + if (!last_range_sort_prefix.empty() && new_sort_prefix) generateSuffixIfNeeded(result_columns, res_fill_columns, res_interpolate_columns, res_sort_prefix_columns, res_other_columns); transformRange( diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 3e2f939af28..cf65b270fc9 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -25,10 +25,11 @@ select * from ts order by sensor_id, timestamp with fill step 1; drop table if exists ts; create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); system stop merges ts; --- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each -insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); -insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); -select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; +-- FillingTransform: 6 rows will be processed in 3 chunks with 2 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1); +insert into ts VALUES (3, 5, 1), (3, 7, 1); +insert into ts VALUES (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; 1 10 1 1 11 0 1 12 1 @@ -41,11 +42,10 @@ select * from ts order by sensor_id, timestamp with fill step 1 settings max_blo drop table if exists ts; create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); system stop merges ts; --- FillingTransform: 6 rows will be processed in 3 chunks with 2 rows each -insert into ts VALUES (1, 10, 1), (1, 12, 1); -insert into ts VALUES (3, 5, 1), (3, 7, 1); -insert into ts VALUES (5, 1, 1), (5, 3, 1); -select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; +-- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); +insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; 1 10 1 1 11 0 1 12 1 diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index 01fc125672a..42f2b93ec73 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -14,14 +14,6 @@ insert into ts VALUES (1, 10, 1), (1, 12, 2), (3, 5, 1), (3, 7, 3), (5, 1, 1), ( -- FillingTransform: 6 rows will be processed in 1 chunks select * from ts order by sensor_id, timestamp with fill step 1; -drop table if exists ts; -create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); -system stop merges ts; --- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each -insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); -insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); -select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; - drop table if exists ts; create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); system stop merges ts; @@ -31,6 +23,14 @@ insert into ts VALUES (3, 5, 1), (3, 7, 1); insert into ts VALUES (5, 1, 1), (5, 3, 1); select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=2; +drop table if exists ts; +create table ts (sensor_id UInt64, timestamp UInt64, value Float64) ENGINE=MergeTree() ORDER BY (sensor_id, timestamp); +system stop merges ts; +-- FillingTransform: 6 rows will be processed in 2 chunks with 3 rows each +insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); +insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); +select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; + select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; From 1d51488d3d98c505cd5c19918850bb75a0449345 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 May 2023 22:14:16 +0000 Subject: [PATCH 0549/2223] Tests: w/o TO, w/o FROM --- ...2730_with_fill_by_sorting_prefix.reference | 51 +++++++++++++++++++ .../02730_with_fill_by_sorting_prefix.sql | 8 +++ 2 files changed, 59 insertions(+) diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index cf65b270fc9..7f1bcc838e7 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -84,5 +84,56 @@ select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 int 3 7 1 5 1 1 5 3 1 +-- without TO +select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999); +1 6 9999 +1 7 9999 +1 8 9999 +1 9 9999 +1 10 1 +1 11 9999 +1 12 1 +3 5 1 +3 6 9999 +3 7 1 +5 1 1 +5 3 1 +select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +0 6 9999 +0 7 9999 +0 8 9999 +0 9 9999 +1 10 1 +0 11 9999 +1 12 1 +3 5 1 +3 7 1 +5 1 1 +5 3 1 +-- -- without FROM +select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999); +1 10 1 +1 12 1 +3 5 1 +3 6 9999 +3 7 1 +3 8 9999 +3 9 9999 +5 1 1 +5 2 9999 +5 3 1 +5 4 9999 +5 5 9999 +5 6 9999 +5 7 9999 +5 8 9999 +5 9 9999 +select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +1 10 1 +1 12 1 +3 5 1 +3 7 1 +5 1 1 +5 3 1 -- checking that sorting prefix columns can't be used in INTERPOLATE SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index 42f2b93ec73..dd0f06e5c8c 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -34,5 +34,13 @@ select * from ts order by sensor_id, timestamp with fill step 1 settings max_blo select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +-- without TO +select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999); +select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; + +-- without FROM +select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999); +select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; + -- checking that sorting prefix columns can't be used in INTERPOLATE SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } From dbf08b25fb8f569a33dc3a8b05862af9e61eb72a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 23 May 2023 01:25:17 +0200 Subject: [PATCH 0550/2223] better scheduling of merge selecting task --- src/Storages/MergeTree/MergeTreeSettings.cpp | 24 +++ src/Storages/MergeTree/MergeTreeSettings.h | 4 +- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 198 +++++++++++------- src/Storages/StorageReplicatedMergeTree.h | 2 + .../test.py | 3 +- .../test_merge_tree_empty_parts/test.py | 2 +- 8 files changed, 157 insertions(+), 81 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 479e50fdebb..6df841059b9 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -175,5 +175,29 @@ void MergeTreeSettings::sanityCheck(size_t background_pool_tasks) const min_bytes_to_rebalance_partition_over_jbod, max_bytes_to_merge_at_max_space_in_pool / 1024); } + + if (max_cleanup_delay_period < cleanup_delay_period) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "The value of max_cleanup_delay_period setting ({}) must be greater than the value of cleanup_delay_period setting ({})", + max_cleanup_delay_period, cleanup_delay_period); + } + + if (max_merge_selecting_sleep_ms < merge_selecting_sleep_ms) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "The value of max_merge_selecting_sleep_ms setting ({}) must be greater than the value of merge_selecting_sleep_ms setting ({})", + max_merge_selecting_sleep_ms, merge_selecting_sleep_ms); + } + + if (merge_selecting_sleep_slowdown_factor < 1.f) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "The value of merge_selecting_sleep_slowdown_factor setting ({}) cannot be less than 1.0", + merge_selecting_sleep_slowdown_factor); + } } } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 78d703e795c..56860342038 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -57,7 +57,9 @@ struct Settings; M(Bool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ M(UInt64, non_replicated_deduplication_window, 0, "How many last blocks of hashes should be kept on disk (0 - disabled).", 0) \ M(UInt64, max_parts_to_merge_at_once, 100, "Max amount of parts which can be merged at once (0 - disabled). Doesn't affect OPTIMIZE FINAL query.", 0) \ - M(UInt64, merge_selecting_sleep_ms, 5000, "Sleep time for merge selecting when no part selected, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ + M(UInt64, merge_selecting_sleep_ms, 5000, "Maximum sleep time for merge selecting, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ + M(UInt64, max_merge_selecting_sleep_ms, 60000, "Maximum sleep time for merge selecting, a lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ + M(Float, merge_selecting_sleep_slowdown_factor, 1.2f, "The sleep time for merge selecting task is multiplied by this factor when there's nothing to merge and divided when a merge was assigned", 0) \ M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ M(UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30, "Remove old broken detached parts in the background if they remained intouched for a specified by this setting period of time.", 0) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 35a860ebb42..bcc4dc749fb 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -108,7 +108,7 @@ void ReplicatedMergeTreeCleanupThread::wakeupEarlierIfNeeded() return; /// Do not re-check all parts too often (avoid constantly calling getNumberOfOutdatedPartsWithExpiredRemovalTime()) - if (!wakeup_check_timer.compareAndRestart(storage_settings->cleanup_delay_period / 4)) + if (!wakeup_check_timer.compareAndRestart(storage_settings->cleanup_delay_period / 4.0)) return; UInt64 prev_run_timestamp_ms = prev_cleanup_timestamp_ms.load(std::memory_order_relaxed); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2c19d3ba122..cb8b78b4e0a 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1298,8 +1298,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign /// which is equal or more fresh than commands themselves. In extremely rare case it can happen that we will have alter /// in between we took snapshot above and selected commands. That is why we take new snapshot here. auto task = std::make_shared(*this, getInMemoryMetadataPtr(), mutate_entry, shared_lock, common_assignee_trigger); - assignee.scheduleMergeMutateTask(task); - return true; + return assignee.scheduleMergeMutateTask(task); } if (has_mutations) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0698ab7bf38..a6152c22148 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -324,6 +325,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// Will be activated if we will achieve leader state. merge_selecting_task->deactivate(); + merge_selecting_sleep_ms = getSettings()->merge_selecting_sleep_ms; mutations_finalizing_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); }); @@ -414,6 +416,19 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( loadDataParts(skip_sanity_checks); + if (attach) + { + /// Provide better initial value of merge_selecting_sleep_ms on server startup + auto settings = getSettings(); + size_t max_parts_in_partition = getMaxPartsCountAndSizeForPartition().first; + if (settings->parts_to_delay_insert && max_parts_in_partition < settings->parts_to_delay_insert) + { + Float64 ratio = 1.0 - static_cast(max_parts_in_partition) / settings->parts_to_delay_insert; + merge_selecting_sleep_ms = static_cast(interpolateLinear(settings->merge_selecting_sleep_ms, + settings->max_merge_selecting_sleep_ms, ratio)); + } + } + if (!current_zookeeper) { if (!attach) @@ -3237,7 +3252,15 @@ void StorageReplicatedMergeTree::mergeSelectingTask() const bool cleanup = (storage_settings_ptr->clean_deleted_rows != CleanDeletedRows::Never); CreateMergeEntryResult create_result = CreateMergeEntryResult::Other; - try + enum class AttemptStatus + { + EntryCreated, + NeedRetry, + Limited, + CannotSelect, + }; + + auto try_assign_merge = [&]() -> AttemptStatus { /// We must select parts for merge under merge_selecting_mutex because other threads /// (OPTIMIZE queries) can assign new merges. @@ -3259,108 +3282,133 @@ void StorageReplicatedMergeTree::mergeSelectingTask() "Current background tasks memory usage: {}.", formatReadableSizeWithBinarySuffix(background_memory_tracker.getSoftLimit()), formatReadableSizeWithBinarySuffix(background_memory_tracker.get())); + return AttemptStatus::Limited; } - else if (merges_and_mutations_sum >= storage_settings_ptr->max_replicated_merges_in_queue) + + if (merges_and_mutations_sum >= storage_settings_ptr->max_replicated_merges_in_queue) { LOG_TRACE(log, "Number of queued merges ({}) and part mutations ({})" " is greater than max_replicated_merges_in_queue ({}), so won't select new parts to merge or mutate.", merges_and_mutations_queued.merges, merges_and_mutations_queued.mutations, storage_settings_ptr->max_replicated_merges_in_queue); + return AttemptStatus::Limited; } - else + + UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( + storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum); + + UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); + + bool merge_with_ttl_allowed = merges_and_mutations_queued.merges_with_ttl < storage_settings_ptr->max_replicated_merges_with_ttl_in_queue && + getTotalMergesWithTTLInMergeList() < storage_settings_ptr->max_number_of_merges_with_ttl_in_pool; + + auto future_merged_part = std::make_shared(); + if (storage_settings.get()->assign_part_uuids) + future_merged_part->uuid = UUIDHelpers::generateV4(); + + bool can_assign_merge = max_source_parts_size_for_merge > 0; + PartitionIdsHint partitions_to_merge_in; + if (can_assign_merge) { - UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( - storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum); + auto lightweight_merge_pred = LocalMergePredicate(queue); + partitions_to_merge_in = merger_mutator.getPartitionsThatMayBeMerged( + max_source_parts_size_for_merge, lightweight_merge_pred, merge_with_ttl_allowed, NO_TRANSACTION_PTR); + if (partitions_to_merge_in.empty()) + can_assign_merge = false; + else + merge_pred.emplace(queue.getMergePredicate(zookeeper, partitions_to_merge_in)); + } - UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); + if (can_assign_merge && + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, *merge_pred, + merge_with_ttl_allowed, NO_TRANSACTION_PTR, nullptr, &partitions_to_merge_in) == SelectPartsDecision::SELECTED) + { + create_result = createLogEntryToMergeParts( + zookeeper, + future_merged_part->parts, + future_merged_part->name, + future_merged_part->uuid, + future_merged_part->part_format, + deduplicate, + deduplicate_by_columns, + cleanup, + nullptr, + merge_pred->getVersion(), + future_merged_part->merge_type); - bool merge_with_ttl_allowed = merges_and_mutations_queued.merges_with_ttl < storage_settings_ptr->max_replicated_merges_with_ttl_in_queue && - getTotalMergesWithTTLInMergeList() < storage_settings_ptr->max_number_of_merges_with_ttl_in_pool; - auto future_merged_part = std::make_shared(); - if (storage_settings.get()->assign_part_uuids) - future_merged_part->uuid = UUIDHelpers::generateV4(); + if (create_result == CreateMergeEntryResult::Ok) + return AttemptStatus::EntryCreated; + if (create_result == CreateMergeEntryResult::LogUpdated) + return AttemptStatus::NeedRetry; + } - bool can_assign_merge = max_source_parts_size_for_merge > 0; - PartitionIdsHint partitions_to_merge_in; - if (can_assign_merge) + /// If there are many mutations in queue, it may happen, that we cannot enqueue enough merges to merge all new parts + if (max_source_part_size_for_mutation == 0 || merges_and_mutations_queued.mutations >= storage_settings_ptr->max_replicated_mutations_in_queue) + return AttemptStatus::Limited; + + if (queue.countMutations() > 0) + { + /// We don't need the list of committing blocks to choose a part to mutate + if (!merge_pred) + merge_pred.emplace(queue.getMergePredicate(zookeeper, PartitionIdsHint{})); + + /// Choose a part to mutate. + DataPartsVector data_parts = getDataPartsVectorForInternalUsage(); + for (const auto & part : data_parts) { - auto lightweight_merge_pred = LocalMergePredicate(queue); - partitions_to_merge_in = merger_mutator.getPartitionsThatMayBeMerged( - max_source_parts_size_for_merge, lightweight_merge_pred, merge_with_ttl_allowed, NO_TRANSACTION_PTR); - if (partitions_to_merge_in.empty()) - can_assign_merge = false; - else - merge_pred.emplace(queue.getMergePredicate(zookeeper, partitions_to_merge_in)); - } + if (part->getBytesOnDisk() > max_source_part_size_for_mutation) + continue; - if (can_assign_merge && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, *merge_pred, - merge_with_ttl_allowed, NO_TRANSACTION_PTR, nullptr, &partitions_to_merge_in) == SelectPartsDecision::SELECTED) - { - create_result = createLogEntryToMergeParts( - zookeeper, - future_merged_part->parts, - future_merged_part->name, + std::optional> desired_mutation_version = merge_pred->getDesiredMutationVersion(part); + if (!desired_mutation_version) + continue; + + create_result = createLogEntryToMutatePart( + *part, future_merged_part->uuid, - future_merged_part->part_format, - deduplicate, - deduplicate_by_columns, - cleanup, - nullptr, - merge_pred->getVersion(), - future_merged_part->merge_type); - } - /// If there are many mutations in queue, it may happen, that we cannot enqueue enough merges to merge all new parts - else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0 - && merges_and_mutations_queued.mutations < storage_settings_ptr->max_replicated_mutations_in_queue) - { - /// We don't need the list of committing blocks to choose a part to mutate - if (!merge_pred) - merge_pred.emplace(queue.getMergePredicate(zookeeper, PartitionIdsHint{})); + desired_mutation_version->first, + desired_mutation_version->second, + merge_pred->getVersion()); - /// Choose a part to mutate. - DataPartsVector data_parts = getDataPartsVectorForInternalUsage(); - for (const auto & part : data_parts) - { - if (part->getBytesOnDisk() > max_source_part_size_for_mutation) - continue; - - std::optional> desired_mutation_version = merge_pred->getDesiredMutationVersion(part); - if (!desired_mutation_version) - continue; - - create_result = createLogEntryToMutatePart( - *part, - future_merged_part->uuid, - desired_mutation_version->first, - desired_mutation_version->second, - merge_pred->getVersion()); - - if (create_result == CreateMergeEntryResult::Ok || - create_result == CreateMergeEntryResult::LogUpdated) - break; - } + if (create_result == CreateMergeEntryResult::Ok) + return AttemptStatus::EntryCreated; + if (create_result == CreateMergeEntryResult::LogUpdated) + return AttemptStatus::NeedRetry; } } + + return AttemptStatus::CannotSelect; + }; + + AttemptStatus result = AttemptStatus::CannotSelect; + try + { + result = try_assign_merge(); } catch (...) { tryLogCurrentException(log, __PRETTY_FUNCTION__); } - if (!is_leader) - return; - if (create_result != CreateMergeEntryResult::Ok - && create_result != CreateMergeEntryResult::LogUpdated) - { - merge_selecting_task->scheduleAfter(storage_settings_ptr->merge_selecting_sleep_ms); - } + if (result == AttemptStatus::EntryCreated || result == AttemptStatus::NeedRetry) + merge_selecting_sleep_ms = static_cast(merge_selecting_sleep_ms / storage_settings_ptr->merge_selecting_sleep_slowdown_factor); + else if (result == AttemptStatus::CannotSelect) + merge_selecting_sleep_ms = static_cast(merge_selecting_sleep_ms * storage_settings_ptr->merge_selecting_sleep_slowdown_factor); + + if (merge_selecting_sleep_ms < storage_settings_ptr->merge_selecting_sleep_ms) + merge_selecting_sleep_ms = storage_settings_ptr->merge_selecting_sleep_ms; + if (merge_selecting_sleep_ms > storage_settings_ptr->max_merge_selecting_sleep_ms) + merge_selecting_sleep_ms = storage_settings_ptr->max_merge_selecting_sleep_ms; + + if (result == AttemptStatus::EntryCreated) + merge_selecting_task->schedule(); else { - merge_selecting_task->schedule(); + LOG_TRACE(log, "Scheduling next merge selecting task after {}ms", merge_selecting_sleep_ms); + merge_selecting_task->scheduleAfter(merge_selecting_sleep_ms); } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 01b86dd1425..5d877e4b7fa 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -456,6 +456,8 @@ private: /// It is acquired for each iteration of the selection of parts to merge or each OPTIMIZE query. std::mutex merge_selecting_mutex; + UInt64 merge_selecting_sleep_ms; + /// A task that marks finished mutations as done. BackgroundSchedulePool::TaskHolder mutations_finalizing_task; diff --git a/tests/integration/test_consistent_parts_after_clone_replica/test.py b/tests/integration/test_consistent_parts_after_clone_replica/test.py index 0c907340090..2771a874d68 100644 --- a/tests/integration/test_consistent_parts_after_clone_replica/test.py +++ b/tests/integration/test_consistent_parts_after_clone_replica/test.py @@ -13,7 +13,8 @@ def fill_nodes(nodes, shard): CREATE TABLE test_table(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) - SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0; + SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, + cleanup_delay_period_random_add=0, cleanup_thread_preferred_points_per_iteration=0; """.format( shard=shard, replica=node.name ) diff --git a/tests/integration/test_merge_tree_empty_parts/test.py b/tests/integration/test_merge_tree_empty_parts/test.py index 0f611408a67..212c0577c13 100644 --- a/tests/integration/test_merge_tree_empty_parts/test.py +++ b/tests/integration/test_merge_tree_empty_parts/test.py @@ -27,7 +27,7 @@ def test_empty_parts_alter_delete(started_cluster): "CREATE TABLE empty_parts_delete (d Date, key UInt64, value String) " "ENGINE = ReplicatedMergeTree('/clickhouse/tables/empty_parts_delete', 'r1') " "PARTITION BY toYYYYMM(d) ORDER BY key " - "SETTINGS old_parts_lifetime = 1" + "SETTINGS old_parts_lifetime = 1, cleanup_delay_period=0, cleanup_thread_preferred_points_per_iteration=0" ) node1.query("INSERT INTO empty_parts_delete VALUES (toDate('2020-10-10'), 1, 'a')") From f8905acb46e52fc599878f141369030ab8730576 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 22 May 2023 23:31:50 +0000 Subject: [PATCH 0551/2223] fix crash with multiif and constant condition and nullable arguments --- src/Interpreters/InterpreterSelectQuery.cpp | 1 - src/Interpreters/TreeOptimizer.cpp | 8 ++++---- src/Interpreters/TreeOptimizer.h | 2 +- src/Interpreters/TreeRewriter.cpp | 13 ++++++++----- .../0_stateless/02751_multiif_to_if_crash.reference | 1 + .../0_stateless/02751_multiif_to_if_crash.sql | 1 + 6 files changed, 15 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02751_multiif_to_if_crash.reference create mode 100644 tests/queries/0_stateless/02751_multiif_to_if_crash.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f7ddf0e1ede..d68f9c8e6a6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -440,7 +440,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (has_input || !joined_tables.resolveTables()) joined_tables.makeFakeTable(storage, metadata_snapshot, source_header); - if (context->getCurrentTransaction() && context->getSettingsRef().throw_on_unsupported_query_inside_transaction) { if (storage) diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index da15b2a7e5f..c38b3c79026 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -703,8 +703,11 @@ void optimizeOrLikeChain(ASTPtr & query) } -void TreeOptimizer::optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif) +void TreeOptimizer::optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif, bool multiif_to_if) { + if (multiif_to_if) + optimizeMultiIfToIf(query); + /// Optimize if with constant condition after constants was substituted instead of scalar subqueries. OptimizeIfWithConstantConditionVisitor(aliases).visit(query); @@ -791,9 +794,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, if (settings.optimize_normalize_count_variants) optimizeCountConstantAndSumOne(query, context); - if (settings.optimize_multiif_to_if) - optimizeMultiIfToIf(query); - if (settings.optimize_rewrite_sum_if_to_count_if) optimizeSumIfFunctions(query); diff --git a/src/Interpreters/TreeOptimizer.h b/src/Interpreters/TreeOptimizer.h index 07ae2fbd12d..74a0a9928c4 100644 --- a/src/Interpreters/TreeOptimizer.h +++ b/src/Interpreters/TreeOptimizer.h @@ -23,7 +23,7 @@ public: const std::vector & tables_with_columns, ContextPtr context); - static void optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif); + static void optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif, bool multiif_to_if); static void optimizeCountConstantAndSumOne(ASTPtr & query, ContextPtr context); static void optimizeGroupByFunctionKeys(ASTSelectQuery * select_query); }; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index bf684077942..0824d360c7f 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1230,11 +1230,14 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( /// Push the predicate expression down to subqueries. The optimization should be applied to both initial and secondary queries. result.rewrite_subqueries = PredicateExpressionsOptimizer(getContext(), tables_with_columns, settings).optimize(*select_query); - TreeOptimizer::optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif); + /// Only apply AST optimization for initial queries. + const bool ast_optimizations_allowed = + getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY + && !select_options.ignore_ast_optimizations; + + bool optimize_multiif_to_if = ast_optimizations_allowed && settings.optimize_multiif_to_if; + TreeOptimizer::optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif, optimize_multiif_to_if); - /// Only apply AST optimization for initial queries. - const bool ast_optimizations_allowed - = getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY && !select_options.ignore_ast_optimizations; if (ast_optimizations_allowed) TreeOptimizer::apply(query, result, tables_with_columns, getContext()); @@ -1341,7 +1344,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( if (settings.legacy_column_name_of_tuple_literal) markTupleLiteralsAsLegacy(query); - TreeOptimizer::optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif); + TreeOptimizer::optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif, false); if (allow_aggregations) { diff --git a/tests/queries/0_stateless/02751_multiif_to_if_crash.reference b/tests/queries/0_stateless/02751_multiif_to_if_crash.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02751_multiif_to_if_crash.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02751_multiif_to_if_crash.sql b/tests/queries/0_stateless/02751_multiif_to_if_crash.sql new file mode 100644 index 00000000000..05233c2c4b9 --- /dev/null +++ b/tests/queries/0_stateless/02751_multiif_to_if_crash.sql @@ -0,0 +1 @@ +SELECT sum(A) FROM (SELECT multiIf(1, 1, NULL) as A); From ab5e16a7137412ec5ea332771088be5c6f6c6ec7 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 23 May 2023 00:27:17 +0000 Subject: [PATCH 0552/2223] Changes after second review iteration --- programs/client/Client.cpp | 4 ++-- programs/local/LocalServer.cpp | 4 ++-- src/Client/ClientBase.cpp | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index b28740ab80e..5870327c3b5 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1381,9 +1381,9 @@ void Client::readArguments( allow_repeated_settings = true; else if (arg == "--allow_merge_tree_settings") allow_merge_tree_settings = true; - else if ((arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-') && arg == "--multiquery") + else if (arg == "--multiquery" && (arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-')) { - /** Transforms from '--multiquery ' into '--multiquery -q ' */ + /// Transform the abbreviated syntax '--multiquery ' into the full syntax '--multiquery -q ' ++arg_num; arg = argv[arg_num]; addMultiquery(arg, common_arguments); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 91e4e077ac7..e026f87279a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -819,9 +819,9 @@ void LocalServer::readArguments(int argc, char ** argv, Arguments & common_argum for (int arg_num = 1; arg_num < argc; ++arg_num) { std::string_view arg = argv[arg_num]; - if ((arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-') && arg == "--multiquery") + if (arg == "--multiquery" && (arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-')) { - /** Transforms from '--multiquery ' into '--multiquery -q ' */ + /// Transform the abbreviated syntax '--multiquery ' into the full syntax '--multiquery -q ' ++arg_num; arg = argv[arg_num]; addMultiquery(arg, common_arguments); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index c978115790e..faddfe4e323 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2604,7 +2604,7 @@ void ClientBase::init(int argc, char ** argv) ("query,q", po::value(), "query") ("queries-file", po::value>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)") - ("multiquery,n", "Indicates that --query can execute several SQL statements. if --query was not specified then SQL statement can be set right after --multiquery. Example --multiquery \"select 1\"") + ("multiquery,n", "If specified, multiple queries separated by semicolons can be listed after --query. For convenience, it is also possible to omit --query and pass the queries directly after --multiquery.") ("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)") ("database,d", po::value(), "database") ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") From c9aa3042b50ae1b691149ec9012c1521b01705ac Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 23 May 2023 02:28:23 +0200 Subject: [PATCH 0553/2223] fix --- .../02427_mutate_and_zero_copy_replication_zookeeper.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02427_mutate_and_zero_copy_replication_zookeeper.sql b/tests/queries/0_stateless/02427_mutate_and_zero_copy_replication_zookeeper.sql index 9b0a52b8dbd..e7e0f2f6c59 100644 --- a/tests/queries/0_stateless/02427_mutate_and_zero_copy_replication_zookeeper.sql +++ b/tests/queries/0_stateless/02427_mutate_and_zero_copy_replication_zookeeper.sql @@ -9,7 +9,7 @@ CREATE TABLE mutate_and_zero_copy_replication1 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02427_mutate_and_zero_copy_replication/alter', '1') ORDER BY tuple() -SETTINGS old_parts_lifetime=0, cleanup_delay_period=300, cleanup_delay_period_random_add=300, min_bytes_for_wide_part = 0; +SETTINGS old_parts_lifetime=0, cleanup_delay_period=300, max_cleanup_delay_period=300, cleanup_delay_period_random_add=300, min_bytes_for_wide_part = 0; CREATE TABLE mutate_and_zero_copy_replication2 ( From 07d9f33b2e1c503438a0b257b8ea321abd70f7e4 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 23 May 2023 04:01:44 +0000 Subject: [PATCH 0554/2223] Improve toFirstDayNumOfWeek infinitesimally --- src/Common/DateLUTImpl.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 93af04456b2..c141ab07d78 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -877,10 +877,11 @@ public: } else { + const auto day_of_week = toDayOfWeek(v); if constexpr (std::is_unsigned_v || std::is_same_v) - return (toDayOfWeek(v) != 7) ? DayNum(saturateMinus(v, toDayOfWeek(v))) : toDayNum(v); + return (day_of_week != 7) ? DayNum(saturateMinus(v, day_of_week)) : toDayNum(v); else - return (toDayOfWeek(v) != 7) ? ExtendedDayNum(v - toDayOfWeek(v)) : toDayNum(v); + return (day_of_week != 7) ? ExtendedDayNum(v - day_of_week) : toDayNum(v); } } From e1b00516c937e566a9149ddf48d2df7441a6a0fb Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 May 2023 05:13:19 +0000 Subject: [PATCH 0555/2223] Update test reference --- .../0_stateless/02730_with_fill_by_sorting_prefix.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 7f1bcc838e7..5ec4245b478 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -110,7 +110,7 @@ select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpola 3 7 1 5 1 1 5 3 1 --- -- without FROM +-- without FROM select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999); 1 10 1 1 12 1 From d9a7227cf414caad0b45f921e1805f7eeaed3520 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 May 2023 06:49:19 +0000 Subject: [PATCH 0556/2223] Fix style check --- src/Functions/repeat.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 88eaebf4f03..93f569d3eea 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int TOO_LARGE_STRING_SIZE; } From b106757a3cb7417b32438706b290fbd1013c8fd7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 May 2023 06:51:59 +0000 Subject: [PATCH 0557/2223] Fix 02415_all_new_functions_must_be_documented --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 79a6ad1fa2d..7ab26982402 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -655,6 +655,7 @@ sleep sleepEachRow snowflakeToDateTime snowflakeToDateTime64 +space splitByChar splitByNonAlpha splitByRegexp From c0bc75eacd0624e38d2a1581e19906778ea8b676 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 May 2023 09:50:34 +0000 Subject: [PATCH 0558/2223] Try to fix test --- .../02586_generate_random_structure.reference | 26 ++++++++++--------- .../02586_generate_random_structure.sql | 10 +++---- 2 files changed, 19 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index e6e2c73ad87..d2929fb4564 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -2,16 +2,18 @@ c1 String, c2 UInt256, c3 String, c4 Decimal128(8), c5 UInt128 String Const(String) ` 90465455320735604871982424534384518837533904778028808627865442405232847164685 5& -303477100882544888461471906106.82821046 75820566154622566322847299106656624693 -c1 Int128 -c2 Decimal(76, 55) -c3 Int256 -c4 UInt32 -c5 UInt256 -c6 Float64 -c7 Map(DateTime, Int128) -c8 IPv6 -c9 Date32 --77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 --77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':127575633389498667752072479581409788016,'2052-11-08 23:07:13':-53938349319031918752329205601430421582,'2104-07-17 19:59:35':84394139582225600743319968813775553298,'2034-11-07 21:10:22':151550220355687100498925996413330909596} 328a:eccb:530f:23c3:275d:7eec:2b1b:9c29 2112-05-13 --77422512305044606600216318673365695785 -178276798036269625488.0220515892112491429818466547307797481494678716313923193 36368120999598076422295038178490120194013353260138466872851513353522727275230 2299358810 12701207658267404852859640589581641341858007661085134086312689265075880787713 -9.78063876538428e-227 {'2063-09-16 00:40:36':166979754159728572703419507823025932071} 8eff:8d3c:5a2c:fa5f:b2bf:2b0e:ff23:beb2 2143-03-03 +c1 FixedString(125) +c2 IPv4 +c3.e1 Array(Enum16(\'e1V3\' = -24827, \'e1V14\' = -24479, \'e1V8\' = -22478, \'e1V10\' = -13735, \'e1V15\' = -12641, \'e1V11\' = -10191, \'e1V0\' = -8579, \'e1V7\' = -8104, \'e1V6\' = 712, \'e1V12\' = 5683, \'e1V13\' = 13678, \'e1V9\' = 19740, \'e1V5\' = 23066, \'e1V2\' = 23292, \'e1V4\' = 23736, \'e1V1\' = 31672)) +c3.e2 Array(Map(Int8, Int32)) +c3.e3 Array(Decimal(76, 64)) +c3.e4 Array(Int32) +c3.e5 Array(Nullable(Int64)) +c3.e6 Array(Int256) +c4 FixedString(183) +c5 IPv4 +c6 UInt256 +Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:777622572,102:-1122882357,62:1647813163,-94:2094022166},{-32:1448633509},{},{},{34:1536340393,19:-2049677851,74:65643868,-46:-1990799930,97:-531041081,46:-2634833,14:1581632600,89:-771229823,-105:1238603584},{47:1458809010,109:1640682510,86:1945730198,85:1505847247,35:-35189402}] [153363749503.3642648494826450951141750747382772821825909005880434540971999557,79828591186.7378041015337066268618633118713347614941338787453473118807106292,81672688565.9633830721322966111551266731935181670389237071708068971548883315,573768486971.1812413548839655834002608768736215115033958693122764224003897029,-393925092368.4893467278351090742501814120269109477445490969167853713051140487,46027399426.0865278566391382610843315130162915324295037009704113636499519839] [755855942,1804001770,-78103159,-866181765,731736602,-79599206] [5253556148991564114,4681434929596395351,-7302160004580855709,-3686747220178471318,6288582051009949273,646864891160092871] [17035203905051045016266537043565487029724162173062647021612805252288722534904,-42105881403933504641593145676742477006499618886131028341247993701618141933523,45346626822580305846120377917274679004279343244238782744860626882886217433843,-3660165069803677989574889324494857545543653453780976182221584349306428201647,-23316760935816288837287058499520670431785615691220162210524162590241529297823,6184785563808848524970564618169964412151721224362412457508264894603779018817] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 +Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:777622572,102:-1122882357,62:1647813163,-94:2094022166},{-32:1448633509},{},{},{34:1536340393,19:-2049677851,74:65643868,-46:-1990799930,97:-531041081,46:-2634833,14:1581632600,89:-771229823,-105:1238603584},{47:1458809010,109:1640682510,86:1945730198,85:1505847247,35:-35189402}] [153363749503.3642648494826450951141750747382772821825909005880434540971999557,79828591186.7378041015337066268618633118713347614941338787453473118807106292,81672688565.9633830721322966111551266731935181670389237071708068971548883315,573768486971.1812413548839655834002608768736215115033958693122764224003897029,-393925092368.4893467278351090742501814120269109477445490969167853713051140487,46027399426.0865278566391382610843315130162915324295037009704113636499519839] [755855942,1804001770,-78103159,-866181765,731736602,-79599206] [5253556148991564114,4681434929596395351,-7302160004580855709,-3686747220178471318,6288582051009949273,646864891160092871] [17035203905051045016266537043565487029724162173062647021612805252288722534904,-42105881403933504641593145676742477006499618886131028341247993701618141933523,45346626822580305846120377917274679004279343244238782744860626882886217433843,-3660165069803677989574889324494857545543653453780976182221584349306428201647,-23316760935816288837287058499520670431785615691220162210524162590241529297823,6184785563808848524970564618169964412151721224362412457508264894603779018817] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 +Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10'] [{}] [825002272867.1157788721157301271303736024856710948164507982705676578804195475] [1865150610] [7514464811443271056] [33504961604882608369857530219353040639899064613284394558131808339620328539033] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 c1 LowCardinality(Nullable(UInt64)), c2 Date32, c3 LowCardinality(Nullable(Float64)), c4 Int256, c5 Date32 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.sql b/tests/queries/0_stateless/02586_generate_random_structure.sql index b9cec1a436a..e2e8409b35c 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.sql +++ b/tests/queries/0_stateless/02586_generate_random_structure.sql @@ -9,11 +9,11 @@ select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMEN select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN} select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN} -desc generateRandom(11); -select * from generateRandom(11) limit 1; -select * from generateRandom(11, 2) limit 1; -select * from generateRandom(11, 2, 2) limit 1; -select * from generateRandom(11, 2, 2, 2) limit 1; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +desc generateRandom(10000000); +select * from generateRandom(10000000) limit 1; +select * from generateRandom(10000000, 2) limit 1; +select * from generateRandom(10000000, 2, 2) limit 1; +select * from generateRandom(10000000, 2, 2, 2) limit 1; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} set allow_suspicious_low_cardinality_types=1; select generateRandomStructure(5, 4); From 66e111a6aa49d0cd9a16f7e6050b23f8ad4a6e68 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 23 May 2023 11:52:44 +0200 Subject: [PATCH 0559/2223] Fix tests --- src/IO/WriteBufferFromPocoSocket.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index c316ff17e2a..cf3944e019d 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -10,7 +10,6 @@ #include #include #include -#include namespace ProfileEvents @@ -107,13 +106,6 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { -#ifndef NDEBUG - if (!finalized) - { - LOG_ERROR(&Poco::Logger::get("WriteBufferFromPocoSocket"), "WriteBufferFromPocoSocket is not finalized in destructor. It's a bug"); - std::terminate(); - } -#else try { finalize(); @@ -122,7 +114,6 @@ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { tryLogCurrentException(__PRETTY_FUNCTION__); } -#endif } } From 0fb9e63f76323ae60520df37e9a947c420664de9 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 May 2023 10:00:00 +0000 Subject: [PATCH 0560/2223] Fix and update broken_tests --- tests/broken_tests.txt | 2 -- tests/queries/0_stateless/01655_plan_optimizations.reference | 1 + 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index 0b4efacba0b..fc60b820f93 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -59,7 +59,6 @@ 01615_random_one_shard_insertion 01624_soft_constraints 01651_bugs_from_15889 -01655_plan_optimizations 01656_test_query_log_factories_info 01681_bloom_filter_nullable_column 01700_system_zookeeper_path_in @@ -101,7 +100,6 @@ 02354_annoy 02366_union_decimal_conversion 02375_rocksdb_with_filters -02377_optimize_sorting_by_input_stream_properties_explain 02382_join_and_filtering_set 02402_merge_engine_with_view 02404_memory_bound_merging diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 9796d2e4f82..34ea2bc20a3 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -1,4 +1,5 @@ Too many optimizations applied to query plan +Too many optimizations applied to query plan > sipHash should be calculated after filtration FUNCTION sipHash64 Filter column: equals From 84a97ca04a0f22becab1459bb1e557fe1a6104a8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 23 May 2023 12:18:41 +0200 Subject: [PATCH 0561/2223] fix --- .../02427_mutate_and_zero_copy_replication_zookeeper.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02427_mutate_and_zero_copy_replication_zookeeper.sql b/tests/queries/0_stateless/02427_mutate_and_zero_copy_replication_zookeeper.sql index e7e0f2f6c59..e3c8583ccf4 100644 --- a/tests/queries/0_stateless/02427_mutate_and_zero_copy_replication_zookeeper.sql +++ b/tests/queries/0_stateless/02427_mutate_and_zero_copy_replication_zookeeper.sql @@ -19,7 +19,7 @@ CREATE TABLE mutate_and_zero_copy_replication2 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_02427_mutate_and_zero_copy_replication/alter', '2') ORDER BY tuple() -SETTINGS old_parts_lifetime=0, cleanup_delay_period=300, cleanup_delay_period_random_add=300; +SETTINGS old_parts_lifetime=0, cleanup_delay_period=300, max_cleanup_delay_period=300, cleanup_delay_period_random_add=300; INSERT INTO mutate_and_zero_copy_replication1 VALUES (1, '1', 1.0); From d26f9e4022ae119d03b612a263f110f637cce93d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 May 2023 10:24:06 +0000 Subject: [PATCH 0562/2223] Test with DESC order in sorting prefix --- ...2730_with_fill_by_sorting_prefix.reference | 88 +++++++++++++++++++ .../02730_with_fill_by_sorting_prefix.sql | 14 +++ 2 files changed, 102 insertions(+) diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference index 5ec4245b478..260f08027f1 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.reference @@ -55,6 +55,8 @@ select * from ts order by sensor_id, timestamp with fill step 1 settings max_blo 5 1 1 5 2 0 5 3 1 +-- FROM and TO +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); 1 6 9999 1 7 9999 @@ -84,7 +86,37 @@ select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 int 3 7 1 5 1 1 5 3 1 +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); +5 1 1 +5 3 1 +5 6 9999 +5 7 9999 +5 8 9999 +5 9 9999 +3 5 1 +3 6 9999 +3 7 1 +3 8 9999 +3 9 9999 +1 6 9999 +1 7 9999 +1 8 9999 +1 9 9999 +1 10 1 +1 12 1 +select * from ts order by sensor_id DESC, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +5 1 1 +5 3 1 +3 5 1 +0 6 9999 +3 7 1 +0 8 9999 +0 9 9999 +1 10 1 +1 12 1 -- without TO +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999); 1 6 9999 1 7 9999 @@ -110,7 +142,33 @@ select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpola 3 7 1 5 1 1 5 3 1 +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill from 6 step 1 interpolate (value as 9999); +5 1 1 +5 3 1 +3 5 1 +3 6 9999 +3 7 1 +1 6 9999 +1 7 9999 +1 8 9999 +1 9 9999 +1 10 1 +1 11 9999 +1 12 1 +select * from ts order by sensor_id DESC, timestamp with fill from 6 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +5 1 1 +5 3 1 +3 5 1 +0 6 9999 +3 7 1 +0 8 9999 +0 9 9999 +1 10 1 +0 11 9999 +1 12 1 -- without FROM +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999); 1 10 1 1 12 1 @@ -135,5 +193,35 @@ select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolat 3 7 1 5 1 1 5 3 1 +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill to 10 step 1 interpolate (value as 9999); +5 1 1 +5 2 9999 +5 3 1 +5 4 9999 +5 5 9999 +5 6 9999 +5 7 9999 +5 8 9999 +5 9 9999 +3 5 1 +3 6 9999 +3 7 1 +3 8 9999 +3 9 9999 +1 10 1 +1 12 1 +select * from ts order by sensor_id DESC, timestamp with fill to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +5 1 1 +0 2 9999 +5 3 1 +0 4 9999 +3 5 1 +0 6 9999 +3 7 1 +0 8 9999 +0 9 9999 +1 10 1 +1 12 1 -- checking that sorting prefix columns can't be used in INTERPOLATE SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } diff --git a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql index dd0f06e5c8c..e2f1ce29d5c 100644 --- a/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql +++ b/tests/queries/0_stateless/02730_with_fill_by_sorting_prefix.sql @@ -31,16 +31,30 @@ insert into ts VALUES (1, 10, 1), (1, 12, 1), (3, 5, 1); insert into ts VALUES (3, 7, 1), (5, 1, 1), (5, 3, 1); select * from ts order by sensor_id, timestamp with fill step 1 settings max_block_size=3; +-- FROM and TO +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999); +select * from ts order by sensor_id DESC, timestamp with fill from 6 to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; + -- without TO +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill from 6 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill from 6 step 1 interpolate (value as 9999); +select * from ts order by sensor_id DESC, timestamp with fill from 6 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; -- without FROM +-- ASC order in sorting prefix select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999); select * from ts order by sensor_id, timestamp with fill to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; +-- DESC order in sorting prefix +select * from ts order by sensor_id DESC, timestamp with fill to 10 step 1 interpolate (value as 9999); +select * from ts order by sensor_id DESC, timestamp with fill to 10 step 1 interpolate (value as 9999) settings use_with_fill_by_sorting_prefix=0; -- checking that sorting prefix columns can't be used in INTERPOLATE SELECT * FROM ts ORDER BY sensor_id, value, timestamp WITH FILL FROM 6 TO 10 INTERPOLATE ( value AS 1 ); -- { serverError INVALID_WITH_FILL_EXPRESSION } From 241e75197ee64ea1c7578d35f5aa3cd3cceda968 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 23 May 2023 13:31:50 +0200 Subject: [PATCH 0563/2223] Fix --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 24 +++++++++++-------- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 3 ++- .../IO/ReadIndirectBufferFromRemoteFS.cpp | 2 +- 3 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 8db93e3ed6e..12fbbbcf747 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -46,7 +47,6 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c } current_object = object; - total_bytes_read_from_current_file = 0; const auto & object_path = object.remote_path; size_t current_read_until_position = read_until_position ? read_until_position : object.bytes_size; @@ -84,7 +84,7 @@ void ReadBufferFromRemoteFSGather::appendUncachedReadInfo() .source_file_path = current_object.remote_path, .file_segment_range = { 0, current_object.bytes_size }, .cache_type = FilesystemCacheLogElement::CacheType::READ_FROM_FS_BYPASSING_CACHE, - .file_segment_size = total_bytes_read_from_current_file, + .file_segment_size = current_object.bytes_size, .read_from_cache_attempted = false, }; cache_log->add(elem); @@ -176,7 +176,7 @@ bool ReadBufferFromRemoteFSGather::moveToNextBuffer() bool ReadBufferFromRemoteFSGather::readImpl() { - swap(*current_buf); + SwapHelper swap(*this, *current_buf); bool result = false; @@ -187,7 +187,6 @@ bool ReadBufferFromRemoteFSGather::readImpl() */ if (bytes_to_ignore) { - total_bytes_read_from_current_file += bytes_to_ignore; current_buf->ignore(bytes_to_ignore); result = current_buf->hasPendingData(); file_offset_of_buffer_end += bytes_to_ignore; @@ -207,14 +206,11 @@ bool ReadBufferFromRemoteFSGather::readImpl() file_offset_of_buffer_end += current_buf->available(); } - swap(*current_buf); - /// Required for non-async reads. if (result) { - assert(available()); - nextimpl_working_buffer_offset = offset(); - total_bytes_read_from_current_file += available(); + assert(current_buf->available()); + nextimpl_working_buffer_offset = current_buf->offset(); } return result; @@ -225,8 +221,16 @@ void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t position) if (position == read_until_position) return; + reset(); read_until_position = position; +} + +void ReadBufferFromRemoteFSGather::reset() +{ + current_object = {}; + current_buf_idx = {}; current_buf.reset(); + bytes_to_ignore = 0; } off_t ReadBufferFromRemoteFSGather::seek(off_t offset, int whence) @@ -234,8 +238,8 @@ off_t ReadBufferFromRemoteFSGather::seek(off_t offset, int whence) if (whence != SEEK_SET) throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only seeking with SEEK_SET is allowed"); + reset(); file_offset_of_buffer_end = offset; - current_buf.reset(); return file_offset_of_buffer_end; } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 08e5b97d018..39b81d6f9ac 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -64,6 +64,8 @@ private: void appendUncachedReadInfo(); + void reset(); + const ReadSettings settings; const StoredObjects blobs_to_read; const ReadBufferCreator read_buffer_creator; @@ -74,7 +76,6 @@ private: size_t read_until_position = 0; size_t file_offset_of_buffer_end = 0; size_t bytes_to_ignore = 0; - size_t total_bytes_read_from_current_file = 0; StoredObject current_object; size_t current_buf_idx = 0; diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp index 04521011599..a559b47f2cc 100644 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp @@ -82,8 +82,8 @@ off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) else throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET or SEEK_CUR modes are allowed."); + impl->seek(impl->file_offset_of_buffer_end, SEEK_SET); resetWorkingBuffer(); - impl->seek(file_offset_of_buffer_end, SEEK_SET); file_offset_of_buffer_end = impl->file_offset_of_buffer_end; return impl->file_offset_of_buffer_end; From 3c1aeaaa793f080e20005bfa42a18746f969ad39 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 May 2023 11:39:40 +0000 Subject: [PATCH 0564/2223] Change default value of handshake_timeout to 10 sec, fix possible use-after-free --- docs/en/operations/settings/settings.md | 2 +- src/Client/Connection.cpp | 4 ++++ src/Core/Settings.h | 2 +- src/IO/TimeoutSetter.cpp | 21 +++++++++++++------ src/IO/TimeoutSetter.h | 6 +++++- .../test_reload_clusters_config/test.py | 2 +- .../test.py | 4 ++-- 7 files changed, 29 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b6ade78b1e1..0810b642039 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1054,7 +1054,7 @@ Default value: 10, 300, 300. Timeout in milliseconds for receiving Hello packet from replicas during handshake. -Default value: 300000. +Default value: 10000. ## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 09145bcdf1b..35423012424 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -361,6 +361,10 @@ void Connection::receiveHello(const Poco::Timespan & handshake_timeout) receiveException()->rethrow(); else { + /// Reset timeout_setter before disconnect, + /// because after disconnect socket will be invalid. + timeout_setter.reset(); + /// Close connection, to not stay in unsynchronised state. disconnect(); throwUnexpectedPacket(packet_type, "Hello or Exception"); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ba3a28af614..03cf32e5aaa 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -55,7 +55,7 @@ class IColumn; M(UInt64, max_query_size, DBMS_DEFAULT_MAX_QUERY_SIZE, "The maximum number of bytes of a query string parsed by the SQL parser. Data in the VALUES clause of INSERT queries is processed by a separate stream parser (that consumes O(1) RAM) and not affected by this restriction.", 0) \ M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \ M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \ - M(Milliseconds, handshake_timeout_ms, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000, "Timeout for receiving HELLO packet from replicas.", 0) \ + M(Milliseconds, handshake_timeout_ms, 10000, "Timeout for receiving HELLO packet from replicas.", 0) \ M(Milliseconds, connect_timeout_with_failover_ms, 1000, "Connection timeout for selecting first healthy replica.", 0) \ M(Milliseconds, connect_timeout_with_failover_secure_ms, 1000, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \ M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Timeout for receiving data from network, in seconds. If no bytes were received in this interval, exception is thrown. If you set this setting on client, the 'send_timeout' for the socket will be also set on the corresponding connection end on the server.", 0) \ diff --git a/src/IO/TimeoutSetter.cpp b/src/IO/TimeoutSetter.cpp index ed21383ccd4..b8b7a814703 100644 --- a/src/IO/TimeoutSetter.cpp +++ b/src/IO/TimeoutSetter.cpp @@ -29,14 +29,12 @@ TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_, Poco::Timespan t TimeoutSetter::~TimeoutSetter() { + if (was_reset) + return; + try { - bool connected = socket.impl()->initialized(); - if (!connected) - return; - - socket.setSendTimeout(old_send_timeout); - socket.setReceiveTimeout(old_receive_timeout); + reset(); } catch (...) { @@ -44,4 +42,15 @@ TimeoutSetter::~TimeoutSetter() } } +void TimeoutSetter::reset() +{ + bool connected = socket.impl()->initialized(); + if (!connected) + return; + + socket.setSendTimeout(old_send_timeout); + socket.setReceiveTimeout(old_receive_timeout); + was_reset = true; +} + } diff --git a/src/IO/TimeoutSetter.h b/src/IO/TimeoutSetter.h index 31c37ea07af..3479986d7fe 100644 --- a/src/IO/TimeoutSetter.h +++ b/src/IO/TimeoutSetter.h @@ -6,7 +6,7 @@ namespace DB { -/// Temporarily overrides socket send/receive timeouts and reset them back into destructor +/// Temporarily overrides socket send/receive timeouts and reset them back into destructor (or manually by calling reset method) /// If "limit_max_timeout" is true, timeouts could be only decreased (maxed by previous value). struct TimeoutSetter { @@ -19,6 +19,9 @@ struct TimeoutSetter ~TimeoutSetter(); + /// Reset timeouts back. + void reset(); + Poco::Net::StreamSocket & socket; Poco::Timespan send_timeout; @@ -26,5 +29,6 @@ struct TimeoutSetter Poco::Timespan old_send_timeout; Poco::Timespan old_receive_timeout; + bool was_reset = false; }; } diff --git a/tests/integration/test_reload_clusters_config/test.py b/tests/integration/test_reload_clusters_config/test.py index a52871890e9..73ca4a01f34 100644 --- a/tests/integration/test_reload_clusters_config/test.py +++ b/tests/integration/test_reload_clusters_config/test.py @@ -169,7 +169,7 @@ test_config3 = """ def send_repeated_query(table, count=5): for i in range(count): node.query_and_get_error( - "SELECT count() FROM {} SETTINGS receive_timeout=1".format(table) + "SELECT count() FROM {} SETTINGS receive_timeout=1, handshake_timeout_ms=1".format(table) ) diff --git a/tests/integration/test_system_clusters_actual_information/test.py b/tests/integration/test_system_clusters_actual_information/test.py index 0658d0c7576..e90a6cdeb3f 100644 --- a/tests/integration/test_system_clusters_actual_information/test.py +++ b/tests/integration/test_system_clusters_actual_information/test.py @@ -40,8 +40,8 @@ def test(started_cluster): cluster.pause_container("node_1") node.query("SYSTEM RELOAD CONFIG") - node.query_and_get_error( - "SELECT count() FROM distributed SETTINGS receive_timeout=1" + error = node.query_and_get_error( + "SELECT count() FROM distributed SETTINGS receive_timeout=1, handshake_timeout_ms=1" ) result = node.query( From b1a3b8536fdcfaeb2f0699ef7d123d8cc15e86c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 May 2023 11:55:27 +0000 Subject: [PATCH 0565/2223] Docs: Update clickhouse-local arguments Follow-up to #49870 --- docs/en/operations/utilities/clickhouse-local.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index c5aea2b98a0..d6587602990 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -188,6 +188,7 @@ Arguments: - `-N`, `--table` — table name where to put output data, `table` by default. - `--format`, `--output-format` — output format, `TSV` by default. - `-d`, `--database` — default database, `_local` by default. +- `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`. - `--stacktrace` — whether to dump debug output in case of exception. - `--echo` — print query before execution. - `--verbose` — more details on query execution. From 285e8f4ae1f8bc3dc8a8b1d7e12bc152debc0650 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 May 2023 12:16:49 +0000 Subject: [PATCH 0566/2223] Protect against DOS --- src/Functions/repeat.cpp | 8 +++----- src/Functions/space.cpp | 13 +++++++++++++ tests/queries/0_stateless/02752_space_function.sql | 1 + 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 93f569d3eea..c1b553ac6b3 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -24,18 +24,16 @@ struct RepeatImpl /// Safety threshold against DoS. static inline void checkRepeatTime(UInt64 repeat_time) { - static constexpr UInt64 max_repeat_times = 1000000; + static constexpr UInt64 max_repeat_times = 1'000'000; if (repeat_time > max_repeat_times) - throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", - std::to_string(repeat_time), std::to_string(max_repeat_times)); + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times); } static inline void checkStringSize(UInt64 size) { static constexpr UInt64 max_string_size = 1 << 30; if (size > max_string_size) - throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size ({}) in function repeat, maximum is: {}", - size, max_string_size); + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size ({}) in function repeat, maximum is: {}", size, max_string_size); } template diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 70e4d3e6794..9d825a8b294 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; + extern const int TOO_LARGE_STRING_SIZE; } namespace @@ -25,6 +26,14 @@ class FunctionSpace : public IFunction private: static constexpr auto space = ' '; + /// Safety threshold against DoS. + static inline void checkRepeatTime(UInt64 repeat_time) + { + static constexpr UInt64 max_repeat_times = 1'000'000; + if (repeat_time > max_repeat_times) + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times); + } + public: static constexpr auto name = "space"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } @@ -60,6 +69,8 @@ public: if (times < 1) times = 0; + checkRepeatTime(times); + res_offsets.resize(col_times->size()); res_chars.resize(col_times->size() * (times + 1)); @@ -101,6 +112,8 @@ public: if (times < 1) times = 0; + checkRepeatTime(times); + if (res_chars.size() + times + 1 >= res_chars.capacity()) res_chars.resize(2 * res_chars.capacity()); diff --git a/tests/queries/0_stateless/02752_space_function.sql b/tests/queries/0_stateless/02752_space_function.sql index d621af13096..b12906927df 100644 --- a/tests/queries/0_stateless/02752_space_function.sql +++ b/tests/queries/0_stateless/02752_space_function.sql @@ -19,6 +19,7 @@ SELECT 'negative tests'; SELECT space('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(['abc']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(('abc')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(30303030303030303030303030303030::UInt64); -- { serverError TOO_LARGE_STRING_SIZE } SELECT 'null'; SELECT space(NULL); From fd62a1473cb673cf11dddb6529e14f1701d1708b Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Tue, 23 May 2023 09:35:23 -0400 Subject: [PATCH 0567/2223] Update build instructions I am testing the build on Ubuntu 22.04 and found that I needed to add some prereqs. --- docs/en/development/build.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 26ee9ce581a..5bf9244d4c2 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -22,7 +22,7 @@ The minimum recommended Ubuntu version for development is 22.04 LTS. ### Install Prerequisites {#install-prerequisites} ``` bash -sudo apt-get install git cmake ccache python3 ninja-build nasm yasm gawk +sudo apt-get install git cmake ccache python3 ninja-build nasm yasm gawk lsb-release wget software-properties-common gnupg ``` ### Install and Use the Clang compiler From f9f98ac50317e54c0081bfc245d5f53bdb06acf1 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 23 May 2023 09:58:59 -0400 Subject: [PATCH 0568/2223] note option --- docs/en/development/build.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 5bf9244d4c2..c35e077a166 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -46,6 +46,11 @@ As of April 2023, any version of Clang >= 15 will work. GCC as a compiler is not supported To build with a specific Clang version: +:::tip +This is optional, if you are following along and just now installed Clang then check +to see what version you have installed before setting this environment variable. +::: + ``` bash export CC=clang-16 export CXX=clang++-16 From 4689412ab3dd1891fc956a1dd0fc48a2e538d5c6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 23 May 2023 16:14:24 +0200 Subject: [PATCH 0569/2223] Change fields destruction order in AsyncTaskExecutor --- src/Common/AsyncTaskExecutor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncTaskExecutor.h b/src/Common/AsyncTaskExecutor.h index 55dc2913c13..10a9556a88b 100644 --- a/src/Common/AsyncTaskExecutor.h +++ b/src/Common/AsyncTaskExecutor.h @@ -113,8 +113,8 @@ private: void createFiber(); void destroyFiber(); - Fiber fiber; FiberStack fiber_stack; + Fiber fiber; std::mutex fiber_lock; std::exception_ptr exception; From 8645af5809e280e89d562476657564fa74e31f7b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 May 2023 14:54:22 +0000 Subject: [PATCH 0570/2223] Hoping to get into next release --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + src/Processors/Transforms/FillingTransform.cpp | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9b9dc43e32e..8967c55b3e2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -729,7 +729,7 @@ class IColumn; M(UInt64, http_max_request_param_data_size, 10_MiB, "Limit on size of request data used as a query parameter in predefined HTTP requests.", 0) \ M(Bool, function_json_value_return_type_allow_nullable, false, "Allow function JSON_VALUE to return nullable type.", 0) \ M(Bool, function_json_value_return_type_allow_complex, false, "Allow function JSON_VALUE to return complex type, such as: struct, array, map.", 0) \ - M(Bool, use_with_fill_by_sorting_prefix, true, "Columns preceding WITH FILL columns in ORDER BY clause from sorting prefix. Gaps for rows with the different values in sorting prefix will be filled independently", 0) \ + M(Bool, use_with_fill_by_sorting_prefix, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 388cad54791..c0f10b13282 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -82,6 +82,7 @@ static std::map sett { {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, + {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 4d28c079dd0..e75f83b8c80 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -14,7 +14,7 @@ namespace DB { -constexpr bool debug_logging_enabled = true; +constexpr bool debug_logging_enabled = false; template void logDebug(String key, const T & value, const char * separator = " : ") From b82ff979d014ef63f5661f83c1cf29309fe340be Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 23 May 2023 23:10:34 +0800 Subject: [PATCH 0571/2223] Fix invalid index analysis for date related keys --- src/Storages/MergeTree/KeyCondition.cpp | 35 +++++++++++++++++-- .../02764_index_analysis_fix.reference | 1 + .../0_stateless/02764_index_analysis_fix.sql | 9 +++++ 3 files changed, 42 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02764_index_analysis_fix.reference create mode 100644 tests/queries/0_stateless/02764_index_analysis_fix.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index b8ef2152a99..dea2091f115 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -943,6 +943,19 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & return {field.columns, field.row_idx, result_idx}; } +static std::set date_time_parsing_functions = { + "toDate", + "toDate32", + "toDateTime", + "toDateTime64", + "ParseDateTimeBestEffort", + "ParseDateTimeBestEffortUS", + "ParseDateTime32BestEffort", + "ParseDateTime64BestEffort", + "parseDateTime", + "parseDateTimeInJodaSyntax", +}; + /** The key functional expression constraint may be inferred from a plain column in the expression. * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, * it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())` @@ -1026,10 +1039,23 @@ bool KeyCondition::transformConstantWithValidFunctions( if (func->type != ActionsDAG::ActionType::FUNCTION) continue; + const auto & func_name = func->function_base->getName(); + auto func_base = func->function_base; + if (date_time_parsing_functions.contains(func_name)) + { + auto func_or_null = FunctionFactory::instance().get(func_name + "OrNull", context); + ColumnsWithTypeAndName arguments; + int i = 0; + for (const auto & type : func->function_base->getArgumentTypes()) + arguments.push_back({nullptr, type, fmt::format("_{}", i++)}); + + func_base = func_or_null->build(arguments); + } + if (func->children.size() == 1) { std::tie(const_value, const_type) - = applyFunctionForFieldOfUnknownType(func->function_base, const_type, const_value); + = applyFunctionForFieldOfUnknownType(func_base, const_type, const_value); } else if (func->children.size() == 2) { @@ -1040,7 +1066,7 @@ bool KeyCondition::transformConstantWithValidFunctions( auto left_arg_type = left->result_type; auto left_arg_value = (*left->column)[0]; std::tie(const_value, const_type) = applyBinaryFunctionForFieldOfUnknownType( - FunctionFactory::instance().get(func->function_base->getName(), context), + FunctionFactory::instance().get(func_base->getName(), context), left_arg_type, left_arg_value, const_type, const_value); } else @@ -1048,10 +1074,13 @@ bool KeyCondition::transformConstantWithValidFunctions( auto right_arg_type = right->result_type; auto right_arg_value = (*right->column)[0]; std::tie(const_value, const_type) = applyBinaryFunctionForFieldOfUnknownType( - FunctionFactory::instance().get(func->function_base->getName(), context), + FunctionFactory::instance().get(func_base->getName(), context), const_type, const_value, right_arg_type, right_arg_value); } } + + if (const_value.isNull()) + return false; } out_key_column_num = it->second; diff --git a/tests/queries/0_stateless/02764_index_analysis_fix.reference b/tests/queries/0_stateless/02764_index_analysis_fix.reference new file mode 100644 index 00000000000..8eeacf99fa8 --- /dev/null +++ b/tests/queries/0_stateless/02764_index_analysis_fix.reference @@ -0,0 +1 @@ +2022-10-01 10:10:10 diff --git a/tests/queries/0_stateless/02764_index_analysis_fix.sql b/tests/queries/0_stateless/02764_index_analysis_fix.sql new file mode 100644 index 00000000000..541a3444ef3 --- /dev/null +++ b/tests/queries/0_stateless/02764_index_analysis_fix.sql @@ -0,0 +1,9 @@ +drop table if exists x; + +create table x (dt String) engine MergeTree partition by toYYYYMM(toDate(dt)) order by tuple(); + +insert into x values ('2022-10-01 10:10:10'); + +select * from x where dt like '2022-10-01%'; + +drop table x; From 64ee8ebb122ea7a348fccf70ac57085febc3f125 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 23 May 2023 18:11:08 +0300 Subject: [PATCH 0572/2223] Update MutateTask.cpp --- src/Storages/MergeTree/MutateTask.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 78ea3e5b246..af643050504 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -152,7 +152,8 @@ static void splitAndModifyMutationCommands( /// But we don't know for sure what happened. auto part_metadata_version = part->getMetadataVersion(); auto table_metadata_version = table_metadata_snapshot->getMetadataVersion(); - if (table_metadata_version <= part_metadata_version) + /// StorageMergeTree does not have metadata version + if (table_metadata_version <= part_metadata_version && part->storage.supportsReplication()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} with metadata version {} contains column {} that is absent " "in table {} with metadata version {}", part->name, part_metadata_version, column.name, From db4b3d19ae1ca074aa0c62019579f6b3aec096f8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 23 May 2023 17:30:27 +0200 Subject: [PATCH 0573/2223] Clearer coordinator log (#50101) --- .../ParallelReplicasReadingCoordinator.cpp | 44 +++++++++++-------- 1 file changed, 25 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 77c280d4710..ee38cecb9c4 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -21,6 +21,30 @@ #include #include +namespace DB +{ +struct Part +{ + mutable RangesInDataPartDescription description; + // FIXME: This is needed to put this struct in set + // and modify through iterator + mutable std::set replicas; + + bool operator<(const Part & rhs) const { return description.info < rhs.description.info; } +}; +} + +template <> +struct fmt::formatter +{ + static constexpr auto parse(format_parse_context & ctx) { return ctx.begin(); } + + template + auto format(const DB::Part & part, FormatContext & ctx) + { + return format_to(ctx.out(), "{} in replicas [{}]", part.description.describe(), fmt::join(part.replicas, ", ")); + } +}; namespace DB { @@ -60,17 +84,6 @@ public: virtual void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) = 0; }; - -struct Part -{ - mutable RangesInDataPartDescription description; - // FIXME: This is needed to put this struct in set - // and modify through iterator - mutable std::set replicas; - - bool operator<(const Part & rhs) const { return description.info < rhs.description.info; } -}; - using Parts = std::set; using PartRefs = std::deque; @@ -207,14 +220,7 @@ void DefaultCoordinator::finalizeReadingState() delayed_parts.pop_front(); } - String description; - for (const auto & part : all_parts_to_read) - { - description += part.description.describe(); - description += fmt::format("Replicas: ({}) --- ", fmt::join(part.replicas, ",")); - } - - LOG_DEBUG(log, "Reading state is fully initialized: {}", description); + LOG_DEBUG(log, "Reading state is fully initialized: {}", fmt::join(all_parts_to_read, "; ")); } From 90ee85d33aff11ba75fa2b63ccc4a0d4a0882019 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Tue, 23 May 2023 23:53:27 +0800 Subject: [PATCH 0574/2223] Errata build osx zh doc (#50054) --- docs/zh/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/development/build-osx.md b/docs/zh/development/build-osx.md index 639df253dce..40b8ebe9e51 100644 --- a/docs/zh/development/build-osx.md +++ b/docs/zh/development/build-osx.md @@ -46,7 +46,7 @@ $ cd .. 为此,请创建以下文件: -/资源库/LaunchDaemons/limit.maxfiles.plist: +/Library/LaunchDaemons/limit.maxfiles.plist: ``` xml From fcd6cb1f80c135c1cda03877b17aae29df22a065 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 23 May 2023 12:53:53 -0400 Subject: [PATCH 0575/2223] add descriptions --- .../en/sql-reference/functions/geo/polygon.md | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index 771fa29667c..646f7f72e9e 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -55,7 +55,7 @@ Polygon ## polygonsWithinSpherical -Returns true or false +Returns true or false depending on whether or not one polygon lies completely inside another polygon. Reference https://www.boost.org/doc/libs/1_62_0/libs/geometry/doc/html/geometry/reference/algorithms/within/within_2.html ### Example @@ -74,7 +74,7 @@ UInt8, 0 for false, 1 for true ## polygonsDistanceSpherical -Calculates +Calculates the minimal distance between two points where one point belongs to the first polygon and the second to another polygon. Spherical means that coordinates are interpreted as coordinates on a pure and ideal sphere, which is not true for the Earth. Using this type of coordinate system speeds up execution, but of course is not precise. ### Example @@ -137,7 +137,7 @@ UInt8, 0 for false, 1 for true ## polygonsSymDifferenceSpherical -Calculates +Calculates the spatial set theoretic symmetric difference (XOR) between two polygons ### Example @@ -153,7 +153,7 @@ SELECT ## polygonsSymDifferenceCartesian -Calculates +The same as `polygonsSymDifferenceSpherical`, but the coordinates are in the Cartesian coordinate system; which is more close to the model of the real Earth. ### Example @@ -169,7 +169,7 @@ SELECT ## polygonsIntersectionSpherical -Calculates +Calculates the intersection (AND) between polygons, coordinates are spherical. ### Example @@ -206,7 +206,9 @@ UInt8, 0 for false, 1 for true ## polygonConvexHullCartesian -Calculates +Calculates a convex hull. [Reference](https://www.boost.org/doc/libs/1_61_0/libs/geometry/doc/html/geometry/reference/algorithms/convex_hull.html) + +Coordinates are in Cartesian coordinate system. ### Example @@ -222,7 +224,7 @@ SELECT ## polygonAreaSpherical -Calculates +Calculates the surface area of a polygon. ### Example @@ -236,9 +238,11 @@ SELECT ### Returned value +MultiPolygon + ## polygonsUnionSpherical -Calculates +Calculates a union (OR). ### Example @@ -343,3 +347,4 @@ SELECT ### Returned value +For more information on geometry systems, see this [presentation](https://archive.fosdem.org/2020/schedule/event/working_with_spatial_trajectories_in_boost_geometry/attachments/slides/3988/export/events/attachments/working_with_spatial_trajectories_in_boost_geometry/slides/3988/FOSDEM20_vissarion.pdf) about the Boost library, which is what ClickHouse uses. From 584c05d8b8b5a187ceaa92292641a9a10d4d3b78 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 23 May 2023 18:54:36 +0200 Subject: [PATCH 0576/2223] fix modify order by when there was no order by cols --- src/Storages/MergeTree/MergeTreeData.cpp | 74 +++++++++---------- .../00754_alter_modify_order_by.sql | 6 ++ 2 files changed, 41 insertions(+), 39 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 440c91e3082..61d6c49c7bd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -506,52 +506,48 @@ void MergeTreeData::checkProperties( auto all_columns = new_metadata.columns.getAllPhysical(); - /// Order by check AST - if (old_metadata.hasSortingKey()) + /// This is ALTER, not CREATE/ATTACH TABLE. Let us check that all new columns used in the sorting key + /// expression have just been added (so that the sorting order is guaranteed to be valid with the new key). + + Names new_primary_key_columns = new_primary_key.column_names; + Names new_sorting_key_columns = new_sorting_key.column_names; + + ASTPtr added_key_column_expr_list = std::make_shared(); + const auto & old_sorting_key_columns = old_metadata.getSortingKeyColumns(); + for (size_t new_i = 0, old_i = 0; new_i < sorting_key_size; ++new_i) { - /// This is ALTER, not CREATE/ATTACH TABLE. Let us check that all new columns used in the sorting key - /// expression have just been added (so that the sorting order is guaranteed to be valid with the new key). - - Names new_primary_key_columns = new_primary_key.column_names; - Names new_sorting_key_columns = new_sorting_key.column_names; - - ASTPtr added_key_column_expr_list = std::make_shared(); - const auto & old_sorting_key_columns = old_metadata.getSortingKeyColumns(); - for (size_t new_i = 0, old_i = 0; new_i < sorting_key_size; ++new_i) + if (old_i < old_sorting_key_columns.size()) { - if (old_i < old_sorting_key_columns.size()) - { - if (new_sorting_key_columns[new_i] != old_sorting_key_columns[old_i]) - added_key_column_expr_list->children.push_back(new_sorting_key.expression_list_ast->children[new_i]); - else - ++old_i; - } - else + if (new_sorting_key_columns[new_i] != old_sorting_key_columns[old_i]) added_key_column_expr_list->children.push_back(new_sorting_key.expression_list_ast->children[new_i]); + else + ++old_i; } + else + added_key_column_expr_list->children.push_back(new_sorting_key.expression_list_ast->children[new_i]); + } - if (!added_key_column_expr_list->children.empty()) + if (!added_key_column_expr_list->children.empty()) + { + auto syntax = TreeRewriter(getContext()).analyze(added_key_column_expr_list, all_columns); + Names used_columns = syntax->requiredSourceColumns(); + + NamesAndTypesList deleted_columns; + NamesAndTypesList added_columns; + old_metadata.getColumns().getAllPhysical().getDifference(all_columns, deleted_columns, added_columns); + + for (const String & col : used_columns) { - auto syntax = TreeRewriter(getContext()).analyze(added_key_column_expr_list, all_columns); - Names used_columns = syntax->requiredSourceColumns(); + if (!added_columns.contains(col) || deleted_columns.contains(col)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Existing column {} is used in the expression that was added to the sorting key. " + "You can add expressions that use only the newly added columns", + backQuoteIfNeed(col)); - NamesAndTypesList deleted_columns; - NamesAndTypesList added_columns; - old_metadata.getColumns().getAllPhysical().getDifference(all_columns, deleted_columns, added_columns); - - for (const String & col : used_columns) - { - if (!added_columns.contains(col) || deleted_columns.contains(col)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Existing column {} is used in the expression that was added to the sorting key. " - "You can add expressions that use only the newly added columns", - backQuoteIfNeed(col)); - - if (new_metadata.columns.getDefaults().contains(col)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Newly added column {} has a default expression, so adding expressions that use " - "it to the sorting key is forbidden", backQuoteIfNeed(col)); - } + if (new_metadata.columns.getDefaults().contains(col)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Newly added column {} has a default expression, so adding expressions that use " + "it to the sorting key is forbidden", backQuoteIfNeed(col)); } } diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by.sql b/tests/queries/0_stateless/00754_alter_modify_order_by.sql index 234bd61902b..9c7eee74c8c 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by.sql @@ -1,6 +1,12 @@ SET send_logs_level = 'fatal'; SET optimize_on_insert = 0; +DROP TABLE IF EXISTS no_order; +CREATE TABLE no_order(a UInt32, b UInt32) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE no_order MODIFY ORDER BY (a); -- { serverError 36} + +DROP TABLE no_order; + DROP TABLE IF EXISTS old_style; set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE old_style(d Date, x UInt32) ENGINE MergeTree(d, x, 8192); From 2c01104c3f6c78eaaab07be8598e9ff174536050 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 May 2023 17:30:22 +0000 Subject: [PATCH 0577/2223] Clarification comment on retries controller behavior --- src/Storages/MergeTree/ZooKeeperRetries.h | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index 7e7d0f08e2c..e55b04c27b3 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -46,6 +46,16 @@ public: retryLoop(f, []() {}); } + /// retryLoop() executes f() until it succeeds/max_retries is reached/non-retrialable error is encountered + /// + /// the callable f() can provide feedback in terms of errors in two ways: + /// 1. throw KeeperException exception: + /// in such case, retries are done only on hardware keeper errors + /// because non-hardware error codes are semantically not really errors, just a response + /// 2. set an error code in the ZooKeeperRetriesControl object (setUserError/setKeeperError) + /// The idea is that if the caller has some semantics on top of non-hardware keeper errors, + /// then it can provide feedback to retries controller via user errors + /// void retryLoop(auto && f, auto && iteration_cleanup) { while (canTry()) From 6de52e9fced5c71c24a2f40a2b13b9c3d5656b14 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 May 2023 18:36:02 +0000 Subject: [PATCH 0578/2223] Fixing some tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 15 ++- src/Interpreters/PreparedSets.cpp | 36 ++++- src/Interpreters/PreparedSets.h | 32 +---- src/Planner/Planner.cpp | 15 ++- src/Processors/QueryPlan/CreatingSetsStep.h | 3 + .../Optimizations/filterPushDown.cpp | 13 ++ .../QueryPlan/ReadFromMergeTree.cpp | 107 +++++++++------ src/Processors/QueryPlan/ReadFromMergeTree.h | 44 +++++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 127 ++++++------------ .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- .../MergeTreeIndexConditionBloomFilter.cpp | 10 ++ .../MergeTree/MergeTreeIndexInverted.cpp | 5 + src/Storages/MergeTree/RPNBuilder.cpp | 20 ++- src/Storages/SelectQueryInfo.h | 2 + 14 files changed, 259 insertions(+), 172 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index bd96ba693fe..8d305c07ce9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3088,12 +3088,17 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - auto step = std::make_unique( - query_plan.getCurrentDataStream(), - prepared_sets->detachSubqueries(context), - context); + auto subqueries = prepared_sets->detachSubqueries(context); - query_plan.addStep(std::move(step)); + if (!subqueries.empty()) + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + std::move(subqueries), + context); + + query_plan.addStep(std::move(step)); + } } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 1d7d90432b0..cd6b2a81ba0 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -226,7 +226,7 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; - // std::cerr << StackTrace().toString() << std::endl; + //std::cerr << StackTrace().toString() << std::endl; auto set_cache = context->getPreparedSetsCache(); if (set_cache) @@ -294,4 +294,38 @@ FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquer FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} +SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, true); + fill(size_limits, settings.transform_null_in, true); + return set; +} + +std::unique_ptr FutureSetFromTuple::build(const ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, false); + fill(size_limits, settings.transform_null_in, false); + return nullptr; +} + +void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) +{ + fill(size_limits, transform_null_in, false); +} + +void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) +{ + //std::cerr << StackTrace().toString() << std::endl; + + if (set) + return; + + set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); + set->insertFromBlock(block.getColumnsWithTypeAndName()); + set->finishInsert(); +} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index b4d01754ea8..ef7aba38f24 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -86,42 +86,18 @@ public: bool isFilled() const override { return true; } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr & context) override - { - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, true); - fill(size_limits, settings.transform_null_in, true); - return set; - } + SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - std::unique_ptr build(const ContextPtr & context) override - { - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, false); - fill(size_limits, settings.transform_null_in, false); - return nullptr; - } + std::unique_ptr build(const ContextPtr & context) override; - void buildForTuple(SizeLimits size_limits, bool transform_null_in) - { - fill(size_limits, transform_null_in, false); - } + void buildForTuple(SizeLimits size_limits, bool transform_null_in); private: Block block; SetPtr set; - void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) - { - if (set) - return; - - set = std::make_shared(size_limits, create_ordered_set, transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(block.getColumnsWithTypeAndName()); - set->finishInsert(); - } + void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); }; /// Information on how to build set for the [GLOBAL] IN section. diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 38d0aa29d24..b1780212e51 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1468,12 +1468,17 @@ void Planner::buildPlanForQueryNode() if (!select_query_options.only_analyze) { - auto step = std::make_unique( - query_plan.getCurrentDataStream(), - planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), - planner_context->getQueryContext()); + auto subqueries = planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()); - query_plan.addStep(std::move(step)); + if (!subqueries.empty()) + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + std::move(subqueries), + planner_context->getQueryContext()); + + query_plan.addStep(std::move(step)); + } //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); //addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 96ab26077fc..244bb27ba78 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -58,6 +58,9 @@ public: static std::vector> makePlansForSets(DelayedCreatingSetsStep && step); + ContextPtr getContext() const { return context; } + PreparedSets::SubqueriesForSets detachSubqueries() { return std::move(subqueries_for_sets); } + private: PreparedSets::SubqueriesForSets subqueries_for_sets; ContextPtr context; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 37bc894339f..63ba3d5b56c 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -268,6 +268,19 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return 2; } + if (auto * delayed = typeid_cast(child.get())) + { + /// CreatingSets does not change header. + /// We can push down filter and update header. + /// Filter - DelayedCreatingSets - Something + child = std::make_unique(filter->getOutputStream(), delayed->detachSubqueries(), delayed->getContext()); + std::swap(parent, child); + std::swap(parent_node->children, child_node->children); + std::swap(parent_node->children.front(), child_node->children.front()); + /// DelayedCreatingSets - Filter - Something + return 2; + } + if (auto * totals_having = typeid_cast(child.get())) { /// If totals step has HAVING expression, skip it for now. diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index ef013a36069..a096538a298 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1127,7 +1127,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } static ActionsDAGPtr buildFilterDAG( @@ -1171,14 +1171,14 @@ static ActionsDAGPtr buildFilterDAG( return ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); } -static void buildKeyCondition( - std::optional & key_condition, +static void buildIndexes( + std::optional & indexes, ActionsDAGPtr filter_actions_dag, const ContextPtr & context, const SelectQueryInfo & query_info, const StorageMetadataPtr & metadata_snapshot) { - key_condition.reset(); + indexes.reset(); // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); @@ -1191,16 +1191,58 @@ static void buildKeyCondition( if (query_info.syntax_analyzer_result) array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); - key_condition.emplace(filter_actions_dag, + indexes.emplace(ReadFromMergeTree::Indexes{{ + filter_actions_dag, context, primary_key_column_names, primary_key.expression, - array_join_name_set); + array_join_name_set}, {}, false}); } else { - key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); + indexes.emplace(ReadFromMergeTree::Indexes{{ + query_info, + context, + primary_key_column_names, + primary_key.expression}, {}, false}); } + + indexes->use_skip_indexes = settings.use_skip_indexes; + bool final = query_info.isFinal(); + + if (final && !settings.use_skip_indexes_if_final) + indexes->use_skip_indexes = false; + + if (!indexes->use_skip_indexes) + return; + + UsefulSkipIndexes skip_indexes; + using Key = std::pair; + std::map merged; + + for (const auto & index : metadata_snapshot->getSecondaryIndices()) + { + auto index_helper = MergeTreeIndexFactory::instance().get(index); + if (index_helper->isMergeable()) + { + auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); + if (inserted) + { + skip_indexes.merged_indices.emplace_back(); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + } + + skip_indexes.merged_indices[it->second].addIndex(index_helper); + } + else + { + auto condition = index_helper->createIndexCondition(query_info, context); + if (!condition->alwaysUnknownOrTrue()) + skip_indexes.useful_indices.emplace_back(index_helper, condition); + } + } + + indexes->skip_indexes = std::move(skip_indexes); } void ReadFromMergeTree::onAddFilterFinish() @@ -1208,7 +1250,7 @@ void ReadFromMergeTree::onAddFilterFinish() if (!filter_nodes.nodes.empty()) { auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); - buildKeyCondition(key_condition, filter_actions_dag, context, query_info, metadata_for_reading); + buildIndexes(indexes, filter_actions_dag, context, query_info, metadata_for_reading); } } @@ -1226,7 +1268,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition) + std::optional & indexes) { const auto & settings = context->getSettingsRef(); if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key) @@ -1246,7 +1288,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } return selectRangesToReadImpl( @@ -1261,7 +1303,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( @@ -1276,7 +1318,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition) + std::optional & indexes) { AnalysisResult result; const auto & settings = context->getSettingsRef(); @@ -1323,10 +1365,10 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( // } // } - if (!key_condition) - buildKeyCondition(key_condition, query_info.filter_actions_dag, context, query_info, metadata_snapshot); + if (!indexes) + buildIndexes(indexes, query_info.filter_actions_dag, context, query_info, metadata_snapshot); - if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue()) + if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue()) { return std::make_shared(MergeTreeDataSelectAnalysisResult{ .result = std::make_exception_ptr(Exception( @@ -1334,9 +1376,9 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( "Primary key ({}) is not used and setting 'force_primary_key' is set", fmt::join(primary_key_column_names, ", ")))}); } - LOG_DEBUG(log, "Key condition: {}", key_condition->toString()); + LOG_DEBUG(log, "Key condition: {}", indexes->key_condition.toString()); - if (key_condition->alwaysFalse()) + if (indexes->key_condition.alwaysFalse()) return std::make_shared(MergeTreeDataSelectAnalysisResult{.result = std::move(result)}); size_t total_marks_pk = 0; @@ -1358,7 +1400,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( query_info, metadata_snapshot->getColumns().getAllPhysical(), parts, - *key_condition, + indexes->key_condition, data, metadata_snapshot, context, @@ -1374,23 +1416,17 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( auto reader_settings = getMergeTreeReaderSettings(context, query_info); - bool use_skip_indexes = settings.use_skip_indexes; - bool final = isFinal(query_info); - - if (final && !settings.use_skip_indexes_if_final) - use_skip_indexes = false; - result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( std::move(parts), metadata_snapshot, - query_info, context, - *key_condition, + indexes->key_condition, + indexes->skip_indexes, reader_settings, log, num_streams, result.index_stats, - use_skip_indexes); + indexes->use_skip_indexes); } catch (...) { @@ -1438,7 +1474,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, /// Disable read-in-order optimization for reverse order with final. /// Otherwise, it can lead to incorrect final behavior because the implementation may rely on the reading in direct order). - if (direction != 1 && isFinal(query_info)) + if (direction != 1 && query_info.isFinal()) return false; auto order_info = std::make_shared(SortDescription{}, prefix_size, direction, limit); @@ -1561,11 +1597,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const bool ReadFromMergeTree::isQueryWithFinal() const { - const auto & select = query_info.query->as(); - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - else - return select.final(); + return query_info.isFinal(); } bool ReadFromMergeTree::isQueryWithSampling() const @@ -2009,15 +2041,6 @@ void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const } } -bool ReadFromMergeTree::isFinal(const SelectQueryInfo & query_info) -{ - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - - const auto & select = query_info.query->as(); - return select.final(); -} - bool MergeTreeDataSelectAnalysisResult::error() const { return std::holds_alternative(result); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index f13f75bfebc..6610b463726 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -24,6 +24,35 @@ struct MergeTreeDataSelectSamplingData ActionsDAGPtr filter_expression; }; +struct UsefulSkipIndexes +{ + struct DataSkippingIndexAndCondition + { + MergeTreeIndexPtr index; + MergeTreeIndexConditionPtr condition; + + DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) + : index(index_), condition(condition_) + { + } + }; + + struct MergedDataSkippingIndexAndCondition + { + std::vector indices; + MergeTreeIndexMergedConditionPtr condition; + + void addIndex(const MergeTreeIndexPtr & index) + { + indices.push_back(index); + condition->addIndex(indices.back()); + } + }; + + std::vector useful_indices; + std::vector merged_indices; +}; + struct MergeTreeDataSelectAnalysisResult; using MergeTreeDataSelectAnalysisResultPtr = std::shared_ptr; @@ -132,6 +161,13 @@ public: UInt64 getSelectedRows() const { return selected_rows; } UInt64 getSelectedMarks() const { return selected_marks; } + struct Indexes + { + KeyCondition key_condition; + UsefulSkipIndexes skip_indexes; + bool use_skip_indexes; + }; + static MergeTreeDataSelectAnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, const PrewhereInfoPtr & prewhere_info, @@ -146,7 +182,7 @@ public: const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition); + std::optional & indexes); MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; @@ -159,8 +195,6 @@ public: bool requestReadingInOrder(size_t prefix_size, int direction, size_t limit); void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value); - - static bool isFinal(const SelectQueryInfo & query_info); bool isQueryWithFinal() const; bool isQueryWithSampling() const; @@ -193,7 +227,7 @@ private: const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition); + std::optional & indexes); int getSortDirection() const { @@ -233,7 +267,7 @@ private: std::shared_ptr max_block_numbers_to_read; /// Pre-computed value, needed to trigger sets creating for PK - mutable std::optional key_condition; + mutable std::optional indexes; Poco::Logger * log; UInt64 selected_parts = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 55dfc56d8ac..f99e15c0fc1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -887,9 +887,9 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, - const SelectQueryInfo & query_info, const ContextPtr & context, const KeyCondition & key_condition, + const UsefulSkipIndexes & skip_indexes, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, @@ -900,66 +900,6 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd parts_with_ranges.resize(parts.size()); const Settings & settings = context->getSettingsRef(); - /// Let's start analyzing all useful indices - - struct IndexStat - { - std::atomic total_granules{0}; - std::atomic granules_dropped{0}; - std::atomic total_parts{0}; - std::atomic parts_dropped{0}; - }; - - struct DataSkippingIndexAndCondition - { - MergeTreeIndexPtr index; - MergeTreeIndexConditionPtr condition; - IndexStat stat; - - DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) - : index(index_), condition(condition_) - { - } - }; - - struct MergedDataSkippingIndexAndCondition - { - std::vector indices; - MergeTreeIndexMergedConditionPtr condition; - IndexStat stat; - - void addIndex(const MergeTreeIndexPtr & index) - { - indices.push_back(index); - condition->addIndex(indices.back()); - } - }; - - std::list useful_indices; - std::map, MergedDataSkippingIndexAndCondition> merged_indices; - - if (use_skip_indexes) - { - for (const auto & index : metadata_snapshot->getSecondaryIndices()) - { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) - { - auto [it, inserted] = merged_indices.try_emplace({index_helper->index.type, index_helper->getGranularity()}); - if (inserted) - it->second.condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); - - it->second.addIndex(index_helper); - } - else - { - auto condition = index_helper->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - useful_indices.emplace_back(index_helper, condition); - } - } - } - if (use_skip_indexes && settings.force_data_skipping_indices.changed) { const auto & indices = settings.force_data_skipping_indices.toString(); @@ -977,7 +917,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices); std::unordered_set useful_indices_names; - for (const auto & useful_index : useful_indices) + for (const auto & useful_index : skip_indexes.useful_indices) useful_indices_names.insert(useful_index.index->index.name); for (const auto & index_name : forced_indices) @@ -992,6 +932,17 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd } } + struct IndexStat + { + std::atomic total_granules{0}; + std::atomic granules_dropped{0}; + std::atomic total_parts{0}; + std::atomic parts_dropped{0}; + }; + + std::vector useful_indices_stat(skip_indexes.useful_indices.size()); + std::vector merged_indices_stat(skip_indexes.merged_indices.size()); + std::atomic sum_marks_pk = 0; std::atomic sum_parts_pk = 0; @@ -1018,12 +969,14 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (!ranges.ranges.empty()) sum_parts_pk.fetch_add(1, std::memory_order_relaxed); - for (auto & index_and_condition : useful_indices) + for (size_t idx = 0; idx < skip_indexes.useful_indices.size(); ++idx) { if (ranges.ranges.empty()) break; - index_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed); + auto & index_and_condition = skip_indexes.useful_indices[idx]; + auto & stat = useful_indices_stat[idx]; + stat.total_parts.fetch_add(1, std::memory_order_relaxed); size_t total_granules = 0; size_t granules_dropped = 0; @@ -1040,19 +993,21 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd uncompressed_cache.get(), log); - index_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); - index_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); + stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); if (ranges.ranges.empty()) - index_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); + stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); } - for (auto & [_, indices_and_condition] : merged_indices) + for (size_t idx = 0; idx < skip_indexes.merged_indices.size(); ++idx) { if (ranges.ranges.empty()) break; - indices_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed); + auto & indices_and_condition = skip_indexes.merged_indices[idx]; + auto & stat = merged_indices_stat[idx]; + stat.total_parts.fetch_add(1, std::memory_order_relaxed); size_t total_granules = 0; size_t granules_dropped = 0; @@ -1063,11 +1018,11 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd total_granules, granules_dropped, mark_cache.get(), uncompressed_cache.get(), log); - indices_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); - indices_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); + stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); if (ranges.ranges.empty()) - indices_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); + stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); } if (!ranges.ranges.empty()) @@ -1134,15 +1089,17 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd .num_granules_after = sum_marks_pk.load(std::memory_order_relaxed)}); } - for (const auto & index_and_condition : useful_indices) + for (size_t idx = 0; idx < skip_indexes.useful_indices.size(); ++idx) { + const auto & index_and_condition = skip_indexes.useful_indices[idx]; + const auto & stat = useful_indices_stat[idx]; const auto & index_name = index_and_condition.index->index.name; LOG_DEBUG( log, "Index {} has dropped {}/{} granules.", backQuote(index_name), - index_and_condition.stat.granules_dropped, - index_and_condition.stat.total_granules); + stat.granules_dropped, + stat.total_granules); std::string description = index_and_condition.index->index.type + " GRANULARITY " + std::to_string(index_and_condition.index->index.granularity); @@ -1151,25 +1108,27 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd .type = ReadFromMergeTree::IndexType::Skip, .name = index_name, .description = std::move(description), - .num_parts_after = index_and_condition.stat.total_parts - index_and_condition.stat.parts_dropped, - .num_granules_after = index_and_condition.stat.total_granules - index_and_condition.stat.granules_dropped}); + .num_parts_after = stat.total_parts - stat.parts_dropped, + .num_granules_after = stat.total_granules - stat.granules_dropped}); } - for (const auto & [type_with_granularity, index_and_condition] : merged_indices) + for (size_t idx = 0; idx < skip_indexes.merged_indices.size(); ++idx) { + const auto & index_and_condition = skip_indexes.merged_indices[idx]; + const auto & stat = merged_indices_stat[idx]; const auto & index_name = "Merged"; LOG_DEBUG(log, "Index {} has dropped {}/{} granules.", backQuote(index_name), - index_and_condition.stat.granules_dropped, index_and_condition.stat.total_granules); + stat.granules_dropped, stat.total_granules); - std::string description = "MERGED GRANULARITY " + std::to_string(type_with_granularity.second); + std::string description = "MERGED GRANULARITY " + std::to_string(index_and_condition.indices.at(0)->index.granularity); index_stats.emplace_back(ReadFromMergeTree::IndexStat{ .type = ReadFromMergeTree::IndexType::Skip, .name = index_name, .description = std::move(description), - .num_parts_after = index_and_condition.stat.total_parts - index_and_condition.stat.parts_dropped, - .num_granules_after = index_and_condition.stat.total_granules - index_and_condition.stat.granules_dropped}); + .num_parts_after = stat.total_parts - stat.parts_dropped, + .num_granules_after = stat.total_granules - stat.granules_dropped}); } return parts_with_ranges; @@ -1291,7 +1250,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - std::optional key_condition; + std::optional indexes; return ReadFromMergeTree::selectRangesToRead( std::move(parts), prewhere_info, @@ -1306,7 +1265,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index a337574bb64..8c8ce59bebe 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -189,9 +189,9 @@ public: static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, - const SelectQueryInfo & query_info, const ContextPtr & context, const KeyCondition & key_condition, + const UsefulSkipIndexes & skip_indexes, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 5e186c25b83..2bd9db12b93 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -310,14 +310,24 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (functionIsInOrGlobalInOperator(function_name)) { + //std::cerr << StackTrace().toString() << std::endl; + auto future_set = rhs_argument.tryGetPreparedSet(); + + //std::cerr << "==== Finding set for MergeTreeBF " << bool(future_set) << std::endl; + if (future_set && !future_set->isReady()) + { + //std::cerr << "==== not ready, building " << std::endl; future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); + } ConstSetPtr prepared_set; if (future_set) prepared_set = future_set->get(); + //std::cerr << "==== Prep set for MergeTreeBF " << bool(prepared_set) << ' ' << (prepared_set ? prepared_set->hasExplicitSetElements() : false) << std::endl; + if (prepared_set && prepared_set->hasExplicitSetElements()) { const auto prepared_info = getPreparedSetInfo(prepared_set); diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 6ffba0ad029..a64f81807ae 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -655,7 +655,12 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( if (key_tuple_mapping.empty()) return false; + //std::cerr << "==== Finding set for MergeTreeConditionInverted\n"; + auto future_set = rhs.tryGetPreparedSet(); + + //std::cerr << "==== Set for MergeTreeConditionInverted" << bool(future_set) << std::endl; + if (future_set && !future_set->isReady()) future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index e8843ff1489..cc7ec45be6a 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -298,11 +298,29 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const if (ast_node && prepared_sets) { - return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); + auto hash = ast_node->getTreeHash(); + auto key = PreparedSetKey::forSubquery(hash); + + // std::cerr << ".........Getting from AST \n" << ast_node->dumpTree() << std::endl + // << key.toString() << std::endl; + + for (const auto & [k, v] : prepared_sets->getSets()) + { + // std::cerr << "........... " << k.toString() << std::endl; + if (k.ast_hash == hash) + return v; + } + + //return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); } else if (dag_node) { + + // std::cerr << "...........Getting from DAG\n"; const auto * node_without_alias = getNodeWithoutAlias(dag_node); + // std::cerr << ".......... node_without_alias : " << node_without_alias->result_name + // << ' ' << node_without_alias->result_type->getName() + // << ' ' << (node_without_alias->column ? node_without_alias->column->getName() : "") << std::endl; return tryGetSetFromDAGNode(node_without_alias); } diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index fb895d04b8f..a8eb00adc87 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -264,5 +264,7 @@ struct SelectQueryInfo { return input_order_info ? input_order_info : (projection ? projection->input_order_info : nullptr); } + + bool isFinal() const; }; } From f3b4959e059640a9b786f421b3fe42f9a1fb4be6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 23 May 2023 19:37:35 +0200 Subject: [PATCH 0579/2223] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 7 +++++-- tests/integration/test_merge_tree_empty_parts/test.py | 2 +- .../queries/0_stateless/02448_clone_replica_lost_part.sql | 1 + 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a6152c22148..fc90ff550c7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3393,10 +3393,13 @@ void StorageReplicatedMergeTree::mergeSelectingTask() } + Float32 new_sleep_ms = merge_selecting_sleep_ms; if (result == AttemptStatus::EntryCreated || result == AttemptStatus::NeedRetry) - merge_selecting_sleep_ms = static_cast(merge_selecting_sleep_ms / storage_settings_ptr->merge_selecting_sleep_slowdown_factor); + new_sleep_ms /= storage_settings_ptr->merge_selecting_sleep_slowdown_factor; else if (result == AttemptStatus::CannotSelect) - merge_selecting_sleep_ms = static_cast(merge_selecting_sleep_ms * storage_settings_ptr->merge_selecting_sleep_slowdown_factor); + new_sleep_ms *= storage_settings_ptr->merge_selecting_sleep_slowdown_factor; + new_sleep_ms *= std::uniform_real_distribution(1.f, 1.1f)(thread_local_rng); + merge_selecting_sleep_ms = static_cast(new_sleep_ms); if (merge_selecting_sleep_ms < storage_settings_ptr->merge_selecting_sleep_ms) merge_selecting_sleep_ms = storage_settings_ptr->merge_selecting_sleep_ms; diff --git a/tests/integration/test_merge_tree_empty_parts/test.py b/tests/integration/test_merge_tree_empty_parts/test.py index 212c0577c13..c6a96f3ed1b 100644 --- a/tests/integration/test_merge_tree_empty_parts/test.py +++ b/tests/integration/test_merge_tree_empty_parts/test.py @@ -48,7 +48,7 @@ def test_empty_parts_summing(started_cluster): "CREATE TABLE empty_parts_summing (d Date, key UInt64, value Int64) " "ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/empty_parts_summing', 'r1') " "PARTITION BY toYYYYMM(d) ORDER BY key " - "SETTINGS old_parts_lifetime = 1" + "SETTINGS old_parts_lifetime = 1, cleanup_delay_period=0, cleanup_thread_preferred_points_per_iteration=0" ) node1.query("INSERT INTO empty_parts_summing VALUES (toDate('2020-10-10'), 1, 1)") diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index 44303a1c532..7ad25d75fbe 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -144,6 +144,7 @@ select sleep(2) format Null; -- increases probability of reproducing the issue -- rmt1 will mimic rmt2, but will not be able to fetch parts for a while system stop replicated sends rmt2; attach table rmt1; +system sync replica rmt1; -- rmt1 should not show the value (200) from dropped part select throwIf(n = 200) from rmt1 format Null; select 11, arraySort(groupArray(n)) from rmt2; From 483a3cc5b16418dd65c8201ea06f64bc5500fcca Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 23 May 2023 15:59:08 -0300 Subject: [PATCH 0580/2223] Update comparison-functions.md --- .../en/sql-reference/functions/comparison-functions.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/comparison-functions.md b/docs/en/sql-reference/functions/comparison-functions.md index 89e0f3a6f04..297d84eb8a5 100644 --- a/docs/en/sql-reference/functions/comparison-functions.md +++ b/docs/en/sql-reference/functions/comparison-functions.md @@ -20,7 +20,7 @@ Strings are compared byte-by-byte. Note that this may lead to unexpected results A string S1 which has another string S2 as prefix is considered longer than S2. -## equals +## equals, `=`, `==` operators **Syntax** @@ -32,7 +32,7 @@ Alias: - `a = b` (operator) - `a == b` (operator) -## notEquals +## notEquals, `!=`, `<>` operators **Syntax** @@ -44,7 +44,7 @@ Alias: - `a != b` (operator) - `a <> b` (operator) -## less +## less, `<` operator **Syntax** @@ -55,7 +55,7 @@ less(a, b) Alias: - `a < b` (operator) -## greater +## greater, `>` operator **Syntax** @@ -66,7 +66,7 @@ greater(a, b) Alias: - `a > b` (operator) -## lessOrEquals +## lessOrEquals, `<=` operator **Syntax** From 4a9b269cc422db3e86f16a36ef8bcd14e8f2d46a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 23 May 2023 16:09:54 -0300 Subject: [PATCH 0581/2223] move least/greatest to conditional-functions --- .../aggregate-functions/reference/greatest.md | 48 ----------- .../aggregate-functions/reference/least.md | 48 ----------- .../functions/conditional-functions.md | 82 +++++++++++++++++++ 3 files changed, 82 insertions(+), 96 deletions(-) delete mode 100644 docs/en/sql-reference/aggregate-functions/reference/greatest.md delete mode 100644 docs/en/sql-reference/aggregate-functions/reference/least.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/greatest.md b/docs/en/sql-reference/aggregate-functions/reference/greatest.md deleted file mode 100644 index d5efea44790..00000000000 --- a/docs/en/sql-reference/aggregate-functions/reference/greatest.md +++ /dev/null @@ -1,48 +0,0 @@ ---- -slug: /en/sql-reference/aggregate-functions/reference/greatest -title: greatest ---- - -Aggregate function that returns the greatest across a list of values. All of the list members must be of comparable types. - -Examples: - -```sql -SELECT - toTypeName(greatest(toUInt8(1), 2, toUInt8(3), 3.)), - greatest(1, 2, toUInt8(3), 3.) -``` -```response -┌─toTypeName(greatest(toUInt8(1), 2, toUInt8(3), 3.))─┬─greatest(1, 2, toUInt8(3), 3.)─┐ -│ Float64 │ 3 │ -└─────────────────────────────────────────────────────┴────────────────────────────────┘ -``` - -:::note -The type returned is a Float64 as the UInt8 must be promoted to 64 bit for the comparison. -::: - -```sql -SELECT greatest(['hello'], ['there'], ['world']) -``` -```response -┌─greatest(['hello'], ['there'], ['world'])─┐ -│ ['world'] │ -└───────────────────────────────────────────┘ -``` - -```sql -SELECT greatest(toDateTime32(now() + toIntervalDay(1)), toDateTime64(now(), 3)) -``` -```response -┌─greatest(toDateTime32(plus(now(), toIntervalDay(1))), toDateTime64(now(), 3))─┐ -│ 2023-05-12 01:16:59.000 │ -└──---──────────────────────────────────────────────────────────────────────────┘ -``` - -:::note -The type returned is a DateTime64 as the DataTime32 must be promoted to 64 bit for the comparison. -::: - -Also see [least](/docs/en/sql-reference/aggregate-functions/reference/least.md). - diff --git a/docs/en/sql-reference/aggregate-functions/reference/least.md b/docs/en/sql-reference/aggregate-functions/reference/least.md deleted file mode 100644 index ae4b1d43182..00000000000 --- a/docs/en/sql-reference/aggregate-functions/reference/least.md +++ /dev/null @@ -1,48 +0,0 @@ ---- -slug: /en/sql-reference/aggregate-functions/reference/least -title: least ---- - -Aggregate function that returns the least across a list of values. All of the list members must be of comparable types. - -Examples: - -```sql -SELECT - toTypeName(least(toUInt8(1), 2, toUInt8(3), 3.)), - least(1, 2, toUInt8(3), 3.) -``` -```response -┌─toTypeName(least(toUInt8(1), 2, toUInt8(3), 3.))─┬─least(1, 2, toUInt8(3), 3.)─┐ -│ Float64 │ 1 │ -└──────────────────────────────────────────────────┴─────────────────────────────┘ -``` - -:::note -The type returned is a Float64 as the UInt8 must be promoted to 64 bit for the comparison. -::: - -```sql -SELECT least(['hello'], ['there'], ['world']) -``` -```response -┌─least(['hello'], ['there'], ['world'])─┐ -│ ['hello'] │ -└────────────────────────────────────────┘ -``` - -```sql -SELECT least(toDateTime32(now() + toIntervalDay(1)), toDateTime64(now(), 3)) -``` -```response -┌─least(toDateTime32(plus(now(), toIntervalDay(1))), toDateTime64(now(), 3))─┐ -│ 2023-05-12 01:16:59.000 │ -└────────────────────────────────────────────────────────────────────────────┘ -``` - -:::note -The type returned is a DateTime64 as the DataTime32 must be promoted to 64 bit for the comparison. -::: - -Also see [greatest](/docs/en/sql-reference/aggregate-functions/reference/greatest.md). - diff --git a/docs/en/sql-reference/functions/conditional-functions.md b/docs/en/sql-reference/functions/conditional-functions.md index eb86a6e551a..eb4e98961f1 100644 --- a/docs/en/sql-reference/functions/conditional-functions.md +++ b/docs/en/sql-reference/functions/conditional-functions.md @@ -152,3 +152,85 @@ FROM LEFT_RIGHT │ 4 │ ᴺᵁᴸᴸ │ Both equal │ └──────┴───────┴──────────────────┘ ``` + +## greatest + +Returns the greatest across a list of values. All of the list members must be of comparable types. + +Examples: + +```sql +SELECT greatest(1, 2, toUInt8(3), 3.) result, toTypeName(result) type; +``` +```response +┌─result─┬─type────┐ +│ 3 │ Float64 │ +└────────┴─────────┘ +``` + +:::note +The type returned is a Float64 as the UInt8 must be promoted to 64 bit for the comparison. +::: + +```sql +SELECT greatest(['hello'], ['there'], ['world']) +``` +```response +┌─greatest(['hello'], ['there'], ['world'])─┐ +│ ['world'] │ +└───────────────────────────────────────────┘ +``` + +```sql +SELECT greatest(toDateTime32(now() + toIntervalDay(1)), toDateTime64(now(), 3)) +``` +```response +┌─greatest(toDateTime32(plus(now(), toIntervalDay(1))), toDateTime64(now(), 3))─┐ +│ 2023-05-12 01:16:59.000 │ +└──---──────────────────────────────────────────────────────────────────────────┘ +``` + +:::note +The type returned is a DateTime64 as the DataTime32 must be promoted to 64 bit for the comparison. +::: + +## least + +Returns the least across a list of values. All of the list members must be of comparable types. + +Examples: + +```sql +SELECT least(1, 2, toUInt8(3), 3.) result, toTypeName(result) type; +``` +```response +┌─result─┬─type────┐ +│ 1 │ Float64 │ +└────────┴─────────┘ +``` + +:::note +The type returned is a Float64 as the UInt8 must be promoted to 64 bit for the comparison. +::: + +```sql +SELECT least(['hello'], ['there'], ['world']) +``` +```response +┌─least(['hello'], ['there'], ['world'])─┐ +│ ['hello'] │ +└────────────────────────────────────────┘ +``` + +```sql +SELECT least(toDateTime32(now() + toIntervalDay(1)), toDateTime64(now(), 3)) +``` +```response +┌─least(toDateTime32(plus(now(), toIntervalDay(1))), toDateTime64(now(), 3))─┐ +│ 2023-05-12 01:16:59.000 │ +└────────────────────────────────────────────────────────────────────────────┘ +``` + +:::note +The type returned is a DateTime64 as the DataTime32 must be promoted to 64 bit for the comparison. +::: From 03bf6e0a75415943c1e4d9f263969b0537e8c15f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 23 May 2023 21:18:01 +0200 Subject: [PATCH 0582/2223] Temporary update broken_tests.txt until #49135 is merged --- tests/broken_tests.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index e61c1316e17..c5a58440a8a 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -137,3 +137,5 @@ 01600_parts_types_metrics_long 01287_max_execution_speed 02703_row_policy_for_database +02721_url_cluster +02534_s3_cluster_insert_select_schema_inference From 885dea36585e47c37403dd5c4ca94ae4d29d4b2b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 23 May 2023 22:13:14 +0200 Subject: [PATCH 0583/2223] Switch to upstream repository of vectorscan Everything that is requried already merged: - https://github.com/VectorCamp/vectorscan/pull/144 - https://github.com/VectorCamp/vectorscan/pull/149 - https://github.com/VectorCamp/vectorscan/pull/148 Signed-off-by: Azat Khuzhin --- .gitmodules | 2 +- contrib/vectorscan | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index dbca3f3f6bc..f0984fec4db 100644 --- a/.gitmodules +++ b/.gitmodules @@ -267,7 +267,7 @@ url = https://github.com/ClickHouse/nats.c [submodule "contrib/vectorscan"] path = contrib/vectorscan - url = https://github.com/ClickHouse/vectorscan.git + url = https://github.com/VectorCamp/vectorscan.git [submodule "contrib/c-ares"] path = contrib/c-ares url = https://github.com/ClickHouse/c-ares diff --git a/contrib/vectorscan b/contrib/vectorscan index 1f4d448314e..38431d11178 160000 --- a/contrib/vectorscan +++ b/contrib/vectorscan @@ -1 +1 @@ -Subproject commit 1f4d448314e581473103187765e4c949d01b4259 +Subproject commit 38431d111781843741a781a57a6381a527d900a4 From 9db7e8ed62415aac040ee5d220d6dabc749ccc7b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 May 2023 20:47:35 +0000 Subject: [PATCH 0584/2223] Fixing build. --- src/Storages/SelectQueryInfo.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 src/Storages/SelectQueryInfo.cpp diff --git a/src/Storages/SelectQueryInfo.cpp b/src/Storages/SelectQueryInfo.cpp new file mode 100644 index 00000000000..665da7fee70 --- /dev/null +++ b/src/Storages/SelectQueryInfo.cpp @@ -0,0 +1,16 @@ +#include +#include + +namespace DB +{ + +bool SelectQueryInfo::isFinal() const +{ + if (table_expression_modifiers) + return table_expression_modifiers->hasFinal(); + + const auto & select = query->as(); + return select.final(); +} + +} From fbeba9b5911dbaf69efbcdddf29c68d0a786ff89 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 May 2023 22:55:58 +0200 Subject: [PATCH 0585/2223] Documentation --- .../statements/select/order-by.md | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 62feca9ecf6..f1efd6c4718 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -544,6 +544,54 @@ Result: └─────┴──────────┴───────┘ ``` +##Filling grouped by sorting prefix + +It can be useful to fill rows which have the same values in particular columns independently, - a good example is filling missing values in time series. +Assume there is the following time series table +``` sql +CREATE TABLE timeseries +( + `sensor_id` UInt64, + `timestamp` DateTime64(3, 'UTC'), + `value` Float64 +) +ENGINE = Memory; + +SELECT * FROM timeseries; + +┌─sensor_id─┬───────────────timestamp─┬─value─┐ +│ 234 │ 2021-12-01 00:00:03.000 │ 3 │ +│ 432 │ 2021-12-01 00:00:01.000 │ 1 │ +│ 234 │ 2021-12-01 00:00:07.000 │ 7 │ +│ 432 │ 2021-12-01 00:00:05.000 │ 5 │ +└───────────┴─────────────────────────┴───────┘ +``` +And we'd like to fill missing values for each sensor independently with 1 second interval. +The way to achieve it is to use `sensor_id` column as sorting prefix for filling column `timestamp` +``` +SELECT * +FROM timeseries +ORDER BY + sensor_id, + timestamp WITH FILL +INTERPOLATE ( value AS 9999 ) + +┌─sensor_id─┬───────────────timestamp─┬─value─┐ +│ 234 │ 2021-12-01 00:00:03.000 │ 3 │ +│ 234 │ 2021-12-01 00:00:04.000 │ 9999 │ +│ 234 │ 2021-12-01 00:00:05.000 │ 9999 │ +│ 234 │ 2021-12-01 00:00:06.000 │ 9999 │ +│ 234 │ 2021-12-01 00:00:07.000 │ 7 │ +│ 432 │ 2021-12-01 00:00:01.000 │ 1 │ +│ 432 │ 2021-12-01 00:00:02.000 │ 9999 │ +│ 432 │ 2021-12-01 00:00:03.000 │ 9999 │ +│ 432 │ 2021-12-01 00:00:04.000 │ 9999 │ +│ 432 │ 2021-12-01 00:00:05.000 │ 5 │ +└───────────┴─────────────────────────┴───────┘ +``` +Here, the `value` column was interpolated with `9999` just to make filled rows more noticeable +This behavior is controlled by setting `use_with_fill_by_sorting_prefix` (enabled by default) + ## Related content - Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse) From da59d8a5b78b54eff73eea74fec2a2cc0ee5bdba Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 24 May 2023 00:27:29 +0200 Subject: [PATCH 0586/2223] Speed-up the shellcheck with parallel xargs --- utils/check-style/shellcheck-run.sh | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/utils/check-style/shellcheck-run.sh b/utils/check-style/shellcheck-run.sh index c0063d4b191..bdb0f681c31 100755 --- a/utils/check-style/shellcheck-run.sh +++ b/utils/check-style/shellcheck-run.sh @@ -1,9 +1,14 @@ #!/usr/bin/env bash ROOT_PATH=$(git rev-parse --show-toplevel) -EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing/|Parsers/New' +NPROC=$(($(nproc) + 3)) # Check sh tests with Shellcheck -(cd $ROOT_PATH/tests/queries/0_stateless/ && shellcheck --check-sourced --external-sources --severity info --exclude SC1071,SC2086,SC2016 *.sh ../1_stateful/*.sh) +( cd "$ROOT_PATH/tests/queries/0_stateless/" && \ + find "$ROOT_PATH/tests/queries/"{0_stateless,1_stateful} -name '*.sh' -print0 | \ + xargs -0 -P "$NPROC" -n 20 shellcheck --check-sourced --external-sources --severity info --exclude SC1071,SC2086,SC2016 +) # Check docker scripts with shellcheck -find "$ROOT_PATH/docker" -executable -type f -exec file -F' ' --mime-type {} \; | awk -F' ' '$2==" text/x-shellscript" {print $1}' | grep -v "entrypoint.alpine.sh" | grep -v "compare.sh"| xargs shellcheck - +find "$ROOT_PATH/docker" -executable -type f -exec file -F' ' --mime-type {} \; | \ + awk -F' ' '$2==" text/x-shellscript" {print $1}' | \ + grep -v "compare.sh" | \ + xargs -P "$NPROC" -n 20 shellcheck From 3edffa3ceddd8637650c7b3ccfefa50373e6a869 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 23 May 2023 19:09:32 -0400 Subject: [PATCH 0587/2223] add examples --- .../en/sql-reference/functions/geo/polygon.md | 64 +++++++++++++++---- 1 file changed, 52 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index 646f7f72e9e..41386bd1183 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -142,15 +142,20 @@ Calculates the spatial set theoretic symmetric difference (XOR) between two poly ### Example ``` sql -SELECT +SELECT wkt(arraySort(polygonsSymDifferenceSpherical([[(50., 50.), (50., -50.), (-50., -50.), (-50., 50.), (50., 50.)], [(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)], [(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]], [[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]))); ``` ```response +MULTIPOLYGON(((-20 -10.3067,-10 -10,-10 -20.8791,-20 -20,-20 -10.3067)),((10 20.8791,20 20,20 10.3067,10 10,10 20.8791)),((50 50,50 -50,-50 -50,-50 50,50 50),(20 10.3067,40 10,40 40,10 40,10 20.8791,-20 20,-20 -10.3067,-40 -10,-40 -40,-10 -40,-10 -20.8791,20 -20,20 10.3067))) ``` ### Input parameters +Polygons + ### Returned value +MultiPolygon + ## polygonsSymDifferenceCartesian The same as `polygonsSymDifferenceSpherical`, but the coordinates are in the Cartesian coordinate system; which is more close to the model of the real Earth. @@ -158,15 +163,20 @@ The same as `polygonsSymDifferenceSpherical`, but the coordinates are in the Car ### Example ``` sql -SELECT +SELECT wkt(polygonsSymDifferenceCartesian([[[(0, 0), (0, 3), (1, 2.9), (2, 2.6), (2.6, 2), (2.9, 1), (3, 0), (0, 0)]]], [[[(1., 1.), (1., 4.), (4., 4.), (4., 1.), (1., 1.)]]])) ``` ```response +MULTIPOLYGON(((1 2.9,1 1,2.9 1,3 0,0 0,0 3,1 2.9)),((1 2.9,1 4,4 4,4 1,2.9 1,2.6 2,2 2.6,1 2.9))) ``` ### Input parameters +Polygons + ### Returned value +MultiPolygon + ## polygonsIntersectionSpherical Calculates the intersection (AND) between polygons, coordinates are spherical. @@ -174,15 +184,20 @@ Calculates the intersection (AND) between polygons, coordinates are spherical. ### Example ``` sql -SELECT +SELECT wkt(arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a), polygonsIntersectionSpherical([[[(4.3613577, 50.8651821), (4.349556, 50.8535879), (4.3602419, 50.8435626), (4.3830299, 50.8428851), (4.3904543, 50.8564867), (4.3613148, 50.8651279)]]], [[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]]))) ``` ```response +MULTIPOLYGON(((4.3666 50.8434,4.36024 50.8436,4.34956 50.8536,4.35268 50.8567,4.36794 50.8525,4.3666 50.8434))) ``` ### Input parameters +Polygons + ### Returned value +MultiPolygon + ## polygonsWithinCartesian Returns true if the second polygon is within the first polygon. @@ -213,15 +228,20 @@ Coordinates are in Cartesian coordinate system. ### Example ``` sql -SELECT +SELECT wkt(polygonConvexHullCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.), (2., 3.)]]])) ``` ```response +POLYGON((0 0,0 5,5 5,5 0,0 0)) ``` ### Input parameters +MultiPolygon + ### Returned value +Polygon + ## polygonAreaSpherical Calculates the surface area of a polygon. @@ -229,16 +249,17 @@ Calculates the surface area of a polygon. ### Example ``` sql -SELECT +SELECT round(polygonAreaSpherical([[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]]), 14) ``` ```response +9.387704e-8 ``` ### Input parameters ### Returned value -MultiPolygon +Float ## polygonsUnionSpherical @@ -247,9 +268,10 @@ Calculates a union (OR). ### Example ``` sql -SELECT +SELECT wkt(polygonsUnionSpherical([[[(4.3613577, 50.8651821), (4.349556, 50.8535879), (4.3602419, 50.8435626), (4.3830299, 50.8428851), (4.3904543, 50.8564867), (4.3613148, 50.8651279)]]], [[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]])) ``` ```response +MULTIPOLYGON(((4.36661 50.8434,4.36623 50.8408,4.34496 50.8333,4.33807 50.8487,4.34669 50.8583,4.35268 50.8567,4.36136 50.8652,4.36131 50.8651,4.39045 50.8565,4.38303 50.8429,4.36661 50.8434))) ``` ### Input parameters @@ -262,10 +284,18 @@ Calculates ### Example +This is the polygon representing Zimbabwe: + + +``` +POLYGON((30.0107 -15.6462,30.0502 -15.6401,30.09 -15.6294,30.1301 -15.6237,30.1699 -15.6322,30.1956 -15.6491,30.2072 -15.6532,30.2231 -15.6497,30.231 -15.6447,30.2461 -15.6321,30.2549 -15.6289,30.2801 -15.6323,30.2962 -15.639,30.3281 -15.6524,30.3567 -15.6515,30.3963 -15.636,30.3977 -15.7168,30.3993 -15.812,30.4013 -15.9317,30.4026 -16.0012,30.5148 -16.0004,30.5866 -16,30.7497 -15.9989,30.8574 -15.9981,30.9019 -16.0071,30.9422 -16.0345,30.9583 -16.0511,30.9731 -16.062,30.9898 -16.0643,31.012 -16.0549,31.0237 -16.0452,31.0422 -16.0249,31.0569 -16.0176,31.0654 -16.0196,31.0733 -16.0255,31.0809 -16.0259,31.089 -16.0119,31.1141 -15.9969,31.1585 -16.0002,31.26 -16.0235,31.2789 -16.0303,31.2953 -16.0417,31.3096 -16.059,31.3284 -16.0928,31.3409 -16.1067,31.3603 -16.1169,31.3703 -16.1237,31.3746 -16.1329,31.3778 -16.1422,31.384 -16.1488,31.3877 -16.1496,31.3956 -16.1477,31.3996 -16.1473,31.4043 -16.1499,31.4041 -16.1545,31.4027 -16.1594,31.4046 -16.1623,31.4241 -16.1647,31.4457 -16.165,31.4657 -16.1677,31.4806 -16.178,31.5192 -16.1965,31.6861 -16.2072,31.7107 -16.2179,31.7382 -16.2398,31.7988 -16.3037,31.8181 -16.3196,31.8601 -16.3408,31.8719 -16.3504,31.8807 -16.368,31.8856 -16.4063,31.8944 -16.4215,31.9103 -16.4289,32.0141 -16.4449,32.2118 -16.4402,32.2905 -16.4518,32.3937 -16.4918,32.5521 -16.5534,32.6718 -16.5998,32.6831 -16.6099,32.6879 -16.6243,32.6886 -16.6473,32.6987 -16.6868,32.7252 -16.7064,32.7309 -16.7087,32.7313 -16.7088,32.7399 -16.7032,32.7538 -16.6979,32.7693 -16.6955,32.8007 -16.6973,32.862 -16.7105,32.8934 -16.7124,32.9096 -16.7081,32.9396 -16.6898,32.9562 -16.6831,32.9685 -16.6816,32.9616 -16.7103,32.9334 -16.8158,32.9162 -16.8479,32.9005 -16.8678,32.8288 -16.9351,32.8301 -16.9415,32.8868 -17.0382,32.9285 -17.1095,32.9541 -17.1672,32.9678 -17.2289,32.9691 -17.2661,32.9694 -17.2761,32.9732 -17.2979,32.9836 -17.3178,32.9924 -17.3247,33.0147 -17.3367,33.0216 -17.3456,33.0225 -17.3615,33.0163 -17.3772,33.0117 -17.384,32.9974 -17.405,32.9582 -17.4785,32.9517 -17.4862,32.943 -17.4916,32.9366 -17.4983,32.9367 -17.5094,32.9472 -17.5432,32.9517 -17.5514,32.9691 -17.5646,33.0066 -17.581,33.0204 -17.5986,33.0245 -17.6192,33.0206 -17.6385,33.0041 -17.6756,33.0002 -17.7139,33.0032 -17.7577,32.9991 -17.7943,32.9736 -17.8106,32.957 -17.818,32.9461 -17.8347,32.9397 -17.8555,32.9369 -17.875,32.9384 -17.8946,32.9503 -17.9226,32.9521 -17.9402,32.9481 -17.9533,32.9404 -17.96,32.9324 -17.9649,32.9274 -17.9729,32.929 -17.9823,32.9412 -17.9963,32.9403 -18.0048,32.9349 -18.0246,32.9371 -18.0471,32.9723 -18.1503,32.9755 -18.1833,32.9749 -18.1908,32.9659 -18.2122,32.9582 -18.2254,32.9523 -18.233,32.9505 -18.2413,32.955 -18.2563,32.9702 -18.2775,33.0169 -18.3137,33.035 -18.3329,33.0428 -18.352,33.0381 -18.3631,33.0092 -18.3839,32.9882 -18.4132,32.9854 -18.4125,32.9868 -18.4223,32.9995 -18.4367,33.003 -18.4469,32.9964 -18.4671,32.9786 -18.4801,32.9566 -18.4899,32.9371 -18.501,32.9193 -18.51,32.9003 -18.5153,32.8831 -18.5221,32.8707 -18.5358,32.8683 -18.5526,32.8717 -18.5732,32.8845 -18.609,32.9146 -18.6659,32.9223 -18.6932,32.9202 -18.7262,32.9133 -18.753,32.9025 -18.7745,32.8852 -18.7878,32.8589 -18.79,32.8179 -18.787,32.7876 -18.7913,32.6914 -18.8343,32.6899 -18.8432,32.6968 -18.8972,32.7032 -18.9119,32.7158 -18.9198,32.7051 -18.9275,32.6922 -18.9343,32.6825 -18.9427,32.6811 -18.955,32.6886 -18.9773,32.6903 -18.9882,32.6886 -19.001,32.6911 -19.0143,32.699 -19.0222,32.7103 -19.026,32.7239 -19.0266,32.786 -19.0177,32.8034 -19.0196,32.8142 -19.0238,32.82 -19.0283,32.823 -19.0352,32.8253 -19.0468,32.8302 -19.0591,32.8381 -19.0669,32.8475 -19.0739,32.8559 -19.0837,32.8623 -19.1181,32.8332 -19.242,32.8322 -19.2667,32.8287 -19.2846,32.8207 -19.3013,32.8061 -19.3234,32.7688 -19.3636,32.7665 -19.3734,32.7685 -19.4028,32.7622 -19.4434,32.7634 -19.464,32.7739 -19.4759,32.7931 -19.4767,32.8113 -19.4745,32.8254 -19.4792,32.8322 -19.5009,32.8325 -19.5193,32.8254 -19.5916,32.8257 -19.6008,32.8282 -19.6106,32.8296 -19.6237,32.8254 -19.6333,32.8195 -19.642,32.8163 -19.6521,32.8196 -19.6743,32.831 -19.6852,32.8491 -19.6891,32.8722 -19.6902,32.8947 -19.6843,32.9246 -19.6553,32.9432 -19.6493,32.961 -19.6588,32.9624 -19.6791,32.9541 -19.7178,32.9624 -19.7354,32.9791 -19.7514,33.0006 -19.7643,33.0228 -19.7731,33.0328 -19.7842,33.0296 -19.8034,33.0229 -19.8269,33.0213 -19.8681,33.002 -19.927,32.9984 -20.0009,33.0044 -20.0243,33.0073 -20.032,32.9537 -20.0302,32.9401 -20.0415,32.9343 -20.0721,32.9265 -20.0865,32.9107 -20.0911,32.8944 -20.094,32.8853 -20.103,32.8779 -20.1517,32.8729 -20.1672,32.8593 -20.1909,32.8571 -20.2006,32.8583 -20.2075,32.8651 -20.2209,32.8656 -20.2289,32.8584 -20.2595,32.853 -20.2739,32.8452 -20.2867,32.8008 -20.3386,32.7359 -20.4142,32.7044 -20.4718,32.6718 -20.5318,32.6465 -20.558,32.6037 -20.5648,32.5565 -20.5593,32.5131 -20.5646,32.4816 -20.603,32.4711 -20.6455,32.4691 -20.6868,32.4835 -20.7942,32.4972 -20.8981,32.491 -20.9363,32.4677 -20.9802,32.4171 -21.0409,32.3398 -21.1341,32.3453 -21.1428,32.3599 -21.1514,32.3689 -21.163,32.3734 -21.1636,32.3777 -21.1634,32.3806 -21.1655,32.3805 -21.1722,32.3769 -21.1785,32.373 -21.184,32.3717 -21.1879,32.4446 -21.3047,32.4458 -21.309,32.4472 -21.3137,32.4085 -21.2903,32.373 -21.3279,32.3245 -21.3782,32.2722 -21.4325,32.2197 -21.4869,32.1673 -21.5413,32.1148 -21.5956,32.0624 -21.65,32.01 -21.7045,31.9576 -21.7588,31.9052 -21.8132,31.8527 -21.8676,31.8003 -21.922,31.7478 -21.9764,31.6955 -22.0307,31.6431 -22.0852,31.5907 -22.1396,31.5382 -22.1939,31.4858 -22.2483,31.4338 -22.302,31.3687 -22.345,31.2889 -22.3973,31.2656 -22.3655,31.2556 -22.358,31.2457 -22.3575,31.2296 -22.364,31.2215 -22.3649,31.2135 -22.3619,31.1979 -22.3526,31.1907 -22.3506,31.1837 -22.3456,31.1633 -22.3226,31.1526 -22.3164,31.1377 -22.3185,31.1045 -22.3334,31.097 -22.3349,31.0876 -22.3369,31.0703 -22.3337,31.0361 -22.3196,30.9272 -22.2957,30.8671 -22.2896,30.8379 -22.2823,30.8053 -22.2945,30.6939 -22.3028,30.6743 -22.3086,30.6474 -22.3264,30.6324 -22.3307,30.6256 -22.3286,30.6103 -22.3187,30.6011 -22.3164,30.5722 -22.3166,30.5074 -22.3096,30.4885 -22.3102,30.4692 -22.3151,30.4317 -22.3312,30.4127 -22.3369,30.3721 -22.3435,30.335 -22.3447,30.3008 -22.337,30.2693 -22.3164,30.2553 -22.3047,30.2404 -22.2962,30.2217 -22.2909,30.197 -22.2891,30.1527 -22.2948,30.1351 -22.2936,30.1111 -22.2823,30.0826 -22.2629,30.0679 -22.2571,30.0381 -22.2538,30.0359 -22.2506,30.0345 -22.2461,30.0155 -22.227,30.0053 -22.2223,29.9838 -22.2177,29.974 -22.214,29.9467 -22.1983,29.9321 -22.1944,29.896 -22.1914,29.8715 -22.1793,29.8373 -22.1724,29.7792 -22.1364,29.7589 -22.1309,29.6914 -22.1341,29.6796 -22.1383,29.6614 -22.1265,29.6411 -22.1292,29.604 -22.1451,29.5702 -22.142,29.551 -22.146,29.5425 -22.1625,29.5318 -22.1724,29.5069 -22.1701,29.4569 -22.1588,29.4361 -22.1631,29.3995 -22.1822,29.378 -22.1929,29.3633 -22.1923,29.3569 -22.1909,29.3501 -22.1867,29.2736 -22.1251,29.2673 -22.1158,29.2596 -22.0961,29.2541 -22.0871,29.2444 -22.0757,29.2393 -22.0726,29.1449 -22.0753,29.108 -22.0692,29.0708 -22.051,29.0405 -22.0209,29.0216 -21.9828,29.0138 -21.9404,29.0179 -21.8981,29.0289 -21.8766,29.0454 -21.8526,29.0576 -21.8292,29.0553 -21.81,29.0387 -21.7979,28.9987 -21.786,28.9808 -21.7748,28.9519 -21.7683,28.891 -21.7649,28.8609 -21.7574,28.7142 -21.6935,28.6684 -21.68,28.6297 -21.6513,28.6157 -21.6471,28.5859 -21.6444,28.554 -21.6366,28.5429 -21.6383,28.5325 -21.6431,28.4973 -21.6515,28.4814 -21.6574,28.4646 -21.6603,28.4431 -21.6558,28.3618 -21.6163,28.3219 -21.6035,28.2849 -21.5969,28.1657 -21.5952,28.0908 -21.5813,28.0329 -21.5779,28.0166 -21.5729,28.0026 -21.5642,27.9904 -21.5519,27.9847 -21.5429,27.9757 -21.5226,27.9706 -21.5144,27.9637 -21.5105,27.9581 -21.5115,27.9532 -21.5105,27.9493 -21.5008,27.9544 -21.4878,27.9504 -21.482,27.9433 -21.4799,27.9399 -21.478,27.9419 -21.4685,27.9496 -21.4565,27.953 -21.4487,27.9502 -21.4383,27.9205 -21.3812,27.9042 -21.3647,27.8978 -21.3554,27.8962 -21.3479,27.8967 -21.3324,27.8944 -21.3243,27.885 -21.3102,27.8491 -21.2697,27.8236 -21.2317,27.7938 -21.1974,27.7244 -21.1497,27.7092 -21.1345,27.6748 -21.0901,27.6666 -21.0712,27.6668 -21.0538,27.679 -21.0007,27.6804 -20.9796,27.6727 -20.9235,27.6726 -20.9137,27.6751 -20.8913,27.6748 -20.8799,27.676 -20.8667,27.6818 -20.8576,27.689 -20.849,27.6944 -20.8377,27.7096 -20.7567,27.7073 -20.7167,27.6825 -20.6373,27.6904 -20.6015,27.7026 -20.5661,27.7056 -20.5267,27.6981 -20.5091,27.6838 -20.4961,27.666 -20.4891,27.6258 -20.4886,27.5909 -20.4733,27.5341 -20.483,27.4539 -20.4733,27.3407 -20.473,27.306 -20.4774,27.2684 -20.4958,27.284 -20.3515,27.266 -20.2342,27.2149 -20.1105,27.2018 -20.093,27.1837 -20.0823,27.1629 -20.0766,27.1419 -20.0733,27.1297 -20.0729,27.1198 -20.0739,27.1096 -20.0732,27.0973 -20.0689,27.0865 -20.0605,27.0692 -20.0374,27.0601 -20.0276,27.0267 -20.0101,26.9943 -20.0068,26.9611 -20.0072,26.9251 -20.0009,26.8119 -19.9464,26.7745 -19.9398,26.7508 -19.9396,26.731 -19.9359,26.7139 -19.9274,26.6986 -19.9125,26.6848 -19.8945,26.6772 -19.8868,26.6738 -19.8834,26.6594 -19.8757,26.6141 -19.8634,26.5956 -19.8556,26.5819 -19.8421,26.5748 -19.8195,26.5663 -19.8008,26.5493 -19.7841,26.5089 -19.7593,26.4897 -19.7519,26.4503 -19.7433,26.4319 -19.7365,26.4128 -19.7196,26.3852 -19.6791,26.3627 -19.6676,26.3323 -19.6624,26.3244 -19.6591,26.3122 -19.6514,26.3125 -19.6496,26.3191 -19.6463,26.3263 -19.6339,26.3335 -19.613,26.331 -19.605,26.3211 -19.592,26.3132 -19.5842,26.3035 -19.5773,26.2926 -19.5725,26.2391 -19.5715,26.1945 -19.5602,26.1555 -19.5372,26.1303 -19.5011,26.0344 -19.2437,26.0114 -19.1998,25.9811 -19.1618,25.9565 -19.1221,25.9486 -19.1033,25.9449 -19.0792,25.9481 -19.0587,25.9644 -19.0216,25.9678 -19.001,25.9674 -18.9999,25.9407 -18.9213,25.8153 -18.814,25.7795 -18.7388,25.7734 -18.6656,25.7619 -18.6303,25.7369 -18.6087,25.6983 -18.5902,25.6695 -18.566,25.6221 -18.5011,25.6084 -18.4877,25.5744 -18.4657,25.5085 -18.3991,25.4956 -18.3789,25.4905 -18.3655,25.4812 -18.3234,25.4732 -18.3034,25.4409 -18.2532,25.4088 -18.176,25.3875 -18.139,25.3574 -18.1158,25.3234 -18.0966,25.2964 -18.0686,25.255 -18.0011,25.2261 -17.9319,25.2194 -17.908,25.2194 -17.8798,25.2598 -17.7941,25.2667 -17.8009,25.2854 -17.8093,25.3159 -17.8321,25.3355 -17.8412,25.3453 -17.8426,25.3765 -17.8412,25.4095 -17.853,25.4203 -17.8549,25.4956 -17.8549,25.5007 -17.856,25.5102 -17.8612,25.5165 -17.8623,25.5221 -17.8601,25.5309 -17.851,25.5368 -17.8487,25.604 -17.8362,25.657 -17.8139,25.6814 -17.8115,25.6942 -17.8194,25.7064 -17.8299,25.7438 -17.8394,25.766 -17.8498,25.786 -17.8622,25.7947 -17.8727,25.8044 -17.8882,25.8497 -17.9067,25.8636 -17.9238,25.8475 -17.9294,25.8462 -17.9437,25.8535 -17.96,25.8636 -17.9716,25.9245 -17.999,25.967 -18.0005,25.9785 -17.999,26.0337 -17.9716,26.0406 -17.9785,26.0466 -17.9663,26.0625 -17.9629,26.0812 -17.9624,26.0952 -17.9585,26.0962 -17.9546,26.0942 -17.9419,26.0952 -17.9381,26.1012 -17.9358,26.1186 -17.9316,26.1354 -17.9226,26.1586 -17.9183,26.1675 -17.9136,26.203 -17.8872,26.2119 -17.8828,26.2211 -17.8863,26.2282 -17.8947,26.2339 -17.904,26.2392 -17.9102,26.2483 -17.9134,26.2943 -17.9185,26.3038 -17.9228,26.312 -17.9284,26.3183 -17.9344,26.3255 -17.936,26.3627 -17.9306,26.4086 -17.939,26.4855 -17.9793,26.5271 -17.992,26.5536 -17.9965,26.5702 -18.0029,26.5834 -18.0132,26.5989 -18.03,26.6127 -18.0412,26.6288 -18.0492,26.6857 -18.0668,26.7 -18.0692,26.7119 -18.0658,26.7406 -18.0405,26.7536 -18.033,26.7697 -18.029,26.794 -18.0262,26.8883 -17.9846,26.912 -17.992,26.9487 -17.9689,26.9592 -17.9647,27.0063 -17.9627,27.0213 -17.9585,27.0485 -17.9443,27.0782 -17.917,27.1154 -17.8822,27.149 -17.8425,27.1465 -17.8189,27.1453 -17.7941,27.147 -17.7839,27.1571 -17.7693,27.4221 -17.5048,27.5243 -17.4151,27.5773 -17.3631,27.6045 -17.3128,27.6249 -17.2333,27.6412 -17.1985,27.7773 -17.0012,27.8169 -16.9596,27.8686 -16.9297,28.023 -16.8654,28.1139 -16.8276,28.2125 -16.7486,28.2801 -16.7065,28.6433 -16.5688,28.6907 -16.5603,28.7188 -16.5603,28.7328 -16.5581,28.7414 -16.5507,28.7611 -16.5323,28.7693 -16.5152,28.8089 -16.4863,28.8225 -16.4708,28.8291 -16.4346,28.8331 -16.4264,28.8572 -16.3882,28.857 -16.3655,28.8405 -16.3236,28.8368 -16.3063,28.8403 -16.2847,28.8642 -16.2312,28.8471 -16.2027,28.8525 -16.1628,28.8654 -16.1212,28.871 -16.0872,28.8685 -16.0822,28.8638 -16.0766,28.8593 -16.0696,28.8572 -16.0605,28.8603 -16.0494,28.8741 -16.0289,28.8772 -16.022,28.8989 -15.9955,28.9324 -15.9637,28.9469 -15.9572,28.9513 -15.9553,28.9728 -15.9514,29.0181 -15.9506,29.0423 -15.9463,29.0551 -15.9344,29.0763 -15.8954,29.0862 -15.8846,29.1022 -15.8709,29.1217 -15.8593,29.1419 -15.8545,29.151 -15.8488,29.1863 -15.8128,29.407 -15.7142,29.4221 -15.711,29.5085 -15.7036,29.5262 -15.6928,29.5634 -15.6621,29.5872 -15.6557,29.6086 -15.6584,29.628 -15.6636,29.6485 -15.6666,29.6728 -15.6633,29.73 -15.6447,29.7733 -15.6381,29.8143 -15.6197,29.8373 -15.6148,29.8818 -15.6188,29.9675 -15.6415,30.0107 -15.6462)) +``` + ``` sql -SELECT +SELECT round(polygonPerimeterSpherical([(30.010654, -15.646227), (30.050238, -15.640129), (30.090029, -15.629381), (30.130129, -15.623696), (30.16992, -15.632171), (30.195552, -15.649121), (30.207231, -15.653152), (30.223147, -15.649741), (30.231002, -15.644677), (30.246091, -15.632068), (30.254876, -15.628864), (30.280094, -15.632275), (30.296196, -15.639042), (30.32805, -15.652428), (30.356679, -15.651498), (30.396263, -15.635995), (30.39771, -15.716817), (30.39926, -15.812005), (30.401327, -15.931688), (30.402568, -16.001244), (30.514809, -16.000418), (30.586587, -16.000004), (30.74973, -15.998867), (30.857424, -15.998144), (30.901865, -16.007136), (30.942173, -16.034524), (30.958296, -16.05106), (30.973075, -16.062016), (30.989767, -16.06429), (31.012039, -16.054885), (31.023718, -16.045169), (31.042218, -16.024912), (31.056895, -16.017574), (31.065421, -16.019641), (31.073328, -16.025532), (31.080872, -16.025946), (31.089037, -16.01189), (31.1141, -15.996904), (31.15849, -16.000211), (31.259983, -16.023465), (31.278897, -16.030287), (31.29533, -16.041655), (31.309592, -16.059019), (31.328351, -16.092815), (31.340908, -16.106664), (31.360339, -16.116896), (31.37026, -16.123718), (31.374601, -16.132916), (31.377754, -16.142218), (31.384006, -16.148832), (31.387727, -16.149556), (31.395582, -16.147695), (31.399613, -16.147282), (31.404315, -16.149866), (31.404057, -16.154517), (31.402713, -16.159374), (31.404574, -16.162268), (31.424107, -16.164749), (31.445708, -16.164955), (31.465655, -16.167746), (31.480641, -16.177978), (31.519192, -16.196478), (31.686107, -16.207227), (31.710705, -16.217872), (31.738197, -16.239783), (31.798761, -16.303655), (31.818088, -16.319571), (31.86005, -16.340759), (31.871935, -16.35037), (31.88072, -16.368044), (31.88563, -16.406284), (31.894363, -16.421477), (31.910279, -16.428919), (32.014149, -16.444938), (32.211759, -16.440184), (32.290463, -16.45176), (32.393661, -16.491757), (32.5521, -16.553355), (32.671783, -16.599761), (32.6831, -16.609889), (32.687906, -16.624255), (32.68863, -16.647303), (32.698655, -16.686784), (32.725217, -16.706421), (32.73095, -16.708656), (32.731314, -16.708798), (32.739893, -16.703217), (32.753845, -16.697946), (32.769348, -16.695466), (32.800664, -16.697326), (32.862004, -16.710452), (32.893372, -16.712415), (32.909598, -16.708075), (32.93957, -16.689781), (32.95621, -16.683063), (32.968509, -16.681615999999998), (32.961585, -16.710348), (32.933369, -16.815768), (32.916213, -16.847911), (32.900503, -16.867755), (32.828776, -16.935141), (32.83012, -16.941549), (32.886757, -17.038184), (32.928512, -17.109497), (32.954143, -17.167168), (32.967786, -17.22887), (32.96909, -17.266115), (32.969439, -17.276102), (32.973212, -17.297909), (32.983599, -17.317753), (32.992384, -17.324678), (33.014656, -17.336667), (33.021633, -17.345555), (33.022459, -17.361471), (33.016258, -17.377181), (33.011651, -17.383991), (32.997448, -17.404983), (32.958174, -17.478467), (32.951663, -17.486218), (32.942981, -17.491593), (32.936573, -17.498311), (32.936676, -17.509369), (32.947218, -17.543166), (32.951663, -17.551434), (32.969129, -17.56456), (33.006646, -17.580993), (33.020392, -17.598563), (33.024526, -17.619233), (33.020599, -17.638457), (33.004063, -17.675561), (33.000238, -17.713905), (33.003184, -17.757726), (32.999102, -17.794313), (32.973573, -17.810643), (32.957037, -17.817981), (32.946082, -17.834724), (32.939674, -17.855498), (32.936883, -17.875032), (32.938433, -17.894566), (32.950267, -17.922574), (32.952128, -17.940247), (32.948149, -17.95327), (32.940397, -17.959988), (32.932439, -17.964949), (32.927375, -17.972907), (32.928977, -17.982312), (32.941224, -17.996265), (32.940294, -18.004843), (32.934919, -18.024583), (32.93709, -18.047114), (32.972282, -18.150261), (32.975537, -18.183333), (32.974865, -18.190775), (32.965925, -18.212169), (32.958174, -18.225398), (32.952283, -18.233046), (32.950525999999996, -18.241314), (32.95497, -18.256301), (32.970163, -18.277488), (33.016878, -18.313661), (33.034965, -18.332885), (33.042768, -18.352005), (33.038066, -18.363064), (33.00923, -18.383941), (32.988198, -18.41319), (32.985356, -18.412467), (32.986803, -18.422285), (32.999515, -18.436651), (33.003029, -18.446883), (32.996414, -18.46714), (32.978586, -18.48006), (32.956624, -18.489878), (32.937142, -18.50104), (32.919313, -18.510032), (32.900296, -18.515303), (32.88314, -18.522124), (32.870737, -18.535767), (32.868257, -18.552613), (32.871668, -18.57318), (32.884483, -18.609044), (32.914559, -18.665888), (32.92231, -18.693173), (32.920243, -18.726246), (32.913267, -18.753014), (32.902518, -18.774512), (32.885207, -18.787844), (32.858852, -18.790015), (32.817924, -18.787018), (32.787642, -18.791255), (32.69142, -18.83425), (32.68987, -18.843241), (32.696794, -18.897192), (32.703202, -18.911868), (32.71576, -18.919826), (32.705063, -18.927474), (32.692247, -18.934295), (32.682532, -18.942667), (32.681085, -18.954966), (32.68863, -18.97729), (32.690283, -18.988246), (32.68863, -19.000958), (32.691058, -19.01429), (32.698965, -19.022249), (32.710282, -19.025969), (32.723873, -19.026589), (32.785988, -19.017701), (32.803351, -19.019561), (32.814203, -19.023799), (32.819991, -19.028346), (32.822988, -19.035168), (32.825262, -19.046847), (32.830223, -19.059146), (32.83813, -19.066897), (32.847483, -19.073925), (32.855906, -19.083744), (32.862262, -19.118057), (32.83322, -19.241977), (32.832187, -19.266678), (32.828673, -19.284558), (32.820715, -19.301301), (32.806142, -19.323419), (32.768831, -19.363623), (32.766454, -19.373442), (32.768521, -19.402794), (32.762217, -19.443412), (32.763354, -19.463979), (32.773947, -19.475864), (32.793119, -19.476691), (32.811309, -19.474521), (32.825365, -19.479172), (32.832187, -19.500876), (32.832497000000004, -19.519273), (32.825365, -19.59162), (32.825675, -19.600818), (32.828156, -19.610636), (32.829603, -19.623659), (32.825365, -19.633271), (32.819474, -19.641952), (32.81627, -19.652081), (32.819629, -19.674302), (32.83105, -19.685154), (32.849137, -19.689081), (32.872184, -19.690218), (32.894715, -19.684327), (32.924584, -19.655285), (32.943188, -19.64929), (32.960964, -19.658799), (32.962411, -19.679056), (32.954143, -19.717813), (32.962411, -19.735383), (32.979051, -19.751403), (33.0006, -19.764322), (33.022769, -19.773107), (33.032795, -19.784166), (33.029642, -19.80339), (33.022873, -19.826851), (33.021322, -19.868088), (33.001995, -19.927), (32.998378, -20.000897), (33.004373, -20.024255), (33.007266, -20.032006), (32.95373, -20.030249), (32.940087, -20.041515), (32.934299, -20.072107), (32.926548, -20.086473), (32.910683, -20.091124), (32.894405, -20.094018), (32.88531, -20.10301), (32.877869, -20.151689), (32.872908, -20.167192), (32.859265, -20.190859), (32.857095, -20.200575), (32.858335, -20.207499), (32.865053, -20.220935), (32.86557, -20.228893), (32.858438, -20.259486), (32.852961, -20.273852), (32.845209, -20.286668), (32.800767, -20.338551), (32.735862, -20.414205), (32.704443, -20.471773), (32.671783, -20.531821), (32.646462, -20.557969), (32.603674, -20.56479), (32.556545, -20.559312), (32.513136, -20.564583), (32.481614, -20.603031), (32.471072, -20.645509), (32.469108, -20.68685), (32.483474, -20.794233), (32.49722, -20.898103), (32.491019, -20.936344), (32.467661, -20.980165), (32.417122, -21.040937), (32.339814, -21.134058), (32.345343, -21.142843), (32.359864, -21.151421), (32.368856, -21.162997), (32.373352, -21.163617), (32.377744, -21.16341), (32.380638, -21.165477), (32.380535, -21.172195), (32.376866, -21.178499), (32.37299, -21.183977), (32.37175, -21.187905), (32.444613, -21.304693), (32.445849, -21.308994), (32.447197, -21.313685), (32.408543, -21.290327), (32.37299, -21.327948), (32.324517, -21.378177), (32.272221, -21.432541), (32.219718, -21.486904), (32.167318, -21.541268), (32.114814, -21.595632), (32.062415, -21.649995), (32.010015, -21.704462), (31.957615, -21.758826), (31.905215, -21.813189), (31.852712, -21.867553), (31.800312, -21.92202), (31.747808, -21.976384), (31.695512, -22.030747), (31.643112, -22.085214), (31.590712, -22.139578), (31.538209, -22.193941), (31.485809, -22.248305), (31.433822, -22.302048), (31.36871, -22.345043), (31.288922, -22.39734), (31.265616, -22.365507), (31.255642, -22.357962), (31.24572, -22.357549), (31.229597, -22.363957), (31.221536, -22.364887), (31.213474, -22.36189), (31.197868, -22.352588), (31.190685, -22.350624), (31.183657, -22.34556), (31.163348, -22.322616), (31.152599, -22.316414), (31.137717, -22.318482), (31.10454, -22.333364), (31.097048, -22.334922), (31.087642, -22.336878), (31.07033, -22.333674), (31.036121, -22.319618), (30.927187, -22.295744), (30.867087, -22.289646), (30.83789, -22.282308), (30.805282, -22.294504), (30.693919, -22.302772), (30.674282, -22.30856), (30.647410999999998, -22.32644), (30.632424, -22.330677), (30.625551, -22.32861), (30.610307, -22.318688), (30.601108, -22.316414), (30.57217, -22.316621), (30.507367, -22.309593), (30.488454, -22.310213), (30.46923, -22.315071), (30.431713, -22.331194), (30.412696, -22.336878), (30.372078, -22.343493), (30.334975, -22.344733), (30.300765, -22.336982), (30.269346, -22.316414), (30.25529, -22.304736), (30.240407, -22.296157), (30.2217, -22.290886), (30.196999, -22.289129), (30.15266, -22.294814), (30.13509, -22.293574), (30.111113, -22.282308), (30.082587, -22.262878), (30.067911, -22.25709), (30.038145, -22.253783), (30.035872, -22.250579), (30.034528, -22.246135), (30.015511, -22.227014), (30.005279, -22.22226), (29.983782, -22.217713), (29.973963, -22.213992), (29.946678, -22.198282), (29.932105, -22.194355), (29.896035, -22.191358), (29.871489, -22.179265), (29.837331, -22.172444), (29.779246, -22.136374), (29.758886, -22.130896), (29.691448, -22.1341), (29.679614, -22.138338), (29.661424, -22.126452), (29.641064, -22.129242), (29.60396, -22.145055), (29.570164, -22.141955), (29.551043, -22.145986), (29.542517, -22.162522), (29.53182, -22.172444), (29.506912, -22.170067), (29.456889, -22.158801), (29.436115, -22.163142), (29.399528, -22.182159), (29.378031, -22.192908), (29.363250999999998, -22.192288), (29.356947, -22.190944000000002), (29.350074, -22.186707), (29.273644, -22.125108), (29.26734, -22.115807), (29.259588, -22.096066), (29.254111, -22.087074), (29.244395, -22.075706), (29.239331, -22.072605), (29.144867, -22.075292), (29.10797, -22.069194), (29.070763, -22.051004), (29.040532, -22.020929), (29.021567, -21.982791), (29.013815, -21.940417), (29.017949, -21.898145), (29.028905, -21.876648), (29.045441, -21.852567), (29.057637, -21.829209), (29.05526, -21.809985), (29.038723, -21.797893), (28.998726, -21.786008), (28.980846, -21.774845), (28.951907, -21.768334), (28.891032, -21.764924), (28.860853, -21.757379), (28.714195, -21.693507), (28.66841, -21.679968), (28.629704, -21.651339), (28.6157, -21.647101), (28.585934, -21.644414), (28.553998, -21.636559), (28.542939, -21.638316), (28.532501, -21.643071), (28.497309, -21.651546), (28.481393, -21.657437), (28.464598, -21.660331), (28.443101, -21.655783), (28.361762, -21.616302), (28.321919, -21.603486), (28.284867, -21.596872), (28.165702, -21.595218), (28.090771, -21.581266), (28.032893, -21.577855), (28.016563, -21.572894), (28.002559, -21.564212), (27.990415, -21.551913), (27.984731, -21.542922), (27.975739, -21.522561), (27.970571, -21.514396), (27.963698, -21.510469), (27.958066, -21.511502), (27.953208, -21.510469), (27.949281, -21.500754), (27.954448, -21.487835), (27.950418, -21.482047), (27.943338, -21.479876), (27.939876, -21.478016), (27.941943, -21.468508), (27.949642, -21.456519), (27.953001, -21.448664), (27.950211, -21.438329), (27.920549, -21.381174), (27.904219, -21.364741), (27.897811, -21.35544), (27.896157, -21.347895), (27.896674, -21.332392), (27.8944, -21.32433), (27.884995, -21.310171), (27.849132, -21.269657), (27.823604, -21.231726), (27.793838, -21.197413), (27.724385, -21.149664), (27.709192, -21.134471), (27.674775, -21.090133), (27.666611, -21.071219), (27.666817, -21.053753), (27.678961, -21.000733), (27.680356, -20.979649), (27.672657, -20.923528), (27.672605, -20.913709), (27.675085, -20.891282), (27.674775, -20.879913), (27.676016, -20.866684), (27.681803, -20.857589), (27.689038, -20.849011), (27.694412, -20.837744999999998), (27.709605, -20.756716), (27.707332, -20.716719), (27.682475, -20.637344), (27.690382, -20.60148), (27.702629, -20.566134), (27.705575, -20.526653), (27.698133, -20.509083), (27.683767, -20.49606), (27.66599, -20.489136), (27.625786, -20.488619), (27.590853, -20.473323), (27.534112, -20.483038), (27.45391, -20.473323), (27.340739, -20.473013), (27.306012, -20.477354), (27.268392, -20.49575), (27.283998, -20.35147), (27.266015, -20.234164), (27.214907, -20.110451), (27.201781, -20.092984), (27.183746, -20.082339), (27.16292, -20.076551), (27.141888, -20.073347), (27.129692, -20.072934), (27.119771, -20.073864), (27.109642, -20.073244), (27.097343, -20.068903), (27.086491, -20.060532), (27.069231, -20.03738), (27.060136, -20.027562), (27.02665, -20.010095), (26.9943, -20.006788), (26.961072, -20.007201), (26.925054, -20.000897), (26.811882, -19.94643), (26.774469, -19.939815), (26.750801, -19.939609), (26.730957, -19.935888), (26.713904, -19.927413), (26.698608, -19.91253), (26.684758, -19.894547), (26.67717, -19.886815), (26.673803, -19.883385), (26.659437, -19.875737), (26.614065, -19.863438), (26.595565, -19.855583), (26.581922, -19.842147), (26.574791, -19.819513), (26.566316, -19.800806), (26.549263, -19.784063), (26.508852, -19.759258), (26.489731, -19.75192), (26.450251, -19.743342), (26.431854, -19.73652), (26.412837, -19.71957), (26.385242, -19.679056), (26.362711, -19.667584), (26.332325, -19.662416), (26.324367, -19.659109), (26.312171, -19.651358), (26.312481, -19.649601), (26.319096, -19.646293), (26.326331, -19.633891), (26.333462, -19.613014), (26.330981, -19.604952), (26.32106, -19.592033), (26.313205, -19.584178), (26.30349, -19.577254), (26.292638, -19.572499), (26.239101, -19.571466), (26.194452, -19.560200000000002), (26.155488, -19.537153), (26.13027, -19.501082), (26.034359, -19.243734), (26.011414, -19.199809), (25.981132, -19.161775), (25.956534, -19.122088), (25.948576, -19.103277), (25.944855, -19.079196), (25.948059, -19.058732), (25.964389, -19.021629), (25.9678, -19.000958), (25.967449, -18.999925), (25.940721, -18.921273), (25.815251, -18.813993), (25.779491, -18.738752), (25.773393, -18.665578), (25.761921, -18.630335), (25.736909, -18.608734), (25.698255, -18.590234), (25.669523, -18.566049), (25.622084, -18.501143), (25.608442, -18.487708), (25.574439, -18.465693), (25.508499, -18.399134), (25.49558, -18.378877), (25.490516, -18.365545), (25.481163, -18.323377), (25.473204, -18.303429), (25.440855, -18.2532), (25.408816, -18.175995), (25.387525, -18.138995), (25.357449, -18.115844), (25.323446, -18.09662), (25.296368, -18.068612), (25.255026, -18.001122), (25.226088, -17.931876), (25.21937, -17.908001), (25.21937, -17.879786), (25.259781, -17.794107), (25.266705, -17.800928), (25.285412, -17.809299), (25.315901, -17.83214), (25.335538, -17.841235), (25.345254, -17.842579), (25.376466, -17.841235), (25.409539, -17.853018), (25.420288, -17.854878), (25.49558, -17.854878), (25.500748, -17.856015), (25.510153, -17.861183), (25.516458, -17.862319), (25.522142, -17.860149), (25.530927, -17.850951), (25.536818, -17.848677), (25.603997, -17.836171), (25.657017, -17.81395), (25.681409, -17.81147), (25.694224, -17.819428), (25.70642, -17.829867), (25.743834, -17.839375), (25.765951, -17.849814), (25.786002, -17.862216), (25.794683, -17.872655), (25.804399, -17.888158), (25.849667, -17.906658), (25.86362, -17.923814), (25.847497, -17.929395), (25.846153, -17.943658), (25.853490999999998, -17.959988), (25.86362, -17.971563), (25.924495, -17.998952), (25.966973, -18.000502), (25.978548, -17.998952), (26.033739, -17.971563), (26.04056, -17.978488), (26.046554, -17.966292), (26.062471, -17.962882), (26.081178, -17.962365), (26.095234, -17.958541), (26.096164, -17.954614), (26.0942, -17.941901), (26.095234, -17.938077), (26.101228, -17.935803), (26.118591, -17.931566), (26.135438, -17.922574), (26.158589, -17.918337), (26.167477, -17.913582), (26.203031, -17.887227), (26.211919, -17.882783), (26.221117, -17.886297), (26.228249, -17.894669), (26.233933, -17.903971), (26.239204, -17.910172), (26.248299, -17.913376), (26.294291, -17.918543), (26.3038, -17.922781), (26.311965, -17.928362), (26.318269, -17.934356), (26.325504, -17.93601), (26.362711, -17.930636), (26.408599, -17.939007), (26.485494, -17.979315), (26.527145, -17.992027), (26.553604, -17.996471), (26.570243, -18.002879), (26.583369, -18.013215), (26.598872, -18.029958), (26.612721, -18.041223), (26.628844, -18.049181), (26.685689, -18.066751), (26.700003, -18.069232), (26.71194, -18.065821), (26.740569, -18.0405), (26.753591, -18.032955), (26.769714, -18.029028), (26.794002, -18.026237), (26.88826, -17.984586), (26.912031, -17.992027), (26.94867, -17.968876), (26.95916, -17.964742), (27.006289, -17.962675), (27.021275, -17.958541), (27.048457, -17.944278), (27.078171, -17.916993), (27.11543, -17.882163), (27.149019, -17.842476), (27.146539, -17.818911), (27.145299, -17.794107), (27.146952, -17.783875), (27.157081, -17.769302), (27.422078, -17.504822), (27.524294, -17.415112), (27.577314, -17.363125), (27.604495, -17.312792), (27.624856, -17.233314), (27.641186, -17.198484), (27.777301, -17.001183), (27.816886, -16.959636), (27.868562, -16.929663), (28.022993, -16.865393), (28.113922, -16.827551), (28.21252, -16.748589), (28.280113, -16.706524), (28.643295, -16.568755), (28.690734, -16.56028), (28.718794, -16.56028), (28.73285, -16.55811), (28.741377, -16.550668), (28.761117, -16.532271), (28.769282, -16.515218), (28.808866, -16.486279), (28.822509, -16.470776), (28.829124, -16.434603), (28.833051, -16.426438), (28.857236, -16.388198), (28.857029, -16.36546), (28.840492, -16.323602), (28.836772, -16.306342), (28.840286, -16.284741), (28.86416, -16.231205), (28.847107, -16.202679), (28.852481, -16.162785), (28.8654, -16.121237), (28.870981, -16.087234), (28.868501, -16.08217), (28.86385, -16.076589), (28.859303, -16.069561), (28.857236, -16.060466), (28.860336, -16.049407), (28.874082, -16.028943), (28.877183, -16.022018), (28.898887, -15.995457), (28.932373, -15.963727), (28.946862, -15.957235), (28.951287, -15.955252), (28.972784, -15.951428), (29.018053, -15.950602), (29.042341, -15.946261), (29.055053, -15.934375), (29.076344, -15.895411), (29.086162, -15.884559), (29.102182, -15.870916), (29.121716, -15.859341), (29.141869, -15.854483), (29.150964, -15.848799), (29.186311, -15.812832), (29.406969, -15.714233), (29.422059, -15.711030000000001), (29.508462, -15.703588), (29.526239, -15.692839), (29.563446, -15.662144), (29.587217, -15.655736), (29.608559, -15.658422999999999), (29.62799, -15.663591), (29.648505, -15.666588), (29.672793, -15.663281), (29.73005, -15.644677), (29.773252, -15.638062), (29.814283, -15.619666), (29.837331, -15.614808), (29.881773, -15.618839), (29.967504, -15.641473), (30.010654, -15.646227)]), 6) ``` ```response +0.45539 ``` ### Input parameters @@ -274,21 +304,25 @@ SELECT ## polygonsIntersectionCartesian -Calculates +Calculates the intersection of polygons. ### Example ``` sql -SELECT +SELECT wkt(polygonsIntersectionCartesian([[[(0., 0.), (0., 3.), (1., 2.9), (2., 2.6), (2.6, 2.), (2.9, 1.), (3., 0.), (0., 0.)]]], [[[(1., 1.), (1., 4.), (4., 4.), (4., 1.), (1., 1.)]]])) ``` ```response +MULTIPOLYGON(((1 2.9,2 2.6,2.6 2,2.9 1,1 1,1 2.9))) ``` ### Input parameters +Polygons ### Returned value +MultiPolygon + ## polygonAreaCartesian Calculates the area of a polygon @@ -333,18 +367,24 @@ Float64 ## polygonsUnionCartesian -Calculates +Calculates the union of polygons. ### Example ``` sql -SELECT +SELECT wkt(polygonsUnionCartesian([[[(0., 0.), (0., 3.), (1., 2.9), (2., 2.6), (2.6, 2.), (2.9, 1), (3., 0.), (0., 0.)]]], [[[(1., 1.), (1., 4.), (4., 4.), (4., 1.), (1., 1.)]]])) ``` ```response +MULTIPOLYGON(((1 2.9,1 4,4 4,4 1,2.9 1,3 0,0 0,0 3,1 2.9))) ``` ### Input parameters +Polygons + ### Returned value +MultiPolygon + For more information on geometry systems, see this [presentation](https://archive.fosdem.org/2020/schedule/event/working_with_spatial_trajectories_in_boost_geometry/attachments/slides/3988/export/events/attachments/working_with_spatial_trajectories_in_boost_geometry/slides/3988/FOSDEM20_vissarion.pdf) about the Boost library, which is what ClickHouse uses. + From 48080468ceed9cbbc82a2b1655e289ecc94ae1c7 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 23 May 2023 19:13:11 -0400 Subject: [PATCH 0588/2223] add description --- docs/en/sql-reference/functions/geo/polygon.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index 41386bd1183..00e7d2f99f9 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -280,7 +280,7 @@ MULTIPOLYGON(((4.36661 50.8434,4.36623 50.8408,4.34496 50.8333,4.33807 50.8487,4 ## polygonPerimeterSpherical -Calculates +Calculates the perimeter of the polygon. ### Example @@ -346,7 +346,7 @@ Float64 ## polygonPerimeterCartesian -Calculates +Calculates the perimeter of a polygon. ### Example From be49281044eba2be91c46666ce12a28da446585c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 24 May 2023 00:48:09 +0000 Subject: [PATCH 0589/2223] Try to fix test --- .../test/integration/runner/compose/docker_compose_mongo.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_mongo.yml b/docker/test/integration/runner/compose/docker_compose_mongo.yml index 8cdcbc421e8..9a6eae6ca8c 100644 --- a/docker/test/integration/runner/compose/docker_compose_mongo.yml +++ b/docker/test/integration/runner/compose/docker_compose_mongo.yml @@ -1,7 +1,7 @@ version: '2.3' services: mongo1: - image: mongo:6.0 + image: mongo:5.0 restart: always environment: MONGO_INITDB_ROOT_USERNAME: root @@ -11,7 +11,7 @@ services: command: --profile=2 --verbose mongo2: - image: mongo:6.0 + image: mongo:5.0 restart: always ports: - ${MONGO_NO_CRED_EXTERNAL_PORT:-27017}:${MONGO_NO_CRED_INTERNAL_PORT:-27017} From b11aa42db9337ea652f8a0b36c9e3f21e9f675af Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 24 May 2023 14:27:49 +0800 Subject: [PATCH 0590/2223] Fix tests --- src/Storages/MergeTree/KeyCondition.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index dea2091f115..923e5237420 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1041,7 +1041,8 @@ bool KeyCondition::transformConstantWithValidFunctions( const auto & func_name = func->function_base->getName(); auto func_base = func->function_base; - if (date_time_parsing_functions.contains(func_name)) + const auto & arg_types = func_base->getArgumentTypes(); + if (date_time_parsing_functions.contains(func_name) && !arg_types.empty() && isStringOrFixedString(arg_types[0])) { auto func_or_null = FunctionFactory::instance().get(func_name + "OrNull", context); ColumnsWithTypeAndName arguments; From 79c5aa23585efb20d410dccd8036af968525a71b Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Wed, 24 May 2023 06:52:22 +0000 Subject: [PATCH 0591/2223] Remove test from broken_tests.txt --- tests/broken_tests.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index cef8f68b210..e61c1316e17 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -137,4 +137,3 @@ 01600_parts_types_metrics_long 01287_max_execution_speed 02703_row_policy_for_database -02732_rename_after_processing From 161afea266bc8f3a13dacfb99f7333ab98f2ac46 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 May 2023 07:36:39 +0000 Subject: [PATCH 0592/2223] Add support for changelog --- programs/keeper/Keeper.cpp | 2 - src/Coordination/Changelog.cpp | 324 +++++++++++------- src/Coordination/Changelog.h | 4 +- src/Coordination/KeeperContext.cpp | 12 +- src/Coordination/KeeperContext.h | 2 + src/Coordination/KeeperSnapshotManager.cpp | 47 ++- src/Coordination/KeeperStateManager.cpp | 3 +- src/Coordination/Standalone/Context.cpp | 4 + src/Core/SettingsFields.cpp | 5 + .../MetadataStorageFromPlainObjectStorage.cpp | 1 + .../MetadataStorageFromPlainObjectStorage.h | 2 +- src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 11 +- src/IO/ZstdDeflatingAppendableWriteBuffer.h | 3 + src/Interpreters/Context.h | 2 +- 14 files changed, 277 insertions(+), 145 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index d5cf61daa6e..002ce413095 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -44,8 +44,6 @@ #include -#include - int mainEntryClickHouseKeeper(int argc, char ** argv) { diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 852a21c1c45..c94633d6dbd 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -3,17 +3,17 @@ #include #include #include +#include #include #include +#include #include #include #include -#include #include #include +#include #include -#include -#include namespace DB @@ -29,50 +29,58 @@ namespace ErrorCodes namespace { + void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr description, DiskPtr disk_to, const std::string & path_to) + { + disk_from->copyFile(description->path, *disk_to, path_to, {}); + disk_from->removeFile(description->path); + description->path = path_to; + description->disk = disk_to; + } -constexpr auto DEFAULT_PREFIX = "changelog"; + constexpr auto DEFAULT_PREFIX = "changelog"; -inline std::string formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) -{ - return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension); -} + inline std::string + formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) + { + return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension); + } -ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path) -{ - // we can have .bin.zstd so we cannot use std::filesystem stem and extension - std::string filename_with_extension = path.filename(); - std::string_view filename_with_extension_view = filename_with_extension; + ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path) + { + // we can have .bin.zstd so we cannot use std::filesystem stem and extension + std::string filename_with_extension = path.filename(); + std::string_view filename_with_extension_view = filename_with_extension; - auto first_dot = filename_with_extension.find('.'); - if (first_dot == std::string::npos) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid changelog file {}", path.generic_string()); + auto first_dot = filename_with_extension.find('.'); + if (first_dot == std::string::npos) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid changelog file {}", path.generic_string()); - Strings filename_parts; - boost::split(filename_parts, filename_with_extension_view.substr(0, first_dot), boost::is_any_of("_")); - if (filename_parts.size() < 3) - throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path.generic_string()); + Strings filename_parts; + boost::split(filename_parts, filename_with_extension_view.substr(0, first_dot), boost::is_any_of("_")); + if (filename_parts.size() < 3) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path.generic_string()); - auto result = std::make_shared(); - result->prefix = filename_parts[0]; - result->from_log_index = parse(filename_parts[1]); - result->to_log_index = parse(filename_parts[2]); - result->extension = std::string(filename_with_extension.substr(first_dot + 1)); - result->path = path.generic_string(); - return result; -} + auto result = std::make_shared(); + result->prefix = filename_parts[0]; + result->from_log_index = parse(filename_parts[1]); + result->to_log_index = parse(filename_parts[2]); + result->extension = std::string(filename_with_extension.substr(first_dot + 1)); + result->path = path.generic_string(); + return result; + } -Checksum computeRecordChecksum(const ChangelogRecord & record) -{ - SipHash hash; - hash.update(record.header.version); - hash.update(record.header.index); - hash.update(record.header.term); - hash.update(record.header.value_type); - hash.update(record.header.blob_size); - if (record.header.blob_size != 0) - hash.update(reinterpret_cast(record.blob->data_begin()), record.blob->size()); - return hash.get64(); -} + Checksum computeRecordChecksum(const ChangelogRecord & record) + { + SipHash hash; + hash.update(record.header.version); + hash.update(record.header.index); + hash.update(record.header.term); + hash.update(record.header.value_type); + hash.update(record.header.blob_size); + if (record.header.blob_size != 0) + hash.update(reinterpret_cast(record.blob->data_begin()), record.blob->size()); + return hash.get64(); + } } @@ -117,25 +125,55 @@ public: // if we wrote at least 1 log in the log file we can rename the file to reflect correctly the // contained logs // file can be deleted from disk earlier by compaction - if (!current_file_description->deleted && last_index_written - && *last_index_written != current_file_description->to_log_index) + if (!current_file_description->deleted) { - auto new_path = formatChangelogPath( - current_file_description->prefix, - current_file_description->from_log_index, - *last_index_written, - current_file_description->extension); - disk->moveFile(current_file_description->path, new_path); - current_file_description->path = std::move(new_path); + auto log_disk = current_file_description->disk; + const auto & path = current_file_description->path; + std::string new_path = path; + if (last_index_written && *last_index_written != current_file_description->to_log_index) + { + new_path = formatChangelogPath( + current_file_description->prefix, + current_file_description->from_log_index, + *last_index_written, + current_file_description->extension); + } + + if (disk == log_disk) + { + if (path != new_path) + { + try + { + disk->moveFile(path, new_path); + } + catch (...) + { + tryLogCurrentException(log, fmt::format("File rename failed on disk {}", disk->getName())); + } + current_file_description->path = std::move(new_path); + } + } + else + { + moveFileBetweenDisks(log_disk, current_file_description, disk, new_path); + } } } - file_buf = disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode); + auto current_log_disk = getCurrentLogDisk(); + assert(file_description->disk == current_log_disk); + file_buf = current_log_disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode); + assert(file_buf); last_index_written.reset(); current_file_description = std::move(file_description); if (log_file_settings.compress_logs) - compressed_buffer = std::make_unique(std::move(file_buf), /* compression level = */ 3, /* append_to_existing_file_ = */ mode == WriteMode::Append); + compressed_buffer = std::make_unique( + std::move(file_buf), + /* compressi)on level = */ 3, + /* append_to_existing_file_ = */ mode == WriteMode::Append, + [current_log_disk, path = current_file_description->path] { return current_log_disk->readFile(path); }); prealloc_done = false; } @@ -147,10 +185,7 @@ public: } /// There is bug when compressed_buffer has value, file_buf's ownership transfer to compressed_buffer - bool isFileSet() const - { - return compressed_buffer != nullptr || file_buf != nullptr; - } + bool isFileSet() const { return compressed_buffer != nullptr || file_buf != nullptr; } bool appendRecord(ChangelogRecord && record) { @@ -236,6 +271,7 @@ public: new_description->from_log_index = new_start_log_index; new_description->to_log_index = new_start_log_index + log_file_settings.rotate_interval - 1; new_description->extension = "bin"; + new_description->disk = getCurrentLogDisk(); if (log_file_settings.compress_logs) new_description->extension += "." + toContentEncodingName(CompressionMethod::Zstd); @@ -259,7 +295,6 @@ public: } private: - void finalizeCurrentFile() { assert(prealloc_done); @@ -279,14 +314,13 @@ private: const auto * file_buffer = tryGetFileBuffer(); - if (log_file_settings.max_size != 0 && isLocalDisk()) + if (log_file_settings.max_size != 0 && file_buffer) { int res = -1; do { res = ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); - } - while (res < 0 && errno == EINTR); + } while (res < 0 && errno == EINTR); if (res != 0) LOG_WARNING(log, "Could not ftruncate file. Error: {}, errno: {}", errnoToString(), errno); @@ -321,10 +355,7 @@ private: return *file_buffer; } - const WriteBufferFromFile * tryGetFileBuffer() const - { - return const_cast(this)->tryGetFileBuffer(); - } + const WriteBufferFromFile * tryGetFileBuffer() const { return const_cast(this)->tryGetFileBuffer(); } WriteBufferFromFile * tryGetFileBuffer() { @@ -344,30 +375,22 @@ private: void tryPreallocateForFile() { - if (log_file_settings.max_size == 0) - { - initial_file_size = 0; - prealloc_done = true; - return; - } - const auto * file_buffer = tryGetFileBuffer(); - if (!file_buffer) + if (log_file_settings.max_size == 0 || !file_buffer) { initial_file_size = 0; prealloc_done = true; - LOG_WARNING(log, "Could not preallocate space on disk {} using fallocate", getDisk()->getName()); return; } #ifdef OS_LINUX - if (isLocalDisk()) { int res = -1; do { - res = fallocate(file_buffer->getFD(), FALLOC_FL_KEEP_SIZE, 0, log_file_settings.max_size + log_file_settings.overallocate_size); + res = fallocate( + file_buffer->getFD(), FALLOC_FL_KEEP_SIZE, 0, log_file_settings.max_size + log_file_settings.overallocate_size); } while (res < 0 && errno == EINTR); if (res != 0) @@ -387,15 +410,11 @@ private: prealloc_done = true; } - DiskPtr getDisk() const - { - return keeper_context->getLogDisk(); - } + DiskPtr getCurrentLogDisk() const { return keeper_context->getCurrentLogDisk(); } - bool isLocalDisk() const - { - return dynamic_cast(getDisk().get()) != nullptr; - } + DiskPtr getDisk() const { return keeper_context->getLogDisk(); } + + bool isLocalDisk() const { return dynamic_cast(getDisk().get()) != nullptr; } std::map & existing_changelogs; @@ -440,9 +459,7 @@ struct ChangelogReadResult class ChangelogReader { public: - explicit ChangelogReader(DiskPtr disk_, const std::string & filepath_) - : disk(disk_) - , filepath(filepath_) + explicit ChangelogReader(DiskPtr disk_, const std::string & filepath_) : disk(disk_), filepath(filepath_) { auto compression_method = chooseCompressionMethod(filepath, ""); auto read_buffer_from_file = disk->readFile(filepath); @@ -546,10 +563,7 @@ private: std::unique_ptr read_buf; }; -Changelog::Changelog( - Poco::Logger * log_, - LogFileSettings log_file_settings, - KeeperContextPtr keeper_context_) +Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, KeeperContextPtr keeper_context_) : changelogs_detached_dir("detached") , rotate_interval(log_file_settings.rotate_interval) , log(log_) @@ -557,18 +571,30 @@ Changelog::Changelog( , append_completion_queue(std::numeric_limits::max()) , keeper_context(std::move(keeper_context_)) { - /// Load all files in changelog directory + /// Load all files on changelog disks + + const auto load_from_disk = [&](const auto & disk) + { + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) + { + if (it->name() == changelogs_detached_dir) + continue; + + auto file_description = getChangelogFileDescription(it->path()); + file_description->disk = disk; + + auto [changelog_it, inserted] = existing_changelogs.insert_or_assign(file_description->from_log_index, std::move(file_description)); + + if (!inserted) + LOG_WARNING(log, "Found duplicate entries for {}, will use the entry from {}", changelog_it->second->path, disk->getName()); + } + }; auto disk = getDisk(); + load_from_disk(disk); - for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) - { - if (it->name() == changelogs_detached_dir) - continue; - - auto file_description = getChangelogFileDescription(it->path()); - existing_changelogs[file_description->from_log_index] = std::move(file_description); - } + auto current_log_disk = getCurrentLogDisk(); + load_from_disk(current_log_disk); if (existing_changelogs.empty()) LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", disk->getPath()); @@ -579,8 +605,7 @@ Changelog::Changelog( append_completion_thread = ThreadFromGlobalPool([this] { appendCompletionThread(); }); - current_writer = std::make_unique( - existing_changelogs, keeper_context, log_file_settings); + current_writer = std::make_unique(existing_changelogs, keeper_context, log_file_settings); } void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) @@ -652,7 +677,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin break; } - ChangelogReader reader(getDisk(), changelog_description.path); + ChangelogReader reader(changelog_description.disk, changelog_description.path); last_log_read_result = reader.readChangelog(logs, start_to_read_from, log); last_log_read_result->log_start_index = changelog_description.from_log_index; @@ -713,13 +738,13 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin assert(existing_changelogs.find(last_log_read_result->log_start_index) != existing_changelogs.end()); assert(existing_changelogs.find(last_log_read_result->log_start_index)->first == existing_changelogs.rbegin()->first); - /// Continue to write into incomplete existing log if it doesn't finished with error + /// Continue to write into incomplete existing log if it doesn't finish with error const auto & description = existing_changelogs[last_log_read_result->log_start_index]; if (last_log_read_result->last_read_index == 0 || last_log_read_result->error) /// If it's broken log then remove it { LOG_INFO(log, "Removing chagelog {} because it's empty or read finished with error", description->path); - getDisk()->removeFile(description->path); + description->disk->removeFile(description->path); existing_changelogs.erase(last_log_read_result->log_start_index); std::erase_if(logs, [last_log_read_result](const auto & item) { return item.first >= last_log_read_result->log_start_index; }); } @@ -728,6 +753,16 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin initWriter(description); } } + else if (last_log_read_result.has_value()) + { + /// check if we need to move it to another disk + auto current_log_disk = getCurrentLogDisk(); + auto disk = getDisk(); + + auto & description = existing_changelogs.at(last_log_read_result->log_start_index); + if (current_log_disk != disk && current_log_disk == description->disk) + moveFileBetweenDisks(current_log_disk, description, disk, description->path); + } /// Start new log if we don't initialize writer from previous log. All logs can be "complete". if (!current_writer->isFileSet()) @@ -740,27 +775,37 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin void Changelog::initWriter(ChangelogFileDescriptionPtr description) { if (description->expectedEntriesCountInLog() != rotate_interval) - LOG_TRACE(log, "Looks like rotate_logs_interval was changed, current {}, expected entries in last log {}", rotate_interval, description->expectedEntriesCountInLog()); + LOG_TRACE( + log, + "Looks like rotate_logs_interval was changed, current {}, expected entries in last log {}", + rotate_interval, + description->expectedEntriesCountInLog()); LOG_TRACE(log, "Continue to write into {}", description->path); + + auto log_disk = description->disk; + auto current_log_disk = getCurrentLogDisk(); + if (log_disk != current_log_disk) + moveFileBetweenDisks(log_disk, description, current_log_disk, description->path); + current_writer->setFile(std::move(description), WriteMode::Append); } namespace { -std::string getCurrentTimestampFolder() -{ - const auto timestamp = LocalDateTime{std::time(nullptr)}; - return fmt::format( - "{:02}{:02}{:02}T{:02}{:02}{:02}", - timestamp.year(), - timestamp.month(), - timestamp.day(), - timestamp.hour(), - timestamp.minute(), - timestamp.second()); -} + std::string getCurrentTimestampFolder() + { + const auto timestamp = LocalDateTime{std::time(nullptr)}; + return fmt::format( + "{:02}{:02}{:02}T{:02}{:02}{:02}", + timestamp.year(), + timestamp.month(), + timestamp.day(), + timestamp.hour(), + timestamp.minute(), + timestamp.second()); + } } @@ -769,6 +814,11 @@ DiskPtr Changelog::getDisk() const return keeper_context->getLogDisk(); } +DiskPtr Changelog::getCurrentLogDisk() const +{ + return keeper_context->getCurrentLogDisk(); +} + void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end) { auto disk = getDisk(); @@ -786,7 +836,23 @@ void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end) LOG_WARNING(log, "Removing changelog {}", itr->second->path); const std::filesystem::path & path = itr->second->path; const auto new_path = timestamp_folder / path.filename(); - disk->moveFile(path.generic_string(), new_path.generic_string()); + + auto changelog_disk = itr->second->disk; + if (changelog_disk == disk) + { + try + { + disk->moveFile(path.generic_string(), new_path.generic_string()); + } + catch (const DB::Exception & e) + { + if (e.code() == DB::ErrorCodes::NOT_IMPLEMENTED) + moveFileBetweenDisks(changelog_disk, itr->second, disk, new_path); + } + } + else + moveFileBetweenDisks(changelog_disk, itr->second, disk, new_path); + itr = existing_changelogs.erase(itr); } } @@ -921,7 +987,6 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before writing records"); { - std::lock_guard lock(writer_mutex); /// This write_at require to overwrite everything in this file and also in previous file(s) const bool go_to_previous_file = index < current_writer->getStartIndex(); @@ -937,13 +1002,18 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) else description = std::prev(index_changelog)->second; + auto log_disk = description->disk; + auto current_log_disk = getCurrentLogDisk(); + if (log_disk != current_log_disk) + moveFileBetweenDisks(log_disk, description, current_log_disk, description->path); + current_writer->setFile(std::move(description), WriteMode::Append); /// Remove all subsequent files if overwritten something in previous one auto to_remove_itr = existing_changelogs.upper_bound(index); for (auto itr = to_remove_itr; itr != existing_changelogs.end();) { - getDisk()->removeFile(itr->second->path); + itr->second->disk->removeFile(itr->second->path); itr = existing_changelogs.erase(itr); } } @@ -993,16 +1063,17 @@ void Changelog::compact(uint64_t up_to_log_index) LOG_INFO(log, "Removing changelog {} because of compaction", changelog_description.path); /// If failed to push to queue for background removing, then we will remove it now - if (!log_files_to_delete_queue.tryPush(changelog_description.path, 1)) + if (!log_files_to_delete_queue.tryPush({changelog_description.path, changelog_description.disk}, 1)) { try { - getDisk()->removeFile(itr->second->path); - LOG_INFO(log, "Removed changelog {} because of compaction.", itr->second->path); + changelog_description.disk->removeFile(changelog_description.path); + LOG_INFO(log, "Removed changelog {} because of compaction.", changelog_description.path); } catch (Exception & e) { - LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", itr->second->path, e.message()); + LOG_WARNING( + log, "Failed to remove changelog {} in compaction, error message: {}", changelog_description.path, e.message()); } catch (...) { @@ -1197,12 +1268,13 @@ Changelog::~Changelog() void Changelog::cleanLogThread() { - std::string path; - while (log_files_to_delete_queue.pop(path)) + std::pair path_with_disk; + while (log_files_to_delete_queue.pop(path_with_disk)) { + const auto & [path, disk] = path_with_disk; try { - getDisk()->removeFile(path); + disk->removeFile(path); LOG_INFO(log, "Removed changelog {} because of compaction.", path); } catch (Exception & e) diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 6f0c4e45605..4054829ef19 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -60,6 +60,7 @@ struct ChangelogFileDescription uint64_t to_log_index; std::string extension; + DiskPtr disk; std::string path; bool deleted = false; @@ -154,6 +155,7 @@ private: static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry); DiskPtr getDisk() const; + DiskPtr getCurrentLogDisk() const; /// Currently existing changelogs std::map existing_changelogs; @@ -187,7 +189,7 @@ private: uint64_t max_log_id = 0; /// For compaction, queue of delete not used logs /// 128 is enough, even if log is not removed, it's not a problem - ConcurrentBoundedQueue log_files_to_delete_queue{128}; + ConcurrentBoundedQueue> log_files_to_delete_queue{128}; ThreadFromGlobalPool clean_log_thread; struct AppendLog diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index ff3f96f199b..bacafe75d13 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -21,6 +21,12 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance()); log_storage = getLogsPathFromConfig(config); + + if (config.has("keeper_server.current_log_storage_disk")) + current_log_storage = config.getString("keeper_server.current_log_storage_disk"); + else + current_log_storage = log_storage; + snapshot_storage = getSnapshotsPathFromConfig(config); state_file_storage = getStatePathFromConfig(config); @@ -57,7 +63,6 @@ DiskPtr KeeperContext::getDisk(const Storage & storage) const return *storage_disk; const auto & disk_name = std::get(storage); - return disk_selector->get(disk_name); } @@ -66,6 +71,11 @@ DiskPtr KeeperContext::getLogDisk() const return getDisk(log_storage); } +DiskPtr KeeperContext::getCurrentLogDisk() const +{ + return getDisk(current_log_storage); +} + DiskPtr KeeperContext::getSnapshotsDisk() const { return getDisk(snapshot_storage); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 1fc01f12bba..aec2061bddf 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -32,6 +32,7 @@ public: bool digestEnabled() const; void setDigestEnabled(bool digest_enabled_); + DiskPtr getCurrentLogDisk() const; DiskPtr getLogDisk() const; DiskPtr getSnapshotsDisk() const; DiskPtr getStateFileDisk() const; @@ -53,6 +54,7 @@ private: std::shared_ptr disk_selector; Storage log_storage; + Storage current_log_storage; Storage snapshot_storage; Storage state_file_storage; diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index d47ea475c42..bfadf3af9aa 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -520,19 +520,41 @@ KeeperSnapshotManager::KeeperSnapshotManager( , keeper_context(keeper_context_) { auto disk = getDisk(); + + std::unordered_set invalid_snapshots; + /// collect invalid snapshots for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { const auto & name = it->name(); if (name.empty()) continue; + if (startsWith(name, "tmp_")) + { + disk->removeFile(it->path()); + invalid_snapshots.insert(name.substr(4)); + continue; + } + + } + + /// process snapshots + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) + { + const auto & name = it->name(); + if (name.empty()) + continue; + + /// Not snapshot file + if (!startsWith(name, "snapshot_")) + continue; + + if (invalid_snapshots.contains(name)) { disk->removeFile(it->path()); continue; } - /// Not snapshot file - if (!startsWith(name, "snapshot_")) - continue; + size_t snapshot_up_to = getSnapshotPathUpToLogIdx(name); existing_snapshots[snapshot_up_to] = it->path(); } @@ -549,11 +571,16 @@ std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; auto disk = getDisk(); - auto plain_buf = disk->writeFile(tmp_snapshot_file_name); + + { + disk->writeFile(tmp_snapshot_file_name); + } + + auto plain_buf = disk->writeFile(snapshot_file_name); copyData(reader, *plain_buf); plain_buf->sync(); - disk->moveFile(tmp_snapshot_file_name, snapshot_file_name); + disk->removeFile(tmp_snapshot_file_name); existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); removeOutdatedSnapshotsIfNeeded(); @@ -673,7 +700,12 @@ std::pair KeeperSnapshotManager::serializeSnapshot auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - auto writer = getDisk()->writeFile(tmp_snapshot_file_name); + auto disk = getDisk(); + { + disk->writeFile(tmp_snapshot_file_name); + } + + auto writer = disk->writeFile(snapshot_file_name); std::unique_ptr compressed_writer; if (compress_snapshots_zstd) compressed_writer = wrapWriteBufferWithCompressionMethod(std::move(writer), CompressionMethod::Zstd, 3); @@ -688,7 +720,8 @@ std::pair KeeperSnapshotManager::serializeSnapshot try { - getDisk()->moveFile(tmp_snapshot_file_name, snapshot_file_name); + std::cout << "Removing file " << tmp_snapshot_file_name << std::endl; + disk->removeFile(tmp_snapshot_file_name); } catch (fs::filesystem_error & e) { diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 352e67e7c43..6c1b14bc94f 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -385,7 +385,7 @@ nuraft::ptr KeeperStateManager::read_state() } auto state = nuraft::srv_state::deserialize(*state_buf); - LOG_INFO(logger, "Read state from {}", disk->getPath() + path); + LOG_INFO(logger, "Read state from {}", fs::path(disk->getPath()) / path); return state; } catch (const std::exception & e) @@ -408,7 +408,6 @@ nuraft::ptr KeeperStateManager::read_state() if (state) { disk->removeFileIfExists(old_path); - return state; } diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 667fabf4d66..84083169df3 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -28,6 +28,10 @@ namespace CurrentMetrics namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} struct ContextSharedPart : boost::noncopyable { diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index c1f9fa00f2a..7711ed7465b 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -381,6 +381,11 @@ void SettingFieldMap::readBinary(ReadBuffer & in) #else +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + SettingFieldMap::SettingFieldMap(const Field &) : value(Map()) {} String SettingFieldMap::toString() const { diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index a680a344746..650fde7bcd1 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -120,6 +120,7 @@ const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getSt void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::string & path) { auto object = StoredObject(metadata_storage.getAbsolutePath(path)); + std::cout << "Removing from plain " << path << std::endl; metadata_storage.object_storage->removeObject(object); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 0beed65879b..fb5b6d0757c 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -63,7 +63,7 @@ public: uint32_t getHardlinkCount(const std::string & /* path */) const override { - return 1; + return 0; } bool supportsChmod() const override { return false; } diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index 406de532216..81be8d8ce4d 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -1,5 +1,6 @@ #include #include +#include "IO/ReadBufferFromFileBase.h" #include namespace DB @@ -14,11 +15,13 @@ ZstdDeflatingAppendableWriteBuffer::ZstdDeflatingAppendableWriteBuffer( std::unique_ptr out_, int compression_level, bool append_to_existing_file_, + std::function()> read_buffer_creator_, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment) , out(std::move(out_)) + , read_buffer_creator(std::move(read_buffer_creator_)) , append_to_existing_file(append_to_existing_file_) { cctx = ZSTD_createCCtx(); @@ -194,13 +197,13 @@ void ZstdDeflatingAppendableWriteBuffer::addEmptyBlock() bool ZstdDeflatingAppendableWriteBuffer::isNeedToAddEmptyBlock() { - ReadBufferFromFile reader(out->getFileName()); - auto fsize = reader.getFileSize(); + auto reader = read_buffer_creator(); + auto fsize = reader->getFileSize(); if (fsize > 3) { std::array result; - reader.seek(fsize - 3, SEEK_SET); - reader.readStrict(result.data(), 3); + reader->seek(fsize - 3, SEEK_SET); + reader->readStrict(result.data(), 3); /// If we don't have correct block in the end, then we need to add it manually. /// NOTE: maybe we can have the same bytes in case of data corruption/unfinished write. diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.h b/src/IO/ZstdDeflatingAppendableWriteBuffer.h index b64e5d5c4cf..d9c4f32d6da 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.h +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.h @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -32,6 +33,7 @@ public: std::unique_ptr out_, int compression_level, bool append_to_existing_file_, + std::function()> read_buffer_creator_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0); @@ -69,6 +71,7 @@ private: void addEmptyBlock(); std::unique_ptr out; + std::function()> read_buffer_creator; bool append_to_existing_file = false; ZSTD_CCtx * cctx; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 0ec39f18757..84e42830dee 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1235,7 +1235,7 @@ struct HTTPContext : public IHTTPContext } -#else +#else #include From 8bbfdcc56c1ad77729529b1bcbb65d4a5b7c2b6d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 24 May 2023 15:47:38 +0800 Subject: [PATCH 0593/2223] Fix index analysis with binary operator null --- src/Storages/MergeTree/KeyCondition.cpp | 4 ++++ ...ndex_analysis_binary_operator_with_null.reference | 0 ...2746_index_analysis_binary_operator_with_null.sql | 12 ++++++++++++ 3 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.reference create mode 100644 tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index b8ef2152a99..239a534ca93 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1334,6 +1334,10 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( arguments.push_back(const_arg); kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST; } + + /// If constant arg of binary operator is NULL, there will be no monotonicity. + if (const_arg.column->isNullAt(0)) + return false; } else arguments.push_back({ nullptr, key_column_type, "" }); diff --git a/tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.reference b/tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.sql b/tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.sql new file mode 100644 index 00000000000..f9613735bbf --- /dev/null +++ b/tests/queries/0_stateless/02746_index_analysis_binary_operator_with_null.sql @@ -0,0 +1,12 @@ +drop table if exists tab; + +create table tab (x DateTime) engine MergeTree order by x; + +SELECT toDateTime(65537, toDateTime(NULL), NULL) +FROM tab +WHERE ((x + CAST('1', 'Nullable(UInt8)')) <= 2) AND ((x + CAST('', 'Nullable(UInt8)')) <= 256) +ORDER BY + toDateTime(toDateTime(-2, NULL, NULL) + 100.0001, NULL, -2, NULL) DESC NULLS LAST, + x ASC NULLS LAST; + +drop table tab; From 5db21607627cf244ce40cf12d8be3dd4430c218a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 May 2023 09:04:12 +0000 Subject: [PATCH 0594/2223] Fix build --- programs/keeper-converter/KeeperConverter.cpp | 3 +- src/Coordination/KeeperContext.cpp | 18 +- src/Coordination/KeeperContext.h | 7 +- src/Coordination/KeeperDispatcher.cpp | 44 +- src/Coordination/KeeperServer.cpp | 3 +- src/Coordination/KeeperServer.h | 1 + src/Coordination/KeeperSnapshotManager.cpp | 2 +- src/Coordination/tests/gtest_coordination.cpp | 463 ++++++++++++------ 8 files changed, 358 insertions(+), 183 deletions(-) diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index 58d090ca8b9..f2389021cb6 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -42,6 +42,7 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) { auto keeper_context = std::make_shared(true); keeper_context->setDigestEnabled(true); + keeper_context->setSnapshotDisk(std::make_shared("Keeper-snapshots", options["output-dir"].as(), 0)); DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); @@ -52,7 +53,7 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(storage.getZXID(), 1, std::make_shared()); DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); - DB::KeeperSnapshotManager manager(std::make_shared("Keeper-snapshots", options["output-dir"].as(), 0), 1, keeper_context); + DB::KeeperSnapshotManager manager(1, keeper_context); auto snp = manager.serializeSnapshotToBuffer(snapshot); auto path = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); std::cout << "Snapshot serialized to path:" << path << std::endl; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index bacafe75d13..3c6411a3a24 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -76,16 +76,32 @@ DiskPtr KeeperContext::getCurrentLogDisk() const return getDisk(current_log_storage); } -DiskPtr KeeperContext::getSnapshotsDisk() const +void KeeperContext::setLogDisk(DiskPtr disk) +{ + log_storage = disk; + current_log_storage = std::move(disk); +} + +DiskPtr KeeperContext::getSnapshotDisk() const { return getDisk(snapshot_storage); } +void KeeperContext::setSnapshotDisk(DiskPtr disk) +{ + snapshot_storage = std::move(disk); +} + DiskPtr KeeperContext::getStateFileDisk() const { return getDisk(state_file_storage); } +void KeeperContext::setStateFileDisk(DiskPtr disk) +{ + state_file_storage = std::move(disk); +} + KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { const auto create_local_disk = [](const auto & path) diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index aec2061bddf..4e538c99649 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -34,8 +34,13 @@ public: DiskPtr getCurrentLogDisk() const; DiskPtr getLogDisk() const; - DiskPtr getSnapshotsDisk() const; + void setLogDisk(DiskPtr disk); + + DiskPtr getSnapshotDisk() const; + void setSnapshotDisk(DiskPtr disk); + DiskPtr getStateFileDisk() const; + void setStateFileDisk(DiskPtr disk); private: /// local disk defined using path or disk name using Storage = std::variant; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 6632e58782f..17a15067301 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -336,28 +336,36 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); - server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, snapshot_s3, [this](const KeeperStorage::RequestForSession & request_for_session) - { - /// check if we have queue of read requests depending on this request to be committed - std::lock_guard lock(read_request_queue_mutex); - if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) + server = std::make_unique( + configuration_and_settings, + config, + responses_queue, + snapshots_queue, + standalone_keeper, + snapshot_s3, + [this](const KeeperStorage::RequestForSession & request_for_session) { - auto & xid_to_request_queue = it->second; - - if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); request_queue_it != xid_to_request_queue.end()) + /// check if we have queue of read requests depending on this request to be committed + std::lock_guard lock(read_request_queue_mutex); + if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) { - for (const auto & read_request : request_queue_it->second) - { - if (server->isLeaderAlive()) - server->putLocalReadRequest(read_request); - else - addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); - } + auto & xid_to_request_queue = it->second; - xid_to_request_queue.erase(request_queue_it); + if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); + request_queue_it != xid_to_request_queue.end()) + { + for (const auto & read_request : request_queue_it->second) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } + + xid_to_request_queue.erase(request_queue_it); + } } - } - }); + }); try { diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 4cacf566df6..31c91e2de80 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -108,13 +108,14 @@ KeeperServer::KeeperServer( const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, + bool standalone_keeper, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) , is_recovering(config.getBool("keeper_server.force_recovery", false)) - , keeper_context{std::make_shared(true)} + , keeper_context{std::make_shared(standalone_keeper)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { if (coordination_settings->quorum_reads) diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index db4e9c1962e..63f9cc2bcea 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -72,6 +72,7 @@ public: const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, + bool standalone_keeper, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index bfadf3af9aa..7d808e88b3d 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -676,7 +676,7 @@ SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() DiskPtr KeeperSnapshotManager::getDisk() const { - return keeper_context->getSnapshotsDisk(); + return keeper_context->getSnapshotDisk(); } void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index de5f2da262b..d3783a83bb0 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2,35 +2,37 @@ #include #include "Common/ZooKeeper/IKeeper.h" -#include "Coordination/KeeperContext.h" -#include "Coordination/KeeperStorage.h" #include "Core/Defines.h" -#include "IO/WriteHelpers.h" #include "config.h" #if USE_NURAFT +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include #include #include -#include #include -#include -#include -#include -#include -#include -#include -#include + +#include #include @@ -39,9 +41,7 @@ struct ChangelogDirTest { std::string path; bool drop; - explicit ChangelogDirTest(std::string path_, bool drop_ = true) - : path(path_) - , drop(drop_) + explicit ChangelogDirTest(std::string path_, bool drop_ = true) : path(path_), drop(drop_) { if (fs::exists(path)) { @@ -66,8 +66,20 @@ struct CompressionParam class CoordinationTest : public ::testing::TestWithParam { protected: - DB::KeeperContextPtr keeper_context = std::make_shared(); + DB::KeeperContextPtr keeper_context = std::make_shared(true); Poco::Logger * log{&Poco::Logger::get("CoordinationTest")}; + + void setLogDirectory(const std::string & path) { keeper_context->setLogDisk(std::make_shared("LogDisk", path, 0)); } + + void setSnapshotDirectory(const std::string & path) + { + keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", path, 0)); + } + + void setStateFileDirectory(const std::string & path) + { + keeper_context->setStateFileDisk(std::make_shared("StateFile", path, 0)); + } }; TEST_P(CoordinationTest, BuildTest) @@ -113,13 +125,14 @@ TEST_P(CoordinationTest, BufferSerde) template struct SimpliestRaftServer { - SimpliestRaftServer(int server_id_, const std::string & hostname_, int port_, const std::string & logs_path, const std::string & state_path) + SimpliestRaftServer( + int server_id_, const std::string & hostname_, int port_, DB::KeeperContextPtr keeper_context) : server_id(server_id_) , hostname(hostname_) , port(port_) , endpoint(hostname + ":" + std::to_string(port)) , state_machine(nuraft::cs_new()) - , state_manager(nuraft::cs_new(server_id, hostname, port, logs_path, state_path)) + , state_manager(nuraft::cs_new(server_id, hostname, port, keeper_context)) { state_manager->loadLogStore(1, 0); nuraft::raft_params params; @@ -135,8 +148,13 @@ struct SimpliestRaftServer nuraft::raft_server::init_options opts; opts.start_server_in_constructor_ = false; raft_instance = launcher.init( - state_machine, state_manager, nuraft::cs_new("ToyRaftLogger", DB::LogsLevel::trace), port, - nuraft::asio_service::options{}, params, opts); + state_machine, + state_manager, + nuraft::cs_new("ToyRaftLogger", DB::LogsLevel::trace), + port, + nuraft::asio_service::options{}, + params, + opts); if (!raft_instance) { @@ -195,15 +213,14 @@ nuraft::ptr getBuffer(int64_t number) return ret; } - TEST_P(CoordinationTest, TestSummingRaft1) { ChangelogDirTest test("./logs"); - SummingRaftServer s1(1, "localhost", 44444, "./logs", "./state"); - SCOPE_EXIT( - if (std::filesystem::exists("./state")) - std::filesystem::remove("./state"); - ); + setLogDirectory("./logs"); + setStateFileDirectory("."); + + SummingRaftServer s1(1, "localhost", 44444, keeper_context); + SCOPE_EXIT(if (std::filesystem::exists("./state")) std::filesystem::remove("./state");); /// Single node is leader EXPECT_EQ(s1.raft_instance->get_leader(), 1); @@ -235,8 +252,10 @@ TEST_P(CoordinationTest, ChangelogTestSimple) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello world", 77); changelog.append(entry); @@ -263,7 +282,10 @@ TEST_P(CoordinationTest, ChangelogTestFile) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello world", 77); changelog.append(entry); @@ -292,7 +314,10 @@ TEST_P(CoordinationTest, ChangelogReadWrite) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) @@ -306,7 +331,8 @@ TEST_P(CoordinationTest, ChangelogReadWrite) waitDurableLogs(changelog); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 10); EXPECT_EQ(changelog_reader.last_entry()->get_term(), changelog.last_entry()->get_term()); @@ -326,7 +352,10 @@ TEST_P(CoordinationTest, ChangelogWriteAt) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) { @@ -348,7 +377,8 @@ TEST_P(CoordinationTest, ChangelogWriteAt) EXPECT_EQ(changelog.entry_at(7)->get_term(), 77); EXPECT_EQ(changelog.next_slot(), 8); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), changelog.size()); @@ -362,7 +392,10 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 7; ++i) { @@ -378,7 +411,8 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 7); @@ -395,7 +429,7 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); size_t logs_count = 0; - for (const auto & _ [[maybe_unused]]: fs::directory_iterator("./logs")) + for (const auto & _ [[maybe_unused]] : fs::directory_iterator("./logs")) logs_count++; EXPECT_EQ(logs_count, 2); @@ -412,7 +446,7 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); logs_count = 0; - for (const auto & _ [[maybe_unused]]: fs::directory_iterator("./logs")) + for (const auto & _ [[maybe_unused]] : fs::directory_iterator("./logs")) logs_count++; EXPECT_EQ(logs_count, 3); @@ -440,7 +474,10 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 3; ++i) @@ -489,7 +526,8 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) EXPECT_EQ(changelog.next_slot(), 8); EXPECT_EQ(changelog.last_entry()->get_term(), 60); /// And we able to read it - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_reader.init(7, 0); EXPECT_EQ(changelog_reader.size(), 1); @@ -502,7 +540,10 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) { @@ -517,7 +558,8 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations) auto entries = changelog.pack(1, 5); - DB::KeeperLogStore apply_changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore apply_changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); apply_changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) @@ -549,23 +591,31 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); - changelog.init(1, 0); - for (size_t i = 0; i < 10; ++i) + setLogDirectory("./logs"); + + nuraft::ptr entries; { - auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.append(entry); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + changelog.init(1, 0); + for (size_t i = 0; i < 10; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); + changelog.append(entry); + } + changelog.end_of_append_batch(0, 0); + + EXPECT_EQ(changelog.size(), 10); + + waitDurableLogs(changelog); + + entries = changelog.pack(5, 5); } - changelog.end_of_append_batch(0, 0); - - EXPECT_EQ(changelog.size(), 10); - - waitDurableLogs(changelog); - - auto entries = changelog.pack(5, 5); ChangelogDirTest test1("./logs1"); - DB::KeeperLogStore changelog_new("./logs1", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + setLogDirectory("./logs1"); + DB::KeeperLogStore changelog_new( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog_new.init(1, 0); EXPECT_EQ(changelog_new.size(), 0); @@ -587,7 +637,8 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) EXPECT_EQ(changelog_new.start_index(), 5); EXPECT_EQ(changelog_new.next_slot(), 11); - DB::KeeperLogStore changelog_reader("./logs1", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog_reader.init(5, 0); } @@ -596,7 +647,10 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) @@ -637,7 +691,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); - DB::KeeperLogStore changelog_read("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_read( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_read.init(1, 0); EXPECT_EQ(changelog_read.size(), 7); EXPECT_EQ(changelog_read.start_index(), 1); @@ -649,7 +704,10 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) @@ -690,7 +748,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); - DB::KeeperLogStore changelog_read("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_read( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_read.init(1, 0); EXPECT_EQ(changelog_read.size(), 11); EXPECT_EQ(changelog_read.start_index(), 1); @@ -702,7 +761,10 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) { @@ -747,7 +809,10 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -768,7 +833,8 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin" + params.extension)); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_reader.init(1, 0); auto entry = getLogEntry("36_hello_world", 360); @@ -792,16 +858,16 @@ namespace { void assertBrokenLogRemoved(const fs::path & log_folder, const fs::path & filename) { - EXPECT_FALSE(fs::exists(log_folder / filename)); - // broken logs are sent to the detached/{timestamp} folder - // we don't know timestamp so we iterate all of them - for (const auto & dir_entry : fs::recursive_directory_iterator(log_folder / "detached")) - { - if (dir_entry.path().filename() == filename) - return; - } + EXPECT_FALSE(fs::exists(log_folder / filename)); + // broken logs are sent to the detached/{timestamp} folder + // we don't know timestamp so we iterate all of them + for (const auto & dir_entry : fs::recursive_directory_iterator(log_folder / "detached")) + { + if (dir_entry.path().filename() == filename) + return; + } - FAIL() << "Broken log " << filename << " was not moved to the detached folder"; + FAIL() << "Broken log " << filename << " was not moved to the detached folder"; } } @@ -812,8 +878,10 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) auto params = GetParam(); ChangelogDirTest test(log_folder); + setLogDirectory(log_folder); - DB::KeeperLogStore changelog(log_folder, DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -833,10 +901,12 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); - DB::WriteBufferFromFile plain_buf("./logs/changelog_11_15.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + DB::WriteBufferFromFile plain_buf( + "./logs/changelog_11_15.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(0); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_reader.init(1, 0); changelog_reader.end_of_append_batch(0, 0); @@ -869,7 +939,8 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) assertBrokenLogRemoved(log_folder, "changelog_26_30.bin" + params.extension); assertBrokenLogRemoved(log_folder, "changelog_31_35.bin" + params.extension); - DB::KeeperLogStore changelog_reader2("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_reader2( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_reader2.init(1, 0); EXPECT_EQ(changelog_reader2.size(), 11); EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); @@ -879,8 +950,10 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -894,10 +967,12 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); - DB::WriteBufferFromFile plain_buf("./logs/changelog_1_20.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + DB::WriteBufferFromFile plain_buf( + "./logs/changelog_1_20.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(30); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 0); @@ -912,7 +987,8 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_EQ(changelog_reader.size(), 1); EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); - DB::KeeperLogStore changelog_reader2("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1}); + DB::KeeperLogStore changelog_reader2( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1}, keeper_context); changelog_reader2.init(1, 0); EXPECT_EQ(changelog_reader2.size(), 1); EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); @@ -922,8 +998,10 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -939,7 +1017,8 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) fs::remove("./logs/changelog_1_20.bin" + params.extension); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); /// It should print error message, but still able to start changelog_reader.init(5, 0); assertBrokenLogRemoved("./logs", "changelog_21_40.bin" + params.extension); @@ -949,8 +1028,10 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -970,7 +1051,8 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2) // we have a gap in our logs, we need to remove all the logs after the gap fs::remove("./logs/changelog_21_30.bin" + params.extension); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, keeper_context); /// It should print error message, but still able to start changelog_reader.init(5, 0); EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension)); @@ -1155,7 +1237,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) EXPECT_EQ(world.getApproximateDataSize(), 0); } -void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner=0) +void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0) { using Node = DB::KeeperStorage::Node; Node node{}; @@ -1163,19 +1245,23 @@ void addNode(DB::KeeperStorage & storage, const std::string & path, const std::s node.stat.ephemeralOwner = ephemeral_owner; storage.container.insertOrReplace(path, node); auto child_it = storage.container.find(path); - auto child_path = DB::getBaseName(child_it->key); - storage.container.updateValue(DB::parentPath(StringRef{path}), [&](auto & parent) - { - parent.addChild(child_path); - parent.stat.numChildren++; - }); + auto child_path = DB::getBaseNodeName(child_it->key); + storage.container.updateValue( + DB::parentNodePath(StringRef{path}), + [&](auto & parent) + { + parent.addChild(child_path); + parent.stat.numChildren++; + }); } TEST_P(CoordinationTest, TestStorageSnapshotSimple) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); addNode(storage, "/hello", "world", 1); @@ -1223,7 +1309,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); storage.getSessionID(130); @@ -1264,7 +1352,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); storage.getSessionID(130); @@ -1303,7 +1393,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); for (size_t i = 0; i < 50; ++i) { @@ -1349,14 +1441,15 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) { EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).getData(), "world_" + std::to_string(i)); } - } TEST_P(CoordinationTest, TestStorageSnapshotBroken) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); for (size_t i = 0; i < 50; ++i) { @@ -1370,7 +1463,8 @@ TEST_P(CoordinationTest, TestStorageSnapshotBroken) EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); /// Let's corrupt file - DB::WriteBufferFromFile plain_buf("./snapshots/snapshot_50.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + DB::WriteBufferFromFile plain_buf( + "./snapshots/snapshot_50.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(34); plain_buf.sync(); @@ -1390,25 +1484,35 @@ nuraft::ptr getBufferFromZKRequest(int64_t session_id, int64_t z return buf.getBuffer(); } -nuraft::ptr getLogEntryFromZKRequest(size_t term, int64_t session_id, int64_t zxid, const Coordination::ZooKeeperRequestPtr & request) +nuraft::ptr +getLogEntryFromZKRequest(size_t term, int64_t session_id, int64_t zxid, const Coordination::ZooKeeperRequestPtr & request) { auto buffer = getBufferFromZKRequest(session_id, zxid, request); return nuraft::cs_new(term, buffer); } -void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint64_t total_logs, bool enable_compression, Coordination::KeeperContextPtr keeper_context) +void testLogAndStateMachine( + Coordination::CoordinationSettingsPtr settings, + uint64_t total_logs, + bool enable_compression, + Coordination::KeeperContextPtr keeper_context) { using namespace Coordination; using namespace DB; ChangelogDirTest snapshots("./snapshots"); + keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", "./snapshots", 0)); ChangelogDirTest logs("./logs"); + keeper_context->setLogDisk(std::make_shared("LogDisk", "./logs", 0)); ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{ + .force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}, + keeper_context); changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items); for (size_t i = 1; i < total_logs + 1; ++i) { @@ -1426,7 +1530,8 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint if (i % settings->snapshot_distance == 0) { nuraft::snapshot s(i, 0, std::make_shared()); - nuraft::async_result::handler_type when_done = [&snapshot_created] (bool & ret, nuraft::ptr &/*exception*/) + nuraft::async_result::handler_type when_done + = [&snapshot_created](bool & ret, nuraft::ptr & /*exception*/) { snapshot_created = ret; LOG_INFO(&Poco::Logger::get("CoordinationTest"), "Snapshot finished"); @@ -1444,17 +1549,21 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint } SnapshotsQueue snapshots_queue1{1}; - auto restore_machine = std::make_shared(queue, snapshots_queue1, "./snapshots", settings, keeper_context, nullptr); + auto restore_machine = std::make_shared(queue, snapshots_queue1, settings, keeper_context, nullptr); restore_machine->init(); EXPECT_EQ(restore_machine->last_commit_index(), total_logs - total_logs % settings->snapshot_distance); - DB::KeeperLogStore restore_changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}); + DB::KeeperLogStore restore_changelog( + DB::LogFileSettings{ + .force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}, + keeper_context); restore_changelog.init(restore_machine->last_commit_index() + 1, settings->reserved_log_items); EXPECT_EQ(restore_changelog.size(), std::min(settings->reserved_log_items + total_logs % settings->snapshot_distance, total_logs)); EXPECT_EQ(restore_changelog.next_slot(), total_logs + 1); if (total_logs > settings->reserved_log_items + 1) - EXPECT_EQ(restore_changelog.start_index(), total_logs - total_logs % settings->snapshot_distance - settings->reserved_log_items + 1); + EXPECT_EQ( + restore_changelog.start_index(), total_logs - total_logs % settings->snapshot_distance - settings->reserved_log_items + 1); else EXPECT_EQ(restore_changelog.start_index(), 1); @@ -1552,11 +1661,13 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) using namespace DB; ChangelogDirTest snapshots("./snapshots"); + setSnapshotDirectory("./snapshots"); + CoordinationSettingsPtr settings = std::make_shared(); ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); std::shared_ptr request_c = std::make_shared(); @@ -1585,11 +1696,12 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte using namespace DB; ChangelogDirTest snapshots("./snapshots"); + setSnapshotDirectory("./snapshots"); CoordinationSettingsPtr settings = std::make_shared(); ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); String user_auth_data = "test_user:test_password"; @@ -1635,11 +1747,13 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) using namespace DB; ChangelogDirTest snapshots("./snapshots"); + setSnapshotDirectory("./snapshots"); + CoordinationSettingsPtr settings = std::make_shared(); ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); String user_auth_data = "test_user:test_password"; @@ -1691,8 +1805,10 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) using namespace Coordination; auto params = GetParam(); ChangelogDirTest snapshots("./logs"); + setLogDirectory("./logs"); { - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -1710,7 +1826,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); - DB::KeeperLogStore changelog_1("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}); + DB::KeeperLogStore changelog_1( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, keeper_context); changelog_1.init(0, 50); for (size_t i = 0; i < 55; ++i) { @@ -1726,7 +1843,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); - DB::KeeperLogStore changelog_2("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 7}); + DB::KeeperLogStore changelog_2( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 7}, keeper_context); changelog_2.init(98, 55); for (size_t i = 0; i < 17; ++i) @@ -1749,7 +1867,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) EXPECT_TRUE(fs::exists("./logs/changelog_118_124.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_125_131.bin" + params.extension)); - DB::KeeperLogStore changelog_3("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_3( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_3.init(116, 3); for (size_t i = 0; i < 17; ++i) { @@ -1796,8 +1915,10 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) { using namespace Coordination; auto test_params = GetParam(); - ChangelogDirTest snapshots("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}); + ChangelogDirTest logs("./logs"); + setLogDirectory("./logs"); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -1811,7 +1932,8 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) waitDurableLogs(changelog); - DB::KeeperLogStore changelog1("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog1( + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog1.init(0, 3); for (size_t i = 55; i < 70; ++i) { @@ -1822,7 +1944,8 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) changelog1.end_of_append_batch(0, 0); } - DB::KeeperLogStore changelog2("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog2( + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog2.init(0, 3); for (size_t i = 70; i < 80; ++i) { @@ -1839,7 +1962,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); addNode(storage, "/hello", "world", 1); @@ -1857,7 +1982,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) manager.serializeSnapshotBufferToDisk(*buf, 2); EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension)); - DB::KeeperSnapshotManager new_manager("./snapshots", 3, keeper_context, !params.enable_compression); + DB::KeeperSnapshotManager new_manager(3, keeper_context, !params.enable_compression); auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2); @@ -1883,9 +2008,10 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); { LOG_INFO(log, "================First time====================="); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog(DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -1896,7 +2022,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { LOG_INFO(log, "================Second time====================="); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -1907,7 +2034,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { LOG_INFO(log, "================Third time====================="); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -1918,7 +2046,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { LOG_INFO(log, "================Fourth time====================="); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -1933,10 +2062,12 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); for (size_t i = 0; i < 36; ++i) { LOG_INFO(log, "================First time====================="); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); for (size_t j = 0; j < 7; ++j) { @@ -1947,7 +2078,8 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth) waitDurableLogs(changelog); } - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); EXPECT_EQ(changelog.next_slot(), 36 * 7 + 1); } @@ -1956,9 +2088,11 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); { LOG_INFO(log, "================First time====================="); - DB::KeeperLogStore changelog1("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog1( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog1.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog1.append(entry); @@ -1969,7 +2103,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { LOG_INFO(log, "================Second time====================="); - DB::KeeperLogStore changelog2("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog2( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog2.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog2.append(entry); @@ -1980,7 +2115,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { LOG_INFO(log, "================Third time====================="); - DB::KeeperLogStore changelog3("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog3( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog3.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog3.append(entry); @@ -1991,7 +2127,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { LOG_INFO(log, "================Fourth time====================="); - DB::KeeperLogStore changelog4("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog4( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog4.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog4.append(entry); @@ -2005,10 +2142,12 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); + setSnapshotDirectory("./snapshots"); + std::optional snapshot_hash; for (size_t i = 0; i < 15; ++i) { - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); addNode(storage, "/hello", ""); @@ -2048,7 +2187,10 @@ TEST_P(CoordinationTest, TestLogGap) using namespace Coordination; auto test_params = GetParam(); ChangelogDirTest logs("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -2060,7 +2202,8 @@ TEST_P(CoordinationTest, TestLogGap) changelog.end_of_append_batch(0, 0); } - DB::KeeperLogStore changelog1("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog1( + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog1.init(61, 3); /// Logs discarded @@ -2220,13 +2363,13 @@ TEST_P(CoordinationTest, TestListRequestTypes) return create_response.path_created; }; - create_path(parentPath(StringRef{test_path}).toString(), false, false); + create_path(parentNodePath(StringRef{test_path}).toString(), false, false); static constexpr size_t persistent_num = 5; std::unordered_set expected_persistent_children; for (size_t i = 0; i < persistent_num; ++i) { - expected_persistent_children.insert(getBaseName(create_path(test_path, false)).toString()); + expected_persistent_children.insert(getBaseNodeName(create_path(test_path, false)).toString()); } ASSERT_EQ(expected_persistent_children.size(), persistent_num); @@ -2234,7 +2377,7 @@ TEST_P(CoordinationTest, TestListRequestTypes) std::unordered_set expected_ephemeral_children; for (size_t i = 0; i < ephemeral_num; ++i) { - expected_ephemeral_children.insert(getBaseName(create_path(test_path, true)).toString()); + expected_ephemeral_children.insert(getBaseNodeName(create_path(test_path, true)).toString()); } ASSERT_EQ(expected_ephemeral_children.size(), ephemeral_num); @@ -2242,7 +2385,7 @@ TEST_P(CoordinationTest, TestListRequestTypes) { const auto list_request = std::make_shared(); int new_zxid = ++zxid; - list_request->path = parentPath(StringRef{test_path}).toString(); + list_request->path = parentNodePath(StringRef{test_path}).toString(); list_request->list_request_type = list_request_type; storage.preprocessRequest(list_request, 1, 0, new_zxid); auto responses = storage.processRequest(list_request, 1, new_zxid); @@ -2270,20 +2413,23 @@ TEST_P(CoordinationTest, TestListRequestTypes) EXPECT_EQ(all_children.size(), ephemeral_num + persistent_num); for (const auto & child : all_children) { - EXPECT_TRUE(expected_ephemeral_children.contains(child) || expected_persistent_children.contains(child)) << "Missing child " << child; + EXPECT_TRUE(expected_ephemeral_children.contains(child) || expected_persistent_children.contains(child)) + << "Missing child " << child; } } TEST_P(CoordinationTest, TestDurableState) { ChangelogDirTest logs("./logs"); + setLogDirectory("./logs"); + setStateFileDirectory("."); auto state = nuraft::cs_new(); std::optional state_manager; const auto reload_state_manager = [&] { - state_manager.emplace(1, "localhost", 9181, "./logs", "./state"); + state_manager.emplace(1, "localhost", 9181, keeper_context); state_manager->loadLogStore(1, 0); }; @@ -2318,11 +2464,11 @@ TEST_P(CoordinationTest, TestDurableState) write_buf.sync(); write_buf.close(); reload_state_manager(); -#ifdef NDEBUG +# ifdef NDEBUG ASSERT_EQ(state_manager->read_state(), nullptr); -#else +# else ASSERT_THROW(state_manager->read_state(), DB::Exception); -#endif +# endif } { @@ -2396,15 +2542,16 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); uint64_t last_entry_index{0}; size_t i{0}; { SCOPED_TRACE("Small rotation interval, big size limit"); DB::KeeperLogStore changelog( - "./logs", DB::LogFileSettings{ - .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20, .max_size = 50 * 1024 * 1024}); + .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20, .max_size = 50 * 1024 * 1024}, + keeper_context); changelog.init(1, 0); for (; i < 100; ++i) @@ -2421,9 +2568,9 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) { SCOPED_TRACE("Large rotation interval, small size limit"); DB::KeeperLogStore changelog( - "./logs", DB::LogFileSettings{ - .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}); + .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}, + keeper_context); changelog.init(1, 0); ASSERT_EQ(changelog.entry_at(last_entry_index)->get_term(), (i - 1 + 44) * 10); @@ -2442,13 +2589,12 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) { SCOPED_TRACE("Final verify all logs"); DB::KeeperLogStore changelog( - "./logs", DB::LogFileSettings{ - .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}); + .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}, + keeper_context); changelog.init(1, 0); ASSERT_EQ(changelog.entry_at(last_entry_index)->get_term(), (i - 1 + 44) * 10); } - } TEST_P(CoordinationTest, TestCheckNotExistsRequest) @@ -2524,13 +2670,10 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest) } } -INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, +INSTANTIATE_TEST_SUITE_P( + CoordinationTestSuite, CoordinationTest, - ::testing::ValuesIn(std::initializer_list{ - CompressionParam{true, ".zstd"}, - CompressionParam{false, ""} - }) -); + ::testing::ValuesIn(std::initializer_list{CompressionParam{true, ".zstd"}, CompressionParam{false, ""}})); int main(int argc, char ** argv) { From 91eb3ad2bca29ec44d1d996740bca4cd026406c2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 May 2023 12:14:15 +0200 Subject: [PATCH 0595/2223] fix clang-tidy build --- src/Disks/IDiskTransaction.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/IDiskTransaction.h b/src/Disks/IDiskTransaction.h index f0c32e04f48..935cd6b2c65 100644 --- a/src/Disks/IDiskTransaction.h +++ b/src/Disks/IDiskTransaction.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { From 486153d581a6b23f412e10caf486578bfe16da27 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 24 May 2023 13:33:28 +0300 Subject: [PATCH 0596/2223] Update MergeTreeData.cpp --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 440c91e3082..ee2ec8867bf 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1081,7 +1081,7 @@ void MergeTreeData::PartLoadingTree::add(const MergeTreePartInfo & info, const S else if (!prev_info.isDisjoint(info)) { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Part {} intersects previous part {}. It is a bug!", + "Part {} intersects previous part {}. It is a bug or a result of manual intervention in the server or ZooKeeper data", name, prev->second->name); } } @@ -1098,7 +1098,7 @@ void MergeTreeData::PartLoadingTree::add(const MergeTreePartInfo & info, const S else if (!next_info.isDisjoint(info)) { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Part {} intersects next part {}. It is a bug!", + "Part {} intersects next part {}. It is a bug or a result of manual intervention in the server or ZooKeeper data", name, it->second->name); } } From 4935b181604b718420fd05e1a3a0a00e6781f963 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 11:07:10 +0000 Subject: [PATCH 0597/2223] Add 02763_last_day_of_week test draft --- .../0_stateless/02763_last_day_of_week.reference | 10 ++++++++++ tests/queries/0_stateless/02763_last_day_of_week.sql | 9 +++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02763_last_day_of_week.reference create mode 100644 tests/queries/0_stateless/02763_last_day_of_week.sql diff --git a/tests/queries/0_stateless/02763_last_day_of_week.reference b/tests/queries/0_stateless/02763_last_day_of_week.reference new file mode 100644 index 00000000000..9c136d856f6 --- /dev/null +++ b/tests/queries/0_stateless/02763_last_day_of_week.reference @@ -0,0 +1,10 @@ +2018-12-25 2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 +2018-12-26 2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 +2018-12-27 2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 +2018-12-28 2018-12-28 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 +2018-12-29 2018-12-29 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 +2018-12-30 2018-12-30 00:00:00 2019-01-05 2019-01-05 2018-12-30 2018-12-30 +2018-12-31 2018-12-31 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 +2019-01-01 2019-01-01 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 +2019-01-02 2019-01-02 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 +2019-01-03 2019-01-03 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 diff --git a/tests/queries/0_stateless/02763_last_day_of_week.sql b/tests/queries/0_stateless/02763_last_day_of_week.sql new file mode 100644 index 00000000000..600cebd2636 --- /dev/null +++ b/tests/queries/0_stateless/02763_last_day_of_week.sql @@ -0,0 +1,9 @@ +SELECT + toDate('2018-12-25') + number AS x, + toDateTime(x) AS x_t, + toLastDayOfWeek(x) AS w0, + toLastDayOfWeek(x_t) AS wt0, + toLastDayOfWeek(x, 3) AS w3, + toLastDayOfWeek(x_t, 3) AS wt3 +FROM numbers(10); + From 3bce2bbc37a070dc967ef071a510564b3a5066b7 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 11:40:41 +0000 Subject: [PATCH 0598/2223] Fix settings.md in regard to enable_extended_results_for_datetime_functions --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cddde2090f8..0147ff6c377 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3791,8 +3791,8 @@ Result: ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Enables or disables returning results of type: -- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md/#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md/#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md/#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md/#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md/#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md/#tomonday) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md/#tolastdayofmonth). -- `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md/#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md/#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md/#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md/#timeslot). +- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). +- `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). Possible values: From 99a7967f11050b1cc4e3a61365ff829b6084012f Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 11:58:39 +0000 Subject: [PATCH 0599/2223] Amend the dicumentation with regard to toLastDayOfWeek --- docs/en/operations/settings/settings.md | 2 +- .../functions/date-time-functions.md | 20 ++++++++++++++++--- docs/ru/operations/settings/settings.md | 2 +- .../functions/date-time-functions.md | 20 +++++++++++++------ 4 files changed, 33 insertions(+), 11 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 0147ff6c377..52ae82b8f36 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3791,7 +3791,7 @@ Result: ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Enables or disables returning results of type: -- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). +- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). - `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). Possible values: diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4710a5617c3..42b25de7f61 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -389,12 +389,14 @@ Result: ``` :::note -The return type of `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` functions described below is determined by the configuration parameter [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) which is `0` by default. +The return type of `toStartOf*`, `toLastDayOf*`, `toMonday`, `timeSlot` functions described below is determined by the configuration parameter [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) which is `0` by default. Behavior for -* `enable_extended_results_for_datetime_functions = 0`: Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime`. Functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` return `DateTime`. Though these functions can take values of the extended types `Date32` and `DateTime64` as an argument, passing them a time outside the normal range (year 1970 to 2149 for `Date` / 2106 for `DateTime`) will produce wrong results. +* `enable_extended_results_for_datetime_functions = 0`: + * Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime`. + * Functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` return `DateTime`. Though these functions can take values of the extended types `Date32` and `DateTime64` as an argument, passing them a time outside the normal range (year 1970 to 2149 for `Date` / 2106 for `DateTime`) will produce wrong results. * `enable_extended_results_for_datetime_functions = 1`: - * Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime` if their argument is a `Date` or `DateTime`, and they return `Date32` or `DateTime64` if their argument is a `Date32` or `DateTime64`. + * Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime` if their argument is a `Date` or `DateTime`, and they return `Date32` or `DateTime64` if their argument is a `Date32` or `DateTime64`. * Functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` return `DateTime` if their argument is a `Date` or `DateTime`, and they return `DateTime64` if their argument is a `Date32` or `DateTime64`. ::: @@ -449,6 +451,18 @@ The mode argument works exactly like the mode argument in function `toWeek()`. I toStartOfWeek(t[, mode[, timezone]]) ``` +## toLastDayOfWeek + +Rounds a date or date with time up to the nearest Saturday or Sunday. +Returns the date. +The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. + +**Syntax** + +``` sql +toLastDayOfWeek(t[, mode[, timezone]]) +``` + ## toStartOfDay Rounds down a date with time to the start of the day. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 066e71c25a5..65b75ff3d74 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3800,7 +3800,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Включает или отключает возвращение результатов типа: -- `Date32` с расширенным диапазоном (по сравнению с типом `Date`) для функций [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday) и [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). +- `Date32` с расширенным диапазоном (по сравнению с типом `Date`) для функций [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) и [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). - `DateTime64` с расширенным диапазоном (по сравнению с типом `DateTime`) для функций [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) и [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). Возможные значения: diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index a7e8a478edb..ee86f0d3d13 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -268,13 +268,15 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; ``` :::note -Тип возвращаемого значения описанными далее функциями `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` определяется конфигурационным параметром [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) имеющим по умолчанию значение `0`. +Тип возвращаемого значения описанными далее функциями `toStartOf*`, `toLastDayOf*`, `toMonday`, `timeSlot` определяется конфигурационным параметром [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) имеющим по умолчанию значение `0`. Поведение для -* `enable_extended_results_for_datetime_functions = 0`: Функции `toStartOf*`, `toLastDayOfMonth`, `toMonday` возвращают `Date` или `DateTime`. Функции `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` возвращают `DateTime`. Хотя эти функции могут принимать значения типа `Date32` или `DateTime64` в качестве аргумента, при обработке аргумента вне нормального диапазона значений (`1970` - `2148` для `Date` и `1970-01-01 00:00:00`-`2106-02-07 08:28:15` для `DateTime`) будет получен некорректный результат. +* `enable_extended_results_for_datetime_functions = 0`: + * Функции `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfWeek`, `toLastDayOfMonth`, `toMonday` возвращают `Date` или `DateTime`. + * Функции `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` возвращают `DateTime`. Хотя эти функции могут принимать значения расширенных типов `Date32` и `DateTime64` в качестве аргумента, при обработке аргумента вне нормального диапазона значений (`1970` - `2148` для `Date` и `1970-01-01 00:00:00`-`2106-02-07 08:28:15` для `DateTime`) будет получен некорректный результат. * `enable_extended_results_for_datetime_functions = 1`: - * Функции `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfMonth`, `toMonday` возвращают `Date` или `DateTime` если их аргумент `Date` или `DateTime` и они возвращают `Date32` или `DateTime64` если их аргумент `Date32` или `DateTime64`. - * Функции `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` возвращают `DateTime` если их аргумент `Date` или `DateTime` и они возвращают `DateTime64` если их аргумент `Date32` или `DateTime64`. + * Функции `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfWeek`, `toLastDayOfMonth`, `toMonday` возвращают `Date` или `DateTime` если их аргумент `Date` или `DateTime` и они возвращают `Date32` или `DateTime64` если их аргумент `Date32` или `DateTime64`. + * Функции `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` возвращают `DateTime`, если их аргумент имеет тип `Date` или `DateTime`, и `DateTime64` если их аргумент имеет тип `Date32` или `DateTime64`. ::: ## toStartOfYear {#tostartofyear} @@ -324,9 +326,15 @@ SELECT toStartOfISOYear(toDate('2017-01-01')) AS ISOYear20170101; Округляет дату или дату-с-временем вниз до ближайшего понедельника. Возвращается дата. -## toStartOfWeek(t[,mode]) {#tostartofweek} +## toStartOfWeek(t[, mode[, timezone]]) -Округляет дату или дату со временем до ближайшего воскресенья или понедельника в соответствии с mode. +Округляет дату или дату-с-временем назад, до ближайшего воскресенья или понедельника, в соответствии с mode. +Возвращается дата. +Аргумент mode работает точно так же, как аргумент mode [toWeek()](#toweek). Если аргумент mode опущен, то используется режим 0. + +## toLastDayOfWeek(t[, mode[, timezone]]) + +Округляет дату или дату-с-временем вперёд, до ближайшей субботы или воскресенья, в соответствии с mode. Возвращается дата. Аргумент mode работает точно так же, как аргумент mode [toWeek()](#toweek). Если аргумент mode опущен, то используется режим 0. From 8bc25c4ea3a1359affc36599bcc982b741ea5360 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 24 May 2023 14:40:27 +0200 Subject: [PATCH 0600/2223] Fix style --- tests/integration/test_reload_clusters_config/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_reload_clusters_config/test.py b/tests/integration/test_reload_clusters_config/test.py index 73ca4a01f34..cb003bbe04e 100644 --- a/tests/integration/test_reload_clusters_config/test.py +++ b/tests/integration/test_reload_clusters_config/test.py @@ -169,7 +169,9 @@ test_config3 = """ def send_repeated_query(table, count=5): for i in range(count): node.query_and_get_error( - "SELECT count() FROM {} SETTINGS receive_timeout=1, handshake_timeout_ms=1".format(table) + "SELECT count() FROM {} SETTINGS receive_timeout=1, handshake_timeout_ms=1".format( + table + ) ) From 2255b0287a7c0e83d1a748f4c716096ac2673581 Mon Sep 17 00:00:00 2001 From: helifu Date: Mon, 8 May 2023 18:01:24 +0800 Subject: [PATCH 0601/2223] Add 'partitions' field for system.query_log --- src/Interpreters/Context.cpp | 14 ++++++++++++++ src/Interpreters/Context.h | 4 ++++ src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Interpreters/QueryLog.cpp | 3 +++ src/Interpreters/QueryLog.h | 1 + src/Interpreters/executeQuery.cpp | 2 ++ src/Processors/QueryPlan/ReadFromMergeTree.cpp | 12 ++++++++++++ src/Storages/SelectQueryInfo.h | 1 + 8 files changed, 38 insertions(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5ec12d7e0ea..2fb065600fc 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1386,6 +1386,20 @@ void Context::addQueryAccessInfo( query_access_info.views.emplace(view_name); } +void Context::addQueryAccessInfo(const Names & partition_names) +{ + if (isGlobalContext()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); + } + + std::lock_guard lock(query_access_info.mutex); + for (const auto & partition_name : partition_names) + { + query_access_info.partitions.emplace(partition_name); + } +} + void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const { if (isGlobalContext()) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3862984bb6f..fcf035cefca 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -295,6 +295,7 @@ private: databases = rhs.databases; tables = rhs.tables; columns = rhs.columns; + partitions = rhs.partitions; projections = rhs.projections; views = rhs.views; } @@ -312,6 +313,7 @@ private: std::swap(databases, rhs.databases); std::swap(tables, rhs.tables); std::swap(columns, rhs.columns); + std::swap(partitions, rhs.partitions); std::swap(projections, rhs.projections); std::swap(views, rhs.views); } @@ -321,6 +323,7 @@ private: std::set databases{}; std::set tables{}; std::set columns{}; + std::set partitions{}; std::set projections{}; std::set views{}; }; @@ -629,6 +632,7 @@ public: const Names & column_names, const String & projection_name = {}, const String & view_name = {}); + void addQueryAccessInfo(const Names & partition_names); /// Supported factories for records in query_log diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d68f9c8e6a6..c410ec998f5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -385,6 +385,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.ignore_projections = options.ignore_projections; query_info.is_projection_query = options.is_projection_query; + query_info.is_internal = options.is_internal; initSettings(); const Settings & settings = context->getSettingsRef(); diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 4746954edf2..ec0315c2f95 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -70,6 +70,7 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"databases", array_low_cardinality_string}, {"tables", array_low_cardinality_string}, {"columns", array_low_cardinality_string}, + {"partitions", array_low_cardinality_string}, {"projections", array_low_cardinality_string}, {"views", array_low_cardinality_string}, {"exception_code", std::make_shared()}, @@ -176,6 +177,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const auto & column_databases = typeid_cast(*columns[i++]); auto & column_tables = typeid_cast(*columns[i++]); auto & column_columns = typeid_cast(*columns[i++]); + auto & column_partitions = typeid_cast(*columns[i++]); auto & column_projections = typeid_cast(*columns[i++]); auto & column_views = typeid_cast(*columns[i++]); @@ -194,6 +196,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const fill_column(query_databases, column_databases); fill_column(query_tables, column_tables); fill_column(query_columns, column_columns); + fill_column(query_partitions, column_partitions); fill_column(query_projections, column_projections); fill_column(query_views, column_views); } diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 44780f530e0..570d1297239 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -65,6 +65,7 @@ struct QueryLogElement std::set query_databases; std::set query_tables; std::set query_columns; + std::set query_partitions; std::set query_projections; std::set query_views; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 9a2750f399c..7ee28fce665 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -837,6 +837,7 @@ static std::tuple executeQueryImpl( elem.query_databases = info.databases; elem.query_tables = info.tables; elem.query_columns = info.columns; + elem.query_partitions = info.partitions; elem.query_projections = info.projections; elem.query_views = info.views; } @@ -901,6 +902,7 @@ static std::tuple executeQueryImpl( element.query_databases.insert(access_info.databases.begin(), access_info.databases.end()); element.query_tables.insert(access_info.tables.begin(), access_info.tables.end()); element.query_columns.insert(access_info.columns.begin(), access_info.columns.end()); + element.query_partitions.insert(access_info.partitions.begin(), access_info.partitions.end()); element.query_projections.insert(access_info.projections.begin(), access_info.projections.end()); element.query_views.insert(access_info.views.begin(), access_info.views.end()); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 1c5b625656c..6180a01dec6 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1618,6 +1618,18 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons result.selected_marks, result.selected_ranges); + // Adding partition info to QueryAccessInfo. + if (context->hasQueryContext() && !query_info.is_internal) + { + Names partition_names; + for (const auto & part : result.parts_with_ranges) + { + partition_names.emplace_back( + fmt::format("{}.{}", data.getStorageID().getFullNameNotQuoted(), part.data_part->info.partition_id)); + } + context->getQueryContext()->addQueryAccessInfo(partition_names); + } + ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts); ProfileEvents::increment(ProfileEvents::SelectedRanges, result.selected_ranges); ProfileEvents::increment(ProfileEvents::SelectedMarks, result.selected_marks); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index fb895d04b8f..b3dfd44b2ad 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -251,6 +251,7 @@ struct SelectQueryInfo bool is_projection_query = false; bool merge_tree_empty_result = false; bool settings_limit_offset_done = false; + bool is_internal = false; Block minmax_count_projection_block; MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr; From e138289fbf12ba0cbc944773f74e3a943c7c1961 Mon Sep 17 00:00:00 2001 From: helifu Date: Tue, 16 May 2023 20:09:49 +0800 Subject: [PATCH 0602/2223] Test the 'partitions' field of system.query_log --- .../02751_query_log_test_partitions.reference | 2 ++ .../02751_query_log_test_partitions.sql | 20 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02751_query_log_test_partitions.reference create mode 100644 tests/queries/0_stateless/02751_query_log_test_partitions.sql diff --git a/tests/queries/0_stateless/02751_query_log_test_partitions.reference b/tests/queries/0_stateless/02751_query_log_test_partitions.reference new file mode 100644 index 00000000000..8419e7f9786 --- /dev/null +++ b/tests/queries/0_stateless/02751_query_log_test_partitions.reference @@ -0,0 +1,2 @@ +3 3 +02751_query_log_test_partitions.3 \ No newline at end of file diff --git a/tests/queries/0_stateless/02751_query_log_test_partitions.sql b/tests/queries/0_stateless/02751_query_log_test_partitions.sql new file mode 100644 index 00000000000..f30b0ee1492 --- /dev/null +++ b/tests/queries/0_stateless/02751_query_log_test_partitions.sql @@ -0,0 +1,20 @@ +set log_queries=1; +set log_queries_min_type='QUERY_FINISH'; + +DROP TABLE IF EXISTS 02751_query_log_test_partitions; +CREATE TABLE 02751_query_log_test_partitions (a Int64, b Int64) ENGINE = MergeTree PARTITION BY a ORDER BY b; + +SYSTEM STOP MERGES 02751_query_log_test_partitions; +INSERT INTO 02751_query_log_test_partitions SELECT number, number FROM numbers(5); + +SELECT * FROM 02751_query_log_test_partitions WHERE a = 3; +SYSTEM FLUSH LOGS; + +SELECT + --Remove the prefix string which is a mutable database name. + arrayStringConcat(arrayPopFront(splitByString('.', partitions[1])), '.') +FROM + system.query_log +WHERE + current_database=currentDatabase() and + query = 'SELECT * FROM 02751_query_log_test_partitions WHERE a = 3;' From 880745453d29e4b5f8d641ccc614b8f8aaf79575 Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 17 May 2023 10:09:04 +0800 Subject: [PATCH 0603/2223] small update for test case --- tests/queries/0_stateless/02751_query_log_test_partitions.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02751_query_log_test_partitions.sql b/tests/queries/0_stateless/02751_query_log_test_partitions.sql index f30b0ee1492..be047d1a46e 100644 --- a/tests/queries/0_stateless/02751_query_log_test_partitions.sql +++ b/tests/queries/0_stateless/02751_query_log_test_partitions.sql @@ -4,10 +4,10 @@ set log_queries_min_type='QUERY_FINISH'; DROP TABLE IF EXISTS 02751_query_log_test_partitions; CREATE TABLE 02751_query_log_test_partitions (a Int64, b Int64) ENGINE = MergeTree PARTITION BY a ORDER BY b; -SYSTEM STOP MERGES 02751_query_log_test_partitions; -INSERT INTO 02751_query_log_test_partitions SELECT number, number FROM numbers(5); +INSERT INTO 02751_query_log_test_partitions SELECT number, number FROM numbers(10); SELECT * FROM 02751_query_log_test_partitions WHERE a = 3; + SYSTEM FLUSH LOGS; SELECT From 4724745b4cbcc6022f796e9b3035c79581d21f76 Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 17 May 2023 16:15:22 +0800 Subject: [PATCH 0604/2223] Add the 'partitions' field description in docs --- docs/en/operations/system-tables/query_log.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 1bcecfeb161..71e1452cef1 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -59,9 +59,10 @@ Columns: - `query_kind` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Type of the query. - `databases` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the databases present in the query. - `tables` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the tables present in the query. -- `views` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the (materialized or live) views present in the query. - `columns` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the columns present in the query. +- `partitions` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the partitions present in the query. - `projections` ([String](../../sql-reference/data-types/string.md)) — Names of the projections used during the query execution. +- `views` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — Names of the (materialized or live) views present in the query. - `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — Code of an exception. - `exception` ([String](../../sql-reference/data-types/string.md)) — Exception message. - `stack_trace` ([String](../../sql-reference/data-types/string.md)) — [Stack trace](https://en.wikipedia.org/wiki/Stack_trace). An empty string, if the query was completed successfully. From 07eedc8ef18c829b884f4fbdd579ed00822a30a8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 24 May 2023 11:32:04 +0200 Subject: [PATCH 0605/2223] Fix --- contrib/libpqxx | 2 +- src/Processors/Sources/PostgreSQLSource.cpp | 8 +++----- tests/integration/test_storage_postgresql/test.py | 10 ++++++++++ 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/contrib/libpqxx b/contrib/libpqxx index a4e83483927..bdd6540fb95 160000 --- a/contrib/libpqxx +++ b/contrib/libpqxx @@ -1 +1 @@ -Subproject commit a4e834839270a8c1f7ff1db351ba85afced3f0e2 +Subproject commit bdd6540fb95ff56c813691ceb5da5a3266cf235d diff --git a/src/Processors/Sources/PostgreSQLSource.cpp b/src/Processors/Sources/PostgreSQLSource.cpp index 77c2fc41aa1..115e24d5740 100644 --- a/src/Processors/Sources/PostgreSQLSource.cpp +++ b/src/Processors/Sources/PostgreSQLSource.cpp @@ -176,12 +176,10 @@ template void PostgreSQLSource::onFinish() { if (stream) - { - stream->complete(); + stream->close(); - if (auto_commit) - tx->commit(); - } + if (tx && auto_commit) + tx->commit(); } template diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 3b7aae1ccdc..7ca10f4466e 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -62,6 +62,16 @@ def test_postgres_select_insert(started_cluster): # for i in range(1, 1000): # assert (node1.query(check1)).rstrip() == '10000', f"Failed on {i}" + result = node1.query( + f""" + INSERT INTO TABLE FUNCTION {table} + SELECT number, concat('name_', toString(number)), 3 from numbers(1000000)""" + ) + check1 = f"SELECT count() FROM {table}" + check2 = f"SELECT count() FROM (SELECT * FROM {table} LIMIT 10)" + assert (node1.query(check1)).rstrip() == "1010000" + assert (node1.query(check2)).rstrip() == "10" + cursor.execute(f"DROP TABLE {table_name} ") From 62208feaf3149e3af94ec25f6887b9e58d524f91 Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 24 May 2023 21:08:25 +0800 Subject: [PATCH 0606/2223] Add a new line to the reference file --- .../0_stateless/02751_query_log_test_partitions.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02751_query_log_test_partitions.reference b/tests/queries/0_stateless/02751_query_log_test_partitions.reference index 8419e7f9786..5a9f2163c0e 100644 --- a/tests/queries/0_stateless/02751_query_log_test_partitions.reference +++ b/tests/queries/0_stateless/02751_query_log_test_partitions.reference @@ -1,2 +1,2 @@ 3 3 -02751_query_log_test_partitions.3 \ No newline at end of file +02751_query_log_test_partitions.3 From f76f989b53645136a5e83a9a1a9ab1335e9a2cbf Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 24 May 2023 13:33:05 +0000 Subject: [PATCH 0607/2223] Implement a uniform way to query processor core IDs --- src/Common/AsynchronousMetrics.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index ac2180103c5..3753aaca405 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -1041,18 +1041,16 @@ void AsynchronousMetrics::update(TimePoint update_time) // It doesn't read the EOL itself. ++cpuinfo->position(); - if (s.rfind("processor", 0) == 0) + static constexpr std::string_view PROCESSOR = "processor"; + if (s.starts_with(PROCESSOR)) { /// s390x example: processor 0: version = FF, identification = 039C88, machine = 3906 /// non s390x example: processor : 0 - if (auto colon = s.find_first_of(':')) - { -#ifdef __s390x__ - core_id = std::stoi(s.substr(10)); /// 10: length of "processor" plus 1 -#else - core_id = std::stoi(s.substr(colon + 2)); -#endif - } + auto core_id_start = std::ssize(PROCESSOR); + while (core_id_start < std::ssize(s) && !std::isdigit(s[core_id_start])) + ++core_id_start; + + core_id = std::stoi(s.substr(core_id_start)); } else if (s.rfind("cpu MHz", 0) == 0) { From 8a00be69b326e8c2eb0abf68e538ed75112e762a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 24 May 2023 10:40:33 -0300 Subject: [PATCH 0608/2223] Update index.md --- docs/en/sql-reference/dictionaries/index.md | 24 ++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index f230cbae100..65f1a81f849 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -865,16 +865,34 @@ LIFETIME(3600); The key must have only one `String` type attribute that contains an allowed IP prefix. Other types are not supported yet. -For queries, you must use the same functions (`dictGetT` with a tuple) as for dictionaries with composite keys. The syntax is: +The syntax is: ``` sql -dictGetT('dict_name', 'attr_name', tuple(ip)) +dictGetT('dict_name', 'attr_name', ip) ``` The function takes either `UInt32` for IPv4, or `FixedString(16)` for IPv6. For example: ``` sql -select dictGet('my_ip_trie_dictionary', 'asn', tuple(IPv6StringToNum('2001:db8::1'))) +SELECT dictGet('my_ip_trie_dictionary', 'cca2', toIPv4('202.79.32.10')) AS result; + +┌─result─┐ +│ NP │ +└────────┘ + + +SELECT dictGet('my_ip_trie_dictionary', 'asn', IPv6StringToNum('2001:db8::1')) AS result; + +┌─result─┐ +│ 65536 │ +└────────┘ + + +SELECT dictGet('my_ip_trie_dictionary', ('asn', 'cca2'), IPv6StringToNum('2001:db8::1')) AS result; + +┌─result───────┐ +│ (65536,'ZZ') │ +└──────────────┘ ``` Other types are not supported yet. The function returns the attribute for the prefix that corresponds to this IP address. If there are overlapping prefixes, the most specific one is returned. From 092cf99147293a8dbb45031cf361a03cbcf5777f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 May 2023 09:46:07 +0000 Subject: [PATCH 0609/2223] Fix build --- programs/keeper/CMakeLists.txt | 2 +- src/Coordination/Changelog.cpp | 134 ++++++++++++------ src/Coordination/KeeperContext.cpp | 20 +++ src/Coordination/KeeperContext.h | 3 + src/Coordination/KeeperSnapshotManager.cpp | 1 - .../MetadataStorageFromPlainObjectStorage.cpp | 1 - utils/keeper-data-dumper/main.cpp | 10 +- 7 files changed, 118 insertions(+), 53 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index a946ea06626..6e97ab324e3 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -143,9 +143,9 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadBufferFromRemoteFSGather.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/IOUringReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferFromTemporaryFile.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferWithFinalizeCallback.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/getThreadPoolReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolRemoteFSReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolReader.cpp diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index c94633d6dbd..875b0758d27 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -24,63 +24,66 @@ namespace ErrorCodes extern const int CHECKSUM_DOESNT_MATCH; extern const int CORRUPTED_DATA; extern const int UNKNOWN_FORMAT_VERSION; + extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; } namespace { - void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr description, DiskPtr disk_to, const std::string & path_to) - { - disk_from->copyFile(description->path, *disk_to, path_to, {}); - disk_from->removeFile(description->path); - description->path = path_to; - description->disk = disk_to; - } - constexpr auto DEFAULT_PREFIX = "changelog"; +void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr description, DiskPtr disk_to, const std::string & path_to) +{ + disk_from->copyFile(description->path, *disk_to, path_to, {}); + disk_from->removeFile(description->path); + description->path = path_to; + description->disk = disk_to; +} - inline std::string - formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) - { - return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension); - } +constexpr auto DEFAULT_PREFIX = "changelog"; - ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path) - { - // we can have .bin.zstd so we cannot use std::filesystem stem and extension - std::string filename_with_extension = path.filename(); - std::string_view filename_with_extension_view = filename_with_extension; +inline std::string +formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) +{ + return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension); +} - auto first_dot = filename_with_extension.find('.'); - if (first_dot == std::string::npos) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid changelog file {}", path.generic_string()); +ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path) +{ + // we can have .bin.zstd so we cannot use std::filesystem stem and extension + std::string filename_with_extension = path.filename(); + std::string_view filename_with_extension_view = filename_with_extension; - Strings filename_parts; - boost::split(filename_parts, filename_with_extension_view.substr(0, first_dot), boost::is_any_of("_")); - if (filename_parts.size() < 3) - throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path.generic_string()); + auto first_dot = filename_with_extension.find('.'); + if (first_dot == std::string::npos) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid changelog file {}", path.generic_string()); - auto result = std::make_shared(); - result->prefix = filename_parts[0]; - result->from_log_index = parse(filename_parts[1]); - result->to_log_index = parse(filename_parts[2]); - result->extension = std::string(filename_with_extension.substr(first_dot + 1)); - result->path = path.generic_string(); - return result; - } + Strings filename_parts; + boost::split(filename_parts, filename_with_extension_view.substr(0, first_dot), boost::is_any_of("_")); + if (filename_parts.size() < 3) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path.generic_string()); - Checksum computeRecordChecksum(const ChangelogRecord & record) - { - SipHash hash; - hash.update(record.header.version); - hash.update(record.header.index); - hash.update(record.header.term); - hash.update(record.header.value_type); - hash.update(record.header.blob_size); - if (record.header.blob_size != 0) - hash.update(reinterpret_cast(record.blob->data_begin()), record.blob->size()); - return hash.get64(); - } + auto result = std::make_shared(); + result->prefix = filename_parts[0]; + result->from_log_index = parse(filename_parts[1]); + result->to_log_index = parse(filename_parts[2]); + result->extension = std::string(filename_with_extension.substr(first_dot + 1)); + result->path = path.generic_string(); + return result; +} + +Checksum computeRecordChecksum(const ChangelogRecord & record) +{ + SipHash hash; + hash.update(record.header.version); + hash.update(record.header.index); + hash.update(record.header.term); + hash.update(record.header.value_type); + hash.update(record.header.blob_size); + if (record.header.blob_size != 0) + hash.update(reinterpret_cast(record.blob->data_begin()), record.blob->size()); + return hash.get64(); +} } @@ -571,6 +574,19 @@ Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, Kee , append_completion_queue(std::numeric_limits::max()) , keeper_context(std::move(keeper_context_)) { + if (auto current_log_disk = getCurrentLogDisk(); + log_file_settings.force_sync && dynamic_cast(current_log_disk.get()) == nullptr) + { + throw DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, + "force_sync is set to true for logs but disk '{}' cannot satisfy such guarantee because it's not of type DiskLocal.\n" + "If you want to use force_sync and same disk for all logs, please set keeper_server.log_storage_disk to a local disk.\n" + "If you want to use force_sync and different disk only for old logs, please set 'keeper_server.log_storage_disk' to any " + "supported disk and 'keeper_server.current_log_storage_disk' to a local disk.\n" + "Otherwise, disable force_sync", + current_log_disk->getName()); + } + /// Load all files on changelog disks const auto load_from_disk = [&](const auto & disk) @@ -590,6 +606,12 @@ Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, Kee } }; + /// Load all files from old disks + for (const auto & disk : keeper_context->getOldLogDisks()) + { + load_from_disk(disk); + } + auto disk = getDisk(); load_from_disk(disk); @@ -738,7 +760,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin assert(existing_changelogs.find(last_log_read_result->log_start_index) != existing_changelogs.end()); assert(existing_changelogs.find(last_log_read_result->log_start_index)->first == existing_changelogs.rbegin()->first); - /// Continue to write into incomplete existing log if it doesn't finish with error + /// Continue to write into incomplete existing log if it didn't finish with error const auto & description = existing_changelogs[last_log_read_result->log_start_index]; if (last_log_read_result->last_read_index == 0 || last_log_read_result->error) /// If it's broken log then remove it @@ -755,7 +777,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin } else if (last_log_read_result.has_value()) { - /// check if we need to move it to another disk + /// check if we need to move completed log to another disk auto current_log_disk = getCurrentLogDisk(); auto disk = getDisk(); @@ -768,6 +790,24 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin if (!current_writer->isFileSet()) current_writer->rotate(max_log_id + 1); + /// Move files to correct disks + auto latest_start_index = current_writer->getStartIndex(); + auto current_log_disk = getCurrentLogDisk(); + auto disk = getDisk(); + for (const auto & [start_index, description] : existing_changelogs) + { + /// latest log should already be on current_log_disk + if (start_index == latest_start_index) + { + chassert(description->disk == current_log_disk); + continue; + } + + if (description->disk != disk) + moveFileBetweenDisks(description->disk, description, disk, description->path); + } + + initialized = true; } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 3c6411a3a24..e27cfc60cff 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -27,6 +28,14 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) else current_log_storage = log_storage; + Poco::Util::AbstractConfiguration::Keys old_log_disk_name_keys; + config.keys("keeper_server", old_log_disk_name_keys); + for (const auto & key : old_log_disk_name_keys) + { + if (key.starts_with("old_log_storage_disk")) + old_log_disk_names.push_back(config.getString("keeper_server." + key)); + } + snapshot_storage = getSnapshotsPathFromConfig(config); state_file_storage = getStatePathFromConfig(config); @@ -71,6 +80,17 @@ DiskPtr KeeperContext::getLogDisk() const return getDisk(log_storage); } +std::vector KeeperContext::getOldLogDisks() const +{ + std::vector old_log_disks; + old_log_disks.reserve(old_log_disk_names.size()); + + for (const auto & disk_name : old_log_disk_names) + old_log_disks.push_back(disk_selector->get(disk_name)); + + return old_log_disks; +} + DiskPtr KeeperContext::getCurrentLogDisk() const { return getDisk(current_log_storage); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 4e538c99649..e04d1cd6b3d 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -34,6 +34,7 @@ public: DiskPtr getCurrentLogDisk() const; DiskPtr getLogDisk() const; + std::vector getOldLogDisks() const; void setLogDisk(DiskPtr disk); DiskPtr getSnapshotDisk() const; @@ -63,6 +64,8 @@ private: Storage snapshot_storage; Storage state_file_storage; + std::vector old_log_disk_names; + bool standalone_keeper; }; diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 7d808e88b3d..7f097c182a1 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -720,7 +720,6 @@ std::pair KeeperSnapshotManager::serializeSnapshot try { - std::cout << "Removing file " << tmp_snapshot_file_name << std::endl; disk->removeFile(tmp_snapshot_file_name); } catch (fs::filesystem_error & e) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 650fde7bcd1..a680a344746 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -120,7 +120,6 @@ const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getSt void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::string & path) { auto object = StoredObject(metadata_storage.getAbsolutePath(path)); - std::cout << "Removing from plain " << path << std::endl; metadata_storage.object_storage->removeObject(object); } diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index e82b21079fe..5a6fd15d72c 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -8,6 +8,7 @@ #include #include #include +#include using namespace Coordination; using namespace DB; @@ -62,15 +63,18 @@ int main(int argc, char *argv[]) ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; CoordinationSettingsPtr settings = std::make_shared(); - KeeperContextPtr keeper_context = std::make_shared(); - auto state_machine = std::make_shared(queue, snapshots_queue, argv[1], settings, keeper_context, nullptr); + KeeperContextPtr keeper_context = std::make_shared(true); + keeper_context->setLogDisk(std::make_shared("LogDisk", argv[2], 0)); + keeper_context->setSnapshotDisk(std::make_shared("LogDisk", argv[1], 0)); + + auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); size_t last_commited_index = state_machine->last_commit_index(); LOG_INFO(logger, "Last committed index: {}", last_commited_index); DB::KeeperLogStore changelog( - argv[2], LogFileSettings{.force_sync = true, .compress_logs = settings->compress_logs, .rotate_interval = 10000000}); + LogFileSettings{.force_sync = true, .compress_logs = settings->compress_logs, .rotate_interval = 10000000}, keeper_context); changelog.init(last_commited_index, 10000000000UL); /// collect all logs if (changelog.size() == 0) LOG_INFO(logger, "Changelog empty"); From 242c3bc9a971b1f9b76df57b7df1ac5d176fe274 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 24 May 2023 16:01:28 +0200 Subject: [PATCH 0610/2223] fix --- tests/integration/test_ttl_replicated/test.py | 22 ++++++++----------- .../02448_clone_replica_lost_part.sql | 7 +++--- 2 files changed, 13 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 4ea4472b812..d78c00a9f9c 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -6,6 +6,7 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV, exec_query_with_retry from helpers.wait_for_helpers import wait_for_delete_inactive_parts from helpers.wait_for_helpers import wait_for_delete_empty_parts +from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance("node1", with_zookeeper=True) @@ -66,7 +67,8 @@ def test_ttl_columns(started_cluster): """ CREATE TABLE test_ttl(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_columns', '{replica}') - ORDER BY id PARTITION BY toDayOfMonth(date) SETTINGS merge_with_ttl_timeout=0, min_bytes_for_wide_part=0; + ORDER BY id PARTITION BY toDayOfMonth(date) + SETTINGS merge_with_ttl_timeout=0, min_bytes_for_wide_part=0, , max_merge_selecting_sleep_ms=6000; """.format( replica=node.name ) @@ -99,7 +101,7 @@ def test_merge_with_ttl_timeout(started_cluster): CREATE TABLE {table}(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) - SETTINGS min_bytes_for_wide_part=0; + SETTINGS min_bytes_for_wide_part=0, max_merge_selecting_sleep_ms=6000; """.format( replica=node.name, table=table ) @@ -134,14 +136,8 @@ def test_merge_with_ttl_timeout(started_cluster): ) ) - time.sleep(15) # TTL merges shall not happen. - - assert ( - node1.query("SELECT countIf(a = 0) FROM {table}".format(table=table)) == "3\n" - ) - assert ( - node2.query("SELECT countIf(a = 0) FROM {table}".format(table=table)) == "3\n" - ) + assert_eq_with_retry(node1, "SELECT countIf(a = 0) FROM {table}".format(table=table), "3\n") + assert_eq_with_retry(node2, "SELECT countIf(a = 0) FROM {table}".format(table=table), "3\n") def test_ttl_many_columns(started_cluster): @@ -155,7 +151,7 @@ def test_ttl_many_columns(started_cluster): _offset Int32 TTL date, _partition Int32 TTL date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_2', '{replica}') - ORDER BY id PARTITION BY toDayOfMonth(date) SETTINGS merge_with_ttl_timeout=0; + ORDER BY id PARTITION BY toDayOfMonth(date) SETTINGS merge_with_ttl_timeout=0, max_merge_selecting_sleep_ms=6000; """.format( replica=node.name ) @@ -213,7 +209,7 @@ def test_ttl_table(started_cluster, delete_suffix): CREATE TABLE test_ttl(date DateTime, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) - TTL date + INTERVAL 1 DAY {delete_suffix} SETTINGS merge_with_ttl_timeout=0; + TTL date + INTERVAL 1 DAY {delete_suffix} SETTINGS merge_with_ttl_timeout=0, max_merge_selecting_sleep_ms=6000; """.format( replica=node.name, delete_suffix=delete_suffix ) @@ -304,7 +300,7 @@ def test_ttl_double_delete_rule_returns_error(started_cluster): CREATE TABLE test_ttl(date DateTime, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_double_delete', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) - TTL date + INTERVAL 1 DAY, date + INTERVAL 2 DAY SETTINGS merge_with_ttl_timeout=0 + TTL date + INTERVAL 1 DAY, date + INTERVAL 2 DAY SETTINGS merge_with_ttl_timeout=0, max_merge_selecting_sleep_ms=6000 """.format( replica=node1.name ) diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index 7ad25d75fbe..1e99e1869cc 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -6,10 +6,12 @@ drop table if exists rmt1; drop table if exists rmt2; create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '1') order by tuple() settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, - cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime=0, max_parts_to_merge_at_once=4; + cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime=0, max_parts_to_merge_at_once=4, + merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=500; create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '2') order by tuple() settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, - cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime=0, max_parts_to_merge_at_once=4; + cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime=0, max_parts_to_merge_at_once=4, + merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=500; -- insert part only on one replica system stop replicated sends rmt1; @@ -144,7 +146,6 @@ select sleep(2) format Null; -- increases probability of reproducing the issue -- rmt1 will mimic rmt2, but will not be able to fetch parts for a while system stop replicated sends rmt2; attach table rmt1; -system sync replica rmt1; -- rmt1 should not show the value (200) from dropped part select throwIf(n = 200) from rmt1 format Null; select 11, arraySort(groupArray(n)) from rmt2; From de0a074545b1af9a44cbafc9d0c0d90d7d6d7bff Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 24 May 2023 16:03:21 +0200 Subject: [PATCH 0611/2223] Don't replicate delete through DDL worker if there is just 1 shard --- src/Databases/DatabaseReplicated.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 2827ec0ce77..9bbf5b9565d 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1455,7 +1455,16 @@ bool DatabaseReplicated::shouldReplicateQuery(const ContextPtr & query_context, } if (query_ptr->as() != nullptr) - return !is_keeper_map_table(query_ptr); + { + if (is_keeper_map_table(query_ptr)) + return false; + + /// If there is only 1 shard then there is no need to replicate DELETE query. + auto current_cluster = tryGetCluster(); + return + !current_cluster || /// Couldn't get the cluster, so we don't know how many shards there are. + current_cluster->getShardsInfo().size() > 1; + } return true; } From a237b8b83958abbb6976fdb72f67790c54442195 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 24 May 2023 14:19:37 +0000 Subject: [PATCH 0612/2223] Automatic style fix --- tests/integration/test_ttl_replicated/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index d78c00a9f9c..d681e81df3a 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -136,8 +136,12 @@ def test_merge_with_ttl_timeout(started_cluster): ) ) - assert_eq_with_retry(node1, "SELECT countIf(a = 0) FROM {table}".format(table=table), "3\n") - assert_eq_with_retry(node2, "SELECT countIf(a = 0) FROM {table}".format(table=table), "3\n") + assert_eq_with_retry( + node1, "SELECT countIf(a = 0) FROM {table}".format(table=table), "3\n" + ) + assert_eq_with_retry( + node2, "SELECT countIf(a = 0) FROM {table}".format(table=table), "3\n" + ) def test_ttl_many_columns(started_cluster): From 9f349e21021e6f1fdacaaabbe410f8f558b614e2 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 24 May 2023 10:33:33 -0400 Subject: [PATCH 0613/2223] add params --- docs/en/sql-reference/functions/geo/polygon.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index 00e7d2f99f9..4a8653965c2 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -257,6 +257,8 @@ SELECT round(polygonAreaSpherical([[[(4.346693, 50.858306), (4.367945, 50.852455 ### Input parameters +Polygon + ### Returned value Float @@ -276,8 +278,12 @@ MULTIPOLYGON(((4.36661 50.8434,4.36623 50.8408,4.34496 50.8333,4.33807 50.8487,4 ### Input parameters +Polygons + ### Returned value +MultiPolygon + ## polygonPerimeterSpherical Calculates the perimeter of the polygon. @@ -338,7 +344,7 @@ SELECT polygonAreaCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.)]]]) ### Input parameters -One polygon +Polygon ### Returned value @@ -359,7 +365,7 @@ SELECT polygonPerimeterCartesian([[[(0., 0.), (0., 5.), (5., 5.), (5., 0.)]]]) ### Input parameters -One polygon +Polygon ### Returned value From d4927e4fe54b01c9ee84f57647ac2ac975285596 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 24 May 2023 10:51:43 -0400 Subject: [PATCH 0614/2223] add docs --- docs/en/operations/settings/settings.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2239084a429..50b114fcb00 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4220,3 +4220,12 @@ Possible values: - false — Disallow. Default value: `false`. + +## zstd_window_log_max + +Allows you to select the max window log of ZSTD (it will not be used for MergeTree family) + +Type: Int64 + +Default: 0 + From 2a3362e0c8d4bc7a49a8031d8bc4860e30bfaa8f Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 24 May 2023 17:27:47 +0200 Subject: [PATCH 0615/2223] Implement encrypted disk transaction and fix shared merge tree with encrypted disk --- src/Disks/DiskEncrypted.cpp | 9 +- src/Disks/DiskEncrypted.h | 29 +-- src/Disks/DiskEncryptedTransaction.cpp | 123 +++++++++++++ src/Disks/DiskEncryptedTransaction.h | 242 +++++++++++++++++++++++++ 4 files changed, 385 insertions(+), 18 deletions(-) create mode 100644 src/Disks/DiskEncryptedTransaction.cpp create mode 100644 src/Disks/DiskEncryptedTransaction.h diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 72f668db00b..a94036ef1a6 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -203,18 +203,19 @@ private: }; DiskEncrypted::DiskEncrypted( - const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_) - : DiskEncrypted(name_, parseDiskEncryptedSettings(name_, config_, config_prefix_, map_)) + const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_, bool use_fake_transaction_) + : DiskEncrypted(name_, parseDiskEncryptedSettings(name_, config_, config_prefix_, map_), use_fake_transaction_) { } -DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr settings_) +DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr settings_, bool use_fake_transaction_) : IDisk(name_) , delegate(settings_->wrapped_disk) , encrypted_name(name_) , disk_path(settings_->disk_path) , disk_absolute_path(settings_->wrapped_disk->getPath() + settings_->disk_path) , current_settings(std::move(settings_)) + , use_fake_transaction(use_fake_transaction_) { delegate->createDirectories(disk_path); } @@ -416,7 +417,7 @@ void registerDiskEncrypted(DiskFactory & factory, bool global_skip_access_check) const DisksMap & map) -> DiskPtr { bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); - DiskPtr disk = std::make_shared(name, config, config_prefix, map); + DiskPtr disk = std::make_shared(name, config, config_prefix, map, config.getBool(config_prefix + ".use_fake_transaction", true)); disk->startup(context, skip_access_check); return disk; }; diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 530d9b2dc02..57de8177a2b 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -6,22 +6,14 @@ #include #include #include +#include namespace DB { + class ReadBufferFromFileBase; class WriteBufferFromFileBase; -namespace FileEncryption { enum class Algorithm; } - -struct DiskEncryptedSettings -{ - DiskPtr wrapped_disk; - String disk_path; - std::unordered_map keys; - UInt64 current_key_id; - FileEncryption::Algorithm current_algorithm; -}; /// Encrypted disk ciphers all written files on the fly and writes the encrypted files to an underlying (normal) disk. /// And when we read files from an encrypted disk it deciphers them automatically, @@ -29,8 +21,8 @@ struct DiskEncryptedSettings class DiskEncrypted : public IDisk { public: - DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_); - DiskEncrypted(const String & name_, std::unique_ptr settings_); + DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_, bool use_fake_transaction_); + DiskEncrypted(const String & name_, std::unique_ptr settings_, bool use_fake_transaction_); const String & getName() const override { return encrypted_name; } const String & getPath() const override { return disk_absolute_path; } @@ -69,7 +61,6 @@ public: delegate->createDirectories(wrapped_path); } - void clearDirectory(const String & path) override { auto wrapped_path = wrappedPath(path); @@ -293,7 +284,16 @@ public: { /// Need to overwrite explicetly because this disk change /// a lot of "delegate" methods. - return std::make_shared(*this); + + if (use_fake_transaction) + { + return std::make_shared(*this); + } + else + { + auto delegate_transaction = delegate->createTransaction(); + return std::make_shared(delegate_transaction, disk_path, *current_settings.get(), delegate.get()); + } } UInt64 getTotalSpace() const override @@ -342,6 +342,7 @@ private: const String disk_path; const String disk_absolute_path; MultiVersion current_settings; + bool use_fake_transaction; }; } diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp new file mode 100644 index 00000000000..7174e98f256 --- /dev/null +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -0,0 +1,123 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int DATA_ENCRYPTION_ERROR; + extern const int NOT_IMPLEMENTED; +} + + +namespace +{ + +FileEncryption::Header readHeader(ReadBufferFromFileBase & read_buffer) +{ + try + { + FileEncryption::Header header; + header.read(read_buffer); + return header; + } + catch (Exception & e) + { + e.addMessage("While reading the header of encrypted file " + quoteString(read_buffer.getFileName())); + throw; + } +} + +String getCurrentKey(const String & path, const DiskEncryptedSettings & settings) +{ + auto it = settings.keys.find(settings.current_key_id); + if (it == settings.keys.end()) + throw Exception( + ErrorCodes::DATA_ENCRYPTION_ERROR, + "Not found a key with the current ID {} required to cipher file {}", + settings.current_key_id, + quoteString(path)); + + return it->second; +} + +String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings) +{ + auto it = settings.keys.find(header.key_id); + if (it == settings.keys.end()) + throw Exception( + ErrorCodes::DATA_ENCRYPTION_ERROR, + "Not found a key with ID {} required to decipher file {}", + header.key_id, + quoteString(path)); + + String key = it->second; + if (FileEncryption::calculateKeyHash(key) != header.key_hash) + throw Exception( + ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key with ID {}, could not decipher file {}", header.key_id, quoteString(path)); + + return key; +} + +} + +void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path) +{ + auto wrapped_from_path = wrappedPath(from_file_path); + auto wrapped_to_path = wrappedPath(to_file_path); + delegate_transaction->copyFile(wrapped_from_path, wrapped_to_path); +} + +std::unique_ptr DiskEncryptedTransaction::writeFile( + const std::string & path, + size_t buf_size, + WriteMode mode, + const WriteSettings & settings, + bool autocommit) +{ + auto wrapped_path = wrappedPath(path); + FileEncryption::Header header; + String key; + UInt64 old_file_size = 0; + if (mode == WriteMode::Append && delegate_disk->exists(path)) + { + old_file_size = delegate_disk->getFileSize(path); + if (old_file_size) + { + /// Append mode: we continue to use the same header. + auto read_buffer = delegate_disk->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize)); + header = readHeader(*read_buffer); + key = getKey(path, header, current_settings); + } + } + if (!old_file_size) + { + /// Rewrite mode: we generate a new header. + key = getCurrentKey(path, current_settings); + header.algorithm = current_settings.current_algorithm; + header.key_id = current_settings.current_key_id; + header.key_hash = FileEncryption::calculateKeyHash(key); + header.init_vector = FileEncryption::InitVector::random(); + } + auto buffer = delegate_transaction->writeFile(wrapped_path, buf_size, mode, settings, autocommit); + return std::make_unique(buf_size, std::move(buffer), key, header, old_file_size); + +} +void DiskEncryptedTransaction::writeFileUsingCustomWriteObject( + const String &, + WriteMode, + std::function & object_attributes)>) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `writeFileUsingCustomWriteObject()` is not implemented"); +} + + + +} diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h new file mode 100644 index 00000000000..d68e0c42807 --- /dev/null +++ b/src/Disks/DiskEncryptedTransaction.h @@ -0,0 +1,242 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace FileEncryption { enum class Algorithm; } + +struct DiskEncryptedSettings +{ + DiskPtr wrapped_disk; + String disk_path; + std::unordered_map keys; + UInt64 current_key_id; + FileEncryption::Algorithm current_algorithm; +}; + +class DiskEncryptedTransaction : public IDiskTransaction +{ +public: + explicit DiskEncryptedTransaction(DiskTransactionPtr delegate_transaction_, const std::string & disk_path_, DiskEncryptedSettings current_settings_, IDisk * delegate_disk_) + : delegate_transaction(delegate_transaction_) + , disk_path(disk_path_) + , current_settings(current_settings_) + , delegate_disk(delegate_disk_) + {} + + /// Tries to commit all accumulated operations simultaneously. + /// If something fails rollback and throw exception. + void commit(const TransactionCommitOptionsVariant & options = NoCommitOptions{}) override // NOLINT + { + delegate_transaction->commit(options); + } + + void undo() override + { + delegate_transaction->undo(); + } + + TransactionCommitOutcomeVariant tryCommit(const TransactionCommitOptionsVariant & options) override + { + return delegate_transaction->tryCommit(options); + } + + ~DiskEncryptedTransaction() override = default; + + /// Create directory. + void createDirectory(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->createDirectory(wrapped_path); + } + + /// Create directory and all parent directories if necessary. + void createDirectories(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->createDirectories(wrapped_path); + } + + /// Remove all files from the directory. Directories are not removed. + void clearDirectory(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->clearDirectory(wrapped_path); + } + + /// Move directory from `from_path` to `to_path`. + void moveDirectory(const std::string & from_path, const std::string & to_path) override + { + auto wrapped_from_path = wrappedPath(from_path); + auto wrapped_to_path = wrappedPath(to_path); + delegate_transaction->moveDirectory(wrapped_from_path, wrapped_to_path); + } + + void moveFile(const std::string & from_path, const std::string & to_path) override + { + auto wrapped_from_path = wrappedPath(from_path); + auto wrapped_to_path = wrappedPath(to_path); + delegate_transaction->moveFile(wrapped_from_path, wrapped_to_path); + + } + + void createFile(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->createFile(wrapped_path); + } + + /// Move the file from `from_path` to `to_path`. + /// If a file with `to_path` path already exists, it will be replaced. + void replaceFile(const std::string & from_path, const std::string & to_path) override + { + auto wrapped_from_path = wrappedPath(from_path); + auto wrapped_to_path = wrappedPath(to_path); + delegate_transaction->replaceFile(wrapped_from_path, wrapped_to_path); + } + + /// Only copy of several files supported now. Disk interface support copy to another disk + /// but it's impossible to implement correctly in transactions because other disk can + /// use different metadata storage. + /// TODO: maybe remove it at all, we don't want copies + void copyFile(const std::string & from_file_path, const std::string & to_file_path) override; + + /// Open the file for write and return WriteBufferFromFileBase object. + std::unique_ptr writeFile( /// NOLINT + const std::string & path, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + WriteMode mode = WriteMode::Rewrite, + const WriteSettings & settings = {}, + bool autocommit = true) override; + + /// Write a file using a custom function to write an object to the disk's object storage. + void writeFileUsingCustomWriteObject( + const String & path, + WriteMode mode, + std::function & object_attributes)> + custom_write_object_function) override; + + /// Remove file. Throws exception if file doesn't exists or it's a directory. + void removeFile(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeFile(wrapped_path); + } + + /// Remove file if it exists. + void removeFileIfExists(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeFileIfExists(wrapped_path); + } + + /// Remove directory. Throws exception if it's not a directory or if directory is not empty. + void removeDirectory(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeDirectory(wrapped_path); + } + + /// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists. + void removeRecursive(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeRecursive(wrapped_path); + } + + /// Remove file. Throws exception if file doesn't exists or if directory is not empty. + /// Differs from removeFile for S3/HDFS disks + /// Second bool param is a flag to remove (true) or keep (false) shared data on S3 + void removeSharedFile(const std::string & path, bool keep_shared_data) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeSharedFile(wrapped_path, keep_shared_data); + } + + /// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists. + /// Differs from removeRecursive for S3/HDFS disks + /// Second bool param is a flag to remove (false) or keep (true) shared data on S3. + /// Third param determines which files cannot be removed even if second is true. + void removeSharedRecursive(const std::string & path, bool keep_all_shared_data, const NameSet & file_names_remove_metadata_only) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeSharedRecursive(wrapped_path, keep_all_shared_data, file_names_remove_metadata_only); + } + + /// Remove file or directory if it exists. + /// Differs from removeFileIfExists for S3/HDFS disks + /// Second bool param is a flag to remove (true) or keep (false) shared data on S3 + void removeSharedFileIfExists(const std::string & path, bool keep_shared_data) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeSharedFileIfExists(wrapped_path, keep_shared_data); + } + + /// Batch request to remove multiple files. + /// May be much faster for blob storage. + /// Second bool param is a flag to remove (true) or keep (false) shared data on S3. + /// Third param determines which files cannot be removed even if second is true. + void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override + { + for (const auto & file : files) + { + auto wrapped_path = wrappedPath(file.path); + bool keep = keep_all_batch_data || file_names_remove_metadata_only.contains(fs::path(file.path).filename()); + if (file.if_exists) + delegate_transaction->removeSharedFileIfExists(wrapped_path, keep); + else + delegate_transaction->removeSharedFile(wrapped_path, keep); + } + } + + /// Set last modified time to file or directory at `path`. + void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->setLastModified(wrapped_path, timestamp); + } + + /// Just chmod. + void chmod(const String & path, mode_t mode) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->chmod(wrapped_path, mode); + } + + /// Set file at `path` as read-only. + void setReadOnly(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->setReadOnly(wrapped_path); + } + + /// Create hardlink from `src_path` to `dst_path`. + void createHardLink(const std::string & src_path, const std::string & dst_path) override + { + auto wrapped_src_path = wrappedPath(src_path); + auto wrapped_dst_path = wrappedPath(dst_path); + delegate_transaction->createHardLink(wrapped_src_path, wrapped_dst_path); + } + +private: + String wrappedPath(const String & path) const + { + // if path starts_with disk_path -> got already wrapped path + if (!disk_path.empty() && path.starts_with(disk_path)) + return path; + return disk_path + path; + } + + DiskTransactionPtr delegate_transaction; + std::string disk_path; + DiskEncryptedSettings current_settings; + IDisk * delegate_disk; +}; + +} From 0219f78630d109fda1be5c7367729a5a6f86c126 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Wed, 24 May 2023 12:40:12 -0400 Subject: [PATCH 0616/2223] Testing algolia index --- docs/en/sql-reference/statements/create/function.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/create/function.md b/docs/en/sql-reference/statements/create/function.md index 15c2356445b..db65cb4448c 100644 --- a/docs/en/sql-reference/statements/create/function.md +++ b/docs/en/sql-reference/statements/create/function.md @@ -2,11 +2,10 @@ slug: /en/sql-reference/statements/create/function sidebar_position: 38 sidebar_label: FUNCTION +title: "CREATE FUNCTION -user defined function (UDF)" --- -# CREATE FUNCTION - user defined function (UDF) - -Creates a user defined function from a lambda expression. The expression must consist of function parameters, constants, operators, or other function calls. +Creates a user defined function (UDF) from a lambda expression. The expression must consist of function parameters, constants, operators, or other function calls. **Syntax** From 8df7a6914769885f137947fc82b3614fb44c4959 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 23 May 2023 06:47:18 -0400 Subject: [PATCH 0617/2223] Automatic backports of important fixes to cloud-release * Automatic backports to cloud-release --------- Co-authored-by: robot-clickhouse --- tests/ci/cherry_pick.py | 138 ++++++++++++++++++++++++++++------------ 1 file changed, 99 insertions(+), 39 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index fd783192ef1..11c8b7e46eb 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -45,8 +45,10 @@ from ssh import SSHKey class Labels: MUST_BACKPORT = "pr-must-backport" + MUST_BACKPORT_CLOUD = "pr-must-backport-cloud" BACKPORT = "pr-backport" BACKPORTS_CREATED = "pr-backports-created" + BACKPORTS_CREATED_CLOUD = "pr-backports-created-cloud" CHERRYPICK = "pr-cherrypick" DO_NOT_TEST = "do not test" @@ -68,9 +70,9 @@ This pull-request will be merged automatically as it reaches the mergeable state ### If the PR was closed and then reopened -If it stuck, check #{pr_number} for `{label_backports_created}` and delete it if \ +If it stuck, check {pr_url} for `{label_backports_created}` and delete it if \ necessary. Manually merging will do nothing, since `{label_backports_created}` \ -prevents the original PR #{pr_number} from being processed. +prevents the original PR {pr_url} from being processed. """ BACKPORT_DESCRIPTION = """This pull-request is a last step of an automated \ backporting. @@ -80,14 +82,17 @@ close it. """ REMOTE = "" - def __init__(self, name: str, pr: PullRequest): + def __init__(self, name: str, pr: PullRequest, repo: Repository): self.name = name self.pr = pr + self.repo = repo + self.cherrypick_branch = f"cherrypick/{name}/{pr.merge_commit_sha}" self.backport_branch = f"backport/{name}/{pr.number}" self.cherrypick_pr = None # type: Optional[PullRequest] self.backport_pr = None # type: Optional[PullRequest] - self._backported = None # type: Optional[bool] + self._backported = False + self.git_prefix = ( # All commits to cherrypick are done as robot-clickhouse "git -c user.email=robot-clickhouse@users.noreply.github.com " "-c user.name=robot-clickhouse -c commit.gpgsign=false" @@ -188,7 +193,7 @@ close it. f"{self.cherrypick_branch} {self.pr.merge_commit_sha}" ) - # Check if there actually any changes between branches. If no, then no + # Check if there are actually any changes between branches. If no, then no # other actions are required. It's possible when changes are backported # manually to the release branch already try: @@ -216,10 +221,11 @@ close it. for branch in [self.cherrypick_branch, self.backport_branch]: git_runner(f"{self.git_prefix} push -f {self.REMOTE} {branch}:{branch}") - self.cherrypick_pr = self.pr.base.repo.create_pull( + self.cherrypick_pr = self.repo.create_pull( title=f"Cherry pick #{self.pr.number} to {self.name}: {self.pr.title}", body=self.CHERRYPICK_DESCRIPTION.format( pr_number=self.pr.number, + pr_url=self.pr.html_url, label_backports_created=Labels.BACKPORTS_CREATED, ), base=self.backport_branch, @@ -253,9 +259,9 @@ close it. f"{self.git_prefix} push -f {self.REMOTE} " f"{self.backport_branch}:{self.backport_branch}" ) - self.backport_pr = self.pr.base.repo.create_pull( + self.backport_pr = self.repo.create_pull( title=title, - body=f"Original pull-request #{self.pr.number}\n" + body=f"Original pull-request {self.pr.url}\n" f"Cherry-pick pull-request #{self.cherrypick_pr.number}\n\n" f"{self.BACKPORT_DESCRIPTION}", base=self.name, @@ -314,22 +320,33 @@ close it. @property def backported(self) -> bool: - if self._backported is not None: - return self._backported - return self.backport_pr is not None + return self._backported or self.backport_pr is not None def __repr__(self): return self.name class Backport: - def __init__(self, gh: GitHub, repo: str, dry_run: bool): + def __init__( + self, + gh: GitHub, + repo: str, + fetch_from: Optional[str], + dry_run: bool, + must_create_backport_label: str, + backport_created_label: str, + ): self.gh = gh self._repo_name = repo + self._fetch_from = fetch_from self.dry_run = dry_run - self._query = f"type:pr repo:{repo}" + self.must_create_backport_label = must_create_backport_label + self.backport_created_label = backport_created_label + self._remote = "" + self._remote_line = "" + self._repo = None # type: Optional[Repository] self.release_prs = [] # type: PullRequests self.release_branches = [] # type: List[str] @@ -338,25 +355,38 @@ class Backport: self.error = None # type: Optional[Exception] @property - def remote(self) -> str: - if not self._remote: + def remote_line(self) -> str: + if not self._remote_line: # lines of "origin git@github.com:ClickHouse/ClickHouse.git (fetch)" remotes = git_runner("git remote -v").split("\n") # We need the first word from the first matching result - self._remote = tuple( - remote.split(maxsplit=1)[0] - for remote in remotes - if f"github.com/{self._repo_name}" in remote # https - or f"github.com:{self._repo_name}" in remote # ssh - )[0] + self._remote_line = next( + iter( + remote + for remote in remotes + if f"github.com/{self._repo_name}" in remote # https + or f"github.com:{self._repo_name}" in remote # ssh + ) + ) + + return self._remote_line + + @property + def remote(self) -> str: + if not self._remote: + self._remote = self.remote_line.split(maxsplit=1)[0] git_runner(f"git fetch {self._remote}") ReleaseBranch.REMOTE = self._remote return self._remote + @property + def is_remote_ssh(self) -> bool: + return "github.com:" in self.remote_line + def receive_release_prs(self): logging.info("Getting release PRs") self.release_prs = self.gh.get_pulls_from_search( - query=f"{self._query} is:open", + query=f"type:pr repo:{self._repo_name} is:open", sort="created", order="asc", label="release", @@ -365,6 +395,14 @@ class Backport: self.labels_to_backport = [ f"v{branch}-must-backport" for branch in self.release_branches ] + + if self._fetch_from: + logging.info("Fetching from %s", self._fetch_from) + fetch_from_repo = self.gh.get_repo(self._fetch_from) + git_runner( + f"git fetch {fetch_from_repo.ssh_url if self.is_remote_ssh else fetch_from_repo.clone_url} {fetch_from_repo.default_branch} --no-tags" + ) + logging.info("Active releases: %s", ", ".join(self.release_branches)) def update_local_release_branches(self): @@ -396,9 +434,10 @@ class Backport: # To not have a possible TZ issues tomorrow = date.today() + timedelta(days=1) logging.info("Receive PRs suppose to be backported") + self.prs_for_backport = self.gh.get_pulls_from_search( - query=f"{self._query} -label:{Labels.BACKPORTS_CREATED}", - label=",".join(self.labels_to_backport + [Labels.MUST_BACKPORT]), + query=f"type:pr repo:{self._fetch_from} -label:{self.backport_created_label}", + label=",".join(self.labels_to_backport + [self.must_create_backport_label]), merged=[since_date, tomorrow], ) logging.info( @@ -418,13 +457,13 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - if Labels.MUST_BACKPORT in pr_labels: + if self.must_create_backport_label in pr_labels: branches = [ - ReleaseBranch(br, pr) for br in self.release_branches + ReleaseBranch(br, pr, self.repo) for br in self.release_branches ] # type: List[ReleaseBranch] else: branches = [ - ReleaseBranch(br, pr) + ReleaseBranch(br, pr, self.repo) for br in [ label.split("-", 1)[0][1:] # v21.8-must-backport for label in pr_labels @@ -452,14 +491,14 @@ class Backport: ] ) bp_cp_prs = self.gh.get_pulls_from_search( - query=f"{self._query} {query_suffix}", + query=f"type:pr repo:{self._repo_name} {query_suffix}", ) for br in branches: br.pop_prs(bp_cp_prs) if bp_cp_prs: # This is definitely some error. All prs must be consumed by - # branches with ReleaseBranch.pop_prs. It also make the whole + # branches with ReleaseBranch.pop_prs. It also makes the whole # program exit code non-zero self.error = Exception( "The following PRs are not filtered by release branches:\n" @@ -483,22 +522,17 @@ class Backport: if self.dry_run: logging.info("DRY RUN: would mark PR #%s as done", pr.number) return - pr.add_to_labels(Labels.BACKPORTS_CREATED) + pr.add_to_labels(self.backport_created_label) logging.info( "PR #%s is successfully labeled with `%s`", pr.number, - Labels.BACKPORTS_CREATED, + self.backport_created_label, ) @property def repo(self) -> Repository: if self._repo is None: - try: - self._repo = self.release_prs[0].base.repo - except IndexError as exc: - raise Exception( - "`repo` is available only after the `receive_release_prs`" - ) from exc + self._repo = self.gh.get_repo(self._repo_name) return self._repo @property @@ -512,7 +546,26 @@ def parse_args(): parser.add_argument( "--repo", default="ClickHouse/ClickHouse", help="repo owner/name" ) + parser.add_argument( + "--from-repo", + help="if set, the commits will be taken from this repo, but PRs will be created in the main repo", + ) parser.add_argument("--dry-run", action="store_true", help="do not create anything") + + parser.add_argument( + "--must-create-backport-label", + default=Labels.MUST_BACKPORT, + choices=(Labels.MUST_BACKPORT, Labels.MUST_BACKPORT_CLOUD), + help="label to filter PRs to backport", + ) + + parser.add_argument( + "--backport-created-label", + default=Labels.BACKPORTS_CREATED, + choices=(Labels.BACKPORTS_CREATED, Labels.BACKPORTS_CREATED_CLOUD), + help="label to mark PRs as backported", + ) + parser.add_argument( "--debug-helpers", action="store_true", @@ -564,7 +617,14 @@ def main(): token = args.token or get_best_robot_token() gh = GitHub(token, create_cache_dir=False) - bp = Backport(gh, args.repo, args.dry_run) + bp = Backport( + gh, + args.repo, + args.from_repo, + args.dry_run, + args.must_create_backport_label, + args.backport_created_label, + ) # https://github.com/python/mypy/issues/3004 bp.gh.cache_path = f"{TEMP_PATH}/gh_cache" # type: ignore bp.receive_release_prs() @@ -577,7 +637,7 @@ def main(): if __name__ == "__main__": - logging.basicConfig(level=logging.INFO) + logging.getLogger().setLevel(level=logging.INFO) assert not is_shallow() with stash(): From bc527c75889b321a01c30f665eb0d4ef47e61d68 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 24 May 2023 17:07:31 +0000 Subject: [PATCH 0618/2223] Don't send head request for all keys in Iceberg schema inference --- src/Storages/StorageS3.cpp | 19 ++++++++++++++----- src/Storages/StorageS3.h | 2 ++ 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index afaafcc75a2..f3cad4de31a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -418,11 +418,13 @@ public: ASTPtr query_, const Block & virtual_header_, ContextPtr context_, + bool need_total_size, KeysWithInfo * read_keys_) : WithContext(context_) , bucket(bucket_) , query(query_) , virtual_header(virtual_header_) + { Strings all_keys = keys_; @@ -458,8 +460,13 @@ public: for (auto && key : all_keys) { - auto info = S3::getObjectInfo(client_, bucket, key, version_id_, request_settings_); - total_size += info.size; + std::optional info; + if (need_total_size) + { + info = S3::getObjectInfo(client_, bucket, key, version_id_, request_settings_); + total_size += info->size; + } + keys.emplace_back(std::move(key), std::move(info)); } @@ -501,10 +508,11 @@ StorageS3Source::KeysIterator::KeysIterator( ASTPtr query, const Block & virtual_header, ContextPtr context, + bool need_total_size, KeysWithInfo * read_keys) : pimpl(std::make_shared( client_, version_id_, keys_, bucket_, request_settings_, - query, virtual_header, context, read_keys)) + query, virtual_header, context, need_total_size, read_keys)) { } @@ -979,6 +987,7 @@ std::shared_ptr StorageS3::createFileIterator( ContextPtr local_context, ASTPtr query, const Block & virtual_block, + bool need_total_size, KeysWithInfo * read_keys) { if (distributed_processing) @@ -997,7 +1006,7 @@ std::shared_ptr StorageS3::createFileIterator( return std::make_shared( *configuration.client, configuration.url.version_id, configuration.keys, configuration.url.bucket, configuration.request_settings, query, - virtual_block, local_context, read_keys); + virtual_block, local_context, need_total_size, read_keys); } } @@ -1442,7 +1451,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( { KeysWithInfo read_keys; - auto file_iterator = createFileIterator(configuration, false, ctx, nullptr, {}, &read_keys); + auto file_iterator = createFileIterator(configuration, false, ctx, nullptr, {}, false, &read_keys); std::optional columns_from_cache; size_t prev_read_keys_size = read_keys.size(); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 12573ab513f..1ca8f80e7a0 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -95,6 +95,7 @@ public: ASTPtr query, const Block & virtual_header, ContextPtr context, + bool need_total_size = true, KeysWithInfo * read_keys = nullptr); KeyWithInfo next() override; @@ -354,6 +355,7 @@ private: ContextPtr local_context, ASTPtr query, const Block & virtual_block, + bool need_total_size = true, KeysWithInfo * read_keys = nullptr); static ColumnsDescription getTableStructureFromDataImpl( From 4c94b3d6bce6bf34a52e83f98b6fec312e4ba79b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 24 May 2023 20:13:37 +0300 Subject: [PATCH 0619/2223] Update test.py --- tests/integration/test_ttl_replicated/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index d681e81df3a..7ba5a4359c7 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -68,7 +68,7 @@ def test_ttl_columns(started_cluster): CREATE TABLE test_ttl(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_columns', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) - SETTINGS merge_with_ttl_timeout=0, min_bytes_for_wide_part=0, , max_merge_selecting_sleep_ms=6000; + SETTINGS merge_with_ttl_timeout=0, min_bytes_for_wide_part=0, max_merge_selecting_sleep_ms=6000; """.format( replica=node.name ) From e66f6272d1dc76859251fde165b2d2d9664dce8f Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 10 May 2023 18:39:38 +0000 Subject: [PATCH 0620/2223] Refactor CapnProto format to improve input/output performance --- src/Core/Settings.h | 2 +- src/Core/SettingsEnums.cpp | 8 +- src/Core/SettingsEnums.h | 2 +- src/Formats/CapnProtoSchema.cpp | 298 ++++ .../{CapnProtoUtils.h => CapnProtoSchema.h} | 13 +- src/Formats/CapnProtoSerializer.cpp | 1218 +++++++++++++++++ src/Formats/CapnProtoSerializer.h | 25 + src/Formats/CapnProtoUtils.cpp | 734 ---------- src/Formats/FormatSettings.h | 6 +- .../Formats/Impl/CapnProtoRowInputFormat.cpp | 253 +--- .../Formats/Impl/CapnProtoRowInputFormat.h | 9 +- .../Formats/Impl/CapnProtoRowOutputFormat.cpp | 266 +--- .../Formats/Impl/CapnProtoRowOutputFormat.h | 17 +- .../Formats/Impl/ProtobufListInputFormat.cpp | 9 +- .../Formats/Impl/ProtobufRowInputFormat.cpp | 9 +- .../queries/0_stateless/02030_capnp_format.sh | 4 +- ...p_case_insensitive_names_matcing.reference | 1 + ...35_capnp_case_insensitive_names_matcing.sh | 10 + ...ing_and_writing_structure_fields.reference | 3 + ...36_reading_and_writing_structure_fields.sh | 24 + ...2735_case_insensitive_names_matching.capnp | 13 + .../02736_nested_structures.capnp | 21 + 22 files changed, 1686 insertions(+), 1259 deletions(-) create mode 100644 src/Formats/CapnProtoSchema.cpp rename src/Formats/{CapnProtoUtils.h => CapnProtoSchema.h} (59%) create mode 100644 src/Formats/CapnProtoSerializer.cpp create mode 100644 src/Formats/CapnProtoSerializer.h delete mode 100644 src/Formats/CapnProtoUtils.cpp create mode 100644 tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.reference create mode 100755 tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.sh create mode 100644 tests/queries/0_stateless/02736_reading_and_writing_structure_fields.reference create mode 100755 tests/queries/0_stateless/02736_reading_and_writing_structure_fields.sh create mode 100644 tests/queries/0_stateless/format_schemas/02735_case_insensitive_names_matching.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02736_nested_structures.capnp diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1df0a8af24f..2863cc9d7a7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -962,7 +962,7 @@ class IColumn; M(Bool, output_format_orc_string_as_string, false, "Use ORC String type instead of Binary for String columns", 0) \ M(ORCCompression, output_format_orc_compression_method, "lz4", "Compression method for ORC output format. Supported codecs: lz4, snappy, zlib, zstd, none (uncompressed)", 0) \ \ - M(EnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::EnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \ + M(CapnProtoEnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::CapnProtoEnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \ \ M(String, input_format_mysql_dump_table_name, "", "Name of the table in MySQL dump from which to read data", 0) \ M(Bool, input_format_mysql_dump_map_column_names, true, "Match columns from table in MySQL dump and columns from ClickHouse table by names", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index e0f16ea00db..a291a23c140 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -144,10 +144,10 @@ IMPLEMENT_SETTING_ENUM(TransactionsWaitCSNMode, ErrorCodes::BAD_ARGUMENTS, {"wait", TransactionsWaitCSNMode::WAIT}, {"wait_unknown", TransactionsWaitCSNMode::WAIT_UNKNOWN}}) -IMPLEMENT_SETTING_ENUM(EnumComparingMode, ErrorCodes::BAD_ARGUMENTS, - {{"by_names", FormatSettings::EnumComparingMode::BY_NAMES}, - {"by_values", FormatSettings::EnumComparingMode::BY_VALUES}, - {"by_names_case_insensitive", FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE}}) +IMPLEMENT_SETTING_ENUM(CapnProtoEnumComparingMode, ErrorCodes::BAD_ARGUMENTS, + {{"by_names", FormatSettings::CapnProtoEnumComparingMode::BY_NAMES}, + {"by_values", FormatSettings::CapnProtoEnumComparingMode::BY_VALUES}, + {"by_names_case_insensitive", FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE}}) IMPLEMENT_SETTING_ENUM(EscapingRule, ErrorCodes::BAD_ARGUMENTS, {{"None", FormatSettings::EscapingRule::None}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 3ae7bfaa673..1c5be910ef7 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -188,7 +188,7 @@ enum class TransactionsWaitCSNMode DECLARE_SETTING_ENUM(TransactionsWaitCSNMode) -DECLARE_SETTING_ENUM_WITH_RENAME(EnumComparingMode, FormatSettings::EnumComparingMode) +DECLARE_SETTING_ENUM_WITH_RENAME(CapnProtoEnumComparingMode, FormatSettings::CapnProtoEnumComparingMode) DECLARE_SETTING_ENUM_WITH_RENAME(EscapingRule, FormatSettings::EscapingRule) diff --git a/src/Formats/CapnProtoSchema.cpp b/src/Formats/CapnProtoSchema.cpp new file mode 100644 index 00000000000..22518d5061a --- /dev/null +++ b/src/Formats/CapnProtoSchema.cpp @@ -0,0 +1,298 @@ +#include + +#if USE_CAPNP + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_CAPN_PROTO_SCHEMA; + extern const int BAD_TYPE_OF_FIELD; + extern const int FILE_DOESNT_EXIST; + extern const int UNKNOWN_EXCEPTION; + extern const int CAPN_PROTO_BAD_TYPE; + extern const int BAD_ARGUMENTS; +} + +capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info) +{ + capnp::ParsedSchema schema; + try + { + int fd; + KJ_SYSCALL(fd = open(schema_info.schemaDirectory().data(), O_RDONLY)); // NOLINT(bugprone-suspicious-semicolon) + auto schema_dir = kj::newDiskDirectory(kj::OsFileHandle(fd)); + schema = impl.parseFromDirectory(*schema_dir, kj::Path::parse(schema_info.schemaPath()), {}); + } + catch (const kj::Exception & e) + { + /// That's not good to determine the type of error by its description, but + /// this is the only way to do it here, because kj doesn't specify the type of error. + auto description = std::string_view(e.getDescription().cStr()); + if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); + + if (description.find("Parse error") != String::npos) + throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "Cannot parse CapnProto schema {}:{}", schema_info.schemaPath(), e.getLine()); + + throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, + "Unknown exception while parsing CapnProto schema: {}, schema dir and file: {}, {}", + description, schema_info.schemaDirectory(), schema_info.schemaPath()); + } + + auto message_maybe = schema.findNested(schema_info.messageName()); + auto * message_schema = kj::_::readMaybe(message_maybe); + if (!message_schema) + throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, + "CapnProto schema doesn't contain message with name {}", schema_info.messageName()); + return message_schema->asStruct(); +} + +bool checkIfStructContainsUnnamedUnion(const capnp::StructSchema & struct_schema) +{ + return struct_schema.getFields().size() != struct_schema.getNonUnionFields().size(); +} + +bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema) +{ + return struct_schema.getFields().size() == struct_schema.getUnionFields().size(); +} + +/// Get full name of type for better exception messages. +String getCapnProtoFullTypeName(const capnp::Type & type) +{ + static const std::map capnp_simple_type_names = + { + {capnp::schema::Type::Which::BOOL, "Bool"}, + {capnp::schema::Type::Which::VOID, "Void"}, + {capnp::schema::Type::Which::INT8, "Int8"}, + {capnp::schema::Type::Which::INT16, "Int16"}, + {capnp::schema::Type::Which::INT32, "Int32"}, + {capnp::schema::Type::Which::INT64, "Int64"}, + {capnp::schema::Type::Which::UINT8, "UInt8"}, + {capnp::schema::Type::Which::UINT16, "UInt16"}, + {capnp::schema::Type::Which::UINT32, "UInt32"}, + {capnp::schema::Type::Which::UINT64, "UInt64"}, + {capnp::schema::Type::Which::FLOAT32, "Float32"}, + {capnp::schema::Type::Which::FLOAT64, "Float64"}, + {capnp::schema::Type::Which::TEXT, "Text"}, + {capnp::schema::Type::Which::DATA, "Data"}, + {capnp::schema::Type::Which::INTERFACE, "Interface"}, + {capnp::schema::Type::Which::ANY_POINTER, "AnyPointer"}, + }; + + switch (type.which()) + { + case capnp::schema::Type::Which::STRUCT: + { + auto struct_schema = type.asStruct(); + + auto non_union_fields = struct_schema.getNonUnionFields(); + std::vector non_union_field_names; + for (auto nested_field : non_union_fields) + non_union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); + + auto union_fields = struct_schema.getUnionFields(); + std::vector union_field_names; + for (auto nested_field : union_fields) + union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); + + String union_name = "Union(" + boost::algorithm::join(union_field_names, ", ") + ")"; + /// Check if the struct is a named union. + if (non_union_field_names.empty()) + return union_name; + + String type_name = "Struct(" + boost::algorithm::join(non_union_field_names, ", "); + /// Check if the struct contains unnamed union. + if (!union_field_names.empty()) + type_name += ", " + union_name; + type_name += ")"; + return type_name; + } + case capnp::schema::Type::Which::LIST: + return "List(" + getCapnProtoFullTypeName(type.asList().getElementType()) + ")"; + case capnp::schema::Type::Which::ENUM: + { + auto enum_schema = type.asEnum(); + String enum_name = "Enum("; + auto enumerants = enum_schema.getEnumerants(); + for (unsigned i = 0; i != enumerants.size(); ++i) + { + enum_name += String(enumerants[i].getProto().getName()) + " = " + std::to_string(enumerants[i].getOrdinal()); + if (i + 1 != enumerants.size()) + enum_name += ", "; + } + enum_name += ")"; + return enum_name; + } + default: + auto it = capnp_simple_type_names.find(type.which()); + if (it == capnp_simple_type_names.end()) + throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Unknown CapnProto type"); + return it->second; + } +} + +namespace +{ + + template + static DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) + { + std::vector> values; + for (auto enumerant : enumerants) + values.emplace_back(enumerant.getProto().getName(), ValueType(enumerant.getOrdinal())); + return std::make_shared>(std::move(values)); + } + + static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) + { + auto enumerants = enum_schema.getEnumerants(); + if (enumerants.size() < 128) + return getEnumDataTypeFromEnumerants(enumerants); + if (enumerants.size() < 32768) + return getEnumDataTypeFromEnumerants(enumerants); + + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "ClickHouse supports only 8 and 16-bit Enums"); + } + + static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) + { + switch (capnp_type.which()) + { + case capnp::schema::Type::INT8: + return std::make_shared(); + case capnp::schema::Type::INT16: + return std::make_shared(); + case capnp::schema::Type::INT32: + return std::make_shared(); + case capnp::schema::Type::INT64: + return std::make_shared(); + case capnp::schema::Type::BOOL: [[fallthrough]]; + case capnp::schema::Type::UINT8: + return std::make_shared(); + case capnp::schema::Type::UINT16: + return std::make_shared(); + case capnp::schema::Type::UINT32: + return std::make_shared(); + case capnp::schema::Type::UINT64: + return std::make_shared(); + case capnp::schema::Type::FLOAT32: + return std::make_shared(); + case capnp::schema::Type::FLOAT64: + return std::make_shared(); + case capnp::schema::Type::DATA: [[fallthrough]]; + case capnp::schema::Type::TEXT: + return std::make_shared(); + case capnp::schema::Type::ENUM: + return getEnumDataTypeFromEnumSchema(capnp_type.asEnum()); + case capnp::schema::Type::LIST: + { + auto list_schema = capnp_type.asList(); + auto nested_type = getDataTypeFromCapnProtoType(list_schema.getElementType(), skip_unsupported_fields); + if (!nested_type) + return nullptr; + return std::make_shared(nested_type); + } + case capnp::schema::Type::STRUCT: + { + auto struct_schema = capnp_type.asStruct(); + + + if (struct_schema.getFields().size() == 0) + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Empty messages are not supported"); + } + + /// Check if it can be Nullable. + if (checkIfStructIsNamedUnion(struct_schema)) + { + auto fields = struct_schema.getUnionFields(); + if (fields.size() != 2 || (!fields[0].getType().isVoid() && !fields[1].getType().isVoid())) + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unions are not supported"); + } + auto value_type = fields[0].getType().isVoid() ? fields[1].getType() : fields[0].getType(); + if (value_type.isStruct() || value_type.isList()) + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Tuples and Lists cannot be inside Nullable"); + } + + auto nested_type = getDataTypeFromCapnProtoType(value_type, skip_unsupported_fields); + if (!nested_type) + return nullptr; + return std::make_shared(nested_type); + } + + if (checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); + + /// Treat Struct as Tuple. + DataTypes nested_types; + Names nested_names; + for (auto field : struct_schema.getNonUnionFields()) + { + auto nested_type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); + if (!nested_type) + continue; + nested_names.push_back(field.getProto().getName()); + nested_types.push_back(nested_type); + } + if (nested_types.empty()) + return nullptr; + return std::make_shared(std::move(nested_types), std::move(nested_names)); + } + default: + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unsupported CapnProtoType: {}", getCapnProtoFullTypeName(capnp_type)); + } + } +} + +} + +NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields) +{ + if (checkIfStructContainsUnnamedUnion(schema)) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); + + NamesAndTypesList names_and_types; + for (auto field : schema.getNonUnionFields()) + { + auto name = field.getProto().getName(); + auto type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); + if (type) + names_and_types.emplace_back(name, type); + } + if (names_and_types.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot convert CapnProto schema to ClickHouse table schema, all fields have unsupported types"); + + return names_and_types; +} + +} + +#endif diff --git a/src/Formats/CapnProtoUtils.h b/src/Formats/CapnProtoSchema.h similarity index 59% rename from src/Formats/CapnProtoUtils.h rename to src/Formats/CapnProtoSchema.h index 2d8cdb418d7..225f6f56207 100644 --- a/src/Formats/CapnProtoUtils.h +++ b/src/Formats/CapnProtoSchema.h @@ -30,17 +30,14 @@ public: capnp::StructSchema getMessageSchema(const FormatSchemaInfo & schema_info); }; -std::pair splitCapnProtoFieldName(const String & name); +bool checkIfStructContainsUnnamedUnion(const capnp::StructSchema & struct_schema); +bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema); -bool compareEnumNames(const String & first, const String & second, FormatSettings::EnumComparingMode mode); - -std::pair getStructBuilderAndFieldByColumnName(capnp::DynamicStruct::Builder struct_builder, const String & name); - -capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Reader & struct_reader, const String & name); - -void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode); +/// Get full name of type for better exception messages. +String getCapnProtoFullTypeName(const capnp::Type & type); NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields); + } #endif diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp new file mode 100644 index 00000000000..e0c8ae2a79a --- /dev/null +++ b/src/Formats/CapnProtoSerializer.cpp @@ -0,0 +1,1218 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int THERE_IS_NO_COLUMN; + extern const int BAD_TYPE_OF_FIELD; + extern const int CAPN_PROTO_BAD_CAST; + extern const int INCORRECT_DATA; + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + std::pair splitFieldName(const String & name) + { + const auto * begin = name.data(); + const auto * end = name.data() + name.size(); + const auto * it = find_first_symbols<'_', '.'>(begin, end); + String first = String(begin, it); + String second = it == end ? "" : String(it + 1, end); + return {first, second}; + } + + std::optional findFieldByName(const capnp::StructSchema & struct_schema, const String & name) + { + const auto & fields = struct_schema.getFields(); + for (auto field : fields) + { + auto field_name = String(field.getProto().getName()); + if (boost::to_lower_copy(name) == boost::to_lower_copy(field_name)) + return field; + } + return std::nullopt; + } + + [[noreturn]] void throwCannotConvert(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type) + { + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto type {}", + name, + type->getName(), + getCapnProtoFullTypeName(capnp_type)); + } + + struct FieldBuilder + { + virtual ~FieldBuilder() = default; + }; + + struct ListBuilder : public FieldBuilder + { + explicit ListBuilder(capnp::DynamicValue::Builder builder) : impl(builder.as()) + { + } + + capnp::DynamicList::Builder impl; + std::vector> nested_builders; + }; + + struct StructBuilder : public FieldBuilder + { + explicit StructBuilder(capnp::DynamicValue::Builder builder, size_t fields_size) : impl(builder.as()), field_builders(fields_size) + { + } + + explicit StructBuilder(capnp::DynamicStruct::Builder struct_builder, size_t fields_size) : impl(std::move(struct_builder)), field_builders(fields_size) + { + } + + capnp::DynamicStruct::Builder impl; + std::vector> field_builders; + }; + + std::unique_ptr initStructFieldBuilderIfNeeded(const ColumnPtr & column, size_t row_num, capnp::DynamicStruct::Builder & struct_builder, const capnp::StructSchema::Field & field, const capnp::Type & capnp_type, size_t nested_fields_size) + { + switch (capnp_type.which()) + { + case capnp::schema::Type::LIST: + { + const auto * array_column = assert_cast(column.get()); + size_t size = array_column->getOffsets()[row_num] - array_column->getOffsets()[row_num - 1]; + return std::make_unique(struct_builder.init(field, static_cast(size))); + } + case capnp::schema::Type::STRUCT: + { + return std::make_unique(struct_builder.init(field), nested_fields_size); + } + default: + return nullptr; + } + } + + class ICapnProtoSerializer + { + public: + virtual std::optional writeRow(const ColumnPtr & column, FieldBuilder * builder, size_t row_num) = 0; + virtual void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) = 0; + + virtual ~ICapnProtoSerializer() = default; + }; + + template + class CapnProtoIntegerSerializer : public ICapnProtoSerializer + { + public: + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::INT) + return capnp::DynamicValue::Reader(column->getInt(row_num)); + if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::UINT) + return capnp::DynamicValue::Reader(column->getUInt(row_num)); + return capnp::DynamicValue::Reader(column->getBool(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + NumericType value; + if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::INT) + value = static_cast(reader.as()); + else if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::UINT) + value = static_cast(reader.as()); + else if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::BOOL) + value = static_cast(reader.as()); + + if constexpr (is_bool_data_type) + assert_cast(column).insertValue(static_cast(value)); + else + assert_cast &>(column).insertValue(value); + } + }; + + template + static std::unique_ptr createIntegerSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + switch (capnp_type.which()) + { + case capnp::schema::Type::INT8: [[fallthrough]]; + case capnp::schema::Type::INT16: [[fallthrough]]; + case capnp::schema::Type::INT32: [[fallthrough]]; + case capnp::schema::Type::INT64: + return std::make_unique>(); + case capnp::schema::Type::UINT8: [[fallthrough]]; + case capnp::schema::Type::UINT16: [[fallthrough]]; + case capnp::schema::Type::UINT32: [[fallthrough]]; + case capnp::schema::Type::UINT64: + return std::make_unique>(); + case capnp::schema::Type::BOOL: + return std::make_unique>(); + default: + throwCannotConvert(data_type, column_name, capnp_type); + } + } + + template + class CapnProtoBigIntegerSerializer : public ICapnProtoSerializer + { + public: + CapnProtoBigIntegerSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(NumericType)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + + private: + DataTypePtr data_type; + }; + + template + class CapnProtoFloatSerializer : public ICapnProtoSerializer + { + public: + CapnProtoFloatSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isFloat32() && !capnp_type.isFloat64()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getFloat64(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast &>(column).insertValue(reader.as()); + } + }; + + template + class CapnProtoEnumSerializer : public ICapnProtoSerializer + { + public: + CapnProtoEnumSerializer( + const DataTypePtr & data_type_, + const String & column_name, + const capnp::Type & capnp_type, + const FormatSettings::CapnProtoEnumComparingMode enum_comparing_mode_) : data_type(data_type_), enum_comparing_mode(enum_comparing_mode_) + { + if (!capnp_type.isEnum()) + throwCannotConvert(data_type, column_name, capnp_type); + + bool to_lower = enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE; + const auto * enum_type = assert_cast *>(data_type.get()); + const auto & enum_values = dynamic_cast &>(*enum_type); + + enum_schema = capnp_type.asEnum(); + auto enumerants = enum_schema.getEnumerants(); + constexpr auto max_value = std::is_same_v ? INT8_MAX : INT16_MAX; + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + { + /// In CapnProto Enum fields are numbered sequentially starting from zero. + if (enumerants.size() > max_value) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Enum from CapnProto schema contains values that are out of range for Clickhouse enum type {}", + data_type->getName()); + + auto values = enum_values.getSetOfAllValues(); + std::unordered_set capn_enum_values; + for (auto enumerant : enumerants) + capn_enum_values.insert(EnumType(enumerant.getOrdinal())); + if (values != capn_enum_values) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "The set of values in Enum from CapnProto schema is different from the set of values in ClickHouse Enum"); + } + else + { + auto names = enum_values.getSetOfAllNames(to_lower); + std::unordered_set capn_enum_names; + + for (auto enumerant : enumerants) + { + String name = enumerant.getProto().getName(); + capn_enum_names.insert(to_lower ? boost::algorithm::to_lower_copy(name) : name); + } + + if (names != capn_enum_names) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"); + } + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + const auto * enum_data_type = assert_cast *>(data_type.get()); + EnumType enum_value = assert_cast &>(*column).getElement(row_num); + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + return capnp::DynamicValue::Reader(capnp::DynamicEnum(enum_schema, enum_value)); + + auto enum_name = enum_data_type->getNameForValue(enum_value); + for (const auto enumerant : enum_schema.getEnumerants()) + { + if (compareEnumNames(String(enum_name), enumerant.getProto().getName(), enum_comparing_mode)) + return capnp::DynamicValue::Reader(capnp::DynamicEnum(enumerant)); + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert CLickHouse Enum value to CapnProto Enum"); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto enum_value = reader.as(); + auto enumerant = *kj::_::readMaybe(enum_value.getEnumerant()); + auto enum_type = assert_cast *>(data_type.get()); + DataTypePtr nested_type = std::make_shared>(); + switch (enum_comparing_mode) + { + case FormatSettings::CapnProtoEnumComparingMode::BY_VALUES: + { + assert_cast &>(column).insertValue(static_cast(enumerant.getOrdinal())); + return; + } + case FormatSettings::CapnProtoEnumComparingMode::BY_NAMES: + { + auto value = enum_type->getValue(String(enumerant.getProto().getName())); + assert_cast &>(column).insertValue(value); + return; + } + case FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE: + { + /// Find the same enum name case insensitive. + String enum_name = enumerant.getProto().getName(); + for (auto & name : enum_type->getAllRegisteredNames()) + { + if (compareEnumNames(name, enum_name, enum_comparing_mode)) + { + assert_cast &>(column).insertValue(enum_type->getValue(name)); + break; + } + } + return; + } + } + } + + private: + bool compareEnumNames(const String & first, const String & second, const FormatSettings::CapnProtoEnumComparingMode mode) + { + if (mode == FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE) + return boost::algorithm::to_lower_copy(first) == boost::algorithm::to_lower_copy(second); + return first == second; + } + + DataTypePtr data_type; + capnp::EnumSchema enum_schema; + const FormatSettings::CapnProtoEnumComparingMode enum_comparing_mode; + }; + + class CapnProtoDateSerializer : public ICapnProtoSerializer + { + public: + CapnProtoDateSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isUInt16()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getUInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + class CapnProtoDate32Serializer : public ICapnProtoSerializer + { + public: + CapnProtoDate32Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isInt32()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + class CapnProtoDateTimeSerializer : public ICapnProtoSerializer + { + public: + CapnProtoDateTimeSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isUInt32()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + class CapnProtoDateTime64Serializer : public ICapnProtoSerializer + { + public: + CapnProtoDateTime64Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isInt64()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + template + class CapnProtoDecimalSerializer : public ICapnProtoSerializer + { + public: + CapnProtoDecimalSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + auto which = WhichDataType(data_type); + if ((!capnp_type.isInt32() && which.isDecimal32()) || (!capnp_type.isInt64() && which.isDecimal64())) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast &>(column).insertValue(reader.as()); + } + }; + + template + class CapnProtoBigDecimalSerializer : public ICapnProtoSerializer + { + public: + CapnProtoBigDecimalSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(DecimalType)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + + private: + DataTypePtr data_type; + }; + + template + class CapnProtoStringSerializer : public ICapnProtoSerializer + { + public: + CapnProtoStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type_) : capnp_type(capnp_type_) + { + if (!capnp_type.isData() && !capnp_type.isText()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + + if constexpr (is_binary) + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + + /// For type TEXT data must be null-terminated, but in String column we always have 0 byte at the end of each value. + return capnp::DynamicValue::Reader(capnp::Text::Reader(data.data, data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + if constexpr (is_binary) + { + auto value = reader.as(); + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + else + { + auto value = reader.as(); + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + } + + private: + capnp::Type capnp_type; + }; + + template + class CapnProtoFixedStringSerializer : public ICapnProtoSerializer + { + public: + CapnProtoFixedStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type_) : capnp_type(capnp_type_) + { + if (!capnp_type.isData() && !capnp_type.isText()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + if constexpr (is_binary) + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + + if (data.data[data.size - 1] == 0) + return capnp::DynamicValue::Reader(capnp::Text::Reader(reinterpret_cast(data.data), data.size)); + + /// In TEXT type data should be null-terminated, but ClickHouse FixedString data could not be. + /// To make data null-terminated we should copy it to temporary String object and use it in capnp::Text::Reader. + /// Note that capnp::Text::Reader works only with pointer to the data and it's size, so we should + /// guarantee that new String object life time is longer than capnp::Text::Reader life time. + tmp_string = data.toString(); + return capnp::DynamicValue::Reader(capnp::Text::Reader(reinterpret_cast(tmp_string.data()), tmp_string.size())); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto & fixed_string_column = assert_cast(column); + if constexpr (is_binary) + { + auto value = reader.as(); + if (value.size() > fixed_string_column.getN()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); + + fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); + } + else + { + auto value = reader.as(); + if (value.size() > fixed_string_column.getN()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); + + fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); + } + } + + private: + String tmp_string; + capnp::Type capnp_type; + }; + + class CapnProtoIPv4Serializer : public ICapnProtoSerializer + { + public: + CapnProtoIPv4Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isUInt32()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(assert_cast(*column).getElement(row_num).toUnderType()); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(IPv4(reader.as())); + } + }; + + class CapnProtoIPv6Serializer : public ICapnProtoSerializer + { + public: + CapnProtoIPv6Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(IPv6)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of IPv6 value: {}", value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + }; + + class CapnProtoUUIDSerializer : public ICapnProtoSerializer + { + public: + CapnProtoUUIDSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(UUID)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of UUID value: {}", value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + }; + + std::unique_ptr createSerializer(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings); + + class CapnProtoLowCardinalitySerializer : public ICapnProtoSerializer + { + public: + CapnProtoLowCardinalitySerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + nested_serializer = createSerializer(assert_cast(*data_type).getDictionaryType(), column_name, capnp_type, settings); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + const auto & low_cardinality_column = assert_cast(*column); + size_t index = low_cardinality_column.getIndexAt(row_num); + const auto & dict_column = low_cardinality_column.getDictionary().getNestedColumn(); + return nested_serializer->writeRow(dict_column, field_builder, index); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto & low_cardinality_column = assert_cast(column); + auto tmp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); + nested_serializer->readRow(*tmp_column, reader); + low_cardinality_column.insertFromFullColumn(*tmp_column, 0); + } + + private: + std::unique_ptr nested_serializer; + }; + + class CapnProtoNullableSerializer : public ICapnProtoSerializer + { + public: + CapnProtoNullableSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + if (!capnp_type.isStruct()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type, got CapnProto type {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + /// Check that struct is a named union of type VOID and one arbitrary type. + auto struct_schema = capnp_type.asStruct(); + if (!checkIfStructIsNamedUnion(struct_schema)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type." + "Given CapnProto struct is not a named union: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + auto union_fields = struct_schema.getUnionFields(); + if (union_fields.size() != 2) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type." + "Given CapnProto union have more than 2 fields: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + auto first = union_fields[0]; + auto second = union_fields[1]; + auto nested_type = assert_cast(data_type.get())->getNestedType(); + if (first.getType().isVoid()) + { + null_field = first; + nested_field = second; + nested_capnp_type = second.getType(); + if (nested_capnp_type.isStruct()) + nested_fields_size = nested_capnp_type.asStruct().getFields().size(); + nested_serializer = createSerializer(nested_type, column_name, nested_capnp_type, settings); + } + else if (second.getType().isVoid()) + { + null_field = second; + nested_field = first; + nested_capnp_type = first.getType(); + if (nested_capnp_type.isStruct()) + nested_fields_size = nested_capnp_type.asStruct().getFields().size(); + nested_serializer = createSerializer(nested_type, column_name, nested_capnp_type, settings); + } + else + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type." + "Given CapnProto union doesn't have field with type Void: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + assert(field_builder); + auto & struct_builder = assert_cast(*field_builder); + const auto & nullable_column = assert_cast(*column); + if (nullable_column.isNullAt(row_num)) + { + struct_builder.impl.set(null_field, capnp::Void()); + } + else + { + struct_builder.impl.clear(nested_field); + const auto & nested_column = nullable_column.getNestedColumnPtr(); + auto nested_field_builder = initStructFieldBuilderIfNeeded(nested_column, row_num, struct_builder.impl, nested_field, nested_capnp_type, nested_fields_size); + auto value = nested_serializer->writeRow(nested_column, nested_field_builder.get(), row_num); + if (value) + struct_builder.impl.set(nested_field, *value); + } + + return std::nullopt; + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto struct_reader = reader.as(); + auto & nullable_column = assert_cast(column); + auto field = *kj::_::readMaybe(struct_reader.which()); + if (field.getType().isVoid()) + nullable_column.insertDefault(); + else + { + auto & nested_column = nullable_column.getNestedColumn(); + auto nested_reader = struct_reader.get(field); + nested_serializer->readRow(nested_column, nested_reader); + nullable_column.getNullMapData().push_back(0); + } + } + + private: + std::unique_ptr nested_serializer; + capnp::StructSchema::Field null_field; + capnp::StructSchema::Field nested_field; + size_t nested_fields_size = 0; + capnp::Type nested_capnp_type; + }; + + class CapnProtoArraySerializer : public ICapnProtoSerializer + { + public: + CapnProtoArraySerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + if (!capnp_type.isList()) + throwCannotConvert(data_type, column_name, capnp_type); + + auto nested_type = assert_cast(data_type.get())->getNestedType(); + element_type = capnp_type.asList().getElementType(); + if (element_type.isStruct()) + element_struct_fields = element_type.asStruct().getFields().size(); + nested_serializer = createSerializer(nested_type, column_name, capnp_type.asList().getElementType(), settings); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + assert(field_builder); + auto & list_builder = assert_cast(*field_builder); + const auto * array_column = assert_cast(column.get()); + const auto & nested_column = array_column->getDataPtr(); + const auto & offsets = array_column->getOffsets(); + auto offset = offsets[row_num - 1]; + size_t size = offsets[row_num] - offset; + bool need_nested_builders = list_builder.nested_builders.empty(); + for (unsigned i = 0; i != static_cast(size); ++i) + { + if (need_nested_builders) + { + /// For nested lists we need to initialize nested list builder. + if (element_type.isList()) + { + const auto & nested_offset = checkAndGetColumn(*nested_column)->getOffsets(); + size_t nested_array_size = nested_offset[offset + i] - nested_offset[offset + i - 1]; + list_builder.nested_builders.emplace_back(std::make_unique(list_builder.impl.init(i, static_cast(nested_array_size)))); + } + else if (element_type.isStruct()) + { + list_builder.nested_builders.emplace_back(std::make_unique(list_builder.impl[i], element_struct_fields)); + } + else + { + list_builder.nested_builders.emplace_back(); + } + } + + auto value = nested_serializer->writeRow(nested_column, list_builder.nested_builders[i].get(), offset + i); + if (value) + list_builder.impl.set(i, *value); + } + + return std::nullopt; + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto list_reader = reader.as(); + auto & column_array = assert_cast(column); + auto & offsets = column_array.getOffsets(); + offsets.push_back(offsets.back() + list_reader.size()); + + auto & nested_column = column_array.getData(); + for (const auto & nested_reader : list_reader) + nested_serializer->readRow(nested_column, nested_reader); + } + + private: + std::unique_ptr nested_serializer; + capnp::Type element_type; + size_t element_struct_fields; + }; + + class CapnProtoMapSerializer : public ICapnProtoSerializer + { + public: + CapnProtoMapSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + /// We output/input Map type as follow CapnProto schema + /// + /// struct Map { + /// struct Entry { + /// key @0: Key; + /// value @1: Value; + /// } + /// entries @0 :List(Entry); + /// } + + if (!capnp_type.isStruct()) + throwCannotConvert(data_type, column_name, capnp_type); + + auto struct_schema = capnp_type.asStruct(); + + if (checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto Struct with unnamed union {}", + column_name, + data_type->getName(), + getCapnProtoFullTypeName(capnp_type)); + + if (struct_schema.getFields().size() != 1) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": Map type can be represented as a Struct with one list field, got struct: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + const auto & field_type = struct_schema.getFields()[0].getType(); + if (!field_type.isList()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": Map type can be represented as a Struct with one list field, got field: {}", + column_name, + getCapnProtoFullTypeName(field_type)); + + auto list_element_type = field_type.asList().getElementType(); + if (!list_element_type.isStruct()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": Field of struct that represents Map should be a list of structs, got list of {}", + column_name, + getCapnProtoFullTypeName(list_element_type)); + + auto key_value_struct = list_element_type.asStruct(); + if (checkIfStructContainsUnnamedUnion(key_value_struct)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": struct that represents Map entries is unnamed union: {}", + column_name, + getCapnProtoFullTypeName(list_element_type)); + + if (key_value_struct.getFields().size() != 2) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": struct that represents Map entries should contain only 2 fields, got struct {}", + column_name, + getCapnProtoFullTypeName(list_element_type)); + + const auto & map_type = assert_cast(*data_type); + DataTypes types = {map_type.getKeyType(), map_type.getValueType()}; + Names names = {"key", "value"}; + auto entries_type = std::make_shared(std::make_shared(types, names)); + entries_field = struct_schema.getFields()[0]; + entries_capnp_type = entries_field.getType(); + nested_serializer = createSerializer(entries_type, column_name, field_type, settings); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + assert(field_builder); + auto & struct_builder = assert_cast(*field_builder); + const auto & entries_column = assert_cast(column.get())->getNestedColumnPtr(); + auto entries_builder = initStructFieldBuilderIfNeeded(entries_column, row_num, struct_builder.impl, entries_field, entries_capnp_type, 0); + nested_serializer->writeRow(entries_column, entries_builder.get(), row_num); + return std::nullopt; + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto struct_reader = reader.as(); + auto & entries_column = assert_cast(column).getNestedColumn(); + nested_serializer->readRow(entries_column, struct_reader.get(entries_field)); + } + + private: + std::unique_ptr nested_serializer; + capnp::StructSchema::Field entries_field; + capnp::Type entries_capnp_type; + }; + + class CapnProtoStructureSerializer : public ICapnProtoSerializer + { + public: + CapnProtoStructureSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + { + if (checkIfStructIsNamedUnion(schema) || checkIfStructContainsUnnamedUnion(schema)) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Root CapnProto Struct cannot be named union/struct with unnamed union"); + + initialize(data_types, names, schema, settings); + } + + CapnProtoStructureSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + if (!capnp_type.isStruct()) + throwCannotConvert(data_type, column_name, capnp_type); + + auto struct_schema = capnp_type.asStruct(); + + if (checkIfStructIsNamedUnion(struct_schema) || checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto named union/struct with unnamed union {}", + column_name, + data_type->getName(), + getCapnProtoFullTypeName(capnp_type)); + + const auto * tuple_data_type = assert_cast(data_type.get()); + auto nested_types = tuple_data_type->getElements(); + Names nested_names; + bool have_explicit_names = tuple_data_type->haveExplicitNames(); + auto structure_fields = struct_schema.getFields(); + if (!have_explicit_names) + { + if (nested_types.size() != structure_fields.size()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto type {}: Tuple and Struct have different sizes {} != {}", + column_name, + data_type->getName(), + getCapnProtoFullTypeName(capnp_type), + nested_types.size(), + structure_fields.size()); + nested_names.reserve(structure_fields.size()); + for (auto field : structure_fields) + nested_names.push_back(field.getProto().getName()); + } + else + { + nested_names = tuple_data_type->getElementNames(); + } + + try + { + initialize(nested_types, nested_names, struct_schema, settings); + } + catch (Exception & e) + { + e.addMessage("(while converting column {})", column_name); + throw e; + } + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * builder, size_t row_num) override + { + assert(builder); + auto & struct_builder = assert_cast(*builder); + if (auto tuple_column = typeid_cast(column.get())) + writeRow(tuple_column->getColumnsCopy(), struct_builder, row_num); + else + writeRow(Columns{column}, struct_builder, row_num); + return std::nullopt; + } + + void writeRow(const Columns & columns, StructBuilder & struct_builder, size_t row_num) + { + for (size_t i = 0; i != columns.size(); ++i) + { + const auto & field = fields[i]; + size_t field_index = field.getIndex(); + if (likely(!struct_builder.field_builders[field_index])) + struct_builder.field_builders[field_index] = initStructFieldBuilderIfNeeded( + columns[i], row_num, struct_builder.impl, field, fields_types[i], nested_field_sizes[i]); + + auto value = field_serializers[i]->writeRow(columns[i], struct_builder.field_builders[field_index].get(), row_num); + if (value) + struct_builder.impl.set(field, *value); + } + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto struct_reader = reader.as(); + if (auto * tuple_column = typeid_cast(&column)) + { + for (size_t i = 0; i != tuple_column->tupleSize(); ++i) + field_serializers[i]->readRow(tuple_column->getColumn(i), struct_reader.get(fields[i])); + } + else + field_serializers[0]->readRow(column, struct_reader.get(fields[0])); + } + + void readRow(MutableColumns & columns, const capnp::DynamicStruct::Reader & reader) + { + for (size_t i = 0; i != columns.size(); ++i) + field_serializers[i]->readRow(*columns[i], reader.get(fields[i])); + } + + private: + void initialize(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + { + field_serializers.reserve(data_types.size()); + fields.reserve(data_types.size()); + fields_types.reserve(data_types.size()); + nested_field_sizes.reserve(data_types.size()); + for (size_t i = 0; i != data_types.size(); ++i) + { + auto [field_name, _] = splitFieldName(names[i]); + auto field = findFieldByName(schema, field_name); + if (!field) + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto schema doesn't contain field with name {}", field_name); + + fields.push_back(*field); + auto capnp_type = field->getType(); + fields_types.push_back(capnp_type); + nested_field_sizes.push_back(capnp_type.isStruct() ? capnp_type.asStruct().getFields().size() : 0); + field_serializers.push_back(createSerializer(data_types[i], names[i], capnp_type, settings)); + } + } + + std::vector> field_serializers; + std::vector fields; + std::vector nested_field_sizes; + std::vector fields_types; + }; + + std::unique_ptr createSerializer(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + auto [field_name, nested_name] = splitFieldName(name); + if (!nested_name.empty() && !capnp_type.isList()) + { + if (!capnp_type.isStruct()) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); + + return std::make_unique(DataTypes{type}, Names{nested_name}, capnp_type.asStruct(), settings); + } + + switch (type->getTypeId()) + { + case TypeIndex::Int8: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt8: + if (isBool(type)) + return createIntegerSerializer(type, name, capnp_type); + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int16: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt16: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int32: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt32: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int64: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt64: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int128: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::UInt128: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Int256: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::UInt256: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Float32: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Float64: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Date: + return std::make_unique(type, name, capnp_type); + case TypeIndex::Date32: + return std::make_unique(type, name, capnp_type); + case TypeIndex::DateTime: + return std::make_unique(type, name, capnp_type); + case TypeIndex::DateTime64: + return std::make_unique(type, name, capnp_type); + case TypeIndex::Decimal32: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Decimal64: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Decimal128: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Decimal256: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::IPv4: + return std::make_unique(type, name, capnp_type); + case TypeIndex::IPv6: + return std::make_unique(type, name, capnp_type); + case TypeIndex::UUID: + return std::make_unique(type, name, capnp_type); + case TypeIndex::Enum8: + return std::make_unique>(type, name, capnp_type, settings.enum_comparing_mode); + case TypeIndex::Enum16: + return std::make_unique>(type, name, capnp_type, settings.enum_comparing_mode); + case TypeIndex::String: + if (capnp_type.isData()) + return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); + case TypeIndex::FixedString: + if (capnp_type.isData()) + return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); + case TypeIndex::LowCardinality: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Nullable: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Array: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Map: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Tuple: + return std::make_unique(type, name, capnp_type, settings); + default: + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Type {} is not supported in CapnProto format", type->getName()); + } + } +} + +class CapnProtoSerializer::Impl +{ +public: + Impl(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + : struct_serializer(std::make_unique(data_types, names, schema, settings)) + , fields_size(schema.getFields().size()) + { + } + + void writeRow(const Columns & columns, capnp::DynamicStruct::Builder builder, size_t row_num) + { + StructBuilder struct_builder(std::move(builder), fields_size); + struct_serializer->writeRow(columns, struct_builder, row_num); + } + + void readRow(MutableColumns & columns, capnp::DynamicStruct::Reader & reader) + { + struct_serializer->readRow(columns, reader); + } + +private: + std::unique_ptr struct_serializer; + size_t fields_size; +}; + +CapnProtoSerializer::CapnProtoSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + : serializer_impl(std::make_unique(data_types, names, schema, settings)) +{ +} + +void CapnProtoSerializer::writeRow(const Columns & columns, capnp::DynamicStruct::Builder builder, size_t row_num) +{ + serializer_impl->writeRow(columns, std::move(builder), row_num); +} + +void CapnProtoSerializer::readRow(MutableColumns & columns, capnp::DynamicStruct::Reader & reader) +{ + serializer_impl->readRow(columns, reader); +} + +CapnProtoSerializer::~CapnProtoSerializer() = default; + +} diff --git a/src/Formats/CapnProtoSerializer.h b/src/Formats/CapnProtoSerializer.h new file mode 100644 index 00000000000..efae797875b --- /dev/null +++ b/src/Formats/CapnProtoSerializer.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class CapnProtoSerializer +{ +public: + CapnProtoSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings); + + void writeRow(const Columns & columns, capnp::DynamicStruct::Builder builder, size_t row_num); + + void readRow(MutableColumns & columns, capnp::DynamicStruct::Reader & reader); + + ~CapnProtoSerializer(); + +private: + class Impl; + std::unique_ptr serializer_impl; +}; + +} diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp deleted file mode 100644 index d6c032408bb..00000000000 --- a/src/Formats/CapnProtoUtils.cpp +++ /dev/null @@ -1,734 +0,0 @@ -#include - -#if USE_CAPNP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_CAPN_PROTO_SCHEMA; - extern const int THERE_IS_NO_COLUMN; - extern const int BAD_TYPE_OF_FIELD; - extern const int CAPN_PROTO_BAD_CAST; - extern const int FILE_DOESNT_EXIST; - extern const int UNKNOWN_EXCEPTION; - extern const int INCORRECT_DATA; - extern const int CAPN_PROTO_BAD_TYPE; - extern const int BAD_ARGUMENTS; -} - -std::pair splitCapnProtoFieldName(const String & name) -{ - const auto * begin = name.data(); - const auto * end = name.data() + name.size(); - const auto * it = find_first_symbols<'_', '.'>(begin, end); - String first = String(begin, it); - String second = it == end ? "" : String(it + 1, end); - return {first, second}; -} - -capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info) -{ - capnp::ParsedSchema schema; - try - { - int fd; - KJ_SYSCALL(fd = open(schema_info.schemaDirectory().data(), O_RDONLY)); // NOLINT(bugprone-suspicious-semicolon) - auto schema_dir = kj::newDiskDirectory(kj::OsFileHandle(fd)); - schema = impl.parseFromDirectory(*schema_dir, kj::Path::parse(schema_info.schemaPath()), {}); - } - catch (const kj::Exception & e) - { - /// That's not good to determine the type of error by its description, but - /// this is the only way to do it here, because kj doesn't specify the type of error. - auto description = std::string_view(e.getDescription().cStr()); - if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); - - if (description.find("Parse error") != String::npos) - throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "Cannot parse CapnProto schema {}:{}", schema_info.schemaPath(), e.getLine()); - - throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, - "Unknown exception while parsing CapnProto schema: {}, schema dir and file: {}, {}", - description, schema_info.schemaDirectory(), schema_info.schemaPath()); - } - - auto message_maybe = schema.findNested(schema_info.messageName()); - auto * message_schema = kj::_::readMaybe(message_maybe); - if (!message_schema) - throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, - "CapnProto schema doesn't contain message with name {}", schema_info.messageName()); - return message_schema->asStruct(); -} - -bool compareEnumNames(const String & first, const String & second, FormatSettings::EnumComparingMode mode) -{ - if (mode == FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE) - return boost::algorithm::to_lower_copy(first) == boost::algorithm::to_lower_copy(second); - return first == second; -} - -static const std::map capnp_simple_type_names = -{ - {capnp::schema::Type::Which::BOOL, "Bool"}, - {capnp::schema::Type::Which::VOID, "Void"}, - {capnp::schema::Type::Which::INT8, "Int8"}, - {capnp::schema::Type::Which::INT16, "Int16"}, - {capnp::schema::Type::Which::INT32, "Int32"}, - {capnp::schema::Type::Which::INT64, "Int64"}, - {capnp::schema::Type::Which::UINT8, "UInt8"}, - {capnp::schema::Type::Which::UINT16, "UInt16"}, - {capnp::schema::Type::Which::UINT32, "UInt32"}, - {capnp::schema::Type::Which::UINT64, "UInt64"}, - {capnp::schema::Type::Which::FLOAT32, "Float32"}, - {capnp::schema::Type::Which::FLOAT64, "Float64"}, - {capnp::schema::Type::Which::TEXT, "Text"}, - {capnp::schema::Type::Which::DATA, "Data"}, - {capnp::schema::Type::Which::INTERFACE, "Interface"}, - {capnp::schema::Type::Which::ANY_POINTER, "AnyPointer"}, -}; - -static bool checkIfStructContainsUnnamedUnion(const capnp::StructSchema & struct_schema) -{ - return struct_schema.getFields().size() != struct_schema.getNonUnionFields().size(); -} - -static bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema) -{ - return struct_schema.getFields().size() == struct_schema.getUnionFields().size(); -} - -/// Get full name of type for better exception messages. -static String getCapnProtoFullTypeName(const capnp::Type & type) -{ - switch (type.which()) - { - case capnp::schema::Type::Which::STRUCT: - { - auto struct_schema = type.asStruct(); - - auto non_union_fields = struct_schema.getNonUnionFields(); - std::vector non_union_field_names; - for (auto nested_field : non_union_fields) - non_union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); - - auto union_fields = struct_schema.getUnionFields(); - std::vector union_field_names; - for (auto nested_field : union_fields) - union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); - - String union_name = "Union(" + boost::algorithm::join(union_field_names, ", ") + ")"; - /// Check if the struct is a named union. - if (non_union_field_names.empty()) - return union_name; - - String type_name = "Struct(" + boost::algorithm::join(non_union_field_names, ", "); - /// Check if the struct contains unnamed union. - if (!union_field_names.empty()) - type_name += ", " + union_name; - type_name += ")"; - return type_name; - } - case capnp::schema::Type::Which::LIST: - return "List(" + getCapnProtoFullTypeName(type.asList().getElementType()) + ")"; - case capnp::schema::Type::Which::ENUM: - { - auto enum_schema = type.asEnum(); - String enum_name = "Enum("; - auto enumerants = enum_schema.getEnumerants(); - for (unsigned i = 0; i != enumerants.size(); ++i) - { - enum_name += String(enumerants[i].getProto().getName()) + " = " + std::to_string(enumerants[i].getOrdinal()); - if (i + 1 != enumerants.size()) - enum_name += ", "; - } - enum_name += ")"; - return enum_name; - } - default: - auto it = capnp_simple_type_names.find(type.which()); - if (it == capnp_simple_type_names.end()) - throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Unknown CapnProto type"); - return it->second; - } -} - -template -static bool checkEnums(const capnp::Type & capnp_type, const DataTypePtr column_type, FormatSettings::EnumComparingMode mode, UInt64 max_value, String & error_message) -{ - if (!capnp_type.isEnum()) - return false; - - auto enum_schema = capnp_type.asEnum(); - bool to_lower = mode == FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE; - const auto * enum_type = assert_cast *>(column_type.get()); - const auto & enum_values = dynamic_cast &>(*enum_type); - - auto enumerants = enum_schema.getEnumerants(); - if (mode == FormatSettings::EnumComparingMode::BY_VALUES) - { - /// In CapnProto Enum fields are numbered sequentially starting from zero. - if (enumerants.size() > max_value) - { - error_message += "Enum from CapnProto schema contains values that is out of range for Clickhouse Enum"; - return false; - } - - auto values = enum_values.getSetOfAllValues(); - std::unordered_set capn_enum_values; - for (auto enumerant : enumerants) - capn_enum_values.insert(Type(enumerant.getOrdinal())); - auto result = values == capn_enum_values; - if (!result) - error_message += "The set of values in Enum from CapnProto schema is different from the set of values in ClickHouse Enum"; - return result; - } - - auto names = enum_values.getSetOfAllNames(to_lower); - std::unordered_set capn_enum_names; - - for (auto enumerant : enumerants) - { - String name = enumerant.getProto().getName(); - capn_enum_names.insert(to_lower ? boost::algorithm::to_lower_copy(name) : name); - } - - auto result = names == capn_enum_names; - if (!result) - error_message += "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"; - return result; -} - -static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name); - -static bool checkNullableType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name) -{ - if (!capnp_type.isStruct()) - return false; - - /// Check that struct is a named union of type VOID and one arbitrary type. - auto struct_schema = capnp_type.asStruct(); - if (!checkIfStructIsNamedUnion(struct_schema)) - return false; - - auto union_fields = struct_schema.getUnionFields(); - if (union_fields.size() != 2) - return false; - - auto first = union_fields[0]; - auto second = union_fields[1]; - - auto nested_type = assert_cast(data_type.get())->getNestedType(); - if (first.getType().isVoid()) - return checkCapnProtoType(second.getType(), nested_type, mode, error_message, column_name); - if (second.getType().isVoid()) - return checkCapnProtoType(first.getType(), nested_type, mode, error_message, column_name); - return false; -} - -static bool checkTupleType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message) -{ - if (!capnp_type.isStruct()) - return false; - auto struct_schema = capnp_type.asStruct(); - - if (checkIfStructIsNamedUnion(struct_schema)) - return false; - - if (checkIfStructContainsUnnamedUnion(struct_schema)) - { - error_message += "CapnProto struct contains unnamed union"; - return false; - } - - const auto * tuple_data_type = assert_cast(data_type.get()); - auto nested_types = tuple_data_type->getElements(); - if (nested_types.size() != struct_schema.getFields().size()) - { - error_message += "Tuple and Struct types have different sizes"; - return false; - } - - bool have_explicit_names = tuple_data_type->haveExplicitNames(); - const auto & nested_names = tuple_data_type->getElementNames(); - for (uint32_t i = 0; i != nested_names.size(); ++i) - { - if (have_explicit_names) - { - KJ_IF_MAYBE (field, struct_schema.findFieldByName(nested_names[i])) - { - if (!checkCapnProtoType(field->getType(), nested_types[tuple_data_type->getPositionByName(nested_names[i])], mode, error_message, nested_names[i])) - return false; - } - else - { - error_message += "CapnProto struct doesn't contain a field with name " + nested_names[i]; - return false; - } - } - else if (!checkCapnProtoType(struct_schema.getFields()[i].getType(), nested_types[tuple_data_type->getPositionByName(nested_names[i])], mode, error_message, nested_names[i])) - return false; - } - - return true; -} - -static bool checkArrayType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name) -{ - if (!capnp_type.isList()) - return false; - auto list_schema = capnp_type.asList(); - auto nested_type = assert_cast(data_type.get())->getNestedType(); - - auto [field_name, nested_name] = splitCapnProtoFieldName(column_name); - if (!nested_name.empty() && list_schema.getElementType().isStruct()) - { - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(field, struct_schema.findFieldByName(nested_name)) - return checkCapnProtoType(field->getType(), nested_type, mode, error_message, nested_name); - - error_message += "Element type of List {} doesn't contain field with name " + nested_name; - return false; - } - - return checkCapnProtoType(list_schema.getElementType(), nested_type, mode, error_message, column_name); -} - -static bool checkMapType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message) -{ - /// We output/input Map type as follow CapnProto schema - /// - /// struct Map { - /// struct Entry { - /// key @0: Key; - /// value @1: Value; - /// } - /// entries @0 :List(Entry); - /// } - - if (!capnp_type.isStruct()) - return false; - auto struct_schema = capnp_type.asStruct(); - - if (checkIfStructContainsUnnamedUnion(struct_schema)) - { - error_message += "CapnProto struct contains unnamed union"; - return false; - } - - if (struct_schema.getFields().size() != 1) - { - error_message += "CapnProto struct that represents Map type can contain only one field"; - return false; - } - - const auto & field_type = struct_schema.getFields()[0].getType(); - if (!field_type.isList()) - { - error_message += "Field of CapnProto struct that represents Map is not a list"; - return false; - } - - auto list_element_type = field_type.asList().getElementType(); - if (!list_element_type.isStruct()) - { - error_message += "Field of CapnProto struct that represents Map is not a list of structs"; - return false; - } - - auto key_value_struct = list_element_type.asStruct(); - if (checkIfStructContainsUnnamedUnion(key_value_struct)) - { - error_message += "CapnProto struct contains unnamed union"; - return false; - } - - if (key_value_struct.getFields().size() != 2) - { - error_message += "Key-value structure for Map struct should have exactly 2 fields"; - return false; - } - - const auto & map_type = assert_cast(*data_type); - DataTypes types = {map_type.getKeyType(), map_type.getValueType()}; - Names names = {"key", "value"}; - - for (size_t i = 0; i != types.size(); ++i) - { - KJ_IF_MAYBE(field, key_value_struct.findFieldByName(names[i])) - { - if (!checkCapnProtoType(field->getType(), types[i], mode, error_message, names[i])) - return false; - } - else - { - error_message += R"(Key-value structure for Map struct should have exactly 2 fields with names "key" and "value")"; - return false; - } - } - - return true; -} - -static bool isCapnInteger(const capnp::Type & capnp_type) -{ - return capnp_type.isInt8() || capnp_type.isUInt8() || capnp_type.isInt16() || capnp_type.isUInt16() || capnp_type.isInt32() - || capnp_type.isUInt32() || capnp_type.isInt64() || capnp_type.isUInt64(); -} - -static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name) -{ - switch (data_type->getTypeId()) - { - case TypeIndex::UInt8: - return capnp_type.isBool() || isCapnInteger(capnp_type); - case TypeIndex::Int8: [[fallthrough]]; - case TypeIndex::Int16: [[fallthrough]]; - case TypeIndex::UInt16: [[fallthrough]]; - case TypeIndex::Int32: [[fallthrough]]; - case TypeIndex::UInt32: [[fallthrough]]; - case TypeIndex::Int64: [[fallthrough]]; - case TypeIndex::UInt64: - /// Allow integer conversions durin input/output. - return isCapnInteger(capnp_type); - case TypeIndex::Date: - return capnp_type.isUInt16(); - case TypeIndex::DateTime: [[fallthrough]]; - case TypeIndex::IPv4: - return capnp_type.isUInt32(); - case TypeIndex::Date32: [[fallthrough]]; - case TypeIndex::Decimal32: - return capnp_type.isInt32() || capnp_type.isUInt32(); - case TypeIndex::DateTime64: [[fallthrough]]; - case TypeIndex::Decimal64: - return capnp_type.isInt64() || capnp_type.isUInt64(); - case TypeIndex::Float32:[[fallthrough]]; - case TypeIndex::Float64: - /// Allow converting between Float32 and isFloat64 - return capnp_type.isFloat32() || capnp_type.isFloat64(); - case TypeIndex::Enum8: - return checkEnums(capnp_type, data_type, mode, INT8_MAX, error_message); - case TypeIndex::Enum16: - return checkEnums(capnp_type, data_type, mode, INT16_MAX, error_message); - case TypeIndex::Int128: [[fallthrough]]; - case TypeIndex::UInt128: [[fallthrough]]; - case TypeIndex::Int256: [[fallthrough]]; - case TypeIndex::UInt256: [[fallthrough]]; - case TypeIndex::Decimal128: [[fallthrough]]; - case TypeIndex::Decimal256: - return capnp_type.isData(); - case TypeIndex::Tuple: - return checkTupleType(capnp_type, data_type, mode, error_message); - case TypeIndex::Nullable: - { - auto result = checkNullableType(capnp_type, data_type, mode, error_message, column_name); - if (!result) - error_message += "Nullable can be represented only as a named union of type Void and nested type"; - return result; - } - case TypeIndex::Array: - return checkArrayType(capnp_type, data_type, mode, error_message, column_name); - case TypeIndex::LowCardinality: - return checkCapnProtoType(capnp_type, assert_cast(data_type.get())->getDictionaryType(), mode, error_message, column_name); - case TypeIndex::FixedString: [[fallthrough]]; - case TypeIndex::IPv6: [[fallthrough]]; - case TypeIndex::String: - return capnp_type.isText() || capnp_type.isData(); - case TypeIndex::Map: - return checkMapType(capnp_type, data_type, mode, error_message); - default: - return false; - } -} - -capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Reader & struct_reader, const String & name) -{ - auto [field_name, nested_name] = splitCapnProtoFieldName(name); - KJ_IF_MAYBE(field, struct_reader.getSchema().findFieldByName(field_name)) - { - capnp::DynamicValue::Reader field_reader; - try - { - field_reader = struct_reader.get(*field); - } - catch (const kj::Exception & e) - { - throw Exception(ErrorCodes::INCORRECT_DATA, - "Cannot extract field value from struct by provided schema, error: " - "{} Perhaps the data was generated by another schema", String(e.getDescription().cStr())); - } - - if (nested_name.empty()) - return field_reader; - - /// Support reading Nested as List of Structs. - if (field_reader.getType() == capnp::DynamicValue::LIST) - { - auto list_schema = field->getType().asList(); - if (!list_schema.getElementType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Element type of List {} is not a struct", field_name); - - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(nested_field, struct_schema.findFieldByName(nested_name)) - return field_reader; - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Element type of List {} doesn't contain field with name \"{}\"", field_name, nested_name); - } - - if (field_reader.getType() != capnp::DynamicValue::STRUCT) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); - - return getReaderByColumnName(field_reader.as(), nested_name); - } - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto struct doesn't contain field with name {}", field_name); -} - -std::pair getStructBuilderAndFieldByColumnName(capnp::DynamicStruct::Builder struct_builder, const String & name) -{ - auto [field_name, nested_name] = splitCapnProtoFieldName(name); - KJ_IF_MAYBE(field, struct_builder.getSchema().findFieldByName(field_name)) - { - if (nested_name.empty()) - return {struct_builder, *field}; - - auto field_builder = struct_builder.get(*field); - - /// Support reading Nested as List of Structs. - if (field_builder.getType() == capnp::DynamicValue::LIST) - { - auto list_schema = field->getType().asList(); - if (!list_schema.getElementType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Element type of List {} is not a struct", field_name); - - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(nested_field, struct_schema.findFieldByName(nested_name)) - return {struct_builder, *field}; - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Element type of List {} doesn't contain field with name \"{}\"", field_name, nested_name); - } - - if (field_builder.getType() != capnp::DynamicValue::STRUCT) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); - - return getStructBuilderAndFieldByColumnName(field_builder.as(), nested_name); - } - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto struct doesn't contain field with name {}", field_name); -} - -static std::pair getFieldByName(const capnp::StructSchema & schema, const String & name) -{ - auto [field_name, nested_name] = splitCapnProtoFieldName(name); - KJ_IF_MAYBE(field, schema.findFieldByName(field_name)) - { - if (nested_name.empty()) - return {*field, name}; - - /// Support reading Nested as List of Structs. - if (field->getType().isList()) - { - auto list_schema = field->getType().asList(); - if (!list_schema.getElementType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Element type of List {} is not a struct", field_name); - - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(nested_field, struct_schema.findFieldByName(nested_name)) - return {*field, name}; - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Element type of List {} doesn't contain field with name \"{}\"", field_name, nested_name); - } - - if (!field->getType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); - - return getFieldByName(field->getType().asStruct(), nested_name); - } - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto schema doesn't contain field with name {}", field_name); -} - -void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode) -{ - /// Firstly check that struct doesn't contain unnamed union, because we don't support it. - if (checkIfStructContainsUnnamedUnion(schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Schema contains unnamed union that is not supported"); - auto names_and_types = header.getNamesAndTypesList(); - String additional_error_message; - for (auto & [name, type] : names_and_types) - { - auto [field, field_name] = getFieldByName(schema, name); - if (!checkCapnProtoType(field.getType(), type, mode, additional_error_message, field_name)) - { - auto e = Exception( - ErrorCodes::CAPN_PROTO_BAD_CAST, - "Cannot convert ClickHouse type {} to CapnProto type {}", - type->getName(), - getCapnProtoFullTypeName(field.getType())); - if (!additional_error_message.empty()) - e.addMessage(additional_error_message); - throw std::move(e); - } - } -} - -template -static DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) -{ - std::vector> values; - for (auto enumerant : enumerants) - values.emplace_back(enumerant.getProto().getName(), ValueType(enumerant.getOrdinal())); - return std::make_shared>(std::move(values)); -} - -static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) -{ - auto enumerants = enum_schema.getEnumerants(); - if (enumerants.size() < 128) - return getEnumDataTypeFromEnumerants(enumerants); - if (enumerants.size() < 32768) - return getEnumDataTypeFromEnumerants(enumerants); - - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "ClickHouse supports only 8 and 16-bit Enums"); -} - -static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) -{ - switch (capnp_type.which()) - { - case capnp::schema::Type::INT8: - return std::make_shared(); - case capnp::schema::Type::INT16: - return std::make_shared(); - case capnp::schema::Type::INT32: - return std::make_shared(); - case capnp::schema::Type::INT64: - return std::make_shared(); - case capnp::schema::Type::BOOL: [[fallthrough]]; - case capnp::schema::Type::UINT8: - return std::make_shared(); - case capnp::schema::Type::UINT16: - return std::make_shared(); - case capnp::schema::Type::UINT32: - return std::make_shared(); - case capnp::schema::Type::UINT64: - return std::make_shared(); - case capnp::schema::Type::FLOAT32: - return std::make_shared(); - case capnp::schema::Type::FLOAT64: - return std::make_shared(); - case capnp::schema::Type::DATA: [[fallthrough]]; - case capnp::schema::Type::TEXT: - return std::make_shared(); - case capnp::schema::Type::ENUM: - return getEnumDataTypeFromEnumSchema(capnp_type.asEnum()); - case capnp::schema::Type::LIST: - { - auto list_schema = capnp_type.asList(); - auto nested_type = getDataTypeFromCapnProtoType(list_schema.getElementType(), skip_unsupported_fields); - if (!nested_type) - return nullptr; - return std::make_shared(nested_type); - } - case capnp::schema::Type::STRUCT: - { - auto struct_schema = capnp_type.asStruct(); - - - if (struct_schema.getFields().size() == 0) - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Empty messages are not supported"); - } - - /// Check if it can be Nullable. - if (checkIfStructIsNamedUnion(struct_schema)) - { - auto fields = struct_schema.getUnionFields(); - if (fields.size() != 2 || (!fields[0].getType().isVoid() && !fields[1].getType().isVoid())) - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unions are not supported"); - } - auto value_type = fields[0].getType().isVoid() ? fields[1].getType() : fields[0].getType(); - if (value_type.isStruct() || value_type.isList()) - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Tuples and Lists cannot be inside Nullable"); - } - - auto nested_type = getDataTypeFromCapnProtoType(value_type, skip_unsupported_fields); - if (!nested_type) - return nullptr; - return std::make_shared(nested_type); - } - - if (checkIfStructContainsUnnamedUnion(struct_schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); - - /// Treat Struct as Tuple. - DataTypes nested_types; - Names nested_names; - for (auto field : struct_schema.getNonUnionFields()) - { - auto nested_type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); - if (!nested_type) - continue; - nested_names.push_back(field.getProto().getName()); - nested_types.push_back(nested_type); - } - if (nested_types.empty()) - return nullptr; - return std::make_shared(std::move(nested_types), std::move(nested_names)); - } - default: - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unsupported CapnProtoType: {}", getCapnProtoFullTypeName(capnp_type)); - } - } -} - -NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields) -{ - if (checkIfStructContainsUnnamedUnion(schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); - - NamesAndTypesList names_and_types; - for (auto field : schema.getNonUnionFields()) - { - auto name = field.getProto().getName(); - auto type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); - if (type) - names_and_types.emplace_back(name, type); - } - if (names_and_types.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Cannot convert CapnProto schema to ClickHouse table schema, all fields have unsupported types"); - - return names_and_types; -} - -} - -#endif diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index c88af650671..475d08e0fe3 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -325,16 +325,16 @@ struct FormatSettings /// For capnProto format we should determine how to /// compare ClickHouse Enum and Enum from schema. - enum class EnumComparingMode + enum class CapnProtoEnumComparingMode { BY_NAMES, // Names in enums should be the same, values can be different. BY_NAMES_CASE_INSENSITIVE, // Case-insensitive name comparison. BY_VALUES, // Values should be the same, names can be different. }; - struct + struct CapnProto { - EnumComparingMode enum_comparing_mode = EnumComparingMode::BY_VALUES; + CapnProtoEnumComparingMode enum_comparing_mode = CapnProtoEnumComparingMode::BY_VALUES; bool skip_fields_with_unsupported_types_in_schema_inference = false; } capn_proto; diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index 2f84e9bde3c..e686ae86997 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -9,23 +9,6 @@ #include #include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include - namespace DB { @@ -35,16 +18,14 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } -CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_) - : IRowInputFormat(std::move(header), in_, std::move(params_)) +CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings) + : IRowInputFormat(std::move(header_), in_, std::move(params_)) , parser(std::make_shared()) - , format_settings(format_settings_) - , column_types(getPort().getHeader().getDataTypes()) - , column_names(getPort().getHeader().getNames()) { // Parse the schema and fetch the root object - root = parser->getMessageSchema(info); - checkCapnProtoSchemaStructure(root, getPort().getHeader(), format_settings.capn_proto.enum_comparing_mode); + schema = parser->getMessageSchema(info); + const auto & header = getPort().getHeader(); + serializer = std::make_unique(header.getDataTypes(), header.getNames(), schema, format_settings.capn_proto); } kj::Array CapnProtoRowInputFormat::readMessage() @@ -82,213 +63,6 @@ kj::Array CapnProtoRowInputFormat::readMessage() return msg; } -static void insertInteger(IColumn & column, const DataTypePtr & column_type, UInt64 value) -{ - switch (column_type->getTypeId()) - { - case TypeIndex::Int8: - assert_cast(column).insertValue(value); - break; - case TypeIndex::UInt8: - assert_cast(column).insertValue(value); - break; - case TypeIndex::Int16: - assert_cast(column).insertValue(value); - break; - case TypeIndex::Date: [[fallthrough]]; - case TypeIndex::UInt16: - assert_cast(column).insertValue(value); - break; - case TypeIndex::Int32: - assert_cast(column).insertValue(static_cast(value)); - break; - case TypeIndex::DateTime: [[fallthrough]]; - case TypeIndex::UInt32: - assert_cast(column).insertValue(static_cast(value)); - break; - case TypeIndex::IPv4: - assert_cast(column).insertValue(IPv4(static_cast(value))); - break; - case TypeIndex::Int64: - assert_cast(column).insertValue(value); - break; - case TypeIndex::UInt64: - assert_cast(column).insertValue(value); - break; - case TypeIndex::DateTime64: - assert_cast &>(column).insertValue(value); - break; - case TypeIndex::Decimal32: - assert_cast &>(column).insertValue(static_cast(value)); - break; - case TypeIndex::Decimal64: - assert_cast &>(column).insertValue(value); - break; - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type {} cannot be parsed from integer", column_type->getName()); - } -} - -static void insertFloat(IColumn & column, const DataTypePtr & column_type, Float64 value) -{ - switch (column_type->getTypeId()) - { - case TypeIndex::Float32: - assert_cast(column).insertValue(static_cast(value)); - break; - case TypeIndex::Float64: - assert_cast(column).insertValue(value); - break; - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type is not a float."); - } -} - -template -static void insertData(IColumn & column, const DataTypePtr & column_type, Value value) -{ - if (column_type->haveMaximumSizeOfValue() && value.size() != column_type->getSizeOfValueInMemory()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", column_type->getName(), value.size()); - - column.insertData(reinterpret_cast(value.begin()), value.size()); -} - -template -static void insertEnum(IColumn & column, const DataTypePtr & column_type, const capnp::DynamicEnum & enum_value, FormatSettings::EnumComparingMode enum_comparing_mode) -{ - auto enumerant = *kj::_::readMaybe(enum_value.getEnumerant()); - auto enum_type = assert_cast *>(column_type.get()); - DataTypePtr nested_type = std::make_shared>(); - switch (enum_comparing_mode) - { - case FormatSettings::EnumComparingMode::BY_VALUES: - insertInteger(column, nested_type, Int64(enumerant.getOrdinal())); - return; - case FormatSettings::EnumComparingMode::BY_NAMES: - insertInteger(column, nested_type, Int64(enum_type->getValue(String(enumerant.getProto().getName())))); - return; - case FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE: - { - /// Find the same enum name case insensitive. - String enum_name = enumerant.getProto().getName(); - for (auto & name : enum_type->getAllRegisteredNames()) - { - if (compareEnumNames(name, enum_name, enum_comparing_mode)) - { - insertInteger(column, nested_type, Int64(enum_type->getValue(name))); - break; - } - } - } - } -} - -static void insertValue(IColumn & column, const DataTypePtr & column_type, const String & column_name, const capnp::DynamicValue::Reader & value, FormatSettings::EnumComparingMode enum_comparing_mode) -{ - if (column_type->lowCardinality()) - { - auto & lc_column = assert_cast(column); - auto tmp_column = lc_column.getDictionary().getNestedColumn()->cloneEmpty(); - auto dict_type = assert_cast(column_type.get())->getDictionaryType(); - insertValue(*tmp_column, dict_type, column_name, value, enum_comparing_mode); - lc_column.insertFromFullColumn(*tmp_column, 0); - return; - } - - switch (value.getType()) - { - case capnp::DynamicValue::Type::INT: - insertInteger(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::UINT: - insertInteger(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::FLOAT: - insertFloat(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::BOOL: - insertInteger(column, column_type, UInt64(value.as())); - break; - case capnp::DynamicValue::Type::DATA: - insertData(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::TEXT: - insertData(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::ENUM: - if (column_type->getTypeId() == TypeIndex::Enum8) - insertEnum(column, column_type, value.as(), enum_comparing_mode); - else - insertEnum(column, column_type, value.as(), enum_comparing_mode); - break; - case capnp::DynamicValue::LIST: - { - auto list_value = value.as(); - auto & column_array = assert_cast(column); - auto & offsets = column_array.getOffsets(); - offsets.push_back(offsets.back() + list_value.size()); - - auto & nested_column = column_array.getData(); - auto nested_type = assert_cast(column_type.get())->getNestedType(); - for (const auto & nested_value : list_value) - insertValue(nested_column, nested_type, column_name, nested_value, enum_comparing_mode); - break; - } - case capnp::DynamicValue::Type::STRUCT: - { - auto struct_value = value.as(); - if (column_type->isNullable()) - { - auto & nullable_column = assert_cast(column); - auto field = *kj::_::readMaybe(struct_value.which()); - if (field.getType().isVoid()) - nullable_column.insertDefault(); - else - { - auto & nested_column = nullable_column.getNestedColumn(); - auto nested_type = assert_cast(column_type.get())->getNestedType(); - auto nested_value = struct_value.get(field); - insertValue(nested_column, nested_type, column_name, nested_value, enum_comparing_mode); - nullable_column.getNullMapData().push_back(0); - } - } - else if (isTuple(column_type)) - { - auto & tuple_column = assert_cast(column); - const auto * tuple_type = assert_cast(column_type.get()); - bool have_explicit_names = tuple_type->haveExplicitNames(); - auto struct_schema = struct_value.getSchema(); - for (uint32_t i = 0; i != tuple_column.tupleSize(); ++i) - insertValue( - tuple_column.getColumn(i), - tuple_type->getElements()[i], - tuple_type->getElementNames()[i], - struct_value.get(have_explicit_names ? struct_schema.getFieldByName(tuple_type->getElementNames()[i]) : struct_schema.getFields()[i]), - enum_comparing_mode); - } - else if (isMap(column_type)) - { - const auto & map_type = assert_cast(*column_type); - DataTypes key_value_types = {map_type.getKeyType(), map_type.getValueType()}; - Names key_value_names = {"key", "value"}; - auto entries_type = std::make_shared(std::make_shared(key_value_types, key_value_names)); - auto & entries_column = assert_cast(column).getNestedColumn(); - auto entries_field = struct_value.getSchema().getFields()[0]; - insertValue(entries_column, entries_type, column_name, struct_value.get(entries_field), enum_comparing_mode); - } - else - { - /// It can be nested column from Nested type. - auto [field_name, nested_name] = splitCapnProtoFieldName(column_name); - insertValue(column, column_type, nested_name, struct_value.get(nested_name), enum_comparing_mode); - } - break; - } - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CapnProto value type."); - } -} - bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { if (in->eof()) @@ -298,12 +72,8 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension { auto array = readMessage(); capnp::FlatArrayMessageReader msg(array); - auto root_reader = msg.getRoot(root); - for (size_t i = 0; i != columns.size(); ++i) - { - auto value = getReaderByColumnName(root_reader, column_names[i]); - insertValue(*columns[i], column_types[i], column_names[i], value, format_settings.capn_proto.enum_comparing_mode); - } + auto root_reader = msg.getRoot(schema); + serializer->readRow(columns, root_reader); } catch (const kj::Exception & e) { @@ -343,7 +113,14 @@ void registerInputFormatCapnProto(FormatFactory & factory) factory.markFormatSupportsSubsetOfColumns("CapnProto"); factory.registerFileExtension("capnp", "CapnProto"); factory.registerAdditionalInfoForSchemaCacheGetter( - "CapnProto", [](const FormatSettings & settings) { return fmt::format("format_schema={}", settings.schema.format_schema); }); + "CapnProto", + [](const FormatSettings & settings) + { + return fmt::format( + "format_schema={}, skip_fields_with_unsupported_types_in_schema_inference={}", + settings.schema.format_schema, + settings.capn_proto.skip_fields_with_unsupported_types_in_schema_inference); + }); } void registerCapnProtoSchemaReader(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h index cf23f22b643..06e94da123f 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h @@ -4,7 +4,8 @@ #if USE_CAPNP #include -#include +#include +#include #include #include @@ -33,10 +34,8 @@ private: kj::Array readMessage(); std::shared_ptr parser; - capnp::StructSchema root; - const FormatSettings format_settings; - DataTypes column_types; - Names column_names; + capnp::StructSchema schema; + std::unique_ptr serializer; }; class CapnProtoSchemaReader : public IExternalSchemaReader diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index 0225680b396..7dd18be27f4 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -1,28 +1,13 @@ #include #if USE_CAPNP -#include +#include #include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - namespace DB { @@ -45,252 +30,25 @@ CapnProtoRowOutputFormat::CapnProtoRowOutputFormat( WriteBuffer & out_, const Block & header_, const FormatSchemaInfo & info, - const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_), column_names(header_.getNames()), column_types(header_.getDataTypes()), output_stream(std::make_unique(out_)), format_settings(format_settings_) + const FormatSettings & format_settings) + : IRowOutputFormat(header_, out_) + , column_names(header_.getNames()) + , column_types(header_.getDataTypes()) + , output_stream(std::make_unique(out_)) { schema = schema_parser.getMessageSchema(info); - checkCapnProtoSchemaStructure(schema, getPort(PortKind::Main).getHeader(), format_settings.capn_proto.enum_comparing_mode); -} - -template -static capnp::DynamicEnum getDynamicEnum( - const ColumnPtr & column, - const DataTypePtr & data_type, - size_t row_num, - const capnp::EnumSchema & enum_schema, - FormatSettings::EnumComparingMode mode) -{ - const auto * enum_data_type = assert_cast *>(data_type.get()); - EnumValue enum_value = column->getInt(row_num); - if (mode == FormatSettings::EnumComparingMode::BY_VALUES) - return capnp::DynamicEnum(enum_schema, enum_value); - - auto enum_name = enum_data_type->getNameForValue(enum_value); - for (const auto enumerant : enum_schema.getEnumerants()) - { - if (compareEnumNames(String(enum_name), enumerant.getProto().getName(), mode)) - return capnp::DynamicEnum(enumerant); - } - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert CLickHouse Enum value to CapnProto Enum"); -} - -static capnp::DynamicValue::Builder initStructFieldBuilder(const ColumnPtr & column, size_t row_num, capnp::DynamicStruct::Builder & struct_builder, capnp::StructSchema::Field field) -{ - if (const auto * array_column = checkAndGetColumn(*column)) - { - size_t size = array_column->getOffsets()[row_num] - array_column->getOffsets()[row_num - 1]; - return struct_builder.init(field, static_cast(size)); - } - - if (field.getType().isStruct()) - return struct_builder.init(field); - - return struct_builder.get(field); -} - -static std::optional convertToDynamicValue( - const ColumnPtr & column, - const DataTypePtr & data_type, - size_t row_num, - const String & column_name, - capnp::DynamicValue::Builder builder, - FormatSettings::EnumComparingMode enum_comparing_mode, - std::vector> & temporary_text_data_storage) -{ - /// Here we don't do any types validation, because we did it in CapnProtoRowOutputFormat constructor. - - if (data_type->lowCardinality()) - { - const auto * lc_column = assert_cast(column.get()); - const auto & dict_type = assert_cast(data_type.get())->getDictionaryType(); - size_t index = lc_column->getIndexAt(row_num); - return convertToDynamicValue(lc_column->getDictionary().getNestedColumn(), dict_type, index, column_name, builder, enum_comparing_mode, temporary_text_data_storage); - } - - switch (builder.getType()) - { - case capnp::DynamicValue::Type::INT: - return capnp::DynamicValue::Reader(column->getInt(row_num)); - case capnp::DynamicValue::Type::UINT: - { - /// IPv4 column doesn't support getUInt method. - if (isIPv4(data_type)) - return capnp::DynamicValue::Reader(assert_cast(column.get())->getElement(row_num)); - return capnp::DynamicValue::Reader(column->getUInt(row_num)); - } - case capnp::DynamicValue::Type::BOOL: - return capnp::DynamicValue::Reader(column->getBool(row_num)); - case capnp::DynamicValue::Type::FLOAT: - return capnp::DynamicValue::Reader(column->getFloat64(row_num)); - case capnp::DynamicValue::Type::ENUM: - { - auto enum_schema = builder.as().getSchema(); - if (data_type->getTypeId() == TypeIndex::Enum8) - return capnp::DynamicValue::Reader( - getDynamicEnum(column, data_type, row_num, enum_schema, enum_comparing_mode)); - return capnp::DynamicValue::Reader( - getDynamicEnum(column, data_type, row_num, enum_schema, enum_comparing_mode)); - } - case capnp::DynamicValue::Type::DATA: - { - auto data = column->getDataAt(row_num); - return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); - } - case capnp::DynamicValue::Type::TEXT: - { - /// In TEXT type data should be null-terminated, but ClickHouse String data could not be. - /// To make data null-terminated we should copy it to temporary String object, but - /// capnp::Text::Reader works only with pointer to the data and it's size, so we should - /// guarantee that new String object life time is longer than capnp::Text::Reader life time. - /// To do this we store new String object in a temporary storage, passed in this function - /// by reference. We use unique_ptr instead of just String to avoid pointers - /// invalidation on vector reallocation. - temporary_text_data_storage.push_back(std::make_unique(column->getDataAt(row_num))); - auto & data = temporary_text_data_storage.back(); - return capnp::DynamicValue::Reader(capnp::Text::Reader(data->data(), data->size())); - } - case capnp::DynamicValue::Type::STRUCT: - { - auto struct_builder = builder.as(); - auto nested_struct_schema = struct_builder.getSchema(); - /// Struct can represent Tuple, Nullable (named union with two fields) or single column when it contains one nested column. - if (data_type->isNullable()) - { - const auto * nullable_type = assert_cast(data_type.get()); - const auto * nullable_column = assert_cast(column.get()); - auto fields = nested_struct_schema.getUnionFields(); - if (nullable_column->isNullAt(row_num)) - { - auto null_field = fields[0].getType().isVoid() ? fields[0] : fields[1]; - struct_builder.set(null_field, capnp::Void()); - } - else - { - auto value_field = fields[0].getType().isVoid() ? fields[1] : fields[0]; - struct_builder.clear(value_field); - const auto & nested_column = nullable_column->getNestedColumnPtr(); - auto value_builder = initStructFieldBuilder(nested_column, row_num, struct_builder, value_field); - auto value = convertToDynamicValue(nested_column, nullable_type->getNestedType(), row_num, column_name, value_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(value_field, *value); - } - } - else if (isTuple(data_type)) - { - const auto * tuple_data_type = assert_cast(data_type.get()); - const auto & nested_types = tuple_data_type->getElements(); - const auto & nested_names = tuple_data_type->getElementNames(); - const auto & nested_columns = assert_cast(column.get())->getColumns(); - bool have_explicit_names = tuple_data_type->haveExplicitNames(); - for (uint32_t i = 0; i != nested_names.size(); ++i) - { - capnp::StructSchema::Field nested_field = have_explicit_names ? nested_struct_schema.getFieldByName(nested_names[i]) : nested_struct_schema.getFields()[i]; - auto field_builder = initStructFieldBuilder(nested_columns[i], row_num, struct_builder, nested_field); - auto value = convertToDynamicValue(nested_columns[i], nested_types[i], row_num, nested_names[i], field_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(nested_field, *value); - } - } - else if (isMap(data_type)) - { - /// We output Map type as follow CapnProto schema - /// - /// struct Map { - /// struct Entry { - /// key @0: Key; - /// value @1: Value; - /// } - /// entries @0 :List(Entry); - /// } - /// - /// And we don't need to check that struct have this form here because we checked it before. - const auto & map_type = assert_cast(*data_type); - DataTypes key_value_types = {map_type.getKeyType(), map_type.getValueType()}; - Names key_value_names = {"key", "value"}; - auto entries_type = std::make_shared(std::make_shared(key_value_types, key_value_names)); - - /// Nested column in Map is actually Array(Tuple), so we can output it according to "entries" field schema. - const auto & entries_column = assert_cast(column.get())->getNestedColumnPtr(); - - auto entries_field = nested_struct_schema.getFields()[0]; - auto field_builder = initStructFieldBuilder(entries_column, row_num, struct_builder, entries_field); - auto entries_value = convertToDynamicValue(entries_column, entries_type, row_num, column_name, field_builder, enum_comparing_mode, temporary_text_data_storage); - if (entries_value) - struct_builder.set(entries_field, *entries_value); - } - else - { - /// It can be nested column from Nested type. - auto [field_name, nested_name] = splitCapnProtoFieldName(column_name); - auto nested_field = nested_struct_schema.getFieldByName(nested_name); - auto field_builder = initStructFieldBuilder(column, row_num, struct_builder, nested_field); - auto value = convertToDynamicValue(column, data_type, row_num, nested_name, field_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(nested_field, *value); - } - return std::nullopt; - } - case capnp::DynamicValue::Type::LIST: - { - auto list_builder = builder.as(); - const auto * array_column = assert_cast(column.get()); - const auto & nested_column = array_column->getDataPtr(); - const auto & nested_type = assert_cast(data_type.get())->getNestedType(); - const auto & offsets = array_column->getOffsets(); - auto offset = offsets[row_num - 1]; - size_t size = offsets[row_num] - offset; - - const auto * nested_array_column = checkAndGetColumn(*nested_column); - for (unsigned i = 0; i != static_cast(size); ++i) - { - capnp::DynamicValue::Builder value_builder; - /// For nested arrays we need to initialize nested list builder. - if (nested_array_column) - { - const auto & nested_offset = nested_array_column->getOffsets(); - size_t nested_array_size = nested_offset[offset + i] - nested_offset[offset + i - 1]; - value_builder = list_builder.init(i, static_cast(nested_array_size)); - } - else - value_builder = list_builder[i]; - - auto value = convertToDynamicValue(nested_column, nested_type, offset + i, column_name, value_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - list_builder.set(i, *value); - } - return std::nullopt; - } - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CapnProto type."); - } + const auto & header = getPort(PortKind::Main).getHeader(); + serializer = std::make_unique(header.getDataTypes(), header.getNames(), schema, format_settings.capn_proto); + capnp::MallocMessageBuilder message; } void CapnProtoRowOutputFormat::write(const Columns & columns, size_t row_num) { capnp::MallocMessageBuilder message; - /// Temporary storage for data that will be outputted in fields with CapnProto type TEXT. - /// See comment in convertToDynamicValue() for more details. - std::vector> temporary_text_data_storage; capnp::DynamicStruct::Builder root = message.initRoot(schema); - - /// Some columns can share same field builder. For example when we have - /// column with Nested type that was flattened into several columns. - std::unordered_map field_builders; - for (size_t i = 0; i != columns.size(); ++i) - { - auto [struct_builder, field] = getStructBuilderAndFieldByColumnName(root, column_names[i]); - if (!field_builders.contains(field.getIndex())) - { - auto field_builder = initStructFieldBuilder(columns[i], row_num, struct_builder, field); - field_builders[field.getIndex()] = field_builder; - } - auto value = convertToDynamicValue(columns[i], column_types[i], row_num, column_names[i], field_builders[field.getIndex()], format_settings.capn_proto.enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(field, *value); - } - + serializer->writeRow(columns, std::move(root), row_num); capnp::writeMessage(*output_stream, message); + } void registerOutputFormatCapnProto(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h index 5cc7099d4c7..dd9dcc6b340 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h @@ -3,15 +3,17 @@ #include "config.h" #if USE_CAPNP -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include namespace DB { + class CapnProtoOutputStream : public kj::OutputStream { public: @@ -43,8 +45,9 @@ private: DataTypes column_types; capnp::StructSchema schema; std::unique_ptr output_stream; - const FormatSettings format_settings; CapnProtoSchemaParser schema_parser; + std::unique_ptr serializer; + }; } diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp index 9777f2361a2..6098923a195 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp @@ -88,7 +88,14 @@ void registerInputFormatProtobufList(FormatFactory & factory) }); factory.markFormatSupportsSubsetOfColumns("ProtobufList"); factory.registerAdditionalInfoForSchemaCacheGetter( - "ProtobufList", [](const FormatSettings & settings) { return fmt::format("format_schema={}", settings.schema.format_schema); }); + "ProtobufList", + [](const FormatSettings & settings) + { + return fmt::format( + "format_schema={}, skip_fields_with_unsupported_types_in_schema_inference={}", + settings.schema.format_schema, + settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference); + }); } void registerProtobufListSchemaReader(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index ee60501dba5..126f3673571 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -128,7 +128,14 @@ void registerProtobufSchemaReader(FormatFactory & factory) for (const auto & name : {"Protobuf", "ProtobufSingle"}) factory.registerAdditionalInfoForSchemaCacheGetter( - name, [](const FormatSettings & settings) { return fmt::format("format_schema={}", settings.schema.format_schema); }); + name, + [](const FormatSettings & settings) + { + return fmt::format( + "format_schema={}, skip_fields_with_unsupported_types_in_schema_inference={}", + settings.schema.format_schema, + settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference); + }); } } diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index c15d6fe442e..625104fb590 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -96,8 +96,8 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a_b U $CLICKHOUSE_CLIENT --query="SELECT number AS a_b, number + 1 AS a_c_d, number + 2 AS a_c_e_f FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_tuples:Message'" > $CAPN_PROTO_FILE $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(f UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" -$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(bb UInt64, c Tuple(d UInt64, e Tuple(f UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(ff UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(bb UInt64, c Tuple(d UInt64, e Tuple(f UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "THERE_IS_NO_COLUMN" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(ff UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "THERE_IS_NO_COLUMN" && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'string String') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'; diff --git a/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.reference b/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.reference new file mode 100644 index 00000000000..f34c857e2f6 --- /dev/null +++ b/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.reference @@ -0,0 +1 @@ +42 (42,42) diff --git a/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.sh b/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.sh new file mode 100755 index 00000000000..c3835948437 --- /dev/null +++ b/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-replicated-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SCHEMADIR=$CURDIR/format_schemas +$CLICKHOUSE_LOCAL -q "select 42 as Field1, (42, 42)::Tuple(Field1 UInt32, Field2 UInt32) as Nested format CapnProto settings format_schema='$SCHEMADIR/02735_case_insensitive_names_matching:Message'" | $CLICKHOUSE_LOCAL --input-format CapnProto --structure "Field1 UInt32, Nested Tuple(Field1 UInt32, Field2 UInt32)" -q "select * from table" --format_schema="$SCHEMADIR/02735_case_insensitive_names_matching:Message" + diff --git a/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.reference b/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.reference new file mode 100644 index 00000000000..b6e6d485929 --- /dev/null +++ b/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.reference @@ -0,0 +1,3 @@ +(42,(42,42),[(42,42),(24,24)]) [(42,(42,42),[(42,42),(24,24)]),(24,(24,24),[(24,24),(42,42)])] +42 42 42 +[42,24] [42,24] [42,24] [[42,24],[24,42]] [[42,24],[24,42]] diff --git a/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.sh b/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.sh new file mode 100755 index 00000000000..c669be2ed33 --- /dev/null +++ b/tests/queries/0_stateless/02736_reading_and_writing_structure_fields.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-replicated-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SCHEMADIR=$CURDIR/format_schemas +DATA_FILE=02736_$CLICKHOUSE_TEST_UNIQUE_NAME.bin + +$CLICKHOUSE_LOCAL -q "select tuple(42, tuple(42, 42), [tuple(42, 42), tuple(24, 24)]) as nested, [tuple(42, tuple(42, 42), [tuple(42, 42), tuple(24, 24)]), tuple(24, tuple(24, 24), [tuple(24, 24), tuple(42, 42)])] as nestedList format CapnProto settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" > $DATA_FILE + +$CLICKHOUSE_LOCAL -q "select * from file($DATA_FILE, CapnProto) settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" + +$CLICKHOUSE_LOCAL -q "select 42 as nested_field1, 42 as nested_nested_field1, 42 as nested_nested_field2 format CapnProto settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" > $DATA_FILE + +$CLICKHOUSE_LOCAL -q "select * from file($DATA_FILE, CapnProto, 'nested_field1 UInt32, nested_nested_field1 UInt32, nested_nested_field2 UInt32') settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" + +$CLICKHOUSE_LOCAL -q "select [42, 24] as nestedList_field1, [42, 24] as nestedList_nested_field1, [42, 24] as nestedList_nested_field2, [[42, 24], [24, 42]] as nestedList_nestedList_field1, [[42, 24], [24, 42]] as nestedList_nestedList_field2 format CapnProto settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" > $DATA_FILE + +$CLICKHOUSE_LOCAL -q "select * from file($DATA_FILE, CapnProto, 'nestedList_field1 Array(UInt32), nestedList_nested_field1 Array(UInt32), nestedList_nested_field2 Array(UInt32), nestedList_nestedList_field1 Array(Array(UInt32)), nestedList_nestedList_field2 Array(Array(UInt32))') settings format_schema='$SCHEMADIR/02736_nested_structures:Message'" + +rm $DATA_FILE + diff --git a/tests/queries/0_stateless/format_schemas/02735_case_insensitive_names_matching.capnp b/tests/queries/0_stateless/format_schemas/02735_case_insensitive_names_matching.capnp new file mode 100644 index 00000000000..6b12aab081a --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02735_case_insensitive_names_matching.capnp @@ -0,0 +1,13 @@ +@0x9ef128e10a8010b8; + +struct Nested +{ + field1 @0 : UInt32; + field2 @1 : UInt32; +} + +struct Message +{ + field1 @0 : UInt32; + nested @1 : Nested; +} diff --git a/tests/queries/0_stateless/format_schemas/02736_nested_structures.capnp b/tests/queries/0_stateless/format_schemas/02736_nested_structures.capnp new file mode 100644 index 00000000000..a03eb27f383 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02736_nested_structures.capnp @@ -0,0 +1,21 @@ +@0x9ef128e10a8010b8; + +struct Nested2 +{ + field1 @0 : UInt32; + field2 @1 : UInt32; +} + +struct Nested +{ + field1 @0 : UInt32; + nested @1 : Nested2; + nestedList @2 : List(Nested2); +} + +struct Message +{ + nested @0 : Nested; + nestedList @1 : List(Nested); +} + From c2eada7ba7bd385281c140dbf225be7eee4f1ff2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 May 2023 21:07:56 +0200 Subject: [PATCH 0621/2223] Fix style --- src/Formats/CapnProtoSerializer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index e0c8ae2a79a..c31623286d0 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -26,7 +26,7 @@ namespace DB namespace ErrorCodes { extern const int THERE_IS_NO_COLUMN; - extern const int BAD_TYPE_OF_FIELD; + extern const int LOGICAL_ERROR; extern const int CAPN_PROTO_BAD_CAST; extern const int INCORRECT_DATA; extern const int ILLEGAL_COLUMN; @@ -293,7 +293,7 @@ namespace return capnp::DynamicValue::Reader(capnp::DynamicEnum(enumerant)); } - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert CLickHouse Enum value to CapnProto Enum"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert ClickHouse Enum value to CapnProto Enum"); } void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override From cc7cfa050f5723fa4bfeca994a04784732950968 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 May 2023 21:08:12 +0200 Subject: [PATCH 0622/2223] Fix style --- src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index e686ae86997..c056ee2b4a4 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int INCORRECT_DATA; } From 1347dc4ede100dbfc7240fa7ead23b13c924d202 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 May 2023 21:08:31 +0200 Subject: [PATCH 0623/2223] Fix style --- src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index 7dd18be27f4..66a7160dd89 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -11,12 +11,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - CapnProtoOutputStream::CapnProtoOutputStream(WriteBuffer & out_) : out(out_) { } From a89a8b8d50f8ff5c05ebbcdb83f19dcac6739dbf Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 May 2023 12:08:50 +0000 Subject: [PATCH 0624/2223] Fix build --- src/Formats/CapnProtoSerializer.cpp | 6 ++++++ src/Formats/CapnProtoSerializer.h | 4 ++++ 2 files changed, 10 insertions(+) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index c31623286d0..00ccfc7717d 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -1,3 +1,7 @@ +#include "config.h" + +#if USE_CAPNP + #include #include #include @@ -1216,3 +1220,5 @@ void CapnProtoSerializer::readRow(MutableColumns & columns, capnp::DynamicStruct CapnProtoSerializer::~CapnProtoSerializer() = default; } + +#endif diff --git a/src/Formats/CapnProtoSerializer.h b/src/Formats/CapnProtoSerializer.h index efae797875b..692f5e5301f 100644 --- a/src/Formats/CapnProtoSerializer.h +++ b/src/Formats/CapnProtoSerializer.h @@ -1,5 +1,7 @@ #pragma once +#if USE_CAPNP + #include #include @@ -23,3 +25,5 @@ private: }; } + +#endif From 5f1ca61d090b70ecee8f70d8e3656195e13f0ee9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 12 May 2023 16:12:01 +0200 Subject: [PATCH 0625/2223] Fix special builds --- src/Formats/CapnProtoSerializer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index 00ccfc7717d..091e70da656 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -1007,7 +1007,7 @@ namespace catch (Exception & e) { e.addMessage("(while converting column {})", column_name); - throw e; + throw std::move(e); } } @@ -1015,7 +1015,7 @@ namespace { assert(builder); auto & struct_builder = assert_cast(*builder); - if (auto tuple_column = typeid_cast(column.get())) + if (auto * tuple_column = typeid_cast(column.get())) writeRow(tuple_column->getColumnsCopy(), struct_builder, row_num); else writeRow(Columns{column}, struct_builder, row_num); From 94ef08977ae88a48f95343a7b27abed6471efbe6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 12 May 2023 18:53:51 +0200 Subject: [PATCH 0626/2223] Fix special build --- src/Formats/CapnProtoSchema.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Formats/CapnProtoSchema.cpp b/src/Formats/CapnProtoSchema.cpp index 22518d5061a..f9ab88d39ed 100644 --- a/src/Formats/CapnProtoSchema.cpp +++ b/src/Formats/CapnProtoSchema.cpp @@ -151,7 +151,7 @@ namespace { template - static DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) + DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) { std::vector> values; for (auto enumerant : enumerants) @@ -159,7 +159,7 @@ namespace return std::make_shared>(std::move(values)); } - static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) + DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) { auto enumerants = enum_schema.getEnumerants(); if (enumerants.size() < 128) @@ -170,7 +170,7 @@ namespace throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "ClickHouse supports only 8 and 16-bit Enums"); } - static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) + DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) { switch (capnp_type.which()) { From f76fc5e06682fb7931fc067bcbc38960e91dea7b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 12 May 2023 18:54:38 +0200 Subject: [PATCH 0627/2223] Fix special build --- src/Formats/CapnProtoSerializer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index 091e70da656..ff3880976c7 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -158,7 +158,7 @@ namespace }; template - static std::unique_ptr createIntegerSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + std::unique_ptr createIntegerSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) { switch (capnp_type.which()) { @@ -1015,7 +1015,7 @@ namespace { assert(builder); auto & struct_builder = assert_cast(*builder); - if (auto * tuple_column = typeid_cast(column.get())) + if (const auto * tuple_column = typeid_cast(column.get())) writeRow(tuple_column->getColumnsCopy(), struct_builder, row_num); else writeRow(Columns{column}, struct_builder, row_num); From 5676a2c8805c9775017e0c40964fbd5fe1235731 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 24 May 2023 19:34:51 +0200 Subject: [PATCH 0628/2223] Small refactoring of encrypted disk --- src/Disks/DiskEncrypted.cpp | 45 --------- src/Disks/DiskEncrypted.h | 130 ++++++++++++++----------- src/Disks/DiskEncryptedTransaction.cpp | 9 -- src/Disks/DiskEncryptedTransaction.h | 36 ++++--- 4 files changed, 93 insertions(+), 127 deletions(-) diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index a94036ef1a6..2415b432e01 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -138,19 +138,6 @@ namespace } } - String getCurrentKey(const String & path, const DiskEncryptedSettings & settings) - { - auto it = settings.keys.find(settings.current_key_id); - if (it == settings.keys.end()) - throw Exception( - ErrorCodes::DATA_ENCRYPTION_ERROR, - "Not found a key with the current ID {} required to cipher file {}", - settings.current_key_id, - quoteString(path)); - - return it->second; - } - String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings) { auto it = settings.keys.find(header.key_id); @@ -310,38 +297,6 @@ std::unique_ptr DiskEncrypted::readFile( return std::make_unique(settings.local_fs_buffer_size, std::move(buffer), key, header); } -std::unique_ptr DiskEncrypted::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) -{ - auto wrapped_path = wrappedPath(path); - FileEncryption::Header header; - String key; - UInt64 old_file_size = 0; - auto settings = current_settings.get(); - if (mode == WriteMode::Append && exists(path)) - { - old_file_size = getFileSize(path); - if (old_file_size) - { - /// Append mode: we continue to use the same header. - auto read_buffer = delegate->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize)); - header = readHeader(*read_buffer); - key = getKey(path, header, *settings); - } - } - if (!old_file_size) - { - /// Rewrite mode: we generate a new header. - key = getCurrentKey(path, *settings); - header.algorithm = settings->current_algorithm; - header.key_id = settings->current_key_id; - header.key_hash = calculateKeyHash(key); - header.init_vector = InitVector::random(); - } - auto buffer = delegate->writeFile(wrapped_path, buf_size, mode); - return std::make_unique(buf_size, std::move(buffer), key, header, old_file_size); -} - - size_t DiskEncrypted::getFileSize(const String & path) const { auto wrapped_path = wrappedPath(path); diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 57de8177a2b..73bbe3fd74a 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -51,27 +51,30 @@ public: void createDirectory(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->createDirectory(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->createDirectory(path); + tx->commit(); } void createDirectories(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->createDirectories(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->createDirectories(path); + tx->commit(); } void clearDirectory(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->clearDirectory(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->clearDirectory(path); + tx->commit(); } void moveDirectory(const String & from_path, const String & to_path) override { - auto wrapped_from_path = wrappedPath(from_path); - auto wrapped_to_path = wrappedPath(to_path); - delegate->moveDirectory(wrapped_from_path, wrapped_to_path); + auto tx = createEncryptedTransaction(); + tx->moveDirectory(from_path, to_path); + tx->commit(); } DirectoryIteratorPtr iterateDirectory(const String & path) const override @@ -82,22 +85,23 @@ public: void createFile(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->createFile(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->createFile(path); + tx->commit(); } void moveFile(const String & from_path, const String & to_path) override { - auto wrapped_from_path = wrappedPath(from_path); - auto wrapped_to_path = wrappedPath(to_path); - delegate->moveFile(wrapped_from_path, wrapped_to_path); + auto tx = createEncryptedTransaction(); + tx->moveFile(from_path, to_path); + tx->commit(); } void replaceFile(const String & from_path, const String & to_path) override { - auto wrapped_from_path = wrappedPath(from_path); - auto wrapped_to_path = wrappedPath(to_path); - delegate->replaceFile(wrapped_from_path, wrapped_to_path); + auto tx = createEncryptedTransaction(); + tx->replaceFile(from_path, to_path); + tx->commit(); } void listFiles(const String & path, std::vector & file_names) const override @@ -120,61 +124,67 @@ public: const String & path, size_t buf_size, WriteMode mode, - const WriteSettings & settings) override; + const WriteSettings & settings) override + { + auto tx = createEncryptedTransaction(); + auto result = tx->writeFile(path, buf_size, mode, settings); + return result; + } void removeFile(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->removeFile(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->removeFile(path); + tx->commit(); } void removeFileIfExists(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->removeFileIfExists(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->removeFileIfExists(path); + tx->commit(); } void removeDirectory(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->removeDirectory(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->removeDirectory(path); + tx->commit(); } void removeRecursive(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->removeRecursive(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->removeRecursive(path); + tx->commit(); } void removeSharedFile(const String & path, bool flag) override { - auto wrapped_path = wrappedPath(path); - delegate->removeSharedFile(wrapped_path, flag); + auto tx = createEncryptedTransaction(); + tx->removeSharedFile(path, flag); + tx->commit(); } void removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override { - auto wrapped_path = wrappedPath(path); - delegate->removeSharedRecursive(wrapped_path, keep_all_batch_data, file_names_remove_metadata_only); + auto tx = createEncryptedTransaction(); + tx->removeSharedRecursive(path, keep_all_batch_data, file_names_remove_metadata_only); + tx->commit(); } void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override { - for (const auto & file : files) - { - auto wrapped_path = wrappedPath(file.path); - bool keep = keep_all_batch_data || file_names_remove_metadata_only.contains(fs::path(file.path).filename()); - if (file.if_exists) - delegate->removeSharedFileIfExists(wrapped_path, keep); - else - delegate->removeSharedFile(wrapped_path, keep); - } + auto tx = createEncryptedTransaction(); + tx->removeSharedFiles(files, keep_all_batch_data, file_names_remove_metadata_only); + tx->commit(); } void removeSharedFileIfExists(const String & path, bool flag) override { - auto wrapped_path = wrappedPath(path); - delegate->removeSharedFileIfExists(wrapped_path, flag); + auto tx = createEncryptedTransaction(); + tx->removeSharedFileIfExists(path, flag); + tx->commit(); } Strings getBlobPath(const String & path) const override @@ -185,8 +195,9 @@ public: void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override { - auto wrapped_path = wrappedPath(path); - delegate->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function)); + auto tx = createEncryptedTransaction(); + tx->writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function)); + tx->commit(); } std::unique_ptr readEncryptedFile(const String & path, const ReadSettings & settings) const override @@ -201,8 +212,9 @@ public: WriteMode mode, const WriteSettings & settings) const override { - auto wrapped_path = wrappedPath(path); - return delegate->writeFile(wrapped_path, buf_size, mode, settings); + auto tx = createEncryptedTransaction(); + auto buf = tx->writeEncryptedFile(path, buf_size, mode, settings); + return buf; } size_t getEncryptedFileSize(const String & path) const override @@ -219,8 +231,9 @@ public: void setLastModified(const String & path, const Poco::Timestamp & timestamp) override { - auto wrapped_path = wrappedPath(path); - delegate->setLastModified(wrapped_path, timestamp); + auto tx = createEncryptedTransaction(); + tx->setLastModified(path, timestamp); + tx->commit(); } Poco::Timestamp getLastModified(const String & path) const override @@ -237,15 +250,16 @@ public: void setReadOnly(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->setReadOnly(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->setReadOnly(path); + tx->commit(); } void createHardLink(const String & src_path, const String & dst_path) override { - auto wrapped_src_path = wrappedPath(src_path); - auto wrapped_dst_path = wrappedPath(dst_path); - delegate->createHardLink(wrapped_src_path, wrapped_dst_path); + auto tx = createEncryptedTransaction(); + tx->createHardLink(src_path, dst_path); + tx->commit(); } void truncateFile(const String & path, size_t size) override; @@ -280,19 +294,21 @@ public: SyncGuardPtr getDirectorySyncGuard(const String & path) const override; + std::shared_ptr createEncryptedTransaction() const + { + auto delegate_transaction = delegate->createTransaction(); + return std::make_shared(delegate_transaction, disk_path, *current_settings.get(), delegate.get()); + } + DiskTransactionPtr createTransaction() override { - /// Need to overwrite explicetly because this disk change - /// a lot of "delegate" methods. - if (use_fake_transaction) { return std::make_shared(*this); } else { - auto delegate_transaction = delegate->createTransaction(); - return std::make_shared(delegate_transaction, disk_path, *current_settings.get(), delegate.get()); + return createEncryptedTransaction(); } } diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 7174e98f256..0c927a027cd 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -110,14 +110,5 @@ std::unique_ptr DiskEncryptedTransaction::writeFile( return std::make_unique(buf_size, std::move(buffer), key, header, old_file_size); } -void DiskEncryptedTransaction::writeFileUsingCustomWriteObject( - const String &, - WriteMode, - std::function & object_attributes)>) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `writeFileUsingCustomWriteObject()` is not implemented"); -} - - } diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index d68e0c42807..0ef8d946f7f 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -23,7 +22,7 @@ struct DiskEncryptedSettings class DiskEncryptedTransaction : public IDiskTransaction { public: - explicit DiskEncryptedTransaction(DiskTransactionPtr delegate_transaction_, const std::string & disk_path_, DiskEncryptedSettings current_settings_, IDisk * delegate_disk_) + DiskEncryptedTransaction(DiskTransactionPtr delegate_transaction_, const std::string & disk_path_, DiskEncryptedSettings current_settings_, IDisk * delegate_disk_) : delegate_transaction(delegate_transaction_) , disk_path(disk_path_) , current_settings(current_settings_) @@ -32,9 +31,9 @@ public: /// Tries to commit all accumulated operations simultaneously. /// If something fails rollback and throw exception. - void commit(const TransactionCommitOptionsVariant & options = NoCommitOptions{}) override // NOLINT + void commit() override // NOLINT { - delegate_transaction->commit(options); + delegate_transaction->commit(); } void undo() override @@ -42,11 +41,6 @@ public: delegate_transaction->undo(); } - TransactionCommitOutcomeVariant tryCommit(const TransactionCommitOptionsVariant & options) override - { - return delegate_transaction->tryCommit(options); - } - ~DiskEncryptedTransaction() override = default; /// Create directory. @@ -115,13 +109,6 @@ public: const WriteSettings & settings = {}, bool autocommit = true) override; - /// Write a file using a custom function to write an object to the disk's object storage. - void writeFileUsingCustomWriteObject( - const String & path, - WriteMode mode, - std::function & object_attributes)> - custom_write_object_function) override; - /// Remove file. Throws exception if file doesn't exists or it's a directory. void removeFile(const std::string & path) override { @@ -224,6 +211,23 @@ public: delegate_transaction->createHardLink(wrapped_src_path, wrapped_dst_path); } + void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function)); + } + + std::unique_ptr writeEncryptedFile( + const String & path, + size_t buf_size, + WriteMode mode, + const WriteSettings & settings) const + { + auto wrapped_path = wrappedPath(path); + return delegate_transaction->writeFile(wrapped_path, buf_size, mode, settings); + } + + private: String wrappedPath(const String & path) const { From 4a4246a8cc87c29e88bceb02baa5ffcfe15864c8 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 24 May 2023 19:39:53 +0200 Subject: [PATCH 0629/2223] Dedup --- src/Disks/DiskEncrypted.h | 5 +---- src/Disks/DiskEncryptedTransaction.h | 16 ++++++++++++---- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 73bbe3fd74a..69d051a9537 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -347,10 +347,7 @@ public: private: String wrappedPath(const String & path) const { - // if path starts_with disk_path -> got already wrapped path - if (!disk_path.empty() && path.starts_with(disk_path)) - return path; - return disk_path + path; + return DiskEncryptedTransaction::wrappedPath(disk_path, path); } DiskPtr delegate; diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index 0ef8d946f7f..9f60ed36c4e 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -19,9 +19,19 @@ struct DiskEncryptedSettings FileEncryption::Algorithm current_algorithm; }; + + class DiskEncryptedTransaction : public IDiskTransaction { public: + static String wrappedPath(const String disk_path, const String & path) + { + // if path starts_with disk_path -> got already wrapped path + if (!disk_path.empty() && path.starts_with(disk_path)) + return path; + return disk_path + path; + } + DiskEncryptedTransaction(DiskTransactionPtr delegate_transaction_, const std::string & disk_path_, DiskEncryptedSettings current_settings_, IDisk * delegate_disk_) : delegate_transaction(delegate_transaction_) , disk_path(disk_path_) @@ -229,12 +239,10 @@ public: private: + String wrappedPath(const String & path) const { - // if path starts_with disk_path -> got already wrapped path - if (!disk_path.empty() && path.starts_with(disk_path)) - return path; - return disk_path + path; + return wrappedPath(disk_path, path); } DiskTransactionPtr delegate_transaction; From 03ca3f96d2bbb455dbef82e7d1937771f49382e2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 17:40:21 +0000 Subject: [PATCH 0630/2223] Add built-in documentation to toStartOfWeek and toLastDayOfWeek functions --- src/Functions/toCustomWeek.cpp | 26 ++++++++++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/src/Functions/toCustomWeek.cpp b/src/Functions/toCustomWeek.cpp index 98e7aaf1d6b..0dbd60b3eff 100644 --- a/src/Functions/toCustomWeek.cpp +++ b/src/Functions/toCustomWeek.cpp @@ -17,8 +17,30 @@ REGISTER_FUNCTION(ToCustomWeek) { factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + + factory.registerFunction(FunctionDocumentation{.description=R"( +Rounds a date or date with time down to the nearest Sunday or Monday. Returns the date. +Syntax: toStartOfWeek(t[, mode[, timezone]]) +The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. + +Example: +[example:typical] +)", + .examples{ + {"typical", "SELECT toStartOfWeek(today(), 1);", ""}}, + .categories{"Dates and Times"}}, FunctionFactory::CaseSensitive); + + factory.registerFunction(FunctionDocumentation{.description=R"( +Rounds a date or date with time up to the nearest Saturday or Sunday. Returns the date. +Syntax: toLastDayOfWeek(t[, mode[, timezone]]) +The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. + +Example: +[example:typical] +)", + .examples{ + {"typical", "SELECT toLastDayOfWeek(today(), 1);", ""}}, + .categories{"Dates and Times"}}, FunctionFactory::CaseSensitive); /// Compatibility aliases for mysql. factory.registerAlias("week", "toWeek", FunctionFactory::CaseInsensitive); From 8875b98e2ac5d3469419d97f231894078741c5b4 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 24 May 2023 13:47:50 -0400 Subject: [PATCH 0631/2223] Added default value for `--from-repo` arg --- tests/ci/cherry_pick.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 11c8b7e46eb..211092190fa 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -548,6 +548,7 @@ def parse_args(): ) parser.add_argument( "--from-repo", + default="ClickHouse/ClickHouse", help="if set, the commits will be taken from this repo, but PRs will be created in the main repo", ) parser.add_argument("--dry-run", action="store_true", help="do not create anything") From d8f39b8df1ef608b9f78844a6665495be3cbec33 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 24 May 2023 17:53:37 +0000 Subject: [PATCH 0632/2223] Fixing more tests. --- src/Interpreters/ActionsVisitor.cpp | 8 ++++++ .../ClusterProxy/SelectStreamFactory.cpp | 8 +++--- .../ClusterProxy/SelectStreamFactory.h | 6 ++++- .../ClusterProxy/executeQuery.cpp | 1 + src/Interpreters/GlobalSubqueriesVisitor.h | 3 ++- src/Interpreters/PreparedSets.cpp | 6 ++++- .../QueryPlan/DistributedCreateLocalPlan.cpp | 3 ++- .../QueryPlan/DistributedCreateLocalPlan.h | 4 +++ src/Processors/QueryPlan/ReadFromRemote.cpp | 6 +++-- src/Processors/QueryPlan/ReadFromRemote.h | 2 ++ src/Storages/StorageDistributed.cpp | 3 ++- src/Storages/StorageMergeTree.cpp | 3 ++- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- src/Storages/VirtualColumnUtils.cpp | 27 +++++++++++++++++++ 14 files changed, 71 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index dcf6c4a461f..1405568aa71 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -3,6 +3,7 @@ #include #include #include +#include "Parsers/queryToString.h" #include #include @@ -1392,8 +1393,15 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool { if (no_subqueries) return {}; + //std::cerr << queryToString(right_in_operand) << std::endl; auto set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); + // std::cerr << set_key.toString() << std::endl; + // std::cerr << data.prepared_sets->getSets().size() << std::endl; + // std::cerr << reinterpret_cast(data.prepared_sets.get()) << std::endl; + // for (const auto & [k, v] : data.prepared_sets->getSets()) + // std::cerr << "... " << k.toString(); + if (auto set = data.prepared_sets->getFuture(set_key)) return set; diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 0cf3f360994..0f8a725e144 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -92,11 +92,13 @@ SelectStreamFactory::SelectStreamFactory( const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, const StorageSnapshotPtr & storage_snapshot_, - QueryProcessingStage::Enum processed_stage_) + QueryProcessingStage::Enum processed_stage_, + PreparedSetsPtr prepared_sets_) : header(header_), objects_by_shard(objects_by_shard_), storage_snapshot(storage_snapshot_), - processed_stage(processed_stage_) + processed_stage(processed_stage_), + prepared_sets(std::move(prepared_sets_)) { } @@ -117,7 +119,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { local_plans.emplace_back(createLocalPlan( - query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); + query_ast, header, context, processed_stage, prepared_sets, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); }; auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index f1a8b3e0984..b19012ddba6 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -26,6 +26,8 @@ using QueryPlanPtr = std::unique_ptr; struct StorageID; +class PreparedSets; +using PreparedSetsPtr = std::shared_ptr; namespace ClusterProxy { @@ -67,7 +69,8 @@ public: const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, const StorageSnapshotPtr & storage_snapshot_, - QueryProcessingStage::Enum processed_stage_); + QueryProcessingStage::Enum processed_stage_, + PreparedSetsPtr prepared_sets_); void createForShard( const Cluster::ShardInfo & shard_info, @@ -92,6 +95,7 @@ public: const ColumnsDescriptionByShardNum objects_by_shard; const StorageSnapshotPtr storage_snapshot; QueryProcessingStage::Enum processed_stage; + PreparedSetsPtr prepared_sets; }; } diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index e2f1dfe8ba7..a9cf3d55392 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -226,6 +226,7 @@ void executeQuery( std::move(remote_shards), header, processed_stage, + stream_factory.prepared_sets, main_table, table_func_ptr, new_context, diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 3a846bb4bc3..a872af529aa 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -181,7 +181,7 @@ public: // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); - auto set_key = PreparedSetKey::forSubquery(subquery_or_table_name->getTreeHash()); + auto set_key = PreparedSetKey::forSubquery(database_and_table_name->getTreeHash()); if (!prepared_sets->getFuture(set_key)) { @@ -190,6 +190,7 @@ public: subquery_for_set.table = std::move(external_storage); subquery_for_set.createSource(*interpreter); + //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); } else diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index cd6b2a81ba0..3b63d942404 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -140,6 +140,10 @@ FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryF if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + // std::cerr << key.toString() << std::endl; + // std::cerr << "========= PreparedSets::addFromSubquery\n"; + // std::cerr << StackTrace().toString() << std::endl; + subqueries.emplace(id, std::move(from_subquery)); return it->second; } @@ -226,7 +230,7 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; - //std::cerr << StackTrace().toString() << std::endl; + // std::cerr << StackTrace().toString() << std::endl; auto set_cache = context->getPreparedSetsCache(); if (set_cache) diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 9b9cc221ca8..62e369659d1 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -43,6 +43,7 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, + PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, @@ -98,7 +99,7 @@ std::unique_ptr createLocalPlan( } else { - auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); + auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options, prepared_sets); interpreter.buildQueryPlan(*query_plan); } diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h index 1afdc07fa4d..cf59027a33f 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h @@ -10,11 +10,15 @@ namespace DB { +class PreparedSets; +using PreparedSetsPtr = std::shared_ptr; + std::unique_ptr createLocalPlan( const ASTPtr & query_ast, const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, + PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index ed740e3e242..16cb06a94d6 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -97,6 +97,7 @@ ReadFromRemote::ReadFromRemote( ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, + PreparedSetsPtr prepared_sets_, StorageID main_table_, ASTPtr table_func_ptr_, ContextMutablePtr context_, @@ -109,6 +110,7 @@ ReadFromRemote::ReadFromRemote( : ISourceStep(DataStream{.header = std::move(header_)}) , shards(std::move(shards_)) , stage(stage_) + , prepared_sets(std::move(prepared_sets_)) , main_table(std::move(main_table_)) , table_func_ptr(std::move(table_func_ptr_)) , context(std::move(context_)) @@ -150,7 +152,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream my_context = context, my_throttler = throttler, my_main_table = main_table, my_table_func_ptr = table_func_ptr, my_scalars = scalars, my_external_tables = external_tables, - my_stage = stage, local_delay = shard.local_delay, + my_stage = stage, my_prepared_sets = prepared_sets, local_delay = shard.local_delay, add_agg_info, add_totals, add_extremes, async_read, async_query_sending]() mutable -> QueryPipelineBuilder { @@ -185,7 +187,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream if (try_results.empty() || local_delay < max_remote_delay) { auto plan = createLocalPlan( - query, header, my_context, my_stage, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); + query, header, my_context, my_stage, my_prepared_sets, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); return std::move(*plan->buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(my_context), diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index d4005d81f1b..405b5727ff2 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -26,6 +26,7 @@ public: ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, + PreparedSetsPtr prepared_sets_, StorageID main_table_, ASTPtr table_func_ptr_, ContextMutablePtr context_, @@ -46,6 +47,7 @@ public: private: ClusterProxy::SelectStreamFactory::Shards shards; QueryProcessingStage::Enum stage; + PreparedSetsPtr prepared_sets; StorageID main_table; ASTPtr table_func_ptr; ContextMutablePtr context; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 86a2599b49c..f45e247f8e7 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1098,7 +1098,8 @@ void StorageDistributed::read( header, snapshot_data.objects_by_shard, storage_snapshot, - processed_stage); + processed_stage, + query_info.prepared_sets); auto settings = local_context->getSettingsRef(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2c19d3ba122..4600532231f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -231,7 +231,8 @@ void StorageMergeTree::read( header, {}, storage_snapshot, - processed_stage); + processed_stage, + query_info.prepared_sets); ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d9c8f09ccf1..ab78ea2f0a4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4663,7 +4663,8 @@ void StorageReplicatedMergeTree::read( header, {}, storage_snapshot, - processed_stage); + processed_stage, + query_info.prepared_sets); ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 31fa1cc6a7e..be2206a78e9 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -23,6 +23,8 @@ #include #include #include +#include +#include #include @@ -202,6 +204,31 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex //buildSets(expression_ast, analyzer); ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes); + for (const auto & node : actions->getNodes()) + { + if (node.type == ActionsDAG::ActionType::COLUMN) + { + const ColumnSet * column_set = checkAndGetColumnConstData(node.column.get()); + if (!column_set) + column_set = checkAndGetColumn(node.column.get()); + + if (column_set) + { + auto future_set = column_set->getData(); + if (!future_set->isFilled()) + { + auto plan = future_set->build(context); + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + } + } + } + } + Block block_with_filter = block; actions->execute(block_with_filter); From 1c627fbcab71e652975c3c2338db67d63ecec149 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 24 May 2023 20:01:36 +0200 Subject: [PATCH 0633/2223] Fxi --- src/Disks/DiskEncryptedTransaction.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 0c927a027cd..deafe88e43d 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int DATA_ENCRYPTION_ERROR; - extern const int NOT_IMPLEMENTED; } From 3e69648268957cd9f2e31183757bc9584ee23b33 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 24 May 2023 20:02:03 +0200 Subject: [PATCH 0634/2223] Fxi moar --- src/Disks/DiskEncryptedTransaction.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index 9f60ed36c4e..197df56c2a2 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -20,7 +20,6 @@ struct DiskEncryptedSettings }; - class DiskEncryptedTransaction : public IDiskTransaction { public: From 026a15d8a7fecfb4356db3eb7881c6728bc1b3c3 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Wed, 24 May 2023 15:01:59 -0400 Subject: [PATCH 0635/2223] Update dns_max_consecutive_failures docs (#50196) Co-authored-by: Nikita Mikhaylov --- src/Core/ServerSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 2a73930836a..1441c22507d 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -63,7 +63,7 @@ namespace DB \ M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \ M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \ - M(UInt32, dns_max_consecutive_failures, 1024, "Max server connections.", 0) \ + M(UInt32, dns_max_consecutive_failures, 1024, "Max connection failures before dropping host from ClickHouse DNS cache.", 0) \ \ M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ From 62fc5bb3eb57b28cc99f942b213d82315bb6c095 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Wed, 24 May 2023 12:54:30 -0700 Subject: [PATCH 0636/2223] get detailed error instead of unknown result --- tests/clickhouse-test | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e61749e5377..a7921160a78 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1047,12 +1047,13 @@ class TestCase: self.reference_file, self.stdout_file, ], + encoding="latin-1", stdout=PIPE, universal_newlines=True, ).communicate()[0] if diff.startswith("Binary files "): diff += "Content of stdout:\n===================\n" - file = open(self.stdout_file, "r") + file = open(self.stdout_file, "rb") diff += str(file.read()) file.close() diff += "===================" From 034a8bd6b30ffc8ca3e614a7ef6e4f7882a0f662 Mon Sep 17 00:00:00 2001 From: Sorck Date: Wed, 24 May 2023 21:14:01 +0100 Subject: [PATCH 0637/2223] Fix: typo in last_value.md (#50148) Correct typo in page title Co-authored-by: Dan Roscigno --- .../sql-reference/aggregate-functions/reference/last_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/last_value.md b/docs/en/sql-reference/aggregate-functions/reference/last_value.md index ebf002e6ae2..7b6e14e4a55 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/last_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/last_value.md @@ -3,7 +3,7 @@ slug: /en/sql-reference/aggregate-functions/reference/last_value sidebar_position: 8 --- -# first_value +# last_value Selects the last encountered value, similar to `anyLast`, but could accept NULL. From 2c0c1ba4af00c3ba2088fc22926344dec276c8fa Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 24 May 2023 23:29:36 +0300 Subject: [PATCH 0638/2223] add comments to build reports (#50200) --- tests/ci/build_report_check.py | 2 ++ tests/ci/ci_config.py | 17 +++++++++++++++++ tests/ci/report.py | 4 ++++ 3 files changed, 23 insertions(+) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 13257eabb71..82c04ce82c5 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -73,6 +73,7 @@ def get_failed_report( sanitizer="unknown", status=message, elapsed_seconds=0, + comment="", ) return [build_result], [[""]], [GITHUB_RUN_URL] @@ -87,6 +88,7 @@ def process_report( sanitizer=build_config["sanitizer"], status="success" if build_report["status"] else "failure", elapsed_seconds=build_report["elapsed_seconds"], + comment=build_config["comment"], ) build_results = [] build_urls = [] diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 5bba85569b4..d829115cfe1 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -17,6 +17,7 @@ CI_CONFIG = { "additional_pkgs": True, "tidy": "disable", "with_coverage": False, + "comment": "", }, "coverity": { "compiler": "clang-16", @@ -26,6 +27,7 @@ CI_CONFIG = { "tidy": "disable", "with_coverage": False, "official": False, + "comment": "A special build for coverity", }, "package_aarch64": { "compiler": "clang-16-aarch64", @@ -36,6 +38,7 @@ CI_CONFIG = { "additional_pkgs": True, "tidy": "disable", "with_coverage": False, + "comment": "", }, "package_asan": { "compiler": "clang-16", @@ -44,6 +47,7 @@ CI_CONFIG = { "package_type": "deb", "tidy": "disable", "with_coverage": False, + "comment": "", }, "package_ubsan": { "compiler": "clang-16", @@ -52,6 +56,7 @@ CI_CONFIG = { "package_type": "deb", "tidy": "disable", "with_coverage": False, + "comment": "", }, "package_tsan": { "compiler": "clang-16", @@ -60,6 +65,7 @@ CI_CONFIG = { "package_type": "deb", "tidy": "disable", "with_coverage": False, + "comment": "", }, "package_msan": { "compiler": "clang-16", @@ -68,6 +74,7 @@ CI_CONFIG = { "package_type": "deb", "tidy": "disable", "with_coverage": False, + "comment": "", }, "package_debug": { "compiler": "clang-16", @@ -76,6 +83,7 @@ CI_CONFIG = { "package_type": "deb", "tidy": "disable", "with_coverage": False, + "comment": "Note: sparse checkout was used", }, "binary_release": { "compiler": "clang-16", @@ -84,6 +92,7 @@ CI_CONFIG = { "package_type": "binary", "tidy": "disable", "with_coverage": False, + "comment": "", }, "binary_tidy": { "compiler": "clang-16", @@ -93,6 +102,7 @@ CI_CONFIG = { "static_binary_name": "debug-amd64", "tidy": "enable", "with_coverage": False, + "comment": "clang-tidy is used for static analysis", }, "binary_darwin": { "compiler": "clang-16-darwin", @@ -102,6 +112,7 @@ CI_CONFIG = { "static_binary_name": "macos", "tidy": "disable", "with_coverage": False, + "comment": "", }, "binary_aarch64": { "compiler": "clang-16-aarch64", @@ -110,6 +121,7 @@ CI_CONFIG = { "package_type": "binary", "tidy": "disable", "with_coverage": False, + "comment": "", }, "binary_aarch64_v80compat": { "compiler": "clang-16-aarch64-v80compat", @@ -119,6 +131,7 @@ CI_CONFIG = { "static_binary_name": "aarch64v80compat", "tidy": "disable", "with_coverage": False, + "comment": "For ARMv8.1 and older", }, "binary_freebsd": { "compiler": "clang-16-freebsd", @@ -128,6 +141,7 @@ CI_CONFIG = { "static_binary_name": "freebsd", "tidy": "disable", "with_coverage": False, + "comment": "", }, "binary_darwin_aarch64": { "compiler": "clang-16-darwin-aarch64", @@ -137,6 +151,7 @@ CI_CONFIG = { "static_binary_name": "macos-aarch64", "tidy": "disable", "with_coverage": False, + "comment": "", }, "binary_ppc64le": { "compiler": "clang-16-ppc64le", @@ -146,6 +161,7 @@ CI_CONFIG = { "static_binary_name": "powerpc64le", "tidy": "disable", "with_coverage": False, + "comment": "", }, "binary_amd64_compat": { "compiler": "clang-16-amd64-compat", @@ -155,6 +171,7 @@ CI_CONFIG = { "static_binary_name": "amd64compat", "tidy": "disable", "with_coverage": False, + "comment": "SSE2-only build", }, }, "builds_report_config": { diff --git a/tests/ci/report.py b/tests/ci/report.py index 89422298083..a9014acec12 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -243,6 +243,7 @@ class BuildResult: sanitizer: str status: str elapsed_seconds: int + comment: str BuildResults = List[BuildResult] @@ -452,6 +453,7 @@ tr:hover td {{filter: brightness(95%);}} Build log Build time Artifacts +Comment {rows} @@ -519,6 +521,8 @@ def create_build_html_report( links = links[: -len(link_separator)] row += f"{links}" + row += f"{build_result.comment}" + row += "" rows += row return HTML_BASE_BUILD_TEMPLATE.format( From d19d36aad1a2c887ec11ce478f3a8c00689a1cb5 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 20:53:05 +0000 Subject: [PATCH 0639/2223] Extend 02763_last_day_of_week test: add Date32 and DateTime64 types --- .../02763_last_day_of_week.reference | 20 +++++++++---------- .../0_stateless/02763_last_day_of_week.sql | 20 +++++++++++++------ 2 files changed, 24 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02763_last_day_of_week.reference b/tests/queries/0_stateless/02763_last_day_of_week.reference index 9c136d856f6..140ee9bfe24 100644 --- a/tests/queries/0_stateless/02763_last_day_of_week.reference +++ b/tests/queries/0_stateless/02763_last_day_of_week.reference @@ -1,10 +1,10 @@ -2018-12-25 2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 -2018-12-26 2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 -2018-12-27 2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 -2018-12-28 2018-12-28 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 -2018-12-29 2018-12-29 00:00:00 2018-12-29 2018-12-29 2018-12-30 2018-12-30 -2018-12-30 2018-12-30 00:00:00 2019-01-05 2019-01-05 2018-12-30 2018-12-30 -2018-12-31 2018-12-31 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 -2019-01-01 2019-01-01 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 -2019-01-02 2019-01-02 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 -2019-01-03 2019-01-03 00:00:00 2019-01-05 2019-01-05 2019-01-06 2019-01-06 +2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-28 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-29 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-30 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-31 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-01 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-02 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-03 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 diff --git a/tests/queries/0_stateless/02763_last_day_of_week.sql b/tests/queries/0_stateless/02763_last_day_of_week.sql index 600cebd2636..533ecfaa5b2 100644 --- a/tests/queries/0_stateless/02763_last_day_of_week.sql +++ b/tests/queries/0_stateless/02763_last_day_of_week.sql @@ -1,9 +1,17 @@ +WITH + toDate('2018-12-25') + number AS d, + toDate32(d) AS d32, + toDateTime(d) AS dt, + toDateTime64(d, 0) AS dt64 SELECT - toDate('2018-12-25') + number AS x, - toDateTime(x) AS x_t, - toLastDayOfWeek(x) AS w0, - toLastDayOfWeek(x_t) AS wt0, - toLastDayOfWeek(x, 3) AS w3, - toLastDayOfWeek(x_t, 3) AS wt3 + dt64, + toLastDayOfWeek(d) AS wd_0, + toLastDayOfWeek(d32) AS wd32_0, + toLastDayOfWeek(dt) AS wdt_0, + toLastDayOfWeek(dt64) AS wdt64_0, + toLastDayOfWeek(d, 3) AS wd_3, + toLastDayOfWeek(d32, 3) AS wd32_3, + toLastDayOfWeek(dt, 3) AS wdt_3, + toLastDayOfWeek(dt64, 3) AS wdt64_3 FROM numbers(10); From d48a7f9b2cbaa6e905611d58321a7515309051ac Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 24 May 2023 21:08:25 +0000 Subject: [PATCH 0640/2223] Grant toStartOfWeek function a permission to get documentation --- .../02415_all_new_functions_must_be_documented.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index eb574c4e4f5..a22a1552156 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -813,7 +813,6 @@ toStartOfNanosecond toStartOfQuarter toStartOfSecond toStartOfTenMinutes -toStartOfWeek toStartOfYear toString toStringCutToZero From e117347ca0c03820802092c3af6dc22ce6fef059 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 24 May 2023 17:36:23 +0200 Subject: [PATCH 0641/2223] Attempt to fix codebrowser with using clang-15 --- tests/ci/codebrowser_check.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/codebrowser_check.py b/tests/ci/codebrowser_check.py index f9883f0f975..2dba5176c8b 100644 --- a/tests/ci/codebrowser_check.py +++ b/tests/ci/codebrowser_check.py @@ -59,6 +59,9 @@ def main(): os.makedirs(temp_path) docker_image = get_image_with_version(IMAGES_PATH, "clickhouse/codebrowser") + # FIXME: the codebrowser is broken with clang-16, workaround with clang-15 + # See https://github.com/ClickHouse/ClickHouse/issues/50077 + docker_image.version = "49701-4dcdcf4c11b5604f1c5d3121c9c6fea3e957b605" s3_helper = S3Helper() result_path = temp_path / "result_path" From 773a5bbbaae066ac83e35b0215897de949447063 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Fri, 19 May 2023 23:58:32 +0800 Subject: [PATCH 0642/2223] Optimize predicate with toYear converter The date converters, such as toYear, are widely used in the where clauses of the SQL queries, however, these conversions are often expensive due to the complexity of the calendar system. The function preimage is found an optimization for the predicates with the converters. Given a predicate, toYear(c) = y, we could convert it to its equivalent form: c >= b AND c <= e, where b is "y-01-01" and e is "y-12-31". The similar transformation applies to other comparisons (<>, <, >, <=, <=). This commit implemented the above transformation at the AST level by adding a new pass in the TreeOptimizer and a new AST visitor for in-place replacing the predicates of toYear with the converted ones. --- .../OptimizeDateFilterVisitor.cpp | 122 ++++++++++++++++++ src/Interpreters/OptimizeDateFilterVisitor.h | 20 +++ src/Interpreters/TreeOptimizer.cpp | 19 +++ ..._date_filter_predicate_optimizer.reference | 39 ++++++ .../02764_date_filter_predicate_optimizer.sql | 17 +++ 5 files changed, 217 insertions(+) create mode 100644 src/Interpreters/OptimizeDateFilterVisitor.cpp create mode 100644 src/Interpreters/OptimizeDateFilterVisitor.h create mode 100644 tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference create mode 100644 tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql diff --git a/src/Interpreters/OptimizeDateFilterVisitor.cpp b/src/Interpreters/OptimizeDateFilterVisitor.cpp new file mode 100644 index 00000000000..4c714751f7d --- /dev/null +++ b/src/Interpreters/OptimizeDateFilterVisitor.cpp @@ -0,0 +1,122 @@ +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 compare_to) +{ + const DateLUTImpl & date_lut = DateLUT::instance(); + + if (converter != "toYear") return {}; + + UInt64 year = compare_to; + String from_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); + String to_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); + + if (comparator == "equals") + { + return makeASTFunction("and", + makeASTFunction("greaterOrEquals", + std::make_shared(column), + std::make_shared(from_date) + ), + makeASTFunction("lessOrEquals", + std::make_shared(column), + std::make_shared(to_date) + ) + ); + } + else if (comparator == "notEquals") + { + return makeASTFunction("or", + makeASTFunction("less", + std::make_shared(column), + std::make_shared(from_date) + ), + makeASTFunction("greater", + std::make_shared(column), + std::make_shared(to_date) + ) + ); + } + else if (comparator == "less" || comparator == "greaterOrEquals") + { + return makeASTFunction(comparator, + std::make_shared(column), + std::make_shared(from_date) + ); + } + else + { + return makeASTFunction(comparator, + std::make_shared(column), + std::make_shared(to_date) + ); + } +} + +bool rewritePredicateInPlace(ASTFunction & function, ASTPtr & ast) +{ + const static std::unordered_map swap_relations = { + {"equals", "equals"}, + {"notEquals", "notEquals"}, + {"less", "greater"}, + {"greater", "less"}, + {"lessOrEquals", "greaterOrEquals"}, + {"greaterOrEquals", "lessOrEquals"}, + }; + + if (!swap_relations.contains(function.name)) return false; + + if (!function.arguments || function.arguments->children.size() != 2) return false; + + size_t func_id = function.arguments->children.size(); + + for (size_t i = 0; i < function.arguments->children.size(); i++) + { + if (const auto * func = function.arguments->children[i]->as(); func) + { + if (func->name == "toYear") + { + func_id = i; + } + } + } + + if (func_id == function.arguments->children.size()) return false; + + size_t literal_id = 1 - func_id; + const auto * literal = function.arguments->children[literal_id]->as(); + + if (!literal || literal->value.getType() != Field::Types::UInt64) return false; + + UInt64 compare_to = literal->value.get(); + String comparator = literal_id > func_id ? function.name : swap_relations.at(function.name); + + const auto * func = function.arguments->children[func_id]->as(); + const auto * column_id = func->arguments->children.at(0)->as(); + + if (!column_id) return false; + + String column = column_id->name(); + + const auto new_ast = generateOptimizedDateFilterAST(comparator, func->name, column, compare_to); + + if (!new_ast) return false; + + ast = new_ast; + return true; +} + +void OptimizeDateFilterInPlaceData::visit(ASTFunction & function, ASTPtr & ast) const +{ + rewritePredicateInPlace(function, ast); +} +} diff --git a/src/Interpreters/OptimizeDateFilterVisitor.h b/src/Interpreters/OptimizeDateFilterVisitor.h new file mode 100644 index 00000000000..84394372901 --- /dev/null +++ b/src/Interpreters/OptimizeDateFilterVisitor.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +namespace DB +{ + +class ASTFunction; + +/// Rewrite the predicates in place +class OptimizeDateFilterInPlaceData +{ +public: + using TypeToVisit = ASTFunction; + void visit(ASTFunction & function, ASTPtr & ast) const; +}; + +using OptimizeDateFilterInPlaceMatcher = OneTypeMatcher; +using OptimizeDateFilterInPlaceVisitor = InDepthNodeVisitor; +} diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index c38b3c79026..825114b20b7 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -677,6 +678,21 @@ void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, ContextPtr context) RemoveInjectiveFunctionsVisitor(data).visit(query); } +void optimizeDateFilters(ASTSelectQuery * select_query) +{ + /// Predicates in HAVING clause has been moved to WHERE clause. + if (select_query->where()) + { + OptimizeDateFilterInPlaceVisitor::Data data; + OptimizeDateFilterInPlaceVisitor(data).visit(select_query->refWhere()); + } + if (select_query->prewhere()) + { + OptimizeDateFilterInPlaceVisitor::Data data; + OptimizeDateFilterInPlaceVisitor(data).visit(select_query->refPrewhere()); + } +} + void transformIfStringsIntoEnum(ASTPtr & query) { std::unordered_set function_names = {"if", "transform"}; @@ -780,6 +796,9 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, tables_with_columns, result.storage_snapshot->metadata, result.storage); } + /// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, toISOWeek, etc. + optimizeDateFilters(select_query); + /// GROUP BY injective function elimination. optimizeGroupBy(select_query, context); diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference new file mode 100644 index 00000000000..e5c608ddc1a --- /dev/null +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference @@ -0,0 +1,39 @@ +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 < \'1993-01-01\') OR (date1 > \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 > \'1993-12-31\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 <= \'1993-12-31\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 >= \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1997-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) OR ((date1 >= \'1994-01-01\') AND (date1 <= \'1994-12-31\'))) AND ((id >= 1) AND (id <= 3)) +SELECT + value1, + toYear(date1) AS year1 +FROM t +WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +PREWHERE (date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\') +WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql new file mode 100644 index 00000000000..563468d4f82 --- /dev/null +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (id UInt32, value1 String, date1 Date) ENGINE ReplacingMergeTree() ORDER BY id; + +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM t WHERE year1 = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; + +DROP TABLE t; From 802b63f2ab244bcdf6a5c86f541fae9a12599ada Mon Sep 17 00:00:00 2001 From: helifu Date: Thu, 11 May 2023 15:44:15 +0800 Subject: [PATCH 0643/2223] Add 'initial_query_id' field for system.processors_profile_log Facilitate profile data association and aggregation for the same query --- src/Interpreters/ProcessorsProfileLog.cpp | 2 ++ src/Interpreters/ProcessorsProfileLog.h | 1 + src/Interpreters/executeQuery.cpp | 1 + 3 files changed, 4 insertions(+) diff --git a/src/Interpreters/ProcessorsProfileLog.cpp b/src/Interpreters/ProcessorsProfileLog.cpp index f6ce801605a..e78a07bb752 100644 --- a/src/Interpreters/ProcessorsProfileLog.cpp +++ b/src/Interpreters/ProcessorsProfileLog.cpp @@ -29,6 +29,7 @@ NamesAndTypesList ProcessorProfileLogElement::getNamesAndTypes() {"plan_step", std::make_shared()}, {"plan_group", std::make_shared()}, + {"initial_query_id", std::make_shared()}, {"query_id", std::make_shared()}, {"name", std::make_shared(std::make_shared())}, {"elapsed_us", std::make_shared()}, @@ -60,6 +61,7 @@ void ProcessorProfileLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(plan_step); columns[i++]->insert(plan_group); + columns[i++]->insertData(initial_query_id.data(), initial_query_id.size()); columns[i++]->insertData(query_id.data(), query_id.size()); columns[i++]->insertData(processor_name.data(), processor_name.size()); columns[i++]->insert(elapsed_us); diff --git a/src/Interpreters/ProcessorsProfileLog.h b/src/Interpreters/ProcessorsProfileLog.h index 07837bdd10e..81d58edd913 100644 --- a/src/Interpreters/ProcessorsProfileLog.h +++ b/src/Interpreters/ProcessorsProfileLog.h @@ -19,6 +19,7 @@ struct ProcessorProfileLogElement UInt64 plan_step{}; UInt64 plan_group{}; + String initial_query_id; String query_id; String processor_name; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 9a2750f399c..37437410b3c 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1003,6 +1003,7 @@ static std::tuple executeQueryImpl( ProcessorProfileLogElement processor_elem; processor_elem.event_time = elem.event_time; processor_elem.event_time_microseconds = elem.event_time_microseconds; + processor_elem.initial_query_id = elem.client_info.initial_query_id; processor_elem.query_id = elem.client_info.current_query_id; auto get_proc_id = [](const IProcessor & proc) -> UInt64 From 515d30caeab92babf612213118cd7f257c5b2bf4 Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 17 May 2023 17:01:00 +0800 Subject: [PATCH 0644/2223] Add test case. --- .../en/operations/system-tables/processors_profile_log.md | 8 +++++--- .../0_stateless/02210_processors_profile_log_2.reference | 1 + .../queries/0_stateless/02210_processors_profile_log_2.sh | 2 ++ 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/system-tables/processors_profile_log.md b/docs/en/operations/system-tables/processors_profile_log.md index a6ff15642a1..5eedb5a5dae 100644 --- a/docs/en/operations/system-tables/processors_profile_log.md +++ b/docs/en/operations/system-tables/processors_profile_log.md @@ -5,16 +5,18 @@ This table contains profiling on processors level (that you can find in [`EXPLAI Columns: - `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the event happened. -- `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time when the event happened. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the event happened. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time with microseconds precision when the event happened. - `id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of processor - `parent_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Parent processors IDs +- `plan_step` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of the query plan step which created this processor. The value is zero if the processor was not added from any step. +- `plan_group` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Group of the processor if it was created by query plan step. A group is a logical partitioning of processors added from the same query plan step. Group is used only for beautifying the result of EXPLAIN PIPELINE result. +- `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the initial query (for distributed query execution). - `query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the query - `name` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Name of the processor. - `elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was executed. - `input_wait_elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was waiting for data (from other processor). - `output_wait_elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was waiting because output port was full. -- `plan_step` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of the query plan step which created this processor. The value is zero if the processor was not added from any step. -- `plan_group` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Group of the processor if it was created by query plan step. A group is a logical partitioning of processors added from the same query plan step. Group is used only for beautifying the result of EXPLAIN PIPELINE result. - `input_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of rows consumed by processor. - `input_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of bytes consumed by processor. - `output_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of rows generated by processor. diff --git a/tests/queries/0_stateless/02210_processors_profile_log_2.reference b/tests/queries/0_stateless/02210_processors_profile_log_2.reference index 5467c7ef2ba..3d0948105d2 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log_2.reference +++ b/tests/queries/0_stateless/02210_processors_profile_log_2.reference @@ -9,3 +9,4 @@ NullSource 0 0 0 0 NumbersMt 0 0 1000000 8000000 Resize 1 8 1 8 Resize 1 8 1 8 +1 \ No newline at end of file diff --git a/tests/queries/0_stateless/02210_processors_profile_log_2.sh b/tests/queries/0_stateless/02210_processors_profile_log_2.sh index 93eabc2f0fe..044954a4e96 100755 --- a/tests/queries/0_stateless/02210_processors_profile_log_2.sh +++ b/tests/queries/0_stateless/02210_processors_profile_log_2.sh @@ -17,3 +17,5 @@ EOF ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} -q "select name, sum(input_rows), sum(input_bytes), sum(output_rows), sum(output_bytes) from system.processors_profile_log where query_id = '${QUERY_ID}' group by name, plan_step, plan_group order by name, sum(input_rows), sum(input_bytes), sum(output_rows), sum(output_bytes)" + +${CLICKHOUSE_CLIENT} -q "select countDistinct(initial_query_id) from system.processors_profile_log where query_id = '${QUERY_ID}'" \ No newline at end of file From cb130a1eb3f3069f1f7153e2ad3488dc06f93d2e Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 24 May 2023 20:41:05 +0800 Subject: [PATCH 0645/2223] Add a new line to the reference file --- .../0_stateless/02210_processors_profile_log_2.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02210_processors_profile_log_2.reference b/tests/queries/0_stateless/02210_processors_profile_log_2.reference index 3d0948105d2..b9a848131fd 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log_2.reference +++ b/tests/queries/0_stateless/02210_processors_profile_log_2.reference @@ -9,4 +9,4 @@ NullSource 0 0 0 0 NumbersMt 0 0 1000000 8000000 Resize 1 8 1 8 Resize 1 8 1 8 -1 \ No newline at end of file +1 From 1bc4eb1a6c26db08b6af33745feb83f92f093e9b Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Thu, 25 May 2023 13:47:03 +0800 Subject: [PATCH 0646/2223] OptimizeDateFilterVisitor: Revise variable names for clarity --- .../OptimizeDateFilterVisitor.cpp | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/OptimizeDateFilterVisitor.cpp b/src/Interpreters/OptimizeDateFilterVisitor.cpp index 4c714751f7d..58e1b3335f9 100644 --- a/src/Interpreters/OptimizeDateFilterVisitor.cpp +++ b/src/Interpreters/OptimizeDateFilterVisitor.cpp @@ -10,26 +10,25 @@ namespace DB { -ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 compare_to) +ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 year) { const DateLUTImpl & date_lut = DateLUT::instance(); if (converter != "toYear") return {}; - UInt64 year = compare_to; - String from_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); - String to_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); + String start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); + String end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); if (comparator == "equals") { return makeASTFunction("and", makeASTFunction("greaterOrEquals", std::make_shared(column), - std::make_shared(from_date) + std::make_shared(start_date) ), makeASTFunction("lessOrEquals", std::make_shared(column), - std::make_shared(to_date) + std::make_shared(end_date) ) ); } @@ -38,11 +37,11 @@ ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & return makeASTFunction("or", makeASTFunction("less", std::make_shared(column), - std::make_shared(from_date) + std::make_shared(start_date) ), makeASTFunction("greater", std::make_shared(column), - std::make_shared(to_date) + std::make_shared(end_date) ) ); } @@ -50,14 +49,14 @@ ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & { return makeASTFunction(comparator, std::make_shared(column), - std::make_shared(from_date) + std::make_shared(start_date) ); } else { return makeASTFunction(comparator, std::make_shared(column), - std::make_shared(to_date) + std::make_shared(end_date) ); } } From 75791d7a63b9a6a579e1f036cdffc321bcc9fa2d Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 25 May 2023 07:51:32 +0000 Subject: [PATCH 0647/2223] Added input_format_csv_trim_whitespaces parameter --- docs/en/interfaces/formats.md | 1 + .../operations/settings/settings-formats.md | 32 +++++ docs/ru/interfaces/formats.md | 17 +++ docs/ru/operations/settings/settings.md | 62 +++++++++ src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/IO/ReadHelpers.cpp | 19 +-- .../Formats/Impl/CSVRowInputFormat.cpp | 17 ++- .../02764_csv_trim_whitespaces.reference | 122 ++++++++++++++++++ .../0_stateless/02764_csv_trim_whitespaces.sh | 55 ++++++++ 11 files changed, 317 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02764_csv_trim_whitespaces.reference create mode 100755 tests/queries/0_stateless/02764_csv_trim_whitespaces.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 2ab9e8caec4..f19fd94dcd8 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -467,6 +467,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [output_format_csv_crlf_end_of_line](/docs/en/operations/settings/settings-formats.md/#output_format_csv_crlf_end_of_line) - if it is set to true, end of line in CSV output format will be `\r\n` instead of `\n`. Default value - `false`. - [input_format_csv_skip_first_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_first_lines) - skip the specified number of lines at the beginning of data. Default value - `0`. - [input_format_csv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_csv_detect_header) - automatically detect header with names and types in CSV format. Default value - `true`. +- [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 3b87b829c92..cb7d98a4876 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -882,6 +882,38 @@ My NULL My NULL ``` +### input_format_csv_trim_whitespaces {#input_format_csv_trim_whitespaces} + +Trims spaces and tabs in non-quoted CSV strings. + +Default value: `true`. + +**Examples** + +Query + +```bash +echo ' string ' |./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true +``` + +Result + +```text +"string" +``` + +Query + +```bash +echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=false +``` + +Result + +```text +" string " +``` + ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index bef5c223281..48a6132170a 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -387,6 +387,23 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR Формат CSV поддерживает вывод totals и extremes аналогично `TabSeparated`. + +### CSV опции форматирования {#csv-format-settings} + +- [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter) - символ, который будет считаться разделителем в CSV данных. Значение по умолчанию - `,`. +- [format_csv_allow_single_quotes](../operations/settings/settings.md#format_csv_allow_single_quotes) - разрешить строки в одинарных кавычках. Значение по умолчанию - `true`. +- [format_csv_allow_double_quotes](../operations/settings/settings.md#format_csv_allow_double_quotes) - разрешить строки в двойных кавычках. Значение по умолчанию - `true`. +- [format_csv_null_representation](../operations/settings/settings.md#format_tsv_null_representation) - пользовательское представление NULL в формате CSV. Значение по умолчанию - `\N`. +- [input_format_csv_empty_as_default](../operations/settings/settings.md#input_format_csv_empty_as_default) - рассматривать пустые поля в CSV в качестве значений по умолчанию. Значение по умолчанию - `true`. Для сложных выражений по умолчанию необходимо также включить [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#input_format_defaults_for_omitted_fields). +- [input_format_csv_enum_as_number](../operations/settings/settings.md#input_format_csv_enum_as_number) - рассматривать вставленные значения enum в форматах CSV как индексы enum. Значение по умолчанию - `false`. +- [input_format_csv_use_best_effort_in_schema_inference](../operations/settings/settings.md#input_format_csv_use_best_effort_in_schema_inference) - использовать некоторые твики и эвристики для вывода схемы в формате CSV. Если параметр отключен, все поля будут определяться как строки. Значение по умолчанию - `true`. +- [input_format_csv_arrays_as_nested_csv](../operations/settings/settings.md#input_format_csv_arrays_as_nested_csv) - при чтении массива из CSV ожидать, что его элементы были сериализованы во вложенный CSV и затем помещены в строку. Значение по умолчанию - `false`. +- [output_format_csv_crlf_end_of_line](../operations/settings/settings.md#output_format_csv_crlf_end_of_line) - если установлено значение true, конец строки в формате вывода CSV будет `\r\n` вместо `\n`. Значение по умолчанию - `false`. +- [input_format_csv_skip_first_lines](../operations/settings/settings.md#input_format_csv_skip_first_lines) - пропустить указанное количество строк в начале данных. Значение по умолчанию - `0`. +- [input_format_csv_detect_header](../operations/settings/settings.md#input_format_csv_detect_header) - обнаружить заголовок с именами и типами в формате CSV. Значение по умолчанию - `true`. +- [input_format_csv_trim_whitespaces](../operations/settings/settings.md#input_format_csv_trim_whitespaces) - удалить пробелы и символы табуляции из строк без кавычек. +Значение по умолчанию - `true`. + ## CSVWithNames {#csvwithnames} Выводит также заголовок, аналогично [TabSeparatedWithNames](#tabseparatedwithnames). diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index fa3ea582c55..e9b7091c8b8 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1589,6 +1589,24 @@ SELECT area/period FROM account_orders FORMAT JSON; Символ, интерпретируемый как разделитель в данных формата CSV. По умолчанию — `,`. +## format_csv_allow_double_quotes {#format_csv_allow_double_quotes} + +Если установлено значение true, разрешить строки в двойных кавычках. + +Включено по умолчанию. + +## input_format_csv_empty_as_default {#input_format_csv_empty_as_default} + +Если включено, заменяет пустые поля ввода в CSV значениями по умолчанию. Для сложных выражений по умолчанию `input_format_defaults_for_omitted_fields` также должен быть включен. + +Включено по умолчанию. + +## input_format_csv_arrays_as_nested_csv {#input_format_csv_arrays_as_nested_csv} + +При чтении массива из CSV ожидайте, что его элементы были сериализованы во вложенный CSV, а затем помещены в строку. Пример: "[""Hello"", ""world"", ""42"""" TV""]". Скобки вокруг массива могут быть опущены. + +По умолчанию отключены. + ## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} Для формата CSV включает или выключает парсинг неэкранированной строки `NULL` как литерала (синоним для `\N`) @@ -1665,6 +1683,50 @@ SELECT * FROM table_with_enum_column_for_csv_insert; Использовать в качестве разделителя строк для CSV формата CRLF (DOS/Windows стиль) вместо LF (Unix стиль). +## input_format_csv_detect_header {#input_format_csv_detect_header} + +Обнаружить заголовок с именами и типами в формате CSV. + +Значение по умолчанию - `true`. + +## input_format_csv_skip_first_lines {#input_format_csv_skip_first_lines} + +Количество строк, пропускаемых в начале данных в формате ввода CSV. + +Значение по умолчанию: `0`. + +## input_format_csv_trim_whitespaces {#input_format_csv_trim_whitespaces} + +Удалить пробелы и символы табуляции из строк без кавычек. + +Значение по умолчанию: `true`. + +**Примеры** + +Запрос + +```bash +echo ' string ' |./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true +``` + +Результат + +```text +"string" +``` + +Запрос + +```bash +echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=false +``` + +Результат + +```text +" string " +``` + ## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} Использовать в качестве разделителя строк для TSV формата CRLF (DOC/Windows стиль) вместо LF (Unix стиль). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1df0a8af24f..750b6e16c4b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -844,6 +844,7 @@ class IColumn; M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ + M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index dd6252b96f1..ada8751545c 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -69,6 +69,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.use_best_effort_in_schema_inference = settings.input_format_csv_use_best_effort_in_schema_inference; format_settings.csv.skip_first_lines = settings.input_format_csv_skip_first_lines; format_settings.csv.try_detect_header = settings.input_format_csv_detect_header; + format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index c88af650671..3ae579cd552 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -136,6 +136,7 @@ struct FormatSettings UInt64 skip_first_lines = 0; String custom_delimiter; bool try_detect_header = true; + bool trim_whitespaces = true; } csv; struct HiveText diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 99d25ee6613..8dc05e75855 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -849,15 +849,18 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & if constexpr (WithResize) { - /** CSV format can contain insignificant spaces and tabs. - * Usually the task of skipping them is for the calling code. - * But in this case, it will be difficult to do this, so remove the trailing whitespace by ourself. - */ - size_t size = s.size(); - while (size > 0 && (s[size - 1] == ' ' || s[size - 1] == '\t')) - --size; + if (settings.trim_whitespaces) [[likely]] + { + /** CSV format can contain insignificant spaces and tabs. + * Usually the task of skipping them is for the calling code. + * But in this case, it will be difficult to do this, so remove the trailing whitespace by ourself. + */ + size_t size = s.size(); + while (size > 0 && (s[size - 1] == ' ' || s[size - 1] == '\t')) + --size; - s.resize(size); + s.resize(size); + } } return; } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index de955d81651..9922bd41442 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -152,7 +153,9 @@ void CSVFormatReader::skipFieldDelimiter() template String CSVFormatReader::readCSVFieldIntoString() { - skipWhitespacesAndTabs(*buf); + if (format_settings.csv.trim_whitespaces) [[likely]] + skipWhitespacesAndTabs(*buf); + String field; if constexpr (read_string) readCSVString(field, *buf, format_settings.csv); @@ -200,7 +203,6 @@ void CSVFormatReader::skipHeaderRow() template std::vector CSVFormatReader::readRowImpl() { - std::vector fields; do { @@ -280,7 +282,16 @@ bool CSVFormatReader::readField( bool is_last_file_column, const String & /*column_name*/) { - skipWhitespacesAndTabs(*buf); + if (format_settings.csv.trim_whitespaces) [[likely]] + skipWhitespacesAndTabs(*buf); + else if (type->isNullable()) + { + auto nested_type = typeid_cast(type.get())->getNestedType(); + if (!isStringOrFixedString(nested_type)) + skipWhitespacesAndTabs(*buf); + } + else if (!isStringOrFixedString(type)) + skipWhitespacesAndTabs(*buf); const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; const bool at_last_column_line_end = is_last_file_column && (buf->eof() || *buf->position() == '\n' || *buf->position() == '\r'); diff --git a/tests/queries/0_stateless/02764_csv_trim_whitespaces.reference b/tests/queries/0_stateless/02764_csv_trim_whitespaces.reference new file mode 100644 index 00000000000..a2ea31ddae7 --- /dev/null +++ b/tests/queries/0_stateless/02764_csv_trim_whitespaces.reference @@ -0,0 +1,122 @@ +" trim_false_tab_left" +"trim_false_tab_right " +"trim_false_ tab_middle" +" trim_false_ tab_everywhere " +" trim_false_fixed_string_ tab_everywhere " +" quoted_trim_false_ tab_everywhere " +" trim_false_csv_field1 ","123 ","5.0 "," 12.0123"," quoted_string1" +" trim_false_csv_field2 "," 321 "," 0.5","21.321 "," quoted_ string2 " +" trim_false_csv_field1_with_ structure ",123,5,12.0123," quoted_string " +" trim_false_csv_field2_with_structure ",321,0.5,21.321," quoted_ _string2 " +" trim_false_space_left" +"trim_false_space_right " +"trim_false_ space_middle" +" trim_false_ space_everywhere " +" trim_false_fixed_string_ space_everywhere " +" quoted_trim_false_ space_everywhere " +" trim_false_csv_field1 ","123 ","5.0 "," 12.0123"," quoted_string1" +" trim_false_csv_field2 "," 321 "," 0.5","21.321 "," quoted_ string2 " +" trim_false_csv_field1_with_ structure ",123,5,12.0123," quoted_string " +" trim_false_csv_field2_with_structure ",321,0.5,21.321," quoted_ _string2 " +" trim_false_tab_space_left" +"trim_false_tab_space_right " +"trim_false_ tab_space_middle" +" trim_false_ tab_space_everywhere " +" trim_false_fixed_string_ tab_space_everywhere " +" quoted_trim_false_ tab_space_everywhere " +" trim_false_csv_field1 ","123 ","5.0 "," 12.0123"," quoted_string1" +" trim_false_csv_field2 "," 321 "," 0.5","21.321 "," quoted_ string2 " +" trim_false_csv_field1_with_ structure ",123,5,12.0123," quoted_string " +" trim_false_csv_field2_with_structure ",321,0.5,21.321," quoted_ _string2 " +8 +8 +16 +16 +32.32 +32.32 +64.64 +64.64 +"2023-05-22" +"2023-05-22" +"2023-05-22" +"2023-05-22" +"2023-05-22 00:00:00" +"2023-05-22 00:00:00" +"2023-05-22 00:00:00.000" +"2023-05-22 00:00:00.000" +"trim_true_tab_left" +"trim_true_tab_right" +"trim_true_ tab_middle" +"trim_true_ tab_everywhere" +"trim_true_fixed_string_ tab_everywhere" +" quoted_trim_true_ tab_everywhere " +"trim_true_csv_field1",123,5,12.0123," quoted_string1" +"trim_true_csv_field2",321,0.5,21.321," quoted_ string2 " +"trim_true_csv_field1_with_ structure",123,5,12.0123," quoted_string " +"trim_true_csv_field2_with_structure",321,0.5,21.321," quoted_ _string2 " +"trim_true_space_left" +"trim_true_space_right" +"trim_true_ space_middle" +"trim_true_ space_everywhere" +"trim_true_fixed_string_ space_everywhere" +" quoted_trim_true_ space_everywhere " +"trim_true_csv_field1",123,5,12.0123," quoted_string1" +"trim_true_csv_field2",321,0.5,21.321," quoted_ string2 " +"trim_true_csv_field1_with_ structure",123,5,12.0123," quoted_string " +"trim_true_csv_field2_with_structure",321,0.5,21.321," quoted_ _string2 " +"trim_true_tab_space_left" +"trim_true_tab_space_right" +"trim_true_ tab_space_middle" +"trim_true_ tab_space_everywhere" +"trim_true_fixed_string_ tab_space_everywhere" +" quoted_trim_true_ tab_space_everywhere " +"trim_true_csv_field1",123,5,12.0123," quoted_string1" +"trim_true_csv_field2",321,0.5,21.321," quoted_ string2 " +"trim_true_csv_field1_with_ structure",123,5,12.0123," quoted_string " +"trim_true_csv_field2_with_structure",321,0.5,21.321," quoted_ _string2 " +8 +8 +16 +16 +32.32 +32.32 +64.64 +64.64 +"2023-05-22" +"2023-05-22" +"2023-05-22" +"2023-05-22" +"2023-05-22 00:00:00" +"2023-05-22 00:00:00" +"2023-05-22 00:00:00.000" +"2023-05-22 00:00:00.000" +" custom_csv_tab_left" +"custom_csv_tab_right " +"custom_csv_ tab_middle" +" custom_csv_ tab_everywhere " +" custom_csv_fixed_string_ tab_everywhere " +" quoted_custom_csv_ tab_everywhere " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " +" custom_csv_space_left" +"custom_csv_space_right " +"custom_csv_ space_middle" +" custom_csv_ space_everywhere " +" custom_csv_fixed_string_ space_everywhere " +" quoted_custom_csv_ space_everywhere " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " +" custom_csv_tab_space_left" +"custom_csv_tab_space_right " +"custom_csv_ tab_space_middle" +" custom_csv_ tab_space_everywhere " +" custom_csv_fixed_string_ tab_space_everywhere " +" quoted_custom_csv_ tab_space_everywhere " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " +" custom_csv_field_with_ structure ",123,5,12.0123," custom_csv_quoted_string " +" custom_csv_field2_with_structure ",321,0.5,21.321," custom_csv_quoted_ _string2 " diff --git a/tests/queries/0_stateless/02764_csv_trim_whitespaces.sh b/tests/queries/0_stateless/02764_csv_trim_whitespaces.sh new file mode 100755 index 00000000000..9b3b7231f6e --- /dev/null +++ b/tests/queries/0_stateless/02764_csv_trim_whitespaces.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +BOOLS=('false' 'true') + +WHITESPACES=( '\t' ' ' '\t ') +WHITESPACES_NAMES=('tab' 'space' 'tab_space') + +DATA_TYPES=( 'Int8' 'Int16' 'Float32' 'Float64' 'Date' 'Date32' 'DateTime' 'DateTime64') +DATA_VALUES=( '8' '16' '32.32' '64.64' '2023-05-22' '2023-05-22' '2023-05-22 00:00:00' '2023-05-22 00:00:00.000') + +for trim in "${BOOLS[@]}" +do + for wsIndex in "${!WHITESPACES[@]}"; + do + whitespace=${WHITESPACES[$wsIndex]} + whitespace_name=${WHITESPACES_NAMES[$wsIndex]} + echo -e "${whitespace}trim_${trim}_${whitespace_name}_left" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "trim_${trim}_${whitespace_name}_right${whitespace}" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "trim_${trim}_${whitespace}${whitespace_name}_middle" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "${whitespace}trim_${trim}_${whitespace}${whitespace_name}_everywhere${whitespace}" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "${whitespace}trim_${trim}_fixed_string_${whitespace}${whitespace_name}_everywhere${whitespace}" | $CLICKHOUSE_LOCAL -S "c1 FixedString(64)" --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select toString(c1) from table FORMAT CSV" + echo -e "\"${whitespace}quoted_trim_${trim}_${whitespace}${whitespace_name}_everywhere${whitespace}\"" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "${whitespace}trim_${trim}_csv_field1${whitespace},123${whitespace},5.0${whitespace},${whitespace}12.0123,\"${whitespace}quoted_string1\"\n${whitespace}trim_${trim}_csv_field2${whitespace},${whitespace}321${whitespace},${whitespace}0.5,21.321${whitespace},\"${whitespace}quoted_${whitespace}string2${whitespace}\"${whitespace}" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + echo -e "${whitespace}trim_${trim}_csv_field1_with_${whitespace}structure${whitespace},${whitespace}123,${whitespace}5.0${whitespace},12.0123${whitespace},\"${whitespace}quoted_string${whitespace}\"\n${whitespace}trim_${trim}_csv_field2_with_structure${whitespace},${whitespace}321${whitespace},0.5,21.321,\"${whitespace}quoted_${whitespace}_string2${whitespace}\"${whitespace}" | $CLICKHOUSE_LOCAL -S "c1 String, c2 Int32, c3 Float, c4 Double, c5 String" --input_format_csv_trim_whitespaces=${trim} --input-format="CSV" -q "select * from table FORMAT CSV" + done + + for type_index in "${!DATA_TYPES[@]}"; + do + type=${DATA_TYPES[$type_index]} + value=${DATA_VALUES[$type_index]} + echo -e "\t ${value} \t" | $CLICKHOUSE_LOCAL -S "c1 ${type}" --input-format="CSV" --input_format_csv_trim_whitespaces=${trim} -q "select * from table FORMAT CSV" + echo -e "\t ${value} \t" | $CLICKHOUSE_LOCAL -S "c1 Nullable(${type})" --input-format="CSV" --input_format_csv_trim_whitespaces=${trim} -q "select * from table FORMAT CSV" + done +done + +## Custom CSV tested with input_format_csv_trim_whitespaces = false. +## Custom CSV with input_format_csv_trim_whitespaces=true doesn't trim whitespaces from the left side at the moment +for wsIndex in "${!WHITESPACES[@]}"; +do + whitespace=${WHITESPACES[$wsIndex]} + whitespace_name=${WHITESPACES_NAMES[$wsIndex]} + echo -e "${whitespace}custom_csv_${whitespace_name}_left" | $CLICKHOUSE_LOCAL --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + echo -e "custom_csv_${whitespace_name}_right${whitespace}" | $CLICKHOUSE_LOCAL --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + echo -e "custom_csv_${whitespace}${whitespace_name}_middle" | $CLICKHOUSE_LOCAL --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + echo -e "${whitespace}custom_csv_${whitespace}${whitespace_name}_everywhere${whitespace}" | $CLICKHOUSE_LOCAL --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + echo -e "${whitespace}custom_csv_fixed_string_${whitespace}${whitespace_name}_everywhere${whitespace}" | $CLICKHOUSE_LOCAL -S "c1 FixedString(64)" --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select toString(c1) from table FORMAT CSV" + echo -e "\"${whitespace}quoted_custom_csv_${whitespace}${whitespace_name}_everywhere${whitespace}\"" | $CLICKHOUSE_LOCAL --input-format="CustomSeparated" --input_format_csv_trim_whitespaces=false --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + + echo -e "${whitespace}custom_csv_field_with_${whitespace}structure${whitespace},123,5.0,12.0123,\"${whitespace}custom_csv_quoted_string${whitespace}\"\n${whitespace}custom_csv_field2_with_structure${whitespace},321,0.5,21.321,\"${whitespace}custom_csv_quoted_${whitespace}_string2${whitespace}\"" | $CLICKHOUSE_LOCAL --input_format_csv_trim_whitespaces=false --input-format="CustomSeparated" --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" + echo -e "${whitespace}custom_csv_field_with_${whitespace}structure${whitespace},123,5.0,12.0123,\"${whitespace}custom_csv_quoted_string${whitespace}\"\n${whitespace}custom_csv_field2_with_structure${whitespace},321,0.5,21.321,\"${whitespace}custom_csv_quoted_${whitespace}_string2${whitespace}\"" | $CLICKHOUSE_LOCAL -S "c1 String, c2 Int32, c3 Float, c4 Double, c5 String" --input_format_csv_trim_whitespaces=false --input-format="CustomSeparated" --format_custom_escaping_rule=CSV --format_custom_field_delimiter=',' --format_csv_delimiter=',' -q "select * from table FORMAT CSV" +done From 4eb944fef11dacc95873f9e5de9949c381d4dc79 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 25 May 2023 08:05:11 +0000 Subject: [PATCH 0648/2223] minor changes in documentation --- docs/en/operations/settings/settings-formats.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index cb7d98a4876..1db1b5066c3 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -893,7 +893,7 @@ Default value: `true`. Query ```bash -echo ' string ' |./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true +echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true ``` Result diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e9b7091c8b8..4c2117b2b87 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1706,7 +1706,7 @@ SELECT * FROM table_with_enum_column_for_csv_insert; Запрос ```bash -echo ' string ' |./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true +echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_trim_whitespaces=true ``` Результат From 9582d9e892d5bfd7b0c6b46b11f758c15f5ab0cc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 09:57:32 +0200 Subject: [PATCH 0649/2223] Fix UB for INTO OUTFILE extensions (APPEND / AND STDOUT) MSAn report: ==38627==WARNING: MemorySanitizer: use-of-uninitialized-value 0 0x555599f5e114 in std::__1::__unique_if::__unique_single std::__1::make_unique[abi:v15000]<> build_docker/./contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:714:32 1 0x555599f5e114 in DB::ClientBase::initOutputFormat() build_docker/./src/Client/ClientBase.cpp:604:21 2 0x555599f590a8 in DB::ClientBase::onData() build_docker/./src/Client/ClientBase.cpp:446:5 3 0x555599f6f36e in DB::ClientBase::receiveAndProcessPacket() build_docker/./src/Client/ClientBase.cpp:1019:17 4 0x555599f6e863 in DB::ClientBase::receiveResult() build_docker/./src/Client/ClientBase.cpp:987:18 5 0x555599f6c05b in DB::ClientBase::processOrdinaryQuery() build_docker/./src/Client/ClientBase.cpp:905:13 6 0x555599f67e05 in DB::ClientBase::processParsedSingleQuery() build_docker/./src/Client/ClientBase.cpp:1711:13 7 0x555599f86fb6 in DB::ClientBase::executeMultiQuery(std::__1::basic_string, std::__1::allocator > const&) build_docker/./src/Client/ClientBase.cpp:1975:21 Uninitialized value was created by a heap allocation 8 0x55559bd3e038 in DB::ParserExplainQuery::parseImpl(DB::IParser::Pos&, std::__1::shared_ptr&, DB::Expected&) build_docker/./src/Parsers/ParserExplainQuery.cpp:53:26 9 0x55559bce31f4 in DB::IParserBase::parse(DB::IParser::Pos&, std::__1::shared_ptr&, DB::Expected&)::$_0::operator()() const build_docker/./src/Parsers/IParserBase.cpp:13:20 .. 21 0x55559be13b5c in DB::parseQueryAndMovePosition(DB::IParser&, char const*&, char const*, std::__1::basic_string, std::__1::allocator > const&, bool, unsigned long, unsigned long) build_docker/./src/Parsers/parseQuery.cpp:357:18 22 0x555599f5673a in DB::ClientBase::parseQuery(char const*&, char const*, bool) const build_docker/./src/Client/ClientBase.cpp:362:15 23 0x555599f84a4f in DB::ClientBase::analyzeMultiQueryText() build_docker/./src/Client/ClientBase.cpp:1821:24 24 0x555599f867b3 in DB::ClientBase::executeMultiQuery(std::__1::basic_string, std::__1::allocator > const&) build_docker/./src/Client/ClientBase.cpp:1910:22 25 0x555599f8a2fd in DB::ClientBase::processQueryText(std::__1::basic_string, std::__1::allocator > const&) build_docker/./src/Client/ClientBase.cpp:2120:12 26 0x555599f94aee in DB::ClientBase::runNonInteractive() build_docker/./src/Client/ClientBase.cpp:2403:9 Signed-off-by: Azat Khuzhin --- src/Parsers/ASTQueryWithOutput.h | 4 ++-- .../02767_into_outfile_extensions_msan.reference | 2 ++ .../0_stateless/02767_into_outfile_extensions_msan.sh | 11 +++++++++++ 3 files changed, 15 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference create mode 100755 tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index 09f08772468..7db021405e7 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -15,8 +15,8 @@ class ASTQueryWithOutput : public IAST { public: ASTPtr out_file; - bool is_into_outfile_with_stdout; - bool is_outfile_append; + bool is_into_outfile_with_stdout = false; + bool is_outfile_append = false; ASTPtr format; ASTPtr settings_ast; ASTPtr compression; diff --git a/tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference new file mode 100644 index 00000000000..0c8b489c2fd --- /dev/null +++ b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference @@ -0,0 +1,2 @@ +Expression ((Projection + Before ORDER BY)) + ReadFromStorage (SystemNumbers) diff --git a/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh new file mode 100755 index 00000000000..d1e7312b150 --- /dev/null +++ b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +out="explain1.$CLICKHOUSE_TEST_UNIQUE_NAME.out" +# only EXPLAIN triggers the problem under MSan +$CLICKHOUSE_CLIENT -q "explain select * from numbers(1) into outfile '$out'" +cat "$out" +rm -f "$out" From c053d75741aead3764e3be9d955f496dc47749d5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 10:09:33 +0200 Subject: [PATCH 0650/2223] Fix formatting of INTO OUTFILE extensions (APPEND / AND STDOUT) Signed-off-by: Azat Khuzhin --- src/Parsers/ASTQueryWithOutput.cpp | 7 +++++++ ...8_into_outfile_extensions_format.reference | 20 +++++++++++++++++++ .../02768_into_outfile_extensions_format.sh | 12 +++++++++++ 3 files changed, 39 insertions(+) create mode 100644 tests/queries/0_stateless/02768_into_outfile_extensions_format.reference create mode 100755 tests/queries/0_stateless/02768_into_outfile_extensions_format.sh diff --git a/src/Parsers/ASTQueryWithOutput.cpp b/src/Parsers/ASTQueryWithOutput.cpp index 9f771ab92e3..5f717715a69 100644 --- a/src/Parsers/ASTQueryWithOutput.cpp +++ b/src/Parsers/ASTQueryWithOutput.cpp @@ -35,6 +35,13 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTO OUTFILE " << (s.hilite ? hilite_none : ""); out_file->formatImpl(s, state, frame); + + s.ostr << (s.hilite ? hilite_keyword : ""); + if (is_outfile_append) + s.ostr << " APPEND"; + if (is_into_outfile_with_stdout) + s.ostr << " AND STDOUT"; + s.ostr << (s.hilite ? hilite_none : ""); } if (format) diff --git a/tests/queries/0_stateless/02768_into_outfile_extensions_format.reference b/tests/queries/0_stateless/02768_into_outfile_extensions_format.reference new file mode 100644 index 00000000000..4ebc1da8865 --- /dev/null +++ b/tests/queries/0_stateless/02768_into_outfile_extensions_format.reference @@ -0,0 +1,20 @@ +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' +; + +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' AND STDOUT +; + +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' APPEND +; + +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' APPEND AND STDOUT +; + diff --git a/tests/queries/0_stateless/02768_into_outfile_extensions_format.sh b/tests/queries/0_stateless/02768_into_outfile_extensions_format.sh new file mode 100755 index 00000000000..756488076f9 --- /dev/null +++ b/tests/queries/0_stateless/02768_into_outfile_extensions_format.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo " +select * from numbers(1) into outfile '/dev/null'; +select * from numbers(1) into outfile '/dev/null' and stdout; +select * from numbers(1) into outfile '/dev/null' append; +select * from numbers(1) into outfile '/dev/null' append and stdout; +" | clickhouse-format -n From b30cfe55030fd819d0dfa1da62ed74ab29ea9e63 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 10:14:47 +0200 Subject: [PATCH 0651/2223] Fix UB in ASTWatchQuery for is_watch_events Signed-off-by: Azat Khuzhin --- src/Parsers/ASTWatchQuery.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTWatchQuery.h b/src/Parsers/ASTWatchQuery.h index 29dadd71675..156fe6828bc 100644 --- a/src/Parsers/ASTWatchQuery.h +++ b/src/Parsers/ASTWatchQuery.h @@ -23,7 +23,7 @@ class ASTWatchQuery : public ASTQueryWithTableAndOutput public: ASTPtr limit_length; - bool is_watch_events; + bool is_watch_events = false; ASTWatchQuery() = default; String getID(char) const override { return "WatchQuery_" + getDatabase() + "_" + getTable(); } From b680697cced2d9a5dabe87219db8b5e75d4867e2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 10:16:54 +0200 Subject: [PATCH 0652/2223] Initialize POD members of ASTs to make it less error-prone The cost of initializing members is insignificant in compare to parsing, while the cost of the error is high. Signed-off-by: Azat Khuzhin --- src/Parsers/ASTColumnDeclaration.h | 2 +- src/Parsers/ASTDictionaryAttributeDeclaration.h | 8 ++++---- src/Parsers/ASTOrderByElement.h | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index 2008e4f99d1..45814551db8 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -16,7 +16,7 @@ public: std::optional null_modifier; String default_specifier; ASTPtr default_expression; - bool ephemeral_default; + bool ephemeral_default = false; ASTPtr comment; ASTPtr codec; ASTPtr ttl; diff --git a/src/Parsers/ASTDictionaryAttributeDeclaration.h b/src/Parsers/ASTDictionaryAttributeDeclaration.h index b6572e89d16..52103650684 100644 --- a/src/Parsers/ASTDictionaryAttributeDeclaration.h +++ b/src/Parsers/ASTDictionaryAttributeDeclaration.h @@ -19,13 +19,13 @@ public: /// Attribute expression ASTPtr expression; /// Is attribute mirrored to the parent identifier - bool hierarchical; + bool hierarchical = false; /// Is hierarchical attribute bidirectional - bool bidirectional; + bool bidirectional = false; /// Flag that shows whether the id->attribute image is injective - bool injective; + bool injective = false; /// MongoDB object ID - bool is_object_id; + bool is_object_id = false; String getID(char delim) const override { return "DictionaryAttributeDeclaration" + (delim + name); } diff --git a/src/Parsers/ASTOrderByElement.h b/src/Parsers/ASTOrderByElement.h index 4d07405c17a..468d2161dff 100644 --- a/src/Parsers/ASTOrderByElement.h +++ b/src/Parsers/ASTOrderByElement.h @@ -11,14 +11,14 @@ namespace DB class ASTOrderByElement : public IAST { public: - int direction; /// 1 for ASC, -1 for DESC - int nulls_direction; /// Same as direction for NULLS LAST, opposite for NULLS FIRST. - bool nulls_direction_was_explicitly_specified; + int direction = 0; /// 1 for ASC, -1 for DESC + int nulls_direction = 0; /// Same as direction for NULLS LAST, opposite for NULLS FIRST. + bool nulls_direction_was_explicitly_specified = false; /** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */ ASTPtr collation; - bool with_fill; + bool with_fill = false; ASTPtr fill_from; ASTPtr fill_to; ASTPtr fill_step; From 7e68f61df1671b0be0b30c8cef33f34c8c971d3b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 08:43:11 +0000 Subject: [PATCH 0653/2223] Fix snapshot recovery --- programs/keeper-converter/KeeperConverter.cpp | 4 +- programs/server/config.d/users.xml | 26 ----- src/Coordination/FourLetterCommand.cpp | 1 + src/Coordination/KeeperContext.cpp | 35 ++++++- src/Coordination/KeeperContext.h | 4 + src/Coordination/KeeperDiskSelector.h | 37 -------- src/Coordination/KeeperDispatcher.cpp | 11 ++- src/Coordination/KeeperDispatcher.h | 7 ++ src/Coordination/KeeperServer.cpp | 4 +- src/Coordination/KeeperServer.h | 2 +- src/Coordination/KeeperSnapshotManager.cpp | 20 +--- src/Coordination/KeeperSnapshotManager.h | 21 ++-- src/Coordination/KeeperSnapshotManagerS3.cpp | 24 ++--- src/Coordination/KeeperSnapshotManagerS3.h | 8 +- src/Coordination/KeeperStateMachine.cpp | 95 ++++++++++++++----- src/Coordination/KeeperStateMachine.h | 2 +- .../test_keeper_four_word_command/test.py | 2 + 17 files changed, 165 insertions(+), 138 deletions(-) delete mode 100644 programs/server/config.d/users.xml delete mode 100644 src/Coordination/KeeperDiskSelector.h diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index f2389021cb6..a049e6bc2b3 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -55,8 +55,8 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) DB::KeeperSnapshotManager manager(1, keeper_context); auto snp = manager.serializeSnapshotToBuffer(snapshot); - auto path = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); - std::cout << "Snapshot serialized to path:" << path << std::endl; + auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); + std::cout << "Snapshot serialized to path:" << fs::path(file_info.disk->getPath()) / file_info.path << std::endl; } catch (...) { diff --git a/programs/server/config.d/users.xml b/programs/server/config.d/users.xml deleted file mode 100644 index d8a62b45baa..00000000000 --- a/programs/server/config.d/users.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - - 10000000000 - 0 - 2 - - - 5000000000 - 20000000000 - - - - - - - - - - - - - default - - - diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 7077e792fd8..b7419bcaccc 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -292,6 +292,7 @@ String ConfCommand::run() StringBuffer buf; keeper_dispatcher.getKeeperConfigurationAndSettings()->dump(buf); + keeper_dispatcher.getKeeperContext()->dumpConfiguration(buf); return buf.str(); } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index e27cfc60cff..da49868f706 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -122,6 +122,35 @@ void KeeperContext::setStateFileDisk(DiskPtr disk) state_file_storage = std::move(disk); } +void KeeperContext::dumpConfiguration(WriteBufferFromOwnString & buf) const +{ + auto dump_disk_info = [&](const std::string_view prefix, const IDisk & disk) + { + writeText(fmt::format("{}_path=", prefix), buf); + writeText(disk.getPath(), buf); + buf.write('\n'); + + writeText(fmt::format("{}_disk=", prefix), buf); + writeText(disk.getName(), buf); + buf.write('\n'); + + }; + + { + auto log_disk = getDisk(log_storage); + dump_disk_info("log_storage", *log_disk); + + auto current_log_disk = getDisk(current_log_storage); + if (log_disk != current_log_disk) + dump_disk_info("current_log_storage", *current_log_disk); + } + + { + auto snapshot_disk = getDisk(snapshot_storage); + dump_disk_info("snapshot_storage", *snapshot_disk); + } +} + KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { const auto create_local_disk = [](const auto & path) @@ -129,7 +158,7 @@ KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::Ab if (!fs::exists(path)) fs::create_directories(path); - return std::make_shared("LogDisk", path, 0); + return std::make_shared("LocalLogDisk", path, 0); }; /// the most specialized path @@ -155,7 +184,7 @@ KeeperContext::Storage KeeperContext::getSnapshotsPathFromConfig(const Poco::Uti if (!fs::exists(path)) fs::create_directories(path); - return std::make_shared("SnapshotDisk", path, 0); + return std::make_shared("LocalSnapshotDisk", path, 0); }; /// the most specialized path @@ -181,7 +210,7 @@ KeeperContext::Storage KeeperContext::getStatePathFromConfig(const Poco::Util::A if (!fs::exists(path)) fs::create_directories(path); - return std::make_shared("SnapshotDisk", path, 0); + return std::make_shared("LocalStateFileDisk", path, 0); }; if (config.has("keeper_server.state_storage_disk")) diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index e04d1cd6b3d..6b7af3a60db 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -2,6 +2,8 @@ #include +#include + #include #include @@ -42,6 +44,8 @@ public: DiskPtr getStateFileDisk() const; void setStateFileDisk(DiskPtr disk); + + void dumpConfiguration(WriteBufferFromOwnString & buf) const; private: /// local disk defined using path or disk name using Storage = std::variant; diff --git a/src/Coordination/KeeperDiskSelector.h b/src/Coordination/KeeperDiskSelector.h deleted file mode 100644 index fa78fec1952..00000000000 --- a/src/Coordination/KeeperDiskSelector.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class KeeperDiskSelector -{ -public: - void initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); - - DiskSelectorPtr updateFromConfig( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - ContextPtr context) const; - - /// Get disk by name - DiskPtr get(const String & name) const; - - DiskPtr tryGet(const String & name) const; - - /// Get all disks with names - const DisksMap & getDisksMap() const; - - void shutdown(); - -private: - mutable std::mutex disk_selector_mutex; - DiskSelectorPtr disk_selector; -}; - -using KeeperDiskSelectorPtr = std::shared_ptr; - - -} diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 17a15067301..4a460777621 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -238,13 +238,13 @@ void KeeperDispatcher::snapshotThread() try { - auto snapshot_path = task.create_snapshot(std::move(task.snapshot)); + auto snapshot_file_info = task.create_snapshot(std::move(task.snapshot)); - if (snapshot_path.empty()) + if (snapshot_file_info.path.empty()) continue; if (isLeader()) - snapshot_s3.uploadSnapshot(snapshot_path); + snapshot_s3.uploadSnapshot(snapshot_file_info); } catch (...) { @@ -336,12 +336,15 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); + keeper_context = std::make_shared(standalone_keeper); + keeper_context->initialize(config); + server = std::make_unique( configuration_and_settings, config, responses_queue, snapshots_queue, - standalone_keeper, + keeper_context, snapshot_s3, [this](const KeeperStorage::RequestForSession & request_for_session) { diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 4b8b134cf8f..1b44f0f6ced 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -81,6 +81,8 @@ private: KeeperSnapshotManagerS3 snapshot_s3; + KeeperContextPtr keeper_context; + /// Thread put requests to raft void requestThread(); /// Thread put responses for subscribed sessions @@ -198,6 +200,11 @@ public: return configuration_and_settings; } + const KeeperContextPtr & getKeeperContext() const + { + return keeper_context; + } + void incrementPacketsSent() { keeper_stats.incrementPacketsSent(); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 31c91e2de80..989455a5a79 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -108,14 +108,14 @@ KeeperServer::KeeperServer( const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - bool standalone_keeper, + KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) , is_recovering(config.getBool("keeper_server.force_recovery", false)) - , keeper_context{std::make_shared(standalone_keeper)} + , keeper_context{std::move(keeper_context_)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { if (coordination_settings->quorum_reads) diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 63f9cc2bcea..8f416b1f48c 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -72,7 +72,7 @@ public: const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - bool standalone_keeper, + KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 7f097c182a1..3bfe700bcd5 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -563,7 +563,7 @@ KeeperSnapshotManager::KeeperSnapshotManager( } -std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) +SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) { ReadBufferFromNuraftBuffer reader(buffer); @@ -585,7 +585,7 @@ std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); removeOutdatedSnapshotsIfNeeded(); - return snapshot_file_name; + return {snapshot_file_name, disk}; } nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk() @@ -694,7 +694,7 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) existing_snapshots.erase(itr); } -std::pair KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot) +SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot) { auto up_to_log_idx = snapshot.snapshot_meta->get_last_log_idx(); auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); @@ -716,22 +716,12 @@ std::pair KeeperSnapshotManager::serializeSnapshot compressed_writer->finalize(); compressed_writer->sync(); - std::error_code ec; - - try - { - disk->removeFile(tmp_snapshot_file_name); - } - catch (fs::filesystem_error & e) - { - ec = e.code(); - return {snapshot_file_name, ec}; - } + disk->removeFile(tmp_snapshot_file_name); existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); removeOutdatedSnapshotsIfNeeded(); - return {snapshot_file_name, ec}; + return {snapshot_file_name, disk}; } } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 7b1129018d8..036c0cab62b 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -87,8 +87,14 @@ public: uint64_t nodes_digest; }; +struct SnapshotFileInfo +{ + std::string path; + DiskPtr disk; +}; + using KeeperStorageSnapshotPtr = std::shared_ptr; -using CreateSnapshotCallback = std::function; +using CreateSnapshotCallback = std::function; using SnapshotMetaAndStorage = std::pair; @@ -112,10 +118,10 @@ public: nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const; /// Serialize already compressed snapshot to disk (return path) - std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); + SnapshotFileInfo serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); /// Serialize snapshot directly to disk - std::pair serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot); + SnapshotFileInfo serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot); SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr buffer) const; @@ -139,7 +145,7 @@ public: return 0; } - std::string getLatestSnapshotPath() const + SnapshotFileInfo getLatestSnapshotInfo() const { if (!existing_snapshots.empty()) { @@ -147,14 +153,15 @@ public: try { - if (getDisk()->exists(path)) - return path; + auto disk = getDisk(); + if (disk->exists(path)) + return {path, disk}; } catch (...) { } } - return ""; + return {"", nullptr}; } private: diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 1afe0b352c5..580e166e302 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -132,8 +132,9 @@ std::shared_ptr KeeperSnapshotManagerS return snapshot_s3_client; } -void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_path) +void KeeperSnapshotManagerS3::uploadSnapshotImpl(const SnapshotFileInfo & snapshot_file_info) { + const auto & [snapshot_path, snapshot_disk] = snapshot_file_info; try { auto s3_client = getSnapshotS3Client(); @@ -154,8 +155,9 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa }; }; - LOG_INFO(log, "Will try to upload snapshot on {} to S3", snapshot_path); - ReadBufferFromFile snapshot_file(snapshot_path); + LOG_INFO(log, "Will try to upload snapshot on {} to S3", snapshot_file_info.path); + + auto snapshot_file = snapshot_disk->readFile(snapshot_file_info.path); auto snapshot_name = fs::path(snapshot_path).filename().string(); auto lock_file = fmt::format(".{}_LOCK", snapshot_name); @@ -222,7 +224,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa }); WriteBufferFromS3 snapshot_writer = create_writer(snapshot_name); - copyData(snapshot_file, snapshot_writer); + copyData(*snapshot_file, snapshot_writer); snapshot_writer.finalize(); LOG_INFO(log, "Successfully uploaded {} to S3", snapshot_path); @@ -240,31 +242,31 @@ void KeeperSnapshotManagerS3::snapshotS3Thread() while (!shutdown_called) { - std::string snapshot_path; - if (!snapshots_s3_queue.pop(snapshot_path)) + SnapshotFileInfo snapshot_file_info; + if (!snapshots_s3_queue.pop(snapshot_file_info)) break; if (shutdown_called) break; - uploadSnapshotImpl(snapshot_path); + uploadSnapshotImpl(snapshot_file_info); } } -void KeeperSnapshotManagerS3::uploadSnapshot(const std::string & path, bool async_upload) +void KeeperSnapshotManagerS3::uploadSnapshot(const SnapshotFileInfo & file_info, bool async_upload) { if (getSnapshotS3Client() == nullptr) return; if (async_upload) { - if (!snapshots_s3_queue.push(path)) - LOG_WARNING(log, "Failed to add snapshot {} to S3 queue", path); + if (!snapshots_s3_queue.push(file_info)) + LOG_WARNING(log, "Failed to add snapshot {} to S3 queue", file_info.path); return; } - uploadSnapshotImpl(path); + uploadSnapshotImpl(file_info); } void KeeperSnapshotManagerS3::startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros) diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index eff7868bba9..908deb76851 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -10,6 +10,8 @@ #include #include +#include + #include #endif @@ -24,13 +26,13 @@ public: /// 'macros' are used to substitute macros in endpoint of disks void updateS3Configuration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); - void uploadSnapshot(const std::string & path, bool async_upload = true); + void uploadSnapshot(const SnapshotFileInfo & file_info, bool async_upload = true); /// 'macros' are used to substitute macros in endpoint of disks void startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void shutdown(); private: - using SnapshotS3Queue = ConcurrentBoundedQueue; + using SnapshotS3Queue = ConcurrentBoundedQueue; SnapshotS3Queue snapshots_s3_queue; /// Upload new snapshots to S3 @@ -48,7 +50,7 @@ private: std::shared_ptr getSnapshotS3Client() const; - void uploadSnapshotImpl(const std::string & snapshot_path); + void uploadSnapshotImpl(const SnapshotFileInfo & snapshot_file_info); /// Thread upload snapshots to S3 in the background void snapshotS3Thread(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 69c15db51da..49243541bc8 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -14,6 +14,8 @@ #include #include "Coordination/KeeperStorage.h" +#include + namespace ProfileEvents { @@ -66,6 +68,16 @@ KeeperStateMachine::KeeperStateMachine( { } +namespace +{ + +bool isLocalDisk(const IDisk & disk) +{ + return dynamic_cast(&disk) != nullptr; +} + +} + void KeeperStateMachine::init() { /// Do everything without mutexes, no other threads exist. @@ -80,9 +92,13 @@ void KeeperStateMachine::init() try { - auto snapshot_deserialization_result - = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(latest_log_index)); - latest_snapshot_path = snapshot_manager.getLatestSnapshotPath(); + latest_snapshot_buf = snapshot_manager.deserializeSnapshotBufferFromDisk(latest_log_index); + auto snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_buf); + latest_snapshot_info = snapshot_manager.getLatestSnapshotInfo(); + + if (isLocalDisk(*latest_snapshot_info.disk)) + latest_snapshot_buf = nullptr; + storage = std::move(snapshot_deserialization_result.storage); latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; cluster_config = snapshot_deserialization_result.cluster_config; @@ -306,8 +322,14 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) { /// deserialize and apply snapshot to storage std::lock_guard lock(storage_and_responses_lock); - auto snapshot_deserialization_result - = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx())); + + SnapshotDeserializationResult snapshot_deserialization_result; + if (latest_snapshot_ptr) + snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr); + else + snapshot_deserialization_result + = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx())); + storage = std::move(snapshot_deserialization_result.storage); latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; cluster_config = snapshot_deserialization_result.cluster_config; @@ -387,19 +409,22 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res } else { - auto [path, error_code] = snapshot_manager.serializeSnapshotToDisk(*snapshot); - if (error_code) + /// we rely on the fact that the snapshot disk cannot be changed during runtime + if (isLocalDisk(*keeper_context->getSnapshotDisk())) { - throw Exception( - ErrorCodes::SYSTEM_ERROR, - "Snapshot {} was created failed, error: {}", - snapshot->snapshot_meta->get_last_log_idx(), - error_code.message()); + latest_snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot); + latest_snapshot_meta = snapshot->snapshot_meta; + latest_snapshot_buf = nullptr; } - latest_snapshot_path = path; - latest_snapshot_meta = snapshot->snapshot_meta; + else + { + auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(*snapshot); + latest_snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx()); + latest_snapshot_buf = std::move(snapshot_buf); + } + ProfileEvents::increment(ProfileEvents::KeeperSnapshotCreations); - LOG_DEBUG(log, "Created persistent snapshot {} with path {}", latest_snapshot_meta->get_last_log_idx(), path); + LOG_DEBUG(log, "Created persistent snapshot {} with path {}", latest_snapshot_meta->get_last_log_idx(), latest_snapshot_info.path); } } @@ -423,19 +448,19 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res when_done(ret, exception); - return ret ? latest_snapshot_path : ""; + return ret ? latest_snapshot_info : SnapshotFileInfo{}; }; if (keeper_context->getServerState() == KeeperContext::Phase::SHUTDOWN) { LOG_INFO(log, "Creating a snapshot during shutdown because 'create_snapshot_on_exit' is enabled."); - auto snapshot_path = snapshot_task.create_snapshot(std::move(snapshot_task.snapshot)); + auto snapshot_file_info = snapshot_task.create_snapshot(std::move(snapshot_task.snapshot)); - if (!snapshot_path.empty() && snapshot_manager_s3) + if (!snapshot_file_info.path.empty() && snapshot_manager_s3) { - LOG_INFO(log, "Uploading snapshot {} during shutdown because 'upload_snapshot_on_exit' is enabled.", snapshot_path); - snapshot_manager_s3->uploadSnapshot(snapshot_path, /* asnyc_upload */ false); + LOG_INFO(log, "Uploading snapshot {} during shutdown because 'upload_snapshot_on_exit' is enabled.", snapshot_file_info.path); + snapshot_manager_s3->uploadSnapshot(snapshot_file_info, /* asnyc_upload */ false); } return; @@ -456,14 +481,20 @@ void KeeperStateMachine::save_logical_snp_obj( nuraft::ptr snp_buf = s.serialize(); nuraft::ptr cloned_meta = nuraft::snapshot::deserialize(*snp_buf); + nuraft::ptr cloned_buffer; + + /// we rely on the fact that the snapshot disk cannot be changed during runtime + if (!isLocalDisk(*keeper_context->getSnapshotDisk())) + cloned_buffer = nuraft::buffer::clone(data); + try { std::lock_guard lock(snapshots_lock); /// Serialize snapshot to disk - auto result_path = snapshot_manager.serializeSnapshotBufferToDisk(data, s.get_last_log_idx()); - latest_snapshot_path = result_path; + latest_snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(data, s.get_last_log_idx()); latest_snapshot_meta = cloned_meta; - LOG_DEBUG(log, "Saved snapshot {} to path {}", s.get_last_log_idx(), result_path); + latest_snapshot_buf = std::move(cloned_buffer); + LOG_DEBUG(log, "Saved snapshot {} to path {}", s.get_last_log_idx(), latest_snapshot_info.path); obj_id++; ProfileEvents::increment(ProfileEvents::KeeperSaveSnapshot); } @@ -523,11 +554,23 @@ int KeeperStateMachine::read_logical_snp_obj( latest_snapshot_meta->get_last_log_idx()); return -1; } - if (bufferFromFile(log, latest_snapshot_path, data_out)) + + const auto & [path, disk] = latest_snapshot_info; + if (isLocalDisk(*disk)) { - LOG_WARNING(log, "Error reading snapshot {} from {}", s.get_last_log_idx(), latest_snapshot_path); - return -1; + auto full_path = fs::path(disk->getPath()) / path; + if (bufferFromFile(log, full_path, data_out)) + { + LOG_WARNING(log, "Error reading snapshot {} from {}", s.get_last_log_idx(), full_path); + return -1; + } } + else + { + chassert(latest_snapshot_buf); + data_out = nuraft::buffer::clone(*latest_snapshot_buf); + } + is_last_obj = true; ProfileEvents::increment(ProfileEvents::KeeperReadSnapshot); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index afe11150f36..834837314df 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -110,7 +110,7 @@ private: /// In our state machine we always have a single snapshot which is stored /// in memory in compressed (serialized) format. SnapshotMetadataPtr latest_snapshot_meta = nullptr; - std::string latest_snapshot_path; + SnapshotFileInfo latest_snapshot_info; nuraft::ptr latest_snapshot_buf = nullptr; CoordinationSettingsPtr coordination_settings; diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 2098daea5fe..aab8a2ccb68 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -252,10 +252,12 @@ def test_cmd_conf(started_cluster): assert result["four_letter_word_allow_list"] == "*" assert result["log_storage_path"] == "/var/lib/clickhouse/coordination/log" + assert result["log_storage_disk"] == "LocalLogDisk" assert ( result["snapshot_storage_path"] == "/var/lib/clickhouse/coordination/snapshots" ) + assert result["snapshot_storage_disk"] == "LocalSnapshotDisk" assert result["session_timeout_ms"] == "30000" assert result["min_session_timeout_ms"] == "10000" From 2651330b12cc73e1134cf84258e2cbca6d4c20e1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 08:50:44 +0000 Subject: [PATCH 0654/2223] README.md: Remove Berlin Meetup from upcoming events --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 4a619eb4fd3..bbedea364fc 100644 --- a/README.md +++ b/README.md @@ -23,7 +23,6 @@ curl https://clickhouse.com/ | sh ## Upcoming Events * [**v23.5 Release Webinar**](https://clickhouse.com/company/events/v23-5-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-05) - May 31 - 23.5 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. -* [**ClickHouse Meetup in Berlin**](https://www.meetup.com/clickhouse-berlin-user-group/events/292892466) - May 16 * [**ClickHouse Meetup in Barcelona**](https://www.meetup.com/clickhouse-barcelona-user-group/events/292892669) - May 25 * [**ClickHouse Meetup in London**](https://www.meetup.com/clickhouse-london-user-group/events/292892824) - May 25 * [**ClickHouse Meetup in San Francisco**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/293426725/) - Jun 7 From 4483602c50918e69d5a4d79c1628412b2c667c0d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 08:52:14 +0000 Subject: [PATCH 0655/2223] Remove double initialization --- src/Coordination/KeeperServer.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 989455a5a79..363e7f9bef1 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -121,8 +121,6 @@ KeeperServer::KeeperServer( if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); - keeper_context->initialize(config); - state_machine = nuraft::cs_new( responses_queue_, snapshots_queue_, From 4f5cf656eb7343cc68e053ee9bcb8a7eaa4e7ae9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 09:36:03 +0000 Subject: [PATCH 0656/2223] Minor adjustment of clickhouse-client/local parameter docs Related to #50210 --- docs/en/interfaces/cli.md | 8 ++++---- docs/en/operations/utilities/clickhouse-local.md | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 6d5395d46e3..f670d464006 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -177,11 +177,11 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--user, -u` – The username. Default value: default. - `--password` – The password. Default value: empty string. - `--ask-password` - Prompt the user to enter a password. -- `--query, -q` – The query to process when using non-interactive mode. You must specify either `query` or `queries-file` option. -- `--queries-file` – file path with queries to execute. You must specify either `query` or `queries-file` option. -- `--database, -d` – Select the current default database. Default value: the current database from the server settings (‘default’ by default). -- `--multiline, -m` – If specified, allow multiline queries (do not send the query on Enter). +- `--query, -q` – The query to process when using non-interactive mode. Cannot be used simultaneously with `--queries-file`. +- `--queries-file` – file path with queries to execute. Cannot be used simultaneously with `--query`. - `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`. +- `--multiline, -m` – If specified, allow multiline queries (do not send the query on Enter). +- `--database, -d` – Select the current default database. Default value: the current database from the server settings (‘default’ by default). - `--format, -f` – Use the specified default format to output the result. - `--vertical, -E` – If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `–format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables. - `--time, -t` – If specified, print the query execution time to ‘stderr’ in non-interactive mode. diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index d6587602990..0443a80cf17 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -183,12 +183,12 @@ Arguments: - `-S`, `--structure` — table structure for input data. - `--input-format` — input format, `TSV` by default. - `-f`, `--file` — path to data, `stdin` by default. -- `-q`, `--query` — queries to execute with `;` as delimiter. You must specify either `query` or `queries-file` option. -- `--queries-file` - file path with queries to execute. You must specify either `query` or `queries-file` option. +- `-q`, `--query` — queries to execute with `;` as delimiter. Cannot be used simultaneously with `--queries-file`. +- `--queries-file` - file path with queries to execute. Cannot be used simultaneously with `--query`. +- `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`. - `-N`, `--table` — table name where to put output data, `table` by default. - `--format`, `--output-format` — output format, `TSV` by default. - `-d`, `--database` — default database, `_local` by default. -- `--multiquery, -n` – If specified, multiple queries separated by semicolons can be listed after the `--query` option. For convenience, it is also possible to omit `--query` and pass the queries directly after `--multiquery`. - `--stacktrace` — whether to dump debug output in case of exception. - `--echo` — print query before execution. - `--verbose` — more details on query execution. From e4c8c4cecfb80c4afdae961c4ab8a64d42dc84d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BD=95=E6=9D=8E=E5=A4=AB?= Date: Thu, 25 May 2023 17:50:14 +0800 Subject: [PATCH 0657/2223] Add zookeeper name in endpoint id (#49780) * Add zookeeper name in endpoint id When we migrate a replicated table from one zookeeper cluster to another (the reason why we migration is that zookeeper's load is too high), we will create a new table with the same zpath, but it will fail and the old table will be in trouble. Here is some infomation: 1.old table: CREATE TABLE a1 (`id` UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/default/a1/{shard}', '{replica}') ORDER BY (id); 2.new table: CREATE TABLE a2 (`id` UInt64) ENGINE = ReplicatedMergeTree('aux1:/clickhouse/tables/default/a1/{shard}', '{replica}') ORDER BY (id); 3.error info: executeQuery: Code: 220. DB::Exception: Duplicate interserver IO endpoint: DataPartsExchange:/clickhouse/tables/default/a1/01/replicas/02. (DUPLICATE_INTERSERVER_IO_ENDPOINT) InterserverIOHTTPHandler: Code: 221. DB::Exception: No interserver IO endpoint named DataPartsExchange:/clickhouse/tables/default/a1/01/replicas/02. (NO_SUCH_INTERSERVER_IO_ENDPOINT) * Revert "Add zookeeper name in endpoint id" This reverts commit 9deb75b249619b7abdd38e3949ca8b3a76c9df8e. * Add zookeeper name in endpoint id When we migrate a replicated table from one zookeeper cluster to another (the reason why we migration is that zookeeper's load is too high), we will create a new table with the same zpath, but it will fail and the old table will be in trouble. * Fix incompatible with a new setting * add a test, fix other issues * Update 02442_auxiliary_zookeeper_endpoint_id.sql * Update 02735_system_zookeeper_connection.reference * Update 02735_system_zookeeper_connection.sql * Update run.sh * Remove the 'no-fasttest' tag * Update 02442_auxiliary_zookeeper_endpoint_id.sql --------- Co-authored-by: Alexander Tokmakov Co-authored-by: Alexander Tokmakov --- docker/test/stateless/run.sh | 3 + docker/test/upgrade/run.sh | 6 ++ src/Common/ZooKeeper/IKeeper.h | 3 +- src/Common/ZooKeeper/TestKeeper.h | 4 +- src/Common/ZooKeeper/ZooKeeper.cpp | 9 ++- src/Common/ZooKeeper/ZooKeeper.h | 4 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 +- src/Common/ZooKeeper/ZooKeeperImpl.h | 4 +- src/Interpreters/Context.cpp | 6 +- src/Storages/MergeTree/DataPartsExchange.cpp | 18 ++++- src/Storages/MergeTree/DataPartsExchange.h | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 66 +++++++++++++------ src/Storages/StorageReplicatedMergeTree.h | 2 + .../StorageSystemZooKeeperConnection.cpp | 5 +- tests/config/config.d/database_replicated.xml | 18 +++++ tests/config/config.d/merge_tree.xml | 1 + tests/config/config.d/zookeeper.xml | 9 +++ ..._auxiliary_zookeeper_endpoint_id.reference | 1 + .../02442_auxiliary_zookeeper_endpoint_id.sql | 21 ++++++ ...2735_system_zookeeper_connection.reference | 3 +- .../02735_system_zookeeper_connection.sql | 6 +- 22 files changed, 147 insertions(+), 48 deletions(-) create mode 100644 tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.reference create mode 100644 tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.sql diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index e509809c028..5d0a7b50741 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -132,6 +132,9 @@ function run_tests() ADDITIONAL_OPTIONS+=('--report-logs-stats') + clickhouse-test "00001_select_1" > /dev/null ||: + clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" ||: + set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 10ba597a33a..bd0c59a12cd 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -65,6 +65,9 @@ sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ > /etc/clickhouse-server/config.d/storage_conf.xml.tmp sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml +# it contains some new settings, but we can safely remove it +rm /etc/clickhouse-server/config.d/merge_tree.xml + start stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log @@ -94,6 +97,9 @@ sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ > /etc/clickhouse-server/config.d/storage_conf.xml.tmp sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml +# it contains some new settings, but we can safely remove it +rm /etc/clickhouse-server/config.d/merge_tree.xml + start clickhouse-client --query="SELECT 'Server version: ', version()" diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 86f9a388644..3eb5819df90 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -466,7 +467,7 @@ public: /// Useful to check owner of ephemeral node. virtual int64_t getSessionID() const = 0; - virtual String getConnectedAddress() const = 0; + virtual Poco::Net::SocketAddress getConnectedAddress() const = 0; /// If the method will throw an exception, callbacks won't be called. /// diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 11e56daf6b4..4bffa4e1d4f 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -39,7 +39,7 @@ public: bool isExpired() const override { return expired; } int64_t getSessionID() const override { return 0; } - String getConnectedAddress() const override { return connected_zk_address; } + Poco::Net::SocketAddress getConnectedAddress() const override { return connected_zk_address; } void create( @@ -127,7 +127,7 @@ private: zkutil::ZooKeeperArgs args; - String connected_zk_address; + Poco::Net::SocketAddress connected_zk_address; std::mutex push_request_mutex; std::atomic expired{false}; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index c423e4fd498..a587ad6caf4 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -112,11 +112,10 @@ void ZooKeeper::init(ZooKeeperArgs args_) else LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(args.hosts, ","), args.chroot); - String address = impl->getConnectedAddress(); + Poco::Net::SocketAddress address = impl->getConnectedAddress(); - size_t colon_pos = address.find(':'); - connected_zk_host = address.substr(0, colon_pos); - connected_zk_port = address.substr(colon_pos + 1); + connected_zk_host = address.host().toString(); + connected_zk_port = address.port(); connected_zk_index = 0; @@ -124,7 +123,7 @@ void ZooKeeper::init(ZooKeeperArgs args_) { for (size_t i = 0; i < args.hosts.size(); i++) { - if (args.hosts[i] == address) + if (args.hosts[i] == address.toString()) { connected_zk_index = i; break; diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 9b85938c726..96f9914b597 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -524,7 +524,7 @@ public: void setServerCompletelyStarted(); String getConnectedZooKeeperHost() const { return connected_zk_host; } - String getConnectedZooKeeperPort() const { return connected_zk_port; } + UInt16 getConnectedZooKeeperPort() const { return connected_zk_port; } size_t getConnectedZooKeeperIndex() const { return connected_zk_index; } private: @@ -591,7 +591,7 @@ private: ZooKeeperArgs args; String connected_zk_host; - String connected_zk_port; + UInt16 connected_zk_port; size_t connected_zk_index; std::mutex mutex; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 34be8aa1332..7f23ac00efe 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -433,7 +433,7 @@ void ZooKeeper::connect( } connected = true; - connected_zk_address = node.address.toString(); + connected_zk_address = node.address; break; } @@ -450,7 +450,7 @@ void ZooKeeper::connect( if (!connected) { WriteBufferFromOwnString message; - connected_zk_address = ""; + connected_zk_address = Poco::Net::SocketAddress(); message << "All connection tries failed while connecting to ZooKeeper. nodes: "; bool first = true; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 6715607ca88..944c5032fac 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -125,7 +125,7 @@ public: /// Useful to check owner of ephemeral node. int64_t getSessionID() const override { return session_id; } - String getConnectedAddress() const override { return connected_zk_address; } + Poco::Net::SocketAddress getConnectedAddress() const override { return connected_zk_address; } void executeGenericRequest( const ZooKeeperRequestPtr & request, @@ -203,7 +203,7 @@ public: private: ACLs default_acls; - String connected_zk_address; + Poco::Net::SocketAddress connected_zk_address; zkutil::ZooKeeperArgs args; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5ec12d7e0ea..b0895939874 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2796,11 +2796,7 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const std::map Context::getAuxiliaryZooKeepers() const { std::lock_guard lock(shared->auxiliary_zookeepers_mutex); - - if (!shared->auxiliary_zookeepers.empty()) - return shared->auxiliary_zookeepers; - else - return std::map(); + return shared->auxiliary_zookeepers; } #if USE_ROCKSDB diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index f2e35e2dcd2..21fdda28767 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -369,6 +369,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const String & part_name, + const String & zookeeper_name, const String & replica_path, const String & host, int port, @@ -401,13 +402,18 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( /// Validation of the input that may come from malicious replica. auto part_info = MergeTreePartInfo::fromPartName(part_name, data.format_version); + String endpoint_id = getEndpointId( + data_settings->enable_the_endpoint_id_with_zookeeper_name_prefix ? + zookeeper_name + ":" + replica_path : + replica_path); + Poco::URI uri; uri.setScheme(interserver_scheme); uri.setHost(host); uri.setPort(port); uri.setQueryParameters( { - {"endpoint", getEndpointId(replica_path)}, + {"endpoint", endpoint_id}, {"part", part_name}, {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION)}, {"compress", "false"} @@ -630,7 +636,15 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( temporary_directory_lock = {}; /// Try again but without zero-copy - return fetchSelectedPart(metadata_snapshot, context, part_name, replica_path, host, port, timeouts, + return fetchSelectedPart( + metadata_snapshot, + context, + part_name, + zookeeper_name, + replica_path, + host, + port, + timeouts, user, password, interserver_scheme, throttler, to_detached, tmp_prefix, nullptr, false, disk); } } diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 57fd0d5cff0..704c1b0a226 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -70,6 +70,7 @@ public: const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const String & part_name, + const String & zookeeper_name, const String & replica_path, const String & host, int port, diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c9e81ce9103..5ea99009756 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -159,6 +159,7 @@ struct Settings; M(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ M(Bool, check_sample_column_is_correct, true, "Check columns or columns by hash for sampling are unsigned integer.", 0) \ M(Bool, allow_vertical_merges_from_compact_to_wide_parts, false, "Allows vertical merges from compact to wide parts. This settings must have the same value on all replicas", 0) \ + M(Bool, enable_the_endpoint_id_with_zookeeper_name_prefix, false, "Enable the endpoint id with zookeeper name prefix for the replicated merge tree table", 0) \ \ /** Experimental/work in progress feature. Unsafe for production. */ \ M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ea9ffee4939..7b9a3093e40 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -240,6 +240,15 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperAndAssertNotReadonl return res; } +String StorageReplicatedMergeTree::getEndpointName() const +{ + const MergeTreeSettings & settings = getContext()->getReplicatedMergeTreeSettings(); + if (settings.enable_the_endpoint_id_with_zookeeper_name_prefix) + return zookeeper_name + ":" + replica_path; + + return replica_path; +} + static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) { return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), {context->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0}); @@ -1841,6 +1850,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che String source_replica_path = fs::path(zookeeper_path) / "replicas" / replica; if (!fetchPart(part_name, metadata_snapshot, + zookeeper_name, source_replica_path, /* to_detached= */ false, entry.quorum, @@ -2341,7 +2351,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) interserver_scheme, address.scheme, address.host); part_desc->res_part = fetcher.fetchSelectedPart( - metadata_snapshot, getContext(), part_desc->found_new_part_name, source_replica_path, + metadata_snapshot, getContext(), part_desc->found_new_part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, false, TMP_PREFIX + "fetch_"); @@ -2458,7 +2468,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr interserver_scheme, address.scheme, address.host); return fetcher.fetchSelectedPart( - metadata_snapshot, getContext(), entry.new_part_name, source_replica_path, + metadata_snapshot, getContext(), entry.new_part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, true); @@ -4042,6 +4052,7 @@ bool StorageReplicatedMergeTree::partIsLastQuorumPart(const MergeTreePartInfo & bool StorageReplicatedMergeTree::fetchPart( const String & part_name, const StorageMetadataPtr & metadata_snapshot, + const String & source_zookeeper_name, const String & source_replica_path, bool to_detached, size_t quorum, @@ -4077,7 +4088,7 @@ bool StorageReplicatedMergeTree::fetchPart( currently_fetching_parts.erase(part_name); }); - LOG_DEBUG(log, "Fetching part {} from {}", part_name, source_replica_path); + LOG_DEBUG(log, "Fetching part {} from {}:{}", part_name, source_zookeeper_name, source_replica_path); auto settings_ptr = getSettings(); TableLockHolder table_lock_holder; @@ -4134,7 +4145,8 @@ bool StorageReplicatedMergeTree::fetchPart( } else { - LOG_INFO(log, "Not checking checksums of part {} with replica {} because part was removed from ZooKeeper", part_name, source_replica_path); + LOG_INFO(log, "Not checking checksums of part {} with replica {}:{} because part was removed from ZooKeeper", + part_name, source_zookeeper_name, source_replica_path); } } @@ -4187,6 +4199,7 @@ bool StorageReplicatedMergeTree::fetchPart( metadata_snapshot, getContext(), part_name, + source_zookeeper_name, source_replica_path, address.host, address.replication_port, @@ -4279,7 +4292,7 @@ bool StorageReplicatedMergeTree::fetchPart( if (part_to_clone) LOG_DEBUG(log, "Cloned part {} from {}{}", part_name, part_to_clone->name, to_detached ? " (to 'detached' directory)" : ""); else - LOG_DEBUG(log, "Fetched part {} from {}{}", part_name, source_replica_path, to_detached ? " (to 'detached' directory)" : ""); + LOG_DEBUG(log, "Fetched part {} from {}:{}{}", part_name, source_zookeeper_name, source_replica_path, to_detached ? " (to 'detached' directory)" : ""); return true; } @@ -4318,7 +4331,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( currently_fetching_parts.erase(part_name); }); - LOG_DEBUG(log, "Fetching already known part {} from {}", part_name, source_replica_path); + LOG_DEBUG(log, "Fetching already known part {} from {}:{}", part_name, zookeeper_name, source_replica_path); TableLockHolder table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -4350,7 +4363,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( "'{}' != '{}', can't fetch part from {}", interserver_scheme, address.scheme, address.host); return fetcher.fetchSelectedPart( - metadata_snapshot, getContext(), part_name, source_replica_path, + metadata_snapshot, getContext(), part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, false, "", nullptr, true, @@ -4387,7 +4400,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches); - LOG_DEBUG(log, "Fetched part {} from {}", part_name, source_replica_path); + LOG_DEBUG(log, "Fetched part {} from {}:{}", part_name, zookeeper_name, source_replica_path); return part->getDataPartStoragePtr(); } @@ -4430,7 +4443,16 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) InterserverIOEndpointPtr data_parts_exchange_ptr = std::make_shared(*this); [[maybe_unused]] auto prev_ptr = std::atomic_exchange(&data_parts_exchange_endpoint, data_parts_exchange_ptr); assert(prev_ptr == nullptr); - getContext()->getInterserverIOHandler().addEndpoint(data_parts_exchange_ptr->getId(replica_path), data_parts_exchange_ptr); + + /// The endpoint id: + /// old format: DataPartsExchange:/clickhouse/tables/default/t1/{shard}/{replica} + /// new format: DataPartsExchange:{zookeeper_name}:/clickhouse/tables/default/t1/{shard}/{replica} + /// Notice: + /// They are incompatible and the default is the old format. + /// If you want to use the new format, please ensure that 'enable_the_endpoint_id_with_zookeeper_name_prefix' of all nodes is true . + /// + getContext()->getInterserverIOHandler().addEndpoint( + data_parts_exchange_ptr->getId(getEndpointName()), data_parts_exchange_ptr); startBeingLeader(); @@ -4555,7 +4577,7 @@ void StorageReplicatedMergeTree::shutdown() auto data_parts_exchange_ptr = std::atomic_exchange(&data_parts_exchange_endpoint, InterserverIOEndpointPtr{}); if (data_parts_exchange_ptr) { - getContext()->getInterserverIOHandler().removeEndpointIfExists(data_parts_exchange_ptr->getId(replica_path)); + getContext()->getInterserverIOHandler().removeEndpointIfExists(data_parts_exchange_ptr->getId(getEndpointName())); /// Ask all parts exchange handlers to finish asap. New ones will fail to start data_parts_exchange_ptr->blocker.cancelForever(); /// Wait for all of them @@ -6237,14 +6259,14 @@ void StorageReplicatedMergeTree::fetchPartition( info.table_id = getStorageID(); info.table_id.uuid = UUIDHelpers::Nil; auto expand_from = query_context->getMacros()->expand(from_, info); - String auxiliary_zookeeper_name = zkutil::extractZooKeeperName(expand_from); + String from_zookeeper_name = zkutil::extractZooKeeperName(expand_from); String from = zkutil::extractZooKeeperPath(expand_from, /* check_starts_with_slash */ true); if (from.empty()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "ZooKeeper path should not be empty"); zkutil::ZooKeeperPtr zookeeper; - if (auxiliary_zookeeper_name != default_zookeeper_name) - zookeeper = getContext()->getAuxiliaryZooKeeper(auxiliary_zookeeper_name); + if (from_zookeeper_name != default_zookeeper_name) + zookeeper = getContext()->getAuxiliaryZooKeeper(from_zookeeper_name); else zookeeper = getZooKeeper(); @@ -6263,12 +6285,12 @@ void StorageReplicatedMergeTree::fetchPartition( */ if (checkIfDetachedPartExists(part_name)) throw Exception(ErrorCodes::DUPLICATE_DATA_PART, "Detached part {} already exists.", part_name); - LOG_INFO(log, "Will fetch part {} from shard {} (zookeeper '{}')", part_name, from_, auxiliary_zookeeper_name); + LOG_INFO(log, "Will fetch part {} from shard {}", part_name, from_); try { /// part name , metadata, part_path , true, 0, zookeeper - if (!fetchPart(part_name, metadata_snapshot, part_path, true, 0, zookeeper, /* try_fetch_shared = */ false)) + if (!fetchPart(part_name, metadata_snapshot, from_zookeeper_name, part_path, true, 0, zookeeper, /* try_fetch_shared = */ false)) throw Exception(ErrorCodes::UNFINISHED, "Failed to fetch part {} from {}", part_name, from_); } catch (const DB::Exception & e) @@ -6283,7 +6305,7 @@ void StorageReplicatedMergeTree::fetchPartition( } String partition_id = getPartitionIDFromQuery(partition, query_context); - LOG_INFO(log, "Will fetch partition {} from shard {} (zookeeper '{}')", partition_id, from_, auxiliary_zookeeper_name); + LOG_INFO(log, "Will fetch partition {} from shard {}", partition_id, from_); /** Let's check that there is no such partition in the `detached` directory (where we will write the downloaded parts). * Unreliable (there is a race condition) - such a partition may appear a little later. @@ -6307,7 +6329,7 @@ void StorageReplicatedMergeTree::fetchPartition( active_replicas.push_back(replica); if (active_replicas.empty()) - throw Exception(ErrorCodes::NO_ACTIVE_REPLICAS, "No active replicas for shard {}", from); + throw Exception(ErrorCodes::NO_ACTIVE_REPLICAS, "No active replicas for shard {}", from_); /** You must select the best (most relevant) replica. * This is a replica with the maximum `log_pointer`, then with the minimum `queue` size. @@ -6361,7 +6383,8 @@ void StorageReplicatedMergeTree::fetchPartition( LOG_INFO(log, "Some of parts ({}) are missing. Will try to fetch covering parts.", missing_parts.size()); if (try_no >= query_context->getSettings().max_fetch_partition_retries_count) - throw Exception(ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS, "Too many retries to fetch parts from {}", best_replica_path); + throw Exception(ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS, + "Too many retries to fetch parts from {}:{}", from_zookeeper_name, best_replica_path); Strings parts = zookeeper->getChildren(fs::path(best_replica_path) / "parts"); ActiveDataPartSet active_parts_set(format_version, parts); @@ -6382,7 +6405,8 @@ void StorageReplicatedMergeTree::fetchPartition( parts_to_fetch = std::move(parts_to_fetch_partition); if (parts_to_fetch.empty()) - throw Exception(ErrorCodes::PARTITION_DOESNT_EXIST, "Partition {} on {} doesn't exist", partition_id, best_replica_path); + throw Exception(ErrorCodes::PARTITION_DOESNT_EXIST, + "Partition {} on {}:{} doesn't exist", partition_id, from_zookeeper_name, best_replica_path); } else { @@ -6392,7 +6416,7 @@ void StorageReplicatedMergeTree::fetchPartition( if (!containing_part.empty()) parts_to_fetch.push_back(containing_part); else - LOG_WARNING(log, "Part {} on replica {} has been vanished.", missing_part, best_replica_path); + LOG_WARNING(log, "Part {} on replica {}:{} has been vanished.", missing_part, from_zookeeper_name, best_replica_path); } } @@ -6405,7 +6429,7 @@ void StorageReplicatedMergeTree::fetchPartition( try { - fetched = fetchPart(part, metadata_snapshot, best_replica_path, true, 0, zookeeper, /* try_fetch_shared = */ false); + fetched = fetchPart(part, metadata_snapshot, from_zookeeper_name, best_replica_path, true, 0, zookeeper, /* try_fetch_shared = */ false); } catch (const DB::Exception & e) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 29b6a4d6817..cb93dd0b5e3 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -382,6 +382,7 @@ private: zkutil::ZooKeeperPtr getZooKeeperIfTableShutDown() const; zkutil::ZooKeeperPtr getZooKeeperAndAssertNotReadonly() const; void setZooKeeper(); + String getEndpointName() const; /// If true, the table is offline and can not be written to it. /// This flag is managed by RestartingThread. @@ -699,6 +700,7 @@ private: bool fetchPart( const String & part_name, const StorageMetadataPtr & metadata_snapshot, + const String & source_zookeeper_name, const String & source_replica_path, bool to_detached, size_t quorum, diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index f249097654e..559e12ad5ee 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -13,7 +13,7 @@ NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() return { {"name", std::make_shared()}, {"host", std::make_shared()}, - {"port", std::make_shared()}, + {"port", std::make_shared()}, {"index", std::make_shared()}, {"connected_time", std::make_shared()}, {"is_expired", std::make_shared()}, @@ -25,7 +25,7 @@ NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - res_columns[0]->insert("default_zookeeper"); + res_columns[0]->insert("default"); res_columns[1]->insert(context->getZooKeeper()->getConnectedZooKeeperHost()); res_columns[2]->insert(context->getZooKeeper()->getConnectedZooKeeperPort()); res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); @@ -38,7 +38,6 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co { res_columns[0]->insert(elem.first); res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); - res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); res_columns[2]->insert(elem.second->getConnectedZooKeeperPort()); res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 3fea87c4fd1..9a405f85908 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -14,6 +14,24 @@
+ + + + localhost + 9181 + + + localhost + 19181 + + + localhost + 29181 + + /test/chroot/auxiliary_zookeeper2 + + + 9181 1 diff --git a/tests/config/config.d/merge_tree.xml b/tests/config/config.d/merge_tree.xml index 43bdb6aa07b..bf2da9b09a2 100644 --- a/tests/config/config.d/merge_tree.xml +++ b/tests/config/config.d/merge_tree.xml @@ -1,5 +1,6 @@ + 1 8 diff --git a/tests/config/config.d/zookeeper.xml b/tests/config/config.d/zookeeper.xml index 63057224ef9..75b4a00fe67 100644 --- a/tests/config/config.d/zookeeper.xml +++ b/tests/config/config.d/zookeeper.xml @@ -7,4 +7,13 @@ 9181
+ + + + localhost + 9181 + + /test/chroot/auxiliary_zookeeper2 + + diff --git a/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.reference b/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.reference new file mode 100644 index 00000000000..4a9341ba3f6 --- /dev/null +++ b/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.reference @@ -0,0 +1 @@ +10013 diff --git a/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.sql b/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.sql new file mode 100644 index 00000000000..ff3552b2a42 --- /dev/null +++ b/tests/queries/0_stateless/02442_auxiliary_zookeeper_endpoint_id.sql @@ -0,0 +1,21 @@ +-- Tags: no-fasttest + +drop table if exists t1_r1 sync; +drop table if exists t1_r2 sync; +drop table if exists t2 sync; + +create table t1_r1 (x Int32) engine=ReplicatedMergeTree('/test/02442/{database}/t', 'r1') order by x; + +create table t1_r2 (x Int32) engine=ReplicatedMergeTree('/test/02442/{database}/t', 'r2') order by x; + +-- create table with same replica_path as t1_r1 +create table t2 (x Int32) engine=ReplicatedMergeTree('zookeeper2:/test/02442/{database}/t', 'r1') order by x; +drop table t2 sync; + +-- insert data into one replica +insert into t1_r1 select * from generateRandom('x Int32') LIMIT 10013; +system sync replica t1_r2; +select count() from t1_r2; + +drop table t1_r1 sync; +drop table t1_r2 sync; diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference index c9cc8adede8..1deabd88b88 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference @@ -1 +1,2 @@ -[ :1]:9181 0 +default ::1 9181 0 0 3 +zookeeper2 ::1 9181 0 0 0 diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql index b409913d183..8b37c428413 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql @@ -5,9 +5,11 @@ DROP TABLE IF EXISTS test_zk_connection_table; CREATE TABLE test_zk_connection_table ( key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/{database}/02731_zk_connection/{shard}', '{replica}') +ENGINE ReplicatedMergeTree('zookeeper2:/clickhouse/{database}/02731_zk_connection/{shard}', '{replica}') ORDER BY tuple(); -select host, port, is_expired from system.zookeeper_connection where name='default_zookeeper'; +-- keeper_api_version will by 0 for auxiliary_zookeeper2, because we fail to get /api_version due to chroot +-- I'm not sure if it's a bug or a useful trick to fallback to basic api +select name, host, port, index, is_expired, keeper_api_version from system.zookeeper_connection order by name; DROP TABLE IF EXISTS test_zk_connection_table; From b2c9611da66254fc10ad4bc8fe30d7929b3c30a3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 May 2023 12:01:24 +0200 Subject: [PATCH 0658/2223] Fix build --- src/Disks/DiskEncryptedTransaction.cpp | 5 +++++ src/Disks/DiskEncryptedTransaction.h | 6 ++++++ src/Disks/tests/gtest_disk_encrypted.cpp | 2 +- 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index deafe88e43d..d7448b986bb 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -1,4 +1,7 @@ #include + + +#if USE_SSL #include #include #include @@ -111,3 +114,5 @@ std::unique_ptr DiskEncryptedTransaction::writeFile( } } + +#endif diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index 197df56c2a2..bae3f2c728c 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -1,5 +1,9 @@ #pragma once +#include "config.h" + +#if USE_SSL + #include #include #include @@ -251,3 +255,5 @@ private: }; } + +#endif diff --git a/src/Disks/tests/gtest_disk_encrypted.cpp b/src/Disks/tests/gtest_disk_encrypted.cpp index 32db5f5c1d9..80a10e8680b 100644 --- a/src/Disks/tests/gtest_disk_encrypted.cpp +++ b/src/Disks/tests/gtest_disk_encrypted.cpp @@ -40,7 +40,7 @@ protected: settings->keys[0] = key; settings->current_key_id = 0; settings->disk_path = path; - encrypted_disk = std::make_shared("encrypted_disk", std::move(settings)); + encrypted_disk = std::make_shared("encrypted_disk", std::move(settings), true); } String getFileNames() From fd49821e98c9324e0a6db56dbbec55b52551225b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 25 May 2023 12:46:53 +0200 Subject: [PATCH 0659/2223] Fix tests --- tests/integration/test_storage_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index d9ac70f51ad..f983bd618e3 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1636,7 +1636,7 @@ def test_ast_auth_headers(started_cluster): filename = "test.csv" result = instance.query_and_get_error( - f"select count() from s3('http://resolver:8080/{bucket}/{filename}', 'CSV')" + f"select count() from s3('http://resolver:8080/{bucket}/{filename}', 'CSV', 'dummy String')" ) assert "HTTP response code: 403" in result From 58610f11ab90a017a1275e1a9a0b843d17e948d8 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 May 2023 10:54:02 +0000 Subject: [PATCH 0660/2223] Fix tests --- tests/queries/0_stateless/01256_negative_generate_random.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/01256_negative_generate_random.sql b/tests/queries/0_stateless/01256_negative_generate_random.sql index 14f1d947108..7e05a394b8d 100644 --- a/tests/queries/0_stateless/01256_negative_generate_random.sql +++ b/tests/queries/0_stateless/01256_negative_generate_random.sql @@ -1,5 +1,4 @@ SELECT * FROM generateRandom('i8', 1, 10, 10); -- { serverError 62 } SELECT * FROM generateRandom; -- { serverError 60 } -SELECT * FROM generateRandom(); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } SELECT * FROM generateRandom('i8 UInt8', 1, 10, 10, 10, 10); -- { serverError 42 } SELECT * FROM generateRandom('', 1, 10, 10); -- { serverError 62 } From 5b76ab4e038c1b9a9fb395d787205b298ecac9d1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 May 2023 13:02:40 +0200 Subject: [PATCH 0661/2223] Fix build --- src/Disks/DiskEncryptedTransaction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index d7448b986bb..7534abe59a8 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -77,7 +77,7 @@ void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, cons delegate_transaction->copyFile(wrapped_from_path, wrapped_to_path); } -std::unique_ptr DiskEncryptedTransaction::writeFile( +std::unique_ptr DiskEncryptedTransaction::writeFile( // NOLINT const std::string & path, size_t buf_size, WriteMode mode, From 2df4130d82eeac51067c8b82dcc1bc1a81d84e91 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 25 May 2023 13:06:40 +0200 Subject: [PATCH 0662/2223] Update src/Interpreters/ClusterDiscovery.cpp --- src/Interpreters/ClusterDiscovery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index ce59dfcae45..884e3b87343 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -130,7 +130,7 @@ ClusterDiscovery::ClusterDiscovery( /* secure= */ config.getBool(prefix + ".secure", false), /* shard_id= */ config.getUInt(prefix + ".shard", 0), /* observer_mode= */ ConfigHelper::getBool(config, prefix + ".observer"), - /* invisible_mode= */ ConfigHelper::getBool(config, prefix + ".invisible") + /* invisible= */ ConfigHelper::getBool(config, prefix + ".invisible") ) ); } From ce99825200c5b59a35e2ad2dc22cb91725fd5351 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 May 2023 11:19:15 +0000 Subject: [PATCH 0663/2223] Fix skipping spaces at end of row in CustomSeparatedIgnoreSpaces format --- src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h | 2 +- .../02752_custom_separated_ignore_spaces_bug.reference | 1 + .../0_stateless/02752_custom_separated_ignore_spaces_bug.sql | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02752_custom_separated_ignore_spaces_bug.reference create mode 100644 tests/queries/0_stateless/02752_custom_separated_ignore_spaces_bug.sql diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h index 26ee32be370..2acf35bd143 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h @@ -76,7 +76,7 @@ public: bool checkEndOfRow(); bool checkForSuffixImpl(bool check_eof); - inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf); } + inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf, true); } EscapingRule getEscapingRule() const override { return format_settings.custom.escaping_rule; } diff --git a/tests/queries/0_stateless/02752_custom_separated_ignore_spaces_bug.reference b/tests/queries/0_stateless/02752_custom_separated_ignore_spaces_bug.reference new file mode 100644 index 00000000000..f52004e62a3 --- /dev/null +++ b/tests/queries/0_stateless/02752_custom_separated_ignore_spaces_bug.reference @@ -0,0 +1 @@ +unquoted_string diff --git a/tests/queries/0_stateless/02752_custom_separated_ignore_spaces_bug.sql b/tests/queries/0_stateless/02752_custom_separated_ignore_spaces_bug.sql new file mode 100644 index 00000000000..62047a704a7 --- /dev/null +++ b/tests/queries/0_stateless/02752_custom_separated_ignore_spaces_bug.sql @@ -0,0 +1 @@ +select * from format(CustomSeparatedIgnoreSpaces, 'x String', ' unquoted_string\n') settings format_custom_escaping_rule='CSV'; From f519aa4613a7f82b30bb39686412343d5a5a939d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 09:10:45 +0000 Subject: [PATCH 0664/2223] Calculate size --- .../KeeperAsynchronousMetrics.cpp | 12 +++---- src/Coordination/KeeperDispatcher.cpp | 32 +++++++++---------- src/Coordination/KeeperSnapshotManagerS3.h | 5 +-- src/Coordination/KeeperStateMachine.cpp | 5 --- 4 files changed, 25 insertions(+), 29 deletions(-) diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 1427130b184..890079e98f7 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -28,8 +28,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM size_t zxid = 0; size_t session_with_watches = 0; size_t paths_watched = 0; - size_t snapshot_dir_size = 0; - size_t log_dir_size = 0; + //size_t snapshot_dir_size = 0; + //size_t log_dir_size = 0; if (keeper_dispatcher.isServerActive()) { @@ -49,8 +49,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM latest_snapshot_size = state_machine.getLatestSnapshotBufSize(); session_with_watches = state_machine.getSessionsWithWatchesCount(); paths_watched = state_machine.getWatchedPathsCount(); - snapshot_dir_size = keeper_dispatcher.getSnapDirSize(); - log_dir_size = keeper_dispatcher.getLogDirSize(); + //snapshot_dir_size = keeper_dispatcher.getSnapDirSize(); + //log_dir_size = keeper_dispatcher.getLogDirSize(); # if defined(__linux__) || defined(__APPLE__) open_file_descriptor_count = getCurrentProcessFDCount(); @@ -85,8 +85,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM new_values["KeeperZxid"] = { zxid, "The current transaction id number (zxid) in ClickHouse Keeper." }; new_values["KeeperSessionWithWatches"] = { session_with_watches, "The number of client sessions of ClickHouse Keeper having watches." }; new_values["KeeperPathsWatched"] = { paths_watched, "The number of different paths watched by the clients of ClickHouse Keeper." }; - new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." }; - new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." }; + //new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." }; + //new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." }; auto keeper_log_info = keeper_dispatcher.getKeeperLogInfo(); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 4a460777621..6d1239c9210 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -777,37 +777,37 @@ void KeeperDispatcher::updateKeeperStatLatency(uint64_t process_time_ms) keeper_stats.updateLatency(process_time_ms); } -static uint64_t getDirSize(const fs::path & dir) +static uint64_t getTotalSize(const DiskPtr & disk, const std::string & path = "") { checkStackSize(); - if (!fs::exists(dir)) - return 0; - fs::directory_iterator it(dir); - fs::directory_iterator end; - - uint64_t size{0}; - while (it != end) + uint64_t size = 0; + for (auto it = disk->iterateDirectory(path); it->isValid(); it->next()) { - if (it->is_regular_file()) - size += fs::file_size(*it); + if (disk->isFile(it->path())) + size += disk->getFileSize(it->path()); else - size += getDirSize(it->path()); - ++it; + size += getTotalSize(disk, it->path()); } + return size; } uint64_t KeeperDispatcher::getLogDirSize() const { - //return getDirSize(configuration_and_settings->log_storage_path); - return 0; + auto log_disk = keeper_context->getLogDisk(); + auto size = getTotalSize(log_disk); + + auto current_log_disk = keeper_context->getCurrentLogDisk(); + if (log_disk != current_log_disk) + size += getTotalSize(current_log_disk); + + return size; } uint64_t KeeperDispatcher::getSnapDirSize() const { - //return getDirSize(configuration_and_settings->snapshot_storage_path); - return 0; + return getTotalSize(keeper_context->getSnapshotDisk()); } Keeper4LWInfo KeeperDispatcher::getKeeper4LWInfo() const diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index 908deb76851..e17cf5a1cfb 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -6,11 +6,12 @@ #include #include +#include + #if USE_AWS_S3 #include #include -#include #include #endif @@ -62,7 +63,7 @@ public: KeeperSnapshotManagerS3() = default; void updateS3Configuration(const Poco::Util::AbstractConfiguration &, const MultiVersion::Version &) {} - void uploadSnapshot(const std::string &, [[maybe_unused]] bool async_upload = true) {} + void uploadSnapshot(const SnapshotFileInfo &, [[maybe_unused]] bool async_upload = true) {} void startup(const Poco::Util::AbstractConfiguration &, const MultiVersion::Version &) {} diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 49243541bc8..5cfc9333a66 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -35,11 +35,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int SYSTEM_ERROR; -} - -namespace -{ } KeeperStateMachine::KeeperStateMachine( From 32cb9931b61df452ade3885fab8006582375640b Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 25 May 2023 13:47:56 +0200 Subject: [PATCH 0665/2223] Update src/Common/AsyncLoader.h Co-authored-by: Antonio Andelic --- src/Common/AsyncLoader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 7c6c934d03e..c3d11b26336 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -59,7 +59,7 @@ public: // Returns pool in which the job is executing (was executed). May differ from initial pool and from current pool. // Value is only valid (and constant) after execution started. - size_t execution_pool() const; + size_t executionPool() const; // Returns current pool of the job. May differ from initial and execution pool. // This value is intended for creating new jobs during this job execution. From 78c89da8bbc2cc75882a7883cd2e32315a4dfc32 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 25 May 2023 13:48:03 +0200 Subject: [PATCH 0666/2223] Update src/Common/AsyncLoader.cpp Co-authored-by: Antonio Andelic --- src/Common/AsyncLoader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 0f75f17465b..6e558f36b7d 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -42,7 +42,7 @@ std::exception_ptr LoadJob::exception() const return load_exception; } -size_t LoadJob::execution_pool() const +size_t LoadJob::executionPool() const { return execution_pool_id; } From 8804dfd4b052bf768d2747fec814c1956f7f40f4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 11:55:23 +0000 Subject: [PATCH 0667/2223] Fix resizing --- src/Functions/space.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 9d825a8b294..7d55d704038 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -27,9 +27,9 @@ private: static constexpr auto space = ' '; /// Safety threshold against DoS. - static inline void checkRepeatTime(UInt64 repeat_time) + static inline void checkRepeatTime(size_t repeat_time) { - static constexpr UInt64 max_repeat_times = 1'000'000; + static constexpr auto max_repeat_times = 1'000'000uz; if (repeat_time > max_repeat_times) throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times); } @@ -114,8 +114,8 @@ public: checkRepeatTime(times); - if (res_chars.size() + times + 1 >= res_chars.capacity()) - res_chars.resize(2 * res_chars.capacity()); + if (pos + times + 1 > res_chars.size()) + res_chars.resize(std::max(2 * res_chars.size(), pos + times + 1)); memset(res_chars.begin() + pos, space, times); pos += times; From f48845fa0d56dc81a44fd6314342849325d78b0b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 25 May 2023 12:23:35 +0000 Subject: [PATCH 0668/2223] Fix test once again --- .../integration/runner/compose/docker_compose_mongo_secure.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/compose/docker_compose_mongo_secure.yml b/docker/test/integration/runner/compose/docker_compose_mongo_secure.yml index f5b0ffed130..193e5d26568 100644 --- a/docker/test/integration/runner/compose/docker_compose_mongo_secure.yml +++ b/docker/test/integration/runner/compose/docker_compose_mongo_secure.yml @@ -1,7 +1,7 @@ version: '2.3' services: mongo1: - image: mongo:3.5 + image: mongo:3.6 restart: always environment: MONGO_INITDB_ROOT_USERNAME: root From e3ce2f834acd13e844b49c3bb7cea7aafe8b63dc Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 25 May 2023 12:35:00 +0000 Subject: [PATCH 0669/2223] fix style --- src/Common/tests/gtest_async_loader.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 2e4c42d6826..6a7d2d727a9 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -704,7 +704,7 @@ TEST(AsyncLoader, DynamicPriorities) { { std::unique_lock lock{schedule_mutex}; - schedule += fmt::format("{}{}", self->name, self->execution_pool()); + schedule += fmt::format("{}{}", self->name, self->executionPool()); } if (prioritize && self->name == "C") @@ -905,7 +905,7 @@ TEST(AsyncLoader, DynamicPools) auto job_func = [&] (const LoadJobPtr & self) { - auto pool_id = self->execution_pool(); + auto pool_id = self->executionPool(); executing[pool_id]++; if (executing[pool_id] > max_threads[0]) boosted = true; From 1c3b6738f426779391e5126d34cd8f7d6cdaccfe Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 May 2023 14:41:04 +0200 Subject: [PATCH 0670/2223] Fixes for parallel replicas (#50195) --- src/Client/HedgedConnections.cpp | 2 +- src/Client/MultiplexedConnections.cpp | 2 +- src/Core/Settings.h | 2 +- src/Interpreters/Context.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 52 +++++++++++++------ src/Interpreters/InterpreterSelectQuery.h | 2 + src/Interpreters/interpretSubquery.cpp | 2 +- src/Planner/Planner.cpp | 22 +++++--- .../QueryPlan/ReadFromMergeTree.cpp | 26 +++------- src/Processors/QueryPlan/ReadFromMergeTree.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 14 +++-- src/Storages/StorageDistributed.cpp | 5 -- src/Storages/StorageMerge.cpp | 15 +----- src/Storages/StorageMerge.h | 1 - tests/broken_tests.txt | 3 ++ .../02535_max_parallel_replicas_custom_key.sh | 2 +- ...lel_replicas_with_final_modifier.reference | 1 - ..._parallel_replicas_with_final_modifier.sql | 5 -- ...rallel_replicas_plain_merge_tree.reference | 0 ...764_parallel_replicas_plain_merge_tree.sql | 16 ++++++ ...parallel_replicas_final_modifier.reference | 0 ...02765_parallel_replicas_final_modifier.sql | 14 +++++ 22 files changed, 113 insertions(+), 76 deletions(-) delete mode 100644 tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.reference delete mode 100644 tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.sql create mode 100644 tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.reference create mode 100644 tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql create mode 100644 tests/queries/0_stateless/02765_parallel_replicas_final_modifier.reference create mode 100644 tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index fe3acd7cc7b..f8966847e5a 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -174,7 +174,7 @@ void HedgedConnections::sendQuery( modified_settings.group_by_two_level_threshold_bytes = 0; } - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas == 0; if (offset_states.size() > 1 && enable_sample_offset_parallel_processing) { diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 7df767a4bcf..71f536b9687 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -142,7 +142,7 @@ void MultiplexedConnections::sendQuery( } } - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas == 0; size_t num_replicas = replica_states.size(); if (num_replicas > 1) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8967c55b3e2..47d4ba452f7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -154,7 +154,7 @@ class IColumn; M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ \ M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \ - M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ + M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a7a064a66d6..c594feb73c2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4324,7 +4324,7 @@ Context::ParallelReplicasMode Context::getParallelReplicasMode() const if (!settings_.parallel_replicas_custom_key.value.empty()) return CUSTOM_KEY; - if (settings_.allow_experimental_parallel_reading_from_replicas + if (settings_.allow_experimental_parallel_reading_from_replicas > 0 && !settings_.use_hedged_requests) return READ_TASKS; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8e02ce347de..ebaf88ea5d5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -116,6 +116,7 @@ namespace ErrorCodes extern const int ACCESS_DENIED; extern const int UNKNOWN_IDENTIFIER; extern const int BAD_ARGUMENTS; + extern const int SUPPORT_IS_DISABLED; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -409,6 +410,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( ApplyWithSubqueryVisitor().visit(query_ptr); } + query_info.query = query_ptr->clone(); query_info.original_query = query_ptr->clone(); if (settings.count_distinct_optimization) @@ -456,25 +458,35 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } - if (joined_tables.tablesCount() > 1 && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas)) + /// Check support for JOINs for parallel replicas + if (joined_tables.tablesCount() > 1 && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas > 0)) { - LOG_WARNING(log, "Joins are not supported with parallel replicas. Query will be executed without using them."); - context->setSetting("allow_experimental_parallel_reading_from_replicas", false); - context->setSetting("parallel_replicas_custom_key", String{""}); + if (settings.allow_experimental_parallel_reading_from_replicas == 1) + { + LOG_WARNING(log, "JOINs are not supported with parallel replicas. Query will be executed without using them."); + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + context->setSetting("parallel_replicas_custom_key", String{""}); + } + else if (settings.allow_experimental_parallel_reading_from_replicas == 2) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "JOINs are not supported with parallel replicas"); + } } - /// Try to execute query without parallel replicas if we find that there is a FINAL modifier there. - bool is_query_with_final = false; - if (query_info.table_expression_modifiers) - is_query_with_final = query_info.table_expression_modifiers->hasFinal(); - else if (query_info.query) - is_query_with_final = query_info.query->as().final(); - - if (is_query_with_final && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas)) + /// Check support for FINAL for parallel replicas + bool is_query_with_final = isQueryWithFinal(query_info); + if (is_query_with_final && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas > 0)) { - LOG_WARNING(log, "FINAL modifier is supported with parallel replicas. Will try to execute the query without using them."); - context->setSetting("allow_experimental_parallel_reading_from_replicas", false); - context->setSetting("parallel_replicas_custom_key", String{""}); + if (settings.allow_experimental_parallel_reading_from_replicas == 1) + { + LOG_WARNING(log, "FINAL modifier is not supported with parallel replicas. Query will be executed without using them."); + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + context->setSetting("parallel_replicas_custom_key", String{""}); + } + else if (settings.allow_experimental_parallel_reading_from_replicas == 2) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "FINAL modifier is not supported with parallel replicas"); + } } /// Rewrite JOINs @@ -3134,4 +3146,14 @@ void InterpreterSelectQuery::initSettings() } } +bool InterpreterSelectQuery::isQueryWithFinal(const SelectQueryInfo & info) +{ + bool result = info.query->as().final(); + if (info.table_expression_modifiers) + result |= info.table_expression_modifiers->hasFinal(); + + return result; +} + + } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 58fddb8ffe9..e39dd675136 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -131,6 +131,8 @@ public: static SortDescription getSortDescription(const ASTSelectQuery & query, const ContextPtr & context); static UInt64 getLimitForSorting(const ASTSelectQuery & query, const ContextPtr & context); + static bool isQueryWithFinal(const SelectQueryInfo & info); + private: InterpreterSelectQuery( const ASTPtr & query_ptr_, diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 2358b0ab42a..550fa2912ba 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -113,7 +113,7 @@ std::shared_ptr interpretSubquery( } /// We don't want to execute reading for subqueries in parallel - subquery_context->setSetting("allow_experimental_parallel_reading_from_replicas", false); + subquery_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); return std::make_shared(query, subquery_context, subquery_options, required_source_columns); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index ba338de4072..3c9df29534d 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -83,6 +83,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int TOO_DEEP_SUBQUERIES; extern const int NOT_IMPLEMENTED; + extern const int SUPPORT_IS_DISABLED; } /** ClickHouse query planner. @@ -1192,16 +1193,25 @@ void Planner::buildPlanForQueryNode() const auto & settings = query_context->getSettingsRef(); if (planner_context->getTableExpressionNodeToData().size() > 1 - && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas)) + && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas > 0)) { - LOG_WARNING( - &Poco::Logger::get("Planner"), "Joins are not supported with parallel replicas. Query will be executed without using them."); + if (settings.allow_experimental_parallel_reading_from_replicas == 1) + { + LOG_WARNING( + &Poco::Logger::get("Planner"), "JOINs are not supported with parallel replicas. Query will be executed without using them."); - auto & mutable_context = planner_context->getMutableQueryContext(); - mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", false); - mutable_context->setSetting("parallel_replicas_custom_key", String{""}); + auto & mutable_context = planner_context->getMutableQueryContext(); + mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + mutable_context->setSetting("parallel_replicas_custom_key", String{""}); + } + else if (settings.allow_experimental_parallel_reading_from_replicas == 2) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "JOINs are not supported with parallel replicas"); + } } + /// TODO: Also disable parallel replicas in case of FINAL + auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context); auto join_tree_query_plan = buildJoinTreeQueryPlan(query_tree, select_query_info, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6180a01dec6..b38c3422be1 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -99,7 +100,6 @@ namespace ErrorCodes extern const int INDEX_NOT_USED; extern const int LOGICAL_ERROR; extern const int TOO_MANY_ROWS; - extern const int SUPPORT_IS_DISABLED; } static MergeTreeReaderSettings getMergeTreeReaderSettings( @@ -1314,7 +1314,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( auto reader_settings = getMergeTreeReaderSettings(context, query_info); bool use_skip_indexes = settings.use_skip_indexes; - bool final = isFinal(query_info); + bool final = InterpreterSelectQuery::isQueryWithFinal(query_info); if (final && !settings.use_skip_indexes_if_final) use_skip_indexes = false; @@ -1377,7 +1377,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, /// Disable read-in-order optimization for reverse order with final. /// Otherwise, it can lead to incorrect final behavior because the implementation may rely on the reading in direct order). - if (direction != 1 && isFinal(query_info)) + if (direction != 1 && isQueryWithFinal()) return false; auto order_info = std::make_shared(SortDescription{}, prefix_size, direction, limit); @@ -1500,11 +1500,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const bool ReadFromMergeTree::isQueryWithFinal() const { - const auto & select = query_info.query->as(); - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - else - return select.final(); + return InterpreterSelectQuery::isQueryWithFinal(query_info); } bool ReadFromMergeTree::isQueryWithSampling() const @@ -1522,7 +1518,7 @@ bool ReadFromMergeTree::isQueryWithSampling() const Pipe ReadFromMergeTree::spreadMarkRanges( RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection) { - bool final = isQueryWithFinal(); + const bool final = isQueryWithFinal(); const auto & input_order_info = query_info.getInputOrderInfo(); Names column_names_to_read = result.column_names_to_read; @@ -1539,8 +1535,7 @@ Pipe ReadFromMergeTree::spreadMarkRanges( if (final) { - if (is_parallel_reading_from_replicas) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "FINAL modifier is not supported with parallel replicas"); + chassert(!is_parallel_reading_from_replicas); if (output_each_partition_through_separate_port) throw Exception(ErrorCodes::LOGICAL_ERROR, "Optimisation isn't supposed to be used for queries with final"); @@ -1960,15 +1955,6 @@ void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const } } -bool ReadFromMergeTree::isFinal(const SelectQueryInfo & query_info) -{ - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - - const auto & select = query_info.query->as(); - return select.final(); -} - bool MergeTreeDataSelectAnalysisResult::error() const { return std::holds_alternative(result); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 5e4ba117967..545ffe84e0c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -159,7 +159,6 @@ public: void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value); - static bool isFinal(const SelectQueryInfo & query_info); bool isQueryWithFinal() const; bool isQueryWithSampling() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3d3e7f2c735..1d5ac21f803 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7198,9 +7198,17 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( if (query_context->getClientInfo().collaborate_with_initiator) return QueryProcessingStage::Enum::FetchColumns; - if (query_context->canUseParallelReplicasOnInitiator() - && to_stage >= QueryProcessingStage::WithMergeableState) - return QueryProcessingStage::Enum::WithMergeableState; + /// Parallel replicas + if (query_context->canUseParallelReplicasOnInitiator() && to_stage >= QueryProcessingStage::WithMergeableState) + { + /// ReplicatedMergeTree + if (supportsReplication()) + return QueryProcessingStage::Enum::WithMergeableState; + + /// For non-replicated MergeTree we allow them only if parallel_replicas_for_non_replicated_merge_tree is enabled + if (query_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) + return QueryProcessingStage::Enum::WithMergeableState; + } if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 86a2599b49c..dd2d3ebfaf0 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -145,7 +145,6 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_NUMBER_OF_COLUMNS; extern const int INFINITE_LOOP; - extern const int ILLEGAL_FINAL; extern const int TYPE_MISMATCH; extern const int TOO_MANY_ROWS; extern const int UNABLE_TO_SKIP_UNUSED_SHARDS; @@ -1045,10 +1044,6 @@ void StorageDistributed::read( const size_t /*max_block_size*/, const size_t /*num_streams*/) { - const auto * select_query = query_info.query->as(); - if (select_query->final() && local_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas) - throw Exception(ErrorCodes::ILLEGAL_FINAL, "Final modifier is not allowed together with parallel reading from replicas feature"); - Block header; ASTPtr query_ast; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 7ecb424673d..b0ed242d14d 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -633,10 +633,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( auto & modified_select = modified_query_info.query->as(); QueryPipelineBuilderPtr builder; - - bool final = isFinal(modified_query_info); - - if (!final && storage->needRewriteQueryWithFinal(real_column_names)) + if (!InterpreterSelectQuery::isQueryWithFinal(modified_query_info) && storage->needRewriteQueryWithFinal(real_column_names)) { /// NOTE: It may not work correctly in some cases, because query was analyzed without final. /// However, it's needed for MaterializedMySQL and it's unlikely that someone will use it with Merge tables. @@ -1010,21 +1007,13 @@ bool ReadFromMerge::requestReadingInOrder(InputOrderInfoPtr order_info_) { /// Disable read-in-order optimization for reverse order with final. /// Otherwise, it can lead to incorrect final behavior because the implementation may rely on the reading in direct order). - if (order_info_->direction != 1 && isFinal(query_info)) + if (order_info_->direction != 1 && InterpreterSelectQuery::isQueryWithFinal(query_info)) return false; order_info = order_info_; return true; } -bool ReadFromMerge::isFinal(const SelectQueryInfo & query_info) -{ - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - const auto & select_query = query_info.query->as(); - return select_query.final(); -} - IStorage::ColumnSizeByName StorageMerge::getColumnSizes() const { ColumnSizeByName column_sizes; diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index c4b6d815935..babf0dd92e8 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -145,7 +145,6 @@ public: /// Returns `false` if requested reading cannot be performed. bool requestReadingInOrder(InputOrderInfoPtr order_info_); - static bool isFinal(const SelectQueryInfo & query_info); private: const size_t required_max_block_size; diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index c5a58440a8a..faee1c5b295 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -139,3 +139,6 @@ 02703_row_policy_for_database 02721_url_cluster 02534_s3_cluster_insert_select_schema_inference +02764_parallel_replicas_plain_merge_tree +02765_parallel_replicas_final_modifier + diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 3035a191c8f..50e89cca4c9 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -41,6 +41,6 @@ run_count_with_custom_key "y" run_count_with_custom_key "cityHash64(y)" run_count_with_custom_key "cityHash64(y) + 1" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "Joins are not supported with parallel replicas" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with parallel replicas" $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" diff --git a/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.reference b/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.reference deleted file mode 100644 index f347e8b5857..00000000000 --- a/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.reference +++ /dev/null @@ -1 +0,0 @@ -1 1 2020-01-01 00:00:00 diff --git a/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.sql b/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.sql deleted file mode 100644 index 1f5daaae189..00000000000 --- a/tests/queries/0_stateless/02709_parallel_replicas_with_final_modifier.sql +++ /dev/null @@ -1,5 +0,0 @@ -DROP TABLE IF EXISTS t_02709; -CREATE TABLE t_02709 (key UInt32, sign Int8, date Datetime) ENGINE=CollapsingMergeTree(sign) PARTITION BY date ORDER BY key; -INSERT INTO t_02709 VALUES (1, 1, '2020-01-01'), (2, 1, '2020-01-02'), (1, -1, '2020-01-01'), (2, -1, '2020-01-02'), (1, 1, '2020-01-01'); -SELECT * FROM t_02709 FINAL ORDER BY key SETTINGS max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; -DROP TABLE t_02709; diff --git a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.reference b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql new file mode 100644 index 00000000000..ea8eb04bd07 --- /dev/null +++ b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql @@ -0,0 +1,16 @@ +CREATE TABLE IF NOT EXISTS parallel_replicas_plain (x String) ENGINE=MergeTree() ORDER BY x; +INSERT INTO parallel_replicas_plain SELECT toString(number) FROM numbers(10); + +SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; + +SET parallel_replicas_for_non_replicated_merge_tree = 0; + +SELECT x FROM parallel_replicas_plain LIMIT 1 FORMAT Null; +SELECT max(length(x)) FROM parallel_replicas_plain FORMAT Null; + +SET parallel_replicas_for_non_replicated_merge_tree = 1; + +SELECT x FROM parallel_replicas_plain LIMIT 1 FORMAT Null; +SELECT max(length(x)) FROM parallel_replicas_plain FORMAT Null; + +DROP TABLE IF EXISTS parallel_replicas_plain; diff --git a/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.reference b/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql b/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql new file mode 100644 index 00000000000..f447051e1e5 --- /dev/null +++ b/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql @@ -0,0 +1,14 @@ +CREATE TABLE IF NOT EXISTS parallel_replicas_final (x String) ENGINE=ReplacingMergeTree() ORDER BY x; + +INSERT INTO parallel_replicas_final SELECT toString(number) FROM numbers(10); + +SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; +SET parallel_replicas_for_non_replicated_merge_tree = 1; + +SELECT * FROM parallel_replicas_final FINAL FORMAT Null; + +SET allow_experimental_parallel_reading_from_replicas=2; + +SELECT * FROM parallel_replicas_final FINAL FORMAT Null; -- { serverError SUPPORT_IS_DISABLED } + +DROP TABLE IF EXISTS parallel_replicas_final; From b8d3e495e51849588ceca6a6976c3931bdd3ef2d Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 25 May 2023 12:42:19 +0000 Subject: [PATCH 0671/2223] add `pool_id` out-of-bound checks --- src/Common/AsyncLoader.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 6e558f36b7d..b5612517cd6 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -272,6 +272,7 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) // Pass 2. Schedule all incoming jobs for (const auto & job : jobs) { + chassert(job->pool() < pools.size()); NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; scheduled_jobs.try_emplace(job); @@ -359,6 +360,7 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool) { if (!job) return; + chassert(new_pool < pools.size()); DENY_ALLOCATIONS_IN_SCOPE; std::unique_lock lock{mutex}; prioritize(job, new_pool, lock); From 3ef6cb7bdc7027db6e40500080c713e490e079e7 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 25 May 2023 13:24:45 +0000 Subject: [PATCH 0672/2223] git-apply #50205 --- src/Common/Priority.h | 11 ++++++++++ src/Common/ThreadPool.cpp | 8 +++---- src/Common/ThreadPool.h | 21 ++++++++++--------- .../CachedCompressedReadBuffer.cpp | 2 +- src/Compression/CachedCompressedReadBuffer.h | 2 +- .../CompressedReadBufferFromFile.cpp | 2 +- .../CompressedReadBufferFromFile.h | 2 +- .../IO/AsynchronousBoundedReadBuffer.cpp | 6 +++--- src/IO/AsynchronousReadBufferFromFile.cpp | 4 ++-- src/IO/AsynchronousReadBufferFromFile.h | 6 +++--- ...ynchronousReadBufferFromFileDescriptor.cpp | 8 +++---- ...AsynchronousReadBufferFromFileDescriptor.h | 9 ++++---- src/IO/AsynchronousReader.h | 3 ++- src/IO/CompressedReadBufferWrapper.h | 2 +- src/IO/ParallelReadBuffer.cpp | 2 +- src/IO/PeekableReadBuffer.h | 2 +- src/IO/ReadBuffer.h | 9 ++++---- src/IO/ReadBufferFromFileDescriptor.cpp | 2 +- src/IO/ReadBufferFromFileDescriptor.h | 2 +- src/IO/ReadSettings.h | 5 +++-- src/IO/S3/copyS3File.cpp | 2 +- src/IO/SeekAvoidingReadBuffer.h | 2 +- src/IO/WriteBufferFromS3TaskTracker.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- .../FilesystemReadPrefetchesLog.cpp | 4 ++-- .../FilesystemReadPrefetchesLog.h | 3 ++- src/Interpreters/threadPoolCallbackRunner.h | 9 ++++---- .../HDFS/AsynchronousReadBufferFromHDFS.cpp | 6 +++--- .../HDFS/AsynchronousReadBufferFromHDFS.h | 6 +++--- src/Storages/MergeTree/IMergeTreeReader.h | 2 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 2 +- .../MergeTree/MergeTreeBlockReadUtils.h | 8 ++----- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 14 ++++++------- .../MergeTree/MergeTreePrefetchedReadPool.h | 3 +-- .../MergeTree/MergeTreeReaderCompact.cpp | 2 +- .../MergeTree/MergeTreeReaderCompact.h | 2 +- .../MergeTree/MergeTreeReaderWide.cpp | 8 +++---- src/Storages/MergeTree/MergeTreeReaderWide.h | 6 +++--- src/Storages/MergeTree/MergeTreeSource.cpp | 2 +- src/Storages/StorageS3.cpp | 4 ++-- 41 files changed, 105 insertions(+), 94 deletions(-) create mode 100644 src/Common/Priority.h diff --git a/src/Common/Priority.h b/src/Common/Priority.h new file mode 100644 index 00000000000..8952fe4dd5a --- /dev/null +++ b/src/Common/Priority.h @@ -0,0 +1,11 @@ +#pragma once + +#include + +/// Common type for priority values. +/// Separate type (rather than `Int64` is used just to avoid implicit conversion errors and to default-initialize +struct Priority +{ + Int64 value = 0; /// Note that lower value means higher priority. + constexpr operator Int64() const { return value; } /// NOLINT +}; diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 63eef72dffb..f91849ead66 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -123,7 +123,7 @@ void ThreadPoolImpl::setQueueSize(size_t value) template template -ReturnType ThreadPoolImpl::scheduleImpl(Job job, ssize_t priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context) +ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context) { auto on_error = [&](const std::string & reason) { @@ -231,19 +231,19 @@ void ThreadPoolImpl::startNewThreadsNoLock() } template -void ThreadPoolImpl::scheduleOrThrowOnError(Job job, ssize_t priority) +void ThreadPoolImpl::scheduleOrThrowOnError(Job job, Priority priority) { scheduleImpl(std::move(job), priority, std::nullopt); } template -bool ThreadPoolImpl::trySchedule(Job job, ssize_t priority, uint64_t wait_microseconds) noexcept +bool ThreadPoolImpl::trySchedule(Job job, Priority priority, uint64_t wait_microseconds) noexcept { return scheduleImpl(std::move(job), priority, wait_microseconds); } template -void ThreadPoolImpl::scheduleOrThrow(Job job, ssize_t priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context) +void ThreadPoolImpl::scheduleOrThrow(Job job, Priority priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context) { scheduleImpl(std::move(job), priority, wait_microseconds, propagate_opentelemetry_tracing_context); } diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index b92cb256b13..70053ff7558 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -18,6 +18,7 @@ #include #include #include +#include #include /** Very simple thread pool similar to boost::threadpool. @@ -59,17 +60,17 @@ public: /// If any thread was throw an exception, first exception will be rethrown from this method, /// and exception will be cleared. /// Also throws an exception if cannot create thread. - /// Priority: greater is higher. + /// Priority: lower is higher. /// NOTE: Probably you should call wait() if exception was thrown. If some previously scheduled jobs are using some objects, /// located on stack of current thread, the stack must not be unwinded until all jobs finished. However, /// if ThreadPool is a local object, it will wait for all scheduled jobs in own destructor. - void scheduleOrThrowOnError(Job job, ssize_t priority = 0); + void scheduleOrThrowOnError(Job job, Priority priority = {}); /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or return false. - bool trySchedule(Job job, ssize_t priority = 0, uint64_t wait_microseconds = 0) noexcept; + bool trySchedule(Job job, Priority priority = {}, uint64_t wait_microseconds = 0) noexcept; /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or throw an exception. - void scheduleOrThrow(Job job, ssize_t priority = 0, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); + void scheduleOrThrow(Job job, Priority priority = {}, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); /// Wait for all currently active jobs to be done. /// You may call schedule and wait many times in arbitrary order. @@ -123,15 +124,15 @@ private: struct JobWithPriority { Job job; - ssize_t priority; + Priority priority; DB::OpenTelemetry::TracingContextOnThread thread_trace_context; - JobWithPriority(Job job_, ssize_t priority_, const DB::OpenTelemetry::TracingContextOnThread& thread_trace_context_) + JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_) : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) {} - bool operator< (const JobWithPriority & rhs) const + bool operator<(const JobWithPriority & rhs) const { - return priority < rhs.priority; + return priority > rhs.priority; // Reversed for `priority_queue` max-heap to yield minimum value (i.e. highest priority) first } }; @@ -141,7 +142,7 @@ private: std::stack on_destroy_callbacks; template - ReturnType scheduleImpl(Job job, ssize_t priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context = true); + ReturnType scheduleImpl(Job job, Priority priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context = true); void worker(typename std::list::iterator thread_it); @@ -227,7 +228,7 @@ public: DB::ThreadStatus thread_status; std::apply(function, arguments); }, - 0, // default priority + {}, // default priority 0, // default wait_microseconds propagate_opentelemetry_context ); diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index fdb2132d134..0febfca75cc 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -28,7 +28,7 @@ void CachedCompressedReadBuffer::initInput() } -void CachedCompressedReadBuffer::prefetch(int64_t priority) +void CachedCompressedReadBuffer::prefetch(Priority priority) { initInput(); file_in->prefetch(priority); diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index ee2728752e9..cce9a8e671c 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -36,7 +36,7 @@ private: bool nextImpl() override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; /// Passed into file_in. ReadBufferFromFileBase::ProfileCallback profile_callback; diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index ca697e2f51e..9dc40b8217c 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -51,7 +51,7 @@ CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr buf, bool allow_different_codecs_ = false); diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index 611fd00a023..aed4c2f82f2 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -83,19 +83,19 @@ bool AsynchronousBoundedReadBuffer::hasPendingDataToRead() } std::future -AsynchronousBoundedReadBuffer::asyncReadInto(char * data, size_t size, int64_t priority) +AsynchronousBoundedReadBuffer::asyncReadInto(char * data, size_t size, Priority priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(*impl, async_read_counters); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = read_settings.priority + priority; + request.priority = Priority{read_settings.priority.value + priority.value}; request.ignore = bytes_to_ignore; return reader.submit(request); } -void AsynchronousBoundedReadBuffer::prefetch(int64_t priority) +void AsynchronousBoundedReadBuffer::prefetch(Priority priority) { if (prefetch_future.valid()) return; diff --git a/src/IO/AsynchronousReadBufferFromFile.cpp b/src/IO/AsynchronousReadBufferFromFile.cpp index 7499b684a89..0e6c8090cb5 100644 --- a/src/IO/AsynchronousReadBufferFromFile.cpp +++ b/src/IO/AsynchronousReadBufferFromFile.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, const std::string & file_name_, size_t buf_size, int flags, @@ -60,7 +60,7 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int & fd_, const std::string & original_file_name, size_t buf_size, diff --git a/src/IO/AsynchronousReadBufferFromFile.h b/src/IO/AsynchronousReadBufferFromFile.h index 0ac0a820e48..d3b7ffbc7dc 100644 --- a/src/IO/AsynchronousReadBufferFromFile.h +++ b/src/IO/AsynchronousReadBufferFromFile.h @@ -17,7 +17,7 @@ protected: public: explicit AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, @@ -28,7 +28,7 @@ public: /// Use pre-opened file descriptor. explicit AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int & fd, /// Will be set to -1 if constructor didn't throw and ownership of file descriptor is passed to the object. const std::string & original_file_name = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, @@ -58,7 +58,7 @@ private: public: AsynchronousReadBufferFromFileWithDescriptorsCache( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp index 743892d24f6..d30773f88f4 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp @@ -40,14 +40,14 @@ std::string AsynchronousReadBufferFromFileDescriptor::getFileName() const } -std::future AsynchronousReadBufferFromFileDescriptor::asyncReadInto(char * data, size_t size, int64_t priority) +std::future AsynchronousReadBufferFromFileDescriptor::asyncReadInto(char * data, size_t size, Priority priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(fd); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = base_priority + priority; + request.priority = Priority{base_priority.value + priority.value}; request.ignore = bytes_to_ignore; bytes_to_ignore = 0; @@ -61,7 +61,7 @@ std::future AsynchronousReadBufferFromFileDescripto } -void AsynchronousReadBufferFromFileDescriptor::prefetch(int64_t priority) +void AsynchronousReadBufferFromFileDescriptor::prefetch(Priority priority) { if (prefetch_future.valid()) return; @@ -151,7 +151,7 @@ void AsynchronousReadBufferFromFileDescriptor::finalize() AsynchronousReadBufferFromFileDescriptor::AsynchronousReadBufferFromFileDescriptor( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int fd_, size_t buf_size, char * existing_memory, diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.h b/src/IO/AsynchronousReadBufferFromFileDescriptor.h index dff56baef4e..4a4130ebab1 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.h +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -18,7 +19,7 @@ class AsynchronousReadBufferFromFileDescriptor : public ReadBufferFromFileBase { protected: IAsynchronousReader & reader; - int64_t base_priority; + Priority base_priority; Memory<> prefetch_buffer; std::future prefetch_future; @@ -39,7 +40,7 @@ protected: public: AsynchronousReadBufferFromFileDescriptor( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int fd_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, @@ -49,7 +50,7 @@ public: ~AsynchronousReadBufferFromFileDescriptor() override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; int getFD() const { @@ -70,7 +71,7 @@ public: size_t getFileSize() override; private: - std::future asyncReadInto(char * data, size_t size, int64_t priority); + std::future asyncReadInto(char * data, size_t size, Priority priority); }; } diff --git a/src/IO/AsynchronousReader.h b/src/IO/AsynchronousReader.h index 91a5db9dcdd..6b8b93fcc20 100644 --- a/src/IO/AsynchronousReader.h +++ b/src/IO/AsynchronousReader.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -47,7 +48,7 @@ public: size_t offset = 0; size_t size = 0; char * buf = nullptr; - int64_t priority = 0; + Priority priority; size_t ignore = 0; }; diff --git a/src/IO/CompressedReadBufferWrapper.h b/src/IO/CompressedReadBufferWrapper.h index 8fe87d22dd9..bb58a7bfeb3 100644 --- a/src/IO/CompressedReadBufferWrapper.h +++ b/src/IO/CompressedReadBufferWrapper.h @@ -19,7 +19,7 @@ public: const ReadBuffer & getWrappedReadBuffer() const { return *in; } ReadBuffer & getWrappedReadBuffer() { return *in; } - void prefetch(int64_t priority) override { in->prefetch(priority); } + void prefetch(Priority priority) override { in->prefetch(priority); } protected: std::unique_ptr in; diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 6188fc210be..fff02db1bd6 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -87,7 +87,7 @@ bool ParallelReadBuffer::addReaderToPool() auto worker = read_workers.emplace_back(std::make_shared(std::move(reader), range_start, size)); ++active_working_reader; - schedule([this, my_worker = std::move(worker)]() mutable { readerThreadFunction(std::move(my_worker)); }, 0); + schedule([this, my_worker = std::move(worker)]() mutable { readerThreadFunction(std::move(my_worker)); }, Priority{}); return true; } diff --git a/src/IO/PeekableReadBuffer.h b/src/IO/PeekableReadBuffer.h index 7af17aff090..78cb319327d 100644 --- a/src/IO/PeekableReadBuffer.h +++ b/src/IO/PeekableReadBuffer.h @@ -20,7 +20,7 @@ public: ~PeekableReadBuffer() override; - void prefetch(int64_t priority) override { sub_buf->prefetch(priority); } + void prefetch(Priority priority) override { sub_buf->prefetch(priority); } /// Sets checkpoint at current position ALWAYS_INLINE inline void setCheckpoint() diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 1a8222001a9..a4ae12f5069 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -20,7 +21,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -static constexpr auto DEFAULT_PREFETCH_PRIORITY = 0; +static constexpr auto DEFAULT_PREFETCH_PRIORITY = Priority{0}; /** A simple abstract class for buffered data reading (char sequences) from somewhere. * Unlike std::istream, it provides access to the internal buffer, @@ -208,10 +209,10 @@ public: /** Do something to allow faster subsequent call to 'nextImpl' if possible. * It's used for asynchronous readers with double-buffering. - * `priority` is the Threadpool priority, with which the prefetch task will be schedules. - * Smaller is more priority. + * `priority` is the `ThreadPool` priority, with which the prefetch task will be scheduled. + * Lower value means higher priority. */ - virtual void prefetch(int64_t /* priority */) {} + virtual void prefetch(Priority) {} /** * Set upper bound for read range [..., position). diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 1e7ea855364..bf44d9d10da 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -124,7 +124,7 @@ bool ReadBufferFromFileDescriptor::nextImpl() } -void ReadBufferFromFileDescriptor::prefetch(int64_t) +void ReadBufferFromFileDescriptor::prefetch(Priority) { #if defined(POSIX_FADV_WILLNEED) /// For direct IO, loading data into page cache is pointless. diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index daf8ac56d13..10f140275bb 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -25,7 +25,7 @@ protected: ThrottlerPtr throttler; bool nextImpl() override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; /// Name or some description of file. std::string getFileName() const override; diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index d1551b16324..e43ecd7f275 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -5,6 +5,7 @@ #include #include #include +#include #include namespace DB @@ -84,8 +85,8 @@ struct ReadSettings size_t mmap_threshold = 0; MMappedFileCache * mmap_cache = nullptr; - /// For 'pread_threadpool'/'io_uring' method. Lower is more priority. - size_t priority = 0; + /// For 'pread_threadpool'/'io_uring' method. Lower value is higher priority. + Priority priority; bool load_marks_asynchronously = true; diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 20490ef9a19..7886b84cd00 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -361,7 +361,7 @@ namespace task->exception = std::current_exception(); } task_finish_notify(); - }, 0); + }, Priority{}); } catch (...) { diff --git a/src/IO/SeekAvoidingReadBuffer.h b/src/IO/SeekAvoidingReadBuffer.h index af2a988ab3a..716d7c5046c 100644 --- a/src/IO/SeekAvoidingReadBuffer.h +++ b/src/IO/SeekAvoidingReadBuffer.h @@ -17,7 +17,7 @@ public: off_t seek(off_t off, int whence) override; - void prefetch(int64_t priority) override { impl->prefetch(priority); } + void prefetch(Priority priority) override { impl->prefetch(priority); } private: UInt64 min_bytes_for_seek; /// Minimum positive seek offset which shall be executed using seek operation. diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 7826747c0a4..6c7bb35fc5a 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -113,7 +113,7 @@ void WriteBufferFromS3::TaskTracker::add(Callback && func) { LOG_TEST(log, "add, in queue {}", futures.size()); - auto future = scheduler(std::move(func), 0); + auto future = scheduler(std::move(func), Priority{}); auto exit_scope = scope_guard( [&future]() { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c594feb73c2..91f34c8eb97 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4269,7 +4269,7 @@ ReadSettings Context::getReadSettings() const res.prefetch_buffer_size = settings.prefetch_buffer_size; res.direct_io_threshold = settings.min_bytes_to_use_direct_io; res.mmap_threshold = settings.min_bytes_to_use_mmap_io; - res.priority = settings.read_priority; + res.priority = Priority{settings.read_priority}; res.remote_throttler = getRemoteReadThrottler(); res.local_throttler = getLocalReadThrottler(); diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.cpp b/src/Interpreters/FilesystemReadPrefetchesLog.cpp index 9dd0fce84b0..221cb11f09a 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.cpp +++ b/src/Interpreters/FilesystemReadPrefetchesLog.cpp @@ -19,7 +19,7 @@ NamesAndTypesList FilesystemReadPrefetchesLogElement::getNamesAndTypes() {"offset", std::make_shared()}, {"size", std::make_shared()}, {"prefetch_submit_time", std::make_shared(6)}, - {"priority", std::make_shared()}, + {"priority", std::make_shared()}, {"prefetch_execution_start_time", std::make_shared(6)}, {"prefetch_execution_end_time", std::make_shared(6)}, {"prefetch_execution_time_us", std::make_shared()}, @@ -40,7 +40,7 @@ void FilesystemReadPrefetchesLogElement::appendToBlock(MutableColumns & columns) columns[i++]->insert(offset); columns[i++]->insert(size); columns[i++]->insert(prefetch_submit_time); - columns[i++]->insert(priority); + columns[i++]->insert(priority.value); if (execution_watch) { columns[i++]->insert(execution_watch->getStart()); diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.h b/src/Interpreters/FilesystemReadPrefetchesLog.h index 7052cf2769d..cf36f513f5a 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.h +++ b/src/Interpreters/FilesystemReadPrefetchesLog.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -25,7 +26,7 @@ struct FilesystemReadPrefetchesLogElement Int64 size; /// -1 means unknown Decimal64 prefetch_submit_time{}; std::optional execution_watch; - size_t priority; + Priority priority; FilesystemPrefetchState state; UInt64 thread_id; String reader_id; diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index b8805722346..55c6a848b77 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -11,13 +11,13 @@ namespace DB /// High-order function to run callbacks (functions with 'void()' signature) somewhere asynchronously. template > -using ThreadPoolCallbackRunner = std::function(Callback &&, int64_t priority)>; +using ThreadPoolCallbackRunner = std::function(Callback &&, Priority)>; /// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'. template > ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name) { - return [my_pool = &pool, thread_group = CurrentThread::getGroup(), thread_name](Callback && callback, int64_t priority) mutable -> std::future + return [my_pool = &pool, thread_group = CurrentThread::getGroup(), thread_name](Callback && callback, Priority priority) mutable -> std::future { auto task = std::make_shared>([thread_group, thread_name, my_callback = std::move(callback)]() mutable -> Result { @@ -44,15 +44,14 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & auto future = task->get_future(); - /// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority". - my_pool->scheduleOrThrow([my_task = std::move(task)]{ (*my_task)(); }, -priority); + my_pool->scheduleOrThrow([my_task = std::move(task)]{ (*my_task)(); }, priority); return future; }; } template -std::future scheduleFromThreadPool(T && task, ThreadPool & pool, const std::string & thread_name, int64_t priority = 0) +std::future scheduleFromThreadPool(T && task, ThreadPool & pool, const std::string & thread_name, Priority priority = {}) { auto schedule = threadPoolCallbackRunner(pool, thread_name); return schedule(std::move(task), priority); diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp index a8502b1bd65..b490c5cac63 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -64,19 +64,19 @@ bool AsynchronousReadBufferFromHDFS::hasPendingDataToRead() return true; } -std::future AsynchronousReadBufferFromHDFS::asyncReadInto(char * data, size_t size, int64_t priority) +std::future AsynchronousReadBufferFromHDFS::asyncReadInto(char * data, size_t size, Priority priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(*impl, nullptr); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = base_priority + priority; + request.priority = Priority{base_priority.value + priority.value}; request.ignore = 0; return reader.submit(request); } -void AsynchronousReadBufferFromHDFS::prefetch(int64_t priority) +void AsynchronousReadBufferFromHDFS::prefetch(Priority priority) { interval_watch.restart(); diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h index 07d32194a93..d89aa60ab71 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h @@ -33,7 +33,7 @@ public: off_t seek(off_t offset_, int whence) override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; size_t getFileSize() override; @@ -50,10 +50,10 @@ private: bool hasPendingDataToRead(); - std::future asyncReadInto(char * data, size_t size, int64_t priority); + std::future asyncReadInto(char * data, size_t size, Priority priority); IAsynchronousReader & reader; - int64_t base_priority; + Priority base_priority; std::shared_ptr impl; std::future prefetch_future; Memory<> prefetch_buffer; diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index c7396e882fa..f4759409c52 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -61,7 +61,7 @@ public: MergeTreeDataPartInfoForReaderPtr data_part_info_for_read; - virtual void prefetchBeginOfRange(int64_t /* priority */) {} + virtual void prefetchBeginOfRange(Priority) {} protected: /// Returns actual column name in part, which can differ from table metadata. diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 4c3d4bc8aa0..9eeb1ec13cf 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -142,7 +142,7 @@ MergeTreeReadTask::MergeTreeReadTask( const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, MergeTreeBlockSizePredictorPtr size_predictor_, - int64_t priority_, + Priority priority_, std::future reader_, std::vector> && pre_reader_for_step_) : data_part{data_part_} diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 5fce95e1714..6eefa993e5e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -71,11 +71,7 @@ struct MergeTreeReadTask std::future reader; std::vector> pre_reader_for_step; - int64_t priority = 0; /// Priority of the task. Bigger value, bigger priority. - bool operator <(const MergeTreeReadTask & rhs) const - { - return priority < rhs.priority; - } + Priority priority; bool isFinished() const { return mark_ranges.empty() && range_reader.isCurrentRangeFinished(); } @@ -86,7 +82,7 @@ struct MergeTreeReadTask const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, MergeTreeBlockSizePredictorPtr size_predictor_, - int64_t priority_ = 0, + Priority priority_ = {}, std::future reader_ = {}, std::vector> && pre_reader_for_step_ = {}); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1d5ac21f803..11aa7c5ec76 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1967,7 +1967,7 @@ try res.part->remove(); else preparePartForRemoval(res.part); - }, 0)); + }, Priority{})); } /// Wait for every scheduled task diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 30f31910bee..6b5214622ff 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -90,7 +90,7 @@ std::future MergeTreePrefetchedReadPool::createPrefetchedRea const IMergeTreeDataPart & data_part, const NamesAndTypesList & columns, const MarkRanges & required_ranges, - int64_t priority) const + Priority priority) const { auto reader = data_part.getReader( columns, storage_snapshot->metadata, required_ranges, @@ -142,7 +142,7 @@ bool MergeTreePrefetchedReadPool::TaskHolder::operator <(const TaskHolder & othe { chassert(task->priority >= 0); chassert(other.task->priority >= 0); - return -task->priority < -other.task->priority; /// Less is better. + return task->priority > other.task->priority; /// Less is better. /// With default std::priority_queue, top() returns largest element. /// So closest to 0 will be on top with this comparator. } @@ -153,7 +153,7 @@ void MergeTreePrefetchedReadPool::startPrefetches() const return; [[maybe_unused]] TaskHolder prev(nullptr, 0); - [[maybe_unused]] const int64_t highest_priority = reader_settings.read_settings.priority + 1; + [[maybe_unused]] const Priority highest_priority{reader_settings.read_settings.priority.value + 1}; assert(prefetch_queue.top().task->priority == highest_priority); while (!prefetch_queue.empty()) { @@ -495,11 +495,11 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr auto need_marks = min_marks_per_thread; /// Priority is given according to the prefetch number for each thread, - /// e.g. the first task of each thread has the same priority and is bigger - /// than second task of each thread, and so on. + /// e.g. the first task of each thread has the same priority and is greater + /// than the second task of each thread, and so on. /// Add 1 to query read priority because higher priority should be given to /// reads from pool which are from reader. - int64_t priority = reader_settings.read_settings.priority + 1; + Priority priority{reader_settings.read_settings.priority.value + 1}; while (need_marks > 0 && part_idx < parts_infos.size()) { @@ -597,7 +597,7 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr { prefetch_queue.emplace(TaskHolder(read_task.get(), i)); } - ++priority; + ++priority.value; result_threads_tasks[i].push_back(std::move(read_task)); } diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index a412dbf0811..7a815777125 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -53,12 +53,11 @@ private: using ThreadTasks = std::deque; using ThreadsTasks = std::map; - /// smaller `priority` means more priority std::future createPrefetchedReader( const IMergeTreeDataPart & data_part, const NamesAndTypesList & columns, const MarkRanges & required_ranges, - int64_t priority) const; + Priority priority) const; void createPrefetchedReaderForTask(MergeTreeReadTask & task) const; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 26a7cb2b50b..d22684eaa9d 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -314,7 +314,7 @@ void MergeTreeReaderCompact::readData( last_read_granule.emplace(from_mark, column_position); } -void MergeTreeReaderCompact::prefetchBeginOfRange(int64_t priority) +void MergeTreeReaderCompact::prefetchBeginOfRange(Priority priority) { if (!initialized) { diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index a994e72d3ff..f180d7508f7 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -38,7 +38,7 @@ public: bool canReadIncompleteGranules() const override { return false; } - void prefetchBeginOfRange(int64_t priority) override; + void prefetchBeginOfRange(Priority priority) override; private: bool isContinuousReading(size_t mark, size_t column_position); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 69617fdf9e3..baacfa55c94 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -58,7 +58,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( } } -void MergeTreeReaderWide::prefetchBeginOfRange(int64_t priority) +void MergeTreeReaderWide::prefetchBeginOfRange(Priority priority) { prefetched_streams.clear(); @@ -90,7 +90,7 @@ void MergeTreeReaderWide::prefetchBeginOfRange(int64_t priority) } void MergeTreeReaderWide::prefetchForAllColumns( - int64_t priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading) + Priority priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading) { bool do_prefetch = data_part_info_for_read->getDataPartStorage()->isStoredOnRemoteDisk() ? settings.read_settings.remote_fs_prefetch @@ -137,7 +137,7 @@ size_t MergeTreeReaderWide::readRows( if (num_columns == 0) return max_rows_to_read; - prefetchForAllColumns(/* priority */0, num_columns, from_mark, current_task_last_mark, continue_reading); + prefetchForAllColumns(Priority{}, num_columns, from_mark, current_task_last_mark, continue_reading); for (size_t pos = 0; pos < num_columns; ++pos) { @@ -305,7 +305,7 @@ void MergeTreeReaderWide::deserializePrefix( } void MergeTreeReaderWide::prefetchForColumn( - int64_t priority, + Priority priority, const NameAndTypePair & name_and_type, const SerializationPtr & serialization, size_t from_mark, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 08bb17c5d56..c31b1baf32e 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -33,14 +33,14 @@ public: bool canReadIncompleteGranules() const override { return true; } - void prefetchBeginOfRange(int64_t priority) override; + void prefetchBeginOfRange(Priority priority) override; using FileStreams = std::map>; private: FileStreams streams; - void prefetchForAllColumns(int64_t priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading); + void prefetchForAllColumns(Priority priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading); void addStreams( const NameAndTypePair & name_and_type, @@ -55,7 +55,7 @@ private: /// Make next readData more simple by calling 'prefetch' of all related ReadBuffers (column streams). void prefetchForColumn( - int64_t priority, + Priority priority, const NameAndTypePair & name_and_type, const SerializationPtr & serialization, size_t from_mark, diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index 500327afd61..53695639769 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -84,7 +84,7 @@ struct MergeTreeSource::AsyncReadingState { try { - callback_runner(std::move(job), 0); + callback_runner(std::move(job), Priority{}); } catch (...) { diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 51acb6a3a7d..80a49b2cd1d 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -356,7 +356,7 @@ private: request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); return outcome; - }, 0); + }, Priority{}); } std::mutex mutex; @@ -619,7 +619,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() std::future StorageS3Source::createReaderAsync() { - return create_reader_scheduler([this] { return createReader(); }, 0); + return create_reader_scheduler([this] { return createReader(); }, Priority{}); } StorageS3Source::ReadBufferOrFactory StorageS3Source::createS3ReadBuffer(const String & key, size_t object_size) From 0ca526c6036a3f007d38b1bef6b1db2b341c1f67 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 25 May 2023 13:04:07 +0000 Subject: [PATCH 0673/2223] Unify priorities: rework IO scheduling subsystem --- src/IO/ISchedulerNode.h | 5 +++-- src/IO/Resource/PriorityPolicy.h | 4 ++-- src/IO/Resource/tests/gtest_resource_class_priority.cpp | 8 ++++---- src/IO/Resource/tests/gtest_resource_manager_static.cpp | 6 +++--- 4 files changed, 12 insertions(+), 11 deletions(-) diff --git a/src/IO/ISchedulerNode.h b/src/IO/ISchedulerNode.h index 266f1ba07a0..1c33c033744 100644 --- a/src/IO/ISchedulerNode.h +++ b/src/IO/ISchedulerNode.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -37,7 +38,7 @@ inline const Poco::Util::AbstractConfiguration & emptyConfig() struct SchedulerNodeInfo { double weight = 1.0; /// Weight of this node among it's siblings - Int64 priority = 0; /// Priority of this node among it's siblings (higher value means higher priority) + Priority priority; /// Priority of this node among it's siblings (lower value means higher priority) /// Arbitrary data accessed/stored by parent union { @@ -65,7 +66,7 @@ struct SchedulerNodeInfo void setPriority(Int64 value) { - priority = value; + priority.value = value; } }; diff --git a/src/IO/Resource/PriorityPolicy.h b/src/IO/Resource/PriorityPolicy.h index 961f5af4d27..3c091dcc85a 100644 --- a/src/IO/Resource/PriorityPolicy.h +++ b/src/IO/Resource/PriorityPolicy.h @@ -26,12 +26,12 @@ class PriorityPolicy : public ISchedulerNode struct Item { ISchedulerNode * child = nullptr; - Int64 priority = 0; // higher value means higher priority + Priority priority; // lower value means higher priority /// For max-heap by priority bool operator<(const Item& rhs) const noexcept { - return priority < rhs.priority; + return priority > rhs.priority; // Reversed for heap top to yield highest priority (lowest value) child first } }; diff --git a/src/IO/Resource/tests/gtest_resource_class_priority.cpp b/src/IO/Resource/tests/gtest_resource_class_priority.cpp index 9e1a55a0741..83902453ed4 100644 --- a/src/IO/Resource/tests/gtest_resource_class_priority.cpp +++ b/src/IO/Resource/tests/gtest_resource_class_priority.cpp @@ -22,9 +22,9 @@ TEST(IOResourcePriorityPolicy, Priorities) ResourceTest t; t.add("/"); - t.add("/A", "1"); + t.add("/A", "3"); t.add("/B", "2"); - t.add("/C", "3"); + t.add("/C", "1"); t.enqueue("/A", {10, 10, 10}); t.enqueue("/B", {10, 10, 10}); @@ -56,9 +56,9 @@ TEST(IOResourcePriorityPolicy, Activation) ResourceTest t; t.add("/"); - t.add("/A", "1"); + t.add("/A", "3"); t.add("/B", "2"); - t.add("/C", "3"); + t.add("/C", "1"); t.enqueue("/A", {10, 10, 10, 10, 10, 10}); t.enqueue("/B", {10}); diff --git a/src/IO/Resource/tests/gtest_resource_manager_static.cpp b/src/IO/Resource/tests/gtest_resource_manager_static.cpp index 35bb78bbbe3..091f6923714 100644 --- a/src/IO/Resource/tests/gtest_resource_manager_static.cpp +++ b/src/IO/Resource/tests/gtest_resource_manager_static.cpp @@ -49,7 +49,7 @@ TEST(IOResourceStaticResourceManager, Prioritization) { // Lock is not required here because this is called during request execution and we have max_requests = 1 if (last_priority) - EXPECT_TRUE(priority <= *last_priority); // Should be true if every queue arrived at the same time at busy period start + EXPECT_TRUE(priority >= *last_priority); // Should be true if every queue arrived at the same time at busy period start last_priority = priority; }; @@ -63,8 +63,8 @@ TEST(IOResourceStaticResourceManager, Prioritization) inflight_limit1 priority - -1 - 1 + 1 + -1 From bb77441acb488bb2a32f691ce724b2950ec0d9ba Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 13:31:11 +0000 Subject: [PATCH 0674/2223] Add support for old storage disks --- src/Coordination/KeeperContext.cpp | 30 ++++-- src/Coordination/KeeperContext.h | 2 + src/Coordination/KeeperSnapshotManager.cpp | 118 +++++++++++++-------- src/Coordination/KeeperSnapshotManager.h | 5 +- src/Coordination/KeeperStateMachine.cpp | 2 +- 5 files changed, 103 insertions(+), 54 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index da49868f706..1d6f1be9bfb 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -8,7 +8,6 @@ namespace DB { - KeeperContext::KeeperContext(bool standalone_keeper_) : disk_selector(std::make_shared()) , standalone_keeper(standalone_keeper_) @@ -28,13 +27,19 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) else current_log_storage = log_storage; - Poco::Util::AbstractConfiguration::Keys old_log_disk_name_keys; - config.keys("keeper_server", old_log_disk_name_keys); - for (const auto & key : old_log_disk_name_keys) + const auto collect_old_disk_names = [&](const std::string_view key_prefix, std::vector & disk_names) { - if (key.starts_with("old_log_storage_disk")) - old_log_disk_names.push_back(config.getString("keeper_server." + key)); - } + Poco::Util::AbstractConfiguration::Keys disk_name_keys; + config.keys("keeper_server", disk_name_keys); + for (const auto & key : disk_name_keys) + { + if (key.starts_with(key_prefix)) + disk_names.push_back(config.getString(fmt::format("keeper_server.{}", key_prefix))); + } + }; + + collect_old_disk_names("old_log_storage_disk", old_log_disk_names); + collect_old_disk_names("old_snapshot_storage_disk", old_snapshot_disk_names); snapshot_storage = getSnapshotsPathFromConfig(config); @@ -107,6 +112,17 @@ DiskPtr KeeperContext::getSnapshotDisk() const return getDisk(snapshot_storage); } +std::vector KeeperContext::getOldSnapshotDisks() const +{ + std::vector old_snapshot_disks; + old_snapshot_disks.reserve(old_snapshot_disk_names.size()); + + for (const auto & disk_name : old_snapshot_disk_names) + old_snapshot_disks.push_back(disk_selector->get(disk_name)); + + return old_snapshot_disks; +} + void KeeperContext::setSnapshotDisk(DiskPtr disk) { snapshot_storage = std::move(disk); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 6b7af3a60db..e41d8e35032 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -40,6 +40,7 @@ public: void setLogDisk(DiskPtr disk); DiskPtr getSnapshotDisk() const; + std::vector getOldSnapshotDisks() const; void setSnapshotDisk(DiskPtr disk); DiskPtr getStateFileDisk() const; @@ -69,6 +70,7 @@ private: Storage state_file_storage; std::vector old_log_disk_names; + std::vector old_snapshot_disk_names; bool standalone_keeper; }; diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 3bfe700bcd5..77abbfb2054 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -519,49 +519,79 @@ KeeperSnapshotManager::KeeperSnapshotManager( , storage_tick_time(storage_tick_time_) , keeper_context(keeper_context_) { + const auto load_snapshot_from_disk = [&](DiskPtr disk) + { + std::unordered_set invalid_snapshots; + /// collect invalid snapshots + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) + { + const auto & name = it->name(); + if (name.empty()) + continue; + + if (startsWith(name, "tmp_")) + { + disk->removeFile(it->path()); + invalid_snapshots.insert(name.substr(4)); + continue; + } + } + + /// process snapshots + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) + { + const auto & name = it->name(); + if (name.empty()) + continue; + + /// Not snapshot file + if (!startsWith(name, "snapshot_")) + continue; + + if (invalid_snapshots.contains(name)) + { + disk->removeFile(it->path()); + continue; + } + + size_t snapshot_up_to = getSnapshotPathUpToLogIdx(name); + auto [_, inserted] = existing_snapshots.insert_or_assign(snapshot_up_to, SnapshotFileInfo{it->path(), disk}); + + if (!inserted) + LOG_WARNING( + &Poco::Logger::get("KeeperSnapshotManager"), + "Found another snapshots with last log idx {}, will use snapshot from disk {}", + snapshot_up_to, + disk->getName()); + } + }; + + for (const auto & disk : keeper_context->getOldSnapshotDisks()) + load_snapshot_from_disk(disk); + auto disk = getDisk(); - - std::unordered_set invalid_snapshots; - /// collect invalid snapshots - for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) - { - const auto & name = it->name(); - if (name.empty()) - continue; - - if (startsWith(name, "tmp_")) - { - disk->removeFile(it->path()); - invalid_snapshots.insert(name.substr(4)); - continue; - } - - } - - /// process snapshots - for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) - { - const auto & name = it->name(); - if (name.empty()) - continue; - - /// Not snapshot file - if (!startsWith(name, "snapshot_")) - continue; - - if (invalid_snapshots.contains(name)) - { - disk->removeFile(it->path()); - continue; - } - - size_t snapshot_up_to = getSnapshotPathUpToLogIdx(name); - existing_snapshots[snapshot_up_to] = it->path(); - } + load_snapshot_from_disk(disk); removeOutdatedSnapshotsIfNeeded(); -} + /// move snapshots from old disks to new one + for (auto & [_, file_info] : existing_snapshots) + { + if (file_info.disk == disk) + continue; + + auto file_path = fs::path(file_info.path); + auto tmp_snapshot_path = file_path.parent_path() / ("tmp_" + file_path.filename().generic_string()); + + { + disk->writeFile(tmp_snapshot_path); + } + + file_info.disk->copyFile(file_info.path, *disk, file_info.path, {}); + disk->removeFile(tmp_snapshot_path); + file_info.disk = disk; + } +} SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) { @@ -599,7 +629,8 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff } catch (const DB::Exception &) { - getDisk()->removeFile(latest_itr->second); + const auto & [path, disk] = latest_itr->second; + disk->removeFile(path); existing_snapshots.erase(latest_itr->first); tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -610,9 +641,9 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const { - const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx); + const auto & [snapshot_path, snapshot_disk] = existing_snapshots.at(up_to_log_idx); WriteBufferFromNuraftBuffer writer; - auto reader = getDisk()->readFile(snapshot_path); + auto reader = snapshot_disk->readFile(snapshot_path); copyData(*reader, writer); return writer.getBuffer(); } @@ -690,7 +721,8 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) auto itr = existing_snapshots.find(log_idx); if (itr == existing_snapshots.end()) throw Exception(ErrorCodes::UNKNOWN_SNAPSHOT, "Unknown snapshot with log index {}", log_idx); - getDisk()->removeFile(itr->second); + const auto & [path, disk] = itr->second; + disk->removeFile(path); existing_snapshots.erase(itr); } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 036c0cab62b..0afe582ef59 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -149,11 +149,10 @@ public: { if (!existing_snapshots.empty()) { - const auto & path = existing_snapshots.at(getLatestSnapshotIndex()); + const auto & [path, disk] = existing_snapshots.at(getLatestSnapshotIndex()); try { - auto disk = getDisk(); if (disk->exists(path)) return {path, disk}; } @@ -176,7 +175,7 @@ private: /// How many snapshots to keep before remove const size_t snapshots_to_keep; /// All existing snapshots in our path (log_index -> path) - std::map existing_snapshots; + std::map existing_snapshots; /// Compress snapshots in common ZSTD format instead of custom ClickHouse block LZ4 format const bool compress_snapshots_zstd; /// Superdigest for deserialization of storage diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5cfc9333a66..9a1cb1941b7 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -404,11 +404,11 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res } else { + latest_snapshot_meta = snapshot->snapshot_meta; /// we rely on the fact that the snapshot disk cannot be changed during runtime if (isLocalDisk(*keeper_context->getSnapshotDisk())) { latest_snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot); - latest_snapshot_meta = snapshot->snapshot_meta; latest_snapshot_buf = nullptr; } else From 30ff5113d98141ab4adb05ad08b7ffb48888c33d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 May 2023 13:33:52 +0000 Subject: [PATCH 0675/2223] Fixing more tests. --- src/Functions/in.cpp | 8 +- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/PreparedSets.cpp | 70 +++++++++++------ src/Interpreters/PreparedSets.h | 17 +++-- src/Interpreters/Set.cpp | 76 +++++++++++-------- src/Interpreters/Set.h | 18 ++++- src/Planner/CollectSets.cpp | 2 +- .../CreateSetAndFilterOnTheFlyTransform.cpp | 2 +- src/Storages/KVStorageUtils.cpp | 4 +- src/Storages/MergeTree/KeyCondition.cpp | 8 +- .../MergeTreeIndexConditionBloomFilter.cpp | 2 +- .../MergeTree/MergeTreeIndexFullText.cpp | 2 +- .../MergeTree/MergeTreeIndexInverted.cpp | 2 +- .../System/StorageSystemZooKeeper.cpp | 4 +- 14 files changed, 136 insertions(+), 81 deletions(-) diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index eb623951bf3..6a88a413c63 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -127,12 +127,12 @@ public: } auto future_set = column_set->getData(); - if (!future_set || !future_set->isFilled()) + if (!future_set || !future_set->isReady()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName()); - if (auto * for_tuple = typeid_cast(future_set.get())) - if (!for_tuple->isReady()) - for_tuple->buildForTuple(size_limits, transform_null_in); + // if (auto * for_tuple = typeid_cast(future_set.get())) + // if (!for_tuple->isReady()) + // for_tuple->buildForTuple(size_limits, transform_null_in); auto set = future_set->get(); auto set_types = set->getDataTypes(); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 1405568aa71..da8666466fc 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -458,7 +458,7 @@ FutureSetPtr makeExplicitSet( else block = createBlockForSet(left_arg_type, right_arg, set_element_types, context); - return prepared_sets.addFromTuple(set_key, block); + return prepared_sets.addFromTuple(set_key, block, context->getSettings()); } class ScopeStack::Index diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 3b63d942404..eebc59e85d5 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int SET_SIZE_LIMIT_EXCEEDED; } PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) @@ -120,9 +121,9 @@ FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set return it->second; } -FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block) +FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings) { - auto from_tuple = std::make_shared(std::move(block)); + auto from_tuple = std::make_shared(std::move(block), settings); auto [it, inserted] = sets.emplace(key, std::move(from_tuple)); if (!inserted) @@ -292,7 +293,25 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } -FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} +FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) +{ + bool create_ordered_set = false; + auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); + + Columns columns; + columns.reserve(block.columns()); + for (const auto & column : block) + columns.emplace_back(column.column); + + set_key_columns.filter = ColumnUInt8::create(block.rows()); + + set->initSetElements(); + set->insertFromColumns(columns, set_key_columns); + set->finishInsert(); + //block(std::move(block_)) +} FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} @@ -301,35 +320,40 @@ FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) { SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, true); - fill(size_limits, settings.transform_null_in, true); + auto limits = getSizeLimitsForSet(settings, true); + + if (!limits.check(set->getTotalRowCount(), set->getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED)) + return nullptr; + + set->initSetElements(); + set->appendSetElements(set_key_columns); return set; } -std::unique_ptr FutureSetFromTuple::build(const ContextPtr & context) +std::unique_ptr FutureSetFromTuple::build(const ContextPtr &) { - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, false); - fill(size_limits, settings.transform_null_in, false); + // const auto & settings = context->getSettingsRef(); + // auto size_limits = getSizeLimitsForSet(settings, false); + // fill(size_limits, settings.transform_null_in, false); return nullptr; } -void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) -{ - fill(size_limits, transform_null_in, false); -} +// void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) +// { +// fill(size_limits, transform_null_in, false); +// } -void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) -{ - //std::cerr << StackTrace().toString() << std::endl; +// void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) +// { +// //std::cerr << StackTrace().toString() << std::endl; - if (set) - return; +// if (set) +// return; - set = std::make_shared(size_limits, create_ordered_set, transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(block.getColumnsWithTypeAndName()); - set->finishInsert(); -} +// set = std::make_shared(size_limits, create_ordered_set, transform_null_in); +// set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); +// set->insertFromBlock(block.getColumnsWithTypeAndName()); +// set->finishInsert(); +// } }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index ef7aba38f24..35bbc1e12fb 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -80,24 +80,23 @@ using FutureSetPtr = std::shared_ptr; class FutureSetFromTuple final : public FutureSet { public: - FutureSetFromTuple(Block block_); + FutureSetFromTuple(Block block, const Settings & settings); - bool isReady() const override { return set != nullptr; } + bool isReady() const override { return true; } bool isFilled() const override { return true; } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - std::unique_ptr build(const ContextPtr & context) override; + std::unique_ptr build(const ContextPtr &) override; - void buildForTuple(SizeLimits size_limits, bool transform_null_in); +/// void buildForTuple(SizeLimits size_limits, bool transform_null_in); private: - Block block; - SetPtr set; + Set::SetKeyColumns set_key_columns; - void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); + //void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); }; /// Information on how to build set for the [GLOBAL] IN section. @@ -145,6 +144,8 @@ public: return nullptr; auto plan = buildPlan(context, true); + if (!plan) + return nullptr; auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); @@ -249,7 +250,7 @@ public: // SizeLimits set_size_limit, bool transform_null_in); FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); - FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block); + FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery); void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index a7bea63bd99..5adf3d07353 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -156,25 +156,46 @@ void Set::setHeader(const ColumnsWithTypeAndName & header) { /// Create empty columns with set values in advance. /// It is needed because set may be empty, so method 'insertFromBlock' will be never called. - set_elements.reserve(keys_size); - for (const auto & type : set_elements_types) - set_elements.emplace_back(type->createColumn()); + initSetElements(); } /// Choose data structure to use for the set. data.init(data.chooseMethod(key_columns, key_sizes)); } +void Set::initSetElements() +{ + set_elements.reserve(keys_size); + for (const auto & type : set_elements_types) + set_elements.emplace_back(type->createColumn()); +} + bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns) { Columns cols; cols.reserve(columns.size()); for (const auto & column : columns) cols.emplace_back(column.column); - return insertFromBlock(cols); + return insertFromColumns(cols); } -bool Set::insertFromBlock(const Columns & columns) +bool Set::insertFromColumns(const Columns & columns) +{ + size_t rows = columns.at(0)->size(); + + SetKeyColumns holder; + /// Filter to extract distinct values from the block. + if (fill_set_elements) + holder.filter = ColumnUInt8::create(rows); + + bool inserted = insertFromColumns(columns, holder); + if (inserted && fill_set_elements) + appendSetElements(holder); + + return inserted; +} + +bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) { std::lock_guard lock(rwlock); @@ -183,15 +204,13 @@ bool Set::insertFromBlock(const Columns & columns) ColumnRawPtrs key_columns; key_columns.reserve(keys_size); - - /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - Columns materialized_columns; + holder.key_columns.reserve(keys_size); /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); - key_columns.emplace_back(materialized_columns.back().get()); + holder.key_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); + key_columns.emplace_back(holder.key_columns.back().get()); } size_t rows = columns.at(0)->size(); @@ -202,40 +221,37 @@ bool Set::insertFromBlock(const Columns & columns) if (!transform_null_in) null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); - /// Filter to extract distinct values from the block. - ColumnUInt8::MutablePtr filter; - if (fill_set_elements) - filter = ColumnUInt8::create(rows); - switch (data.type) { case SetVariants::Type::EMPTY: break; #define M(NAME) \ case SetVariants::Type::NAME: \ - insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, filter ? &filter->getData() : nullptr); \ + insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ break; APPLY_FOR_SET_VARIANTS(M) #undef M } - if (fill_set_elements) - { - for (size_t i = 0; i < keys_size; ++i) - { - auto filtered_column = key_columns[i]->filter(filter->getData(), rows); - if (set_elements[i]->empty()) - set_elements[i] = filtered_column; - else - set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size()); - if (transform_null_in && null_map_holder) - set_elements[i]->insert(Null{}); - } - } - return limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); } +void Set::appendSetElements(SetKeyColumns & holder) +{ + //std::cerr << "========= " << keys_size << ' ' << holder.key_columns.size() << std::endl; + size_t rows = holder.key_columns.at(0)->size(); + for (size_t i = 0; i < keys_size; ++i) + { + auto filtered_column = holder.key_columns[i]->filter(holder.filter->getData(), rows); + if (set_elements[i]->empty()) + set_elements[i] = filtered_column; + else + set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size()); + if (transform_null_in && holder.null_map_holder) + set_elements[i]->insert(Null{}); + } +} + void Set::checkIsCreated() const { if (!is_created.load()) diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index fff5fa4e1b1..cb47fde7f7d 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -45,9 +45,23 @@ public: void setHeader(const ColumnsWithTypeAndName & header); /// Returns false, if some limit was exceeded and no need to insert more data. - bool insertFromBlock(const Columns & columns); + bool insertFromColumns(const Columns & columns); bool insertFromBlock(const ColumnsWithTypeAndName & columns); + + struct SetKeyColumns + { + //ColumnRawPtrs key_columns; + /// The constant columns to the right of IN are not supported directly. For this, they first materialize. + Columns key_columns; + ColumnPtr null_map_holder; + ColumnUInt8::MutablePtr filter; + }; + + void initSetElements(); + bool insertFromColumns(const Columns & columns, SetKeyColumns & holder); + void appendSetElements(SetKeyColumns & holder); + /// Call after all blocks were inserted. To get the information that set is already created. void finishInsert() { is_created = true; } @@ -68,7 +82,7 @@ public: const DataTypes & getDataTypes() const { return data_types; } const DataTypes & getElementsTypes() const { return set_elements_types; } - bool hasExplicitSetElements() const { return fill_set_elements; } + bool hasExplicitSetElements() const { return fill_set_elements || (!set_elements.empty() && set_elements.front()->size() == data.getTotalRowCount()); } Columns getSetElements() const { checkIsCreated(); return { set_elements.begin(), set_elements.end() }; } void checkColumnsNumber(size_t num_key_columns) const; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 6c970e0e91b..d66424f89ec 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -82,7 +82,7 @@ public: auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); - sets.addFromTuple(set_key, std::move(set)); + sets.addFromTuple(set_key, std::move(set), settings); //planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); } diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 4278eb8e8b2..59c4b9a6a87 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -106,7 +106,7 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (chunk.getNumRows()) { Columns key_columns = getColumnsByIndices(chunk, key_column_indices); - bool limit_exceeded = !set->insertFromBlock(key_columns); + bool limit_exceeded = !set->insertFromColumns(key_columns); if (limit_exceeded) { auto prev_state = set->state.exchange(SetWithState::State::Suspended); diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 281236e631e..f4a4b6e9af1 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -76,8 +76,8 @@ bool traverseASTFilter( if (!future_set) return false; - if (!future_set->isReady()) - future_set->buildOrderedSetInplace(context); + //if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); auto set = future_set->get(); if (!set) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 4b053a37aa2..7fcc111ced9 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1212,11 +1212,11 @@ bool KeyCondition::tryPrepareSetIndex( // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); - if (!future_set->isReady()) - { + //if (!future_set->isReady()) + //{ // LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); - future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); - } + future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); + //} auto prepared_set = future_set->get(); if (!prepared_set) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 2bd9db12b93..3d0883a1241 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -316,7 +316,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo //std::cerr << "==== Finding set for MergeTreeBF " << bool(future_set) << std::endl; - if (future_set && !future_set->isReady()) + if (future_set) // && !future_set->isReady()) { //std::cerr << "==== not ready, building " << std::endl; future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 5e1d23df3c7..d14ff4fa20b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -625,7 +625,7 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( return false; auto future_set = right_argument.tryGetPreparedSet(data_types); - if (future_set && !future_set->isReady()) + if (future_set) // && !future_set->isReady()) future_set->buildOrderedSetInplace(right_argument.getTreeContext().getQueryContext()); ConstSetPtr prepared_set; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index a64f81807ae..1d3b923056a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -661,7 +661,7 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( //std::cerr << "==== Set for MergeTreeConditionInverted" << bool(future_set) << std::endl; - if (future_set && !future_set->isReady()) + if (future_set) // && !future_set->isReady()) future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); ConstSetPtr prepared_set; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 0f45f6825f6..34b463eadee 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -317,8 +317,8 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP if (!future_set) return; - if (!future_set->isReady()) - future_set->buildOrderedSetInplace(context); + //if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); auto set = future_set->get(); if (!set) From 20287055be4dada6d5eef2bc21004911f2649628 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 25 May 2023 13:43:05 +0000 Subject: [PATCH 0676/2223] Add toLastDayOfWeek function to 02403_enable_extended_results_for_datetime_functions group test --- ...able_extended_results_for_datetime_functions.reference | 8 ++++++++ ..._enable_extended_results_for_datetime_functions.sql.j2 | 2 +- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference index 025191c234a..c830d790000 100644 --- a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference +++ b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.reference @@ -22,6 +22,10 @@ toMonday;toDate32;true 1920-02-02 type;toMonday;toDate32;true Date32 toMonday;toDateTime64;true 1920-02-02 type;toMonday;toDateTime64;true Date32 +toLastDayOfWeek;toDate32;true 1920-02-07 +type;toLastDayOfWeek;toDate32;true Date32 +toLastDayOfWeek;toDateTime64;true 1920-02-07 +type;toLastDayOfWeek;toDateTime64;true Date32 toLastDayOfMonth;toDate32;true 1920-02-29 type;toLastDayOfMonth;toDate32;true Date32 toLastDayOfMonth;toDateTime64;true 1920-02-29 @@ -66,6 +70,10 @@ toMonday;toDate32;false 2099-07-08 type;toMonday;toDate32;false Date toMonday;toDateTime64;false 2099-07-08 type;toMonday;toDateTime64;false Date +toLastDayOfWeek;toDate32;false 2099-07-13 +type;toLastDayOfWeek;toDate32;false Date +toLastDayOfWeek;toDateTime64;false 2099-07-13 +type;toLastDayOfWeek;toDateTime64;false Date toLastDayOfMonth;toDate32;false 2099-08-04 type;toLastDayOfMonth;toDate32;false Date toLastDayOfMonth;toDateTime64;false 2099-08-04 diff --git a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.sql.j2 b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.sql.j2 index 00a1a0030aa..27e950e4fce 100644 --- a/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.sql.j2 +++ b/tests/queries/0_stateless/02403_enable_extended_results_for_datetime_functions.sql.j2 @@ -1,5 +1,5 @@ {% for option_value in ['true', 'false'] -%} -{% for date_fun in ['toStartOfYear', 'toStartOfISOYear', 'toStartOfQuarter', 'toStartOfMonth', 'toStartOfWeek', 'toMonday', 'toLastDayOfMonth'] -%} +{% for date_fun in ['toStartOfYear', 'toStartOfISOYear', 'toStartOfQuarter', 'toStartOfMonth', 'toStartOfWeek', 'toMonday', 'toLastDayOfWeek', 'toLastDayOfMonth'] -%} SELECT '{{ date_fun }};toDate32;{{ option_value }}', {{ date_fun }}(toDate32('1920-02-02')) SETTINGS enable_extended_results_for_datetime_functions = {{ option_value }}; SELECT 'type;{{ date_fun }};toDate32;{{ option_value }}', toTypeName({{ date_fun }}(toDate32('1920-02-02'))) SETTINGS enable_extended_results_for_datetime_functions = {{ option_value }}; SELECT '{{ date_fun }};toDateTime64;{{ option_value }}', {{ date_fun }}(toDateTime64('1920-02-02 10:20:30', 3)) SETTINGS enable_extended_results_for_datetime_functions = {{ option_value }}; From 3fefacbf20d10b274af22cc3ae57260ed78b0f5b Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 25 May 2023 15:46:05 +0200 Subject: [PATCH 0677/2223] Fix --- src/Storages/DataLakes/IcebergMetadataParser.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Storages/DataLakes/IcebergMetadataParser.cpp b/src/Storages/DataLakes/IcebergMetadataParser.cpp index 3ab90e271cf..144dace2fc1 100644 --- a/src/Storages/DataLakes/IcebergMetadataParser.cpp +++ b/src/Storages/DataLakes/IcebergMetadataParser.cpp @@ -244,7 +244,13 @@ struct IcebergMetadataParser::Impl const auto * str_col = assert_cast(col_str.get()); for (size_t i = 0; i < str_col->size(); ++i) - keys.emplace_back(str_col->getDataAt(i).toView()); + { + const auto data_path = std::string(str_col->getDataAt(i).toView()); + const auto pos = data_path.find(configuration.url.key); + if (pos == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration.url.key, data_path); + keys.emplace_back(data_path.substr(pos)); + } } return keys; From 243207aa4e140aba8e708453744e5101b12cf731 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 25 May 2023 16:00:50 +0200 Subject: [PATCH 0678/2223] Update S3 sdk to v1.11.61 (#50037) --- contrib/aws | 2 +- contrib/aws-c-auth | 2 +- contrib/aws-c-common | 2 +- contrib/aws-c-event-stream | 2 +- contrib/aws-c-http | 2 +- contrib/aws-c-io | 2 +- contrib/aws-c-mqtt | 2 +- contrib/aws-c-s3 | 2 +- contrib/aws-c-sdkutils | 2 +- contrib/aws-checksums | 2 +- contrib/aws-cmake/CMakeLists.txt | 6 +++--- contrib/aws-crt-cpp | 2 +- contrib/aws-s2n-tls | 2 +- contrib/sparse-checkout/update-aws.sh | 4 ++-- 14 files changed, 17 insertions(+), 17 deletions(-) diff --git a/contrib/aws b/contrib/aws index ecccfc026a4..ca02358dcc7 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit ecccfc026a42b30023289410a67024d561f4bf3e +Subproject commit ca02358dcc7ce3ab733dd4cbcc32734eecfa4ee3 diff --git a/contrib/aws-c-auth b/contrib/aws-c-auth index 30df6c407e2..97133a2b5db 160000 --- a/contrib/aws-c-auth +++ b/contrib/aws-c-auth @@ -1 +1 @@ -Subproject commit 30df6c407e2df43bd244e2c34c9b4a4b87372bfb +Subproject commit 97133a2b5dbca1ccdf88cd6f44f39d0531d27d12 diff --git a/contrib/aws-c-common b/contrib/aws-c-common index 324fd1d973c..45dcb2849c8 160000 --- a/contrib/aws-c-common +++ b/contrib/aws-c-common @@ -1 +1 @@ -Subproject commit 324fd1d973ccb25c813aa747bf1759cfde5121c5 +Subproject commit 45dcb2849c891dba2100b270b4676765c92949ff diff --git a/contrib/aws-c-event-stream b/contrib/aws-c-event-stream index 39bfa94a14b..2f9b60c42f9 160000 --- a/contrib/aws-c-event-stream +++ b/contrib/aws-c-event-stream @@ -1 +1 @@ -Subproject commit 39bfa94a14b7126bf0c1330286ef8db452d87e66 +Subproject commit 2f9b60c42f90840ec11822acda3d8cdfa97a773d diff --git a/contrib/aws-c-http b/contrib/aws-c-http index 2c5a2a7d555..dd344619879 160000 --- a/contrib/aws-c-http +++ b/contrib/aws-c-http @@ -1 +1 @@ -Subproject commit 2c5a2a7d5556600b9782ffa6c9d7e09964df1abc +Subproject commit dd34461987947672444d0bc872c5a733dfdb9711 diff --git a/contrib/aws-c-io b/contrib/aws-c-io index 5d32c453560..d58ed4f272b 160000 --- a/contrib/aws-c-io +++ b/contrib/aws-c-io @@ -1 +1 @@ -Subproject commit 5d32c453560d0823df521a686bf7fbacde7f9be3 +Subproject commit d58ed4f272b1cb4f89ac9196526ceebe5f2b0d89 diff --git a/contrib/aws-c-mqtt b/contrib/aws-c-mqtt index 882c689561a..33c3455cec8 160000 --- a/contrib/aws-c-mqtt +++ b/contrib/aws-c-mqtt @@ -1 +1 @@ -Subproject commit 882c689561a3db1466330ccfe3b63637e0a575d3 +Subproject commit 33c3455cec82b16feb940e12006cefd7b3ef4194 diff --git a/contrib/aws-c-s3 b/contrib/aws-c-s3 index a41255ece72..d7bfe602d69 160000 --- a/contrib/aws-c-s3 +++ b/contrib/aws-c-s3 @@ -1 +1 @@ -Subproject commit a41255ece72a7c887bba7f9d998ca3e14f4c8a1b +Subproject commit d7bfe602d6925948f1fff95784e3613cca6a3900 diff --git a/contrib/aws-c-sdkutils b/contrib/aws-c-sdkutils index 25bf5cf225f..208a701fa01 160000 --- a/contrib/aws-c-sdkutils +++ b/contrib/aws-c-sdkutils @@ -1 +1 @@ -Subproject commit 25bf5cf225f977c3accc6a05a0a7a181ef2a4a30 +Subproject commit 208a701fa01e99c7c8cc3dcebc8317da71362972 diff --git a/contrib/aws-checksums b/contrib/aws-checksums index 48e7c0e0147..ad53be196a2 160000 --- a/contrib/aws-checksums +++ b/contrib/aws-checksums @@ -1 +1 @@ -Subproject commit 48e7c0e01479232f225c8044d76c84e74192889d +Subproject commit ad53be196a25bbefa3700a01187fdce573a7d2d0 diff --git a/contrib/aws-cmake/CMakeLists.txt b/contrib/aws-cmake/CMakeLists.txt index 52533cd6483..950a0e06cd0 100644 --- a/contrib/aws-cmake/CMakeLists.txt +++ b/contrib/aws-cmake/CMakeLists.txt @@ -52,8 +52,8 @@ endif() # Directories. SET(AWS_SDK_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws") -SET(AWS_SDK_CORE_DIR "${AWS_SDK_DIR}/aws-cpp-sdk-core") -SET(AWS_SDK_S3_DIR "${AWS_SDK_DIR}/aws-cpp-sdk-s3") +SET(AWS_SDK_CORE_DIR "${AWS_SDK_DIR}/src/aws-cpp-sdk-core") +SET(AWS_SDK_S3_DIR "${AWS_SDK_DIR}/generated/src/aws-cpp-sdk-s3") SET(AWS_AUTH_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-auth") SET(AWS_CAL_DIR "${ClickHouse_SOURCE_DIR}/contrib/aws-c-cal") @@ -118,7 +118,7 @@ configure_file("${AWS_SDK_CORE_DIR}/include/aws/core/SDKConfig.h.in" list(APPEND AWS_PUBLIC_COMPILE_DEFS "-DAWS_SDK_VERSION_MAJOR=1") list(APPEND AWS_PUBLIC_COMPILE_DEFS "-DAWS_SDK_VERSION_MINOR=10") list(APPEND AWS_PUBLIC_COMPILE_DEFS "-DAWS_SDK_VERSION_PATCH=36") - + list(APPEND AWS_SOURCES ${AWS_SDK_CORE_SRC} ${AWS_SDK_CORE_NET_SRC} ${AWS_SDK_CORE_PLATFORM_SRC}) list(APPEND AWS_PUBLIC_INCLUDES diff --git a/contrib/aws-crt-cpp b/contrib/aws-crt-cpp index ec0bea288f4..8a301b7e842 160000 --- a/contrib/aws-crt-cpp +++ b/contrib/aws-crt-cpp @@ -1 +1 @@ -Subproject commit ec0bea288f451d884c0d80d534bc5c66241c39a4 +Subproject commit 8a301b7e842f1daed478090c869207300972379f diff --git a/contrib/aws-s2n-tls b/contrib/aws-s2n-tls index 0f1ba9e5c4a..71f4794b758 160000 --- a/contrib/aws-s2n-tls +++ b/contrib/aws-s2n-tls @@ -1 +1 @@ -Subproject commit 0f1ba9e5c4a67cb3898de0c0b4f911d4194dc8de +Subproject commit 71f4794b7580cf780eb4aca77d69eded5d3c7bb4 diff --git a/contrib/sparse-checkout/update-aws.sh b/contrib/sparse-checkout/update-aws.sh index c8d4c5a89c2..f86acb54d95 100755 --- a/contrib/sparse-checkout/update-aws.sh +++ b/contrib/sparse-checkout/update-aws.sh @@ -5,8 +5,8 @@ echo "Using sparse checkout for aws" FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout echo '/*' > $FILES_TO_CHECKOUT echo '!/*/*' >> $FILES_TO_CHECKOUT -echo '/aws-cpp-sdk-core/*' >> $FILES_TO_CHECKOUT -echo '/aws-cpp-sdk-s3/*' >> $FILES_TO_CHECKOUT +echo '/src/aws-cpp-sdk-core/*' >> $FILES_TO_CHECKOUT +echo '/generated/src/aws-cpp-sdk-s3/*' >> $FILES_TO_CHECKOUT git config core.sparsecheckout true git checkout $1 From 0580859e6fa70102d3cde058040c4722d51170fc Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 May 2023 14:05:44 +0000 Subject: [PATCH 0679/2223] Better --- src/Parsers/TokenIterator.cpp | 4 ++-- src/Parsers/TokenIterator.h | 2 +- src/Parsers/parseQuery.cpp | 4 ++-- src/Parsers/parseQuery.h | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index 6b798f6f576..fa792e7c8b5 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -4,7 +4,7 @@ namespace DB { -Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool skipp_insignificant) +Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool skip_insignificant) { Lexer lexer(begin, end, max_query_size); @@ -13,7 +13,7 @@ Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool { Token token = lexer.nextToken(); stop = token.isEnd() || token.type == TokenType::ErrorMaxQuerySizeExceeded; - if (token.isSignificant() || (!skipp_insignificant && !data.empty() && data.back().isSignificant())) + if (token.isSignificant() || (!skip_insignificant && !data.empty() && data.back().isSignificant())) data.emplace_back(std::move(token)); } while (!stop); } diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index 31cb644d879..192f2f55e6a 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -24,7 +24,7 @@ private: std::size_t last_accessed_index = 0; public: - Tokens(const char * begin, const char * end, size_t max_query_size = 0, bool skipp_insignificant = true); + Tokens(const char * begin, const char * end, size_t max_query_size = 0, bool skip_insignificant = true); ALWAYS_INLINE inline const Token & operator[](size_t index) { diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 9f688f204a2..dd9a6023b0b 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -234,10 +234,10 @@ ASTPtr tryParseQuery( bool allow_multi_statements, size_t max_query_size, size_t max_parser_depth, - bool skipp_insignificant) + bool skip_insignificant) { const char * query_begin = _out_query_end; - Tokens tokens(query_begin, all_queries_end, max_query_size, skipp_insignificant); + Tokens tokens(query_begin, all_queries_end, max_query_size, skip_insignificant); /// NOTE: consider use UInt32 for max_parser_depth setting. IParser::Pos token_iterator(tokens, static_cast(max_parser_depth)); diff --git a/src/Parsers/parseQuery.h b/src/Parsers/parseQuery.h index 30f43261103..a087f145d2c 100644 --- a/src/Parsers/parseQuery.h +++ b/src/Parsers/parseQuery.h @@ -19,7 +19,7 @@ ASTPtr tryParseQuery( size_t max_query_size, /// If (end - pos) > max_query_size and query is longer than max_query_size then throws "Max query size exceeded". /// Disabled if zero. Is used in order to check query size if buffer can contains data for INSERT query. size_t max_parser_depth, - bool skipp_insignificant = true); /// If true, lexer will skip all insignificant tokens (e.g. whitespaces) + bool skip_insignificant = true); /// If true, lexer will skip all insignificant tokens (e.g. whitespaces) /// Parse query or throw an exception with error message. From e94b0c8e5e30451d2ab29fd3365bc8285bbec40e Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 May 2023 16:38:19 +0200 Subject: [PATCH 0680/2223] Fix bug --- src/Disks/DiskEncryptedTransaction.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 7534abe59a8..7797e11bf39 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -88,9 +88,10 @@ std::unique_ptr DiskEncryptedTransaction::writeFile( // FileEncryption::Header header; String key; UInt64 old_file_size = 0; - if (mode == WriteMode::Append && delegate_disk->exists(path)) + if (mode == WriteMode::Append && delegate_disk->exists(wrapped_path)) { - old_file_size = delegate_disk->getFileSize(path); + size_t size = delegate_disk->getFileSize(wrapped_path); + old_file_size = size > FileEncryption::Header::kSize ? (size - FileEncryption::Header::kSize) : 0; if (old_file_size) { /// Append mode: we continue to use the same header. From ea395e9554e29f5eaa73d9fcd632f87aa4371d42 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 May 2023 15:24:02 +0000 Subject: [PATCH 0681/2223] Make better --- src/Formats/CapnProtoSerializer.cpp | 1152 ++++++++++++++++++--------- src/Formats/CapnProtoSerializer.h | 1 + 2 files changed, 757 insertions(+), 396 deletions(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index ff3880976c7..91e207a1846 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -77,7 +76,7 @@ namespace struct ListBuilder : public FieldBuilder { - explicit ListBuilder(capnp::DynamicValue::Builder builder) : impl(builder.as()) + explicit ListBuilder(capnp::DynamicValue::Builder builder, UInt32 elements_size) : impl(builder.as()), nested_builders(elements_size) { } @@ -87,10 +86,6 @@ namespace struct StructBuilder : public FieldBuilder { - explicit StructBuilder(capnp::DynamicValue::Builder builder, size_t fields_size) : impl(builder.as()), field_builders(fields_size) - { - } - explicit StructBuilder(capnp::DynamicStruct::Builder struct_builder, size_t fields_size) : impl(std::move(struct_builder)), field_builders(fields_size) { } @@ -99,136 +94,144 @@ namespace std::vector> field_builders; }; - std::unique_ptr initStructFieldBuilderIfNeeded(const ColumnPtr & column, size_t row_num, capnp::DynamicStruct::Builder & struct_builder, const capnp::StructSchema::Field & field, const capnp::Type & capnp_type, size_t nested_fields_size) - { - switch (capnp_type.which()) - { - case capnp::schema::Type::LIST: - { - const auto * array_column = assert_cast(column.get()); - size_t size = array_column->getOffsets()[row_num] - array_column->getOffsets()[row_num - 1]; - return std::make_unique(struct_builder.init(field, static_cast(size))); - } - case capnp::schema::Type::STRUCT: - { - return std::make_unique(struct_builder.init(field), nested_fields_size); - } - default: - return nullptr; - } - } - class ICapnProtoSerializer { public: - virtual std::optional writeRow(const ColumnPtr & column, FieldBuilder * builder, size_t row_num) = 0; - virtual void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) = 0; + virtual void writeRow( + const ColumnPtr & column, + std::unique_ptr & builder, + capnp::DynamicStruct::Builder & parent_struct_builder, + UInt32 slot_offset, + size_t row_num) = 0; + + virtual void writeRow( + const ColumnPtr & column, + std::unique_ptr & builder, + capnp::DynamicList::Builder & parent_list_builder, + UInt32 array_index, + size_t row_num) = 0; + + virtual void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) = 0; + + virtual void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) = 0; virtual ~ICapnProtoSerializer() = default; }; - template + template class CapnProtoIntegerSerializer : public ICapnProtoSerializer { public: - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::INT) - return capnp::DynamicValue::Reader(column->getInt(row_num)); - if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::UINT) - return capnp::DynamicValue::Reader(column->getUInt(row_num)); - return capnp::DynamicValue::Reader(column->getBool(row_num)); + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + CapnProtoNumericType value = static_cast(assert_cast &>(*column).getElement(row_num)); + builder_impl.setDataField(slot_offset, value); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - NumericType value; - if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::INT) - value = static_cast(reader.as()); - else if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::UINT) - value = static_cast(reader.as()); - else if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::BOOL) - value = static_cast(reader.as()); + auto & builder_impl = parent_list_builder.getBuilderImpl(); + CapnProtoNumericType value = static_cast(assert_cast &>(*column).getElement(row_num)); + builder_impl.setDataElement(array_index, value); + } - if constexpr (is_bool_data_type) + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + CapnProtoNumericType value = reader_impl.getDataField(slot_offset); + if constexpr (convert_to_bool_on_read) assert_cast(column).insertValue(static_cast(value)); else - assert_cast &>(column).insertValue(value); + assert_cast &>(column).insertValue(static_cast(value)); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + CapnProtoNumericType value = reader_impl.getDataElement(array_index); + if constexpr (convert_to_bool_on_read) + assert_cast(column).insertValue(static_cast(value)); + else + assert_cast &>(column).insertValue(static_cast(value)); } }; - template + template std::unique_ptr createIntegerSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) { switch (capnp_type.which()) { - case capnp::schema::Type::INT8: [[fallthrough]]; - case capnp::schema::Type::INT16: [[fallthrough]]; - case capnp::schema::Type::INT32: [[fallthrough]]; + case capnp::schema::Type::INT8: + return std::make_unique>(); + case capnp::schema::Type::INT16: + return std::make_unique>(); + case capnp::schema::Type::INT32: + return std::make_unique>(); case capnp::schema::Type::INT64: - return std::make_unique>(); - case capnp::schema::Type::UINT8: [[fallthrough]]; - case capnp::schema::Type::UINT16: [[fallthrough]]; - case capnp::schema::Type::UINT32: [[fallthrough]]; + return std::make_unique>(); + case capnp::schema::Type::UINT8: + return std::make_unique>(); + case capnp::schema::Type::UINT16: + return std::make_unique>(); + case capnp::schema::Type::UINT32: + return std::make_unique>(); case capnp::schema::Type::UINT64: - return std::make_unique>(); + return std::make_unique>(); case capnp::schema::Type::BOOL: - return std::make_unique>(); + return std::make_unique>(); default: throwCannotConvert(data_type, column_name, capnp_type); } } - template - class CapnProtoBigIntegerSerializer : public ICapnProtoSerializer - { - public: - CapnProtoBigIntegerSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) - { - if (!capnp_type.isData()) - throwCannotConvert(data_type, column_name, capnp_type); - } - - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override - { - auto data = column->getDataAt(row_num); - return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); - } - - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override - { - auto value = reader.as(); - if (value.size() != sizeof(NumericType)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); - - column.insertData(reinterpret_cast(value.begin()), value.size()); - } - - private: - DataTypePtr data_type; - }; - - template + template class CapnProtoFloatSerializer : public ICapnProtoSerializer { public: - CapnProtoFloatSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - if (!capnp_type.isFloat32() && !capnp_type.isFloat64()) - throwCannotConvert(data_type, column_name, capnp_type); + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + CapnProtoFloatType value = static_cast(assert_cast &>(*column).getElement(row_num)); + builder_impl.setDataField(slot_offset, value); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - return capnp::DynamicValue::Reader(column->getFloat64(row_num)); + auto & builder_impl = parent_list_builder.getBuilderImpl(); + CapnProtoFloatType value = static_cast(assert_cast &>(*column).getElement(row_num)); + builder_impl.setDataElement(array_index, value); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - assert_cast &>(column).insertValue(reader.as()); + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + CapnProtoFloatType value = reader_impl.getDataField(slot_offset); + assert_cast &>(column).insertValue(static_cast(value)); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + CapnProtoFloatType value = reader_impl.getDataElement(array_index); + assert_cast &>(column).insertValue(static_cast(value)); } }; + template + std::unique_ptr createFloatSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + switch (capnp_type.which()) + { + case capnp::schema::Type::FLOAT32: + return std::make_unique>(); + case capnp::schema::Type::FLOAT64: + return std::make_unique>(); + default: + throwCannotConvert(data_type, column_name, capnp_type); + } + } + template class CapnProtoEnumSerializer : public ICapnProtoSerializer { @@ -267,86 +270,90 @@ namespace } else { - auto names = enum_values.getSetOfAllNames(to_lower); - std::unordered_set capn_enum_names; - - for (auto enumerant : enumerants) - { - String name = enumerant.getProto().getName(); - capn_enum_names.insert(to_lower ? boost::algorithm::to_lower_copy(name) : name); - } - - if (names != capn_enum_names) + auto all_values = enum_values.getValues(); + if (all_values.size() != enumerants.size()) throw Exception( ErrorCodes::CAPN_PROTO_BAD_CAST, "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"); + + std::unordered_map ch_name_to_value; + for (auto & [name, value] : all_values) + ch_name_to_value[to_lower ? boost::algorithm::to_lower_copy(name) : name] = value; + + for (auto enumerant : enumerants) + { + String capnp_name = enumerant.getProto().getName(); + UInt16 capnp_value = enumerant.getOrdinal(); + auto it = ch_name_to_value.find(to_lower ? boost::algorithm::to_lower_copy(capnp_name) : capnp_name); + if (it == ch_name_to_value.end()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"); + + ch_to_capnp_values[it->second] = capnp_value; + capnp_to_ch_values[capnp_value] = it->second; + } } } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - const auto * enum_data_type = assert_cast *>(data_type.get()); + auto & builder_impl = parent_struct_builder.getBuilderImpl(); EnumType enum_value = assert_cast &>(*column).getElement(row_num); + UInt16 capnp_value; if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) - return capnp::DynamicValue::Reader(capnp::DynamicEnum(enum_schema, enum_value)); + capnp_value = static_cast(enum_value); + else + capnp_value = ch_to_capnp_values[enum_value]; - auto enum_name = enum_data_type->getNameForValue(enum_value); - for (const auto enumerant : enum_schema.getEnumerants()) - { - if (compareEnumNames(String(enum_name), enumerant.getProto().getName(), enum_comparing_mode)) - return capnp::DynamicValue::Reader(capnp::DynamicEnum(enumerant)); - } - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert ClickHouse Enum value to CapnProto Enum"); + builder_impl.setDataField(slot_offset, capnp_value); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto enum_value = reader.as(); - auto enumerant = *kj::_::readMaybe(enum_value.getEnumerant()); - auto enum_type = assert_cast *>(data_type.get()); - DataTypePtr nested_type = std::make_shared>(); - switch (enum_comparing_mode) - { - case FormatSettings::CapnProtoEnumComparingMode::BY_VALUES: - { - assert_cast &>(column).insertValue(static_cast(enumerant.getOrdinal())); - return; - } - case FormatSettings::CapnProtoEnumComparingMode::BY_NAMES: - { - auto value = enum_type->getValue(String(enumerant.getProto().getName())); - assert_cast &>(column).insertValue(value); - return; - } - case FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE: - { - /// Find the same enum name case insensitive. - String enum_name = enumerant.getProto().getName(); - for (auto & name : enum_type->getAllRegisteredNames()) - { - if (compareEnumNames(name, enum_name, enum_comparing_mode)) - { - assert_cast &>(column).insertValue(enum_type->getValue(name)); - break; - } - } - return; - } - } + auto & builder_impl = parent_list_builder.getBuilderImpl(); + EnumType enum_value = assert_cast &>(*column).getElement(row_num); + UInt16 capnp_value; + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + capnp_value = static_cast(enum_value); + else + capnp_value = ch_to_capnp_values[enum_value]; + + builder_impl.setDataElement(array_index, capnp_value); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + UInt16 capnp_value = reader_impl.getDataField(slot_offset); + EnumType value; + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + value = static_cast(capnp_value); + else + value = capnp_to_ch_values[capnp_value]; + + assert_cast &>(column).insertValue(value); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + UInt16 capnp_value = reader_impl.getDataElement(array_index); + EnumType value; + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + value = static_cast(capnp_value); + else + value = capnp_to_ch_values[capnp_value]; + + assert_cast &>(column).insertValue(value); } private: - bool compareEnumNames(const String & first, const String & second, const FormatSettings::CapnProtoEnumComparingMode mode) - { - if (mode == FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE) - return boost::algorithm::to_lower_copy(first) == boost::algorithm::to_lower_copy(second); - return first == second; - } - DataTypePtr data_type; capnp::EnumSchema enum_schema; const FormatSettings::CapnProtoEnumComparingMode enum_comparing_mode; + std::unordered_map ch_to_capnp_values; + std::unordered_map capnp_to_ch_values; }; class CapnProtoDateSerializer : public ICapnProtoSerializer @@ -358,14 +365,32 @@ namespace throwCannotConvert(data_type, column_name, capnp_type); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - return capnp::DynamicValue::Reader(column->getUInt(row_num)); + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + UInt16 value = assert_cast(*column).getElement(row_num); + builder_impl.setDataField(slot_offset, value); + } + + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override + { + auto & builder_impl = parent_list_builder.getBuilderImpl(); + UInt16 value = assert_cast(*column).getElement(row_num); + builder_impl.setDataElement(array_index, value); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - assert_cast(column).insertValue(reader.as()); + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + UInt16 value = reader_impl.getDataField(slot_offset); + assert_cast(column).insertValue(value); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + UInt16 value = reader_impl.getDataElement(array_index); + assert_cast(column).insertValue(value); } }; @@ -378,14 +403,32 @@ namespace throwCannotConvert(data_type, column_name, capnp_type); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - return capnp::DynamicValue::Reader(column->getInt(row_num)); + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + Int32 value = assert_cast(*column).getElement(row_num); + builder_impl.setDataField(slot_offset, value); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - assert_cast(column).insertValue(reader.as()); + auto & builder_impl = parent_list_builder.getBuilderImpl(); + Int32 value = assert_cast(*column).getElement(row_num); + builder_impl.setDataElement(array_index, value); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + Int32 value = reader_impl.getDataField(slot_offset); + assert_cast(column).insertValue(value); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + Int32 value = reader_impl.getDataElement(array_index); + assert_cast(column).insertValue(value); } }; @@ -398,14 +441,32 @@ namespace throwCannotConvert(data_type, column_name, capnp_type); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - return capnp::DynamicValue::Reader(column->getInt(row_num)); + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + UInt32 value = assert_cast(*column).getElement(row_num); + builder_impl.setDataField(slot_offset, value); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - assert_cast(column).insertValue(reader.as()); + auto & builder_impl = parent_list_builder.getBuilderImpl(); + UInt32 value = assert_cast(*column).getElement(row_num); + builder_impl.setDataElement(array_index, value); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + UInt32 value = reader_impl.getDataField(slot_offset); + assert_cast(column).insertValue(value); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + UInt32 value = reader_impl.getDataElement(array_index); + assert_cast(column).insertValue(value); } }; @@ -418,14 +479,32 @@ namespace throwCannotConvert(data_type, column_name, capnp_type); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - return capnp::DynamicValue::Reader(column->getInt(row_num)); + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + Int64 value = assert_cast(*column).getElement(row_num); + builder_impl.setDataField(slot_offset, value); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - assert_cast(column).insertValue(reader.as()); + auto & builder_impl = parent_list_builder.getBuilderImpl(); + Int64 value = assert_cast(*column).getElement(row_num); + builder_impl.setDataElement(array_index, value); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + Int64 value = reader_impl.getDataField(slot_offset); + assert_cast(column).insertValue(value); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + Int64 value = reader_impl.getDataElement(array_index); + assert_cast(column).insertValue(value); } }; @@ -433,6 +512,8 @@ namespace class CapnProtoDecimalSerializer : public ICapnProtoSerializer { public: + using NativeType = typename DecimalType::NativeType; + CapnProtoDecimalSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) { auto which = WhichDataType(data_type); @@ -440,37 +521,79 @@ namespace throwCannotConvert(data_type, column_name, capnp_type); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - return capnp::DynamicValue::Reader(column->getInt(row_num)); + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + DecimalType value = assert_cast &>(*column).getElement(row_num); + builder_impl.setDataField(slot_offset, value); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - assert_cast &>(column).insertValue(reader.as()); + auto & builder_impl = parent_list_builder.getBuilderImpl(); + DecimalType value = assert_cast &>(*column).getElement(row_num); + builder_impl.setDataElement(array_index, value); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + NativeType value = reader_impl.getDataField(slot_offset); + assert_cast &>(column).insertValue(value); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + NativeType value = reader_impl.getDataElement(array_index); + assert_cast &>(column).insertValue(value); } }; - template - class CapnProtoBigDecimalSerializer : public ICapnProtoSerializer + template + class CapnProtoFixedSizeRawDataSerializer : public ICapnProtoSerializer { + private: + static constexpr size_t value_size = sizeof(T); + public: - CapnProtoBigDecimalSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) + CapnProtoFixedSizeRawDataSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) { if (!capnp_type.isData()) throwCannotConvert(data_type, column_name, capnp_type); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { + auto & builder_impl = parent_struct_builder.getBuilderImpl(); auto data = column->getDataAt(row_num); - return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); + builder_impl.getPointerField(slot_offset).template setBlob(value); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_struct_builder, UInt32 array_index, size_t row_num) override { - auto value = reader.as(); - if (value.size() != sizeof(DecimalType)) + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + auto data = column->getDataAt(row_num); + capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); + builder_impl.getPointerElement(array_index).setBlob(value); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + capnp::Data::Reader value = reader_impl.getPointerField(slot_offset).template getBlob(nullptr, 0); + if (value.size() != value_size) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + capnp::Data::Reader value = reader_impl.getPointerElement(array_index).getBlob(nullptr, 0); + if (value.size() != value_size) throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); column.insertData(reinterpret_cast(value.begin()), value.size()); @@ -484,39 +607,73 @@ namespace class CapnProtoStringSerializer : public ICapnProtoSerializer { public: - CapnProtoStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type_) : capnp_type(capnp_type_) + CapnProtoStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) { if (!capnp_type.isData() && !capnp_type.isText()) throwCannotConvert(data_type, column_name, capnp_type); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { + auto & builder_impl = parent_struct_builder.getBuilderImpl(); auto data = column->getDataAt(row_num); - - if constexpr (is_binary) - return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); - - /// For type TEXT data must be null-terminated, but in String column we always have 0 byte at the end of each value. - return capnp::DynamicValue::Reader(capnp::Text::Reader(data.data, data.size)); - } - - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override - { if constexpr (is_binary) { - auto value = reader.as(); + capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); + builder_impl.getPointerField(slot_offset).setBlob(value); + } + else + { + capnp::Text::Reader value = capnp::Text::Reader(data.data, data.size); + builder_impl.getPointerField(slot_offset).setBlob(value); + } + } + + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_struct_builder, UInt32 array_index, size_t row_num) override + { + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + auto data = column->getDataAt(row_num); + if constexpr (is_binary) + { + capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); + builder_impl.getPointerElement(array_index).setBlob(value); + } + else + { + capnp::Text::Reader value = capnp::Text::Reader(data.data, data.size); + builder_impl.getPointerElement(array_index).setBlob(value); + } + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + if constexpr (is_binary) + { + capnp::Data::Reader value = reader_impl.getPointerField(slot_offset).getBlob(nullptr, 0); column.insertData(reinterpret_cast(value.begin()), value.size()); } else { - auto value = reader.as(); + capnp::Text::Reader value = reader_impl.getPointerField(slot_offset).getBlob(nullptr, 0); column.insertData(reinterpret_cast(value.begin()), value.size()); } } - private: - capnp::Type capnp_type; + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + if constexpr (is_binary) + { + capnp::Data::Reader value = reader_impl.getPointerElement(array_index).getBlob(nullptr, 0); + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + else + { + capnp::Text::Reader value = reader_impl.getPointerElement(array_index).getBlob(nullptr, 0); + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + } }; template @@ -529,29 +686,71 @@ namespace throwCannotConvert(data_type, column_name, capnp_type); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { + auto & builder_impl = parent_struct_builder.getBuilderImpl(); auto data = column->getDataAt(row_num); if constexpr (is_binary) - return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); - - if (data.data[data.size - 1] == 0) - return capnp::DynamicValue::Reader(capnp::Text::Reader(reinterpret_cast(data.data), data.size)); - - /// In TEXT type data should be null-terminated, but ClickHouse FixedString data could not be. - /// To make data null-terminated we should copy it to temporary String object and use it in capnp::Text::Reader. - /// Note that capnp::Text::Reader works only with pointer to the data and it's size, so we should - /// guarantee that new String object life time is longer than capnp::Text::Reader life time. - tmp_string = data.toString(); - return capnp::DynamicValue::Reader(capnp::Text::Reader(reinterpret_cast(tmp_string.data()), tmp_string.size())); + { + capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); + builder_impl.getPointerField(slot_offset).setBlob(value); + } + else + { + if (data.data[data.size - 1] == 0) + { + capnp::Text::Reader value = capnp::Text::Reader(data.data, data.size); + builder_impl.getPointerField(slot_offset).setBlob(value); + } + else + { + /// In TEXT type data should be null-terminated, but ClickHouse FixedString data could not be. + /// To make data null-terminated we should copy it to temporary String object and use it in capnp::Text::Reader. + /// Note that capnp::Text::Reader works only with pointer to the data and it's size, so we should + /// guarantee that new String object life time is longer than capnp::Text::Reader life time. + tmp_string = data.toString(); + capnp::Text::Reader value = capnp::Text::Reader(tmp_string.data(), tmp_string.size()); + builder_impl.getPointerField(slot_offset).setBlob(value); + } + } } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_struct_builder, UInt32 array_index, size_t row_num) override { + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + auto data = column->getDataAt(row_num); + if constexpr (is_binary) + { + capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); + builder_impl.getPointerElement(array_index).setBlob(value); + } + else + { + if (data.data[data.size - 1] == 0) + { + capnp::Text::Reader value = capnp::Text::Reader(data.data, data.size); + builder_impl.getPointerElement(array_index).setBlob(value); + } + else + { + /// In TEXT type data should be null-terminated, but ClickHouse FixedString data could not be. + /// To make data null-terminated we should copy it to temporary String object and use it in capnp::Text::Reader. + /// Note that capnp::Text::Reader works only with pointer to the data and it's size, so we should + /// guarantee that new String object life time is longer than capnp::Text::Reader life time. + tmp_string = data.toString(); + capnp::Text::Reader value = capnp::Text::Reader(tmp_string.data(), tmp_string.size()); + builder_impl.getPointerElement(array_index).setBlob(value); + } + } + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + const auto & reader_impl = parent_struct_reader.getReaderImpl(); auto & fixed_string_column = assert_cast(column); if constexpr (is_binary) { - auto value = reader.as(); + capnp::Data::Reader value = reader_impl.getPointerField(slot_offset).getBlob(nullptr, 0); if (value.size() > fixed_string_column.getN()) throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); @@ -559,7 +758,29 @@ namespace } else { - auto value = reader.as(); + capnp::Text::Reader value = reader_impl.getPointerField(slot_offset).getBlob(nullptr, 0); + if (value.size() > fixed_string_column.getN()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); + + fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); + } + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + auto & fixed_string_column = assert_cast(column); + if constexpr (is_binary) + { + capnp::Data::Reader value = reader_impl.getPointerElement(array_index).getBlob(nullptr, 0); + if (value.size() > fixed_string_column.getN()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); + + fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); + } + else + { + capnp::Text::Reader value = reader_impl.getPointerElement(array_index).getBlob(nullptr, 0); if (value.size() > fixed_string_column.getN()) throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); @@ -581,64 +802,32 @@ namespace throwCannotConvert(data_type, column_name, capnp_type); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - return capnp::DynamicValue::Reader(assert_cast(*column).getElement(row_num).toUnderType()); + auto & builder_impl = parent_struct_builder.getBuilderImpl(); + UInt32 value = assert_cast(*column).getElement(row_num); + builder_impl.setDataField(slot_offset, value); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - assert_cast(column).insertValue(IPv4(reader.as())); - } - }; - - class CapnProtoIPv6Serializer : public ICapnProtoSerializer - { - public: - CapnProtoIPv6Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) - { - if (!capnp_type.isData()) - throwCannotConvert(data_type, column_name, capnp_type); + auto & builder_impl = parent_list_builder.getBuilderImpl(); + UInt32 value = assert_cast(*column).getElement(row_num); + builder_impl.setDataElement(array_index, value); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - auto data = column->getDataAt(row_num); - return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + UInt32 value = reader_impl.getDataField(slot_offset); + assert_cast(column).insertValue(IPv4(value)); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - auto value = reader.as(); - if (value.size() != sizeof(IPv6)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of IPv6 value: {}", value.size()); - - column.insertData(reinterpret_cast(value.begin()), value.size()); - } - }; - - class CapnProtoUUIDSerializer : public ICapnProtoSerializer - { - public: - CapnProtoUUIDSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) - { - if (!capnp_type.isData()) - throwCannotConvert(data_type, column_name, capnp_type); - } - - std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override - { - auto data = column->getDataAt(row_num); - return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); - } - - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override - { - auto value = reader.as(); - if (value.size() != sizeof(UUID)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of UUID value: {}", value.size()); - - column.insertData(reinterpret_cast(value.begin()), value.size()); + const auto & reader_impl = parent_list_reader.getReaderImpl(); + UInt32 value = reader_impl.getDataElement(array_index); + assert_cast(column).insertValue(IPv4(value)); } }; @@ -652,19 +841,35 @@ namespace nested_serializer = createSerializer(assert_cast(*data_type).getDictionaryType(), column_name, capnp_type, settings); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { const auto & low_cardinality_column = assert_cast(*column); size_t index = low_cardinality_column.getIndexAt(row_num); const auto & dict_column = low_cardinality_column.getDictionary().getNestedColumn(); - return nested_serializer->writeRow(dict_column, field_builder, index); + nested_serializer->writeRow(dict_column, field_builder, parent_struct_builder, slot_offset, index); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override + { + const auto & low_cardinality_column = assert_cast(*column); + size_t index = low_cardinality_column.getIndexAt(row_num); + const auto & dict_column = low_cardinality_column.getDictionary().getNestedColumn(); + nested_serializer->writeRow(dict_column, field_builder, parent_list_builder, array_index, index); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { auto & low_cardinality_column = assert_cast(column); auto tmp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); - nested_serializer->readRow(*tmp_column, reader); + nested_serializer->readRow(*tmp_column, parent_struct_reader, slot_offset); + low_cardinality_column.insertFromFullColumn(*tmp_column, 0); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + auto & low_cardinality_column = assert_cast(column); + auto tmp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); + nested_serializer->readRow(*tmp_column, parent_list_reader, array_index); low_cardinality_column.insertFromFullColumn(*tmp_column, 0); } @@ -685,7 +890,10 @@ namespace getCapnProtoFullTypeName(capnp_type)); /// Check that struct is a named union of type VOID and one arbitrary type. - auto struct_schema = capnp_type.asStruct(); + struct_schema = capnp_type.asStruct(); + auto node = struct_schema.getProto().getStruct(); + struct_size = capnp::_::StructSize(node.getDataWordCount(), node.getPointerCount()); + discriminant_offset = node.getDiscriminantOffset(); if (!checkIfStructIsNamedUnion(struct_schema)) throw Exception( ErrorCodes::CAPN_PROTO_BAD_CAST, @@ -706,23 +914,18 @@ namespace auto first = union_fields[0]; auto second = union_fields[1]; auto nested_type = assert_cast(data_type.get())->getNestedType(); + nested_slot_offset = first.getProto().getSlot().getOffset(); /// Both fields have the same offset. if (first.getType().isVoid()) { - null_field = first; - nested_field = second; - nested_capnp_type = second.getType(); - if (nested_capnp_type.isStruct()) - nested_fields_size = nested_capnp_type.asStruct().getFields().size(); - nested_serializer = createSerializer(nested_type, column_name, nested_capnp_type, settings); + nested_serializer = createSerializer(nested_type, column_name, second.getType(), settings); + null_discriminant = 0; + nested_discriminant = 1; } else if (second.getType().isVoid()) { - null_field = second; - nested_field = first; - nested_capnp_type = first.getType(); - if (nested_capnp_type.isStruct()) - nested_fields_size = nested_capnp_type.asStruct().getFields().size(); - nested_serializer = createSerializer(nested_type, column_name, nested_capnp_type, settings); + nested_serializer = createSerializer(nested_type, column_name, first.getType(), settings); + null_discriminant = 1; + nested_discriminant = 0; } else throw Exception( @@ -733,50 +936,102 @@ namespace getCapnProtoFullTypeName(capnp_type)); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - assert(field_builder); + if (!field_builder) + { + auto builder_impl = parent_struct_builder.getBuilderImpl(); + auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getPointerField(slot_offset).initStruct(struct_size)); + field_builder = std::make_unique(std::move(struct_builder_impl), 1); + } + auto & struct_builder = assert_cast(*field_builder); + const auto & nullable_column = assert_cast(*column); if (nullable_column.isNullAt(row_num)) { - struct_builder.impl.set(null_field, capnp::Void()); + auto struct_builder_impl = struct_builder.impl.getBuilderImpl(); + struct_builder_impl.setDataField(discriminant_offset, null_discriminant); + struct_builder_impl.setDataField(nested_slot_offset, capnp::Void()); } else { - struct_builder.impl.clear(nested_field); const auto & nested_column = nullable_column.getNestedColumnPtr(); - auto nested_field_builder = initStructFieldBuilderIfNeeded(nested_column, row_num, struct_builder.impl, nested_field, nested_capnp_type, nested_fields_size); - auto value = nested_serializer->writeRow(nested_column, nested_field_builder.get(), row_num); - if (value) - struct_builder.impl.set(nested_field, *value); + struct_builder.impl.getBuilderImpl().setDataField(discriminant_offset, nested_discriminant); + nested_serializer->writeRow(nested_column, struct_builder.field_builders[0], struct_builder.impl, nested_slot_offset, row_num); } - - return std::nullopt; } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override + { + if (!field_builder) + { + auto builder_impl = parent_list_builder.getBuilderImpl(); + auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getStructElement(array_index)); + field_builder = std::make_unique(std::move(struct_builder_impl), 1); + } + + auto & struct_builder = assert_cast(*field_builder); + + const auto & nullable_column = assert_cast(*column); + if (nullable_column.isNullAt(row_num)) + { + auto struct_builder_impl = struct_builder.impl.getBuilderImpl(); + struct_builder_impl.setDataField(discriminant_offset, null_discriminant); + struct_builder_impl.setDataField(nested_slot_offset, capnp::Void()); + } + else + { + const auto & nested_column = nullable_column.getNestedColumnPtr(); + struct_builder.impl.getBuilderImpl().setDataField(discriminant_offset, nested_discriminant); + nested_serializer->writeRow(nested_column, struct_builder.field_builders[0], struct_builder.impl, nested_slot_offset, row_num); + } + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - auto struct_reader = reader.as(); auto & nullable_column = assert_cast(column); - auto field = *kj::_::readMaybe(struct_reader.which()); - if (field.getType().isVoid()) + auto reader_impl = parent_struct_reader.getReaderImpl(); + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getPointerField(slot_offset).getStruct(nullptr)); + + auto discriminant = struct_reader.getReaderImpl().getDataField(discriminant_offset); + + if (discriminant == null_discriminant) nullable_column.insertDefault(); else { auto & nested_column = nullable_column.getNestedColumn(); - auto nested_reader = struct_reader.get(field); - nested_serializer->readRow(nested_column, nested_reader); + nested_serializer->readRow(nested_column, struct_reader, nested_slot_offset); + nullable_column.getNullMapData().push_back(0); + } + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + auto & nullable_column = assert_cast(column); + auto reader_impl = parent_list_reader.getReaderImpl(); + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getStructElement(array_index)); + + auto discriminant = struct_reader.getReaderImpl().getDataField(discriminant_offset); + + if (discriminant == null_discriminant) + nullable_column.insertDefault(); + else + { + auto & nested_column = nullable_column.getNestedColumn(); + nested_serializer->readRow(nested_column, struct_reader, nested_slot_offset); nullable_column.getNullMapData().push_back(0); } } private: std::unique_ptr nested_serializer; - capnp::StructSchema::Field null_field; - capnp::StructSchema::Field nested_field; - size_t nested_fields_size = 0; - capnp::Type nested_capnp_type; + capnp::StructSchema struct_schema; + capnp::_::StructSize struct_size; + UInt32 discriminant_offset; + UInt16 null_discriminant; + UInt16 nested_discriminant; + UInt32 nested_slot_offset; }; class CapnProtoArraySerializer : public ICapnProtoSerializer @@ -788,67 +1043,102 @@ namespace throwCannotConvert(data_type, column_name, capnp_type); auto nested_type = assert_cast(data_type.get())->getNestedType(); - element_type = capnp_type.asList().getElementType(); + list_schema = capnp_type.asList(); + auto element_type = list_schema.getElementType(); + element_size = capnp::elementSizeFor(element_type.which()); if (element_type.isStruct()) - element_struct_fields = element_type.asStruct().getFields().size(); + { + element_is_struct = true; + auto node = element_type.asStruct().getProto().getStruct(); + element_struct_size = capnp::_::StructSize(node.getDataWordCount(), node.getPointerCount()); + } + nested_serializer = createSerializer(nested_type, column_name, capnp_type.asList().getElementType(), settings); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - assert(field_builder); - auto & list_builder = assert_cast(*field_builder); const auto * array_column = assert_cast(column.get()); const auto & nested_column = array_column->getDataPtr(); const auto & offsets = array_column->getOffsets(); auto offset = offsets[row_num - 1]; - size_t size = offsets[row_num] - offset; - bool need_nested_builders = list_builder.nested_builders.empty(); - for (unsigned i = 0; i != static_cast(size); ++i) - { - if (need_nested_builders) - { - /// For nested lists we need to initialize nested list builder. - if (element_type.isList()) - { - const auto & nested_offset = checkAndGetColumn(*nested_column)->getOffsets(); - size_t nested_array_size = nested_offset[offset + i] - nested_offset[offset + i - 1]; - list_builder.nested_builders.emplace_back(std::make_unique(list_builder.impl.init(i, static_cast(nested_array_size)))); - } - else if (element_type.isStruct()) - { - list_builder.nested_builders.emplace_back(std::make_unique(list_builder.impl[i], element_struct_fields)); - } - else - { - list_builder.nested_builders.emplace_back(); - } - } + UInt32 size = static_cast(offsets[row_num] - offset); - auto value = nested_serializer->writeRow(nested_column, list_builder.nested_builders[i].get(), offset + i); - if (value) - list_builder.impl.set(i, *value); + if (!field_builder) + { + auto builder_impl = parent_struct_builder.getBuilderImpl(); + capnp::DynamicList::Builder list_builder_impl; + if (element_is_struct) + list_builder_impl = capnp::DynamicList::Builder(list_schema, builder_impl.getPointerField(slot_offset).initStructList(size, element_struct_size)); + else + list_builder_impl = capnp::DynamicList::Builder(list_schema, builder_impl.getPointerField(slot_offset).initList(element_size, size)); + field_builder = std::make_unique(std::move(list_builder_impl), size); } - return std::nullopt; + auto & list_builder = assert_cast(*field_builder); + for (UInt32 i = 0; i != size; ++i) + nested_serializer->writeRow(nested_column, list_builder.nested_builders[i], list_builder.impl, i, offset + i); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto list_reader = reader.as(); + const auto * array_column = assert_cast(column.get()); + const auto & nested_column = array_column->getDataPtr(); + const auto & offsets = array_column->getOffsets(); + auto offset = offsets[row_num - 1]; + UInt32 size = static_cast(offsets[row_num] - offset); + + if (!field_builder) + { + auto builder_impl = parent_list_builder.getBuilderImpl(); + capnp::DynamicList::Builder list_builder_impl; + if (element_is_struct) + list_builder_impl = capnp::DynamicList::Builder(list_schema, builder_impl.getPointerElement(array_index).initStructList(size, element_struct_size)); + else + list_builder_impl = capnp::DynamicList::Builder(list_schema, builder_impl.getPointerElement(array_index).initList(element_size, size)); + field_builder = std::make_unique(std::move(list_builder_impl), size); + } + + auto & list_builder = assert_cast(*field_builder); + for (UInt32 i = 0; i != size; ++i) + nested_serializer->writeRow(nested_column, list_builder.nested_builders[i], list_builder.impl, i, offset + i); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + const auto & reader_impl = parent_struct_reader.getReaderImpl(); + auto list_reader = capnp::DynamicList::Reader(list_schema, reader_impl.getPointerField(slot_offset).getList(element_size, nullptr)); + UInt32 size = list_reader.size(); auto & column_array = assert_cast(column); auto & offsets = column_array.getOffsets(); offsets.push_back(offsets.back() + list_reader.size()); auto & nested_column = column_array.getData(); - for (const auto & nested_reader : list_reader) - nested_serializer->readRow(nested_column, nested_reader); + for (UInt32 i = 0; i != size; ++i) + nested_serializer->readRow(nested_column, list_reader, i); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + const auto & reader_impl = parent_list_reader.getReaderImpl(); + auto list_reader = capnp::DynamicList::Reader(list_schema, reader_impl.getPointerElement(array_index).getList(element_size, nullptr)); + UInt32 size = list_reader.size(); + auto & column_array = assert_cast(column); + auto & offsets = column_array.getOffsets(); + offsets.push_back(offsets.back() + list_reader.size()); + + auto & nested_column = column_array.getData(); + for (UInt32 i = 0; i != size; ++i) + nested_serializer->readRow(nested_column, list_reader, i); } private: + capnp::ListSchema list_schema; std::unique_ptr nested_serializer; - capnp::Type element_type; - size_t element_struct_fields; + capnp::ElementSize element_size; + capnp::_::StructSize element_struct_size; + bool element_is_struct = false; + }; class CapnProtoMapSerializer : public ICapnProtoSerializer @@ -869,7 +1159,9 @@ namespace if (!capnp_type.isStruct()) throwCannotConvert(data_type, column_name, capnp_type); - auto struct_schema = capnp_type.asStruct(); + struct_schema = capnp_type.asStruct(); + auto node = struct_schema.getProto().getStruct(); + struct_size = capnp::_::StructSize(node.getDataWordCount(), node.getPointerCount()); if (checkIfStructContainsUnnamedUnion(struct_schema)) throw Exception( @@ -921,43 +1213,70 @@ namespace DataTypes types = {map_type.getKeyType(), map_type.getValueType()}; Names names = {"key", "value"}; auto entries_type = std::make_shared(std::make_shared(types, names)); - entries_field = struct_schema.getFields()[0]; - entries_capnp_type = entries_field.getType(); nested_serializer = createSerializer(entries_type, column_name, field_type, settings); + entries_slot_offset = struct_schema.getFields()[0].getProto().getSlot().getOffset(); } - std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - assert(field_builder); + if (!field_builder) + { + auto builder_impl = parent_struct_builder.getBuilderImpl(); + auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getPointerField(slot_offset).initStruct(struct_size)); + field_builder = std::make_unique(std::move(struct_builder_impl), 1); + } + auto & struct_builder = assert_cast(*field_builder); const auto & entries_column = assert_cast(column.get())->getNestedColumnPtr(); - auto entries_builder = initStructFieldBuilderIfNeeded(entries_column, row_num, struct_builder.impl, entries_field, entries_capnp_type, 0); - nested_serializer->writeRow(entries_column, entries_builder.get(), row_num); - return std::nullopt; + nested_serializer->writeRow(entries_column, struct_builder.field_builders[0], struct_builder.impl, entries_slot_offset, row_num); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto struct_reader = reader.as(); + if (!field_builder) + { + auto builder_impl = parent_list_builder.getBuilderImpl(); + auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getStructElement(array_index)); + field_builder = std::make_unique(std::move(struct_builder_impl), 1); + } + + auto & struct_builder = assert_cast(*field_builder); + const auto & entries_column = assert_cast(column.get())->getNestedColumnPtr(); + nested_serializer->writeRow(entries_column, struct_builder.field_builders[0], struct_builder.impl, entries_slot_offset, row_num); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + auto reader_impl = parent_struct_reader.getReaderImpl(); + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getPointerField(slot_offset).getStruct(nullptr)); auto & entries_column = assert_cast(column).getNestedColumn(); - nested_serializer->readRow(entries_column, struct_reader.get(entries_field)); + nested_serializer->readRow(entries_column, struct_reader, entries_slot_offset); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + auto reader_impl = parent_list_reader.getReaderImpl(); + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getStructElement(array_index)); + auto & entries_column = assert_cast(column).getNestedColumn(); + nested_serializer->readRow(entries_column, struct_reader, entries_slot_offset); } private: std::unique_ptr nested_serializer; - capnp::StructSchema::Field entries_field; - capnp::Type entries_capnp_type; + capnp::StructSchema struct_schema; + capnp::_::StructSize struct_size; + UInt32 entries_slot_offset; }; class CapnProtoStructureSerializer : public ICapnProtoSerializer { public: - CapnProtoStructureSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + CapnProtoStructureSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) : struct_schema(schema) { if (checkIfStructIsNamedUnion(schema) || checkIfStructContainsUnnamedUnion(schema)) throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Root CapnProto Struct cannot be named union/struct with unnamed union"); - initialize(data_types, names, schema, settings); + initialize(data_types, names, settings); } CapnProtoStructureSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) @@ -965,7 +1284,7 @@ namespace if (!capnp_type.isStruct()) throwCannotConvert(data_type, column_name, capnp_type); - auto struct_schema = capnp_type.asStruct(); + struct_schema = capnp_type.asStruct(); if (checkIfStructIsNamedUnion(struct_schema) || checkIfStructContainsUnnamedUnion(struct_schema)) throw Exception( @@ -1002,7 +1321,7 @@ namespace try { - initialize(nested_types, nested_names, struct_schema, settings); + initialize(nested_types, nested_names, settings); } catch (Exception & e) { @@ -1011,77 +1330,118 @@ namespace } } - std::optional writeRow(const ColumnPtr & column, FieldBuilder * builder, size_t row_num) override + void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - assert(builder); - auto & struct_builder = assert_cast(*builder); + if (!field_builder) + { + auto builder_impl = parent_struct_builder.getBuilderImpl(); + auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getPointerField(slot_offset).initStruct(struct_size)); + field_builder = std::make_unique(std::move(struct_builder_impl), fields_count); + } + + auto & struct_builder = assert_cast(*field_builder); if (const auto * tuple_column = typeid_cast(column.get())) - writeRow(tuple_column->getColumnsCopy(), struct_builder, row_num); + { + const auto & columns = tuple_column->getColumns(); + for (size_t i = 0; i != columns.size(); ++i) + fields_serializers[i]->writeRow(columns[i], struct_builder.field_builders[fields_indexes[i]], struct_builder.impl, fields_offsets[i], row_num); + } else - writeRow(Columns{column}, struct_builder, row_num); - return std::nullopt; + { + fields_serializers[0]->writeRow(column, struct_builder.field_builders[fields_indexes[0]], struct_builder.impl, fields_offsets[0], row_num); + } + } + + void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override + { + if (!field_builder) + { + auto builder_impl = parent_list_builder.getBuilderImpl(); + auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getStructElement(array_index)); + field_builder = std::make_unique(std::move(struct_builder_impl), fields_count); + } + + auto & struct_builder = assert_cast(*field_builder); + if (const auto * tuple_column = typeid_cast(column.get())) + { + const auto & columns = tuple_column->getColumns(); + for (size_t i = 0; i != columns.size(); ++i) + fields_serializers[i]->writeRow(columns[i], struct_builder.field_builders[fields_indexes[i]], struct_builder.impl, fields_offsets[i], row_num); + } + else + { + fields_serializers[0]->writeRow(column, struct_builder.field_builders[fields_indexes[0]], struct_builder.impl, fields_offsets[0], row_num); + } } void writeRow(const Columns & columns, StructBuilder & struct_builder, size_t row_num) { for (size_t i = 0; i != columns.size(); ++i) - { - const auto & field = fields[i]; - size_t field_index = field.getIndex(); - if (likely(!struct_builder.field_builders[field_index])) - struct_builder.field_builders[field_index] = initStructFieldBuilderIfNeeded( - columns[i], row_num, struct_builder.impl, field, fields_types[i], nested_field_sizes[i]); - - auto value = field_serializers[i]->writeRow(columns[i], struct_builder.field_builders[field_index].get(), row_num); - if (value) - struct_builder.impl.set(field, *value); - } + fields_serializers[i]->writeRow(columns[i], struct_builder.field_builders[fields_indexes[i]], struct_builder.impl, fields_offsets[i], row_num); } - void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - auto struct_reader = reader.as(); + auto reader_impl = parent_struct_reader.getReaderImpl(); + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getPointerField(slot_offset).getStruct(nullptr)); if (auto * tuple_column = typeid_cast(&column)) { for (size_t i = 0; i != tuple_column->tupleSize(); ++i) - field_serializers[i]->readRow(tuple_column->getColumn(i), struct_reader.get(fields[i])); + fields_serializers[i]->readRow(tuple_column->getColumn(i), struct_reader, fields_offsets[i]); } else - field_serializers[0]->readRow(column, struct_reader.get(fields[0])); + fields_serializers[0]->readRow(column, struct_reader, fields_offsets[0]); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + auto reader_impl = parent_list_reader.getReaderImpl(); + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getStructElement(array_index)); + if (auto * tuple_column = typeid_cast(&column)) + { + for (size_t i = 0; i != tuple_column->tupleSize(); ++i) + fields_serializers[i]->readRow(tuple_column->getColumn(i), struct_reader, fields_offsets[i]); + } + else + fields_serializers[0]->readRow(column, struct_reader, fields_offsets[0]); } void readRow(MutableColumns & columns, const capnp::DynamicStruct::Reader & reader) { for (size_t i = 0; i != columns.size(); ++i) - field_serializers[i]->readRow(*columns[i], reader.get(fields[i])); + fields_serializers[i]->readRow(*columns[i], reader, fields_offsets[i]); } private: - void initialize(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + void initialize(const DataTypes & data_types, const Names & names, const FormatSettings::CapnProto & settings) { - field_serializers.reserve(data_types.size()); - fields.reserve(data_types.size()); - fields_types.reserve(data_types.size()); - nested_field_sizes.reserve(data_types.size()); + auto node = struct_schema.getProto().getStruct(); + struct_size = capnp::_::StructSize(node.getDataWordCount(), node.getPointerCount()); + fields_count = struct_schema.getFields().size(); + fields_serializers.reserve(data_types.size()); + fields_offsets.reserve(data_types.size()); + fields_indexes.reserve(data_types.size()); for (size_t i = 0; i != data_types.size(); ++i) { auto [field_name, _] = splitFieldName(names[i]); - auto field = findFieldByName(schema, field_name); + auto field = findFieldByName(struct_schema, field_name); if (!field) throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto schema doesn't contain field with name {}", field_name); - fields.push_back(*field); auto capnp_type = field->getType(); - fields_types.push_back(capnp_type); - nested_field_sizes.push_back(capnp_type.isStruct() ? capnp_type.asStruct().getFields().size() : 0); - field_serializers.push_back(createSerializer(data_types[i], names[i], capnp_type, settings)); + fields_serializers.push_back(createSerializer(data_types[i], names[i], capnp_type, settings)); + fields_offsets.push_back(field->getProto().getSlot().getOffset()); + fields_indexes.push_back(field->getIndex()); } } - std::vector> field_serializers; - std::vector fields; - std::vector nested_field_sizes; - std::vector fields_types; + capnp::StructSchema struct_schema; + capnp::_::StructSize struct_size; + size_t fields_count; + std::vector> fields_serializers; + std::vector fields_offsets; + std::vector fields_indexes; + }; std::unique_ptr createSerializer(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) @@ -1116,17 +1476,17 @@ namespace case TypeIndex::UInt64: return createIntegerSerializer(type, name, capnp_type); case TypeIndex::Int128: - return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); case TypeIndex::UInt128: - return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); case TypeIndex::Int256: - return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); case TypeIndex::UInt256: - return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); case TypeIndex::Float32: - return std::make_unique>(type, name, capnp_type); + return createFloatSerializer(type, name, capnp_type); case TypeIndex::Float64: - return std::make_unique>(type, name, capnp_type); + return createFloatSerializer(type, name, capnp_type); case TypeIndex::Date: return std::make_unique(type, name, capnp_type); case TypeIndex::Date32: @@ -1140,15 +1500,15 @@ namespace case TypeIndex::Decimal64: return std::make_unique>(type, name, capnp_type); case TypeIndex::Decimal128: - return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); case TypeIndex::Decimal256: - return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); case TypeIndex::IPv4: return std::make_unique(type, name, capnp_type); case TypeIndex::IPv6: - return std::make_unique(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); case TypeIndex::UUID: - return std::make_unique(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); case TypeIndex::Enum8: return std::make_unique>(type, name, capnp_type, settings.enum_comparing_mode); case TypeIndex::Enum16: diff --git a/src/Formats/CapnProtoSerializer.h b/src/Formats/CapnProtoSerializer.h index 692f5e5301f..5bdd1a0e554 100644 --- a/src/Formats/CapnProtoSerializer.h +++ b/src/Formats/CapnProtoSerializer.h @@ -4,6 +4,7 @@ #include #include +#include namespace DB { From e4c1e2f2326c6f52cb42c1b1b7eaa1a333784a61 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 May 2023 17:37:09 +0200 Subject: [PATCH 0682/2223] Fix build while it's not failing locally --- src/Disks/DiskEncryptedTransaction.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 7797e11bf39..4a613374ccf 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { From 4ce722854b4216c1a6e443327bf1a2cd248011b8 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 25 May 2023 11:51:47 -0400 Subject: [PATCH 0683/2223] Fix URL in backport comment --- tests/ci/cherry_pick.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 211092190fa..2fa562a1386 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -261,7 +261,7 @@ close it. ) self.backport_pr = self.repo.create_pull( title=title, - body=f"Original pull-request {self.pr.url}\n" + body=f"Original pull-request {self.pr.html_url}\n" f"Cherry-pick pull-request #{self.cherrypick_pr.number}\n\n" f"{self.BACKPORT_DESCRIPTION}", base=self.name, From 6a8a21e09a49303919d2357f0b7f9ca38c79684b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 16:01:40 +0000 Subject: [PATCH 0684/2223] Add test for s3 disk restore --- src/Coordination/Changelog.cpp | 3 +- src/Coordination/KeeperSnapshotManager.cpp | 4 +- .../configs/disk_s3_storage.xml | 26 +++ .../configs/enable_keeper1.xml | 2 - .../configs/enable_keeper2.xml | 2 - .../configs/enable_keeper3.xml | 2 - .../configs/keeper1_snapshot_disk.xml | 5 + .../configs/keeper2_snapshot_disk.xml | 5 + .../configs/keeper3_snapshot_disk.xml | 5 + .../configs/local_storage_path.xml | 6 + .../test_keeper_restore_from_snapshot/test.py | 12 +- .../test_disk_s3.py | 152 ++++++++++++++++++ 12 files changed, 212 insertions(+), 12 deletions(-) create mode 100644 tests/integration/test_keeper_restore_from_snapshot/configs/disk_s3_storage.xml create mode 100644 tests/integration/test_keeper_restore_from_snapshot/configs/keeper1_snapshot_disk.xml create mode 100644 tests/integration/test_keeper_restore_from_snapshot/configs/keeper2_snapshot_disk.xml create mode 100644 tests/integration/test_keeper_restore_from_snapshot/configs/keeper3_snapshot_disk.xml create mode 100644 tests/integration/test_keeper_restore_from_snapshot/configs/local_storage_path.xml create mode 100644 tests/integration/test_keeper_restore_from_snapshot/test_disk_s3.py diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 875b0758d27..b289e1dc9f2 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -616,7 +616,8 @@ Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, Kee load_from_disk(disk); auto current_log_disk = getCurrentLogDisk(); - load_from_disk(current_log_disk); + if (disk != current_log_disk) + load_from_disk(current_log_disk); if (existing_changelogs.empty()) LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", disk->getPath()); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 77abbfb2054..c3371501976 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -612,7 +612,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu disk->removeFile(tmp_snapshot_file_name); - existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); + existing_snapshots.emplace(up_to_log_idx, SnapshotFileInfo{snapshot_file_name, disk}); removeOutdatedSnapshotsIfNeeded(); return {snapshot_file_name, disk}; @@ -750,7 +750,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor disk->removeFile(tmp_snapshot_file_name); - existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); + existing_snapshots.emplace(up_to_log_idx, SnapshotFileInfo{snapshot_file_name, disk}); removeOutdatedSnapshotsIfNeeded(); return {snapshot_file_name, disk}; diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/disk_s3_storage.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/disk_s3_storage.xml new file mode 100644 index 00000000000..2ec4bcd77fb --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/disk_s3_storage.xml @@ -0,0 +1,26 @@ + + + + + s3_plain + http://minio1:9001/root/data/snapshots1/ + minio + minio123 + + + s3_plain + http://minio1:9001/root/data/snapshots2/ + minio + minio123 + + + s3_plain + http://minio1:9001/root/data/snapshots3/ + minio + minio123 + + + + + + diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml index 4ea543e6f31..510913b5dc4 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml @@ -2,8 +2,6 @@ 9181 1 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots 5000 diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml index 4bf3083c1fa..0d50573df01 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml @@ -2,8 +2,6 @@ 9181 2 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots 5000 diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml index b9e2a2d0422..8fa322cb9e7 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml @@ -2,8 +2,6 @@ 9181 3 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots 5000 diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/keeper1_snapshot_disk.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper1_snapshot_disk.xml new file mode 100644 index 00000000000..01759e2771a --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper1_snapshot_disk.xml @@ -0,0 +1,5 @@ + + + snapshot_s3_plain1 + + \ No newline at end of file diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/keeper2_snapshot_disk.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper2_snapshot_disk.xml new file mode 100644 index 00000000000..6c33c5fb300 --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper2_snapshot_disk.xml @@ -0,0 +1,5 @@ + + + snapshot_s3_plain2 + + \ No newline at end of file diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/keeper3_snapshot_disk.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper3_snapshot_disk.xml new file mode 100644 index 00000000000..5016ccd581b --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper3_snapshot_disk.xml @@ -0,0 +1,5 @@ + + + snapshot_s3_plain3 + + \ No newline at end of file diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/local_storage_path.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/local_storage_path.xml new file mode 100644 index 00000000000..63cb958c88e --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/local_storage_path.xml @@ -0,0 +1,6 @@ + + + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + \ No newline at end of file diff --git a/tests/integration/test_keeper_restore_from_snapshot/test.py b/tests/integration/test_keeper_restore_from_snapshot/test.py index bc33689dd20..e4d5793bb17 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/test.py +++ b/tests/integration/test_keeper_restore_from_snapshot/test.py @@ -9,13 +9,19 @@ import time cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/enable_keeper1.xml"], stay_alive=True + "node1", + main_configs=["configs/enable_keeper1.xml", "configs/local_storage_path.xml"], + stay_alive=True, ) node2 = cluster.add_instance( - "node2", main_configs=["configs/enable_keeper2.xml"], stay_alive=True + "node2", + main_configs=["configs/enable_keeper2.xml", "configs/local_storage_path.xml"], + stay_alive=True, ) node3 = cluster.add_instance( - "node3", main_configs=["configs/enable_keeper3.xml"], stay_alive=True + "node3", + main_configs=["configs/enable_keeper3.xml", "configs/local_storage_path.xml"], + stay_alive=True, ) from kazoo.client import KazooClient, KazooState diff --git a/tests/integration/test_keeper_restore_from_snapshot/test_disk_s3.py b/tests/integration/test_keeper_restore_from_snapshot/test_disk_s3.py new file mode 100644 index 00000000000..1226df75203 --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/test_disk_s3.py @@ -0,0 +1,152 @@ +#!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +import random +import string +import os +import time + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=[ + "configs/enable_keeper1.xml", + "configs/disk_s3_storage.xml", + "configs/keeper1_snapshot_disk.xml", + ], + stay_alive=True, + with_minio=True, +) +node2 = cluster.add_instance( + "node2", + main_configs=[ + "configs/enable_keeper2.xml", + "configs/disk_s3_storage.xml", + "configs/keeper2_snapshot_disk.xml", + ], + stay_alive=True, + with_minio=True, +) +node3 = cluster.add_instance( + "node3", + main_configs=[ + "configs/enable_keeper3.xml", + "configs/disk_s3_storage.xml", + "configs/keeper3_snapshot_disk.xml", + ], + stay_alive=True, + with_minio=True, +) + +from kazoo.client import KazooClient, KazooState + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def stop_zk(zk): + try: + if zk: + zk.stop() + zk.close() + except: + pass + + +def test_recover_from_snapshot_with_disk_s3(started_cluster): + try: + node1_zk = node2_zk = node3_zk = None + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3") + + node1_zk.create("/test_snapshot_multinode_recover", "somedata".encode()) + + node2_zk.sync("/test_snapshot_multinode_recover") + node3_zk.sync("/test_snapshot_multinode_recover") + + assert node1_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + assert node2_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + assert node3_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + + node3.stop_clickhouse(kill=True) + + # at least we will have 2 snapshots + for i in range(435): + node1_zk.create( + "/test_snapshot_multinode_recover" + str(i), + ("somedata" + str(i)).encode(), + ) + + for i in range(435): + if i % 10 == 0: + node1_zk.delete("/test_snapshot_multinode_recover" + str(i)) + + finally: + for zk in [node1_zk, node2_zk, node3_zk]: + stop_zk(zk) + + # stale node should recover from leader's snapshot + # with some sanitizers can start longer than 5 seconds + node3.start_clickhouse(20) + keeper_utils.wait_until_connected(cluster, node3) + print("Restarted") + + try: + node1_zk = node2_zk = node3_zk = None + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3") + + node1_zk.sync("/test_snapshot_multinode_recover") + node2_zk.sync("/test_snapshot_multinode_recover") + node3_zk.sync("/test_snapshot_multinode_recover") + + assert node1_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + assert node2_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + assert node3_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + + for i in range(435): + if i % 10 != 0: + assert ( + node1_zk.get("/test_snapshot_multinode_recover" + str(i))[0] + == ("somedata" + str(i)).encode() + ) + assert ( + node2_zk.get("/test_snapshot_multinode_recover" + str(i))[0] + == ("somedata" + str(i)).encode() + ) + assert ( + node3_zk.get("/test_snapshot_multinode_recover" + str(i))[0] + == ("somedata" + str(i)).encode() + ) + else: + assert ( + node1_zk.exists("/test_snapshot_multinode_recover" + str(i)) is None + ) + assert ( + node2_zk.exists("/test_snapshot_multinode_recover" + str(i)) is None + ) + assert ( + node3_zk.exists("/test_snapshot_multinode_recover" + str(i)) is None + ) + finally: + for zk in [node1_zk, node2_zk, node3_zk]: + stop_zk(zk) From eca08438f42ed43b1e393b2f1dd4b4b1aefefd5e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 17:05:18 +0000 Subject: [PATCH 0685/2223] Fix macos build --- src/Functions/space.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 7d55d704038..009bc20e065 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -115,7 +115,7 @@ public: checkRepeatTime(times); if (pos + times + 1 > res_chars.size()) - res_chars.resize(std::max(2 * res_chars.size(), pos + times + 1)); + res_chars.resize(std::max(2 * res_chars.size(), static_cast(pos + times + 1))); memset(res_chars.begin() + pos, space, times); pos += times; From ad4a21034f79ba296153ea153a34d8829e7d1f6b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 17:41:38 +0000 Subject: [PATCH 0686/2223] Fix msan issue in keyed siphash Issue: https://s3.amazonaws.com/clickhouse-test-reports/0/ffdd91669471f4934704f98f0191524496b4e85b/fuzzer_astfuzzermsan/report.html Repro: SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1) Minimal repro: SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2) --- docs/en/sql-reference/functions/hash-functions.md | 8 ++++---- src/Functions/FunctionsHashing.h | 10 ++++++++-- .../queries/0_stateless/02534_keyed_siphash.reference | 2 ++ tests/queries/0_stateless/02534_keyed_siphash.sql | 4 ++++ 4 files changed, 18 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 635c8f4e0ec..8dfa03ceaf2 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -51,7 +51,7 @@ Calculates the MD5 from a string and returns the resulting set of bytes as Fixed If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 (#hash_functions-siphash64) +## sipHash64 {#hash_functions-siphash64} Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. @@ -63,9 +63,9 @@ This is a cryptographic hash function. It works at least three times faster than The function [interprets](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. It then combines the hashes by the following algorithm: -1. The first and the second hash value are concatenated to an array which is hashed. -2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. -3. This calculation is repeated for all remaining hash values of the original input. +1. The first and the second hash value are concatenated to an array which is hashed. +2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. +3. This calculation is repeated for all remaining hash values of the original input. **Arguments** diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 3de757bfa3f..b8122ebac85 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -91,12 +91,18 @@ namespace impl throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64"); if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) - ret.key0 = key0col->get64(0); + { + const auto & key0col_data = key0col->getData(); + ret.key0 = key0col_data[0]; + } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) - ret.key1 = key1col->get64(0); + { + const auto & key1col_data = key1col->getData(); + ret.key1 = key1col_data[0]; + } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 3606b9a41db..d4ab4a4e88f 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -194,3 +194,5 @@ E28DBDE7FE22E41C 1 E28DBDE7FE22E41C 1 +1CE422FEE7BD8DE20000000000000000 +7766709361750702608 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 9c914f586f0..09489be39c2 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -272,3 +272,7 @@ select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; select hex(sipHash64Keyed()); SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; + +-- Crashed with memory sanitizer +SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); +SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); From 231d52d0e6d3fc16511522228e4241d95005d15f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 18:27:31 +0000 Subject: [PATCH 0687/2223] Document system.build_options --- .../operations/system-tables/build_options.md | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) create mode 100644 docs/en/operations/system-tables/build_options.md diff --git a/docs/en/operations/system-tables/build_options.md b/docs/en/operations/system-tables/build_options.md new file mode 100644 index 00000000000..5225d0ff99d --- /dev/null +++ b/docs/en/operations/system-tables/build_options.md @@ -0,0 +1,27 @@ +--- +slug: /en/operations/system-tables/build_options +--- +# build_options + +Contains information about the ClickHouse server's build options. + +Columns: + +- `name` (String) — Name of the build option, e.g. `USE_ODBC` +- `value` (String) — Value of the build option, e.g. `1` + +**Example** + +``` sql +SELECT * FROM system.build_options LIMIT 5 +``` + +``` text +┌─name─────────────┬─value─┐ +│ USE_BROTLI │ 1 │ +│ USE_BZIP2 │ 1 │ +│ USE_CAPNP │ 1 │ +│ USE_CASSANDRA │ 1 │ +│ USE_DATASKETCHES │ 1 │ +└──────────────────┴───────┘ +``` From 8c4f0c763e676ebde0d89803701d4aaa16a966f1 Mon Sep 17 00:00:00 2001 From: Stanislav Dobrovolschii Date: Thu, 25 May 2023 19:50:36 +0200 Subject: [PATCH 0688/2223] Update SAMPLE clause documentation links --- docs/en/sql-reference/statements/select/sample.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/select/sample.md b/docs/en/sql-reference/statements/select/sample.md index a0c9257ed94..137f86cc8b9 100644 --- a/docs/en/sql-reference/statements/select/sample.md +++ b/docs/en/sql-reference/statements/select/sample.md @@ -34,7 +34,7 @@ For the `SAMPLE` clause the following syntax is supported: | `SAMPLE k OFFSET m` | Here `k` and `m` are the numbers from 0 to 1. The query is executed on a sample of `k` fraction of the data. The data used for the sample is offset by `m` fraction. [Read more](#select-sample-offset) | -## SAMPLE K +## SAMPLE K {#select-sample-k} Here `k` is the number from 0 to 1 (both fractional and decimal notations are supported). For example, `SAMPLE 1/2` or `SAMPLE 0.5`. @@ -54,7 +54,7 @@ ORDER BY PageViews DESC LIMIT 1000 In this example, the query is executed on a sample from 0.1 (10%) of data. Values of aggregate functions are not corrected automatically, so to get an approximate result, the value `count()` is manually multiplied by 10. -## SAMPLE N +## SAMPLE N {#select-sample-n} Here `n` is a sufficiently large integer. For example, `SAMPLE 10000000`. @@ -90,7 +90,7 @@ FROM visits SAMPLE 10000000 ``` -## SAMPLE K OFFSET M +## SAMPLE K OFFSET M {#select-sample-offset} Here `k` and `m` are numbers from 0 to 1. Examples are shown below. From 7ebe19f5fb1390bc5f2dac3a5cca7db9c76bdd98 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 May 2023 19:18:11 +0000 Subject: [PATCH 0689/2223] Fixing more tests --- src/Interpreters/ActionsVisitor.cpp | 7 ++- src/Interpreters/Context.cpp | 15 +++++ src/Interpreters/Context.h | 1 + src/Interpreters/DatabaseCatalog.h | 3 + src/Interpreters/GlobalSubqueriesVisitor.h | 19 ++++--- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/PreparedSets.cpp | 55 +++++++++++++++---- src/Interpreters/PreparedSets.h | 35 ++++-------- src/Planner/CollectSets.cpp | 3 +- src/Planner/Planner.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 8 ++- src/Processors/QueryPlan/CreatingSetsStep.cpp | 4 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 4 +- .../QueryPlan/ReadFromMemoryStorageStep.cpp | 6 +- .../QueryPlan/ReadFromMemoryStorageStep.h | 2 + .../Transforms/CreatingSetsTransform.cpp | 13 +++-- .../Transforms/CreatingSetsTransform.h | 5 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 +- src/QueryPipeline/QueryPipelineBuilder.h | 5 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMemory.h | 2 + 21 files changed, 132 insertions(+), 65 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index da8666466fc..133fcbbfe87 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1405,6 +1405,8 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (auto set = data.prepared_sets->getFuture(set_key)) return set; + FutureSetPtr external_table_set; + /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). if (identifier) @@ -1417,6 +1419,9 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (StorageSet * storage_set = dynamic_cast(table.get())) return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } + + if (auto tmp_table = data.getContext()->findExternalTable(table_id.getShortName())) + external_table_set = tmp_table->future_set; } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. @@ -1438,7 +1443,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool subquery_for_set.createSource(*interpreter); } - return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); + return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), std::move(external_table_set)); } else { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d9f450191bc..f8fe99fbcc6 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1319,6 +1319,21 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder & external_tables_mapping.emplace(table_name, std::make_shared(std::move(temporary_table))); } +std::shared_ptr Context::findExternalTable(const String & table_name) const +{ + if (isGlobalContext()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have external tables"); + + std::shared_ptr holder; + { + auto lock = getLock(); + auto iter = external_tables_mapping.find(table_name); + if (iter == external_tables_mapping.end()) + return {}; + holder = iter->second; + } + return holder; +} std::shared_ptr Context::removeExternalTable(const String & table_name) { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 1be662e0958..1a0ee891699 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -611,6 +611,7 @@ public: Tables getExternalTables() const; void addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table); + std::shared_ptr findExternalTable(const String & table_name) const; std::shared_ptr removeExternalTable(const String & table_name); const Scalars & getScalars() const; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 51e9fbdb936..258ea2dee7c 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -79,6 +79,8 @@ private: using DDLGuardPtr = std::unique_ptr; +class FutureSet; +using FutureSetPtr = std::shared_ptr; /// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID. /// Such table can be accessed from everywhere by its ID. @@ -111,6 +113,7 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext IDatabase * temporary_tables = nullptr; UUID id = UUIDHelpers::Nil; + FutureSetPtr future_set; }; ///TODO maybe remove shared_ptr from here? diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index a872af529aa..195839da04f 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -64,6 +64,7 @@ public: void addExternalStorage(ASTPtr & ast, bool set_alias = false) { + // std::cerr << "=============== addExternalStorage is remote " << is_remote << std::endl; /// With nondistributed queries, creating temporary tables does not make sense. if (!is_remote) return; @@ -163,10 +164,10 @@ public: /// We need to materialize external tables immediately because reading from distributed /// tables might generate local plans which can refer to external tables during index /// analysis. It's too late to populate the external table via CreatingSetsTransform. - if (is_explain) - { - /// Do not materialize external tables if it's explain statement. - } + // if (is_explain) + // { + // /// Do not materialize external tables if it's explain statement. + // } // else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) // { // auto external_table = external_storage_holder->getTable(); @@ -176,13 +177,15 @@ public: // CompletedPipelineExecutor executor(io.pipeline); // executor.execute(); // } - else + // else { // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); auto set_key = PreparedSetKey::forSubquery(database_and_table_name->getTreeHash()); + // std::cerr << "====== Adding key " << set_key.toString() << std::endl; + if (!prepared_sets->getFuture(set_key)) { SubqueryForSet subquery_for_set; @@ -191,10 +194,12 @@ public: subquery_for_set.createSource(*interpreter); //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; - prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); + auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + external_storage_holder->future_set = std::move(future_set); } else - prepared_sets->addStorageToSubquery(key, std::move(external_storage)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "!!!!!!!!"); + //prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8d305c07ce9..28a0e27384a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3088,7 +3088,7 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - auto subqueries = prepared_sets->detachSubqueries(context); + auto subqueries = prepared_sets->detachSubqueries(); if (!subqueries.empty()) { diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index eebc59e85d5..f2b70462227 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -101,14 +101,14 @@ String PreparedSetKey::toString() const /// If the subquery is not associated with any set, create default-constructed SubqueryForSet. /// It's aimed to fill external table passed to SubqueryForSet::createSource. -void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) -{ - auto it = subqueries.find(subquery_id); - if (it == subqueries.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); +// void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) +// { +// auto it = subqueries.find(subquery_id); +// if (it == subqueries.end()) +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); - it->second->addStorage(std::move(storage)); -} +// it->second->addStorage(std::move(storage)); +// } FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set_) { @@ -132,10 +132,10 @@ FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, return it->second; } -FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery) +FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set) { auto id = subquery.key; - auto from_subquery = std::make_shared(std::move(subquery)); + auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set)); auto [it, inserted] = sets.emplace(key, from_subquery); if (!inserted) @@ -145,7 +145,7 @@ FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryF // std::cerr << "========= PreparedSets::addFromSubquery\n"; // std::cerr << StackTrace().toString() << std::endl; - subqueries.emplace(id, std::move(from_subquery)); + subqueries.emplace_back(SetAndName{.name = id, .set = std::move(from_subquery)}); return it->second; } @@ -176,7 +176,7 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return res; // } -PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries(const ContextPtr &) +PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() { auto res = std::move(subqueries); subqueries = SubqueriesForSets(); @@ -226,6 +226,36 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo return promise_to_fill_set; } +SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) +{ + if (!context->getSettingsRef().use_index_for_in_with_subqueries) + return nullptr; + + if (set) + { + if (set->hasExplicitSetElements()) + return set; + + return nullptr; + } + + if (external_table_set) + return set = external_table_set->buildOrderedSetInplace(context); + + auto plan = buildPlan(context, true); + if (!plan) + return nullptr; + + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + + return set; +} + std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) { if (set) @@ -313,7 +343,8 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) //block(std::move(block_)) } -FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_) + : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)) {} FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 35bbc1e12fb..c510e690e25 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -132,30 +132,13 @@ public: class FutureSetFromSubquery : public FutureSet { public: - FutureSetFromSubquery(SubqueryForSet subquery_); + FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_); bool isReady() const override { return set != nullptr; } bool isFilled() const override { return isReady(); } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr & context) override - { - if (!context->getSettingsRef().use_index_for_in_with_subqueries) - return nullptr; - - auto plan = buildPlan(context, true); - if (!plan) - return nullptr; - - auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); - - CompletedPipelineExecutor executor(pipeline); - executor.execute(); - - return set; - } + SetPtr buildOrderedSetInplace(const ContextPtr & context) override; std::unique_ptr build(const ContextPtr & context) override { @@ -167,6 +150,7 @@ public: private: SetPtr set; SubqueryForSet subquery; + FutureSetPtr external_table_set; std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; @@ -244,16 +228,21 @@ struct PreparedSetKey class PreparedSets { public: - using SubqueriesForSets = std::unordered_map>; + struct SetAndName + { + String name; + std::shared_ptr set; + }; + using SubqueriesForSets = std::vector; // SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, // SizeLimits set_size_limit, bool transform_null_in); FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery); + FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set); - void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); + //void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); FutureSetPtr getFuture(const PreparedSetKey & key) const; //SubqueryForSet & getSubquery(const String & subquery_id); @@ -262,7 +251,7 @@ public: /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. /// SetPtr would still be available for consumers of PreparedSets. - SubqueriesForSets detachSubqueries(const ContextPtr &); + SubqueriesForSets detachSubqueries(); /// Returns all sets that match the given ast hash not checking types /// Used in KeyCondition and MergeTreeIndexConditionBloomFilter to make non exact match for types in PreparedSetKey diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index d66424f89ec..19dc569ff0c 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -107,7 +107,8 @@ public: subquery_for_set.key = planner_context.createSetKey(in_second_argument); subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); - sets.addFromSubquery(set_key, std::move(subquery_for_set)); + /// TODO + sets.addFromSubquery(set_key, std::move(subquery_for_set), nullptr); //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index b1780212e51..5c3fc82a80b 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1468,7 +1468,7 @@ void Planner::buildPlanForQueryNode() if (!select_query_options.only_analyze) { - auto subqueries = planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()); + auto subqueries = planner_context->getPreparedSets().detachSubqueries(); if (!subqueries.empty()) { diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index e0844a6d2b1..bbac9bf5c4b 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -632,7 +632,13 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma DataTypes set_element_types; auto in_second_argument_node_type = in_second_argument->getNodeType(); - if (!(in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION)) + // std::cerr << "=========== " << in_second_argument->getNodeTypeName() << std::endl; + bool subquery_or_table = + in_second_argument_node_type == QueryTreeNodeType::QUERY || + in_second_argument_node_type == QueryTreeNodeType::UNION || + in_second_argument_node_type == QueryTreeNodeType::TABLE; + + if (!subquery_or_table) { set_element_types = {in_first_argument->getResultType()}; const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 34018b2144c..38506412531 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -48,7 +48,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context); + pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context->getPreparedSetsCache()); } void CreatingSetStep::updateOutputStream() @@ -189,7 +189,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, if (!prepared_sets || prepared_sets->empty()) return; - addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(context), context); + addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(), context); } DelayedCreatingSetsStep::DelayedCreatingSetsStep( diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 62e369659d1..8e2221d564d 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -43,7 +43,7 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, - PreparedSetsPtr prepared_sets, + [[maybe_unused]] PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, @@ -99,7 +99,7 @@ std::unique_ptr createLocalPlan( } else { - auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options, prepared_sets); + auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); //, prepared_sets); interpreter.buildQueryPlan(*query_plan); } diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp index 2080c31d253..4ad3cc7373a 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp @@ -96,11 +96,13 @@ private: }; ReadFromMemoryStorageStep::ReadFromMemoryStorageStep(const Names & columns_to_read_, + StoragePtr storage_, const StorageSnapshotPtr & storage_snapshot_, const size_t num_streams_, const bool delay_read_for_global_sub_queries_) : SourceStepWithFilter(DataStream{.header=storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}), columns_to_read(columns_to_read_), + storage(std::move(storage_)), storage_snapshot(storage_snapshot_), num_streams(num_streams_), delay_read_for_global_sub_queries(delay_read_for_global_sub_queries_) @@ -142,9 +144,9 @@ Pipe ReadFromMemoryStorageStep::makePipe() storage_snapshot, nullptr /* data */, nullptr /* parallel execution index */, - [current_data](std::shared_ptr & data_to_initialize) + [storage = storage](std::shared_ptr & data_to_initialize) { - data_to_initialize = current_data; + data_to_initialize = static_cast(*storage).data.get(); })); } diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h index cec523ed58b..1122bfbb2a5 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h @@ -16,6 +16,7 @@ class ReadFromMemoryStorageStep final : public SourceStepWithFilter { public: ReadFromMemoryStorageStep(const Names & columns_to_read_, + StoragePtr storage_, const StorageSnapshotPtr & storage_snapshot_, size_t num_streams_, bool delay_read_for_global_sub_queries_); @@ -35,6 +36,7 @@ private: static constexpr auto name = "ReadFromMemoryStorage"; Names columns_to_read; + StoragePtr storage; StorageSnapshotPtr storage_snapshot; size_t num_streams; bool delay_read_for_global_sub_queries; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 6626d4b9795..24e17ff210f 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -27,11 +27,11 @@ CreatingSetsTransform::CreatingSetsTransform( Block out_header_, SubqueryForSet subquery_for_set_, SizeLimits network_transfer_limits_, - ContextPtr context_) + PreparedSetsCachePtr prepared_sets_cache_) : IAccumulatingTransform(std::move(in_header_), std::move(out_header_)) - , WithContext(context_) , subquery(std::move(subquery_for_set_)) , network_transfer_limits(std::move(network_transfer_limits_)) + , prepared_sets_cache(std::move(prepared_sets_cache_)) { } @@ -52,14 +52,13 @@ void CreatingSetsTransform::work() void CreatingSetsTransform::startSubquery() { /// Lookup the set in the cache if we don't need to build table. - auto ctx = context.lock(); - if (ctx && ctx->getPreparedSetsCache() && !subquery.table) + if (prepared_sets_cache && !subquery.table) { /// Try to find the set in the cache and wait for it to be built. /// Retry if the set from cache fails to be built. while (true) { - auto from_cache = ctx->getPreparedSetsCache()->findOrPromiseToBuild(subquery.key); + auto from_cache = prepared_sets_cache->findOrPromiseToBuild(subquery.key); if (from_cache.index() == 0) { promise_to_build = std::move(std::get<0>(from_cache)); @@ -89,9 +88,11 @@ void CreatingSetsTransform::startSubquery() if (subquery.table) LOG_TRACE(log, "Filling temporary table."); + // std::cerr << StackTrace().toString() << std::endl; + if (subquery.table) /// TODO: make via port - table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); + table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), nullptr)); done_with_set = !subquery.set; done_with_table = !subquery.table; diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index 26bbc45933d..ef586b0655c 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -23,7 +23,7 @@ class PushingPipelineExecutor; /// Don't return any data. Sets are created when Finish status is returned. /// In general, several work() methods need to be called to finish. /// Independent processors is created for each subquery. -class CreatingSetsTransform : public IAccumulatingTransform, WithContext +class CreatingSetsTransform : public IAccumulatingTransform { public: CreatingSetsTransform( @@ -31,7 +31,7 @@ public: Block out_header_, SubqueryForSet subquery_for_set_, SizeLimits network_transfer_limits_, - ContextPtr context_); + PreparedSetsCachePtr prepared_sets_cache_); ~CreatingSetsTransform() override; @@ -55,6 +55,7 @@ private: bool done_with_table = true; SizeLimits network_transfer_limits; + PreparedSetsCachePtr prepared_sets_cache; size_t rows_to_transfer = 0; size_t bytes_to_transfer = 0; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 764997e7b7e..4898501b980 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -569,7 +569,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe return left; } -void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context) +void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache) { resize(1); @@ -578,7 +578,7 @@ void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, Su res_header, std::move(subquery_for_set), limits, - context); + std::move(prepared_sets_cache)); InputPort * totals_port = nullptr; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 3a5d65d4388..85b6f5c6772 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -33,6 +33,9 @@ class TableJoin; class QueryPipelineBuilder; using QueryPipelineBuilderPtr = std::unique_ptr; +class PreparedSetsCache; +using PreparedSetsCachePtr = std::shared_ptr; + class QueryPipelineBuilder { public: @@ -138,7 +141,7 @@ public: /// This is used for CreatingSets. void addPipelineBefore(QueryPipelineBuilder pipeline); - void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context); + void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache); PipelineExecutorPtr execute(); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 31e45db55cb..1392ed5c684 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -155,7 +155,7 @@ void StorageMemory::read( size_t /*max_block_size*/, size_t num_streams) { - query_plan.addStep(std::make_unique(column_names, storage_snapshot, num_streams, delay_read_for_global_subqueries)); + query_plan.addStep(std::make_unique(column_names, shared_from_this(), storage_snapshot, num_streams, delay_read_for_global_subqueries)); } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index ce8a59b8bcd..31c8ee31f8b 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -132,6 +132,8 @@ private: std::atomic total_size_rows = 0; bool compress; + + friend class ReadFromMemoryStorageStep; }; } From de3b08aa5b6308d51ef62bc386f322ee27ad1e55 Mon Sep 17 00:00:00 2001 From: johanngan Date: Thu, 25 May 2023 14:35:24 -0500 Subject: [PATCH 0690/2223] Clean up regexp tree dictionary documentation dictGetOrNull() relies on IDictionary::hasKeys(), which RegExpTreeDictionary doesn't implement, so this probably never worked. If you try to use it, an exception is thrown. The docs shouldn't indicate that this is supported. Also fix a markdown hyperlink in the docs. --- docs/en/sql-reference/dictionaries/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 65f1a81f849..f7b4be64851 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2234,7 +2234,7 @@ Result: ## Regular Expression Tree Dictionary {#regexp-tree-dictionary} -Regular expression tree dictionaries are a special type of dictionary which represent the mapping from key to attributes using a tree of regular expressions. There are some use cases, e.g. parsing of (user agent)[https://en.wikipedia.org/wiki/User_agent] strings, which can be expressed elegantly with regexp tree dictionaries. +Regular expression tree dictionaries are a special type of dictionary which represent the mapping from key to attributes using a tree of regular expressions. There are some use cases, e.g. parsing of [user agent](https://en.wikipedia.org/wiki/User_agent) strings, which can be expressed elegantly with regexp tree dictionaries. ### Use Regular Expression Tree Dictionary in ClickHouse Open-Source @@ -2280,7 +2280,7 @@ This config consists of a list of regular expression tree nodes. Each node has t - The value of an attribute may contain **back references**, referring to capture groups of the matched regular expression. In the example, the value of attribute `version` in the first node consists of a back-reference `\1` to capture group `(\d+[\.\d]*)` in the regular expression. Back-reference numbers range from 1 to 9 and are written as `$1` or `\1` (for number 1). The back reference is replaced by the matched capture group during query execution. - **child nodes**: a list of children of a regexp tree node, each of which has its own attributes and (potentially) children nodes. String matching proceeds in a depth-first fashion. If a string matches a regexp node, the dictionary checks if it also matches the nodes' child nodes. If that is the case, the attributes of the deepest matching node are assigned. Attributes of a child node overwrite equally named attributes of parent nodes. The name of child nodes in YAML files can be arbitrary, e.g. `versions` in above example. -Regexp tree dictionaries only allow access using functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`. +Regexp tree dictionaries only allow access using the functions `dictGet` and `dictGetOrDefault`. Example: From 36e8f13242edfd83e3650369b6b71bc9e8fc2e64 Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Thu, 25 May 2023 20:10:02 +0000 Subject: [PATCH 0691/2223] Added docs for feature --- docs/en/operations/settings/settings.md | 26 +++++++++++++++++++ docs/en/sql-reference/table-functions/file.md | 5 ++-- docs/ru/operations/settings/settings.md | 23 ++++++++++++++++ docs/ru/sql-reference/table-functions/file.md | 1 + 4 files changed, 53 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2239084a429..df9e8eb2fe2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4200,6 +4200,32 @@ Possible values: Default value: `false`. +## rename_files_after_processing + +- **Type:** String + +- **Default value:** Empty string + +This setting allows to specify renaming pattern for files processed by `file` table function. When option is set, all files read by `file` table function will be renamed according to specified pattern with placeholders, only if files processing was successful. + +### Placeholders + +- `%f` — Original filename without extension (e.g., "sample"). +- `%e` — Original file extension with dot (e.g., ".csv"). +- `%t` — Timestamp (in microseconds). +- `%%` — Percentage sign ("%"). + +### Example +- Option: `--rename_files_after_processing="processed_%f_%t%e"` + +- Query: `SELECT * FROM file('sample.csv')` + + +If reading `sample.csv` is successful, file will be renamed to `processed_sample_1683473210851438.csv` + + + + ## function_json_value_return_type_allow_complex Control whether allow to return complex type (such as: struct, array, map) for json_value function. diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 28c2dc9f1f3..577e2e6aa1d 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -40,7 +40,7 @@ VALUES (1, 2, 3), (3, 2, 1), (1, 3, 2) As a result, the data is written into the file `test.tsv`: ```bash -# cat /var/lib/clickhouse/user_files/test.tsv +# cat /var/lib/clickhouse/user_files/test.tsv 1 2 3 3 2 1 1 3 2 @@ -163,7 +163,7 @@ Query the number of rows in all files of these two directories: SELECT count(*) FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32'); ``` -:::note +:::note If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: @@ -199,3 +199,4 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 **See Also** - [Virtual columns](/docs/en/engines/table-engines/index.md#table_engines-virtual_columns) +- [Rename files after processing](/docs/en/operations/settings/settings.md#rename_files_after_processing) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index fa3ea582c55..7cab607bb3b 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4064,3 +4064,26 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca Возможные значения:: `true`, `false` Значение по умолчанию: `false` + +## rename_files_after_processing + +- **Тип:** Строка + +- **Значение по умолчанию:** Пустая строка + +Этот параметр позволяет задать паттерн для переименования файлов, обрабатываемых табличной функцией `file`. Когда опция установлена, все файлы, прочитанные табличной функцией `file`, будут переименованы в соответствии с указанным шаблоном, если обработка и чтение файла завершились успешно. + +### Шаблон +Шаблон поддерживает следующие виды плейсхолдеров: + +- `%f` — Исходное имя файла без расширения (например "sample"). +- `%e` — Оригинальное расширение файла с точкой (например ".csv"). +- `%t` — Текущее время (в микросекундах). +- `%%` — Знак процента ("%"). + +### Пример +- Значение аргумента: `--rename_files_after_processing="processed_%f_%t%e"` + +- Запрос: `SELECT * FROM file('sample.csv')` + +Если чтение и обработка `sample.csv` прошли успешно, файл будет переименован в `processed_sample_1683473210851438.csv`. diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 94bc734a8fb..0983c51d954 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -126,3 +126,4 @@ SELECT count(*) FROM file('big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, **Смотрите также** - [Виртуальные столбцы](index.md#table_engines-virtual_columns) +- [Переименование файлов после обработки](/docs/ru/operations/settings/settings.md#rename_files_after_processing) From 42e1e3ae208ed6488b7f30e6e87c88be38b07b17 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 May 2023 20:24:03 +0000 Subject: [PATCH 0692/2223] Fix working with secure socket after async connection --- src/Client/Connection.cpp | 5 ----- src/IO/ReadBufferFromPocoSocket.cpp | 28 +++++++++++++++++++++------- src/IO/ReadBufferFromPocoSocket.h | 2 ++ src/IO/WriteBufferFromPocoSocket.cpp | 27 +++++++++++++++++++++------ src/IO/WriteBufferFromPocoSocket.h | 2 ++ 5 files changed, 46 insertions(+), 18 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index d39148d3016..451d29d4091 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -138,11 +138,6 @@ void Connection::connect(const ConnectionTimeouts & timeouts) socket->impl()->error(err); // Throws an exception socket->setBlocking(true); - -#if USE_SSL - if (static_cast(secure)) - static_cast(socket.get())->completeHandshake(); -#endif } else { diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index d0fba2c28e8..4ceba347707 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -49,16 +50,18 @@ bool ReadBufferFromPocoSocket::nextImpl() { CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive); - /// If async_callback is specified, and read will block, run async_callback and try again later. - /// It is expected that file descriptor may be polled externally. - /// Note that receive timeout is not checked here. External code should check it while polling. - while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR)) - async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); - if (internal_buffer.size() > INT_MAX) throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + bytes_read = readFromSocket(); + + /// In case of non-blocking connect for secure socket receiveBytes can return ERR_SSL_WANT_READ, + /// in this case we should call receiveBytes again when socket is ready. + if (socket.secure()) + { + while (bytes_read == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ) + bytes_read = readFromSocket(); + } } catch (const Poco::Net::NetException & e) { @@ -86,6 +89,17 @@ bool ReadBufferFromPocoSocket::nextImpl() return true; } +ssize_t ReadBufferFromPocoSocket::readFromSocket() +{ + /// If async_callback is specified, and read will block, run async_callback and try again later. + /// It is expected that file descriptor may be polled externally. + /// Note that receive timeout is not checked here. External code should check it while polling. + while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR)) + async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); + + return socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); +} + ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index dab4ac86295..3c4bc424334 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -30,6 +30,8 @@ public: void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } private: + ssize_t readFromSocket(); + AsyncCallback async_callback; std::string socket_description; }; diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 039110dfb62..27fc78e5fe3 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace ProfileEvents @@ -62,13 +63,15 @@ void WriteBufferFromPocoSocket::nextImpl() if (size > INT_MAX) throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - /// If async_callback is specified, and write will block, run async_callback and try again later. - /// It is expected that file descriptor may be polled externally. - /// Note that send timeout is not checked here. External code should check it while polling. - while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_WRITE | Poco::Net::Socket::SELECT_ERROR)) - async_callback(socket.impl()->sockfd(), socket.getSendTimeout(), AsyncEventTimeoutType::SEND, socket_description, AsyncTaskExecutor::Event::WRITE | AsyncTaskExecutor::Event::ERROR); + res = writeToSocket(pos, size); - res = socket.impl()->sendBytes(pos, static_cast(size)); + /// In case of non-blocking connect for secure socket sendBytes can return ERR_SSL_WANT_READ, + /// in this case we should call sendBytes again when socket is ready. + if (socket.secure()) + { + while (res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE) + res = writeToSocket(pos, size); + } } catch (const Poco::Net::NetException & e) { @@ -95,6 +98,18 @@ void WriteBufferFromPocoSocket::nextImpl() } } +ssize_t WriteBufferFromPocoSocket::writeToSocket(char * data, size_t size) +{ + /// If async_callback is specified, and write will block, run async_callback and try again later. + /// It is expected that file descriptor may be polled externally. + /// Note that send timeout is not checked here. External code should check it while polling. + while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_WRITE | Poco::Net::Socket::SELECT_ERROR)) + async_callback(socket.impl()->sockfd(), socket.getSendTimeout(), AsyncEventTimeoutType::SEND, socket_description, AsyncTaskExecutor::Event::WRITE | AsyncTaskExecutor::Event::ERROR); + + return socket.impl()->sendBytes(data, static_cast(size)); + +} + WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) diff --git a/src/IO/WriteBufferFromPocoSocket.h b/src/IO/WriteBufferFromPocoSocket.h index ecb61020357..0f03e816af5 100644 --- a/src/IO/WriteBufferFromPocoSocket.h +++ b/src/IO/WriteBufferFromPocoSocket.h @@ -35,6 +35,8 @@ protected: Poco::Net::SocketAddress our_address; private: + ssize_t writeToSocket(char * data, size_t size); + AsyncCallback async_callback; std::string socket_description; }; From 1964d1bb7e55c5827837aa4ac083da3c2c8f39db Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 25 May 2023 22:30:16 +0200 Subject: [PATCH 0693/2223] Fix comment --- src/IO/WriteBufferFromPocoSocket.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 27fc78e5fe3..ed58e37ddee 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -65,7 +65,7 @@ void WriteBufferFromPocoSocket::nextImpl() res = writeToSocket(pos, size); - /// In case of non-blocking connect for secure socket sendBytes can return ERR_SSL_WANT_READ, + /// In case of non-blocking connect for secure socket sendBytes can return ERR_SSL_WANT_WRITE, /// in this case we should call sendBytes again when socket is ready. if (socket.secure()) { From 4f0aeee2364cbfb118742774307a3a619e53518c Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 25 May 2023 20:49:17 +0000 Subject: [PATCH 0694/2223] fix more conflicts --- src/Disks/IO/AsynchronousBoundedReadBuffer.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.h b/src/Disks/IO/AsynchronousBoundedReadBuffer.h index 45256cdfac2..cafc5b6d691 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.h +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.h @@ -39,7 +39,7 @@ public: off_t seek(off_t offset_, int whence) override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; void setReadUntilPosition(size_t position) override; /// [..., position). @@ -72,7 +72,7 @@ private: struct LastPrefetchInfo { UInt64 submit_time = 0; - size_t priority = 0; + Priority priority; }; LastPrefetchInfo last_prefetch_info; @@ -87,7 +87,7 @@ private: int64_t size, const std::unique_ptr & execution_watch); - std::future asyncReadInto(char * data, size_t size, int64_t priority); + std::future asyncReadInto(char * data, size_t size, Priority priority); void resetPrefetch(FilesystemPrefetchState state); From 613568423d7b34a80d9c5a1688865122f1136a07 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuk Date: Fri, 26 May 2023 07:49:45 +1000 Subject: [PATCH 0695/2223] Update src/Processors/Formats/Impl/CSVRowInputFormat.cpp Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 9922bd41442..6593567a581 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -282,15 +282,7 @@ bool CSVFormatReader::readField( bool is_last_file_column, const String & /*column_name*/) { - if (format_settings.csv.trim_whitespaces) [[likely]] - skipWhitespacesAndTabs(*buf); - else if (type->isNullable()) - { - auto nested_type = typeid_cast(type.get())->getNestedType(); - if (!isStringOrFixedString(nested_type)) - skipWhitespacesAndTabs(*buf); - } - else if (!isStringOrFixedString(type)) + if (format_settings.csv.trim_whitespaces || !isStringOrFixedString(RemoveNullable(type))) [[likely]] skipWhitespacesAndTabs(*buf); const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; From a9082b24b40a9b09ce057836b4b5d55bf7bb491b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 May 2023 00:20:20 +0200 Subject: [PATCH 0696/2223] Fix build --- src/IO/ReadBufferFromPocoSocket.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 4ceba347707..11acd3a9136 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -9,7 +9,10 @@ #include #include #include + +#if USE_SSL #include +#endif namespace ProfileEvents { From 67b78829fcc74407418db922f1ef58cfe8e1b6ad Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 May 2023 00:21:14 +0200 Subject: [PATCH 0697/2223] Fix build --- src/IO/ReadBufferFromPocoSocket.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 11acd3a9136..c051478afc5 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -58,6 +58,7 @@ bool ReadBufferFromPocoSocket::nextImpl() bytes_read = readFromSocket(); +#if USE_SSL /// In case of non-blocking connect for secure socket receiveBytes can return ERR_SSL_WANT_READ, /// in this case we should call receiveBytes again when socket is ready. if (socket.secure()) @@ -65,6 +66,7 @@ bool ReadBufferFromPocoSocket::nextImpl() while (bytes_read == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ) bytes_read = readFromSocket(); } +#endif } catch (const Poco::Net::NetException & e) { From 6b98e028576ac8564336fab1a9021482a5129739 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 25 May 2023 19:21:32 -0300 Subject: [PATCH 0698/2223] Update settings.md --- .../settings.md | 26 +++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index e3ca04f5b9b..99630043095 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -917,9 +917,9 @@ We recommend using this option in macOS since the `getrlimit()` function returns Restriction on deleting tables. -If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_table_size_to_drop` (in bytes), you can’t delete it using a DROP query. +If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_table_size_to_drop` (in bytes), you can’t delete it using a [DROP](sql-reference/statements/drop.md) query or [TRUNCATE](../../sql-reference/statements/truncate.md) query. -If you still need to delete the table without restarting the ClickHouse server, create the `/flags/force_drop_table` file and run the DROP query. +This setting does not require a restart of the Clickhouse server to apply. Another way to disable the restriction is to create the `/flags/force_drop_table` file. Default value: 50 GB. @@ -931,6 +931,28 @@ The value 0 means that you can delete all tables without any restrictions. 0 ``` +## max_partition_size_to_drop {#max-partition-size-to-drop} + +Restriction on dropping partitions. + +If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_partition_size_to_drop` (in bytes), you can’t drop a partition using a [DROP PARTITION](../../sql-reference/statements/alter/partition.md#drop-partitionpart) query. + +This setting does not require a restart of the Clickhouse server to apply. Another way to disable the restriction is to create the `/flags/force_drop_table` file. + +Default value: 50 GB. + +The value 0 means that you can drop partitions without any restrictions. + +:::note +This limitation does not restrict drop table and truncate table, see [max_table_size_to_drop](#max_table_size_to_drop) +::: + +**Example** + +``` xml +0 +``` + ## max_thread_pool_size {#max-thread-pool-size} ClickHouse uses threads from the Global Thread pool to process queries. If there is no idle thread to process a query, then a new thread is created in the pool. `max_thread_pool_size` limits the maximum number of threads in the pool. From f03ca41b081630061f57d3c3b1b1938c4ddcb759 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 May 2023 00:21:46 +0200 Subject: [PATCH 0699/2223] Fix build --- src/IO/WriteBufferFromPocoSocket.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index ed58e37ddee..e566c228534 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -10,8 +10,10 @@ #include #include #include -#include +#if USE_SSL +#include +#endif namespace ProfileEvents { @@ -65,6 +67,7 @@ void WriteBufferFromPocoSocket::nextImpl() res = writeToSocket(pos, size); +#if USE_SSL /// In case of non-blocking connect for secure socket sendBytes can return ERR_SSL_WANT_WRITE, /// in this case we should call sendBytes again when socket is ready. if (socket.secure()) @@ -72,6 +75,7 @@ void WriteBufferFromPocoSocket::nextImpl() while (res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE) res = writeToSocket(pos, size); } +#endif } catch (const Poco::Net::NetException & e) { From 01f3a46cf06d168452a8cbbca76b1c083355d63b Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 25 May 2023 22:49:36 +0000 Subject: [PATCH 0700/2223] fixed wrong case in removeNullable --- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 6593567a581..8b4dbbffe1d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -282,7 +282,7 @@ bool CSVFormatReader::readField( bool is_last_file_column, const String & /*column_name*/) { - if (format_settings.csv.trim_whitespaces || !isStringOrFixedString(RemoveNullable(type))) [[likely]] + if (format_settings.csv.trim_whitespaces || !isStringOrFixedString(removeNullable(type))) [[likely]] skipWhitespacesAndTabs(*buf); const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; From 612173e734fc72d86640e8e0c9da3eb3967e079a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 25 May 2023 22:54:54 +0000 Subject: [PATCH 0701/2223] refactoring near alter conversions --- .../ExecuteScalarSubqueriesVisitor.cpp | 46 ++-- .../ExecuteScalarSubqueriesVisitor.h | 4 + src/Interpreters/InterpreterAlterQuery.cpp | 7 +- src/Interpreters/InterpreterDeleteQuery.cpp | 3 +- src/Interpreters/InterpreterSelectQuery.cpp | 5 +- src/Interpreters/MutationsInterpreter.cpp | 205 ++++++------------ src/Interpreters/MutationsInterpreter.h | 50 +++-- .../MutationsNonDeterministicHelpers.cpp | 100 +++++++++ .../MutationsNonDeterministicHelpers.h | 21 ++ src/Interpreters/TreeRewriter.cpp | 6 +- src/Interpreters/executeQuery.cpp | 10 +- .../optimizeUseAggregateProjection.cpp | 5 +- .../optimizeUseNormalProjection.cpp | 20 +- .../Optimizations/projectionsCommon.cpp | 2 +- src/Processors/QueryPlan/PartsSplitter.cpp | 4 + .../QueryPlan/ReadFromMergeTree.cpp | 29 ++- src/Processors/QueryPlan/ReadFromMergeTree.h | 17 +- src/Storages/IStorage.h | 6 + src/Storages/MergeTree/AlterConversions.cpp | 9 +- src/Storages/MergeTree/AlterConversions.h | 26 ++- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 + .../IMergeTreeDataPartInfoForReader.h | 2 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 7 +- src/Storages/MergeTree/IMergeTreeReader.h | 6 +- .../LoadedMergeTreeDataPartInfoForReader.h | 12 +- src/Storages/MergeTree/MergeTask.cpp | 11 +- .../MergeTreeBaseSelectProcessor.cpp | 110 +++++----- .../MergeTree/MergeTreeBaseSelectProcessor.h | 24 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 78 ++++--- .../MergeTree/MergeTreeBlockReadUtils.h | 7 +- src/Storages/MergeTree/MergeTreeData.cpp | 41 ++-- src/Storages/MergeTree/MergeTreeData.h | 5 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 3 +- .../MergeTree/MergeTreeDataPartCompact.h | 1 + .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 +- .../MergeTree/MergeTreeDataPartInMemory.h | 1 + .../MergeTree/MergeTreeDataPartWide.cpp | 3 +- .../MergeTree/MergeTreeDataPartWide.h | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 64 ++++-- .../MergeTree/MergeTreeDataSelectExecutor.h | 6 +- .../MergeTreeInOrderSelectProcessor.cpp | 7 +- .../MergeTree/MergeTreeIndexReader.cpp | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 36 +-- .../MergeTree/MergeTreePrefetchedReadPool.h | 1 + .../MergeTree/MergeTreeRangeReader.cpp | 180 ++++++++------- src/Storages/MergeTree/MergeTreeRangeReader.h | 22 +- src/Storages/MergeTree/MergeTreeReadPool.cpp | 24 +- src/Storages/MergeTree/MergeTreeReadPool.h | 3 +- .../MergeTreeReverseSelectProcessor.cpp | 18 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 13 +- .../MergeTree/MergeTreeSelectProcessor.h | 6 +- .../MergeTree/MergeTreeSequentialSource.cpp | 23 +- src/Storages/MergeTree/MergeTreeSink.cpp | 2 +- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 15 +- .../MergeTreeThreadSelectProcessor.cpp | 2 - src/Storages/MergeTree/MutateTask.cpp | 30 ++- src/Storages/MergeTree/RangesInDataPart.h | 4 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 22 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 6 +- src/Storages/MutationCommands.cpp | 1 - .../RocksDB/StorageEmbeddedRocksDB.cpp | 18 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageJoin.cpp | 3 +- src/Storages/StorageKeeperMap.cpp | 18 +- src/Storages/StorageMemory.cpp | 5 +- src/Storages/StorageMergeTree.cpp | 20 +- src/Storages/StorageMergeTree.h | 2 - src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageSnapshot.h | 21 +- .../01786_explain_merge_tree.reference | 10 +- 71 files changed, 887 insertions(+), 596 deletions(-) create mode 100644 src/Interpreters/MutationsNonDeterministicHelpers.cpp create mode 100644 src/Interpreters/MutationsNonDeterministicHelpers.h diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 3d5f14f4723..3d821b60e81 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -19,6 +19,8 @@ #include #include #include +#include +#include namespace ProfileEvents { @@ -68,17 +70,6 @@ void ExecuteScalarSubqueriesMatcher::visit(ASTPtr & ast, Data & data) visit(*t, ast, data); } -/// Converting to literal values might take a fair amount of overhead when the value is large, (e.g. -/// Array, BitMap, etc.), This conversion is required for constant folding, index lookup, branch -/// elimination. However, these optimizations should never be related to large values, thus we -/// blacklist them here. -static bool worthConvertingToLiteral(const Block & scalar) -{ - const auto * scalar_type_name = scalar.safeGetByPosition(0).type->getFamilyName(); - static const std::set useless_literal_types = {"Array", "Tuple", "AggregateFunction", "Function", "Set", "LowCardinality"}; - return !useless_literal_types.contains(scalar_type_name); -} - static auto getQueryInterpreter(const ASTSubquery & subquery, ExecuteScalarSubqueriesMatcher::Data & data) { auto subquery_context = Context::createCopy(data.getContext()); @@ -255,7 +246,9 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr const Settings & settings = data.getContext()->getSettingsRef(); // Always convert to literals when there is no query context. - if (data.only_analyze || !settings.enable_scalar_subquery_optimization || worthConvertingToLiteral(scalar) + if (data.only_analyze + || !settings.enable_scalar_subquery_optimization + || worthConvertingScalarToLiteral(scalar, data.max_literal_size) || !data.getContext()->hasQueryContext()) { /// subquery and ast can be the same object and ast will be moved. @@ -278,7 +271,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr ast = std::move(func); } } - else + else if (!data.replace_only_to_literals) { auto func = makeASTFunction("__getScalar", std::make_shared(scalar_query_hash_str)); func->alias = subquery.alias; @@ -318,4 +311,31 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTFunction & func, ASTPtr & as Visitor(data).visit(*add_node); } +static size_t getSizeOfSerializedLiteral(const Field & field) +{ + auto field_str = applyVisitor(FieldVisitorToString(), field); + return field_str.size(); +} + +bool worthConvertingScalarToLiteral(const Block & scalar, std::optional max_literal_size) +{ + /// Converting to literal values might take a fair amount of overhead when the value is large, (e.g. + /// Array, BitMap, etc.), This conversion is required for constant folding, index lookup, branch + /// elimination. However, these optimizations should never be related to large values, thus we blacklist them here. + const auto * scalar_type_name = scalar.safeGetByPosition(0).type->getFamilyName(); + static const std::set maybe_large_literal_types = {"Array", "Tuple", "AggregateFunction", "Function", "Set", "LowCardinality"}; + + if (!maybe_large_literal_types.contains(scalar_type_name)) + return true; + + if (!max_literal_size) + return false; + + /// Size of serialized literal cannot be less than size in bytes. + if (scalar.bytes() > *max_literal_size) + return false; + + return getSizeOfSerializedLiteral((*scalar.safeGetByPosition(0).column)[0]) <= *max_literal_size; +} + } diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.h b/src/Interpreters/ExecuteScalarSubqueriesVisitor.h index 577140f731f..7e2e06d8f8e 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.h +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.h @@ -37,6 +37,8 @@ public: Scalars & local_scalars; bool only_analyze; bool is_create_parameterized_view; + bool replace_only_to_literals; + std::optional max_literal_size; }; static bool needChildVisit(ASTPtr & node, const ASTPtr &); @@ -49,4 +51,6 @@ private: using ExecuteScalarSubqueriesVisitor = ExecuteScalarSubqueriesMatcher::Visitor; +bool worthConvertingScalarToLiteral(const Block & scalar, std::optional max_literal_size); + } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 21f0fbadd09..ec2145b38bf 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -8,12 +8,14 @@ #include #include #include +#include #include #include #include #include #include #include +#include #include #include #include @@ -67,7 +69,6 @@ BlockIO InterpreterAlterQuery::execute() throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown alter object type"); } - BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) { BlockIO res; @@ -156,7 +157,8 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (mutation_commands.hasNonEmptyMutationCommands()) { table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); + MutationsInterpreter::Settings settings(false); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), settings).validate(); table->mutate(mutation_commands, getContext()); } @@ -236,6 +238,7 @@ BlockIO InterpreterAlterQuery::executeToDatabase(const ASTAlterQuery & alter) return res; } + AccessRightsElements InterpreterAlterQuery::getRequiredAccess() const { AccessRightsElements required_access; diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index fabd1c38ce3..52723416400 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -72,7 +72,8 @@ BlockIO InterpreterDeleteQuery::execute() mutation_commands.emplace_back(mut_command); table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); - MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); + MutationsInterpreter::Settings settings(false); + MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), settings).validate(); table->mutate(mutation_commands, getContext()); return {}; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ebaf88ea5d5..b010b0c8f0f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -437,7 +437,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (!metadata_snapshot) metadata_snapshot = storage->getInMemoryMetadataPtr(); - storage_snapshot = storage->getStorageSnapshotForQuery(metadata_snapshot, query_ptr, context); + if (options.only_analyze) + storage_snapshot = storage->getStorageSnapshotWithoutData(metadata_snapshot, context); + else + storage_snapshot = storage->getStorageSnapshotForQuery(metadata_snapshot, query_ptr, context); } if (has_input || !joined_tables.resolveTables()) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 2f9e647b99e..713ebade1d5 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1,9 +1,9 @@ #include #include -#include #include #include #include +#include #include #include #include @@ -31,7 +31,6 @@ #include #include #include -#include #include #include #include @@ -53,90 +52,12 @@ namespace ErrorCodes extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int CANNOT_UPDATE_COLUMN; extern const int UNEXPECTED_EXPRESSION; + extern const int THERE_IS_NO_COLUMN; } namespace { -/// Helps to detect situations, where non-deterministic functions may be used in mutations of Replicated*MergeTree. -class FirstNonDeterministicFunctionMatcher -{ -public: - struct Data - { - ContextPtr context; - std::optional nondeterministic_function_name; - bool subquery = false; - }; - - static bool needChildVisit(const ASTPtr & /*node*/, const ASTPtr & /*child*/) - { - return true; - } - - static void visit(const ASTPtr & node, Data & data) - { - if (data.nondeterministic_function_name || data.subquery) - return; - - if (node->as()) - { - /// We cannot determine if subquery is deterministic or not, - /// so we do not allow to use subqueries in mutation without allow_nondeterministic_mutations=1 - data.subquery = true; - } - else if (const auto * function = typeid_cast(node.get())) - { - /// Property of being deterministic for lambda expression is completely determined - /// by the contents of its definition, so we just proceed to it. - if (function->name != "lambda") - { - /// NOTE It may be an aggregate function, so get(...) may throw. - /// However, an aggregate function can be used only in subquery and we do not go into subquery. - const auto func = FunctionFactory::instance().get(function->name, data.context); - if (!func->isDeterministic()) - data.nondeterministic_function_name = func->getName(); - } - } - } -}; - -using FirstNonDeterministicFunctionFinder = InDepthNodeVisitor; -using FirstNonDeterministicFunctionData = FirstNonDeterministicFunctionMatcher::Data; - -FirstNonDeterministicFunctionData findFirstNonDeterministicFunctionName(const MutationCommand & command, ContextPtr context) -{ - FirstNonDeterministicFunctionMatcher::Data finder_data{context, std::nullopt, false}; - - switch (command.type) - { - case MutationCommand::UPDATE: - { - auto update_assignments_ast = command.ast->as().update_assignments->clone(); - FirstNonDeterministicFunctionFinder(finder_data).visit(update_assignments_ast); - - if (finder_data.nondeterministic_function_name) - return finder_data; - - /// Currently UPDATE and DELETE both always have predicates so we can use fallthrough - [[fallthrough]]; - } - - case MutationCommand::DELETE: - { - auto predicate_ast = command.predicate->clone(); - FirstNonDeterministicFunctionFinder(finder_data).visit(predicate_ast); - - return finder_data; - } - - default: - break; - } - - return {}; -} - ASTPtr prepareQueryAffectedAST(const std::vector & commands, const StoragePtr & storage, ContextPtr context) { /// Execute `SELECT count() FROM storage WHERE predicate1 OR predicate2 OR ...` query. @@ -326,10 +247,10 @@ MutationsInterpreter::Source::Source(MergeTreeData & storage_, MergeTreeData::Da StorageSnapshotPtr MutationsInterpreter::Source::getStorageSnapshot(const StorageMetadataPtr & snapshot_, const ContextPtr & context_) const { - if (data) - return data->getStorageSnapshot(snapshot_, context_); + if (const auto * merge_tree = getMergeTreeData()) + return merge_tree->getStorageSnapshotWithoutData(snapshot_, context_); - return storage->getStorageSnapshot(snapshot_, context_); + return storage->getStorageSnapshotWithoutData(snapshot_, context_); } StoragePtr MutationsInterpreter::Source::getStorage() const @@ -367,20 +288,27 @@ bool MutationsInterpreter::Source::materializeTTLRecalculateOnly() const return data && data->getSettings()->materialize_ttl_recalculate_only; } +static Names getAvailableColumnsWithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage) +{ + auto all_columns = metadata_snapshot->getColumns().getNamesOfPhysical(); + for (const auto & column : storage.getVirtuals()) + all_columns.push_back(column.name); + return all_columns; +} + MutationsInterpreter::MutationsInterpreter( StoragePtr storage_, - const StorageMetadataPtr & metadata_snapshot_, + StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, ContextPtr context_, - bool can_execute_, - bool return_all_columns_, - bool return_mutated_rows_) + Settings settings_) : MutationsInterpreter( - Source(std::move(storage_)), - metadata_snapshot_, std::move(commands_), std::move(context_), - can_execute_, return_all_columns_, return_mutated_rows_) + Source(storage_), + metadata_snapshot_, std::move(commands_), + getAvailableColumnsWithVirtuals(metadata_snapshot_, *storage_), + std::move(context_), std::move(settings_)) { - if (can_execute_ && dynamic_cast(source.getStorage().get())) + if (settings.can_execute && dynamic_cast(source.getStorage().get())) { throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -392,37 +320,34 @@ MutationsInterpreter::MutationsInterpreter( MutationsInterpreter::MutationsInterpreter( MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, - const StorageMetadataPtr & metadata_snapshot_, + StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, + Names available_columns_, ContextPtr context_, - bool can_execute_, - bool return_all_columns_, - bool return_mutated_rows_) + Settings settings_) : MutationsInterpreter( Source(storage_, std::move(source_part_)), - metadata_snapshot_, std::move(commands_), std::move(context_), - can_execute_, return_all_columns_, return_mutated_rows_) + std::move(metadata_snapshot_), std::move(commands_), + std::move(available_columns_), std::move(context_), std::move(settings_)) { } MutationsInterpreter::MutationsInterpreter( Source source_, - const StorageMetadataPtr & metadata_snapshot_, + StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, + Names available_columns_, ContextPtr context_, - bool can_execute_, - bool return_all_columns_, - bool return_mutated_rows_) + Settings settings_) : source(std::move(source_)) , metadata_snapshot(metadata_snapshot_) , commands(std::move(commands_)) + , available_columns(std::move(available_columns_)) , context(Context::createCopy(context_)) - , can_execute(can_execute_) - , select_limits(SelectQueryOptions().analyze(!can_execute).ignoreLimits().ignoreProjections()) - , return_all_columns(return_all_columns_) - , return_mutated_rows(return_mutated_rows_) + , settings(std::move(settings_)) + , select_limits(SelectQueryOptions().analyze(!settings.can_execute).ignoreLimits().ignoreProjections()) { - prepare(!can_execute); + prepare(!settings.can_execute); } static NameSet getKeyColumns(const MutationsInterpreter::Source & source, const StorageMetadataPtr & metadata_snapshot) @@ -546,16 +471,18 @@ void MutationsInterpreter::prepare(bool dry_run) const ColumnsDescription & columns_desc = metadata_snapshot->getColumns(); const IndicesDescription & indices_desc = metadata_snapshot->getSecondaryIndices(); const ProjectionsDescription & projections_desc = metadata_snapshot->getProjections(); - NamesAndTypesList all_columns = columns_desc.getAllPhysical(); + + auto storage_snapshot = std::make_shared(*source.getStorage(), metadata_snapshot); + auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withVirtuals(); + + auto all_columns = storage_snapshot->getColumnsByNames(options, available_columns); + NameSet available_columns_set(available_columns.begin(), available_columns.end()); /// Add _row_exists column if it is physically present in the part if (source.hasLightweightDeleteMask()) - all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN}); - - if (return_all_columns) { - for (const auto & column : source.getStorage()->getVirtuals()) - all_columns.push_back(column); + all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN}); + available_columns_set.insert(LightweightDeleteDescription::FILTER_COLUMN.name); } NameSet updated_columns; @@ -567,9 +494,13 @@ void MutationsInterpreter::prepare(bool dry_run) || command.type == MutationCommand::Type::DELETE) materialize_ttl_recalculate_only = false; - for (const auto & kv : command.column_to_update_expression) + for (const auto & [name, _] : command.column_to_update_expression) { - updated_columns.insert(kv.first); + if (!available_columns_set.contains(name) && name != LightweightDeleteDescription::FILTER_COLUMN.name) + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, + "Column {} is updated but not requested to read", name); + + updated_columns.insert(name); } } @@ -580,29 +511,28 @@ void MutationsInterpreter::prepare(bool dry_run) { for (const auto & column : columns_desc) { - if (column.default_desc.kind == ColumnDefaultKind::Materialized) + if (column.default_desc.kind == ColumnDefaultKind::Materialized && available_columns_set.contains(column.name)) { auto query = column.default_desc.expression->clone(); auto syntax_result = TreeRewriter(context).analyze(query, all_columns); - for (const String & dependency : syntax_result->requiredSourceColumns()) - { + for (const auto & dependency : syntax_result->requiredSourceColumns()) if (updated_columns.contains(dependency)) column_to_affected_materialized[dependency].push_back(column.name); - } } } validateUpdateColumns(source, metadata_snapshot, updated_columns, column_to_affected_materialized); } - dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns); + if (settings.recalculate_dependencies_of_updated_columns) + dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns); std::vector read_columns; /// First, break a sequence of commands into stages. for (auto & command : commands) { // we can return deleted rows only if it's the only present command - assert(command.type == MutationCommand::DELETE || command.type == MutationCommand::UPDATE || !return_mutated_rows); + assert(command.type == MutationCommand::DELETE || command.type == MutationCommand::UPDATE || !settings.return_mutated_rows); if (command.type == MutationCommand::DELETE) { @@ -612,7 +542,7 @@ void MutationsInterpreter::prepare(bool dry_run) auto predicate = getPartitionAndPredicateExpressionForMutationCommand(command); - if (!return_mutated_rows) + if (!settings.return_mutated_rows) predicate = makeASTFunction("isZeroOrNull", predicate); stages.back().filters.push_back(predicate); @@ -700,7 +630,7 @@ void MutationsInterpreter::prepare(bool dry_run) stages.back().column_to_updated.emplace(column, updated_column); - if (condition && return_mutated_rows) + if (condition && settings.return_mutated_rows) stages.back().filters.push_back(condition); } @@ -909,17 +839,15 @@ void MutationsInterpreter::prepare(bool dry_run) } is_prepared = true; - prepareMutationStages(stages, dry_run); } void MutationsInterpreter::prepareMutationStages(std::vector & prepared_stages, bool dry_run) { auto storage_snapshot = source.getStorageSnapshot(metadata_snapshot, context); - auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); - if (return_all_columns) - options.withVirtuals(); - auto all_columns = storage_snapshot->getColumns(options); + auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects().withVirtuals(); + + auto all_columns = storage_snapshot->getColumnsByNames(options, available_columns); /// Add _row_exists column if it is present in the part if (source.hasLightweightDeleteMask()) @@ -928,7 +856,7 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s /// Next, for each stage calculate columns changed by this and previous stages. for (size_t i = 0; i < prepared_stages.size(); ++i) { - if (return_all_columns || !prepared_stages[i].filters.empty()) + if (settings.return_all_columns || !prepared_stages[i].filters.empty()) { for (const auto & column : all_columns) prepared_stages[i].output_columns.insert(column.name); @@ -1054,8 +982,7 @@ struct VirtualColumns { if (columns_to_read[i] == LightweightDeleteDescription::FILTER_COLUMN.name) { - LoadedMergeTreeDataPartInfoForReader part_info_reader(part); - if (!part_info_reader.getColumns().contains(LightweightDeleteDescription::FILTER_COLUMN.name)) + if (!part->getColumns().contains(LightweightDeleteDescription::FILTER_COLUMN.name)) { ColumnWithTypeAndName mask_column; mask_column.type = LightweightDeleteDescription::FILTER_COLUMN.type; @@ -1144,7 +1071,6 @@ void MutationsInterpreter::Source::read( ActionsDAGPtr filter; if (!first_stage.filter_column_names.empty()) { - ActionsDAG::NodeRawConstPtrs nodes(num_filters); for (size_t i = 0; i < num_filters; ++i) nodes[i] = &steps[i]->actions()->findInOutputs(names[i]); @@ -1155,7 +1081,9 @@ void MutationsInterpreter::Source::read( VirtualColumns virtual_columns(std::move(required_columns), part); createMergeTreeSequentialSource( - plan, *data, storage_snapshot, part, std::move(virtual_columns.columns_to_read), apply_deleted_mask_, filter, context_, + plan, *data, storage_snapshot, part, + std::move(virtual_columns.columns_to_read), + apply_deleted_mask_, filter, context_, &Poco::Logger::get("MutationsInterpreter")); virtual_columns.addVirtuals(plan); @@ -1208,7 +1136,7 @@ void MutationsInterpreter::Source::read( void MutationsInterpreter::initQueryPlan(Stage & first_stage, QueryPlan & plan) { - source.read(first_stage, plan, metadata_snapshot, context, apply_deleted_mask, can_execute); + source.read(first_stage, plan, metadata_snapshot, context, settings.apply_deleted_mask, settings.can_execute); addCreatingSetsStep(plan, first_stage.analyzer->getPreparedSets(), context); } @@ -1221,6 +1149,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v const auto & step = stage.expressions_chain.steps[i]; if (step->actions()->hasArrayJoin()) throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "arrayJoin is not allowed in mutations"); + if (i < stage.filter_column_names.size()) { /// Execute DELETEs. @@ -1253,15 +1182,13 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v void MutationsInterpreter::validate() { - const Settings & settings = context->getSettingsRef(); - /// For Replicated* storages mutations cannot employ non-deterministic functions /// because that produces inconsistencies between replicas - if (startsWith(source.getStorage()->getName(), "Replicated") && !settings.allow_nondeterministic_mutations) + if (startsWith(source.getStorage()->getName(), "Replicated") && !context->getSettingsRef().allow_nondeterministic_mutations) { for (const auto & command : commands) { - const auto nondeterministic_func_data = findFirstNonDeterministicFunctionName(command, context); + const auto nondeterministic_func_data = findFirstNonDeterministicFunction(command, context); if (nondeterministic_func_data.subquery) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ALTER UPDATE/ALTER DELETE statement with subquery may be nondeterministic, " "see allow_nondeterministic_mutations setting"); @@ -1281,7 +1208,7 @@ void MutationsInterpreter::validate() QueryPipelineBuilder MutationsInterpreter::execute() { - if (!can_execute) + if (!settings.can_execute) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot execute mutations interpreter because can_execute flag set to false"); QueryPlan plan; diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 8bb90365050..49ba07641d9 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -36,30 +36,44 @@ ASTPtr getPartitionAndPredicateExpressionForMutationCommand( /// to this data. class MutationsInterpreter { +private: struct Stage; public: + struct Settings + { + explicit Settings(bool can_execute_) : can_execute(can_execute_) {} + + /// If false only analyze mutation expressions. + bool can_execute = false; + /// Whether all columns should be returned, not just updated + bool return_all_columns = false; + /// Whether we should return mutated or all existing rows + bool return_mutated_rows = false; + /// Where we should filter deleted rows by lightweight DELETE. + bool apply_deleted_mask = true; + /// Where we should recalculate skip indexes, TTL expressions, etc. that depend on updated columns. + bool recalculate_dependencies_of_updated_columns = true; + }; + /// Storage to mutate, array of mutations commands and context. If you really want to execute mutation /// use can_execute = true, in other cases (validation, amount of commands) it can be false MutationsInterpreter( StoragePtr storage_, - const StorageMetadataPtr & metadata_snapshot_, + StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, ContextPtr context_, - bool can_execute_, - bool return_all_columns_ = false, - bool return_mutated_rows_ = false); + Settings settings_); /// Special case for *MergeTree MutationsInterpreter( MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, - const StorageMetadataPtr & metadata_snapshot_, + StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, + Names available_columns_, ContextPtr context_, - bool can_execute_, - bool return_all_columns_ = false, - bool return_mutated_rows_ = false); + Settings settings_); void validate(); size_t evaluateCommandsSize(); @@ -93,8 +107,6 @@ public: MutationKind::MutationKindEnum getMutationKind() const { return mutation_kind.mutation_kind; } - void setApplyDeletedMask(bool apply) { apply_deleted_mask = apply; } - /// Internal class which represents a data part for MergeTree /// or just storage for other storages. /// The main idea is to create a dedicated reading from MergeTree part. @@ -131,12 +143,11 @@ public: private: MutationsInterpreter( Source source_, - const StorageMetadataPtr & metadata_snapshot_, + StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, + Names available_columns_, ContextPtr context_, - bool can_execute_, - bool return_all_columns_, - bool return_mutated_rows_); + Settings settings_); void prepare(bool dry_run); @@ -151,12 +162,11 @@ private: Source source; StorageMetadataPtr metadata_snapshot; MutationCommands commands; + Names available_columns; ContextPtr context; - bool can_execute; + Settings settings; SelectQueryOptions select_limits; - bool apply_deleted_mask = true; - /// A sequence of mutation commands is executed as a sequence of stages. Each stage consists of several /// filters, followed by updating values of some columns. Commands can reuse expressions calculated by the /// previous commands in the same stage, but at the end of each stage intermediate columns are thrown away @@ -206,12 +216,6 @@ private: /// Columns, that we need to read for calculation of skip indices, projections or TTL expressions. ColumnDependencies dependencies; - - // whether all columns should be returned, not just updated - bool return_all_columns; - - // whether we should return mutated or all existing rows - bool return_mutated_rows; }; } diff --git a/src/Interpreters/MutationsNonDeterministicHelpers.cpp b/src/Interpreters/MutationsNonDeterministicHelpers.cpp new file mode 100644 index 00000000000..46c203e15bc --- /dev/null +++ b/src/Interpreters/MutationsNonDeterministicHelpers.cpp @@ -0,0 +1,100 @@ +#include "Parsers/IAST_fwd.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +/// Helps to detect situations, where non-deterministic functions may be used in mutations. +class FirstNonDeterministicFunctionMatcher +{ +public: + struct Data + { + ContextPtr context; + FirstNonDeterministicFunctionResult result; + }; + + static bool needChildVisit(const ASTPtr & /*node*/, const ASTPtr & /*child*/) + { + return true; + } + + static void visit(const ASTPtr & node, Data & data) + { + if (data.result.nondeterministic_function_name || data.result.subquery) + return; + + if (node->as()) + { + /// We cannot determine if subquery is deterministic or not, + /// so we do not allow to use subqueries in mutation without allow_nondeterministic_mutations=1 + data.result.subquery = true; + } + else if (const auto * function = typeid_cast(node.get())) + { + /// Property of being deterministic for lambda expression is completely determined + /// by the contents of its definition, so we just proceed to it. + if (function->name != "lambda") + { + /// NOTE It may be an aggregate function, so get(...) may throw. + /// However, an aggregate function can be used only in subquery and we do not go into subquery. + const auto func = FunctionFactory::instance().get(function->name, data.context); + if (!func->isDeterministic()) + data.result.nondeterministic_function_name = func->getName(); + } + } + } +}; + +using FirstNonDeterministicFunctionFinder = InDepthNodeVisitor; +using FirstNonDeterministicFunctionData = FirstNonDeterministicFunctionMatcher::Data; + +} + +FirstNonDeterministicFunctionResult findFirstNonDeterministicFunction(const MutationCommand & command, ContextPtr context) +{ + FirstNonDeterministicFunctionMatcher::Data finder_data{context, {}}; + + switch (command.type) + { + case MutationCommand::UPDATE: + { + auto update_assignments_ast = command.ast->as().update_assignments->clone(); + FirstNonDeterministicFunctionFinder(finder_data).visit(update_assignments_ast); + + if (finder_data.result.nondeterministic_function_name) + return finder_data.result; + + /// Currently UPDATE and DELETE both always have predicates so we can use fallthrough + [[fallthrough]]; + } + + case MutationCommand::DELETE: + { + auto predicate_ast = command.predicate->clone(); + FirstNonDeterministicFunctionFinder(finder_data).visit(predicate_ast); + return finder_data.result; + } + + default: + break; + } + + return {}; +} + +} diff --git a/src/Interpreters/MutationsNonDeterministicHelpers.h b/src/Interpreters/MutationsNonDeterministicHelpers.h new file mode 100644 index 00000000000..eb3a3b1c7a4 --- /dev/null +++ b/src/Interpreters/MutationsNonDeterministicHelpers.h @@ -0,0 +1,21 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +struct MutationCommand; + +struct FirstNonDeterministicFunctionResult +{ + std::optional nondeterministic_function_name; + bool subquery = false; +}; + +/// Searches for non-deterministic functions and subqueries which +/// may also be non-deterministic in expressions of mutation command. +FirstNonDeterministicFunctionResult findFirstNonDeterministicFunction(const MutationCommand & command, ContextPtr context); + +} diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 0824d360c7f..cda5ceeb164 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -455,7 +455,11 @@ void executeScalarSubqueries( ASTPtr & query, ContextPtr context, size_t subquery_depth, Scalars & scalars, Scalars & local_scalars, bool only_analyze, bool is_create_parameterized_view) { LogAST log; - ExecuteScalarSubqueriesVisitor::Data visitor_data{WithContext{context}, subquery_depth, scalars, local_scalars, only_analyze, is_create_parameterized_view}; + ExecuteScalarSubqueriesVisitor::Data visitor_data{ + WithContext{context}, subquery_depth, scalars, + local_scalars, only_analyze, is_create_parameterized_view, + /*replace_only_to_literals=*/ false, /*max_literal_size=*/ std::nullopt}; + ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query); } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 08753b9dfb7..6e13afa9f43 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -666,9 +666,13 @@ static std::tuple executeQueryImpl( interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); - if (context->getCurrentTransaction() && !interpreter->supportsTransactions() && - context->getSettingsRef().throw_on_unsupported_query_inside_transaction) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query ({})", ast->getID()); + const auto & query_settings = context->getSettingsRef(); + if (context->getCurrentTransaction() && query_settings.throw_on_unsupported_query_inside_transaction) + { + if (!interpreter->supportsTransactions()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query ({})", ast->getID()); + + } if (!interpreter->ignoreQuota() && !quota_checked) { diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 931e1ae1dd8..2959178b2e5 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -582,7 +582,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & ContextPtr context = reading->getContext(); MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); - auto ordinary_reading_select_result = reading->selectRangesToRead(parts); + auto ordinary_reading_select_result = reading->selectRangesToRead(parts, /* alter_conversions = */ {}); size_t ordinary_reading_marks = ordinary_reading_select_result->marks(); /// Selecting best candidate. @@ -640,7 +640,8 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & query_info_copy.prewhere_info = nullptr; projection_reading = reader.readFromParts( - {}, + /* parts = */ {}, + /* alter_conversions = */ {}, best_candidate->dag->getRequiredColumnsNames(), proj_snapshot, query_info_copy, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index eed3707fe9c..dd7a5d449bc 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -110,10 +110,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) return false; if (query.dag) - { query.dag->removeUnusedActions(); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", query.dag->dumpDAG()); - } } std::list candidates; @@ -125,12 +122,9 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) ContextPtr context = reading->getContext(); MergeTreeDataSelectExecutor reader(reading->getMergeTreeData()); - auto ordinary_reading_select_result = reading->selectRangesToRead(parts); + auto ordinary_reading_select_result = reading->selectRangesToRead(parts, /* alter_conversions = */ {}); size_t ordinary_reading_marks = ordinary_reading_select_result->marks(); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), - // "Marks for ordinary reading {}", ordinary_reading_marks); - std::shared_ptr max_added_blocks = getMaxAddedBlocks(reading); for (const auto * projection : normal_projections) @@ -152,9 +146,6 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!analyzed) continue; - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), - // "Marks for projection {} {}", projection->name ,candidate.sum_marks); - if (candidate.sum_marks >= ordinary_reading_marks) continue; @@ -173,14 +164,12 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data); proj_snapshot->addProjection(best_candidate->projection); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}", - // proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString()); - auto query_info_copy = query_info; query_info_copy.prewhere_info = nullptr; auto projection_reading = reader.readFromParts( - {}, + /*parts=*/ {}, + /*alter_conversions=*/ {}, required_columns, proj_snapshot, query_info_copy, @@ -201,9 +190,6 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (has_ordinary_parts) reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr)); - // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}", - // projection_reading->getOutputStream().header.dumpStructure()); - projection_reading->setStepDescription(best_candidate->projection->name); auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)}); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 9252137f649..2f73e14b2a0 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -248,7 +248,7 @@ bool analyzeProjectionCandidate( if (!normal_parts.empty()) { - auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts)); + auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts), /* alter_conversions = */ {}); if (normal_result_ptr->error()) return false; diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 3024a79bcbb..936182f8c00 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -142,8 +142,10 @@ std::pair, std::vector> split(RangesInDat { result_layers.back().emplace_back( parts[part_idx].data_part, + parts[part_idx].alter_conversions, parts[part_idx].part_index_in_query, MarkRanges{{current_part_range_begin[part_idx], current.range.end}}); + current_part_range_begin.erase(part_idx); current_part_range_end.erase(part_idx); continue; @@ -170,8 +172,10 @@ std::pair, std::vector> split(RangesInDat { result_layers.back().emplace_back( parts[part_idx].data_part, + parts[part_idx].alter_conversions, parts[part_idx].part_index_in_query, MarkRanges{{current_part_range_begin[part_idx], last_mark + 1}}); + current_part_range_begin[part_idx] = current_part_range_end[part_idx]; } } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index b38c3422be1..2415507a6eb 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -171,6 +171,7 @@ void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, c ReadFromMergeTree::ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, + std::vector alter_conversions_, Names real_column_names_, Names virt_column_names_, const MergeTreeData & data_, @@ -191,6 +192,7 @@ ReadFromMergeTree::ReadFromMergeTree( virt_column_names_)}) , reader_settings(getMergeTreeReaderSettings(context_, query_info_)) , prepared_parts(std::move(parts_)) + , alter_conversions_for_parts(std::move(alter_conversions_)) , real_column_names(std::move(real_column_names_)) , virt_column_names(std::move(virt_column_names_)) , data(data_) @@ -307,8 +309,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( reader_settings, required_columns, virt_column_names, - min_marks_for_concurrent_read - ); + min_marks_for_concurrent_read); Pipes pipes; const auto & settings = context->getSettingsRef(); @@ -459,7 +460,7 @@ ProcessorPtr ReadFromMergeTree::createSource( bool set_rows_approx = !is_parallel_reading_from_replicas && !reader_settings.read_in_order; auto algorithm = std::make_unique( - data, storage_snapshot, part.data_part, max_block_size, preferred_block_size_bytes, + data, storage_snapshot, part.data_part, part.alter_conversions, max_block_size, preferred_block_size_bytes, preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info, actions_settings, reader_settings, pool, virt_column_names, part.part_index_in_query, has_limit_below_one_block); @@ -808,7 +809,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( } ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); - new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); + new_parts.emplace_back(part.data_part, part.alter_conversions, part.part_index_in_query, std::move(ranges_to_get_from_part)); } splitted_parts_and_ranges.emplace_back(std::move(new_parts)); @@ -1000,7 +1001,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( { for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) { - new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); + new_parts.emplace_back(part_it->data_part, part_it->alter_conversions, part_it->part_index_in_query, part_it->ranges); } } @@ -1111,10 +1112,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( return Pipe::unitePipes(std::move(partition_pipes)); } -MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(MergeTreeData::DataPartsVector parts) const +MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( + MergeTreeData::DataPartsVector parts, + std::vector alter_conversions) const { return selectRangesToRead( std::move(parts), + std::move(alter_conversions), prewhere_info, filter_nodes, storage_snapshot->metadata, @@ -1131,6 +1135,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( MergeTreeData::DataPartsVector parts, + std::vector alter_conversions, const PrewhereInfoPtr & prewhere_info, const ActionDAGNodes & added_filter_nodes, const StorageMetadataPtr & metadata_snapshot_base, @@ -1182,7 +1187,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( updated_query_info_with_filter_dag.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); return selectRangesToReadImpl( - parts, + std::move(parts), + std::move(alter_conversions), metadata_snapshot_base, metadata_snapshot, updated_query_info_with_filter_dag, @@ -1196,7 +1202,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( } return selectRangesToReadImpl( - parts, + std::move(parts), + std::move(alter_conversions), metadata_snapshot_base, metadata_snapshot, query_info, @@ -1211,6 +1218,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( MergeTreeData::DataPartsVector parts, + std::vector alter_conversions, const StorageMetadataPtr & metadata_snapshot_base, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -1284,6 +1292,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( { MergeTreeDataSelectExecutor::filterPartsByPartition( parts, + alter_conversions, part_values, metadata_snapshot_base, data, @@ -1321,6 +1330,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( std::move(parts), + std::move(alter_conversions), metadata_snapshot, query_info, context, @@ -1491,7 +1501,7 @@ bool ReadFromMergeTree::requestOutputEachPartitionThroughSeparatePort() ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const { - auto result_ptr = analyzed_result_ptr ? analyzed_result_ptr : selectRangesToRead(prepared_parts); + auto result_ptr = analyzed_result_ptr ? analyzed_result_ptr : selectRangesToRead(prepared_parts, alter_conversions_for_parts); if (std::holds_alternative(result_ptr->result)) std::rethrow_exception(std::get(result_ptr->result)); @@ -1720,7 +1730,6 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); - pipeline.init(std::move(pipe)); // Attach QueryIdHolder if needed if (query_id_holder) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 545ffe84e0c..45beaaaf013 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -97,6 +98,7 @@ public: ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, + std::vector alter_conversions_, Names real_column_names_, Names virt_column_names_, const MergeTreeData & data_, @@ -134,6 +136,7 @@ public: static MergeTreeDataSelectAnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, + std::vector alter_conversions, const PrewhereInfoPtr & prewhere_info, const ActionDAGNodes & added_filter_nodes, const StorageMetadataPtr & metadata_snapshot_base, @@ -147,7 +150,9 @@ public: bool sample_factor_column_queried, Poco::Logger * log); - MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; + MergeTreeDataSelectAnalysisResultPtr selectRangesToRead( + MergeTreeData::DataPartsVector parts, + std::vector alter_conversions) const; ContextPtr getContext() const { return context; } const SelectQueryInfo & getQueryInfo() const { return query_info; } @@ -168,7 +173,12 @@ public: bool hasAnalyzedResult() const { return analyzed_result_ptr != nullptr; } void setAnalyzedResult(MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } - void resetParts(MergeTreeData::DataPartsVector parts) { prepared_parts = std::move(parts); } + + void resetParts(MergeTreeData::DataPartsVector parts) + { + prepared_parts = std::move(parts); + alter_conversions_for_parts = {}; + } const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } const MergeTreeData & getMergeTreeData() const { return data; } @@ -179,6 +189,7 @@ public: private: static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl( MergeTreeData::DataPartsVector parts, + std::vector alter_conversions, const StorageMetadataPtr & metadata_snapshot_base, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -202,6 +213,8 @@ private: MergeTreeReaderSettings reader_settings; MergeTreeData::DataPartsVector prepared_parts; + std::vector alter_conversions_for_parts; + Names real_column_names; Names virt_column_names; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 15b45f30891..c163e8107ac 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -661,6 +661,12 @@ public: return getStorageSnapshot(metadata_snapshot, query_context); } + /// Creates a storage snapshot but without holding a data specific to storage. + virtual StorageSnapshotPtr getStorageSnapshotWithoutData(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const + { + return getStorageSnapshot(metadata_snapshot, query_context); + } + /// A helper to implement read() static void readFromPipe( QueryPlan & query_plan, diff --git a/src/Storages/MergeTree/AlterConversions.cpp b/src/Storages/MergeTree/AlterConversions.cpp index 7a298b0f6ca..a98cd6d99f9 100644 --- a/src/Storages/MergeTree/AlterConversions.cpp +++ b/src/Storages/MergeTree/AlterConversions.cpp @@ -9,6 +9,13 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +void AlterConversions::addMutationCommand(const MutationCommand & command) +{ + /// Currently only RENAME_COLUMN is applied on-fly. + if (command.type == MutationCommand::Type::RENAME_COLUMN) + rename_map.emplace_back(RenamePair{command.rename_to, command.column_name}); +} + bool AlterConversions::columnHasNewName(const std::string & old_name) const { for (const auto & [new_name, prev_name] : rename_map) @@ -31,7 +38,6 @@ std::string AlterConversions::getColumnNewName(const std::string & old_name) con throw Exception(ErrorCodes::LOGICAL_ERROR, "Column {} was not renamed", old_name); } - bool AlterConversions::isColumnRenamed(const std::string & new_name) const { for (const auto & [name_to, name_from] : rename_map) @@ -41,6 +47,7 @@ bool AlterConversions::isColumnRenamed(const std::string & new_name) const } return false; } + /// Get column old name before rename (lookup by key in rename_map) std::string AlterConversions::getColumnOldName(const std::string & new_name) const { diff --git a/src/Storages/MergeTree/AlterConversions.h b/src/Storages/MergeTree/AlterConversions.h index ada385d6100..4410b9c56e2 100644 --- a/src/Storages/MergeTree/AlterConversions.h +++ b/src/Storages/MergeTree/AlterConversions.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -7,20 +8,23 @@ namespace DB { -/// Alter conversions which should be applied on-fly for part. Build from of -/// the most recent mutation commands for part. Now we have only rename_map -/// here (from ALTER_RENAME) command, because for all other type of alters -/// we can deduce conversions for part from difference between -/// part->getColumns() and storage->getColumns(). -struct AlterConversions + +/// Alter conversions which should be applied on-fly for part. +/// Built from of the most recent mutation commands for part. +/// Now only ALTER RENAME COLUMN is applied. +class AlterConversions : private boost::noncopyable { +public: + AlterConversions() = default; + struct RenamePair { std::string rename_to; std::string rename_from; }; - /// Rename map new_name -> old_name - std::vector rename_map; + + void addMutationCommand(const MutationCommand & command); + const std::vector & getRenameMap() const { return rename_map; } /// Column was renamed (lookup by value in rename_map) bool columnHasNewName(const std::string & old_name) const; @@ -30,6 +34,12 @@ struct AlterConversions bool isColumnRenamed(const std::string & new_name) const; /// Get column old name before rename (lookup by key in rename_map) std::string getColumnOldName(const std::string & new_name) const; + +private: + /// Rename map new_name -> old_name. + std::vector rename_map; }; +using AlterConversionsPtr = std::shared_ptr; + } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 388d96314c0..411de3af982 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -92,6 +93,7 @@ public: const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, + const AlterConversionsPtr & alter_conversions, const MergeTreeReaderSettings & reader_settings_, const ValueSizeMap & avg_value_size_hints_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_) const = 0; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h b/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h index 648c3cfbb6b..55a9c7b1509 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h @@ -52,7 +52,7 @@ public: virtual const MergeTreeDataPartChecksums & getChecksums() const = 0; - virtual AlterConversions getAlterConversions() const = 0; + virtual AlterConversionsPtr getAlterConversions() const = 0; virtual size_t getMarksCount() const = 0; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index d3b623d656e..dc9e9001439 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -5,8 +6,6 @@ #include #include #include -#include -#include namespace DB @@ -133,9 +132,9 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns String IMergeTreeReader::getColumnNameInPart(const NameAndTypePair & required_column) const { auto name_in_storage = required_column.getNameInStorage(); - if (alter_conversions.isColumnRenamed(name_in_storage)) + if (alter_conversions->isColumnRenamed(name_in_storage)) { - name_in_storage = alter_conversions.getColumnOldName(name_in_storage); + name_in_storage = alter_conversions->getColumnOldName(name_in_storage); return Nested::concatenateName(name_in_storage, required_column.getSubcolumnName()); } diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index c7396e882fa..61a976aba38 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -50,8 +50,8 @@ public: /// Evaluate defaulted columns if necessary. void evaluateMissingDefaults(Block additional_columns, Columns & res_columns) const; - /// If part metadata is not equal to storage metadata, than - /// try to perform conversions of columns. + /// If part metadata is not equal to storage metadata, + /// then try to perform conversions of columns. void performRequiredConversions(Columns & res_columns) const; const NamesAndTypesList & getColumns() const { return requested_columns; } @@ -104,7 +104,7 @@ protected: private: /// Alter conversions, which must be applied on fly if required - AlterConversions alter_conversions; + AlterConversionsPtr alter_conversions; /// Columns that are requested to read. NamesAndTypesList requested_columns; diff --git a/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h b/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h index 3363c75dd6f..aec102f3f7d 100644 --- a/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h +++ b/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h @@ -1,4 +1,5 @@ #pragma once +#include #include #include @@ -9,9 +10,11 @@ namespace DB class LoadedMergeTreeDataPartInfoForReader final : public IMergeTreeDataPartInfoForReader { public: - explicit LoadedMergeTreeDataPartInfoForReader(MergeTreeData::DataPartPtr data_part_) + LoadedMergeTreeDataPartInfoForReader( + MergeTreeData::DataPartPtr data_part_, AlterConversionsPtr alter_conversions_) : IMergeTreeDataPartInfoForReader(data_part_->storage.getContext()) - , data_part(data_part_) + , data_part(std::move(data_part_)) + , alter_conversions(std::move(alter_conversions_)) { } @@ -33,7 +36,7 @@ public: std::optional getColumnPosition(const String & column_name) const override { return data_part->getColumnPosition(column_name); } - AlterConversions getAlterConversions() const override { return data_part->storage.getAlterConversionsForPart(data_part); } + AlterConversionsPtr getAlterConversions() const override { return alter_conversions; } String getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const override { return data_part->getColumnNameWithMinimumCompressedSize(with_subcolumns); } @@ -53,8 +56,11 @@ public: SerializationPtr getSerialization(const NameAndTypePair & column) const override { return data_part->getSerialization(column.name); } + MergeTreeData::DataPartPtr getDataPart() const { return data_part; } + private: MergeTreeData::DataPartPtr data_part; + AlterConversionsPtr alter_conversions; }; } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index e58a9fc283c..8f39c31eae0 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -193,8 +193,9 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->storage_columns = global_ctx->metadata_snapshot->getColumns().getAllPhysical(); auto object_columns = MergeTreeData::getConcreteObjectColumns(global_ctx->future_part->parts, global_ctx->metadata_snapshot->getColumns()); - global_ctx->storage_snapshot = std::make_shared(*global_ctx->data, global_ctx->metadata_snapshot, object_columns); + extendObjectColumns(global_ctx->storage_columns, object_columns, false); + global_ctx->storage_snapshot = std::make_shared(*global_ctx->data, global_ctx->metadata_snapshot, std::move(object_columns)); extractMergingAndGatheringColumns( global_ctx->storage_columns, @@ -544,8 +545,8 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const global_ctx->future_part->parts[part_num], column_names, ctx->read_with_direct_io, - true, - false, + /*take_column_types_from_storage=*/ true, + /*quiet=*/ false, global_ctx->input_rows_filtered); pipes.emplace_back(std::move(pipe)); @@ -896,8 +897,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() part, global_ctx->merging_column_names, ctx->read_with_direct_io, - true, - false, + /*take_column_types_from_storage=*/ true, + /*quiet=*/ false, global_ctx->input_rows_filtered); if (global_ctx->metadata_snapshot->hasSortingKey()) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 24980cdb82c..48adf36e678 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -80,49 +80,51 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm( result_header = header_without_const_virtual_columns; injectPartConstVirtualColumns(0, result_header, nullptr, partition_value_type, virt_column_names); - if (prewhere_actions) - LOG_TRACE(log, "PREWHERE condition was split into {} steps: {}", prewhere_actions->steps.size(), prewhere_actions->dumpConditions()); + if (!prewhere_actions.steps.empty()) + LOG_TRACE(log, "PREWHERE condition was split into {} steps: {}", prewhere_actions.steps.size(), prewhere_actions.dumpConditions()); if (prewhere_info) LOG_TEST(log, "Original PREWHERE DAG:\n{}\nPREWHERE actions:\n{}", (prewhere_info->prewhere_actions ? prewhere_info->prewhere_actions->dumpDAG(): std::string("")), - (prewhere_actions ? prewhere_actions->dump() : std::string(""))); + (!prewhere_actions.steps.empty() ? prewhere_actions.dump() : std::string(""))); } bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere); -std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps) +PrewhereExprInfo IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps) { - std::unique_ptr prewhere_actions; + PrewhereExprInfo prewhere_actions; if (prewhere_info) { - prewhere_actions = std::make_unique(); - if (prewhere_info->row_level_filter) { PrewhereExprStep row_level_filter_step { + .type = PrewhereExprStep::Filter, .actions = std::make_shared(prewhere_info->row_level_filter, actions_settings), - .column_name = prewhere_info->row_level_column_name, - .remove_column = true, - .need_filter = true + .filter_column_name = prewhere_info->row_level_column_name, + .remove_filter_column = true, + .need_filter = true, + .perform_alter_conversions = true, }; - prewhere_actions->steps.emplace_back(std::move(row_level_filter_step)); + prewhere_actions.steps.emplace_back(std::make_shared(std::move(row_level_filter_step))); } if (!enable_multiple_prewhere_read_steps || - !tryBuildPrewhereSteps(prewhere_info, actions_settings, *prewhere_actions)) + !tryBuildPrewhereSteps(prewhere_info, actions_settings, prewhere_actions)) { PrewhereExprStep prewhere_step { + .type = PrewhereExprStep::Filter, .actions = std::make_shared(prewhere_info->prewhere_actions, actions_settings), - .column_name = prewhere_info->prewhere_column_name, - .remove_column = prewhere_info->remove_prewhere_column, - .need_filter = prewhere_info->need_filter + .filter_column_name = prewhere_info->prewhere_column_name, + .remove_filter_column = prewhere_info->remove_prewhere_column, + .need_filter = prewhere_info->need_filter, + .perform_alter_conversions = true, }; - prewhere_actions->steps.emplace_back(std::move(prewhere_step)); + prewhere_actions.steps.emplace_back(std::make_shared(std::move(prewhere_step))); } } @@ -213,7 +215,7 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask( reader = task->data_part->getReader( task->task_columns.columns, metadata_snapshot, task->mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), - reader_settings, value_size_map, profile_callback); + task->alter_conversions, reader_settings, value_size_map, profile_callback); } if (!task->pre_reader_for_step.empty()) @@ -226,13 +228,15 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask( else { initializeMergeTreePreReadersForPart( - task->data_part, task->task_columns, metadata_snapshot, + task->data_part, task->alter_conversions, + task->task_columns, metadata_snapshot, task->mark_ranges, value_size_map, profile_callback); } } void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForPart( - MergeTreeData::DataPartPtr & data_part, + const MergeTreeData::DataPartPtr & data_part, + const AlterConversionsPtr & alter_conversions, const MergeTreeReadTaskColumns & task_columns, const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, @@ -242,15 +246,16 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForPart( reader = data_part->getReader( task_columns.columns, metadata_snapshot, mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), - reader_settings, value_size_map, profile_callback); + alter_conversions, reader_settings, value_size_map, profile_callback); initializeMergeTreePreReadersForPart( - data_part, task_columns, metadata_snapshot, + data_part, alter_conversions, task_columns, metadata_snapshot, mark_ranges, value_size_map, profile_callback); } void IMergeTreeSelectAlgorithm::initializeMergeTreePreReadersForPart( - MergeTreeData::DataPartPtr & data_part, + const MergeTreeData::DataPartPtr & data_part, + const AlterConversionsPtr & alter_conversions, const MergeTreeReadTaskColumns & task_columns, const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, @@ -266,36 +271,37 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreePreReadersForPart( data_part->getReader( {LightweightDeleteDescription::FILTER_COLUMN}, metadata_snapshot, mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), - reader_settings, value_size_map, profile_callback)); + alter_conversions, reader_settings, value_size_map, profile_callback)); } - if (prewhere_info) + for (const auto & pre_columns_per_step : task_columns.pre_columns) { - for (const auto & pre_columns_per_step : task_columns.pre_columns) - { - pre_reader_for_step.push_back( - data_part->getReader( - pre_columns_per_step, metadata_snapshot, mark_ranges, - owned_uncompressed_cache.get(), owned_mark_cache.get(), - reader_settings, value_size_map, profile_callback)); - } + pre_reader_for_step.push_back( + data_part->getReader( + pre_columns_per_step, metadata_snapshot, mark_ranges, + owned_uncompressed_cache.get(), owned_mark_cache.get(), + alter_conversions, reader_settings, value_size_map, profile_callback)); } } void IMergeTreeSelectAlgorithm::initializeRangeReaders(MergeTreeReadTask & current_task) { return initializeRangeReadersImpl( - current_task.range_reader, current_task.pre_range_readers, prewhere_info, prewhere_actions.get(), + current_task.range_reader, current_task.pre_range_readers, prewhere_actions, reader.get(), current_task.data_part->hasLightweightDelete(), reader_settings, pre_reader_for_step, lightweight_delete_filter_step, non_const_virtual_column_names); } void IMergeTreeSelectAlgorithm::initializeRangeReadersImpl( - MergeTreeRangeReader & range_reader, std::deque & pre_range_readers, - PrewhereInfoPtr prewhere_info, const PrewhereExprInfo * prewhere_actions, - IMergeTreeReader * reader, bool has_lightweight_delete, const MergeTreeReaderSettings & reader_settings, + MergeTreeRangeReader & range_reader, + std::deque & pre_range_readers, + const PrewhereExprInfo & prewhere_actions, + IMergeTreeReader * reader, + bool has_lightweight_delete, + const MergeTreeReaderSettings & reader_settings, const std::vector> & pre_reader_for_step, - const PrewhereExprStep & lightweight_delete_filter_step, const Names & non_const_virtual_column_names) + const PrewhereExprStep & lightweight_delete_filter_step, + const Names & non_const_virtual_column_names) { MergeTreeRangeReader * prev_reader = nullptr; bool last_reader = false; @@ -310,25 +316,25 @@ void IMergeTreeSelectAlgorithm::initializeRangeReadersImpl( pre_readers_shift++; } - if (prewhere_info) + if (prewhere_actions.steps.size() + pre_readers_shift != pre_reader_for_step.size()) { - if (prewhere_actions->steps.size() + pre_readers_shift != pre_reader_for_step.size()) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "PREWHERE steps count mismatch, actions: {}, readers: {}", - prewhere_actions->steps.size(), pre_reader_for_step.size()); - } + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "PREWHERE steps count mismatch, actions: {}, readers: {}", + prewhere_actions.steps.size(), pre_reader_for_step.size()); + } - for (size_t i = 0; i < prewhere_actions->steps.size(); ++i) - { - last_reader = reader->getColumns().empty() && (i + 1 == prewhere_actions->steps.size()); + for (size_t i = 0; i < prewhere_actions.steps.size(); ++i) + { + last_reader = reader->getColumns().empty() && (i + 1 == prewhere_actions.steps.size()); - MergeTreeRangeReader current_reader(pre_reader_for_step[i + pre_readers_shift].get(), prev_reader, &prewhere_actions->steps[i], last_reader, non_const_virtual_column_names); + MergeTreeRangeReader current_reader( + pre_reader_for_step[i + pre_readers_shift].get(), + prev_reader, prewhere_actions.steps[i].get(), + last_reader, non_const_virtual_column_names); - pre_range_readers.push_back(std::move(current_reader)); - prev_reader = &pre_range_readers.back(); - } + pre_range_readers.push_back(std::move(current_reader)); + prev_reader = &pre_range_readers.back(); } if (!last_reader) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index e838a860d93..7b6dc50060a 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -74,7 +74,7 @@ public: virtual std::string getName() const = 0; - static std::unique_ptr getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps); + static PrewhereExprInfo getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps); protected: /// This struct allow to return block with no columns but with non-zero number of rows similar to Chunk @@ -110,8 +110,7 @@ protected: static void initializeRangeReadersImpl( MergeTreeRangeReader & range_reader, std::deque & pre_range_readers, - PrewhereInfoPtr prewhere_info, - const PrewhereExprInfo * prewhere_actions, + const PrewhereExprInfo & prewhere_actions, IMergeTreeReader * reader, bool has_lightweight_delete, const MergeTreeReaderSettings & reader_settings, @@ -126,7 +125,8 @@ protected: const ReadBufferFromFileBase::ProfileCallback & profile_callback); void initializeMergeTreeReadersForPart( - MergeTreeData::DataPartPtr & data_part, + const MergeTreeData::DataPartPtr & data_part, + const AlterConversionsPtr & alter_conversions, const MergeTreeReadTaskColumns & task_columns, const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, @@ -140,10 +140,19 @@ protected: StorageSnapshotPtr storage_snapshot; /// This step is added when the part has lightweight delete mask - const PrewhereExprStep lightweight_delete_filter_step { nullptr, LightweightDeleteDescription::FILTER_COLUMN.name, true, true }; + const PrewhereExprStep lightweight_delete_filter_step + { + .type = PrewhereExprStep::Filter, + .actions = nullptr, + .filter_column_name = LightweightDeleteDescription::FILTER_COLUMN.name, + .remove_filter_column = true, + .need_filter = true, + .perform_alter_conversions = true, + }; + PrewhereInfoPtr prewhere_info; ExpressionActionsSettings actions_settings; - std::unique_ptr prewhere_actions; + PrewhereExprInfo prewhere_actions; UInt64 max_block_size_rows; UInt64 preferred_block_size_bytes; @@ -195,7 +204,8 @@ private: /// Initialize pre readers. void initializeMergeTreePreReadersForPart( - MergeTreeData::DataPartPtr & data_part, + const MergeTreeData::DataPartPtr & data_part, + const AlterConversionsPtr & alter_conversions, const MergeTreeReadTaskColumns & task_columns, const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 4c3d4bc8aa0..465868ef853 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -30,7 +31,7 @@ namespace bool injectRequiredColumnsRecursively( const String & column_name, const StorageSnapshotPtr & storage_snapshot, - const AlterConversions & alter_conversions, + const AlterConversionsPtr & alter_conversions, const IMergeTreeDataPartInfoForReader & data_part_info_for_reader, const GetColumnsOptions & options, Names & columns, @@ -46,8 +47,8 @@ bool injectRequiredColumnsRecursively( if (column_in_storage) { auto column_name_in_part = column_in_storage->getNameInStorage(); - if (alter_conversions.isColumnRenamed(column_name_in_part)) - column_name_in_part = alter_conversions.getColumnOldName(column_name_in_part); + if (alter_conversions && alter_conversions->isColumnRenamed(column_name_in_part)) + column_name_in_part = alter_conversions->getColumnOldName(column_name_in_part); auto column_in_part = data_part_info_for_reader.getColumns().tryGetByName(column_name_in_part); @@ -98,13 +99,14 @@ NameSet injectRequiredColumns( NameSet injected_columns; bool have_at_least_one_physical_column = false; - AlterConversions alter_conversions; + AlterConversionsPtr alter_conversions; if (!data_part_info_for_reader.isProjectionPart()) alter_conversions = data_part_info_for_reader.getAlterConversions(); auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical) .withExtendedObjects() .withSystemColumns(); + if (with_subcolumns) options.withSubcolumns(); @@ -137,6 +139,7 @@ NameSet injectRequiredColumns( MergeTreeReadTask::MergeTreeReadTask( const DataPartPtr & data_part_, + const AlterConversionsPtr & alter_conversions_, const MarkRanges & mark_ranges_, size_t part_index_in_query_, const NameSet & column_name_set_, @@ -146,6 +149,7 @@ MergeTreeReadTask::MergeTreeReadTask( std::future reader_, std::vector> && pre_reader_for_step_) : data_part{data_part_} + , alter_conversions{alter_conversions_} , mark_ranges{mark_ranges_} , part_index_in_query{part_index_in_query_} , column_name_set{column_name_set_} @@ -306,10 +310,8 @@ MergeTreeReadTaskColumns getReadTaskColumns( /// Read system columns such as lightweight delete mask "_row_exists" if it is persisted in the part for (const auto & name : system_columns) - { if (data_part_info_for_reader.getColumns().contains(name)) column_to_read_after_prewhere.push_back(name); - } /// Inject columns required for defaults evaluation injectRequiredColumns( @@ -319,44 +321,50 @@ MergeTreeReadTaskColumns getReadTaskColumns( auto options = GetColumnsOptions(GetColumnsOptions::All) .withExtendedObjects() .withSystemColumns(); + if (with_subcolumns) options.withSubcolumns(); + NameSet columns_from_previous_steps; + auto add_step = [&](const PrewhereExprStep & step) + { + Names step_column_names = step.actions->getActionsDAG().getRequiredColumnsNames(); + + injectRequiredColumns( + data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names); + + Names columns_to_read_in_step; + for (const auto & name : step_column_names) + { + if (columns_from_previous_steps.contains(name)) + continue; + + columns_to_read_in_step.push_back(name); + columns_from_previous_steps.insert(name); + } + + result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, columns_to_read_in_step)); + }; + if (prewhere_info) { auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions( - prewhere_info, actions_settings, reader_settings.enable_multiple_prewhere_read_steps); + prewhere_info, + actions_settings, + reader_settings.enable_multiple_prewhere_read_steps); - NameSet columns_from_previous_steps; - - for (const auto & step : prewhere_actions->steps) - { - Names step_column_names = step.actions->getActionsDAG().getRequiredColumnsNames(); - - injectRequiredColumns( - data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names); - - Names columns_to_read_in_step; - for (const auto & name : step_column_names) - { - if (columns_from_previous_steps.contains(name)) - continue; - columns_to_read_in_step.push_back(name); - columns_from_previous_steps.insert(name); - } - - result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, columns_to_read_in_step)); - } - - /// Remove columns read in prewehere from the list of columns to read - Names post_column_names; - for (const auto & name : column_to_read_after_prewhere) - if (!columns_from_previous_steps.contains(name)) - post_column_names.push_back(name); - - column_to_read_after_prewhere = std::move(post_column_names); + for (const auto & step : prewhere_actions.steps) + add_step(*step); } + /// Remove columns read in prewehere from the list of columns to read + Names post_column_names; + for (const auto & name : column_to_read_after_prewhere) + if (!columns_from_previous_steps.contains(name)) + post_column_names.push_back(name); + + column_to_read_after_prewhere = std::move(post_column_names); + /// Rest of the requested columns result.columns = storage_snapshot->getColumnsByNames(options, column_to_read_after_prewhere); return result; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 5fce95e1714..0a3462db7f3 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -35,7 +36,6 @@ NameSet injectRequiredColumns( bool with_subcolumns, Names & columns); - struct MergeTreeReadTaskColumns { /// column names to read during WHERE @@ -49,8 +49,10 @@ struct MergeTreeReadTaskColumns /// A batch of work for MergeTreeThreadSelectProcessor struct MergeTreeReadTask { - /// data part which should be read while performing this task + /// Data part which should be read while performing this task DataPartPtr data_part; + /// Alter converversionss that should be applied on-fly for part. + AlterConversionsPtr alter_conversions; /// Ranges to read from `data_part`. MarkRanges mark_ranges; /// for virtual `part_index` virtual column @@ -81,6 +83,7 @@ struct MergeTreeReadTask MergeTreeReadTask( const DataPartPtr & data_part_, + const AlterConversionsPtr & alter_conversions_, const MarkRanges & mark_ranges_, size_t part_index_in_query_, const NameSet & column_name_set_, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1d5ac21f803..0d89acb4cdf 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1,4 +1,3 @@ -#include "Storages/MergeTree/MergeTreeDataPartBuilder.h" #include #include @@ -76,6 +75,8 @@ #include #include #include +#include +#include #include #include @@ -7982,25 +7983,14 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S return true; } -AlterConversions MergeTreeData::getAlterConversionsForPart(const MergeTreeDataPartPtr part) const +AlterConversionsPtr MergeTreeData::getAlterConversionsForPart(MergeTreeDataPartPtr part) const { - std::map commands_map = getAlterMutationCommandsForPart(part); + auto commands_map = getAlterMutationCommandsForPart(part); - AlterConversions result{}; - auto & rename_map = result.rename_map; - for (const auto & [version, commands] : commands_map) - { + auto result = std::make_shared(); + for (const auto & [_, commands] : commands_map) for (const auto & command : commands) - { - /// Currently we need explicit conversions only for RENAME alter - /// all other conversions can be deduced from diff between part columns - /// and columns in storage. - if (command.type == MutationCommand::Type::RENAME_COLUMN) - { - rename_map.emplace_back(AlterConversions::RenamePair{command.rename_to, command.column_name}); - } - } - } + result->addMutationCommand(command); return result; } @@ -8331,13 +8321,22 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const { auto snapshot_data = std::make_unique(); + ColumnsDescription object_columns_copy; - auto lock = lockParts(); - snapshot_data->parts = getVisibleDataPartsVectorUnlocked(query_context, lock); - return std::make_shared(*this, metadata_snapshot, object_columns, std::move(snapshot_data)); + { + auto lock = lockParts(); + snapshot_data->parts = getVisibleDataPartsVectorUnlocked(query_context, lock); + object_columns_copy = object_columns; + } + + snapshot_data->alter_conversions.reserve(snapshot_data->parts.size()); + for (const auto & part : snapshot_data->parts) + snapshot_data->alter_conversions.push_back(getAlterConversionsForPart(part)); + + return std::make_shared(*this, metadata_snapshot, std::move(object_columns_copy), std::move(snapshot_data)); } -StorageSnapshotPtr MergeTreeData::getStorageSnapshotWithoutParts(const StorageMetadataPtr & metadata_snapshot) const +StorageSnapshotPtr MergeTreeData::getStorageSnapshotWithoutData(const StorageMetadataPtr & metadata_snapshot, ContextPtr) const { auto lock = lockParts(); return std::make_shared(*this, metadata_snapshot, object_columns, std::make_unique()); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5488ce72631..fce7d989a2f 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -443,12 +443,13 @@ public: struct SnapshotData : public StorageSnapshot::Data { DataPartsVector parts; + std::vector alter_conversions; }; StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override; /// The same as above but does not hold vector of data parts. - StorageSnapshotPtr getStorageSnapshotWithoutParts(const StorageMetadataPtr & metadata_snapshot) const; + StorageSnapshotPtr getStorageSnapshotWithoutData(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override; /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); @@ -914,7 +915,7 @@ public: Disks getDisks() const { return getStoragePolicy()->getDisks(); } /// Return alter conversions for part which must be applied on fly. - AlterConversions getAlterConversionsForPart(MergeTreeDataPartPtr part) const; + AlterConversionsPtr getAlterConversionsForPart(MergeTreeDataPartPtr part) const; /// Returns destination disk or volume for the TTL rule according to current storage policy. SpacePtr getDestinationForMoveTTL(const TTLDescription & move_ttl) const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index a5fb11244c7..07e20f16a9f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -34,11 +34,12 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, + const AlterConversionsPtr & alter_conversions, const MergeTreeReaderSettings & reader_settings, const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const { - auto read_info = std::make_shared(shared_from_this()); + auto read_info = std::make_shared(shared_from_this(), alter_conversions); auto * load_marks_threadpool = reader_settings.read_settings.load_marks_asynchronously ? &read_info->getContext()->getLoadMarksThreadpool() : nullptr; return std::make_unique( diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index ef553d290f7..b115692a7cf 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -34,6 +34,7 @@ public: const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, + const AlterConversionsPtr & alter_conversions, const MergeTreeReaderSettings & reader_settings_, const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 5b1054d0a0e..468747a6c36 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -36,12 +36,14 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( const MarkRanges & mark_ranges, UncompressedCache * /* uncompressed_cache */, MarkCache * /* mark_cache */, + const AlterConversionsPtr & alter_conversions, const MergeTreeReaderSettings & reader_settings, const ValueSizeMap & /* avg_value_size_hints */, const ReadBufferFromFileBase::ProfileCallback & /* profile_callback */) const { - auto read_info = std::make_shared(shared_from_this()); + auto read_info = std::make_shared(shared_from_this(), alter_conversions); auto ptr = std::static_pointer_cast(shared_from_this()); + return std::make_unique( read_info, ptr, columns_to_read, metadata_snapshot, mark_ranges, reader_settings); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 525c966476b..db7244d8e99 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -23,6 +23,7 @@ public: const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, + const AlterConversionsPtr & alter_conversions, const MergeTreeReaderSettings & reader_settings_, const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 016aa2b7984..f44cbdd8628 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -33,11 +33,12 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, + const AlterConversionsPtr & alter_conversions, const MergeTreeReaderSettings & reader_settings, const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const { - auto read_info = std::make_shared(shared_from_this()); + auto read_info = std::make_shared(shared_from_this(), alter_conversions); return std::make_unique( read_info, columns_to_read, metadata_snapshot, uncompressed_cache, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 3f0d52d831e..5ee497b9b21 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -29,6 +29,7 @@ public: const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, + const AlterConversionsPtr & alter_conversions, const MergeTreeReaderSettings & reader_settings_, const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index dea22813bd1..2b16ea43179 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -160,11 +160,13 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( const auto & snapshot_data = assert_cast(*storage_snapshot->data); const auto & parts = snapshot_data.parts; + const auto & alter_conversions = snapshot_data.alter_conversions; if (!query_info.projection) { auto step = readFromParts( query_info.merge_tree_select_result_ptr ? MergeTreeData::DataPartsVector{} : parts, + query_info.merge_tree_select_result_ptr ? std::vector{} : alter_conversions, column_names_to_return, storage_snapshot, query_info, @@ -207,7 +209,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( { LOG_DEBUG(log, "projection required columns: {}", fmt::join(query_info.projection->required_columns, ", ")); projection_plan->addStep(readFromParts( - {}, + /*parts=*/ {}, + /*alter_conversions=*/ {}, query_info.projection->required_columns, storage_snapshot, query_info, @@ -794,6 +797,7 @@ std::optional> MergeTreeDataSelectExecutor::filterPar void MergeTreeDataSelectExecutor::filterPartsByPartition( MergeTreeData::DataPartsVector & parts, + std::vector & alter_conversions, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, @@ -803,6 +807,8 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( Poco::Logger * log, ReadFromMergeTree::IndexStats & index_stats) { + chassert(alter_conversions.empty() || parts.size() == alter_conversions.size()); + const Settings & settings = context->getSettingsRef(); std::optional partition_pruner; @@ -836,6 +842,7 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( if (query_context->getSettingsRef().allow_experimental_query_deduplication) selectPartsToReadWithUUIDFilter( parts, + alter_conversions, part_values, data.getPinnedPartUUIDs(), minmax_idx_condition, @@ -848,6 +855,7 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( else selectPartsToRead( parts, + alter_conversions, part_values, minmax_idx_condition, minmax_columns_types, @@ -886,6 +894,7 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, + std::vector && alter_conversions, StorageMetadataPtr metadata_snapshot, const SelectQueryInfo & query_info, const ContextPtr & context, @@ -896,6 +905,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd ReadFromMergeTree::IndexStats & index_stats, bool use_skip_indexes) { + chassert(alter_conversions.empty() || parts.size() == alter_conversions.size()); + RangesInDataParts parts_with_ranges; parts_with_ranges.resize(parts.size()); const Settings & settings = context->getSettingsRef(); @@ -1003,9 +1014,11 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd auto process_part = [&](size_t part_index) { auto & part = parts[part_index]; + auto alter_conversions_for_part = !alter_conversions.empty() + ? alter_conversions[part_index] + : std::make_shared(); - RangesInDataPart ranges(part, part_index); - + RangesInDataPart ranges(part, alter_conversions_for_part, part_index); size_t total_marks_count = part->index_granularity.getMarksCountWithoutFinal(); if (metadata_snapshot->hasPrimaryKey()) @@ -1024,8 +1037,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd break; index_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed); + index_and_condition.stat.total_granules.fetch_add(ranges.ranges.getNumberOfMarks(), std::memory_order_relaxed); - size_t total_granules = 0; size_t granules_dropped = 0; ranges.ranges = filterMarksUsingIndex( index_and_condition.index, @@ -1034,15 +1047,12 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd ranges.ranges, settings, reader_settings, - total_granules, granules_dropped, mark_cache.get(), uncompressed_cache.get(), log); - index_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); index_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); - if (ranges.ranges.empty()) index_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); } @@ -1293,6 +1303,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar return ReadFromMergeTree::selectRangesToRead( std::move(parts), + /*alter_conversions=*/ {}, prewhere_info, added_filter_nodes, metadata_snapshot_base, @@ -1309,6 +1320,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, + std::vector alter_conversions, const Names & column_names_to_return, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info, @@ -1338,6 +1350,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( return std::make_unique( std::move(parts), + std::move(alter_conversions), real_column_names, virt_column_names, data, @@ -1609,7 +1622,6 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( const MarkRanges & ranges, const Settings & settings, const MergeTreeReaderSettings & reader_settings, - size_t & total_granules, size_t & granules_dropped, MarkCache * mark_cache, UncompressedCache * uncompressed_cache, @@ -1670,8 +1682,6 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( if (last_index_mark != index_range.begin || !granule) reader.seek(index_range.begin); - total_granules += index_range.end - index_range.begin; - for (size_t index_mark = index_range.begin; index_mark < index_range.end; ++index_mark) { if (index_mark != index_range.begin || !granule || last_index_mark != index_range.begin) @@ -1833,6 +1843,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex( void MergeTreeDataSelectExecutor::selectPartsToRead( MergeTreeData::DataPartsVector & parts, + std::vector & alter_conversions, const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, @@ -1841,10 +1852,14 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( PartFilterCounters & counters) { MergeTreeData::DataPartsVector prev_parts; + std::vector prev_conversions; + std::swap(prev_parts, parts); - for (const auto & part_or_projection : prev_parts) + std::swap(prev_conversions, alter_conversions); + + for (size_t i = 0; i < prev_parts.size(); ++i) { - const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); + const auto * part = prev_parts[i]->isProjectionPart() ? prev_parts[i]->getParentPart() : prev_parts[i].get(); if (part_values && part_values->find(part->name) == part_values->end()) continue; @@ -1881,12 +1896,15 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( counters.num_parts_after_partition_pruner += 1; counters.num_granules_after_partition_pruner += num_granules; - parts.push_back(part_or_projection); + parts.push_back(prev_parts[i]); + if (!prev_conversions.empty()) + alter_conversions.push_back(prev_conversions[i]); } } void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, + std::vector & alter_conversions, const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, @@ -1899,16 +1917,22 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( { /// process_parts prepare parts that have to be read for the query, /// returns false if duplicated parts' UUID have been met - auto select_parts = [&] (MergeTreeData::DataPartsVector & selected_parts) -> bool + auto select_parts = [&] ( + MergeTreeData::DataPartsVector & selected_parts, + std::vector & selected_conversions) -> bool { auto ignored_part_uuids = query_context->getIgnoredPartUUIDs(); std::unordered_set temp_part_uuids; MergeTreeData::DataPartsVector prev_parts; + std::vector prev_conversions; + std::swap(prev_parts, selected_parts); - for (const auto & part_or_projection : prev_parts) + std::swap(prev_conversions, selected_conversions); + + for (size_t i = 0; i < prev_parts.size(); ++i) { - const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); + const auto * part = prev_parts[i]->isProjectionPart() ? prev_parts[i]->getParentPart() : prev_parts[i].get(); if (part_values && part_values->find(part->name) == part_values->end()) continue; @@ -1958,7 +1982,9 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( throw Exception(ErrorCodes::LOGICAL_ERROR, "Found a part with the same UUID on the same replica."); } - selected_parts.push_back(part_or_projection); + selected_parts.push_back(prev_parts[i]); + if (!prev_conversions.empty()) + selected_conversions.push_back(prev_conversions[i]); } if (!temp_part_uuids.empty()) @@ -1977,7 +2003,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( }; /// Process parts that have to be read for a query. - auto needs_retry = !select_parts(parts); + auto needs_retry = !select_parts(parts, alter_conversions); /// If any duplicated part UUIDs met during the first step, try to ignore them in second pass. /// This may happen when `prefer_localhost_replica` is set and "distributed" stage runs in the same process with "remote" stage. @@ -1988,7 +2014,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( counters = PartFilterCounters(); /// Second attempt didn't help, throw an exception - if (!select_parts(parts)) + if (!select_parts(parts, alter_conversions)) throw Exception(ErrorCodes::DUPLICATED_PART_UUIDS, "Found duplicate UUIDs while processing query."); } } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index a337574bb64..18fe312598c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -41,6 +41,7 @@ public: /// The same as read, but with specified set of parts. QueryPlanStepPtr readFromParts( MergeTreeData::DataPartsVector parts, + std::vector alter_conversions, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info, @@ -92,7 +93,6 @@ private: const MarkRanges & ranges, const Settings & settings, const MergeTreeReaderSettings & reader_settings, - size_t & total_granules, size_t & granules_dropped, MarkCache * mark_cache, UncompressedCache * uncompressed_cache, @@ -125,6 +125,7 @@ private: /// as well as `max_block_number_to_read`. static void selectPartsToRead( MergeTreeData::DataPartsVector & parts, + std::vector & alter_conversions, const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, @@ -135,6 +136,7 @@ private: /// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded. static void selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, + std::vector & alter_conversions, const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, @@ -174,6 +176,7 @@ public: /// Filter parts using minmax index and partition key. static void filterPartsByPartition( MergeTreeData::DataPartsVector & parts, + std::vector & alter_conversions, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, @@ -188,6 +191,7 @@ public: /// If 'check_limits = true' it will throw exception if the amount of data exceed the limits from settings. static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, + std::vector && alter_conversions, StorageMetadataPtr metadata_snapshot, const SelectQueryInfo & query_info, const ContextPtr & context, diff --git a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp index 813f144ee98..2676adfac2d 100644 --- a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp @@ -55,7 +55,12 @@ try : getSizePredictor(data_part, task_columns, sample_block); task = std::make_unique( - data_part, mark_ranges_for_task, part_index_in_query, column_name_set, task_columns, + data_part, + alter_conversions, + mark_ranges_for_task, + part_index_in_query, + column_name_set, + task_columns, std::move(size_predictor)); return true; diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.cpp b/src/Storages/MergeTree/MergeTreeIndexReader.cpp index 88fbc8c2488..ab1fd8dfa74 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexReader.cpp @@ -21,7 +21,7 @@ std::unique_ptr makeIndexReader( auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; return std::make_unique( - std::make_shared(part), + std::make_shared(part, std::make_shared()), index->getFileName(), extension, marks_count, all_mark_ranges, std::move(settings), mark_cache, uncompressed_cache, diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 30f31910bee..3a75a86ec0b 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include #include @@ -71,6 +73,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( struct MergeTreePrefetchedReadPool::PartInfo { MergeTreeData::DataPartPtr data_part; + AlterConversionsPtr alter_conversions; size_t part_index_in_query; size_t sum_marks = 0; MarkRanges ranges; @@ -89,12 +92,13 @@ struct MergeTreePrefetchedReadPool::PartInfo std::future MergeTreePrefetchedReadPool::createPrefetchedReader( const IMergeTreeDataPart & data_part, const NamesAndTypesList & columns, + const AlterConversionsPtr & alter_conversions, const MarkRanges & required_ranges, int64_t priority) const { auto reader = data_part.getReader( columns, storage_snapshot->metadata, required_ranges, - uncompressed_cache, mark_cache, reader_settings, + uncompressed_cache, mark_cache, alter_conversions, reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); /// In order to make a prefetch we need to wait for marks to be loaded. But we just created @@ -120,21 +124,18 @@ void MergeTreePrefetchedReadPool::createPrefetchedReaderForTask(MergeTreeReadTas if (task.reader.valid()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Task already has a reader"); - task.reader = createPrefetchedReader(*task.data_part, task.task_columns.columns, task.mark_ranges, task.priority); + task.reader = createPrefetchedReader(*task.data_part, task.task_columns.columns, task.alter_conversions, task.mark_ranges, task.priority); if (reader_settings.apply_deleted_mask && task.data_part->hasLightweightDelete()) { - auto pre_reader = createPrefetchedReader(*task.data_part, {LightweightDeleteDescription::FILTER_COLUMN}, task.mark_ranges, task.priority); + auto pre_reader = createPrefetchedReader(*task.data_part, {LightweightDeleteDescription::FILTER_COLUMN}, task.alter_conversions, task.mark_ranges, task.priority); task.pre_reader_for_step.push_back(std::move(pre_reader)); } - if (prewhere_info) + for (const auto & pre_columns_per_step : task.task_columns.pre_columns) { - for (const auto & pre_columns_per_step : task.task_columns.pre_columns) - { - auto pre_reader = createPrefetchedReader(*task.data_part, pre_columns_per_step, task.mark_ranges, task.priority); - task.pre_reader_for_step.push_back(std::move(pre_reader)); - } + auto pre_reader = createPrefetchedReader(*task.data_part, pre_columns_per_step, task.alter_conversions, task.mark_ranges, task.priority); + task.pre_reader_for_step.push_back(std::move(pre_reader)); } } @@ -335,27 +336,28 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf auto part_info = std::make_unique(); part_info->data_part = part.data_part; + part_info->alter_conversions = part.alter_conversions; part_info->part_index_in_query = part.part_index_in_query; part_info->ranges = part.ranges; std::sort(part_info->ranges.begin(), part_info->ranges.end()); + LoadedMergeTreeDataPartInfoForReader part_reader_info(part.data_part, part_info->alter_conversions); + /// Sum up total size of all mark ranges in a data part. for (const auto & range : part.ranges) - { part_info->sum_marks += range.end - range.begin; - } part_info->approx_size_of_mark = getApproxSizeOfGranule(*part_info->data_part); const auto task_columns = getReadTaskColumns( - LoadedMergeTreeDataPartInfoForReader(part.data_part), + part_reader_info, storage_snapshot, column_names, virtual_column_names, prewhere_info, actions_settings, reader_settings, - /* with_subcolumns */true); + /*with_subcolumns=*/ true); part_info->size_predictor = !predict_block_size_bytes ? nullptr @@ -571,8 +573,12 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr : std::make_unique(*part.size_predictor); /// make a copy auto read_task = std::make_unique( - part.data_part, ranges_to_get_from_part, part.part_index_in_query, - part.column_name_set, part.task_columns, + part.data_part, + part.alter_conversions, + ranges_to_get_from_part, + part.part_index_in_query, + part.column_name_set, + part.task_columns, std::move(curr_task_size_predictor)); read_task->priority = priority; diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index a412dbf0811..421c7654ca6 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -57,6 +57,7 @@ private: std::future createPrefetchedReader( const IMergeTreeDataPart & data_part, const NamesAndTypesList & columns, + const AlterConversionsPtr & alter_conversions, const MarkRanges & required_ranges, int64_t priority) const; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 3f8a90ca175..586d7aa918a 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -842,8 +843,8 @@ MergeTreeRangeReader::MergeTreeRangeReader( if (step.actions) step.actions->execute(result_sample_block, true); - if (step.remove_column) - result_sample_block.erase(step.column_name); + if (step.remove_filter_column) + result_sample_block.erase(step.filter_column_name); } } @@ -981,12 +982,9 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar /// Calculate and update read bytes size_t total_bytes = 0; for (auto & column : columns) - { if (column) - { total_bytes += column->byteSize(); - } - } + read_result.addNumBytesRead(total_bytes); if (!columns.empty()) @@ -1000,8 +998,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar /// some columns (e.g. arrays) might be only partially filled and thus not be valid and /// fillMissingColumns() fixes this. bool should_evaluate_missing_defaults; - merge_tree_reader->fillMissingColumns(columns, should_evaluate_missing_defaults, - num_read_rows); + merge_tree_reader->fillMissingColumns(columns, should_evaluate_missing_defaults, num_read_rows); if (read_result.total_rows_per_granule == num_read_rows && read_result.num_rows != num_read_rows) { @@ -1026,7 +1023,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar } /// If columns not empty, then apply on-fly alter conversions if any required - merge_tree_reader->performRequiredConversions(columns); + if (!prewhere_info || prewhere_info->perform_alter_conversions) + merge_tree_reader->performRequiredConversions(columns); } read_result.columns.reserve(read_result.columns.size() + columns.size()); @@ -1050,15 +1048,15 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar Columns physical_columns(read_result.columns.begin(), read_result.columns.begin() + physical_columns_count); bool should_evaluate_missing_defaults; - merge_tree_reader->fillMissingColumns(physical_columns, should_evaluate_missing_defaults, - read_result.num_rows); + merge_tree_reader->fillMissingColumns(physical_columns, should_evaluate_missing_defaults, read_result.num_rows); /// If some columns absent in part, then evaluate default values if (should_evaluate_missing_defaults) merge_tree_reader->evaluateMissingDefaults({}, physical_columns); /// If result not empty, then apply on-fly alter conversions if any required - merge_tree_reader->performRequiredConversions(physical_columns); + if (!prewhere_info || prewhere_info->perform_alter_conversions) + merge_tree_reader->performRequiredConversions(physical_columns); for (size_t i = 0; i < physical_columns.size(); ++i) read_result.columns[i] = std::move(physical_columns[i]); @@ -1317,91 +1315,87 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r "Invalid number of columns passed to MergeTreeRangeReader. Expected {}, got {}", num_columns, result.columns.size()); - /// Filter computed at the current step. Its size is equal to num_rows which is <= total_rows_per_granule - ColumnPtr current_step_filter; - size_t prewhere_column_pos; + /// Restore block from columns list. + Block block; + size_t pos = 0; + if (prev_reader) { - /// Restore block from columns list. - Block block; - size_t pos = 0; - - if (prev_reader) + for (const auto & col : prev_reader->getSampleBlock()) { - for (const auto & col : prev_reader->getSampleBlock()) - { - block.insert({result.columns[pos], col.type, col.name}); - ++pos; - } + block.insert({result.columns[pos], col.type, col.name}); + ++pos; } - - for (auto name_and_type = header.begin(); name_and_type != header.end() && pos < result.columns.size(); ++pos, ++name_and_type) - block.insert({result.columns[pos], name_and_type->type, name_and_type->name}); - - { - /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. - Block additional_columns = block; - - if (prewhere_info->actions) - { - const String dummy_column = addDummyColumnWithRowCount(block, result.num_rows); - - LOG_TEST(log, "Executing prewhere actions on block: {}", block.dumpStructure()); - - prewhere_info->actions->execute(block); - - if (!dummy_column.empty()) - block.erase(dummy_column); - } - - result.additional_columns.clear(); - /// Additional columns might only be needed if there are more steps in the chain. - if (!last_reader_in_chain) - { - for (auto & col : additional_columns) - { - /// Exclude columns that are present in the result block to avoid storing them and filtering twice. - /// TODO: also need to exclude the columns that are not needed for the next steps. - if (block.has(col.name)) - continue; - result.additional_columns.insert(col); - } - } - } - - prewhere_column_pos = block.getPositionByName(prewhere_info->column_name); - - result.columns.clear(); - result.columns.reserve(block.columns()); - for (auto & col : block) - result.columns.emplace_back(std::move(col.column)); - - current_step_filter = result.columns[prewhere_column_pos]; } - /// In case when we are returning prewhere column the caller expects it to serve as a final filter: - /// it must contain 0s not only from the current step but also from all the previous steps. - /// One way to achieve this is to apply the final_filter if we know that the final_filter was not applied at - /// several previous steps but was accumulated instead. - result.can_return_prewhere_column_without_filtering = result.filterWasApplied(); + for (auto name_and_type = header.begin(); name_and_type != header.end() && pos < result.columns.size(); ++pos, ++name_and_type) + block.insert({result.columns[pos], name_and_type->type, name_and_type->name}); - if (prewhere_info->remove_column) - result.columns.erase(result.columns.begin() + prewhere_column_pos); - - FilterWithCachedCount current_filter(current_step_filter); - - result.optimize(current_filter, merge_tree_reader->canReadIncompleteGranules()); - - if (prewhere_info->need_filter && !result.filterWasApplied()) { - /// Depending on whether the final filter was applied at the previous step or not we need to apply either - /// just the current step filter or the accumulated filter. - FilterWithCachedCount filter_to_apply = - current_filter.size() == result.total_rows_per_granule ? - result.final_filter : - current_filter; + /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. + Block additional_columns = block; + + if (prewhere_info->actions) + { + const String dummy_column = addDummyColumnWithRowCount(block, result.num_rows); + + LOG_TEST(log, "Executing prewhere actions on block: {}", block.dumpStructure()); + + prewhere_info->actions->execute(block); + + if (!dummy_column.empty()) + block.erase(dummy_column); + } + + result.additional_columns.clear(); + /// Additional columns might only be needed if there are more steps in the chain. + if (!last_reader_in_chain) + { + for (auto & col : additional_columns) + { + /// Exclude columns that are present in the result block to avoid storing them and filtering twice. + /// TODO: also need to exclude the columns that are not needed for the next steps. + if (block.has(col.name)) + continue; + result.additional_columns.insert(col); + } + } + } + + result.columns.clear(); + result.columns.reserve(block.columns()); + for (auto & col : block) + result.columns.emplace_back(std::move(col.column)); + + if (prewhere_info->type == PrewhereExprStep::Filter) + { + /// Filter computed at the current step. Its size is equal to num_rows which is <= total_rows_per_granule + size_t filter_column_pos = block.getPositionByName(prewhere_info->filter_column_name); + auto current_step_filter = result.columns[filter_column_pos]; + + /// In case when we are returning prewhere column the caller expects it to serve as a final filter: + /// it must contain 0s not only from the current step but also from all the previous steps. + /// One way to achieve this is to apply the final_filter if we know that the final_filter was not applied at + /// several previous steps but was accumulated instead. + result.can_return_prewhere_column_without_filtering = result.filterWasApplied(); + + if (prewhere_info->remove_filter_column) + result.columns.erase(result.columns.begin() + filter_column_pos); + + FilterWithCachedCount current_filter(current_step_filter); + result.optimize(current_filter, merge_tree_reader->canReadIncompleteGranules()); + + if (prewhere_info->need_filter && !result.filterWasApplied()) + { + /// Depending on whether the final filter was applied at the previous step or not we need to apply either + /// just the current step filter or the accumulated filter. + FilterWithCachedCount filter_to_apply = + current_filter.size() == result.total_rows_per_granule + ? result.final_filter + : current_filter; result.applyFilter(filter_to_apply); + } } LOG_TEST(log, "After execute prewhere {}", result.dumpInfo()); @@ -1415,12 +1409,12 @@ std::string PrewhereExprInfo::dump() const for (size_t i = 0; i < steps.size(); ++i) { s << "STEP " << i << ":\n" - << " ACTIONS: " << (steps[i].actions ? - (indent + boost::replace_all_copy(steps[i].actions->dumpActions(), "\n", indent)) : + << " ACTIONS: " << (steps[i]->actions ? + (indent + boost::replace_all_copy(steps[i]->actions->dumpActions(), "\n", indent)) : "nullptr") << "\n" - << " COLUMN: " << steps[i].column_name << "\n" - << " REMOVE_COLUMN: " << steps[i].remove_column << "\n" - << " NEED_FILTER: " << steps[i].need_filter << "\n\n"; + << " COLUMN: " << steps[i]->filter_column_name << "\n" + << " REMOVE_COLUMN: " << steps[i]->remove_filter_column << "\n" + << " NEED_FILTER: " << steps[i]->need_filter << "\n\n"; } return s.str(); @@ -1431,7 +1425,7 @@ std::string PrewhereExprInfo::dumpConditions() const WriteBufferFromOwnString s; for (size_t i = 0; i < steps.size(); ++i) - s << (i == 0 ? "\"" : ", \"") << steps[i].column_name << "\""; + s << (i == 0 ? "\"" : ", \"") << steps[i]->filter_column_name << "\""; return s.str(); } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index f180787c869..04d42138963 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -22,19 +22,33 @@ using ExpressionActionsPtr = std::shared_ptr; struct PrewhereExprStep { + enum Type + { + Filter, + Expression, + }; + + Type type = Type::Filter; ExpressionActionsPtr actions; - String column_name; - bool remove_column = false; + String filter_column_name; + + bool remove_filter_column = false; bool need_filter = false; + + /// Some PREWHERE steps should be executed without conversions. + /// A step without alter conversion cannot be executed after step with alter conversions. + bool perform_alter_conversions = false; }; +using PrewhereExprStepPtr = std::shared_ptr; +using PrewhereExprSteps = std::vector; + /// The same as PrewhereInfo, but with ExpressionActions instead of ActionsDAG struct PrewhereExprInfo { - std::vector steps; + PrewhereExprSteps steps; std::string dump() const; - std::string dumpConditions() const; }; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index c26958e12b6..0ffc91aca57 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -92,16 +93,18 @@ std::vector MergeTreeReadPool::fillPerPartInfo( per_part_sum_marks.push_back(sum_marks); + auto & per_part = per_part_params.emplace_back(); + per_part.data_part = part; + + LoadedMergeTreeDataPartInfoForReader part_info(part.data_part, part.alter_conversions); auto task_columns = getReadTaskColumns( - LoadedMergeTreeDataPartInfoForReader(part.data_part), storage_snapshot, - column_names, virtual_column_names, prewhere_info, actions_settings, reader_settings, /*with_subcolumns=*/ true); + part_info, storage_snapshot, column_names, virtual_column_names, + prewhere_info, actions_settings, + reader_settings, /*with_subcolumns=*/ true); auto size_predictor = !predict_block_size_bytes ? nullptr : IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block); - auto & per_part = per_part_params.emplace_back(); - - per_part.data_part = part; per_part.size_predictor = std::move(size_predictor); /// will be used to distinguish between PREWHERE and WHERE columns when applying filter @@ -202,13 +205,16 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t thread) } const auto & per_part = per_part_params[part_idx]; - auto curr_task_size_predictor = !per_part.size_predictor ? nullptr : std::make_unique(*per_part.size_predictor); /// make a copy return std::make_unique( - part.data_part, ranges_to_get_from_part, part.part_index_in_query, - per_part.column_name_set, per_part.task_columns, + part.data_part, + part.alter_conversions, + ranges_to_get_from_part, + part.part_index_in_query, + per_part.column_name_set, + per_part.task_columns, std::move(curr_task_size_predictor)); } @@ -456,6 +462,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t thread) return std::make_unique( part.data_part, + part.alter_conversions, ranges_to_read, part.part_index_in_query, per_part.column_name_set, @@ -515,5 +522,4 @@ MarkRanges MergeTreeInOrderReadPoolParallelReplicas::getNewTask(RangesInDataPart return {}; } - } diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index b3356ec3351..e621a02a9d6 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -8,6 +8,8 @@ #include #include #include +#include +#include #include @@ -164,7 +166,6 @@ private: class MergeTreeReadPoolParallelReplicas : public IMergeTreeReadPool { public: - MergeTreeReadPoolParallelReplicas( StorageSnapshotPtr storage_snapshot_, size_t threads_, diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index da2d0b0ae4a..bb515bcd10f 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -48,8 +48,13 @@ bool MergeTreeReverseSelectAlgorithm::getNewTaskOrdinaryReading() : getSizePredictor(data_part, task_columns, sample_block); task = std::make_unique( - data_part, mark_ranges_for_task, part_index_in_query, column_name_set, - task_columns, std::move(size_predictor)); + data_part, + alter_conversions, + mark_ranges_for_task, + part_index_in_query, + column_name_set, + task_columns, + std::move(size_predictor)); return true; @@ -86,8 +91,13 @@ bool MergeTreeReverseSelectAlgorithm::getNewTaskParallelReplicas() : getSizePredictor(data_part, task_columns, sample_block); task = std::make_unique( - data_part, mark_ranges_for_task, part_index_in_query, column_name_set, - task_columns, std::move(size_predictor)); + data_part, + alter_conversions, + mark_ranges_for_task, + part_index_in_query, + column_name_set, + task_columns, + std::move(size_predictor)); return true; } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 83e9cc90547..ce4ba69c08c 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -12,6 +12,7 @@ MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm( const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, const MergeTreeData::DataPartPtr & owned_data_part_, + const AlterConversionsPtr & alter_conversions_, UInt64 max_block_size_rows_, size_t preferred_block_size_bytes_, size_t preferred_max_column_in_block_size_bytes_, @@ -32,6 +33,7 @@ MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm( reader_settings_, use_uncompressed_cache_, virt_column_names_}, required_columns{std::move(required_columns_)}, data_part{owned_data_part_}, + alter_conversions(alter_conversions_), sample_block(storage_snapshot_->metadata->getSampleBlock()), all_mark_ranges(std::move(mark_ranges_)), part_index_in_query(part_index_in_query_), @@ -44,9 +46,13 @@ MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm( void MergeTreeSelectAlgorithm::initializeReaders() { + LoadedMergeTreeDataPartInfoForReader part_info(data_part, alter_conversions); + task_columns = getReadTaskColumns( - LoadedMergeTreeDataPartInfoForReader(data_part), storage_snapshot, - required_columns, virt_column_names, prewhere_info, actions_settings, reader_settings, /*with_subcolumns=*/ true); + part_info, storage_snapshot, + required_columns, virt_column_names, + prewhere_info, + actions_settings, reader_settings, /*with_subcolumns=*/ true); /// Will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & column_names = task_columns.columns.getNames(); @@ -58,7 +64,8 @@ void MergeTreeSelectAlgorithm::initializeReaders() owned_mark_cache = storage.getContext()->getMarkCache(); initializeMergeTreeReadersForPart( - data_part, task_columns, storage_snapshot->getMetadataForQuery(), all_mark_ranges, {}, {}); + data_part, alter_conversions, task_columns, + storage_snapshot->getMetadataForQuery(), all_mark_ranges, {}, {}); } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 981c42574e0..5f4f49bf075 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -20,7 +20,8 @@ public: MergeTreeSelectAlgorithm( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot_, - const MergeTreeData::DataPartPtr & owned_data_part, + const MergeTreeData::DataPartPtr & owned_data_part_, + const AlterConversionsPtr & alter_conversions_, UInt64 max_block_size_rows, size_t preferred_block_size_bytes, size_t preferred_max_column_in_block_size_bytes, @@ -54,6 +55,9 @@ protected: /// Data part will not be removed if the pointer owns it MergeTreeData::DataPartPtr data_part; + /// Alter converversionss that should be applied on-fly for part. + AlterConversionsPtr alter_conversions; + /// Cache getSampleBlock call, which might be heavy. Block sample_block; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index ef6013f5c65..e27354f9d16 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -111,12 +111,14 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( data_part->getMarksCount(), data_part->name, data_part->rows_count); } + auto alter_conversions = storage.getAlterConversionsForPart(data_part); + /// Note, that we don't check setting collaborate_with_coordinator presence, because this source /// is only used in background merges. addTotalRowsApprox(data_part->rows_count); /// Add columns because we don't want to read empty blocks - injectRequiredColumns(LoadedMergeTreeDataPartInfoForReader(data_part), storage_snapshot, /*with_subcolumns=*/ false, columns_to_read); + injectRequiredColumns(LoadedMergeTreeDataPartInfoForReader(data_part, alter_conversions), storage_snapshot, /*with_subcolumns=*/ false, columns_to_read); NamesAndTypesList columns_for_reader; if (take_column_types_from_storage) @@ -146,9 +148,10 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( if (!mark_ranges) mark_ranges.emplace(MarkRanges{MarkRange(0, data_part->getMarksCount())}); - reader = data_part->getReader(columns_for_reader, storage_snapshot->metadata, - *mark_ranges, - /* uncompressed_cache = */ nullptr, mark_cache.get(), reader_settings, {}, {}); + reader = data_part->getReader( + columns_for_reader, storage_snapshot->metadata, + *mark_ranges, /* uncompressed_cache = */ nullptr, + mark_cache.get(), alter_conversions, reader_settings, {}, {}); } Chunk MergeTreeSequentialSource::generate() @@ -241,10 +244,9 @@ Pipe createMergeTreeSequentialSource( if (need_to_filter_deleted_rows) columns.emplace_back(LightweightDeleteDescription::FILTER_COLUMN.name); - bool apply_deleted_mask = false; - auto column_part_source = std::make_shared( - storage, storage_snapshot, data_part, columns, std::optional{}, apply_deleted_mask, read_with_direct_io, take_column_types_from_storage, quiet); + storage, storage_snapshot, data_part, columns, std::optional{}, + /*apply_deleted_mask=*/ false, read_with_direct_io, take_column_types_from_storage, quiet); Pipe pipe(std::move(column_part_source)); @@ -316,7 +318,8 @@ public: } auto source = std::make_unique( - storage, storage_snapshot, data_part, columns_to_read, std::move(mark_ranges), apply_deleted_mask, false, true); + storage, storage_snapshot, data_part, columns_to_read, + std::move(mark_ranges), apply_deleted_mask, false, true); pipeline.init(Pipe(std::move(source))); } @@ -344,7 +347,9 @@ void createMergeTreeSequentialSource( Poco::Logger * log) { auto reading = std::make_unique( - storage, storage_snapshot, std::move(data_part), std::move(columns_to_read), apply_deleted_mask, filter, std::move(context), log); + storage, storage_snapshot, std::move(data_part), + std::move(columns_to_read), apply_deleted_mask, + filter, std::move(context), log); plan.addStep(std::move(reading)); } diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 981eb1af280..d62fe5024f4 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -39,7 +39,7 @@ MergeTreeSink::MergeTreeSink( , metadata_snapshot(metadata_snapshot_) , max_parts_per_block(max_parts_per_block_) , context(context_) - , storage_snapshot(storage.getStorageSnapshotWithoutParts(metadata_snapshot)) + , storage_snapshot(storage.getStorageSnapshotWithoutData(metadata_snapshot, context_)) { } diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index bfc674a7eef..7ca8983bfda 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -330,15 +330,20 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction { for (const auto & step : steps) { - prewhere.steps.push_back( + PrewhereExprStep new_step { + .type = PrewhereExprStep::Filter, .actions = std::make_shared(step.actions, actions_settings), - .column_name = step.column_name, - .remove_column = !all_output_names.contains(step.column_name), /// Don't remove if it's in the list of original outputs + .filter_column_name = step.column_name, + .remove_filter_column = !all_output_names.contains(step.column_name), /// Don't remove if it's in the list of original outputs .need_filter = false, - }); + .perform_alter_conversions = true, + }; + + prewhere.steps.push_back(std::make_shared(std::move(new_step))); } - prewhere.steps.back().need_filter = prewhere_info->need_filter; + + prewhere.steps.back()->need_filter = prewhere_info->need_filter; } return true; diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp index 5c46c906b31..01094d65ac5 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp @@ -57,9 +57,7 @@ void MergeTreeThreadSelectAlgorithm::finalizeNewTask() /// task->reader.valid() means there is a prefetched reader in this test, use it. const bool init_new_readers = !reader || task->reader.valid() || part_name != last_read_part_name; if (init_new_readers) - { initializeMergeTreeReadersForCurrentTask(metadata_snapshot, value_size_map, profile_callback); - } last_read_part_name = part_name; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index af643050504..76096d00641 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -56,10 +56,10 @@ static bool checkOperationIsNotCanceled(ActionBlocker & merges_blocker, MergeLis */ static void splitAndModifyMutationCommands( MergeTreeData::DataPartPtr part, + StorageMetadataPtr metadata_snapshot, const MutationCommands & commands, MutationCommands & for_interpreter, MutationCommands & for_file_renames, - const StorageMetadataPtr & table_metadata_snapshot, Poco::Logger * log) { auto part_columns = part->getColumnsDescription(); @@ -114,7 +114,7 @@ static void splitAndModifyMutationCommands( /// It's important because required renames depend not only on part's data version (i.e. mutation version) /// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter /// can be deduced based on difference between part's schema and table schema. - for (const auto & [rename_to, rename_from] : alter_conversions.rename_map) + for (const auto & [rename_to, rename_from] : alter_conversions->getRenameMap()) { if (part_columns.has(rename_from)) { @@ -144,14 +144,14 @@ static void splitAndModifyMutationCommands( { if (!mutated_columns.contains(column.name)) { - if (!table_metadata_snapshot->getColumns().has(column.name) && !part->storage.getVirtuals().contains(column.name)) + if (!metadata_snapshot->getColumns().has(column.name) && !part->storage.getVirtuals().contains(column.name)) { /// We cannot add the column because there's no such column in table. /// It's okay if the column was dropped. It may also absent in dropped_columns /// if the corresponding MUTATE_PART entry was not created yet or was created separately from current MUTATE_PART. /// But we don't know for sure what happened. auto part_metadata_version = part->getMetadataVersion(); - auto table_metadata_version = table_metadata_snapshot->getMetadataVersion(); + auto table_metadata_version = metadata_snapshot->getMetadataVersion(); /// StorageMergeTree does not have metadata version if (table_metadata_version <= part_metadata_version && part->storage.supportsReplication()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} with metadata version {} contains column {} that is absent " @@ -219,7 +219,7 @@ static void splitAndModifyMutationCommands( /// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter /// can be deduced based on difference between part's schema and table schema. - for (const auto & [rename_to, rename_from] : alter_conversions.rename_map) + for (const auto & [rename_to, rename_from] : alter_conversions->getRenameMap()) { for_file_renames.push_back({.type = MutationCommand::Type::RENAME_COLUMN, .column_name = rename_from, .rename_to = rename_to}); } @@ -1653,7 +1653,7 @@ MutateTask::MutateTask( ctx->source_part = ctx->future_part->parts[0]; ctx->need_prefix = need_prefix_; - auto storage_snapshot = ctx->data->getStorageSnapshot(ctx->metadata_snapshot, context_); + auto storage_snapshot = ctx->data->getStorageSnapshotWithoutData(ctx->metadata_snapshot, context_); extendObjectColumns(ctx->storage_columns, storage_snapshot->object_columns, /*with_subcolumns=*/ false); } @@ -1748,6 +1748,7 @@ bool MutateTask::prepare() /// Allow mutations to work when force_index_by_date or force_primary_key is on. context_for_reading->setSetting("force_index_by_date", false); context_for_reading->setSetting("force_primary_key", false); + context_for_reading->setSetting("apply_mutations_on_fly", false); /// Skip using large sets in KeyCondition context_for_reading->setSetting("use_index_for_in_with_subqueries_max_values", 100000); @@ -1800,20 +1801,25 @@ bool MutateTask::prepare() context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false); context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0)); - MutationHelpers::splitAndModifyMutationCommands(ctx->source_part, ctx->commands_for_part, ctx->for_interpreter, - ctx->for_file_renames, ctx->metadata_snapshot, ctx->log); + MutationHelpers::splitAndModifyMutationCommands( + ctx->source_part, ctx->metadata_snapshot, + ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames, ctx->log); ctx->stage_progress = std::make_unique(1.0); if (!ctx->for_interpreter.empty()) { + /// Always disable filtering in mutations: we want to read and write all rows because for updates we rewrite only some of the + /// columns and preserve the columns that are not affected, but after the update all columns must have the same number of row + MutationsInterpreter::Settings settings(true); + settings.apply_deleted_mask = false; + ctx->interpreter = std::make_unique( - *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->for_interpreter, context_for_reading, true); + *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->for_interpreter, + ctx->metadata_snapshot->getColumns().getNamesOfPhysical(), context_for_reading, settings); + ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); ctx->materialized_projections = ctx->interpreter->grabMaterializedProjections(); - /// Always disable filtering in mutations: we want to read and write all rows because for updates we rewrite only some of the - /// columns and preserve the columns that are not affected, but after the update all columns must have the same number of rows. - ctx->interpreter->setApplyDeletedMask(false); ctx->mutating_pipeline_builder = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress); diff --git a/src/Storages/MergeTree/RangesInDataPart.h b/src/Storages/MergeTree/RangesInDataPart.h index 9c8ab4859a0..afb3ad33762 100644 --- a/src/Storages/MergeTree/RangesInDataPart.h +++ b/src/Storages/MergeTree/RangesInDataPart.h @@ -5,6 +5,7 @@ #include #include #include +#include "Storages/MergeTree/AlterConversions.h" #include "Storages/MergeTree/MergeTreePartInfo.h" @@ -40,6 +41,7 @@ struct RangesInDataPartsDescription: public std::deque ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const { std::unique_lock lock(state_mutex); + auto in_partition = mutations_by_partition.find(part->info.partition_id); if (in_partition == mutations_by_partition.end()) return {}; + Int64 part_data_version = part->info.getDataVersion(); Int64 part_metadata_version = part->getMetadataVersion(); + LOG_DEBUG(log, "Looking for mutations for part {} (part data version {}, part metadata version {})", part->name, part_data_version, part_metadata_version); + std::map result; /// Here we return mutation commands for part which has bigger alter version than part metadata version. /// Please note, we don't use getDataVersion(). It's because these alter commands are used for in-fly conversions /// of part's metadata. for (const auto & [mutation_version, mutation_status] : in_partition->second | std::views::reverse) { - int32_t alter_version = mutation_status->entry->alter_version; + auto alter_version = mutation_status->entry->alter_version; if (alter_version != -1) { if (alter_version > storage.getInMemoryMetadataPtr()->getMetadataVersion()) continue; - /// we take commands with bigger metadata version + /// We take commands with bigger metadata version if (alter_version > part_metadata_version) - { result[mutation_version] = mutation_status->entry->commands; - } - else - { - /// entries are ordered, we processing them in reverse order so we can break - break; - } + } + else if (mutation_version > part_data_version) + { + result[mutation_version] = mutation_status->entry->commands; } } + LOG_TRACE(log, "Got {} commands for part {} (part data version {}, part metadata version {})", + result.size(), part->name, part_data_version, part_metadata_version); + return result; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 1c39b53dcc2..a38e9eba844 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -275,7 +275,7 @@ ReplicatedMergeTreeSinkImpl::ReplicatedMergeTreeSinkImpl( , deduplicate(deduplicate_) , log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) , context(context_) - , storage_snapshot(storage.getStorageSnapshotWithoutParts(metadata_snapshot)) + , storage_snapshot(storage.getStorageSnapshotWithoutData(metadata_snapshot, context_)) { /// The quorum value `1` has the same meaning as if it is disabled. if (required_quorum_size == 1) diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 2e0ad116d70..17078d3e73b 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -28,6 +29,7 @@ public: explicit StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) : IStorage(getIDFromPart(part_)) , parts({part_}) + , alter_conversions({part_->storage.getAlterConversionsForPart(part_)}) , storage(part_->storage) , partition_id(part_->info.partition_id) { @@ -54,7 +56,7 @@ public: parts.begin(), parts.end(), storage_columns, [](const auto & part) -> const auto & { return part->getColumns(); }); - return std::make_shared(*this, metadata_snapshot, object_columns); + return std::make_shared(*this, metadata_snapshot, std::move(object_columns)); } void read( @@ -70,6 +72,7 @@ public: query_plan.addStep(MergeTreeDataSelectExecutor(storage) .readFromParts( parts, + alter_conversions, column_names, storage_snapshot, query_info, @@ -126,6 +129,7 @@ public: private: const MergeTreeData::DataPartsVector parts; + const std::vector alter_conversions; const MergeTreeData & storage; const String partition_id; const MergeTreeDataSelectAnalysisResultPtr analysis_result_ptr; diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index aa77988348d..6eb345b449e 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -185,7 +185,6 @@ void MutationCommands::readText(ReadBuffer & in) String commands_str; readEscapedString(commands_str, in); - ParserAlterCommandList p_alter_commands; auto commands_ast = parseQuery( p_alter_commands, commands_str.data(), commands_str.data() + commands_str.length(), "mutation commands list", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index a6fe10ef288..d1195a9132e 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -230,14 +230,17 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt if (commands.front().type == MutationCommand::Type::DELETE) { + MutationsInterpreter::Settings settings(true); + settings.return_all_columns = true; + settings.return_mutated_rows = true; + auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, context_, - /*can_execute_*/ true, - /*return_all_columns_*/ true, - /*return_mutated_rows*/ true); + settings); + auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); @@ -278,14 +281,17 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt if (commands.front().column_to_update_expression.contains(primary_key)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key cannot be updated (cannot update column {})", primary_key); + MutationsInterpreter::Settings settings(true); + settings.return_all_columns = true; + settings.return_mutated_rows = true; + auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, context_, - /*can_execute_*/ true, - /*return_all_columns*/ true, - /*return_mutated_rows*/ true); + settings); + auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index dd2d3ebfaf0..fc552a5ab6d 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -644,7 +644,7 @@ StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( metadata_snapshot->getColumns(), [](const auto & shard_num_and_columns) -> const auto & { return shard_num_and_columns.second; }); - return std::make_shared(*this, metadata_snapshot, object_columns, std::move(snapshot_data)); + return std::make_shared(*this, metadata_snapshot, std::move(object_columns), std::move(snapshot_data)); } namespace diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 23bcdd23484..5113320548d 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -138,7 +138,8 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) // New scope controls lifetime of pipeline. { auto storage_ptr = DatabaseCatalog::instance().getTable(getStorageID(), context); - auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, true); + MutationsInterpreter::Settings settings(true); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context, settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index d81a04610de..deebb9e0096 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -857,14 +857,17 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca if (commands.front().type == MutationCommand::Type::DELETE) { + MutationsInterpreter::Settings settings(true); + settings.return_all_columns = true; + settings.return_mutated_rows = true; + auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, local_context, - /*can_execute_*/ true, - /*return_all_columns_*/ true, - /*return_mutated_rows*/ true); + settings); + auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); @@ -926,14 +929,17 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca if (commands.front().column_to_update_expression.contains(primary_key)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key cannot be updated (cannot update column {})", primary_key); + MutationsInterpreter::Settings settings(true); + settings.return_all_columns = true; + settings.return_mutated_rows = true; + auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, local_context, - /*can_execute_*/ true, - /*return_all_columns*/ true, - /*return_mutated_rows*/ true); + settings); + auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 31e45db55cb..caeefa5d96d 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -142,7 +142,7 @@ StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot->getColumns(), [](const auto & block) -> const auto & { return block.getColumnsWithTypeAndName(); }); - return std::make_shared(*this, metadata_snapshot, object_columns, std::move(snapshot_data)); + return std::make_shared(*this, metadata_snapshot, std::move(object_columns), std::move(snapshot_data)); } void StorageMemory::read( @@ -200,7 +200,8 @@ void StorageMemory::mutate(const MutationCommands & commands, ContextPtr context new_context->setSetting("max_streams_to_max_threads_ratio", 1); new_context->setSetting("max_threads", 1); - auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, new_context, true); + MutationsInterpreter::Settings settings(true); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, new_context, settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2c19d3ba122..a721dd30cd7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -254,6 +254,7 @@ void StorageMergeTree::read( /// reset them to avoid holding them. auto & snapshot_data = assert_cast(*storage_snapshot->data); snapshot_data.parts = {}; + snapshot_data.alter_conversions = {}; } std::optional StorageMergeTree::totalRows(const Settings &) const @@ -1164,8 +1165,9 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( auto fake_query_context = Context::createCopy(getContext()); fake_query_context->makeQueryContext(); fake_query_context->setCurrentQueryId(""); + MutationsInterpreter::Settings settings(false); MutationsInterpreter interpreter( - shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, false); + shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, settings); commands_size += interpreter.evaluateCommandsSize(); } catch (...) @@ -2193,17 +2195,17 @@ std::map StorageMergeTree::getAlterMutationCommandsFo { std::lock_guard lock(currently_processing_in_background_mutex); - Int64 part_data_version = part->info.getDataVersion(); - + UInt64 part_data_version = part->info.getDataVersion(); std::map result; - if (!current_mutations_by_version.empty()) + + for (const auto & [mutation_version, entry] : current_mutations_by_version | std::views::reverse) { - const auto & [latest_mutation_id, latest_commands] = *current_mutations_by_version.rbegin(); - if (part_data_version < static_cast(latest_mutation_id)) - { - result[latest_mutation_id] = latest_commands.commands; - } + if (mutation_version > part_data_version) + result[mutation_version] = entry.commands; + else + break; } + return result; } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 9e75d6a9373..be9f5426bbd 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -196,7 +196,6 @@ private: void waitForMutation(Int64 version, const String & mutation_id, bool wait_for_another_mutation = false); void setMutationCSN(const String & mutation_id, CSN csn) override; - friend struct CurrentlyMergingPartsTagger; MergeMutateSelectedEntryPtr selectPartsToMerge( @@ -280,7 +279,6 @@ private: protected: - std::map getAlterMutationCommandsForPart(const DataPartPtr & part) const override; }; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7b9a3093e40..0f79e9f8f19 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4668,6 +4668,7 @@ void StorageReplicatedMergeTree::read( /// reset them to avoid holding them. auto & snapshot_data = assert_cast(*storage_snapshot->data); snapshot_data.parts = {}; + snapshot_data.alter_conversions = {}; }); /** The `select_sequential_consistency` setting has two meanings: @@ -8116,7 +8117,6 @@ std::unique_ptr StorageReplicatedMergeTree::getDefaultSetting return std::make_unique(getContext()->getReplicatedMergeTreeSettings()); } - String StorageReplicatedMergeTree::getTableSharedID() const { std::lock_guard lock(table_shared_id_mutex); diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index 723b30e49e6..946e8a98cf2 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -30,27 +30,32 @@ struct StorageSnapshot StorageSnapshot( const IStorage & storage_, - const StorageMetadataPtr & metadata_) - : storage(storage_), metadata(metadata_) + StorageMetadataPtr metadata_) + : storage(storage_), metadata(std::move(metadata_)) { init(); } StorageSnapshot( const IStorage & storage_, - const StorageMetadataPtr & metadata_, - const ColumnsDescription & object_columns_) - : storage(storage_), metadata(metadata_), object_columns(object_columns_) + StorageMetadataPtr metadata_, + ColumnsDescription object_columns_) + : storage(storage_) + , metadata(std::move(metadata_)) + , object_columns(std::move(object_columns_)) { init(); } StorageSnapshot( const IStorage & storage_, - const StorageMetadataPtr & metadata_, - const ColumnsDescription & object_columns_, + StorageMetadataPtr metadata_, + ColumnsDescription object_columns_, DataPtr data_) - : storage(storage_), metadata(metadata_), object_columns(object_columns_), data(std::move(data_)) + : storage(storage_) + , metadata(std::move(metadata_)) + , object_columns(std::move(object_columns_)) + , data(std::move(data_)) { init(); } diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.reference b/tests/queries/0_stateless/01786_explain_merge_tree.reference index 4a3fe99710b..e6628813dbd 100644 --- a/tests/queries/0_stateless/01786_explain_merge_tree.reference +++ b/tests/queries/0_stateless/01786_explain_merge_tree.reference @@ -24,12 +24,12 @@ Name: t_minmax Description: minmax GRANULARITY 2 Parts: 1/2 - Granules: 2/6 + Granules: 4/6 Skip Name: t_set Description: set GRANULARITY 2 Parts: 1/1 - Granules: 1/2 + Granules: 2/4 ----------------- "Node Type": "ReadFromMergeTree", "Description": "default.test_index", @@ -68,7 +68,7 @@ "Initial Parts": 2, "Selected Parts": 1, "Initial Granules": 6, - "Selected Granules": 2 + "Selected Granules": 4 }, { "Type": "Skip", @@ -76,8 +76,8 @@ "Description": "set GRANULARITY 2", "Initial Parts": 1, "Selected Parts": 1, - "Initial Granules": 2, - "Selected Granules": 1 + "Initial Granules": 4, + "Selected Granules": 2 } ] } From b3a96de533c27283540d0ecad2054a35f86c9357 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 26 May 2023 00:04:24 +0100 Subject: [PATCH 0702/2223] Pure parallel replicas: JOIN support (#49544) --- src/Interpreters/ExpressionAnalyzer.cpp | 9 + src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 14 +- src/Interpreters/InterpreterSelectQuery.cpp | 31 +-- src/Interpreters/interpretSubquery.cpp | 2 - .../02535_max_parallel_replicas_custom_key.sh | 2 +- ...708_parallel_replicas_not_found_column.sql | 1 + ..._parallel_replicas_join_subquery.reference | 44 +++++ .../02731_parallel_replicas_join_subquery.sql | 182 ++++++++++++++++++ ...l_replicas_bug_chunkinfo_not_set.reference | 0 ...arallel_replicas_bug_chunkinfo_not_set.sql | 43 +++++ ...764_parallel_replicas_plain_merge_tree.sql | 2 +- 12 files changed, 312 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference create mode 100644 tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql create mode 100644 tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.reference create mode 100644 tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 96a86df7ffd..c7c66f6f414 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -969,6 +969,15 @@ const ASTSelectQuery * ExpressionAnalyzer::getSelectQuery() const return select_query; } +bool ExpressionAnalyzer::isRemoteStorage() const +{ + const Settings & csettings = getContext()->getSettingsRef(); + // Consider any storage used in parallel replicas as remote, so the query is executed in multiple servers + const bool enable_parallel_processing_of_joins + = csettings.max_parallel_replicas > 1 && csettings.allow_experimental_parallel_reading_from_replicas > 0; + return syntax->is_remote_storage || enable_parallel_processing_of_joins; +} + const ASTSelectQuery * SelectQueryExpressionAnalyzer::getAggregatingQuery() const { if (!has_aggregation) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 1b6e8e24091..00cd353aa66 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -201,7 +201,7 @@ protected: const ASTSelectQuery * getSelectQuery() const; - bool isRemoteStorage() const { return syntax->is_remote_storage; } + bool isRemoteStorage() const; NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns); NamesAndTypesList analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index b105cae31c6..08862032007 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -205,10 +205,19 @@ public: } private: + static bool shouldBeExecutedGlobally(const Data & data) + { + const Settings & settings = data.getContext()->getSettingsRef(); + /// For parallel replicas we reinterpret JOIN as GLOBAL JOIN as a way to broadcast data + const bool enable_parallel_processing_of_joins = data.getContext()->canUseParallelReplicasOnInitiator(); + return settings.prefer_global_in_and_join || enable_parallel_processing_of_joins; + } + + /// GLOBAL IN static void visit(ASTFunction & func, ASTPtr &, Data & data) { - if ((data.getContext()->getSettingsRef().prefer_global_in_and_join + if ((shouldBeExecutedGlobally(data) && (func.name == "in" || func.name == "notIn" || func.name == "nullIn" || func.name == "notNullIn")) || func.name == "globalIn" || func.name == "globalNotIn" || func.name == "globalNullIn" || func.name == "globalNotNullIn") { @@ -238,8 +247,7 @@ private: static void visit(ASTTablesInSelectQueryElement & table_elem, ASTPtr &, Data & data) { if (table_elem.table_join - && (table_elem.table_join->as().locality == JoinLocality::Global - || data.getContext()->getSettingsRef().prefer_global_in_and_join)) + && (table_elem.table_join->as().locality == JoinLocality::Global || shouldBeExecutedGlobally(data))) { data.addExternalStorage(table_elem.table_expression, true); data.has_global_subqueries = true; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ebaf88ea5d5..d4ca2e405e6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -458,19 +458,11 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } - /// Check support for JOINs for parallel replicas - if (joined_tables.tablesCount() > 1 && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas > 0)) + /// Check support for JOIN for parallel replicas with custom key + if (joined_tables.tablesCount() > 1 && !settings.parallel_replicas_custom_key.value.empty()) { - if (settings.allow_experimental_parallel_reading_from_replicas == 1) - { - LOG_WARNING(log, "JOINs are not supported with parallel replicas. Query will be executed without using them."); - context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); - context->setSetting("parallel_replicas_custom_key", String{""}); - } - else if (settings.allow_experimental_parallel_reading_from_replicas == 2) - { - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "JOINs are not supported with parallel replicas"); - } + LOG_WARNING(log, "JOINs are not supported with parallel_replicas_custom_key. Query will be executed without using them."); + context->setSetting("parallel_replicas_custom_key", String{""}); } /// Check support for FINAL for parallel replicas @@ -489,6 +481,21 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } + /// Check support for parallel replicas for non-replicated storage (plain MergeTree) + bool is_plain_merge_tree = storage && storage->isMergeTree() && !storage->supportsReplication(); + if (is_plain_merge_tree && settings.allow_experimental_parallel_reading_from_replicas > 0 && !settings.parallel_replicas_for_non_replicated_merge_tree) + { + if (settings.allow_experimental_parallel_reading_from_replicas == 1) + { + LOG_WARNING(log, "To use parallel replicas with plain MergeTree tables please enable setting `parallel_replicas_for_non_replicated_merge_tree`. For now query will be executed without using them."); + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + } + else if (settings.allow_experimental_parallel_reading_from_replicas == 2) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "To use parallel replicas with plain MergeTree tables please enable setting `parallel_replicas_for_non_replicated_merge_tree`"); + } + } + /// Rewrite JOINs if (!has_input && joined_tables.tablesCount() > 1) { diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 550fa2912ba..5f00be07fa5 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -112,8 +112,6 @@ std::shared_ptr interpretSubquery( subquery_options.removeDuplicates(); } - /// We don't want to execute reading for subqueries in parallel - subquery_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); return std::make_shared(query, subquery_context, subquery_options, required_source_columns); } diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 50e89cca4c9..9850406eb3a 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -41,6 +41,6 @@ run_count_with_custom_key "y" run_count_with_custom_key "cityHash64(y)" run_count_with_custom_key "cityHash64(y) + 1" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with parallel replicas" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" diff --git a/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql b/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql index 8900025502c..ff7b53ce01f 100644 --- a/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql +++ b/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql @@ -1,3 +1,4 @@ CREATE TABLE IF NOT EXISTS t_02708(x DateTime) ENGINE = MergeTree ORDER BY tuple(); +SET send_logs_level='error'; SELECT count() FROM t_02708 SETTINGS allow_experimental_parallel_reading_from_replicas=1; DROP TABLE t_02708; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference new file mode 100644 index 00000000000..df606679523 --- /dev/null +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -0,0 +1,44 @@ +=============== INNER QUERY (NO PARALLEL) =============== +0 PJFiUe#J2O _s\' 14427935816175499794 +1 >T%O ,z< 17537932797009027240 +12 D[6,P #}Lmb[ ZzU 6394957109822140795 +18 $_N- 24422838680427462 +2 bX?}ix [ Ny]2 G 16242612901291874718 +20 VE] Y 15120036904703536841 +22 Ti~3)N)< A!( 3 18361093572663329113 +23 Sx>b:^UG XpedE)Q: 7433019734386307503 +29 2j&S)ba?XG QuQj 17163829389637435056 +3 UlI+1 14144472852965836438 +=============== INNER QUERY (PARALLEL) =============== +0 PJFiUe#J2O _s\' 14427935816175499794 +1 >T%O ,z< 17537932797009027240 +12 D[6,P #}Lmb[ ZzU 6394957109822140795 +18 $_N- 24422838680427462 +2 bX?}ix [ Ny]2 G 16242612901291874718 +20 VE] Y 15120036904703536841 +22 Ti~3)N)< A!( 3 18361093572663329113 +23 Sx>b:^UG XpedE)Q: 7433019734386307503 +29 2j&S)ba?XG QuQj 17163829389637435056 +3 UlI+1 14144472852965836438 +=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== +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; +=============== OUTER QUERY (NO PARALLEL) =============== +>T%O ,z< 10 +NQTpY# W\\Xx4 10 +PJFiUe#J2O _s\' 10 +U c 10 +UlI+1 10 +bX?}ix [ Ny]2 G 10 +tT%O ,z< 10 +NQTpY# W\\Xx4 10 +PJFiUe#J2O _s\' 10 +U c 10 +UlI+1 10 +bX?}ix [ Ny]2 G 10 +t toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` +0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_11888098645495698704_17868075224240210014` 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; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql new file mode 100644 index 00000000000..29c20980c14 --- /dev/null +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -0,0 +1,182 @@ +-- Tags: zookeeper + +CREATE TABLE join_inner_table +( + id UUID, + key String, + number Int64, + value1 String, + value2 String, + time Int64 +) +ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/join_inner_table', 'r1') +ORDER BY (id, number, key); + +INSERT INTO join_inner_table +SELECT + '833c9e22-c245-4eb5-8745-117a9a1f26b1'::UUID as id, + rowNumberInAllBlocks()::String as key, + * FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) +LIMIT 100; + +SET allow_experimental_analyzer = 0; +SET max_parallel_replicas = 3; +SET prefer_localhost_replica = 1; +SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +SET use_hedged_requests = 0; +SET joined_subquery_requires_alias = 0; + +SELECT '=============== INNER QUERY (NO PARALLEL) ==============='; + +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY key, value1, value2 +ORDER BY key, value1, value2 +LIMIT 10; + +SELECT '=============== INNER QUERY (PARALLEL) ==============='; + +-- Parallel inner query alone +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table +PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY key, value1, value2 +ORDER BY key, value1, value2 +LIMIT 10 +SETTINGS allow_experimental_parallel_reading_from_replicas = 1; + +SELECT '=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE ==============='; + +SYSTEM FLUSH LOGS; +-- There should be 4 queries. The main query as received by the initiator and the 3 equal queries sent to each replica +SELECT is_initial_query, count() as c, query, +FROM system.query_log +WHERE + event_date >= yesterday() + AND type = 'QueryFinish' + AND initial_query_id = + ( + SELECT query_id + FROM system.query_log + WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND type = 'QueryFinish' + AND query LIKE '-- Parallel inner query alone%' + ) +GROUP BY is_initial_query, query +ORDER BY is_initial_query, c, query; + +---- Query with JOIN + +CREATE TABLE join_outer_table +( + id UUID, + key String, + otherValue1 String, + otherValue2 String, + time Int64 +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/join_outer_table', 'r1') +ORDER BY (id, time, key); + +INSERT INTO join_outer_table +SELECT + '833c9e22-c245-4eb5-8745-117a9a1f26b1'::UUID as id, + (rowNumberInAllBlocks() % 10)::String as key, + * FROM generateRandom('otherValue1 String, otherValue2 String, time Int64', 1, 10, 2) +LIMIT 100; + + +SELECT '=============== OUTER QUERY (NO PARALLEL) ==============='; + +SELECT + value1, + value2, + avg(count) AS avg +FROM +( + SELECT + key, + value1, + value2, + count() AS count + FROM join_outer_table + INNER JOIN + ( + SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts + FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) + GROUP BY key, value1, value2 + ) USING (key) + GROUP BY key, value1, value2 +) +GROUP BY value1, value2 +ORDER BY value1, value2; + +SELECT '=============== OUTER QUERY (PARALLEL) ==============='; + +-- Parallel full query +SELECT + value1, + value2, + avg(count) AS avg +FROM + ( + SELECT + key, + value1, + value2, + count() AS count + FROM join_outer_table + INNER JOIN + ( + SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts + FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) + GROUP BY key, value1, value2 + ) USING (key) + GROUP BY key, value1, value2 + ) +GROUP BY value1, value2 +ORDER BY value1, value2 +SETTINGS allow_experimental_parallel_reading_from_replicas = 1; + +SYSTEM FLUSH LOGS; + +-- There should be 7 queries. The main query as received by the initiator, the 3 equal queries to execute the subquery +-- in the inner join and the 3 queries executing the whole query (but replacing the subquery with a temp table) +SELECT is_initial_query, count() as c, query, +FROM system.query_log +WHERE + event_date >= yesterday() + AND type = 'QueryFinish' + AND initial_query_id = + ( + SELECT query_id + FROM system.query_log + WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND type = 'QueryFinish' + AND query LIKE '-- Parallel full query%' + ) +GROUP BY is_initial_query, query +ORDER BY is_initial_query, c, query; diff --git a/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.reference b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql new file mode 100644 index 00000000000..2ea2cecc7b5 --- /dev/null +++ b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql @@ -0,0 +1,43 @@ +CREATE TABLE join_inner_table__fuzz_1 +( + `id` UUID, + `key` Nullable(Date), + `number` Int64, + `value1` LowCardinality(String), + `value2` LowCardinality(String), + `time` Int128 +) +ENGINE = MergeTree +ORDER BY (id, number, key) +SETTINGS allow_nullable_key = 1; + +INSERT INTO join_inner_table__fuzz_1 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 100; + +SET max_parallel_replicas = 3, prefer_localhost_replica = 1, use_hedged_requests = 0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 1; + +-- SELECT query will write a Warning to the logs +SET send_logs_level='error'; + +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table__fuzz_1 +PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY + key, + value1, + value2 + WITH ROLLUP +ORDER BY + key ASC, + value1 ASC, + value2 ASC NULLS LAST +LIMIT 10 +FORMAT Null; diff --git a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql index ea8eb04bd07..aaf68dfd300 100644 --- a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql +++ b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql @@ -2,7 +2,7 @@ CREATE TABLE IF NOT EXISTS parallel_replicas_plain (x String) ENGINE=MergeTree() INSERT INTO parallel_replicas_plain SELECT toString(number) FROM numbers(10); SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; - +SET send_logs_level='error'; SET parallel_replicas_for_non_replicated_merge_tree = 0; SELECT x FROM parallel_replicas_plain LIMIT 1 FORMAT Null; From 08051917c773da3ca10e51f70fb798bf2836eabb Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 25 May 2023 21:35:51 -0300 Subject: [PATCH 0703/2223] Update named-collections.md --- docs/en/operations/named-collections.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 7db8f2b642b..a521a369721 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -167,9 +167,9 @@ user = 'myuser', password = 'mypass', host = '127.0.0.1', port = 3306, -database = 'test' -connection_pool_size = 8 -on_duplicate_clause = 1 +database = 'test', +connection_pool_size = 8, +on_duplicate_clause = 1, replace_query = 1 ``` From 2a2c35e4c11fe1ef245577cb8c1f2cc1db5f9284 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Thu, 25 May 2023 18:14:16 +0300 Subject: [PATCH 0704/2223] Fix changed IP for https session --- .../Net/include/Poco/Net/HTTPClientSession.h | 3 + src/IO/HTTPCommon.cpp | 29 ++++-- .../configs/listen_host.xml | 3 + .../test_https_replication/test_change_ip.py | 96 +++++++++++++++++++ 4 files changed, 123 insertions(+), 8 deletions(-) create mode 100644 tests/integration/test_https_replication/configs/listen_host.xml create mode 100644 tests/integration/test_https_replication/test_change_ip.py diff --git a/base/poco/Net/include/Poco/Net/HTTPClientSession.h b/base/poco/Net/include/Poco/Net/HTTPClientSession.h index 6f67918d2c8..d495d662f75 100644 --- a/base/poco/Net/include/Poco/Net/HTTPClientSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPClientSession.h @@ -127,6 +127,9 @@ namespace Net void setResolvedHost(std::string resolved_host) { _resolved_host.swap(resolved_host); } + std::string getResolvedHost() const { return _resolved_host; } + /// Returns the resolved IP address of the target HTTP server. + Poco::UInt16 getPort() const; /// Returns the port number of the target HTTP server. diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 4bea646a42b..3ec9b3d0a83 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -68,7 +68,8 @@ namespace if (https) { #if USE_SSL - String resolved_host = resolve_host ? DNSResolver::instance().resolveHost(host).toString() : host; + /// Cannot resolve host in advance, otherwise SNI won't work in Poco. + /// For more information about SNI, see the https://en.wikipedia.org/wiki/Server_Name_Indication auto https_session = std::make_shared(host, port); if (resolve_host) https_session->setResolvedHost(DNSResolver::instance().resolveHost(host).toString()); @@ -184,6 +185,24 @@ namespace std::mutex mutex; std::unordered_map endpoints_pool; + void updateHostIfIpChanged(Entry & session, const String & new_ip) + { + const auto old_ip = session->getResolvedHost().empty() ? session->getHost() : session->getResolvedHost(); + + if (new_ip != old_ip) + { + session->reset(); + if (session->getResolvedHost().empty()) + { + session->setHost(new_ip); + } + else + { + session->setResolvedHost(new_ip); + } + } + } + protected: HTTPSessionPool() = default; @@ -238,13 +257,7 @@ namespace if (resolve_host) { - /// Host can change IP - const auto ip = DNSResolver::instance().resolveHost(host).toString(); - if (ip != session->getHost()) - { - session->reset(); - session->setHost(ip); - } + updateHostIfIpChanged(session, DNSResolver::instance().resolveHost(host).toString()); } } /// Reset the message, once it has been printed, diff --git a/tests/integration/test_https_replication/configs/listen_host.xml b/tests/integration/test_https_replication/configs/listen_host.xml new file mode 100644 index 00000000000..f94e5c88568 --- /dev/null +++ b/tests/integration/test_https_replication/configs/listen_host.xml @@ -0,0 +1,3 @@ + + :: + diff --git a/tests/integration/test_https_replication/test_change_ip.py b/tests/integration/test_https_replication/test_change_ip.py new file mode 100644 index 00000000000..14fe5351c8d --- /dev/null +++ b/tests/integration/test_https_replication/test_change_ip.py @@ -0,0 +1,96 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +""" +Both ssl_conf.xml and no_ssl_conf.xml have the same port +""" + + +def _fill_nodes(nodes, shard): + for node in nodes: + node.query( + """ + CREATE DATABASE test; + + CREATE TABLE test_table(date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; + """.format( + shard=shard, replica=node.name + ) + ) + + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=[ + "configs/remote_servers.xml", + "configs/listen_host.xml", + "configs/ssl_conf.xml", + "configs/server.crt", + "configs/server.key", + "configs/dhparam.pem", + ], + with_zookeeper=True, + ipv6_address="2001:3984:3989::1:1111", +) +node2 = cluster.add_instance( + "node2", + main_configs=[ + "configs/remote_servers.xml", + "configs/listen_host.xml", + "configs/ssl_conf.xml", + "configs/server.crt", + "configs/server.key", + "configs/dhparam.pem", + ], + with_zookeeper=True, + ipv6_address="2001:3984:3989::1:1112", +) + + +@pytest.fixture(scope="module") +def both_https_cluster(): + try: + cluster.start() + + _fill_nodes([node1, node2], 1) + + yield cluster + + finally: + cluster.shutdown() + + +def test_replication_when_node_ip_changed(both_https_cluster): + """ + Test for a bug when replication over HTTPS stops working when the IP of the source replica was changed. + + node1 is a source node + node2 fethes data from node1 + """ + node1.query("truncate table test_table") + node2.query("truncate table test_table") + + # First we check, that normal replication works + node1.query( + "INSERT INTO test_table VALUES ('2022-10-01', 1), ('2022-10-02', 2), ('2022-10-03', 3)" + ) + assert node1.query("SELECT count(*) from test_table") == "3\n" + assert_eq_with_retry(node2, "SELECT count(*) from test_table", "3") + + # We change source node ip + cluster.restart_instance_with_ip_change(node1, "2001:3984:3989::1:7777") + + # Put some data to source node1 + node1.query( + "INSERT INTO test_table VALUES ('2018-10-01', 4), ('2018-10-02', 4), ('2018-10-03', 6)" + ) + # Check that data is placed on node1 + assert node1.query("SELECT count(*) from test_table") == "6\n" + + # drop DNS cache + node2.query("SYSTEM DROP DNS CACHE") + # Data is fetched + assert_eq_with_retry(node2, "SELECT count(*) from test_table", "6") From c6c4ded5e176e79bbc8aff81eebc87d6f754f986 Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Fri, 26 May 2023 10:12:34 +0200 Subject: [PATCH 0705/2223] Fixes for WITH FILL grouped by sorting prefix --- docs/en/sql-reference/statements/select/order-by.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index f1efd6c4718..712395a0357 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -544,10 +544,10 @@ Result: └─────┴──────────┴───────┘ ``` -##Filling grouped by sorting prefix +## Filling grouped by sorting prefix It can be useful to fill rows which have the same values in particular columns independently, - a good example is filling missing values in time series. -Assume there is the following time series table +Assume there is the following time series table: ``` sql CREATE TABLE timeseries ( @@ -567,7 +567,7 @@ SELECT * FROM timeseries; └───────────┴─────────────────────────┴───────┘ ``` And we'd like to fill missing values for each sensor independently with 1 second interval. -The way to achieve it is to use `sensor_id` column as sorting prefix for filling column `timestamp` +The way to achieve it is to use `sensor_id` column as sorting prefix for filling column `timestamp`: ``` SELECT * FROM timeseries @@ -589,7 +589,7 @@ INTERPOLATE ( value AS 9999 ) │ 432 │ 2021-12-01 00:00:05.000 │ 5 │ └───────────┴─────────────────────────┴───────┘ ``` -Here, the `value` column was interpolated with `9999` just to make filled rows more noticeable +Here, the `value` column was interpolated with `9999` just to make filled rows more noticeable. This behavior is controlled by setting `use_with_fill_by_sorting_prefix` (enabled by default) ## Related content From 0d0e53ecc0874530a791fc454bb58684b79244d5 Mon Sep 17 00:00:00 2001 From: Mohammad Arab Anvari Date: Fri, 26 May 2023 13:07:37 +0330 Subject: [PATCH 0706/2223] Update distributed.md Fix broken link in `**See Also**` section. --- docs/en/engines/table-engines/special/distributed.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index 43b8e387ba2..d1a0b13b363 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -258,4 +258,4 @@ Since [remote](../../../sql-reference/table-functions/remote.md) and [cluster](. - [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) description - [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting -- [shardNum()](../../../sql-reference/functions/other-functions.md#shard-num) and [shardCount()](../../../sql-reference/functions/other-functions.md#shard-count) functions +- [shardNum()](../../../sql-reference/functions/other-functions.md#shardnum) and [shardCount()](../../../sql-reference/functions/other-functions.md#shardcount) functions From 03652efe58469351f87e87c0ca47a6789776710f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 May 2023 12:44:28 +0200 Subject: [PATCH 0707/2223] Add missing include --- src/IO/ReadBufferFromPocoSocket.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index c051478afc5..d6790439683 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -10,6 +10,8 @@ #include #include +#include "config.h" + #if USE_SSL #include #endif From 67c8c5c561668f0fca8d5ab2545d2ee82178fbdb Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 May 2023 12:44:43 +0200 Subject: [PATCH 0708/2223] Add missing include --- src/IO/WriteBufferFromPocoSocket.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index e566c228534..6e7c67cc054 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -11,6 +11,8 @@ #include #include +#include "config.h" + #if USE_SSL #include #endif From 3d98e591baeddb831329db012de70b155d9d0f82 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 May 2023 09:30:57 +0000 Subject: [PATCH 0709/2223] Handle backwards compatibility --- src/Coordination/CoordinationSettings.h | 2 +- src/Coordination/KeeperServer.cpp | 48 ++++++++++++++++++++----- src/Coordination/KeeperStateMachine.cpp | 30 +++++++++++----- src/Coordination/KeeperStateMachine.h | 17 ++++++++- src/Coordination/KeeperStorage.h | 2 +- 5 files changed, 78 insertions(+), 21 deletions(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 24567177ff1..81be3c3eaa4 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -48,7 +48,7 @@ struct Settings; M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0) \ M(UInt64, max_log_file_size, 50 * 1024 * 1024, "Max size of the Raft log file. If possible, each created log file will preallocate this amount of bytes on disk. Set to 0 to disable the limit", 0) \ M(UInt64, log_file_overallocate_size, 50 * 1024 * 1024, "If max_log_file_size is not set to 0, this value will be added to it for preallocating bytes on disk. If a log record is larger than this value, it could lead to uncaught out-of-space issues so a larger value is preferred", 0) \ - M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests", 0) + M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests, set to 0 to disable", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 6af8f967c0c..2d809a67a9b 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -478,9 +478,11 @@ nuraft::ptr getZooKeeperLogEntry(const KeeperStorage::RequestFor DB::writeIntBinary(request_for_session.session_id, write_buf); request_for_session.request->write(write_buf); DB::writeIntBinary(request_for_session.time, write_buf); - DB::writeIntBinary(static_cast(0), write_buf); - DB::writeIntBinary(KeeperStorage::DigestVersion::NO_DIGEST, write_buf); - DB::writeIntBinary(static_cast(0), write_buf); + /// we fill with dummy values to eliminate unecessary copy later on when we will write correct values + DB::writeIntBinary(static_cast(0), write_buf); /// zxid + DB::writeIntBinary(KeeperStorage::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag + DB::writeIntBinary(static_cast(0), write_buf); /// digest value + /// if new fields are added, update KeeperStateMachine::ZooKeeperLogSerializationVersion along with parseRequest function and PreAppendLog callback handler return write_buf.getBuffer(); } @@ -619,19 +621,47 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ assert(entry->get_val_type() == nuraft::app_log); auto next_zxid = state_machine->getNextZxid(); - auto & entry_buf = entry->get_buf(); - auto request_for_session = state_machine->parseRequest(entry_buf); + auto entry_buf = entry->get_buf_ptr(); + + KeeperStateMachine::ZooKeeperLogSerializationVersion serialization_version; + auto request_for_session = state_machine->parseRequest(*entry_buf, /*final=*/false, &serialization_version); request_for_session->zxid = next_zxid; if (!state_machine->preprocess(*request_for_session)) return nuraft::cb_func::ReturnCode::ReturnNull; request_for_session->digest = state_machine->getNodesDigest(); - static constexpr size_t write_buffer_size - = sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value); - auto * buffer_start = reinterpret_cast(entry_buf.data_begin() + entry_buf.size() - write_buffer_size); + using enum KeeperStateMachine::ZooKeeperLogSerializationVersion; + + /// older versions of Keeper can send logs that are missing some fields + size_t bytes_missing = 0; + if (serialization_version < WITH_TIME) + bytes_missing += sizeof(request_for_session->time); + + if (serialization_version < WITH_ZXID_DIGEST) + bytes_missing += sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value); + + if (bytes_missing != 0) + { + auto new_buffer = nuraft::buffer::alloc(entry_buf->size() + bytes_missing); + memcpy(new_buffer->data_begin(), entry_buf->data_begin(), entry_buf->size()); + entry_buf = std::move(new_buffer); + entry = nuraft::cs_new(entry->get_term(), entry_buf, entry->get_val_type()); + } + + size_t write_buffer_header_size + = sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value); + + if (serialization_version < WITH_TIME) + write_buffer_header_size += sizeof(request_for_session->time); + + auto * buffer_start = reinterpret_cast(entry_buf->data_begin() + entry_buf->size() - write_buffer_header_size); + + WriteBuffer write_buf(buffer_start, write_buffer_header_size); + + if (serialization_version < WITH_TIME) + writeIntBinary(request_for_session->time, write_buf); - WriteBuffer write_buf(buffer_start, write_buffer_size); writeIntBinary(request_for_session->zxid, write_buf); writeIntBinary(request_for_session->digest->version, write_buf); if (request_for_session->digest->version != KeeperStorage::NO_DIGEST) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5e182228170..6635c74149a 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -150,7 +150,7 @@ void assertDigest( nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data) { - auto request_for_session = parseRequest(data); + auto request_for_session = parseRequest(data, /*final=*/false); if (!request_for_session->zxid) request_for_session->zxid = log_idx; @@ -158,7 +158,7 @@ nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nur return nullptr; } -std::shared_ptr KeeperStateMachine::parseRequest(nuraft::buffer & data, bool final) +std::shared_ptr KeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version) { ReadBufferFromNuraftBuffer buffer(data); auto request_for_session = std::make_shared(); @@ -177,10 +177,10 @@ std::shared_ptr KeeperStateMachine::parseReque Coordination::CLOSE_XID, }; - const bool should_cache = request_for_session->session_id != -1 && data.size() > min_request_size_to_cache - && std::all_of(non_cacheable_xids.begin(), - non_cacheable_xids.end(), - [&](const auto non_cacheable_xid) { return xid != non_cacheable_xid; }); + const bool should_cache + = min_request_size_to_cache != 0 && request_for_session->session_id != -1 && data.size() >= min_request_size_to_cache + && std::all_of( + non_cacheable_xids.begin(), non_cacheable_xids.end(), [&](const auto non_cacheable_xid) { return xid != non_cacheable_xid; }); if (should_cache) { @@ -212,23 +212,35 @@ std::shared_ptr KeeperStateMachine::parseReque request_for_session->request->xid = xid; request_for_session->request->readImpl(buffer); + using enum ZooKeeperLogSerializationVersion; + ZooKeeperLogSerializationVersion version = INITIAL; + if (!buffer.eof()) + { + version = WITH_TIME; readIntBinary(request_for_session->time, buffer); - else /// backward compatibility + } + else request_for_session->time = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); if (!buffer.eof()) + { + version = WITH_ZXID_DIGEST; + readIntBinary(request_for_session->zxid, buffer); - if (!buffer.eof()) - { + chassert(!buffer.eof()); + request_for_session->digest.emplace(); readIntBinary(request_for_session->digest->version, buffer); if (request_for_session->digest->version != KeeperStorage::DigestVersion::NO_DIGEST || !buffer.eof()) readIntBinary(request_for_session->digest->value, buffer); } + if (serialization_version) + *serialization_version = version; + if (should_cache && !final) { std::lock_guard lock(request_cache_mutex); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 398c767d010..1f7ac9b77d4 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -36,7 +36,22 @@ public: /// Read state from the latest snapshot void init(); - std::shared_ptr parseRequest(nuraft::buffer & data, bool final = false); + enum ZooKeeperLogSerializationVersion + { + INITIAL = 0, + WITH_TIME = 1, + WITH_ZXID_DIGEST = 2, + }; + + /// lifetime of a parsed request is: + /// [preprocess/PreAppendLog -> commit] + /// [preprocess/PreAppendLog -> rollback] + /// on events like commit and rollback we can remove the parsed request to keep the memory usage at minimum + /// request cache is also cleaned on session close in case somethign strange happened + /// + /// final - whether it's the final time we will fetch the request so we can safely remove it from cache + /// serialization_version - information about which fields were parsed from the buffer so we can modify the buffer accordingly + std::shared_ptr parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr); bool preprocess(const KeeperStorage::RequestForSession & request_for_session); diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index cfacdfc84de..7eb10be3847 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -110,7 +110,7 @@ public: struct RequestForSession { int64_t session_id; - int64_t time; + int64_t time{0}; Coordination::ZooKeeperRequestPtr request; int64_t zxid{0}; std::optional digest; From a9569478e6a7a8d821d05d24f2377899da322758 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 May 2023 13:09:09 +0200 Subject: [PATCH 0710/2223] fix typo --- src/Coordination/KeeperServer.cpp | 2 +- src/Coordination/KeeperStateMachine.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 2d809a67a9b..b07c2edc53a 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -478,7 +478,7 @@ nuraft::ptr getZooKeeperLogEntry(const KeeperStorage::RequestFor DB::writeIntBinary(request_for_session.session_id, write_buf); request_for_session.request->write(write_buf); DB::writeIntBinary(request_for_session.time, write_buf); - /// we fill with dummy values to eliminate unecessary copy later on when we will write correct values + /// we fill with dummy values to eliminate unnecessary copy later on when we will write correct values DB::writeIntBinary(static_cast(0), write_buf); /// zxid DB::writeIntBinary(KeeperStorage::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag DB::writeIntBinary(static_cast(0), write_buf); /// digest value diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 1f7ac9b77d4..fbd97fd8631 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -47,7 +47,7 @@ public: /// [preprocess/PreAppendLog -> commit] /// [preprocess/PreAppendLog -> rollback] /// on events like commit and rollback we can remove the parsed request to keep the memory usage at minimum - /// request cache is also cleaned on session close in case somethign strange happened + /// request cache is also cleaned on session close in case something strange happened /// /// final - whether it's the final time we will fetch the request so we can safely remove it from cache /// serialization_version - information about which fields were parsed from the buffer so we can modify the buffer accordingly From 050b93363b36e9dbb47e140f00ffc10c2c5b18ff Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 26 May 2023 12:25:11 +0000 Subject: [PATCH 0711/2223] Add unit tests --- src/Common/tests/gtest_DateLUTImpl.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index 6ae934b2296..04f63403ec2 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -148,6 +148,8 @@ TEST(DateLUTTest, TimeValuesInMiddleOfRange) EXPECT_EQ(lut.addYears(time, 10), 1884270011 /*time_t*/); EXPECT_EQ(lut.timeToString(time), "2019-09-16 19:20:11" /*std::string*/); EXPECT_EQ(lut.dateToString(time), "2019-09-16" /*std::string*/); + EXPECT_EQ(lut.toLastDayOfWeek(time), 1569099600 /*time_t*/); + EXPECT_EQ(lut.toLastDayNumOfWeek(time), DayNum(18161) /*DayNum*/); EXPECT_EQ(lut.toLastDayOfMonth(time), 1569790800 /*time_t*/); EXPECT_EQ(lut.toLastDayNumOfMonth(time), DayNum(18169) /*DayNum*/); } @@ -211,6 +213,8 @@ TEST(DateLUTTest, TimeValuesAtLeftBoderOfRange) EXPECT_EQ(lut.addYears(time, 10), 315532800 /*time_t*/); EXPECT_EQ(lut.timeToString(time), "1970-01-01 00:00:00" /*std::string*/); EXPECT_EQ(lut.dateToString(time), "1970-01-01" /*std::string*/); + EXPECT_EQ(lut.toLastDayOfWeek(time), 259200 /*time_t*/); + EXPECT_EQ(lut.toLastDayNumOfWeek(time), DayNum(3) /*DayNum*/); EXPECT_EQ(lut.toLastDayOfMonth(time), 2592000 /*time_t*/); EXPECT_EQ(lut.toLastDayNumOfMonth(time), DayNum(30) /*DayNum*/); } @@ -276,6 +280,8 @@ TEST(DateLUTTest, TimeValuesAtRightBoderOfRangeOfOldLUT) EXPECT_EQ(lut.timeToString(time), "2106-01-31 01:17:53" /*std::string*/); EXPECT_EQ(lut.dateToString(time), "2106-01-31" /*std::string*/); + EXPECT_EQ(lut.toLastDayOfWeek(time), 4294339200 /*time_t*/); + EXPECT_EQ(lut.toLastDayNumOfWeek(time), DayNum(49703) /*DayNum*/); EXPECT_EQ(lut.toLastDayOfMonth(time), 4294339200 /*time_t*/); // 2106-01-01 EXPECT_EQ(lut.toLastDayNumOfMonth(time), DayNum(49703)); } From 0d1f2e297b4de987877f79ded7f1aa7c6798565d Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 26 May 2023 15:55:30 +0200 Subject: [PATCH 0712/2223] Unify priorities: lower value means higher priority (#50205) --- src/Common/Priority.h | 11 ++++++++++ src/Common/ThreadPool.cpp | 8 +++---- src/Common/ThreadPool.h | 21 ++++++++++--------- .../CachedCompressedReadBuffer.cpp | 2 +- src/Compression/CachedCompressedReadBuffer.h | 2 +- .../CompressedReadBufferFromFile.cpp | 2 +- .../CompressedReadBufferFromFile.h | 2 +- .../IO/AsynchronousBoundedReadBuffer.cpp | 6 +++--- src/Disks/IO/AsynchronousBoundedReadBuffer.h | 6 +++--- src/IO/AsynchronousReadBufferFromFile.cpp | 4 ++-- src/IO/AsynchronousReadBufferFromFile.h | 6 +++--- ...ynchronousReadBufferFromFileDescriptor.cpp | 8 +++---- ...AsynchronousReadBufferFromFileDescriptor.h | 9 ++++---- src/IO/AsynchronousReader.h | 3 ++- src/IO/CompressedReadBufferWrapper.h | 2 +- src/IO/ParallelReadBuffer.cpp | 2 +- src/IO/PeekableReadBuffer.h | 2 +- src/IO/ReadBuffer.h | 9 ++++---- src/IO/ReadBufferFromFileDescriptor.cpp | 2 +- src/IO/ReadBufferFromFileDescriptor.h | 2 +- src/IO/ReadSettings.h | 5 +++-- src/IO/S3/copyS3File.cpp | 2 +- src/IO/SeekAvoidingReadBuffer.h | 2 +- src/IO/WriteBufferFromS3TaskTracker.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- .../FilesystemReadPrefetchesLog.cpp | 4 ++-- .../FilesystemReadPrefetchesLog.h | 3 ++- src/Interpreters/threadPoolCallbackRunner.h | 9 ++++---- .../HDFS/AsynchronousReadBufferFromHDFS.cpp | 6 +++--- .../HDFS/AsynchronousReadBufferFromHDFS.h | 6 +++--- src/Storages/MergeTree/IMergeTreeReader.h | 2 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 2 +- .../MergeTree/MergeTreeBlockReadUtils.h | 8 ++----- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 14 ++++++------- .../MergeTree/MergeTreePrefetchedReadPool.h | 3 +-- .../MergeTree/MergeTreeReaderCompact.cpp | 2 +- .../MergeTree/MergeTreeReaderCompact.h | 2 +- .../MergeTree/MergeTreeReaderWide.cpp | 8 +++---- src/Storages/MergeTree/MergeTreeReaderWide.h | 6 +++--- src/Storages/MergeTree/MergeTreeSource.cpp | 2 +- src/Storages/StorageS3.cpp | 4 ++-- 42 files changed, 108 insertions(+), 97 deletions(-) create mode 100644 src/Common/Priority.h diff --git a/src/Common/Priority.h b/src/Common/Priority.h new file mode 100644 index 00000000000..8952fe4dd5a --- /dev/null +++ b/src/Common/Priority.h @@ -0,0 +1,11 @@ +#pragma once + +#include + +/// Common type for priority values. +/// Separate type (rather than `Int64` is used just to avoid implicit conversion errors and to default-initialize +struct Priority +{ + Int64 value = 0; /// Note that lower value means higher priority. + constexpr operator Int64() const { return value; } /// NOLINT +}; diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 63eef72dffb..f91849ead66 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -123,7 +123,7 @@ void ThreadPoolImpl::setQueueSize(size_t value) template template -ReturnType ThreadPoolImpl::scheduleImpl(Job job, ssize_t priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context) +ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context) { auto on_error = [&](const std::string & reason) { @@ -231,19 +231,19 @@ void ThreadPoolImpl::startNewThreadsNoLock() } template -void ThreadPoolImpl::scheduleOrThrowOnError(Job job, ssize_t priority) +void ThreadPoolImpl::scheduleOrThrowOnError(Job job, Priority priority) { scheduleImpl(std::move(job), priority, std::nullopt); } template -bool ThreadPoolImpl::trySchedule(Job job, ssize_t priority, uint64_t wait_microseconds) noexcept +bool ThreadPoolImpl::trySchedule(Job job, Priority priority, uint64_t wait_microseconds) noexcept { return scheduleImpl(std::move(job), priority, wait_microseconds); } template -void ThreadPoolImpl::scheduleOrThrow(Job job, ssize_t priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context) +void ThreadPoolImpl::scheduleOrThrow(Job job, Priority priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context) { scheduleImpl(std::move(job), priority, wait_microseconds, propagate_opentelemetry_tracing_context); } diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index b92cb256b13..70053ff7558 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -18,6 +18,7 @@ #include #include #include +#include #include /** Very simple thread pool similar to boost::threadpool. @@ -59,17 +60,17 @@ public: /// If any thread was throw an exception, first exception will be rethrown from this method, /// and exception will be cleared. /// Also throws an exception if cannot create thread. - /// Priority: greater is higher. + /// Priority: lower is higher. /// NOTE: Probably you should call wait() if exception was thrown. If some previously scheduled jobs are using some objects, /// located on stack of current thread, the stack must not be unwinded until all jobs finished. However, /// if ThreadPool is a local object, it will wait for all scheduled jobs in own destructor. - void scheduleOrThrowOnError(Job job, ssize_t priority = 0); + void scheduleOrThrowOnError(Job job, Priority priority = {}); /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or return false. - bool trySchedule(Job job, ssize_t priority = 0, uint64_t wait_microseconds = 0) noexcept; + bool trySchedule(Job job, Priority priority = {}, uint64_t wait_microseconds = 0) noexcept; /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or throw an exception. - void scheduleOrThrow(Job job, ssize_t priority = 0, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); + void scheduleOrThrow(Job job, Priority priority = {}, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); /// Wait for all currently active jobs to be done. /// You may call schedule and wait many times in arbitrary order. @@ -123,15 +124,15 @@ private: struct JobWithPriority { Job job; - ssize_t priority; + Priority priority; DB::OpenTelemetry::TracingContextOnThread thread_trace_context; - JobWithPriority(Job job_, ssize_t priority_, const DB::OpenTelemetry::TracingContextOnThread& thread_trace_context_) + JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_) : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) {} - bool operator< (const JobWithPriority & rhs) const + bool operator<(const JobWithPriority & rhs) const { - return priority < rhs.priority; + return priority > rhs.priority; // Reversed for `priority_queue` max-heap to yield minimum value (i.e. highest priority) first } }; @@ -141,7 +142,7 @@ private: std::stack on_destroy_callbacks; template - ReturnType scheduleImpl(Job job, ssize_t priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context = true); + ReturnType scheduleImpl(Job job, Priority priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context = true); void worker(typename std::list::iterator thread_it); @@ -227,7 +228,7 @@ public: DB::ThreadStatus thread_status; std::apply(function, arguments); }, - 0, // default priority + {}, // default priority 0, // default wait_microseconds propagate_opentelemetry_context ); diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index fdb2132d134..0febfca75cc 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -28,7 +28,7 @@ void CachedCompressedReadBuffer::initInput() } -void CachedCompressedReadBuffer::prefetch(int64_t priority) +void CachedCompressedReadBuffer::prefetch(Priority priority) { initInput(); file_in->prefetch(priority); diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index ee2728752e9..cce9a8e671c 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -36,7 +36,7 @@ private: bool nextImpl() override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; /// Passed into file_in. ReadBufferFromFileBase::ProfileCallback profile_callback; diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index ca697e2f51e..9dc40b8217c 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -51,7 +51,7 @@ CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr buf, bool allow_different_codecs_ = false); diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index 611fd00a023..aed4c2f82f2 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -83,19 +83,19 @@ bool AsynchronousBoundedReadBuffer::hasPendingDataToRead() } std::future -AsynchronousBoundedReadBuffer::asyncReadInto(char * data, size_t size, int64_t priority) +AsynchronousBoundedReadBuffer::asyncReadInto(char * data, size_t size, Priority priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(*impl, async_read_counters); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = read_settings.priority + priority; + request.priority = Priority{read_settings.priority.value + priority.value}; request.ignore = bytes_to_ignore; return reader.submit(request); } -void AsynchronousBoundedReadBuffer::prefetch(int64_t priority) +void AsynchronousBoundedReadBuffer::prefetch(Priority priority) { if (prefetch_future.valid()) return; diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.h b/src/Disks/IO/AsynchronousBoundedReadBuffer.h index 45256cdfac2..cafc5b6d691 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.h +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.h @@ -39,7 +39,7 @@ public: off_t seek(off_t offset_, int whence) override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; void setReadUntilPosition(size_t position) override; /// [..., position). @@ -72,7 +72,7 @@ private: struct LastPrefetchInfo { UInt64 submit_time = 0; - size_t priority = 0; + Priority priority; }; LastPrefetchInfo last_prefetch_info; @@ -87,7 +87,7 @@ private: int64_t size, const std::unique_ptr & execution_watch); - std::future asyncReadInto(char * data, size_t size, int64_t priority); + std::future asyncReadInto(char * data, size_t size, Priority priority); void resetPrefetch(FilesystemPrefetchState state); diff --git a/src/IO/AsynchronousReadBufferFromFile.cpp b/src/IO/AsynchronousReadBufferFromFile.cpp index 7499b684a89..0e6c8090cb5 100644 --- a/src/IO/AsynchronousReadBufferFromFile.cpp +++ b/src/IO/AsynchronousReadBufferFromFile.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, const std::string & file_name_, size_t buf_size, int flags, @@ -60,7 +60,7 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int & fd_, const std::string & original_file_name, size_t buf_size, diff --git a/src/IO/AsynchronousReadBufferFromFile.h b/src/IO/AsynchronousReadBufferFromFile.h index 0ac0a820e48..d3b7ffbc7dc 100644 --- a/src/IO/AsynchronousReadBufferFromFile.h +++ b/src/IO/AsynchronousReadBufferFromFile.h @@ -17,7 +17,7 @@ protected: public: explicit AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, @@ -28,7 +28,7 @@ public: /// Use pre-opened file descriptor. explicit AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int & fd, /// Will be set to -1 if constructor didn't throw and ownership of file descriptor is passed to the object. const std::string & original_file_name = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, @@ -58,7 +58,7 @@ private: public: AsynchronousReadBufferFromFileWithDescriptorsCache( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp index 743892d24f6..d30773f88f4 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp @@ -40,14 +40,14 @@ std::string AsynchronousReadBufferFromFileDescriptor::getFileName() const } -std::future AsynchronousReadBufferFromFileDescriptor::asyncReadInto(char * data, size_t size, int64_t priority) +std::future AsynchronousReadBufferFromFileDescriptor::asyncReadInto(char * data, size_t size, Priority priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(fd); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = base_priority + priority; + request.priority = Priority{base_priority.value + priority.value}; request.ignore = bytes_to_ignore; bytes_to_ignore = 0; @@ -61,7 +61,7 @@ std::future AsynchronousReadBufferFromFileDescripto } -void AsynchronousReadBufferFromFileDescriptor::prefetch(int64_t priority) +void AsynchronousReadBufferFromFileDescriptor::prefetch(Priority priority) { if (prefetch_future.valid()) return; @@ -151,7 +151,7 @@ void AsynchronousReadBufferFromFileDescriptor::finalize() AsynchronousReadBufferFromFileDescriptor::AsynchronousReadBufferFromFileDescriptor( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int fd_, size_t buf_size, char * existing_memory, diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.h b/src/IO/AsynchronousReadBufferFromFileDescriptor.h index dff56baef4e..4a4130ebab1 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.h +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -18,7 +19,7 @@ class AsynchronousReadBufferFromFileDescriptor : public ReadBufferFromFileBase { protected: IAsynchronousReader & reader; - int64_t base_priority; + Priority base_priority; Memory<> prefetch_buffer; std::future prefetch_future; @@ -39,7 +40,7 @@ protected: public: AsynchronousReadBufferFromFileDescriptor( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int fd_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, @@ -49,7 +50,7 @@ public: ~AsynchronousReadBufferFromFileDescriptor() override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; int getFD() const { @@ -70,7 +71,7 @@ public: size_t getFileSize() override; private: - std::future asyncReadInto(char * data, size_t size, int64_t priority); + std::future asyncReadInto(char * data, size_t size, Priority priority); }; } diff --git a/src/IO/AsynchronousReader.h b/src/IO/AsynchronousReader.h index 91a5db9dcdd..6b8b93fcc20 100644 --- a/src/IO/AsynchronousReader.h +++ b/src/IO/AsynchronousReader.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -47,7 +48,7 @@ public: size_t offset = 0; size_t size = 0; char * buf = nullptr; - int64_t priority = 0; + Priority priority; size_t ignore = 0; }; diff --git a/src/IO/CompressedReadBufferWrapper.h b/src/IO/CompressedReadBufferWrapper.h index 8fe87d22dd9..bb58a7bfeb3 100644 --- a/src/IO/CompressedReadBufferWrapper.h +++ b/src/IO/CompressedReadBufferWrapper.h @@ -19,7 +19,7 @@ public: const ReadBuffer & getWrappedReadBuffer() const { return *in; } ReadBuffer & getWrappedReadBuffer() { return *in; } - void prefetch(int64_t priority) override { in->prefetch(priority); } + void prefetch(Priority priority) override { in->prefetch(priority); } protected: std::unique_ptr in; diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 6188fc210be..fff02db1bd6 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -87,7 +87,7 @@ bool ParallelReadBuffer::addReaderToPool() auto worker = read_workers.emplace_back(std::make_shared(std::move(reader), range_start, size)); ++active_working_reader; - schedule([this, my_worker = std::move(worker)]() mutable { readerThreadFunction(std::move(my_worker)); }, 0); + schedule([this, my_worker = std::move(worker)]() mutable { readerThreadFunction(std::move(my_worker)); }, Priority{}); return true; } diff --git a/src/IO/PeekableReadBuffer.h b/src/IO/PeekableReadBuffer.h index 7af17aff090..78cb319327d 100644 --- a/src/IO/PeekableReadBuffer.h +++ b/src/IO/PeekableReadBuffer.h @@ -20,7 +20,7 @@ public: ~PeekableReadBuffer() override; - void prefetch(int64_t priority) override { sub_buf->prefetch(priority); } + void prefetch(Priority priority) override { sub_buf->prefetch(priority); } /// Sets checkpoint at current position ALWAYS_INLINE inline void setCheckpoint() diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 1a8222001a9..a4ae12f5069 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -20,7 +21,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -static constexpr auto DEFAULT_PREFETCH_PRIORITY = 0; +static constexpr auto DEFAULT_PREFETCH_PRIORITY = Priority{0}; /** A simple abstract class for buffered data reading (char sequences) from somewhere. * Unlike std::istream, it provides access to the internal buffer, @@ -208,10 +209,10 @@ public: /** Do something to allow faster subsequent call to 'nextImpl' if possible. * It's used for asynchronous readers with double-buffering. - * `priority` is the Threadpool priority, with which the prefetch task will be schedules. - * Smaller is more priority. + * `priority` is the `ThreadPool` priority, with which the prefetch task will be scheduled. + * Lower value means higher priority. */ - virtual void prefetch(int64_t /* priority */) {} + virtual void prefetch(Priority) {} /** * Set upper bound for read range [..., position). diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 1e7ea855364..bf44d9d10da 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -124,7 +124,7 @@ bool ReadBufferFromFileDescriptor::nextImpl() } -void ReadBufferFromFileDescriptor::prefetch(int64_t) +void ReadBufferFromFileDescriptor::prefetch(Priority) { #if defined(POSIX_FADV_WILLNEED) /// For direct IO, loading data into page cache is pointless. diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index daf8ac56d13..10f140275bb 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -25,7 +25,7 @@ protected: ThrottlerPtr throttler; bool nextImpl() override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; /// Name or some description of file. std::string getFileName() const override; diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index d1551b16324..e43ecd7f275 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -5,6 +5,7 @@ #include #include #include +#include #include namespace DB @@ -84,8 +85,8 @@ struct ReadSettings size_t mmap_threshold = 0; MMappedFileCache * mmap_cache = nullptr; - /// For 'pread_threadpool'/'io_uring' method. Lower is more priority. - size_t priority = 0; + /// For 'pread_threadpool'/'io_uring' method. Lower value is higher priority. + Priority priority; bool load_marks_asynchronously = true; diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 20490ef9a19..7886b84cd00 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -361,7 +361,7 @@ namespace task->exception = std::current_exception(); } task_finish_notify(); - }, 0); + }, Priority{}); } catch (...) { diff --git a/src/IO/SeekAvoidingReadBuffer.h b/src/IO/SeekAvoidingReadBuffer.h index af2a988ab3a..716d7c5046c 100644 --- a/src/IO/SeekAvoidingReadBuffer.h +++ b/src/IO/SeekAvoidingReadBuffer.h @@ -17,7 +17,7 @@ public: off_t seek(off_t off, int whence) override; - void prefetch(int64_t priority) override { impl->prefetch(priority); } + void prefetch(Priority priority) override { impl->prefetch(priority); } private: UInt64 min_bytes_for_seek; /// Minimum positive seek offset which shall be executed using seek operation. diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 7826747c0a4..6c7bb35fc5a 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -113,7 +113,7 @@ void WriteBufferFromS3::TaskTracker::add(Callback && func) { LOG_TEST(log, "add, in queue {}", futures.size()); - auto future = scheduler(std::move(func), 0); + auto future = scheduler(std::move(func), Priority{}); auto exit_scope = scope_guard( [&future]() { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c594feb73c2..91f34c8eb97 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4269,7 +4269,7 @@ ReadSettings Context::getReadSettings() const res.prefetch_buffer_size = settings.prefetch_buffer_size; res.direct_io_threshold = settings.min_bytes_to_use_direct_io; res.mmap_threshold = settings.min_bytes_to_use_mmap_io; - res.priority = settings.read_priority; + res.priority = Priority{settings.read_priority}; res.remote_throttler = getRemoteReadThrottler(); res.local_throttler = getLocalReadThrottler(); diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.cpp b/src/Interpreters/FilesystemReadPrefetchesLog.cpp index 9dd0fce84b0..221cb11f09a 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.cpp +++ b/src/Interpreters/FilesystemReadPrefetchesLog.cpp @@ -19,7 +19,7 @@ NamesAndTypesList FilesystemReadPrefetchesLogElement::getNamesAndTypes() {"offset", std::make_shared()}, {"size", std::make_shared()}, {"prefetch_submit_time", std::make_shared(6)}, - {"priority", std::make_shared()}, + {"priority", std::make_shared()}, {"prefetch_execution_start_time", std::make_shared(6)}, {"prefetch_execution_end_time", std::make_shared(6)}, {"prefetch_execution_time_us", std::make_shared()}, @@ -40,7 +40,7 @@ void FilesystemReadPrefetchesLogElement::appendToBlock(MutableColumns & columns) columns[i++]->insert(offset); columns[i++]->insert(size); columns[i++]->insert(prefetch_submit_time); - columns[i++]->insert(priority); + columns[i++]->insert(priority.value); if (execution_watch) { columns[i++]->insert(execution_watch->getStart()); diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.h b/src/Interpreters/FilesystemReadPrefetchesLog.h index 7052cf2769d..cf36f513f5a 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.h +++ b/src/Interpreters/FilesystemReadPrefetchesLog.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -25,7 +26,7 @@ struct FilesystemReadPrefetchesLogElement Int64 size; /// -1 means unknown Decimal64 prefetch_submit_time{}; std::optional execution_watch; - size_t priority; + Priority priority; FilesystemPrefetchState state; UInt64 thread_id; String reader_id; diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index b8805722346..55c6a848b77 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -11,13 +11,13 @@ namespace DB /// High-order function to run callbacks (functions with 'void()' signature) somewhere asynchronously. template > -using ThreadPoolCallbackRunner = std::function(Callback &&, int64_t priority)>; +using ThreadPoolCallbackRunner = std::function(Callback &&, Priority)>; /// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'. template > ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name) { - return [my_pool = &pool, thread_group = CurrentThread::getGroup(), thread_name](Callback && callback, int64_t priority) mutable -> std::future + return [my_pool = &pool, thread_group = CurrentThread::getGroup(), thread_name](Callback && callback, Priority priority) mutable -> std::future { auto task = std::make_shared>([thread_group, thread_name, my_callback = std::move(callback)]() mutable -> Result { @@ -44,15 +44,14 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & auto future = task->get_future(); - /// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority". - my_pool->scheduleOrThrow([my_task = std::move(task)]{ (*my_task)(); }, -priority); + my_pool->scheduleOrThrow([my_task = std::move(task)]{ (*my_task)(); }, priority); return future; }; } template -std::future scheduleFromThreadPool(T && task, ThreadPool & pool, const std::string & thread_name, int64_t priority = 0) +std::future scheduleFromThreadPool(T && task, ThreadPool & pool, const std::string & thread_name, Priority priority = {}) { auto schedule = threadPoolCallbackRunner(pool, thread_name); return schedule(std::move(task), priority); diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp index a8502b1bd65..b490c5cac63 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -64,19 +64,19 @@ bool AsynchronousReadBufferFromHDFS::hasPendingDataToRead() return true; } -std::future AsynchronousReadBufferFromHDFS::asyncReadInto(char * data, size_t size, int64_t priority) +std::future AsynchronousReadBufferFromHDFS::asyncReadInto(char * data, size_t size, Priority priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(*impl, nullptr); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = base_priority + priority; + request.priority = Priority{base_priority.value + priority.value}; request.ignore = 0; return reader.submit(request); } -void AsynchronousReadBufferFromHDFS::prefetch(int64_t priority) +void AsynchronousReadBufferFromHDFS::prefetch(Priority priority) { interval_watch.restart(); diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h index 07d32194a93..d89aa60ab71 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h @@ -33,7 +33,7 @@ public: off_t seek(off_t offset_, int whence) override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; size_t getFileSize() override; @@ -50,10 +50,10 @@ private: bool hasPendingDataToRead(); - std::future asyncReadInto(char * data, size_t size, int64_t priority); + std::future asyncReadInto(char * data, size_t size, Priority priority); IAsynchronousReader & reader; - int64_t base_priority; + Priority base_priority; std::shared_ptr impl; std::future prefetch_future; Memory<> prefetch_buffer; diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index c7396e882fa..f4759409c52 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -61,7 +61,7 @@ public: MergeTreeDataPartInfoForReaderPtr data_part_info_for_read; - virtual void prefetchBeginOfRange(int64_t /* priority */) {} + virtual void prefetchBeginOfRange(Priority) {} protected: /// Returns actual column name in part, which can differ from table metadata. diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 4c3d4bc8aa0..9eeb1ec13cf 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -142,7 +142,7 @@ MergeTreeReadTask::MergeTreeReadTask( const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, MergeTreeBlockSizePredictorPtr size_predictor_, - int64_t priority_, + Priority priority_, std::future reader_, std::vector> && pre_reader_for_step_) : data_part{data_part_} diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 5fce95e1714..6eefa993e5e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -71,11 +71,7 @@ struct MergeTreeReadTask std::future reader; std::vector> pre_reader_for_step; - int64_t priority = 0; /// Priority of the task. Bigger value, bigger priority. - bool operator <(const MergeTreeReadTask & rhs) const - { - return priority < rhs.priority; - } + Priority priority; bool isFinished() const { return mark_ranges.empty() && range_reader.isCurrentRangeFinished(); } @@ -86,7 +82,7 @@ struct MergeTreeReadTask const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, MergeTreeBlockSizePredictorPtr size_predictor_, - int64_t priority_ = 0, + Priority priority_ = {}, std::future reader_ = {}, std::vector> && pre_reader_for_step_ = {}); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1d5ac21f803..11aa7c5ec76 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1967,7 +1967,7 @@ try res.part->remove(); else preparePartForRemoval(res.part); - }, 0)); + }, Priority{})); } /// Wait for every scheduled task diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 30f31910bee..6b5214622ff 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -90,7 +90,7 @@ std::future MergeTreePrefetchedReadPool::createPrefetchedRea const IMergeTreeDataPart & data_part, const NamesAndTypesList & columns, const MarkRanges & required_ranges, - int64_t priority) const + Priority priority) const { auto reader = data_part.getReader( columns, storage_snapshot->metadata, required_ranges, @@ -142,7 +142,7 @@ bool MergeTreePrefetchedReadPool::TaskHolder::operator <(const TaskHolder & othe { chassert(task->priority >= 0); chassert(other.task->priority >= 0); - return -task->priority < -other.task->priority; /// Less is better. + return task->priority > other.task->priority; /// Less is better. /// With default std::priority_queue, top() returns largest element. /// So closest to 0 will be on top with this comparator. } @@ -153,7 +153,7 @@ void MergeTreePrefetchedReadPool::startPrefetches() const return; [[maybe_unused]] TaskHolder prev(nullptr, 0); - [[maybe_unused]] const int64_t highest_priority = reader_settings.read_settings.priority + 1; + [[maybe_unused]] const Priority highest_priority{reader_settings.read_settings.priority.value + 1}; assert(prefetch_queue.top().task->priority == highest_priority); while (!prefetch_queue.empty()) { @@ -495,11 +495,11 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr auto need_marks = min_marks_per_thread; /// Priority is given according to the prefetch number for each thread, - /// e.g. the first task of each thread has the same priority and is bigger - /// than second task of each thread, and so on. + /// e.g. the first task of each thread has the same priority and is greater + /// than the second task of each thread, and so on. /// Add 1 to query read priority because higher priority should be given to /// reads from pool which are from reader. - int64_t priority = reader_settings.read_settings.priority + 1; + Priority priority{reader_settings.read_settings.priority.value + 1}; while (need_marks > 0 && part_idx < parts_infos.size()) { @@ -597,7 +597,7 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr { prefetch_queue.emplace(TaskHolder(read_task.get(), i)); } - ++priority; + ++priority.value; result_threads_tasks[i].push_back(std::move(read_task)); } diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index a412dbf0811..7a815777125 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -53,12 +53,11 @@ private: using ThreadTasks = std::deque; using ThreadsTasks = std::map; - /// smaller `priority` means more priority std::future createPrefetchedReader( const IMergeTreeDataPart & data_part, const NamesAndTypesList & columns, const MarkRanges & required_ranges, - int64_t priority) const; + Priority priority) const; void createPrefetchedReaderForTask(MergeTreeReadTask & task) const; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 26a7cb2b50b..d22684eaa9d 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -314,7 +314,7 @@ void MergeTreeReaderCompact::readData( last_read_granule.emplace(from_mark, column_position); } -void MergeTreeReaderCompact::prefetchBeginOfRange(int64_t priority) +void MergeTreeReaderCompact::prefetchBeginOfRange(Priority priority) { if (!initialized) { diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index a994e72d3ff..f180d7508f7 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -38,7 +38,7 @@ public: bool canReadIncompleteGranules() const override { return false; } - void prefetchBeginOfRange(int64_t priority) override; + void prefetchBeginOfRange(Priority priority) override; private: bool isContinuousReading(size_t mark, size_t column_position); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 69617fdf9e3..baacfa55c94 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -58,7 +58,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( } } -void MergeTreeReaderWide::prefetchBeginOfRange(int64_t priority) +void MergeTreeReaderWide::prefetchBeginOfRange(Priority priority) { prefetched_streams.clear(); @@ -90,7 +90,7 @@ void MergeTreeReaderWide::prefetchBeginOfRange(int64_t priority) } void MergeTreeReaderWide::prefetchForAllColumns( - int64_t priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading) + Priority priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading) { bool do_prefetch = data_part_info_for_read->getDataPartStorage()->isStoredOnRemoteDisk() ? settings.read_settings.remote_fs_prefetch @@ -137,7 +137,7 @@ size_t MergeTreeReaderWide::readRows( if (num_columns == 0) return max_rows_to_read; - prefetchForAllColumns(/* priority */0, num_columns, from_mark, current_task_last_mark, continue_reading); + prefetchForAllColumns(Priority{}, num_columns, from_mark, current_task_last_mark, continue_reading); for (size_t pos = 0; pos < num_columns; ++pos) { @@ -305,7 +305,7 @@ void MergeTreeReaderWide::deserializePrefix( } void MergeTreeReaderWide::prefetchForColumn( - int64_t priority, + Priority priority, const NameAndTypePair & name_and_type, const SerializationPtr & serialization, size_t from_mark, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 08bb17c5d56..c31b1baf32e 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -33,14 +33,14 @@ public: bool canReadIncompleteGranules() const override { return true; } - void prefetchBeginOfRange(int64_t priority) override; + void prefetchBeginOfRange(Priority priority) override; using FileStreams = std::map>; private: FileStreams streams; - void prefetchForAllColumns(int64_t priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading); + void prefetchForAllColumns(Priority priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading); void addStreams( const NameAndTypePair & name_and_type, @@ -55,7 +55,7 @@ private: /// Make next readData more simple by calling 'prefetch' of all related ReadBuffers (column streams). void prefetchForColumn( - int64_t priority, + Priority priority, const NameAndTypePair & name_and_type, const SerializationPtr & serialization, size_t from_mark, diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index 500327afd61..53695639769 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -84,7 +84,7 @@ struct MergeTreeSource::AsyncReadingState { try { - callback_runner(std::move(job), 0); + callback_runner(std::move(job), Priority{}); } catch (...) { diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 51acb6a3a7d..80a49b2cd1d 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -356,7 +356,7 @@ private: request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); return outcome; - }, 0); + }, Priority{}); } std::mutex mutex; @@ -619,7 +619,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() std::future StorageS3Source::createReaderAsync() { - return create_reader_scheduler([this] { return createReader(); }, 0); + return create_reader_scheduler([this] { return createReader(); }, Priority{}); } StorageS3Source::ReadBufferOrFactory StorageS3Source::createS3ReadBuffer(const String & key, size_t object_size) From 3ef04ad4177091302c57772f5e4f9c6fc32f9855 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 26 May 2023 10:56:24 -0300 Subject: [PATCH 0713/2223] Doc. clarify dns_max_consecutive_failures one more time (#50212) --- src/Core/ServerSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 1441c22507d..511b8d68f6d 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -63,7 +63,7 @@ namespace DB \ M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \ M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \ - M(UInt32, dns_max_consecutive_failures, 1024, "Max connection failures before dropping host from ClickHouse DNS cache.", 0) \ + M(UInt32, dns_max_consecutive_failures, 1024, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \ \ M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ From 4c218f28924fbd021341aafd8a27dbdd38c52e74 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 26 May 2023 09:59:03 -0400 Subject: [PATCH 0714/2223] Upgrade boost submodule (#50188) --- contrib/boost | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boost b/contrib/boost index 8fe7b3326ef..aec12eea7fc 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit 8fe7b3326ef482ee6ecdf5a4f698f2b8c2780f98 +Subproject commit aec12eea7fc762721ae16943d1361340c66c9c17 From b9ef5151ad5aefa44a41e14302cdaca1815749b3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 26 May 2023 11:18:25 -0300 Subject: [PATCH 0715/2223] Update settings.md --- .../en/operations/server-configuration-parameters/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 99630043095..76ff8ea049e 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -917,7 +917,7 @@ We recommend using this option in macOS since the `getrlimit()` function returns Restriction on deleting tables. -If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_table_size_to_drop` (in bytes), you can’t delete it using a [DROP](sql-reference/statements/drop.md) query or [TRUNCATE](../../sql-reference/statements/truncate.md) query. +If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_table_size_to_drop` (in bytes), you can’t delete it using a [DROP](../../sql-reference/statements/drop.md) query or [TRUNCATE](../../sql-reference/statements/truncate.md) query. This setting does not require a restart of the Clickhouse server to apply. Another way to disable the restriction is to create the `/flags/force_drop_table` file. @@ -944,7 +944,7 @@ Default value: 50 GB. The value 0 means that you can drop partitions without any restrictions. :::note -This limitation does not restrict drop table and truncate table, see [max_table_size_to_drop](#max_table_size_to_drop) +This limitation does not restrict drop table and truncate table, see [max_table_size_to_drop](#max-table-size-to-drop) ::: **Example** From 0740bfbe4bfb3d186c8803e15aaffd2042c7f1fa Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 May 2023 14:31:09 +0000 Subject: [PATCH 0716/2223] Better disks moving --- programs/keeper/CMakeLists.txt | 3 +- src/Coordination/Changelog.cpp | 111 +++++++++++++++++++------- src/Coordination/Changelog.h | 2 +- src/Coordination/KeeperContext.cpp | 20 ++--- src/Coordination/KeeperContext.h | 4 +- src/Coordination/KeeperDispatcher.cpp | 6 +- 6 files changed, 98 insertions(+), 48 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 6e97ab324e3..f775e8a5a22 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -131,6 +131,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/StoredObject.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/registerDiskS3.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -144,7 +145,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/IOUringReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferFromTemporaryFile.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferWithFinalizeCallback.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousBoundedReadBuffer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/getThreadPoolReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolRemoteFSReader.cpp diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index b289e1dc9f2..bcce9982e04 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -32,9 +32,21 @@ namespace ErrorCodes namespace { +constexpr std::string_view tmp_prefix = "tmp_"; + void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr description, DiskPtr disk_to, const std::string & path_to) { - disk_from->copyFile(description->path, *disk_to, path_to, {}); + /// we use empty file with prefix tmp_ to detect incomplete copies + /// if a copy is complete we don't care from which disk we use the same file + /// so it's okay if a failure happens after removing of tmp file but before we remove + /// the changelog from the source disk + auto from_path = fs::path(description->path); + auto tmp_changelog_name = from_path.parent_path() / (std::string{tmp_prefix} + from_path.filename().string()); + { + disk_to->writeFile(tmp_changelog_name); + } + disk_from->copyFile(from_path, *disk_to, path_to, {}); + disk_to->removeFile(tmp_changelog_name); disk_from->removeFile(description->path); description->path = path_to; description->disk = disk_to; @@ -164,9 +176,9 @@ public: } } - auto current_log_disk = getCurrentLogDisk(); - assert(file_description->disk == current_log_disk); - file_buf = current_log_disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode); + auto latest_log_disk = getLatestLogDisk(); + assert(file_description->disk == latest_log_disk); + file_buf = latest_log_disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode); assert(file_buf); last_index_written.reset(); current_file_description = std::move(file_description); @@ -176,7 +188,7 @@ public: std::move(file_buf), /* compressi)on level = */ 3, /* append_to_existing_file_ = */ mode == WriteMode::Append, - [current_log_disk, path = current_file_description->path] { return current_log_disk->readFile(path); }); + [latest_log_disk, path = current_file_description->path] { return latest_log_disk->readFile(path); }); prealloc_done = false; } @@ -274,7 +286,7 @@ public: new_description->from_log_index = new_start_log_index; new_description->to_log_index = new_start_log_index + log_file_settings.rotate_interval - 1; new_description->extension = "bin"; - new_description->disk = getCurrentLogDisk(); + new_description->disk = getLatestLogDisk(); if (log_file_settings.compress_logs) new_description->extension += "." + toContentEncodingName(CompressionMethod::Zstd); @@ -413,7 +425,7 @@ private: prealloc_done = true; } - DiskPtr getCurrentLogDisk() const { return keeper_context->getCurrentLogDisk(); } + DiskPtr getLatestLogDisk() const { return keeper_context->getLatestLogDisk(); } DiskPtr getDisk() const { return keeper_context->getLogDisk(); } @@ -574,50 +586,87 @@ Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, Kee , append_completion_queue(std::numeric_limits::max()) , keeper_context(std::move(keeper_context_)) { - if (auto current_log_disk = getCurrentLogDisk(); - log_file_settings.force_sync && dynamic_cast(current_log_disk.get()) == nullptr) + if (auto latest_log_disk = getLatestLogDisk(); + log_file_settings.force_sync && dynamic_cast(latest_log_disk.get()) == nullptr) { throw DB::Exception( DB::ErrorCodes::BAD_ARGUMENTS, "force_sync is set to true for logs but disk '{}' cannot satisfy such guarantee because it's not of type DiskLocal.\n" "If you want to use force_sync and same disk for all logs, please set keeper_server.log_storage_disk to a local disk.\n" "If you want to use force_sync and different disk only for old logs, please set 'keeper_server.log_storage_disk' to any " - "supported disk and 'keeper_server.current_log_storage_disk' to a local disk.\n" + "supported disk and 'keeper_server.latest_log_storage_disk' to a local disk.\n" "Otherwise, disable force_sync", - current_log_disk->getName()); + latest_log_disk->getName()); } /// Load all files on changelog disks const auto load_from_disk = [&](const auto & disk) { + LOG_TRACE(log, "Reading from disk {}", disk->getName()); + std::unordered_map incomplete_files; + + const auto clean_incomplete_file = [&](const auto & file_path) + { + if (auto incomplete_it = incomplete_files.find(fs::path(file_path).filename()); incomplete_it != incomplete_files.end()) + { + LOG_TRACE(log, "Removing {} from {}", file_path, disk->getName()); + disk->removeFile(file_path); + disk->removeFile(incomplete_it->second); + incomplete_files.erase(incomplete_it); + return true; + } + + return false; + }; + + std::vector changelog_files; for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { if (it->name() == changelogs_detached_dir) continue; - auto file_description = getChangelogFileDescription(it->path()); + if (it->name().starts_with(tmp_prefix)) + { + incomplete_files.emplace(it->name().substr(tmp_prefix.size()), it->path()); + continue; + } + + if (clean_incomplete_file(it->path())) + continue; + + changelog_files.push_back(it->path()); + } + + for (const auto & changelog_file : changelog_files) + { + if (clean_incomplete_file(fs::path(changelog_file).filename())) + continue; + + auto file_description = getChangelogFileDescription(changelog_file); file_description->disk = disk; + LOG_TRACE(log, "Found {} on {}", changelog_file, disk->getName()); auto [changelog_it, inserted] = existing_changelogs.insert_or_assign(file_description->from_log_index, std::move(file_description)); if (!inserted) LOG_WARNING(log, "Found duplicate entries for {}, will use the entry from {}", changelog_it->second->path, disk->getName()); } + + for (const auto & [name, path] : incomplete_files) + disk->removeFile(path); }; /// Load all files from old disks for (const auto & disk : keeper_context->getOldLogDisks()) - { load_from_disk(disk); - } auto disk = getDisk(); load_from_disk(disk); - auto current_log_disk = getCurrentLogDisk(); - if (disk != current_log_disk) - load_from_disk(current_log_disk); + auto latest_log_disk = getLatestLogDisk(); + if (disk != latest_log_disk) + load_from_disk(latest_log_disk); if (existing_changelogs.empty()) LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", disk->getPath()); @@ -779,12 +828,12 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin else if (last_log_read_result.has_value()) { /// check if we need to move completed log to another disk - auto current_log_disk = getCurrentLogDisk(); + auto latest_log_disk = getLatestLogDisk(); auto disk = getDisk(); auto & description = existing_changelogs.at(last_log_read_result->log_start_index); - if (current_log_disk != disk && current_log_disk == description->disk) - moveFileBetweenDisks(current_log_disk, description, disk, description->path); + if (latest_log_disk != disk && latest_log_disk == description->disk) + moveFileBetweenDisks(latest_log_disk, description, disk, description->path); } /// Start new log if we don't initialize writer from previous log. All logs can be "complete". @@ -793,14 +842,14 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin /// Move files to correct disks auto latest_start_index = current_writer->getStartIndex(); - auto current_log_disk = getCurrentLogDisk(); + auto latest_log_disk = getLatestLogDisk(); auto disk = getDisk(); for (const auto & [start_index, description] : existing_changelogs) { - /// latest log should already be on current_log_disk + /// latest log should already be on latest_log_disk if (start_index == latest_start_index) { - chassert(description->disk == current_log_disk); + chassert(description->disk == latest_log_disk); continue; } @@ -825,9 +874,9 @@ void Changelog::initWriter(ChangelogFileDescriptionPtr description) LOG_TRACE(log, "Continue to write into {}", description->path); auto log_disk = description->disk; - auto current_log_disk = getCurrentLogDisk(); - if (log_disk != current_log_disk) - moveFileBetweenDisks(log_disk, description, current_log_disk, description->path); + auto latest_log_disk = getLatestLogDisk(); + if (log_disk != latest_log_disk) + moveFileBetweenDisks(log_disk, description, latest_log_disk, description->path); current_writer->setFile(std::move(description), WriteMode::Append); } @@ -855,9 +904,9 @@ DiskPtr Changelog::getDisk() const return keeper_context->getLogDisk(); } -DiskPtr Changelog::getCurrentLogDisk() const +DiskPtr Changelog::getLatestLogDisk() const { - return keeper_context->getCurrentLogDisk(); + return keeper_context->getLatestLogDisk(); } void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end) @@ -1044,9 +1093,9 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) description = std::prev(index_changelog)->second; auto log_disk = description->disk; - auto current_log_disk = getCurrentLogDisk(); - if (log_disk != current_log_disk) - moveFileBetweenDisks(log_disk, description, current_log_disk, description->path); + auto latest_log_disk = getLatestLogDisk(); + if (log_disk != latest_log_disk) + moveFileBetweenDisks(log_disk, description, latest_log_disk, description->path); current_writer->setFile(std::move(description), WriteMode::Append); diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 4054829ef19..d20eaa1666c 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -155,7 +155,7 @@ private: static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry); DiskPtr getDisk() const; - DiskPtr getCurrentLogDisk() const; + DiskPtr getLatestLogDisk() const; /// Currently existing changelogs std::map existing_changelogs; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 1d6f1be9bfb..9d7e62ffae7 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -22,10 +22,10 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) log_storage = getLogsPathFromConfig(config); - if (config.has("keeper_server.current_log_storage_disk")) - current_log_storage = config.getString("keeper_server.current_log_storage_disk"); + if (config.has("keeper_server.latest_log_storage_disk")) + latest_log_storage = config.getString("keeper_server.latest_log_storage_disk"); else - current_log_storage = log_storage; + latest_log_storage = log_storage; const auto collect_old_disk_names = [&](const std::string_view key_prefix, std::vector & disk_names) { @@ -34,7 +34,7 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) for (const auto & key : disk_name_keys) { if (key.starts_with(key_prefix)) - disk_names.push_back(config.getString(fmt::format("keeper_server.{}", key_prefix))); + disk_names.push_back(config.getString(fmt::format("keeper_server.{}", key))); } }; @@ -96,15 +96,15 @@ std::vector KeeperContext::getOldLogDisks() const return old_log_disks; } -DiskPtr KeeperContext::getCurrentLogDisk() const +DiskPtr KeeperContext::getLatestLogDisk() const { - return getDisk(current_log_storage); + return getDisk(latest_log_storage); } void KeeperContext::setLogDisk(DiskPtr disk) { log_storage = disk; - current_log_storage = std::move(disk); + latest_log_storage = std::move(disk); } DiskPtr KeeperContext::getSnapshotDisk() const @@ -156,9 +156,9 @@ void KeeperContext::dumpConfiguration(WriteBufferFromOwnString & buf) const auto log_disk = getDisk(log_storage); dump_disk_info("log_storage", *log_disk); - auto current_log_disk = getDisk(current_log_storage); - if (log_disk != current_log_disk) - dump_disk_info("current_log_storage", *current_log_disk); + auto latest_log_disk = getDisk(latest_log_storage); + if (log_disk != latest_log_disk) + dump_disk_info("latest_log_storage", *latest_log_disk); } { diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index e41d8e35032..00607fd904e 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -34,7 +34,7 @@ public: bool digestEnabled() const; void setDigestEnabled(bool digest_enabled_); - DiskPtr getCurrentLogDisk() const; + DiskPtr getLatestLogDisk() const; DiskPtr getLogDisk() const; std::vector getOldLogDisks() const; void setLogDisk(DiskPtr disk); @@ -65,7 +65,7 @@ private: std::shared_ptr disk_selector; Storage log_storage; - Storage current_log_storage; + Storage latest_log_storage; Storage snapshot_storage; Storage state_file_storage; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 6d1239c9210..a5b0f0b2c01 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -798,9 +798,9 @@ uint64_t KeeperDispatcher::getLogDirSize() const auto log_disk = keeper_context->getLogDisk(); auto size = getTotalSize(log_disk); - auto current_log_disk = keeper_context->getCurrentLogDisk(); - if (log_disk != current_log_disk) - size += getTotalSize(current_log_disk); + auto latest_log_disk = keeper_context->getLatestLogDisk(); + if (log_disk != latest_log_disk) + size += getTotalSize(latest_log_disk); return size; } From c57fa9d4965ae7460c233d112c3458681cf89c43 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Fri, 26 May 2023 16:50:43 +0200 Subject: [PATCH 0717/2223] Docs: Add PostgreSQL and ODBC related settings --- docs/en/operations/settings/settings.md | 32 ++++++++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 50b114fcb00..4f3b4e43358 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1187,6 +1187,36 @@ Disable limit on kafka_num_consumers that depends on the number of available CPU Default value: false. +## postgresql_connection_pool_size {#postgresql-connection-pool-size} + +Connection pool size for PostgreSQL table engine and database engine. + +Default value: 16 + +## postgresql_connection_pool_size {#postgresql-connection-pool-size} + +Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool. + +Default value: 5000 + +## postgresql_connection_pool_auto_close_connection {#postgresql-connection-pool-auto-close-connection} + +Close connection before returning connection to the pool. + +Default value: true. + +## odbc_bridge_connection_pool_size {#odbc-bridge-connection-pool-size} + +Connection pool size for each connection settings string in ODBC bridge. + +Default value: 16 + +## odbc_bridge_use_connection_pooling {#odbc-bridge-use-connection-pooling} + +Use connection pooling in ODBC bridge. If set to false, a new connection is created every time. + +Default value: true + ## use_uncompressed_cache {#setting-use_uncompressed_cache} Whether to use a cache of uncompressed blocks. Accepts 0 or 1. By default, 0 (disabled). @@ -3563,7 +3593,7 @@ SETTINGS index_granularity = 8192 │ ## external_table_functions_use_nulls {#external-table-functions-use-nulls} -Defines how [mysql](../../sql-reference/table-functions/mysql.md), [postgresql](../../sql-reference/table-functions/postgresql.md) and [odbc](../../sql-reference/table-functions/odbc.md)] table functions use Nullable columns. +Defines how [mysql](../../sql-reference/table-functions/mysql.md), [postgresql](../../sql-reference/table-functions/postgresql.md) and [odbc](../../sql-reference/table-functions/odbc.md) table functions use Nullable columns. Possible values: From 4bdce2a6690dedcfe2b292732bb39e1a74e5918b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 May 2023 15:02:50 +0000 Subject: [PATCH 0718/2223] Fix test for new analyzer --- src/Planner/Planner.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 3c9df29534d..c6ae54b4280 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1192,13 +1192,14 @@ void Planner::buildPlanForQueryNode() const auto & settings = query_context->getSettingsRef(); - if (planner_context->getTableExpressionNodeToData().size() > 1 - && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas > 0)) + /// Check support for JOIN for parallel replicas with custom key + if (planner_context->getTableExpressionNodeToData().size() > 1) { - if (settings.allow_experimental_parallel_reading_from_replicas == 1) + if (settings.allow_experimental_parallel_reading_from_replicas == 1 || !settings.parallel_replicas_custom_key.value.empty()) { - LOG_WARNING( - &Poco::Logger::get("Planner"), "JOINs are not supported with parallel replicas. Query will be executed without using them."); + LOG_WARNING( + &Poco::Logger::get("Planner"), + "JOINs are not supported with parallel replicas. Query will be executed without using them."); auto & mutable_context = planner_context->getMutableQueryContext(); mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); From edffedbabf84f2e4d539b684a65b3085430a45aa Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 26 May 2023 15:10:08 +0000 Subject: [PATCH 0719/2223] fix --- src/Common/AsyncLoader.h | 12 ++++++++++-- src/Common/tests/gtest_async_loader.cpp | 4 +++- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index c3d11b26336..15f7ae9722b 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -235,6 +235,14 @@ inline LoadJobSet getGoals(const LoadTaskPtrs & tasks) return result; } +inline LoadJobSet getGoalsOr(const LoadTaskPtrs & tasks, const LoadJobSet & alternative) +{ + LoadJobSet result; + for (const auto & task : tasks) + result.insert(task->goals().begin(), task->goals().end()); + return result.empty() ? alternative : result; +} + inline LoadJobSet joinJobs(const LoadJobSet & jobs1, const LoadJobSet & jobs2) { LoadJobSet result; @@ -368,7 +376,7 @@ public: // WARNING: all tasks instances should be destructed before associated AsyncLoader. ~AsyncLoader(); - // Start workers to execute scheduled load jobs. + // Start workers to execute scheduled load jobs. Note that AsyncLoader is constructed as already started. void start(); // Wait for all load jobs to finish, including all new jobs. So at first take care to stop adding new jobs. @@ -390,7 +398,7 @@ public: void schedule(const LoadTaskPtr & task); // Schedule all tasks atomically. To ensure only highest priority jobs among all tasks are run first. - void schedule(const std::vector & tasks); + void schedule(const LoadTaskPtrs & tasks); // Increase priority of a job and all its dependencies recursively. // Jobs from higher (than `new_pool`) priority pools are not changed. diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 6a7d2d727a9..af685b10f4e 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -44,7 +44,9 @@ struct AsyncLoaderTest explicit AsyncLoaderTest(std::vector initializers) : loader(getPoolInitializers(initializers), /* log_failures = */ false, /* log_progress = */ false) - {} + { + loader.stop(); // All tests call `start()` manually to better control ordering + } explicit AsyncLoaderTest(size_t max_threads = 1) : AsyncLoaderTest({{.max_threads = max_threads}}) From f49160ef4e99fadbff0fcfdad5b4f3e3e6e539f0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 26 May 2023 13:46:59 +0000 Subject: [PATCH 0720/2223] Build partial key positions from entire training data --- .../sql-reference/functions/hash-functions.md | 39 +++- src/Functions/EntropyLearnedHash.cpp | 196 ++++++++++++++---- .../02734_entropy_learned_hashing.reference | 6 + .../02734_entropy_learned_hashing.sql | 27 ++- 4 files changed, 209 insertions(+), 59 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 6758c7e0114..91c45f90018 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -560,7 +560,9 @@ Result: └───────────────────────────┘ ``` -## Entropy-learned hashing +## Entropy-learned hashing (experimental) + +Entropy-learned hashing is disabled by default, to enable: `SET allow_experimental_hash_functions=1`. Entropy-learned hashing is not a standalone hash function like `metroHash64`, `cityHash64`, `sipHash64` etc. Instead, it aims to preprocess the data to be hashed in a way that a standalone hash function can be computed more efficiently while not compromising the hash quality, @@ -570,15 +572,25 @@ bytes, then a hash function will be 95% less expensive to evaluate. For details Time Hashing with Controllable Uniformity](https://doi.org/10.1145/3514221.3517894). Entropy-learned hashing has two phases: -1. A training phase on a representative but typically small set of Strings to be hashed. Function `trainEntropyLearnedHash(data, id)` - calculates a minimal partial sub-key of `data` and stores it as `id`. The training step outputs dummy `0` values. + +1. A training phase on a representative but typically small set of Strings to be hashed. Training consists of two steps: + + - Function `prepareTrainEntropyLearnedHash(data, id)` caches the training data in a global state under a given `id`. It returns dummy + value `0` on every row. + - Function `trainEntropyLearnedHash(id)` computes a minimal partial sub-key of the training data stored stored under `id` in the global + state. The result is stored in the global state as well. It returns dummy value `0` on every row. + 2. An evaluation phase where hashes are computed using the previously calculated partial sub-keys. Function `entropyLearnedHash(data, id)` hashes `data` using the partial subkey stored as `id`. CityHash64 is used as hash function. +The reason that the training phase comprises two steps is that ClickHouse processes data at chunk granularity but entropy-learned hashing +needs to process the entire training set at once. + **Syntax** ``` sql -trainEntropyLearnedHash(data, id); +prepareTrainEntropyLearnedHash(data, id); +trainEntropyLearnedHash(id); entropyLearnedHash(data, id); ``` @@ -588,18 +600,25 @@ entropyLearnedHash(data, id); CREATE TABLE tab (col String) ENGINE=Memory; INSERT INTO tab VALUES ('aa'), ('ba'), ('ca'); -SELECT trainEntropyLearnedHash(col, 'id1') AS trained FROM tab; +SELECT prepareTrainEntropyLearnedHash(col, 'id1') AS prepared FROM tab; +SELECT trainEntropyLearnedHash('id1') AS trained FROM tab; SELECT entropyLearnedHash(col, 'id1') as hashes FROM tab; ``` Result: ``` response -┌─trained─┐ -│ 0 │ -│ 0 │ -│ 0 │ -└─────────┘ +┌─prepared─┐ +│ 0 │ +│ 0 │ +│ 0 │ +└──────────┘ + +┌─trained─┐ +│ 0 │ +│ 0 │ +│ 0 │ +└─────────┘ ┌───────────────hashes─┐ │ 2603192927274642682 │ diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index 9c517a51f77..a80a8fe0228 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; + extern const int SUPPORT_IS_DISABLED; } namespace @@ -116,49 +117,117 @@ std::pair chooseBytes(const std::vectorinsertRangeFrom(*other_training_sample, 0, other_training_sample->size()); + } + + const ColumnString * concatenated_training_sample_string = checkAndGetColumn(*concatenated_training_sample); + if (!concatenated_training_sample_string) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column"); + + const size_t num_rows = concatenated_training_sample_string->size(); + std::vector training_data; + for (size_t i = 0; i < num_rows; ++i) + { + std::string_view string_view = concatenated_training_sample_string->getDataAt(i).toView(); + training_data.emplace_back(string_view); + } + + PartialKeyPositions partial_key_positions = chooseBytes(training_data).first; + + ids_for_user[id].partial_key_positions = partial_key_positions; + training_samples.clear(); + } + + const PartialKeyPositions & getPartialKeyPositions(const String & user_name, const String & id) const + { + std::lock_guard lock(mutex); + auto it_user = global_state.find(user_name); + if (it_user == global_state.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Id {} not registered for user in entropy learned hashing", id); auto it_id = it_user->second.find(id); if (it_id == it_user->second.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Id {} not registered for user in entropy learned hashing", id); - return it_id->second; + return it_id->second.partial_key_positions; } private: mutable std::mutex mutex; - /// Map: user name --> (Map: dataset id --> byte positions to hash) - std::map> partial_key_positions_by_id TSA_GUARDED_BY(mutex); + + /// The state. + struct ColumnsAndPartialKeyPositions + { + /// Caches training data chunks. Filled by prepareTrainEntropyLearnedHash(), cleared by trainEntropyLearnedHash(). + MutableColumns training_samples; + /// The result of the training phase. Filled by trainEntropyLearnedHash(). + PartialKeyPositions partial_key_positions; + }; + + /// Maps a state id to the state. + using IdToColumnsAndPartialKeyPositions = std::map; + + /// Maps the user name to a state id. As a result, the state id is unique at user scope. + using UserNameToId = std::map; + + UserNameToId global_state TSA_GUARDED_BY(mutex); }; } -class FunctionTrainEntropyLearnedHash : public IFunction + +/// Copies all chunks of the training sample column into the global state under a given id. +class FunctionPrepareTrainEntropyLearnedHash : public IFunction { public: - static constexpr auto name = "trainEntropyLearnedHash"; - static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getUserName()); } - explicit FunctionTrainEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} + static constexpr auto name = "prepareTrainEntropyLearnedHash"; + static FunctionPtr create(ContextPtr context) + { + if(!context->getSettings().allow_experimental_hash_functions) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Entropy-learned hashing is experimental. Set `allow_experimental_hash_functions` setting to enable it"); + + return std::make_shared(context->getUserName()); + } + explicit FunctionPrepareTrainEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} String getName() const override { return name; } - bool isVariadic() const override { return false; } size_t getNumberOfArguments() const override { return 2; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } @@ -182,42 +251,88 @@ public: const ColumnConst * id_col_const = checkAndGetColumn(id_col); const String id = id_col_const->getValue(); - const auto * data_col = arguments[0].column.get(); - if (const ColumnString * col_data_string = checkAndGetColumn(data_col)) - { - const size_t num_rows = col_data_string->size(); + IColumn::Ptr data_col = arguments[0].column; + IColumn::MutablePtr data_col_mutable = IColumn::mutate(data_col); - std::vector training_data; - for (size_t i = 0; i < num_rows; ++i) - { - std::string_view string_view = col_data_string->getDataAt(i).toView(); - training_data.emplace_back(string_view); - } + auto & global_state = EntropyLearnedHashGlobalState::instance(); + global_state.cacheTrainingSample(user_name, id, std::move(data_col_mutable)); - PartialKeyPositions partial_key_positions = chooseBytes(training_data).first; - auto & id_manager = IdManager::instance(); - id_manager.setPartialKeyPositionsForId(user_name, id, partial_key_positions); - - return result_type->createColumnConst(num_rows, 0u)->convertToFullColumnIfConst(); - } - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - arguments.begin()->column->getName(), getName()); + const size_t num_rows = data_col->size(); + return result_type->createColumnConst(num_rows, 0u); /// dummy output } private: const String user_name; }; +/// 1. Concatenates the training samples of a given id in the global state. +/// 2. Computes the partial key positions from the concatenated training samples and stores that in the global state. +/// 3. clear()-s the training samples in the global state. +class FunctionTrainEntropyLearnedHash : public IFunction +{ +public: + static constexpr auto name = "trainEntropyLearnedHash"; + static FunctionPtr create(ContextPtr context) + { + if(!context->getSettings().allow_experimental_hash_functions) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Entropy-learned hashing is experimental. Set `allow_experimental_hash_functions` setting to enable it"); + return std::make_shared(context->getUserName()); + } + explicit FunctionTrainEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return false; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"id", &isString, nullptr, "String"} + }; + + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override + { + const IColumn * id_col = arguments[0].column.get(); + const ColumnConst * id_col_const = checkAndGetColumn(id_col); + if (!id_col_const) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", + arguments.begin()->column->getName(), getName()); + + auto & global_state = EntropyLearnedHashGlobalState::instance(); + + const String id = id_col_const->getValue(); + global_state.train(user_name, id); + + const size_t num_rows = id_col->size(); + return result_type->createColumnConst(num_rows, 0u); /// dummy output + } +private: + const String user_name; +}; + + +/// Hashes input strings using partial key positions stored in the global state. class FunctionEntropyLearnedHash : public IFunction { public: static constexpr auto name = "entropyLearnedHash"; - static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getUserName()); } + static FunctionPtr create(ContextPtr context) + { + if(!context->getSettings().allow_experimental_hash_functions) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Entropy-learned hashing experimental. Set `allow_experimental_hash_functions` setting to enable it"); + return std::make_shared(context->getUserName()); + } explicit FunctionEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} String getName() const override { return name; } - bool isVariadic() const override { return false; } size_t getNumberOfArguments() const override { return 2; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } @@ -242,8 +357,8 @@ public: const ColumnConst * id_col_const = checkAndGetColumn(id_col); const String id = id_col_const->getValue(); - const auto & id_manager = IdManager::instance(); - const auto & partial_key_positions = id_manager.getPartialKeyPositionsForId(user_name, id); + const auto & global_state = EntropyLearnedHashGlobalState::instance(); + const auto & partial_key_positions = global_state.getPartialKeyPositions(user_name, id); const auto * data_col = arguments[0].column.get(); if (const auto * col_data_string = checkAndGetColumn(data_col)) @@ -272,6 +387,7 @@ private: REGISTER_FUNCTION(EntropyLearnedHash) { + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); } diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference b/tests/queries/0_stateless/02734_entropy_learned_hashing.reference index 2240da7fea4..f558e3cd444 100644 --- a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference +++ b/tests/queries/0_stateless/02734_entropy_learned_hashing.reference @@ -1,12 +1,18 @@ 0 0 0 +0 +0 +0 2603192927274642682 4947675599669400333 10783339242466472992 0 0 0 +0 +0 +0 2603192927274642682 4947675599669400333 10783339242466472992 diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql b/tests/queries/0_stateless/02734_entropy_learned_hashing.sql index c0c07b26938..ae829fa03c3 100644 --- a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql +++ b/tests/queries/0_stateless/02734_entropy_learned_hashing.sql @@ -1,21 +1,30 @@ +-- Tags: no-parallel +-- no-parallel because entropy-learned hash uses global state + +SET allow_experimental_hash_functions = 1; + +-- no commonalities between keys DROP TABLE IF EXISTS tbl1; CREATE TABLE tbl1 (x String) ENGINE=Memory; --- no commonalities between keys INSERT INTO tbl1 VALUES ('a'), ('b'), ('c'); -SELECT trainEntropyLearnedHash(x, 'id1') FROM tbl1; +SELECT prepareTrainEntropyLearnedHash(x, 'id1') FROM tbl1; +SELECT trainEntropyLearnedHash('id1') FROM tbl1; SELECT entropyLearnedHash(x, 'id1') FROM tbl1; -SELECT trainEntropyLearnedHash(x, 1) FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } --- SELECT trainEntropyLearnedHash(x, NULL) FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT trainEntropyLearnedHash(1, 'id1') FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT entropyLearnedHash(x, 'non-existing id') FROM tbl1; -- { serverError BAD_ARGUMENTS } - +-- with commonalities between keys DROP TABLE IF EXISTS tbl2; CREATE TABLE tbl2 (x String) ENGINE=Memory; --- with commonalities between keys INSERT INTO tbl2 VALUES ('aa'), ('ba'), ('ca'); -SELECT trainEntropyLearnedHash(x, 'id2') FROM tbl2; +SELECT prepareTrainEntropyLearnedHash(x, 'id2') FROM tbl2; +SELECT trainEntropyLearnedHash('id2') FROM tbl2; SELECT entropyLearnedHash(x, 'id2') FROM tbl2; +-- negative tests +SELECT prepareTrainEntropyLearnedHash(x, 1) FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT prepareTrainEntropyLearnedHash(1, 'id1') FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT trainEntropyLearnedHash(1) FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT entropyLearnedHash(1, 'id1') FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT entropyLearnedHash(x, 'non-existing id') FROM tbl1; -- { serverError BAD_ARGUMENTS } + DROP TABLE tbl1; DROP TABLE tbl2; From ac1dc7237ec9301c94ac167303bd4e0b80b3114c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 26 May 2023 15:37:33 +0000 Subject: [PATCH 0721/2223] Update docs --- docs/en/sql-reference/functions/hash-functions.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 91c45f90018..663b8468a94 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -578,7 +578,7 @@ Entropy-learned hashing has two phases: - Function `prepareTrainEntropyLearnedHash(data, id)` caches the training data in a global state under a given `id`. It returns dummy value `0` on every row. - Function `trainEntropyLearnedHash(id)` computes a minimal partial sub-key of the training data stored stored under `id` in the global - state. The result is stored in the global state as well. It returns dummy value `0` on every row. + state. The cached training data in the global state is replaced by the partial key. Dummy value `0` is returned on every row. 2. An evaluation phase where hashes are computed using the previously calculated partial sub-keys. Function `entropyLearnedHash(data, id)` hashes `data` using the partial subkey stored as `id`. CityHash64 is used as hash function. @@ -586,6 +586,9 @@ Entropy-learned hashing has two phases: The reason that the training phase comprises two steps is that ClickHouse processes data at chunk granularity but entropy-learned hashing needs to process the entire training set at once. +Since functions `prepareTrainEntropyLearnedHash()` and `trainEntropyLearnedHash()` access global state, they should not be called in +parallel with the same `id`. + **Syntax** ``` sql @@ -597,6 +600,7 @@ entropyLearnedHash(data, id); **Example** ```sql +SET allow_experimental_hash_functions=1; CREATE TABLE tab (col String) ENGINE=Memory; INSERT INTO tab VALUES ('aa'), ('ba'), ('ca'); From 6b5933a3db9491036b7bd933b177e5590a329eb1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 May 2023 17:58:42 +0200 Subject: [PATCH 0722/2223] Add new profile events for queries with subqueries Differs from SelectQuery/InsertQuery that it could be > 1 (in case of MATERIALIZED VIEW attached to the table), since each local SELECT/INSERT query is accounted, not only initial. Signed-off-by: Azat Khuzhin --- src/Common/ProfileEvents.cpp | 3 + src/Interpreters/InterpreterFactory.cpp | 10 ++ src/Interpreters/InterpreterInsertQuery.cpp | 10 ++ src/Interpreters/InterpreterSelectQuery.cpp | 12 +- src/Planner/Planner.cpp | 10 ++ ...s_with_subqueries_profile_events.reference | 24 ++++ ..._queries_with_subqueries_profile_events.sh | 120 ++++++++++++++++++ 7 files changed, 188 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference create mode 100755 tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index dd96b724793..ab96106be7a 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -8,6 +8,9 @@ M(Query, "Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries.") \ M(SelectQuery, "Same as Query, but only for SELECT queries.") \ M(InsertQuery, "Same as Query, but only for INSERT queries.") \ + M(QueriesWithSubqueries, "Count queries with all subqueries") \ + M(SelectQueriesWithSubqueries, "Count SELECT queries with all subqueries") \ + M(InsertQueriesWithSubqueries, "Count INSERT queries with all subqueries") \ M(AsyncInsertQuery, "Same as InsertQuery, but only for asynchronous INSERT queries.") \ M(AsyncInsertBytes, "Data size in bytes of asynchronous INSERT queries.") \ M(AsyncInsertRows, "Number of rows inserted by asynchronous INSERT queries.") \ diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index eec73fa56e2..9cd1f2a251c 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -114,6 +114,7 @@ namespace ProfileEvents { extern const Event Query; + extern const Event QueriesWithSubqueries; extern const Event SelectQuery; extern const Event InsertQuery; } @@ -131,6 +132,15 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { ProfileEvents::increment(ProfileEvents::Query); + /// SELECT and INSERT query will handle QueriesWithSubqueries on their own. + if (!(query->as() || + query->as() || + query->as() || + query->as())) + { + ProfileEvents::increment(ProfileEvents::QueriesWithSubqueries); + } + if (query->as()) { if (context->getSettingsRef().allow_experimental_analyzer) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b5b159f37d5..e87b16f0e9d 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -34,8 +34,15 @@ #include #include #include +#include +namespace ProfileEvents +{ + extern const Event InsertQueriesWithSubqueries; + extern const Event QueriesWithSubqueries; +} + namespace DB { @@ -234,6 +241,9 @@ Chain InterpreterInsertQuery::buildChain( ThreadStatusesHolderPtr thread_status_holder, std::atomic_uint64_t * elapsed_counter_ms) { + ProfileEvents::increment(ProfileEvents::InsertQueriesWithSubqueries); + ProfileEvents::increment(ProfileEvents::QueriesWithSubqueries); + ThreadGroupPtr running_group; if (current_thread) running_group = current_thread->getThreadGroup(); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d4ca2e405e6..fd9ecb0d7b1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -93,11 +94,17 @@ #include #include #include -#include #include +#include #include "config_version.h" +namespace ProfileEvents +{ + extern const Event SelectQueriesWithSubqueries; + extern const Event QueriesWithSubqueries; +} + namespace DB { @@ -1329,6 +1336,9 @@ static bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query) void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional prepared_pipe) { + ProfileEvents::increment(ProfileEvents::SelectQueriesWithSubqueries); + ProfileEvents::increment(ProfileEvents::QueriesWithSubqueries); + /** Streams of data. When the query is executed in parallel, we have several data streams. * If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then * if there is an ORDER BY, then glue the streams using ResizeProcessor, and then MergeSorting transforms, diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 3c9df29534d..6b69afc9b47 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -2,6 +2,7 @@ #include #include +#include #include @@ -73,6 +74,12 @@ #include #include +namespace ProfileEvents +{ + extern const Event SelectQueriesWithSubqueries; + extern const Event QueriesWithSubqueries; +} + namespace DB { @@ -1155,6 +1162,9 @@ void Planner::buildPlanForUnionNode() void Planner::buildPlanForQueryNode() { + ProfileEvents::increment(ProfileEvents::SelectQueriesWithSubqueries); + ProfileEvents::increment(ProfileEvents::QueriesWithSubqueries); + auto & query_node = query_tree->as(); const auto & query_context = planner_context->getQueryContext(); diff --git a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference new file mode 100644 index 00000000000..1d49b598382 --- /dev/null +++ b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference @@ -0,0 +1,24 @@ +view allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 1 0 1 2 3 +subquery allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 0 1 0 2 2 +CSE allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 0 1 0 2 2 +CSE_Multi allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 0 1 0 2 2 +CTE allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 0 1 0 2 2 +CTE_Multi allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 0 1 0 4 4 +view allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 1 1 0 1 3 4 +subquery allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 1 0 1 0 2 2 +CSE allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 1 0 1 0 2 2 +CSE_Multi allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 1 0 1 0 2 2 +CTE allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 1 0 1 0 2 2 +CTE_Multi allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries +1 1 0 1 0 4 4 diff --git a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh new file mode 100755 index 00000000000..4a7877a68b8 --- /dev/null +++ b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh @@ -0,0 +1,120 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS mv; + DROP TABLE IF EXISTS output; + DROP TABLE IF EXISTS input; + + CREATE TABLE input (key Int) Engine=Null; + CREATE TABLE output AS input Engine=Null; + CREATE MATERIALIZED VIEW mv TO output AS SELECT * FROM input; +" + +for allow_experimental_analyzer in 0 1; do + query_id="$(random_str 10)" + $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "INSERT INTO input SELECT * FROM numbers(1)" + $CLICKHOUSE_CLIENT -mn -q " + SYSTEM FLUSH LOGS; + SELECT + 1 view, + Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + ProfileEvents['InsertQuery'] InsertQuery, + ProfileEvents['SelectQuery'] SelectQuery, + ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, + -- FIXME: for analyzer it will have one more for sample block + ProfileEvents['SelectQueriesWithSubqueries'] SelectQueriesWithSubqueries, + ProfileEvents['QueriesWithSubqueries'] QueriesWithSubqueries + FROM system.query_log + WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query_id = '$query_id' + FORMAT TSVWithNames; + " + + query_id="$(random_str 10)" + $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "SELECT * FROM system.one WHERE dummy IN (SELECT * FROM system.one) FORMAT Null" + $CLICKHOUSE_CLIENT -mn -q " + SYSTEM FLUSH LOGS; + SELECT + 1 subquery, + Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + ProfileEvents['InsertQuery'] InsertQuery, + ProfileEvents['SelectQuery'] SelectQuery, + ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, + ProfileEvents['SelectQueriesWithSubqueries'] SelectQueriesWithSubqueries, + ProfileEvents['QueriesWithSubqueries'] QueriesWithSubqueries + FROM system.query_log + WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query_id = '$query_id' + FORMAT TSVWithNames; + " + + query_id="$(random_str 10)" + $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "WITH (SELECT * FROM system.one) AS x SELECT x FORMAT Null" + $CLICKHOUSE_CLIENT -mn -q " + SYSTEM FLUSH LOGS; + SELECT + 1 CSE, + Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + ProfileEvents['InsertQuery'] InsertQuery, + ProfileEvents['SelectQuery'] SelectQuery, + ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, + ProfileEvents['SelectQueriesWithSubqueries'] SelectQueriesWithSubqueries, + ProfileEvents['QueriesWithSubqueries'] QueriesWithSubqueries + FROM system.query_log + WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query_id = '$query_id' + FORMAT TSVWithNames; + " + + query_id="$(random_str 10)" + $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "WITH (SELECT * FROM system.one) AS x SELECT x, x FORMAT Null" + $CLICKHOUSE_CLIENT -mn -q " + SYSTEM FLUSH LOGS; + SELECT + 1 CSE_Multi, + Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + ProfileEvents['InsertQuery'] InsertQuery, + ProfileEvents['SelectQuery'] SelectQuery, + ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, + ProfileEvents['SelectQueriesWithSubqueries'] SelectQueriesWithSubqueries, + ProfileEvents['QueriesWithSubqueries'] QueriesWithSubqueries + FROM system.query_log + WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query_id = '$query_id' + FORMAT TSVWithNames; + " + + query_id="$(random_str 10)" + $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "WITH x AS (SELECT * FROM system.one) SELECT * FROM x FORMAT Null" + $CLICKHOUSE_CLIENT -mn -q " + SYSTEM FLUSH LOGS; + SELECT + 1 CTE, + Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + ProfileEvents['InsertQuery'] InsertQuery, + ProfileEvents['SelectQuery'] SelectQuery, + ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, + ProfileEvents['SelectQueriesWithSubqueries'] SelectQueriesWithSubqueries, + ProfileEvents['QueriesWithSubqueries'] QueriesWithSubqueries + FROM system.query_log + WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query_id = '$query_id' + FORMAT TSVWithNames; + " + + query_id="$(random_str 10)" + $CLICKHOUSE_CLIENT --allow_experimental_analyzer "$allow_experimental_analyzer" --query_id "$query_id" -q "WITH x AS (SELECT * FROM system.one) SELECT * FROM x UNION ALL SELECT * FROM x FORMAT Null" + $CLICKHOUSE_CLIENT -mn -q " + SYSTEM FLUSH LOGS; + SELECT + 1 CTE_Multi, + Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + ProfileEvents['InsertQuery'] InsertQuery, + ProfileEvents['SelectQuery'] SelectQuery, + ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, + ProfileEvents['SelectQueriesWithSubqueries'] SelectQueriesWithSubqueries, + ProfileEvents['QueriesWithSubqueries'] QueriesWithSubqueries + FROM system.query_log + WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query_id = '$query_id' + FORMAT TSVWithNames; + " +done From c4687a8b9375f3818e46db82f018cbae78cdefea Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 26 May 2023 16:46:59 +0000 Subject: [PATCH 0723/2223] Unify priorities: rework AsyncLoader --- src/Common/AsyncLoader.cpp | 14 +++--- src/Common/AsyncLoader.h | 30 +++++++------ src/Common/tests/gtest_async_loader.cpp | 60 +++++++++++++------------ 3 files changed, 54 insertions(+), 50 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index b5612517cd6..86edcdc8f3d 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -433,7 +433,7 @@ const String & AsyncLoader::getPoolName(size_t pool) const return pools[pool].name; // NOTE: lock is not needed because `name` is const and `pools` are immutable } -ssize_t AsyncLoader::getPoolPriority(size_t pool) const +Priority AsyncLoader::getPoolPriority(size_t pool) const { return pools[pool].priority; // NOTE: lock is not needed because `priority` is const and `pools` are immutable } @@ -576,7 +576,7 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::un { Pool & old_pool = pools[job->pool_id]; Pool & new_pool = pools[new_pool_id]; - if (old_pool.priority >= new_pool.priority) + if (old_pool.priority <= new_pool.priority) return; // Never lower priority or change pool leaving the same priority // Update priority and push job forward through ready queue if needed @@ -590,7 +590,7 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::un spawn(new_pool, lock); } - // Set user-facing pool and priority (may affect executing jobs) + // Set user-facing pool (may affect executing jobs) job->pool_id.store(new_pool_id); // Recurse into dependencies @@ -621,7 +621,7 @@ bool AsyncLoader::canSpawnWorker(Pool & pool, std::unique_lock &) return is_running && !pool.ready_queue.empty() && pool.workers < pool.max_threads - && (!current_priority || *current_priority <= pool.priority); + && (!current_priority || *current_priority >= pool.priority); } bool AsyncLoader::canWorkerLive(Pool & pool, std::unique_lock &) @@ -629,17 +629,17 @@ bool AsyncLoader::canWorkerLive(Pool & pool, std::unique_lock &) return is_running && !pool.ready_queue.empty() && pool.workers <= pool.max_threads - && (!current_priority || *current_priority <= pool.priority); + && (!current_priority || *current_priority >= pool.priority); } void AsyncLoader::updateCurrentPriorityAndSpawn(std::unique_lock & lock) { // Find current priority. // NOTE: We assume low number of pools, so O(N) scans are fine. - std::optional priority; + std::optional priority; for (Pool & pool : pools) { - if (pool.isActive() && (!priority || *priority < pool.priority)) + if (pool.isActive() && (!priority || *priority > pool.priority)) priority = pool.priority; } current_priority = priority; diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 15f7ae9722b..322a4482e4e 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -268,10 +269,10 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs & // `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks job dependencies and priorities. // Basic usage example: -// // Start async_loader with two thread pools (0=bg, 1=fg): +// // Start async_loader with two thread pools (0=fg, 1=bg): // AsyncLoader async_loader({ -// {"BgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 1, .priority = 0} -// {"FgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 2, .priority = 1} +// {"FgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 2, .priority{0}} +// {"BgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 1, .priority{1}} // }); // // // Create and schedule a task consisting of three jobs. Job1 has no dependencies and is run first. @@ -279,19 +280,19 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs & // auto job_func = [&] (const LoadJobPtr & self) { // LOG_TRACE(log, "Executing load job '{}' in pool '{}'", self->name, async_loader->getPoolName(self->pool())); // }; -// auto job1 = makeLoadJob({}, "job1", /* pool_id = */ 0, job_func); -// auto job2 = makeLoadJob({ job1 }, "job2", /* pool_id = */ 0, job_func); -// auto job3 = makeLoadJob({ job1 }, "job3", /* pool_id = */ 0, job_func); +// auto job1 = makeLoadJob({}, "job1", /* pool_id = */ 1, job_func); +// auto job2 = makeLoadJob({ job1 }, "job2", /* pool_id = */ 1, job_func); +// auto job3 = makeLoadJob({ job1 }, "job3", /* pool_id = */ 1, job_func); // auto task = makeLoadTask(async_loader, { job1, job2, job3 }); // task.schedule(); // // // Another thread may prioritize a job by changing its pool and wait for it: -// async_loader->prioritize(job3, /* pool_id = */ 1); // higher priority jobs are run first, default priority is zero. -// job3->wait(); // blocks until job completion or cancellation and rethrow an exception (if any) +// async_loader->prioritize(job3, /* pool_id = */ 0); // Increase priority: 1 -> 0 (lower is better) +// job3->wait(); // Blocks until job completion or cancellation and rethrow an exception (if any) // // Every job has a pool associated with it. AsyncLoader starts every job in its thread pool. // Each pool has a constant priority and a mutable maximum number of threads. -// Higher priority (greater `pool.priority` value) jobs are run first. +// Higher priority (lower `pool.priority` value) jobs are run first. // No job with lower priority is started while there is at least one higher priority job ready or running. // // Job priority can be elevated (but cannot be lowered) @@ -301,7 +302,8 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs & // this also leads to a priority inheritance for all the dependencies. // Value stored in load job `pool_id` field is atomic and can be changed even during job execution. // Job is, of course, not moved from its initial thread pool, but it should use `self->pool()` for -// all new jobs it create to avoid priority inversion. +// all new jobs it create to avoid priority inversion. To obtain pool in which job is being executed +// call `self->execution_pool()` instead. // // === IMPLEMENTATION DETAILS === // All possible states and statuses of a job: @@ -335,7 +337,7 @@ private: struct Pool { const String name; - const ssize_t priority; + const Priority priority; std::unique_ptr thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools. std::map ready_queue; // FIFO queue of jobs to be executed in this pool. Map is used for faster erasing. Key is `ready_seqno` size_t max_threads; // Max number of workers to be spawn @@ -367,7 +369,7 @@ public: Metric metric_threads; Metric metric_active_threads; size_t max_threads; - ssize_t priority; + Priority priority; }; AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_); @@ -412,7 +414,7 @@ public: size_t getMaxThreads(size_t pool) const; const String & getPoolName(size_t pool) const; - ssize_t getPoolPriority(size_t pool) const; + Priority getPoolPriority(size_t pool) const; size_t getScheduledJobCount() const; @@ -451,7 +453,7 @@ private: mutable std::mutex mutex; // Guards all the fields below. bool is_running = true; - std::optional current_priority; // highest priority among active pools + std::optional current_priority; // highest priority among active pools UInt64 last_ready_seqno = 0; // Increasing counter for ready queue keys. std::unordered_map scheduled_jobs; // Full set of scheduled pending jobs along with scheduling info. std::vector pools; // Thread pools for job execution and ready queues diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index af685b10f4e..a5cdde881fd 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -32,7 +32,7 @@ namespace DB::ErrorCodes struct Initializer { size_t max_threads = 1; - ssize_t priority = 0; + Priority priority; }; struct AsyncLoaderTest @@ -144,11 +144,11 @@ struct AsyncLoaderTest TEST(AsyncLoader, Smoke) { AsyncLoaderTest t({ - {.max_threads = 2, .priority = 0}, - {.max_threads = 2, .priority = -1}, + {.max_threads = 2, .priority = Priority{0}}, + {.max_threads = 2, .priority = Priority{1}}, }); - static constexpr ssize_t low_priority_pool = 1; + static constexpr size_t low_priority_pool = 1; std::atomic jobs_done{0}; std::atomic low_priority_jobs_done{0}; @@ -419,6 +419,8 @@ TEST(AsyncLoader, CancelExecutingTask) } } +// This test is disabled due to `MemorySanitizer: use-of-uninitialized-value` issue in `collectSymbolsFromProgramHeaders` function +// More details: https://github.com/ClickHouse/ClickHouse/pull/48923#issuecomment-1545415482 TEST(AsyncLoader, DISABLED_JobFailure) { AsyncLoaderTest t; @@ -595,16 +597,16 @@ TEST(AsyncLoader, TestOverload) TEST(AsyncLoader, StaticPriorities) { AsyncLoaderTest t({ - {.max_threads = 1, .priority = 0}, - {.max_threads = 1, .priority = 1}, - {.max_threads = 1, .priority = 2}, - {.max_threads = 1, .priority = 3}, - {.max_threads = 1, .priority = 4}, - {.max_threads = 1, .priority = 5}, - {.max_threads = 1, .priority = 6}, - {.max_threads = 1, .priority = 7}, - {.max_threads = 1, .priority = 8}, - {.max_threads = 1, .priority = 9}, + {.max_threads = 1, .priority{0}}, + {.max_threads = 1, .priority{-1}}, + {.max_threads = 1, .priority{-2}}, + {.max_threads = 1, .priority{-3}}, + {.max_threads = 1, .priority{-4}}, + {.max_threads = 1, .priority{-5}}, + {.max_threads = 1, .priority{-6}}, + {.max_threads = 1, .priority{-7}}, + {.max_threads = 1, .priority{-8}}, + {.max_threads = 1, .priority{-9}}, }); std::string schedule; @@ -634,9 +636,9 @@ TEST(AsyncLoader, StaticPriorities) TEST(AsyncLoader, SimplePrioritization) { AsyncLoaderTest t({ - {.max_threads = 1, .priority = 0}, - {.max_threads = 1, .priority = 1}, - {.max_threads = 1, .priority = 2}, + {.max_threads = 1, .priority{0}}, + {.max_threads = 1, .priority{-1}}, + {.max_threads = 1, .priority{-2}}, }); t.loader.start(); @@ -674,16 +676,16 @@ TEST(AsyncLoader, SimplePrioritization) TEST(AsyncLoader, DynamicPriorities) { AsyncLoaderTest t({ - {.max_threads = 1, .priority = 0}, - {.max_threads = 1, .priority = 1}, - {.max_threads = 1, .priority = 2}, - {.max_threads = 1, .priority = 3}, - {.max_threads = 1, .priority = 4}, - {.max_threads = 1, .priority = 5}, - {.max_threads = 1, .priority = 6}, - {.max_threads = 1, .priority = 7}, - {.max_threads = 1, .priority = 8}, - {.max_threads = 1, .priority = 9}, + {.max_threads = 1, .priority{0}}, + {.max_threads = 1, .priority{-1}}, + {.max_threads = 1, .priority{-2}}, + {.max_threads = 1, .priority{-3}}, + {.max_threads = 1, .priority{-4}}, + {.max_threads = 1, .priority{-5}}, + {.max_threads = 1, .priority{-6}}, + {.max_threads = 1, .priority{-7}}, + {.max_threads = 1, .priority{-8}}, + {.max_threads = 1, .priority{-9}}, }); for (bool prioritize : {false, true}) @@ -890,8 +892,8 @@ TEST(AsyncLoader, DynamicPools) const size_t max_threads[] { 2, 10 }; const int jobs_in_chain = 16; AsyncLoaderTest t({ - {.max_threads = max_threads[0], .priority = 0}, - {.max_threads = max_threads[1], .priority = 1}, + {.max_threads = max_threads[0], .priority{0}}, + {.max_threads = max_threads[1], .priority{-1}}, }); t.loader.start(); From 4249bda449fdb971bf0fcc28e05317962607b85a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 17 May 2023 15:40:36 +0200 Subject: [PATCH 0724/2223] s3_max_inflight_parts_for_one_file with tests --- src/Core/Settings.h | 1 + src/IO/WriteBufferFromS3.cpp | 13 +- src/IO/WriteBufferFromS3TaskTracker.cpp | 108 +++++++---- src/IO/WriteBufferFromS3TaskTracker.h | 21 ++- src/Storages/StorageS3Settings.cpp | 5 + src/Storages/StorageS3Settings.h | 1 + .../configs/config.d/storage_conf.xml | 2 + .../test_merge_tree_s3/configs/config.xml | 11 ++ .../test_merge_tree_s3/s3_mocks/broken_s3.py | 175 +++++++++++++++--- tests/integration/test_merge_tree_s3/test.py | 107 +++++++++++ 10 files changed, 379 insertions(+), 65 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 47d4ba452f7..6f8132ead91 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -78,6 +78,7 @@ class IColumn; M(UInt64, s3_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_upload_part_size_multiply_factor, 2, "Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.", 0) \ M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3 s3_min_upload_part_size multiplied by s3_upload_part_size_multiply_factor.", 0) \ + M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index c7a3b131d09..680292b16a4 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -92,8 +92,11 @@ WriteBufferFromS3::WriteBufferFromS3( , write_settings(write_settings_) , client_ptr(std::move(client_ptr_)) , object_metadata(std::move(object_metadata_)) - , buffer_allocation_policy(ChooseBufferPolicy(request_settings_.getUploadSettings())) - , task_tracker(std::make_unique(std::move(schedule_))) + , buffer_allocation_policy(ChooseBufferPolicy(upload_settings)) + , task_tracker( + std::make_unique( + std::move(schedule_), + upload_settings.s3_max_inflight_parts_for_one_file)) { LOG_TRACE(log, "Create WriteBufferFromS3, {}", getLogDetails()); @@ -110,7 +113,7 @@ void WriteBufferFromS3::nextImpl() "Cannot write to prefinalized buffer for S3, the file could have been created with PutObjectRequest"); /// Make sense to call to before adding new async task to check if there is an exception - task_tracker->waitReady(); + task_tracker->consumeReady(); hidePartialData(); @@ -134,8 +137,6 @@ void WriteBufferFromS3::preFinalize() LOG_TRACE(log, "preFinalize WriteBufferFromS3. {}", getLogDetails()); - task_tracker->waitReady(); - hidePartialData(); if (hidden_size > 0) @@ -234,7 +235,7 @@ WriteBufferFromS3::~WriteBufferFromS3() { LOG_TRACE(log, "Close WriteBufferFromS3. {}.", getLogDetails()); - // That descructor could be call with finalized=false in case of exceptions + // That destructor could be call with finalized=false in case of exceptions if (!finalized) { LOG_ERROR(log, "WriteBufferFromS3 is not finalized in destructor. It could be if an exception occurs. File is not written to S3. {}.", getLogDetails()); diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 6c7bb35fc5a..091d429c45d 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -7,9 +7,10 @@ namespace DB { -WriteBufferFromS3::TaskTracker::TaskTracker(ThreadPoolCallbackRunner scheduler_) +WriteBufferFromS3::TaskTracker::TaskTracker(ThreadPoolCallbackRunner scheduler_, size_t max_tasks_inflight_) : is_async(bool(scheduler_)) , scheduler(scheduler_ ? std::move(scheduler_) : syncRunner()) + , max_tasks_inflight(max_tasks_inflight_) {} WriteBufferFromS3::TaskTracker::~TaskTracker() @@ -28,34 +29,36 @@ ThreadPoolCallbackRunner WriteBufferFromS3::TaskTracker::syncRunner() }; } -void WriteBufferFromS3::TaskTracker::waitReady() +size_t WriteBufferFromS3::TaskTracker::consumeReady() { - LOG_TEST(log, "waitReady, in queue {}", futures.size()); + LOG_TEST(log, "consumeReady, in queue {}", futures.size()); + + size_t consumed = 0; - /// Exceptions are propagated - auto it = futures.begin(); - while (it != futures.end()) { - chassert(it->valid()); - if (it->wait_for(std::chrono::seconds(0)) != std::future_status::ready) + std::unique_lock lock(mutex); + + for (auto it : finished_futures) { - ++it; - continue; + try + { + it->get(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + throw; + } + + futures.erase(it); } - try - { - it->get(); - } catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - throw; - } - - it = futures.erase(it); + consumed = finished_futures.size(); + finished_futures.clear(); } - LOG_TEST(log, "waitReady ended, in queue {}", futures.size()); + LOG_TEST(log, "consumeReady ended, in queue {}", futures.size()); + return consumed; } void WriteBufferFromS3::TaskTracker::waitAll() @@ -75,6 +78,9 @@ void WriteBufferFromS3::TaskTracker::waitAll() } } futures.clear(); + + /// no concurrent tasks, no mutex required + finished_futures.clear(); } void WriteBufferFromS3::TaskTracker::safeWaitAll() @@ -106,25 +112,65 @@ void WriteBufferFromS3::TaskTracker::safeWaitAll() } } futures.clear(); + + /// no concurrent tasks, no mutex required + finished_futures.clear(); + LOG_TEST(log, "safeWaitAll ended, get in queue {}", futures.size()); } +void WriteBufferFromS3::TaskTracker::waitAny() +{ + LOG_TEST(log, "waitAny, in queue {}", futures.size()); + + while (futures.size() > 0 && consumeReady() == 0) + { + std::unique_lock lock(mutex); + cond_var.wait(lock, [&] () { return finished_futures.size() > 0; }); + } + + LOG_TEST(log, "waitAny ended, in queue {}", futures.size()); +} + void WriteBufferFromS3::TaskTracker::add(Callback && func) { - LOG_TEST(log, "add, in queue {}", futures.size()); + futures.emplace_back(); + auto future_placeholder = std::prev(futures.end()); + FinishedList pre_allocated_finished {future_placeholder}; - auto future = scheduler(std::move(func), Priority{}); - auto exit_scope = scope_guard( - [&future]() - { - future.wait(); - } - ); + Callback func_with_notification = [&, func=std::move(func), pre_allocated_finished=std::move(pre_allocated_finished)] () mutable + { + SCOPE_EXIT({ + DENY_ALLOCATIONS_IN_SCOPE; - futures.push_back(std::move(future)); + std::unique_lock lock(mutex); + finished_futures.splice(finished_futures.end(), pre_allocated_finished, pre_allocated_finished.begin()); + cond_var.notify_one(); + }); + + func(); + }; + + *future_placeholder = scheduler(std::move(func_with_notification), Priority{}); - exit_scope.release(); LOG_TEST(log, "add ended, in queue {}", futures.size()); + + waitInFlight(); +} + +void WriteBufferFromS3::TaskTracker::waitInFlight() +{ + if (!max_tasks_inflight) + return; + + LOG_TEST(log, "waitInFlight, in queue {}", futures.size()); + + while (futures.size() >= max_tasks_inflight) + { + waitAny(); + } + + LOG_TEST(log, "waitInFlight ended, in queue {}", futures.size()); } bool WriteBufferFromS3::TaskTracker::isAsync() const diff --git a/src/IO/WriteBufferFromS3TaskTracker.h b/src/IO/WriteBufferFromS3TaskTracker.h index c978b9a78f0..b6dc7a75368 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.h +++ b/src/IO/WriteBufferFromS3TaskTracker.h @@ -6,6 +6,8 @@ #include "WriteBufferFromS3.h" +#include + namespace DB { @@ -20,22 +22,33 @@ class WriteBufferFromS3::TaskTracker public: using Callback = std::function; - explicit TaskTracker(ThreadPoolCallbackRunner scheduler_); + explicit TaskTracker(ThreadPoolCallbackRunner scheduler_, size_t max_tasks_inflight_ = 0); ~TaskTracker(); static ThreadPoolCallbackRunner syncRunner(); bool isAsync() const; - void waitReady(); + size_t consumeReady(); + void waitAny(); void waitAll(); void safeWaitAll(); void add(Callback && func); private: - bool is_async; + void waitInFlight(); + + const bool is_async; ThreadPoolCallbackRunner scheduler; - std::list> futures; + const size_t max_tasks_inflight; + + using FutureList = std::list>; + FutureList futures; Poco::Logger * log = &Poco::Logger::get("TaskTracker"); + + std::mutex mutex; + std::condition_variable cond_var; + using FinishedList = std::list; + FinishedList finished_futures; }; } diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 4db44b7b4f7..85d38d7a9b2 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -37,6 +37,7 @@ S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings( max_upload_part_size = config.getUInt64(key + "max_upload_part_size", max_upload_part_size); upload_part_size_multiply_factor = config.getUInt64(key + "upload_part_size_multiply_factor", upload_part_size_multiply_factor); upload_part_size_multiply_parts_count_threshold = config.getUInt64(key + "upload_part_size_multiply_parts_count_threshold", upload_part_size_multiply_parts_count_threshold); + s3_max_inflight_parts_for_one_file = config.getUInt64(key + "s3_max_inflight_parts_for_one_file", s3_max_inflight_parts_for_one_file); max_part_number = config.getUInt64(key + "max_part_number", max_part_number); max_single_part_upload_size = config.getUInt64(key + "max_single_part_upload_size", max_single_part_upload_size); max_single_operation_copy_size = config.getUInt64(key + "max_single_operation_copy_size", max_single_operation_copy_size); @@ -55,6 +56,7 @@ S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const NamedC max_single_part_upload_size = collection.getOrDefault("max_single_part_upload_size", max_single_part_upload_size); upload_part_size_multiply_factor = collection.getOrDefault("upload_part_size_multiply_factor", upload_part_size_multiply_factor); upload_part_size_multiply_parts_count_threshold = collection.getOrDefault("upload_part_size_multiply_parts_count_threshold", upload_part_size_multiply_parts_count_threshold); + s3_max_inflight_parts_for_one_file = collection.getOrDefault("s3_max_inflight_parts_for_one_file", s3_max_inflight_parts_for_one_file); /// This configuration is only applicable to s3. Other types of object storage are not applicable or have different meanings. storage_class_name = collection.getOrDefault("s3_storage_class", storage_class_name); @@ -80,6 +82,9 @@ void S3Settings::RequestSettings::PartUploadSettings::updateFromSettingsImpl(con if (!if_changed || settings.s3_upload_part_size_multiply_parts_count_threshold.changed) upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; + if (!if_changed || settings.s3_max_inflight_parts_for_one_file.changed) + s3_max_inflight_parts_for_one_file = settings.s3_max_inflight_parts_for_one_file; + if (!if_changed || settings.s3_max_single_part_upload_size.changed) max_single_part_upload_size = settings.s3_max_single_part_upload_size; } diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index cd5be1626b5..f6ed9db4ee0 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -33,6 +33,7 @@ struct S3Settings size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; size_t upload_part_size_multiply_factor = 2; size_t upload_part_size_multiply_parts_count_threshold = 500; + size_t s3_max_inflight_parts_for_one_file = 20; size_t max_part_number = 10000; size_t max_single_part_upload_size = 32 * 1024 * 1024; size_t max_single_operation_copy_size = 5ULL * 1024 * 1024 * 1024; diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index 68b9b6bf1bf..ab05dbd205b 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -29,6 +29,8 @@ minio123 1 20000 + 30000 + true local diff --git a/tests/integration/test_merge_tree_s3/configs/config.xml b/tests/integration/test_merge_tree_s3/configs/config.xml index 8055a5dd0a5..db5e28bce95 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.xml @@ -1,3 +1,14 @@ + + + 1 + 1 + 60 + 60 + 60 + 60 + + + true diff --git a/tests/integration/test_merge_tree_s3/s3_mocks/broken_s3.py b/tests/integration/test_merge_tree_s3/s3_mocks/broken_s3.py index e6def138371..3adf3ba0047 100644 --- a/tests/integration/test_merge_tree_s3/s3_mocks/broken_s3.py +++ b/tests/integration/test_merge_tree_s3/s3_mocks/broken_s3.py @@ -1,4 +1,8 @@ +import logging import sys +import threading +import random +import time import urllib.parse import http.server import socketserver @@ -9,16 +13,66 @@ UPSTREAM_PORT = 9001 class ServerRuntime: + class SlowPut: + def __init__( + self, probability_=None, timeout_=None, minimal_length_=None, count_=None + ): + self.probability = probability_ if probability_ is not None else 1 + self.timeout = timeout_ if timeout_ is not None else 0.1 + self.minimal_length = minimal_length_ if minimal_length_ is not None else 0 + self.count = count_ if count_ is not None else 2**32 + + def __str__(self): + return ( + f"probability:{self.probability}" + f" timeout:{self.timeout}" + f" minimal_length:{self.minimal_length}" + f" count:{self.count}" + ) + + def get_timeout(self, content_length): + if content_length > self.minimal_length: + if self.count > 0: + if ( + runtime.slow_put.probability == 1 + or random.random() <= runtime.slow_put.probability + ): + self.count -= 1 + return runtime.slow_put.timeout + return None + def __init__(self): + self.lock = threading.Lock() self.error_at_put_when_length_bigger = None + self.fake_put_when_length_bigger = None + self.fake_uploads = dict() + self.slow_put = None + + def register_fake_upload(self, upload_id, key): + with self.lock: + self.fake_uploads[upload_id] = key + + def is_fake_upload(self, upload_id, key): + with self.lock: + if upload_id in self.fake_uploads: + return self.fake_uploads[upload_id] == key + return False def reset(self): self.error_at_put_when_length_bigger = None + self.fake_put_when_length_bigger = None + self.fake_uploads = dict() + self.slow_put = None runtime = ServerRuntime() +def and_then(value, func): + assert callable(func) + return None if value is None else func(value) + + class RequestHandler(http.server.BaseHTTPRequestHandler): def _ok(self): self.send_response(200) @@ -55,51 +109,124 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.send_response(500) self.send_header("Content-Type", "text/xml") self.end_headers() - self.wfile.write(data) + self.wfile.write(bytes(data, "UTF-8")) + + def _fake_put_ok(self): + self._read_out() + + self.send_response(200) + self.send_header("Content-Type", "text/xml") + self.send_header("ETag", "b54357faf0632cce46e942fa68356b38") + self.send_header("Content-Length", 0) + self.end_headers() + + def _fake_post_ok(self, path): + self._read_out() + + parts = [x for x in path.split("/") if x] + bucket = parts[0] + key = "/".join(parts[1:]) + location = "http://Example-Bucket.s3.Region.amazonaws.com/" + path + data = ( + '\n' + "\n" + f"{location}\n" + f"{bucket}\n" + f"{key}\n" + f'"3858f62230ac3c915f300c664312c11f-9"\n' + f"\n" + ) + + self.send_response(200) + self.send_header("Content-Type", "text/xml") + self.send_header("Content-Length", len(data)) + self.end_headers() + + self.wfile.write(bytes(data, "UTF-8")) def _mock_settings(self): parts = urllib.parse.urlsplit(self.path) path = [x for x in parts.path.split("/") if x] assert path[0] == "mock_settings", path + if len(path) < 2: + return self._error("_mock_settings: wrong command") + if path[1] == "error_at_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) runtime.error_at_put_when_length_bigger = int( params.get("when_length_bigger", [1024 * 1024])[0] ) - self._ok() - elif path[1] == "reset": + return self._ok() + if path[1] == "fake_put": + params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) + runtime.fake_put_when_length_bigger = int( + params.get("when_length_bigger", [1024 * 1024])[0] + ) + return self._ok() + if path[1] == "slow_put": + params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) + runtime.slow_put = ServerRuntime.SlowPut( + minimal_length_=and_then(params.get("minimal_length", [None])[0], int), + probability_=and_then(params.get("probability", [None])[0], float), + timeout_=and_then(params.get("timeout", [None])[0], float), + count_=and_then(params.get("count", [None])[0], int), + ) + self.log_message("set slow put %s", runtime.slow_put) + return self._ok() + if path[1] == "reset": runtime.reset() - self._ok() - else: - self._error("_mock_settings: wrong command") + return self._ok() + + return self._error("_mock_settings: wrong command") def do_GET(self): if self.path == "/": - self._ping() - elif self.path.startswith("/mock_settings"): - self._mock_settings() - else: - self._redirect() + return self._ping() + + if self.path.startswith("/mock_settings"): + return self._mock_settings() + + return self._redirect() def do_PUT(self): + content_length = int(self.headers.get("Content-Length", 0)) + + if runtime.slow_put is not None: + timeout = runtime.slow_put.get_timeout(content_length) + if timeout is not None: + self.log_message("slow put %s", timeout) + time.sleep(timeout) + if runtime.error_at_put_when_length_bigger is not None: - content_length = int(self.headers.get("Content-Length", 0)) if content_length > runtime.error_at_put_when_length_bigger: - self._error( - b'' - b"" - b"ExpectedError" - b"mock s3 injected error" - b"txfbd566d03042474888193-00608d7537" - b"" + return self._error( + '' + "" + "ExpectedError" + "mock s3 injected error" + "txfbd566d03042474888193-00608d7537" + "" ) - else: - self._redirect() - else: - self._redirect() + + parts = urllib.parse.urlsplit(self.path) + params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) + upload_id = params.get("uploadId", [None])[0] + if runtime.fake_put_when_length_bigger is not None and upload_id is not None: + if content_length > runtime.fake_put_when_length_bigger: + runtime.register_fake_upload(upload_id, parts.path) + return self._fake_put_ok() + + return self._redirect() def do_POST(self): - self._redirect() + parts = urllib.parse.urlsplit(self.path) + params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) + upload_id = params.get("uploadId", [None])[0] + + if runtime.is_fake_upload(upload_id, parts.path): + return self._fake_post_ok(parts.path) + + return self._redirect() def do_HEAD(self): self._redirect() diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index ee774f6632b..f98b2594e01 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -961,3 +961,110 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, node_name): check_no_objects_after_drop( cluster, table_name="merge_canceled_by_s3_errors_when_move", node_name=node_name ) + + +def value_or(value, default): + assert default is not None + return value if value is not None else default + + +class BrokenS3: + @staticmethod + def reset(cluster): + response = cluster.exec_in_container( + cluster.get_container_id("resolver"), + [ + "curl", + "-s", + f"http://localhost:8083/mock_settings/reset", + ], + nothrow=True, + ) + assert response == "OK" + + @staticmethod + def setup_fake_upload(cluster, part_length): + response = cluster.exec_in_container( + cluster.get_container_id("resolver"), + [ + "curl", + "-s", + f"http://localhost:8083/mock_settings/fake_put?when_length_bigger={part_length}", + ], + nothrow=True, + ) + assert response == "OK" + + @staticmethod + def setup_slow_answers( + cluster, minimal_length=0, timeout=None, probability=None, count=None + ): + url = f"http://localhost:8083/" \ + f"mock_settings/slow_put" \ + f"?minimal_length={minimal_length}" + + if timeout is not None: + url += f"&timeout={timeout}" + + if probability is not None: + url += f"&probability={probability}" + + if count is not None: + url += f"&count={count}" + + response = cluster.exec_in_container( + cluster.get_container_id("resolver"), + ["curl", "-s", url], + nothrow=True, + ) + assert response == "OK" + + +@pytest.fixture(autouse=True, scope="function") +def reset_broken_s3(cluster): + BrokenS3.reset(cluster) + + yield + + +@pytest.mark.parametrize("node_name", ["node"]) +@pytest.mark.parametrize( + "in_flight_memory", [(10, 245918115), (5, 156786752), (1, 106426187)] +) +def test_heavy_write_check_mem(cluster, node_name, in_flight_memory): + in_flight = in_flight_memory[0] + memory = in_flight_memory[1] + + node = cluster.instances[node_name] + node.query("DROP TABLE IF EXISTS s3_test SYNC") + node.query( + "CREATE TABLE s3_test" + " (" + " key UInt32 CODEC(NONE), value String CODEC(NONE)" + " )" + " ENGINE S3('http://resolver:8083/root/data/test-upload.csv', 'minio', 'minio123', 'CSV')", + ) + + BrokenS3.setup_fake_upload(cluster, 1000) + BrokenS3.setup_slow_answers(cluster, 10 * 1024 * 1024, timeout=10, count=10) + + query_id = f"INSERT_INTO_S3_QUERY_ID_{in_flight}" + node.query( + "INSERT INTO s3_test SELECT number, toString(number) FROM numbers(50000000)" + f" SETTINGS max_memory_usage={2*memory}, s3_max_inflight_parts_for_one_file={in_flight}", + query_id=query_id, + ) + + node.query("SYSTEM FLUSH LOGS") + + result = node.query( + "SELECT memory_usage" + " FROM system.query_log" + f" WHERE query_id='{query_id}'" + " AND type!='QueryStart'" + ) + + assert int(result) < 1.1 * memory + assert int(result) > 0.9 * memory + + check_no_objects_after_drop(cluster, node_name=node_name) From 206efee9b744c46d0a78ae8ecc592efb6606fd0e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 21 May 2023 12:42:28 +0200 Subject: [PATCH 0725/2223] add test with merge tree table --- src/Core/Settings.h | 1 + src/Disks/ObjectStorages/S3/diskSettings.cpp | 3 +- src/IO/WriteBufferFromS3.cpp | 2 +- src/IO/WriteBufferFromS3TaskTracker.cpp | 2 +- src/Storages/StorageS3.cpp | 15 +- src/Storages/StorageS3Settings.cpp | 13 +- src/Storages/StorageS3Settings.h | 4 +- .../configs/config.d/storage_conf.xml | 6 +- .../configs/config.d/users.xml | 8 + .../test_merge_tree_s3/configs/config.xml | 17 +- tests/integration/test_merge_tree_s3/test.py | 224 ++++++++++-------- .../configs/config.d/storage_conf.xml | 2 +- 12 files changed, 173 insertions(+), 124 deletions(-) create mode 100644 tests/integration/test_merge_tree_s3/configs/config.d/users.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6f8132ead91..30f3394bcd8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -94,6 +94,7 @@ class IColumn; M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(UInt64, s3_retry_attempts, 10, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 2aa0a22e4f5..409eb2a3dc3 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -146,7 +146,8 @@ std::unique_ptr getClient( S3::ServerSideEncryptionKMSConfig sse_kms_config = S3::getSSEKMSConfig(config_prefix, config); client_configuration.retryStrategy - = std::make_shared(config.getUInt(config_prefix + ".retry_attempts", 10)); + = std::make_shared( + config.getUInt64(config_prefix + ".retry_attempts", settings.request_settings.retry_attempts)); return S3::ClientFactory::instance().create( client_configuration, diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 680292b16a4..7f8e718afca 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -96,7 +96,7 @@ WriteBufferFromS3::WriteBufferFromS3( , task_tracker( std::make_unique( std::move(schedule_), - upload_settings.s3_max_inflight_parts_for_one_file)) + upload_settings.max_inflight_parts_for_one_file)) { LOG_TRACE(log, "Create WriteBufferFromS3, {}", getLogDetails()); diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 091d429c45d..d114c0febbb 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -153,7 +153,7 @@ void WriteBufferFromS3::TaskTracker::add(Callback && func) *future_placeholder = scheduler(std::move(func_with_notification), Priority{}); - LOG_TEST(log, "add ended, in queue {}", futures.size()); + LOG_TEST(log, "add ended, in queue {}, limit {}", futures.size(), max_tasks_inflight); waitInFlight(); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 80a49b2cd1d..2a2192d9cfe 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1263,6 +1263,11 @@ void StorageS3::Configuration::connect(ContextPtr context) if (!headers_from_ast.empty()) headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end()); + client_configuration.requestTimeoutMs = request_settings.request_timeout_ms; + + client_configuration.retryStrategy + = std::make_shared(request_settings.retry_attempts); + auto credentials = Aws::Auth::AWSCredentials(auth_settings.access_key_id, auth_settings.secret_access_key); client = S3::ClientFactory::instance().create( client_configuration, @@ -1273,11 +1278,11 @@ void StorageS3::Configuration::connect(ContextPtr context) auth_settings.server_side_encryption_kms_config, std::move(headers), S3::CredentialsConfiguration{ - auth_settings.use_environment_credentials.value_or(context->getConfigRef().getBool("s3.use_environment_credentials", true)), - auth_settings.use_insecure_imds_request.value_or(context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), - auth_settings.expiration_window_seconds.value_or( - context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), - auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), + auth_settings.use_environment_credentials.value_or(context->getConfigRef().getBool("s3.use_environment_credentials", true)), + auth_settings.use_insecure_imds_request.value_or(context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), + auth_settings.expiration_window_seconds.value_or( + context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), + auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), }); } diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 85d38d7a9b2..23b4630707c 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -37,7 +37,7 @@ S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings( max_upload_part_size = config.getUInt64(key + "max_upload_part_size", max_upload_part_size); upload_part_size_multiply_factor = config.getUInt64(key + "upload_part_size_multiply_factor", upload_part_size_multiply_factor); upload_part_size_multiply_parts_count_threshold = config.getUInt64(key + "upload_part_size_multiply_parts_count_threshold", upload_part_size_multiply_parts_count_threshold); - s3_max_inflight_parts_for_one_file = config.getUInt64(key + "s3_max_inflight_parts_for_one_file", s3_max_inflight_parts_for_one_file); + max_inflight_parts_for_one_file = config.getUInt64(key + "max_inflight_parts_for_one_file", max_inflight_parts_for_one_file); max_part_number = config.getUInt64(key + "max_part_number", max_part_number); max_single_part_upload_size = config.getUInt64(key + "max_single_part_upload_size", max_single_part_upload_size); max_single_operation_copy_size = config.getUInt64(key + "max_single_operation_copy_size", max_single_operation_copy_size); @@ -56,7 +56,7 @@ S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const NamedC max_single_part_upload_size = collection.getOrDefault("max_single_part_upload_size", max_single_part_upload_size); upload_part_size_multiply_factor = collection.getOrDefault("upload_part_size_multiply_factor", upload_part_size_multiply_factor); upload_part_size_multiply_parts_count_threshold = collection.getOrDefault("upload_part_size_multiply_parts_count_threshold", upload_part_size_multiply_parts_count_threshold); - s3_max_inflight_parts_for_one_file = collection.getOrDefault("s3_max_inflight_parts_for_one_file", s3_max_inflight_parts_for_one_file); + max_inflight_parts_for_one_file = collection.getOrDefault("max_inflight_parts_for_one_file", max_inflight_parts_for_one_file); /// This configuration is only applicable to s3. Other types of object storage are not applicable or have different meanings. storage_class_name = collection.getOrDefault("s3_storage_class", storage_class_name); @@ -83,7 +83,7 @@ void S3Settings::RequestSettings::PartUploadSettings::updateFromSettingsImpl(con upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; if (!if_changed || settings.s3_max_inflight_parts_for_one_file.changed) - s3_max_inflight_parts_for_one_file = settings.s3_max_inflight_parts_for_one_file; + max_inflight_parts_for_one_file = settings.s3_max_inflight_parts_for_one_file; if (!if_changed || settings.s3_max_single_part_upload_size.changed) max_single_part_upload_size = settings.s3_max_single_part_upload_size; @@ -198,6 +198,8 @@ S3Settings::RequestSettings::RequestSettings( check_objects_after_upload = config.getBool(key + "check_objects_after_upload", settings.s3_check_objects_after_upload); list_object_keys_size = config.getUInt64(key + "list_object_keys_size", settings.s3_list_object_keys_size); throw_on_zero_files_match = config.getBool(key + "throw_on_zero_files_match", settings.s3_throw_on_zero_files_match); + retry_attempts = config.getUInt64(key + "retry_attempts", settings.s3_retry_attempts); + request_timeout_ms = config.getUInt64(key + "request_timeout_ms", request_timeout_ms); /// NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, /// which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. @@ -248,8 +250,11 @@ void S3Settings::RequestSettings::updateFromSettingsImpl(const Settings & settin put_request_throttler = std::make_shared( settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); - if (!if_changed || settings.s3_throw_on_zero_files_match) + if (!if_changed || settings.s3_throw_on_zero_files_match.changed) throw_on_zero_files_match = settings.s3_throw_on_zero_files_match; + + if (!if_changed || settings.s3_retry_attempts.changed) + retry_attempts = settings.s3_retry_attempts; } void S3Settings::RequestSettings::updateFromSettings(const Settings & settings) diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index f6ed9db4ee0..41489927e7f 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -33,7 +33,7 @@ struct S3Settings size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; size_t upload_part_size_multiply_factor = 2; size_t upload_part_size_multiply_parts_count_threshold = 500; - size_t s3_max_inflight_parts_for_one_file = 20; + size_t max_inflight_parts_for_one_file = 20; size_t max_part_number = 10000; size_t max_single_part_upload_size = 32 * 1024 * 1024; size_t max_single_operation_copy_size = 5ULL * 1024 * 1024 * 1024; @@ -68,6 +68,8 @@ struct S3Settings size_t list_object_keys_size = 1000; ThrottlerPtr get_request_throttler; ThrottlerPtr put_request_throttler; + size_t retry_attempts = 10; + size_t request_timeout_ms = 30000; bool throw_on_zero_files_match = false; diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index ab05dbd205b..b02136b92eb 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -27,10 +27,10 @@ http://resolver:8083/root/data/ minio minio123 - 1 + 0 20000 - 30000 - true + 20000 + 1 local diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/users.xml b/tests/integration/test_merge_tree_s3/configs/config.d/users.xml new file mode 100644 index 00000000000..3daa6f06a78 --- /dev/null +++ b/tests/integration/test_merge_tree_s3/configs/config.d/users.xml @@ -0,0 +1,8 @@ + + + + 1 + 20 + + + diff --git a/tests/integration/test_merge_tree_s3/configs/config.xml b/tests/integration/test_merge_tree_s3/configs/config.xml index db5e28bce95..314f23f5788 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.xml @@ -1,14 +1,11 @@ - - - 1 - 1 - 60 - 60 - 60 - 60 - - + + + http://resolver:8083/root/data/ + 0 + 20000 + + true diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index f98b2594e01..43c924abb42 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -26,6 +26,9 @@ def cluster(): "configs/config.d/storage_conf.xml", "configs/config.d/bg_processing_pool_conf.xml", ], + user_configs=[ + "configs/config.d/users.xml", + ], stay_alive=True, with_minio=True, ) @@ -139,14 +142,76 @@ def clear_minio(cluster): yield +class BrokenS3: + @staticmethod + def reset(cluster): + response = cluster.exec_in_container( + cluster.get_container_id("resolver"), + [ + "curl", + "-s", + f"http://localhost:8083/mock_settings/reset", + ], + nothrow=True, + ) + assert response == "OK" + + @staticmethod + def setup_fail_upload(cluster, part_length): + response = cluster.exec_in_container( + cluster.get_container_id("resolver"), + [ + "curl", + "-s", + f"http://localhost:8083/mock_settings/error_at_put?when_length_bigger={part_length}", + ], + nothrow=True, + ) + assert response == "OK" + + @staticmethod + def setup_fake_upload(cluster, part_length): + response = cluster.exec_in_container( + cluster.get_container_id("resolver"), + [ + "curl", + "-s", + f"http://localhost:8083/mock_settings/fake_put?when_length_bigger={part_length}", + ], + nothrow=True, + ) + assert response == "OK" + + @staticmethod + def setup_slow_answers( + cluster, minimal_length=0, timeout=None, probability=None, count=None + ): + url = ( + f"http://localhost:8083/" + f"mock_settings/slow_put" + f"?minimal_length={minimal_length}" + ) + + if timeout is not None: + url += f"&timeout={timeout}" + + if probability is not None: + url += f"&probability={probability}" + + if count is not None: + url += f"&count={count}" + + response = cluster.exec_in_container( + cluster.get_container_id("resolver"), + ["curl", "-s", url], + nothrow=True, + ) + assert response == "OK" + + @pytest.fixture(autouse=True, scope="function") -def reset_mock_broken_s3(cluster): - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - ["curl", "-s", f"http://localhost:8083/mock_settings/reset"], - nothrow=True, - ) - assert response == "OK" +def reset_broken_s3(cluster): + BrokenS3.reset(cluster) yield @@ -886,16 +951,7 @@ def test_merge_canceled_by_s3_errors(cluster, node_name): min_key = node.query("SELECT min(key) FROM test_merge_canceled_by_s3_errors") assert int(min_key) == 0, min_key - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - [ - "curl", - "-s", - f"http://localhost:8083/mock_settings/error_at_put?when_length_bigger=50000", - ], - nothrow=True, - ) - assert response == "OK" + BrokenS3.setup_fail_upload(cluster, 50000) node.query("SYSTEM START MERGES test_merge_canceled_by_s3_errors") @@ -938,16 +994,7 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, node_name): settings={"materialize_ttl_after_modify": 0}, ) - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - [ - "curl", - "-s", - f"http://localhost:8083/mock_settings/error_at_put?when_length_bigger=10000", - ], - nothrow=True, - ) - assert response == "OK" + BrokenS3.setup_fail_upload(cluster, 10000) node.query("SYSTEM START MERGES merge_canceled_by_s3_errors_when_move") @@ -963,75 +1010,11 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, node_name): ) -def value_or(value, default): - assert default is not None - return value if value is not None else default - - -class BrokenS3: - @staticmethod - def reset(cluster): - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - [ - "curl", - "-s", - f"http://localhost:8083/mock_settings/reset", - ], - nothrow=True, - ) - assert response == "OK" - - @staticmethod - def setup_fake_upload(cluster, part_length): - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - [ - "curl", - "-s", - f"http://localhost:8083/mock_settings/fake_put?when_length_bigger={part_length}", - ], - nothrow=True, - ) - assert response == "OK" - - @staticmethod - def setup_slow_answers( - cluster, minimal_length=0, timeout=None, probability=None, count=None - ): - url = f"http://localhost:8083/" \ - f"mock_settings/slow_put" \ - f"?minimal_length={minimal_length}" - - if timeout is not None: - url += f"&timeout={timeout}" - - if probability is not None: - url += f"&probability={probability}" - - if count is not None: - url += f"&count={count}" - - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - ["curl", "-s", url], - nothrow=True, - ) - assert response == "OK" - - -@pytest.fixture(autouse=True, scope="function") -def reset_broken_s3(cluster): - BrokenS3.reset(cluster) - - yield - - @pytest.mark.parametrize("node_name", ["node"]) @pytest.mark.parametrize( "in_flight_memory", [(10, 245918115), (5, 156786752), (1, 106426187)] ) -def test_heavy_write_check_mem(cluster, node_name, in_flight_memory): +def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): in_flight = in_flight_memory[0] memory = in_flight_memory[1] @@ -1046,12 +1029,13 @@ def test_heavy_write_check_mem(cluster, node_name, in_flight_memory): ) BrokenS3.setup_fake_upload(cluster, 1000) - BrokenS3.setup_slow_answers(cluster, 10 * 1024 * 1024, timeout=10, count=10) + BrokenS3.setup_slow_answers(cluster, 10 * 1024 * 1024, timeout=15, count=10) - query_id = f"INSERT_INTO_S3_QUERY_ID_{in_flight}" + query_id = f"INSERT_INTO_S3_ENGINE_QUERY_ID_{in_flight}" node.query( "INSERT INTO s3_test SELECT number, toString(number) FROM numbers(50000000)" - f" SETTINGS max_memory_usage={2*memory}, s3_max_inflight_parts_for_one_file={in_flight}", + f" SETTINGS max_memory_usage={2*memory}" + f", s3_max_inflight_parts_for_one_file={in_flight}", query_id=query_id, ) @@ -1064,7 +1048,53 @@ def test_heavy_write_check_mem(cluster, node_name, in_flight_memory): " AND type!='QueryStart'" ) - assert int(result) < 1.1 * memory - assert int(result) > 0.9 * memory + assert int(result) < 1.01 * memory + assert int(result) > 0.99 * memory + + check_no_objects_after_drop(cluster, node_name=node_name) + + +@pytest.mark.parametrize("node_name", ["node"]) +def test_s3_disk_heavy_write_check_mem(cluster, node_name): + memory = 2279055040 + + node = cluster.instances[node_name] + node.query("DROP TABLE IF EXISTS s3_test SYNC") + node.query( + "CREATE TABLE s3_test" + " (" + " key UInt32, value String" + " )" + " ENGINE=MergeTree()" + " ORDER BY key" + " SETTINGS" + " storage_policy='broken_s3'", + ) + node.query("SYSTEM STOP MERGES s3_test") + + BrokenS3.setup_fake_upload(cluster, 1000) + BrokenS3.setup_slow_answers(cluster, 10 * 1024 * 1024, timeout=10, count=50) + + query_id = f"INSERT_INTO_S3_DISK_QUERY_ID" + node.query( + "INSERT INTO s3_test SELECT number, toString(number) FROM numbers(50000000)" + f" SETTINGS max_memory_usage={2*memory}" + f", max_insert_block_size=50000000" + f", min_insert_block_size_rows=50000000" + f", min_insert_block_size_bytes=1000000000000", + query_id=query_id, + ) + + node.query("SYSTEM FLUSH LOGS") + + result = node.query( + "SELECT memory_usage" + " FROM system.query_log" + f" WHERE query_id='{query_id}'" + " AND type!='QueryStart'" + ) + + assert int(result) < 1.01 * memory + assert int(result) > 0.99 * memory check_no_objects_after_drop(cluster, node_name=node_name) diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml index 74af657c783..4480327c4b5 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml @@ -32,7 +32,7 @@ true - 1 + 1 1 20000 From 0b781bd4f177bf8533f60c52bbd2af1323966a4c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 21 May 2023 23:45:10 +0200 Subject: [PATCH 0726/2223] fix test_merge_canceled_by_s3_errors --- src/IO/WriteBufferFromS3.cpp | 5 +++-- src/IO/WriteBufferFromS3TaskTracker.cpp | 2 ++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 7f8e718afca..87453e1d323 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -137,6 +137,9 @@ void WriteBufferFromS3::preFinalize() LOG_TRACE(log, "preFinalize WriteBufferFromS3. {}", getLogDetails()); + /// This function should not be run again if an exception has occurred + is_prefinalized = true; + hidePartialData(); if (hidden_size > 0) @@ -167,8 +170,6 @@ void WriteBufferFromS3::preFinalize() { writeMultipartUpload(); } - - is_prefinalized = true; } void WriteBufferFromS3::finalizeImpl() diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index d114c0febbb..c330ac295d5 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -46,6 +46,8 @@ size_t WriteBufferFromS3::TaskTracker::consumeReady() } catch (...) { + futures.erase(it); + tryLogCurrentException(__PRETTY_FUNCTION__); throw; } From 7031796d3f1beefcf0332e5b021b70ef9d590a72 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 22 May 2023 12:50:12 +0200 Subject: [PATCH 0727/2223] fixing tests --- src/Disks/IDisk.cpp | 2 +- .../test_merge_tree_s3/configs/config.d/storage_conf.xml | 1 + tests/integration/test_merge_tree_s3/test.py | 8 ++++---- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 88dd65bfde7..bca867fec76 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -188,12 +188,12 @@ try try { file->write(payload.data(), payload.size()); + file->finalize(); } catch (...) { /// Log current exception, because finalize() can throw a different exception. tryLogCurrentException(__PRETTY_FUNCTION__); - file->finalize(); throw; } } diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index b02136b92eb..cca80143548 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -27,6 +27,7 @@ http://resolver:8083/root/data/ minio minio123 + true 0 20000 20000 diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 43c924abb42..05bed86c814 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -1048,8 +1048,8 @@ def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): " AND type!='QueryStart'" ) - assert int(result) < 1.01 * memory - assert int(result) > 0.99 * memory + assert int(result) < 1.1 * memory + assert int(result) > 0.9 * memory check_no_objects_after_drop(cluster, node_name=node_name) @@ -1094,7 +1094,7 @@ def test_s3_disk_heavy_write_check_mem(cluster, node_name): " AND type!='QueryStart'" ) - assert int(result) < 1.01 * memory - assert int(result) > 0.99 * memory + assert int(result) < 1.1 * memory + assert int(result) > 0.9 * memory check_no_objects_after_drop(cluster, node_name=node_name) From b0eff95388e0285cee3d3bb6248b9a14c6b5f21c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 24 May 2023 13:43:48 +0200 Subject: [PATCH 0728/2223] update doc, add profile event WriteBufferFromS3WaitInflightLimitMicroseconds --- docs/en/engines/table-engines/integrations/s3.md | 1 + .../table-engines/mergetree-family/mergetree.md | 3 ++- src/Common/ProfileEvents.cpp | 2 +- src/IO/WriteBufferFromS3TaskTracker.cpp | 14 ++++++++++++-- tests/integration/test_merge_tree_s3/test.py | 12 +++++++----- 5 files changed, 23 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index cf887a498ea..a2ea4560109 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -139,6 +139,7 @@ The following settings can be set before query execution or placed into configur - `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`. - `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). - `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. +- `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurenly for one object. The value `0` means unlimited. Default value is `20`. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 70e57acd548..79ced0b6ce5 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1219,11 +1219,12 @@ Authentication parameters (the disk will try all available methods **and** Manag * `account_name` and `account_key` - For authentication using Shared Key. Limit parameters (mainly for internal usage): -* `max_single_part_upload_size` - Limits the size of a single block upload to Blob Storage. +* `s3_max_single_part_upload_size` - Limits the size of a single block upload to Blob Storage. * `min_bytes_for_seek` - Limits the size of a seekable region. * `max_single_read_retries` - Limits the number of attempts to read a chunk of data from Blob Storage. * `max_single_download_retries` - Limits the number of attempts to download a readable buffer from Blob Storage. * `thread_pool_size` - Limits the number of threads with which `IDiskRemote` is instantiated. +* `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurenly for one object. Other parameters: * `metadata_path` - Path on local FS to store metadata files for Blob Storage. Default value is `/var/lib/clickhouse/disks//`. diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index dd96b724793..e3db6307ea2 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -366,7 +366,7 @@ The server successfully detected this situation and will download merged part fr M(WriteBufferFromS3Microseconds, "Time spent on writing to S3.") \ M(WriteBufferFromS3Bytes, "Bytes written to S3.") \ M(WriteBufferFromS3RequestsErrors, "Number of exceptions while writing to S3.") \ - \ + M(WriteBufferFromS3WaitInflightLimitMicroseconds, "Time spent on waiting while some of the current requests are done when its number reached the limit defined by s3_max_inflight_parts_for_one_file.") \ M(QueryMemoryLimitExceeded, "Number of times when memory limit exceeded for query.") \ \ M(CachedReadBufferReadFromSourceMicroseconds, "Time reading from filesystem cache source (from remote filesystem, etc)") \ diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index c330ac295d5..414844bfbae 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -4,6 +4,11 @@ #include +namespace ProfileEvents +{ + extern const Event WriteBufferFromS3WaitInflightLimitMicroseconds; +} + namespace DB { @@ -125,10 +130,10 @@ void WriteBufferFromS3::TaskTracker::waitAny() { LOG_TEST(log, "waitAny, in queue {}", futures.size()); - while (futures.size() > 0 && consumeReady() == 0) + while (!futures.empty() && consumeReady() == 0) { std::unique_lock lock(mutex); - cond_var.wait(lock, [&] () { return finished_futures.size() > 0; }); + cond_var.wait(lock, [&] () { return !finished_futures.empty(); }); } LOG_TEST(log, "waitAny ended, in queue {}", futures.size()); @@ -167,11 +172,16 @@ void WriteBufferFromS3::TaskTracker::waitInFlight() LOG_TEST(log, "waitInFlight, in queue {}", futures.size()); + Stopwatch watch; + while (futures.size() >= max_tasks_inflight) { waitAny(); } + watch.stop(); + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); + LOG_TEST(log, "waitInFlight ended, in queue {}", futures.size()); } diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 05bed86c814..f87644a6876 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -1041,15 +1041,17 @@ def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): node.query("SYSTEM FLUSH LOGS") - result = node.query( - "SELECT memory_usage" + memory_usage, wait_inflight = node.query( + "SELECT memory_usage, ProfileEvents['WriteBufferFromS3WaitInflightLimitMicroseconds']" " FROM system.query_log" f" WHERE query_id='{query_id}'" " AND type!='QueryStart'" - ) + ).split() - assert int(result) < 1.1 * memory - assert int(result) > 0.9 * memory + assert int(memory_usage) < 1.1 * memory + assert int(memory_usage) > 0.9 * memory + + assert int(wait_inflight) > 10 * 1000 * 1000 check_no_objects_after_drop(cluster, node_name=node_name) From 7c9d2ab6d0a6cfc58015a17d09767fa6f421e252 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 26 May 2023 12:50:33 +0200 Subject: [PATCH 0729/2223] working with review comments --- .../engines/table-engines/integrations/s3.md | 8 +- src/Core/Settings.h | 2 +- src/IO/WriteBufferFromS3.cpp | 7 +- src/IO/WriteBufferFromS3TaskTracker.cpp | 132 ++++++++---------- src/IO/WriteBufferFromS3TaskTracker.h | 26 +++- 5 files changed, 91 insertions(+), 84 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index a2ea4560109..595bc0c344f 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -131,15 +131,17 @@ CREATE TABLE table_with_asterisk (name String, value UInt32) The following settings can be set before query execution or placed into configuration file. -- `s3_max_single_part_upload_size` — The maximum size of object to upload using singlepart upload to S3. Default value is `64Mb`. -- `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `512Mb`. +- `s3_max_single_part_upload_size` — The maximum size of object to upload using singlepart upload to S3. Default value is `32Mb`. +- `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `16Mb`. - `s3_max_redirects` — Max number of S3 redirects hops allowed. Default value is `10`. - `s3_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. - `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited). - `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`. - `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). - `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. -- `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurenly for one object. The value `0` means unlimited. Default value is `20`. +- `s3_upload_part_size_multiply_factor` - Multiply `s3_min_upload_part_size` by this factor each time `s3_multiply_parts_count_threshold` parts were uploaded from a single write to S3. Default values is `2`. +- `s3_upload_part_size_multiply_parts_count_threshold` - Each time this number of parts was uploaded to S3 `s3_min_upload_part_size multiplied` by `s3_upload_part_size_multiply_factor`. DEfault value us `500`. +- `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurenly for one object. Its number should be limited. The value `0` means unlimited. Default value is `20`. Each inflight part has a buffer with size `s3_min_upload_part_size` for the first `s3_upload_part_size_multiply_factor` parts and more when file is big enought, see `upload_part_size_multiply_factor`. With default settings one uploaded file consumes not more than `320Mb` for a file which is less than `8G`. The consumption is greater for a larger file. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 30f3394bcd8..607be1522db 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -78,7 +78,7 @@ class IColumn; M(UInt64, s3_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_upload_part_size_multiply_factor, 2, "Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.", 0) \ M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3 s3_min_upload_part_size multiplied by s3_upload_part_size_multiply_factor.", 0) \ - M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ + M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited. You ", 0) \ M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 87453e1d323..682cda860da 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -112,8 +112,11 @@ void WriteBufferFromS3::nextImpl() ErrorCodes::LOGICAL_ERROR, "Cannot write to prefinalized buffer for S3, the file could have been created with PutObjectRequest"); - /// Make sense to call to before adding new async task to check if there is an exception - task_tracker->consumeReady(); + /// Make sense to call waitIfAny before adding new async task to check if there is an exception + /// The faster the exception is propagated the lesser time is spent for cancellation + /// Despite the fact that `task_tracker->add()` collects tasks statuses and propagates their exceptions + /// that call is necessary for the case when the is no in-flight limitation and therefore `task_tracker->add()` doesn't wait anything + task_tracker->waitIfAny(); hidePartialData(); diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 414844bfbae..7ae31044012 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -34,40 +34,6 @@ ThreadPoolCallbackRunner WriteBufferFromS3::TaskTracker::syncRunner() }; } -size_t WriteBufferFromS3::TaskTracker::consumeReady() -{ - LOG_TEST(log, "consumeReady, in queue {}", futures.size()); - - size_t consumed = 0; - - { - std::unique_lock lock(mutex); - - for (auto it : finished_futures) - { - try - { - it->get(); - } - catch (...) - { - futures.erase(it); - - tryLogCurrentException(__PRETTY_FUNCTION__); - throw; - } - - futures.erase(it); - } - - consumed = finished_futures.size(); - finished_futures.clear(); - } - - LOG_TEST(log, "consumeReady ended, in queue {}", futures.size()); - return consumed; -} - void WriteBufferFromS3::TaskTracker::waitAll() { LOG_TEST(log, "waitAll, in queue {}", futures.size()); @@ -75,18 +41,11 @@ void WriteBufferFromS3::TaskTracker::waitAll() /// Exceptions are propagated for (auto & future : futures) { - try - { - future.get(); - } catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - throw; - } + future.get(); } futures.clear(); - /// no concurrent tasks, no mutex required + std::lock_guard lock(mutex); finished_futures.clear(); } @@ -94,55 +53,72 @@ void WriteBufferFromS3::TaskTracker::safeWaitAll() { LOG_TEST(log, "safeWaitAll, wait in queue {}", futures.size()); - /// Exceptions are not propagated - for (auto & future : futures) - { - LOG_TEST(log, "safeWaitAll, wait future"); - - if (future.valid()) - future.wait(); - } - - LOG_TEST(log, "safeWaitAll, get in queue {}", futures.size()); - for (auto & future : futures) { if (future.valid()) { try { + /// Exceptions are not propagated future.get(); } catch (...) { + /// But at least they are printed tryLogCurrentException(__PRETTY_FUNCTION__); } } } futures.clear(); - /// no concurrent tasks, no mutex required + std::lock_guard lock(mutex); finished_futures.clear(); - - LOG_TEST(log, "safeWaitAll ended, get in queue {}", futures.size()); } -void WriteBufferFromS3::TaskTracker::waitAny() +void WriteBufferFromS3::TaskTracker::waitIfAny() { - LOG_TEST(log, "waitAny, in queue {}", futures.size()); + LOG_TEST(log, "waitIfAny, in queue {}", futures.size()); + if (futures.empty()) + return; + + Stopwatch watch; - while (!futures.empty() && consumeReady() == 0) { - std::unique_lock lock(mutex); - cond_var.wait(lock, [&] () { return !finished_futures.empty(); }); + std::lock_guard lock(mutex); + for (auto & it : finished_futures) + { + /// actually that call might lock this thread until the future is set finally + /// however that won't lock us for long, the task is about to finish when the pointer appears in the `finished_futures` + it->get(); + + /// in case of exception in `it->get()` + /// it it not necessary to remove `it` from list `futures` + /// `TaskTracker` has to be destroyed after any exception occurs, for this `safeWaitAll` is called. + /// `safeWaitAll` handles invalid futures in the list `futures` + futures.erase(it); + } + finished_futures.clear(); } - LOG_TEST(log, "waitAny ended, in queue {}", futures.size()); + watch.stop(); + ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); + + LOG_TEST(log, "waitIfAny ended, in queue {}", futures.size()); } void WriteBufferFromS3::TaskTracker::add(Callback && func) { + /// All this fuzz is about 2 things. This is the most critical place of TaskTracker. + /// The first is not to fail insertion in the list `futures`. + /// In order to face it, the element is allocated at the end of the list `futures` in advance. + /// The second is not to fail the notification of the task. + /// In order to face it, the list element, which would be inserted to the list `finished_futures`, + /// is allocated in advance as an other list `pre_allocated_finished` with one element inside. + + /// preallocation for the first issue futures.emplace_back(); auto future_placeholder = std::prev(futures.end()); + + /// preallocation for the second issue FinishedList pre_allocated_finished {future_placeholder}; Callback func_with_notification = [&, func=std::move(func), pre_allocated_finished=std::move(pre_allocated_finished)] () mutable @@ -150,39 +126,53 @@ void WriteBufferFromS3::TaskTracker::add(Callback && func) SCOPE_EXIT({ DENY_ALLOCATIONS_IN_SCOPE; - std::unique_lock lock(mutex); - finished_futures.splice(finished_futures.end(), pre_allocated_finished, pre_allocated_finished.begin()); - cond_var.notify_one(); + std::lock_guard lock(mutex); + finished_futures.splice(finished_futures.end(), pre_allocated_finished); + has_finished.notify_one(); }); func(); }; + /// this move is nothrow *future_placeholder = scheduler(std::move(func_with_notification), Priority{}); LOG_TEST(log, "add ended, in queue {}, limit {}", futures.size(), max_tasks_inflight); - waitInFlight(); + waitTilInflightShrink(); } -void WriteBufferFromS3::TaskTracker::waitInFlight() +void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() { if (!max_tasks_inflight) return; - LOG_TEST(log, "waitInFlight, in queue {}", futures.size()); + LOG_TEST(log, "waitTilInflightShrink, in queue {}", futures.size()); Stopwatch watch; + /// Alternative approach is to wait until at least futures.size() - max_tasks_inflight element are finished + /// However the faster finished task is collected the faster CH checks if there is an exception + /// The faster an exception is propagated the lesser time is spent for cancellation while (futures.size() >= max_tasks_inflight) { - waitAny(); + std::unique_lock lock(mutex); + + has_finished.wait(lock, [this] () TSA_REQUIRES(mutex) { return !finished_futures.empty(); }); + + for (auto & it : finished_futures) + { + it->get(); + futures.erase(it); + } + + finished_futures.clear(); } watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); - LOG_TEST(log, "waitInFlight ended, in queue {}", futures.size()); + LOG_TEST(log, "waitTilInflightShrink ended, in queue {}", futures.size()); } bool WriteBufferFromS3::TaskTracker::isAsync() const diff --git a/src/IO/WriteBufferFromS3TaskTracker.h b/src/IO/WriteBufferFromS3TaskTracker.h index b6dc7a75368..ba741cd3e2b 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.h +++ b/src/IO/WriteBufferFromS3TaskTracker.h @@ -14,28 +14,40 @@ namespace DB /// That class is used only in WriteBufferFromS3 for now. /// Therefore it declared as a part of WriteBufferFromS3. /// TaskTracker takes a Callback which is run by scheduler in some external shared ThreadPool. -/// TaskTracker brings the methods waitReady, waitAll/safeWaitAll +/// TaskTracker brings the methods waitIfAny, waitAll/safeWaitAll /// to help with coordination of the running tasks. +/// Basic exception safety is provided. If exception occurred the object has to be destroyed. +/// No thread safety is provided. Use this object with no concurrency. + class WriteBufferFromS3::TaskTracker { public: using Callback = std::function; - explicit TaskTracker(ThreadPoolCallbackRunner scheduler_, size_t max_tasks_inflight_ = 0); + TaskTracker(ThreadPoolCallbackRunner scheduler_, size_t max_tasks_inflight_); ~TaskTracker(); static ThreadPoolCallbackRunner syncRunner(); bool isAsync() const; - size_t consumeReady(); - void waitAny(); + + /// waitIfAny collects statuses from already finished tasks + /// There could be no finished tasks yet, so waitIfAny do nothing useful in that case + /// the first exception is thrown if any task has failed + void waitIfAny(); + + /// Well, waitAll waits all the tasks until they finish and collects they statuses void waitAll(); + + /// safeWaitAll do the same as waitAll but do not rethrow the exceptions void safeWaitAll(); + void add(Callback && func); private: - void waitInFlight(); + /// waitTilInflightShrink waits til the number of in-flight tasks beyond the limit `max_tasks_inflight`. + void waitTilInflightShrink() TSA_NO_THREAD_SAFETY_ANALYSIS; const bool is_async; ThreadPoolCallbackRunner scheduler; @@ -46,9 +58,9 @@ private: Poco::Logger * log = &Poco::Logger::get("TaskTracker"); std::mutex mutex; - std::condition_variable cond_var; + std::condition_variable has_finished TSA_GUARDED_BY(mutex); using FinishedList = std::list; - FinishedList finished_futures; + FinishedList finished_futures TSA_GUARDED_BY(mutex); }; } From 8523cd54a0e5f04c1848f119be244b1c5c50dc3c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 26 May 2023 12:50:33 +0200 Subject: [PATCH 0730/2223] working with review comments --- src/IO/WriteBufferFromS3TaskTracker.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/WriteBufferFromS3TaskTracker.h b/src/IO/WriteBufferFromS3TaskTracker.h index ba741cd3e2b..c7dc562e78d 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.h +++ b/src/IO/WriteBufferFromS3TaskTracker.h @@ -36,11 +36,11 @@ public: /// There could be no finished tasks yet, so waitIfAny do nothing useful in that case /// the first exception is thrown if any task has failed void waitIfAny(); - - /// Well, waitAll waits all the tasks until they finish and collects they statuses + + /// Well, waitAll waits all the tasks until they finish and collects their statuses void waitAll(); - /// safeWaitAll do the same as waitAll but do not rethrow the exceptions + /// safeWaitAll does the same as waitAll but mutes the exceptions void safeWaitAll(); void add(Callback && func); From 1588219ffd803b174dcb4a1711e2df967e9d6203 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 26 May 2023 18:38:45 +0200 Subject: [PATCH 0731/2223] comments --- src/IO/WriteBufferFromS3.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 682cda860da..954c996d929 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -217,8 +217,8 @@ String WriteBufferFromS3::getLogDetails() const multipart_upload_details = fmt::format(", upload id {}, upload has finished {}" , multipart_upload_id, multipart_upload_finished); - return fmt::format("Details: bucket {}, key {}, total size {}, count {}, hidden_size {}, offset {}, with pool: {}, finalized {}{}", - bucket, key, total_size, count(), hidden_size, offset(), task_tracker->isAsync(), finalized, multipart_upload_details); + return fmt::format("Details: bucket {}, key {}, total size {}, count {}, hidden_size {}, offset {}, with pool: {}, prefinalized {}, finalized {}{}", + bucket, key, total_size, count(), hidden_size, offset(), task_tracker->isAsync(), is_prefinalized, finalized, multipart_upload_details); } void WriteBufferFromS3::tryToAbortMultipartUpload() From 6ed7a3b73feadcf5c3d72ed4a137fccd55e10fbb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 May 2023 19:25:33 +0000 Subject: [PATCH 0732/2223] Fixing more tests. --- src/Interpreters/ActionsVisitor.cpp | 6 +++- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 4 +++ src/Interpreters/PreparedSets.cpp | 7 +++- src/Interpreters/Set.cpp | 17 ++++++---- src/Interpreters/Set.h | 3 +- .../QueryPlan/ReadFromMergeTree.cpp | 27 ++++++++++++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 3 ++ src/Storages/MergeTree/KeyCondition.cpp | 18 +++++++++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 20 ++++------- .../MergeTree/MergeTreeDataSelectExecutor.h | 3 +- .../MergeTreeIndexConditionBloomFilter.cpp | 11 ++++++ src/Storages/MergeTree/PartitionPruner.cpp | 16 +++++++++ src/Storages/MergeTree/PartitionPruner.h | 1 + .../System/StorageSystemZooKeeper.cpp | 34 +++++++++++-------- 15 files changed, 127 insertions(+), 45 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 133fcbbfe87..23c6867a868 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1420,8 +1420,12 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } - if (auto tmp_table = data.getContext()->findExternalTable(table_id.getShortName())) + // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; + if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) + { external_table_set = tmp_table->future_set; + // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; + } } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 8d3fa91a7fe..dccf2978f49 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -110,7 +110,7 @@ TemporaryTableHolder::TemporaryTableHolder( } TemporaryTableHolder::TemporaryTableHolder(TemporaryTableHolder && rhs) noexcept - : WithContext(rhs.context), temporary_tables(rhs.temporary_tables), id(rhs.id) + : WithContext(rhs.context), temporary_tables(rhs.temporary_tables), id(rhs.id), future_set(std::move(rhs.future_set)) { rhs.id = UUIDHelpers::Nil; } diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 195839da04f..8d2f2204e84 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -30,6 +30,7 @@ namespace DB namespace ErrorCodes { extern const int WRONG_GLOBAL_SUBQUERY; + extern const int LOGICAL_ERROR; } class GlobalSubqueriesMatcher @@ -159,6 +160,8 @@ public: /*create_for_global_subquery*/ true); StoragePtr external_storage = external_storage_holder->getTable(); + // std::cerr << "......... adding external table " << external_table_name << std::endl; + external_tables.emplace(external_table_name, external_storage_holder); /// We need to materialize external tables immediately because reading from distributed @@ -195,6 +198,7 @@ public: //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + // std::cerr << "... Future set " << reinterpret_cast(external_storage_holder.get()) << " " << reinterpret_cast(future_set.get()) << std::endl; external_storage_holder->future_set = std::move(future_set); } else diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index f2b70462227..c71273e0baa 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -239,6 +239,8 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) return nullptr; } + // std::cerr << "... external_table_set " << reinterpret_cast(external_table_set.get()) << std::endl; + if (external_table_set) return set = external_table_set->buildOrderedSetInplace(context); @@ -337,7 +339,7 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set_key_columns.filter = ColumnUInt8::create(block.rows()); - set->initSetElements(); + //set->initSetElements(); set->insertFromColumns(columns, set_key_columns); set->finishInsert(); //block(std::move(block_)) @@ -350,6 +352,9 @@ FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) { SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { + if (set->hasExplicitSetElements()) + return set; + const auto & settings = context->getSettingsRef(); auto limits = getSizeLimitsForSet(settings, true); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 5adf3d07353..bd9cafc66eb 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -202,15 +202,14 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) if (data.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Method Set::setHeader must be called before Set::insertFromBlock"); - ColumnRawPtrs key_columns; - key_columns.reserve(keys_size); holder.key_columns.reserve(keys_size); + holder.materialized_columns.reserve(keys_size); /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - holder.key_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); - key_columns.emplace_back(holder.key_columns.back().get()); + holder.materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); + holder.key_columns.emplace_back(holder.materialized_columns.back().get()); } size_t rows = columns.at(0)->size(); @@ -219,7 +218,7 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) ConstNullMapPtr null_map{}; ColumnPtr null_map_holder; if (!transform_null_in) - null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); + null_map_holder = extractNestedColumnsAndNullMap(holder.key_columns, null_map); switch (data.type) { @@ -227,7 +226,7 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) break; #define M(NAME) \ case SetVariants::Type::NAME: \ - insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ + insertFromBlockImpl(*data.NAME, holder.key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ break; APPLY_FOR_SET_VARIANTS(M) #undef M @@ -445,6 +444,11 @@ void Set::checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) c MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector && indexes_mapping_) : has_all_keys(set_elements.size() == indexes_mapping_.size()), indexes_mapping(std::move(indexes_mapping_)) { + // std::cerr << "MergeTreeSetIndex::MergeTreeSetIndex " + // << set_elements.size() << ' ' << indexes_mapping.size() << std::endl; + // for (const auto & vv : indexes_mapping) + // std::cerr << vv.key_index << ' ' << vv.tuple_index << std::endl; + ::sort(indexes_mapping.begin(), indexes_mapping.end(), [](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r) { @@ -487,6 +491,7 @@ MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector & key_ranges, const DataTypes & data_types, bool single_point) const { size_t tuple_size = indexes_mapping.size(); + // std::cerr << "MergeTreeSetIndex::checkInRange " << single_point << ' ' << tuple_size << ' ' << has_all_keys << std::endl; FieldValues left_point; FieldValues right_point; diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index cb47fde7f7d..c2931d79de0 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -53,7 +53,8 @@ public: { //ColumnRawPtrs key_columns; /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - Columns key_columns; + ColumnRawPtrs key_columns; + Columns materialized_columns; ColumnPtr null_map_holder; ColumnUInt8::MutablePtr filter; }; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index acd9147b613..4940cbd032c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1174,6 +1174,7 @@ static ActionsDAGPtr buildFilterDAG( static void buildIndexes( std::optional & indexes, ActionsDAGPtr filter_actions_dag, + const MergeTreeData & data, const ContextPtr & context, const SelectQueryInfo & query_info, const StorageMetadataPtr & metadata_snapshot) @@ -1196,7 +1197,7 @@ static void buildIndexes( context, primary_key_column_names, primary_key.expression, - array_join_name_set}, {}, false}); + array_join_name_set}, {}, {}, {}, false}); } else { @@ -1204,7 +1205,22 @@ static void buildIndexes( query_info, context, primary_key_column_names, - primary_key.expression}, {}, false}); + primary_key.expression}, {}, {}, {}, false}); + } + + if (metadata_snapshot->hasPartitionKey()) + { + const auto & partition_key = metadata_snapshot->getPartitionKey(); + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); + auto minmax_expression_actions = data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context)); + // minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); + + // if (context->getSettingsRef().allow_experimental_analyzer) + indexes->minmax_idx_condition.emplace(filter_actions_dag, context, minmax_columns_names, minmax_expression_actions, NameSet()); + // else + // indexes->minmax_idx_condition.emplace(query_info, context, minmax_columns_names, minmax_expression_actions); + + indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */); } indexes->use_skip_indexes = settings.use_skip_indexes; @@ -1250,7 +1266,7 @@ void ReadFromMergeTree::onAddFilterFinish() if (!filter_nodes.nodes.empty()) { auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); - buildIndexes(indexes, filter_actions_dag, context, query_info, metadata_for_reading); + buildIndexes(indexes, filter_actions_dag, data, context, query_info, metadata_for_reading); } } @@ -1366,7 +1382,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( // } if (!indexes) - buildIndexes(indexes, query_info.filter_actions_dag, context, query_info, metadata_snapshot); + buildIndexes(indexes, query_info.filter_actions_dag, data, context, query_info, metadata_snapshot); if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue()) { @@ -1386,11 +1402,12 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( try { MergeTreeDataSelectExecutor::filterPartsByPartition( + indexes->partition_pruner, + indexes->minmax_idx_condition, parts, part_values, metadata_snapshot_base, data, - query_info, context, max_block_numbers_to_read.get(), log, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 6610b463726..5ff9d2f046c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -164,6 +165,8 @@ public: struct Indexes { KeyCondition key_condition; + std::optional partition_pruner; + std::optional minmax_idx_condition; UsefulSkipIndexes skip_indexes; bool use_skip_indexes; }; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7fcc111ced9..fe87198dcf5 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1157,6 +1157,8 @@ bool KeyCondition::tryPrepareSetIndex( RPNElement & out, size_t & out_key_column_num) { + // std::cerr << "::: tryPrepareSetIndex for " << func.getColumnName() << std::endl; + // std::cerr << StackTrace().toString() << std::endl; const auto & left_arg = func.getArgumentAt(0); out_key_column_num = 0; @@ -1200,7 +1202,10 @@ bool KeyCondition::tryPrepareSetIndex( } if (indexes_mapping.empty()) + { + // std::cerr << ".. index mapping is empty\n"; return false; + } const auto right_arg = func.getArgumentAt(1); @@ -1208,7 +1213,10 @@ bool KeyCondition::tryPrepareSetIndex( auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); if (!future_set) + { + // std::cerr << ".. no future set\n"; return false; + } // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); @@ -1220,13 +1228,21 @@ bool KeyCondition::tryPrepareSetIndex( auto prepared_set = future_set->get(); if (!prepared_set) + { + + // std::cerr << ".. no prepared set\n"; return false; + } // LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) + { + + // std::cerr << ".. no explicit elements\n"; return false; + } // LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); @@ -1235,7 +1251,7 @@ bool KeyCondition::tryPrepareSetIndex( prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]); out.set_index = std::make_shared(prepared_set->getSetElements(), std::move(indexes_mapping)); - + // std::cerr << ".. can use\n"; return true; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f99e15c0fc1..f42fada4222 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -793,38 +793,28 @@ std::optional> MergeTreeDataSelectExecutor::filterPar } void MergeTreeDataSelectExecutor::filterPartsByPartition( + std::optional & partition_pruner, + std::optional & minmax_idx_condition, MergeTreeData::DataPartsVector & parts, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, - const SelectQueryInfo & query_info, const ContextPtr & context, const PartitionIdToMaxBlock * max_block_numbers_to_read, Poco::Logger * log, ReadFromMergeTree::IndexStats & index_stats) { const Settings & settings = context->getSettingsRef(); - - std::optional partition_pruner; - std::optional minmax_idx_condition; DataTypes minmax_columns_types; if (metadata_snapshot->hasPartitionKey()) { const auto & partition_key = metadata_snapshot->getPartitionKey(); - auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); - auto minmax_expression_actions = data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context)); minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); - if (context->getSettingsRef().allow_experimental_analyzer) - minmax_idx_condition.emplace(query_info.filter_actions_dag, context, minmax_columns_names, minmax_expression_actions, NameSet()); - else - minmax_idx_condition.emplace(query_info, context, minmax_columns_names, minmax_expression_actions); - - partition_pruner.emplace(metadata_snapshot, query_info, context, false /* strict */); - if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) { + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); throw Exception(ErrorCodes::INDEX_NOT_USED, "Neither MinMax index by columns ({}) nor partition expr is used and setting 'force_index_by_date' is set", fmt::join(minmax_columns_names, ", ")); @@ -1835,7 +1825,9 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( if (partition_pruner) { - if (partition_pruner->canBePruned(*part)) + auto val = partition_pruner->canBePruned(*part); + // std::cerr << "... part " << part->getNameWithState() << " cbp ? " << val << std::endl; + if (val) continue; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 8c8ce59bebe..f1efbdf0310 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -173,11 +173,12 @@ public: /// Filter parts using minmax index and partition key. static void filterPartsByPartition( + std::optional & partition_pruner, + std::optional & minmax_idx_condition, MergeTreeData::DataPartsVector & parts, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, - const SelectQueryInfo & query_info, const ContextPtr & context, const PartitionIdToMaxBlock * max_block_numbers_to_read, Poco::Logger * log, diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 3d0883a1241..e2bf9bde674 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -31,12 +31,21 @@ namespace ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set) { + // std::cerr << "====== " << prepared_set->getDataTypes().size() << std::endl; if (prepared_set->getDataTypes().size() == 1) return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; Columns set_elements; for (auto & set_element : prepared_set->getSetElements()) + { + // std::cerr << set_element->dumpStructure() << std::endl; set_elements.emplace_back(set_element->convertToFullColumnIfConst()); + } + + // for (auto & set_element : prepared_set->getElementsTypes()) + // { + // // std::cerr << set_element->getName() << std::endl; + // } return {ColumnTuple::create(set_elements), std::make_shared(prepared_set->getElementsTypes()), "dummy"}; } @@ -331,6 +340,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (prepared_set && prepared_set->hasExplicitSetElements()) { const auto prepared_info = getPreparedSetInfo(prepared_set); + // std::cerr << "...... " << prepared_info.dumpStructure() << std::endl; if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) maybe_useful = true; } @@ -377,6 +387,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( size_t row_size = column->size(); size_t position = header.getPositionByName(key_node_column_name); const DataTypePtr & index_type = header.getByPosition(position).type; + // std::cerr << "::::: " << ColumnWithTypeAndName{column, type, ""}.dumpStructure() << " -> " << index_type->getName() << std::endl; const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index 35b2d5db3b5..a397a1475d1 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -24,6 +24,19 @@ PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, const Sele { } +PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAGPtr filter_actions_dag, ContextPtr context, bool strict) + : partition_key(MergeTreePartition::adjustPartitionKey(metadata, context)) + , partition_condition(filter_actions_dag, context, partition_key.column_names, partition_key.expression, {}, true /* single_point */, strict) + , useless(strict ? partition_condition.anyUnknownOrAlwaysTrue() : partition_condition.alwaysUnknownOrTrue()) +{ + // auto description = getKeyCondition().getDescription(); + // std::cerr << ".... " << description.condition << std::endl; + // std::cerr << filter_actions_dag->dumpDAG() << std::endl; + // for (const auto & name : partition_key.column_names) + // std::cerr << ". " << name << std::endl; + // std::cerr << partition_key.expression->dumpActions() << std::endl; +} + bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) { if (part.isEmpty()) @@ -39,6 +52,8 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) else { const auto & partition_value = part.partition.value; + // for (const auto & val : partition_value) + // std::cerr << val.dump() << std::endl; std::vector index_value(partition_value.begin(), partition_value.end()); for (auto & field : index_value) { @@ -49,6 +64,7 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) is_valid = partition_condition.mayBeTrueInRange( partition_value.size(), index_value.data(), index_value.data(), partition_key.data_types); + // std::cerr << "may be true " << is_valid << std::endl; partition_filter_map.emplace(partition_id, is_valid); if (!is_valid) diff --git a/src/Storages/MergeTree/PartitionPruner.h b/src/Storages/MergeTree/PartitionPruner.h index 3a986923321..7f1b74795c4 100644 --- a/src/Storages/MergeTree/PartitionPruner.h +++ b/src/Storages/MergeTree/PartitionPruner.h @@ -14,6 +14,7 @@ class PartitionPruner { public: PartitionPruner(const StorageMetadataPtr & metadata, const SelectQueryInfo & query_info, ContextPtr context, bool strict); + PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAGPtr filter_actions_dag, ContextPtr context, bool strict); bool canBePruned(const IMergeTreeDataPart & part); diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 34b463eadee..1e89427071c 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -161,6 +161,17 @@ public: } }; +/// Type of path to be fetched +enum class ZkPathType +{ + Exact, /// Fetch all nodes under this path + Prefix, /// Fetch all nodes starting with this prefix, recursively (multiple paths may match prefix) + Recurse, /// Fatch all nodes under this path, recursively +}; + +/// List of paths to be feched from zookeeper +using Paths = std::deque>; + class ReadFromSystemZooKeeper final : public SourceStepWithFilter { public: @@ -170,11 +181,14 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + void onAddFilterFinish() override; + private: - void fillData(MutableColumns & res_columns) const; + void fillData(MutableColumns & res_columns); std::shared_ptr storage_limits; ContextPtr context; + Paths paths; }; StorageSystemZooKeeper::StorageSystemZooKeeper(const StorageID & table_id_) @@ -246,17 +260,6 @@ NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes() }; } -/// Type of path to be fetched -enum class ZkPathType -{ - Exact, /// Fetch all nodes under this path - Prefix, /// Fetch all nodes starting with this prefix, recursively (multiple paths may match prefix) - Recurse, /// Fatch all nodes under this path, recursively -}; - -/// List of paths to be feched from zookeeper -using Paths = std::deque>; - static String pathCorrected(const String & path) { String path_corrected; @@ -421,10 +424,13 @@ static Paths extractPath(const ActionsDAG::NodeRawConstPtrs & filter_nodes, Cont } -void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns) const +void ReadFromSystemZooKeeper::onAddFilterFinish() { - Paths paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); + paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); +} +void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns) +{ zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper(); if (paths.empty()) From ef9bae50b9cc83a885a6e0f8c0d82a88ee2c791b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 26 May 2023 23:11:57 +0000 Subject: [PATCH 0733/2223] Fix bugs in Poco, use true non-blocking IO --- base/poco/Net/src/SocketImpl.cpp | 10 +++- .../include/Poco/Net/SecureSocketImpl.h | 10 ++++ .../include/Poco/Net/SecureStreamSocketImpl.h | 10 ++++ .../NetSSL_OpenSSL/src/SecureSocketImpl.cpp | 10 ++++ .../src/SecureStreamSocketImpl.cpp | 10 ++++ src/Common/checkSSLError.h | 8 +++ src/Common/checkSSLReturnCode.cpp | 29 ++++++++++ src/Common/checkSSLReturnCode.h | 12 ++++ src/IO/ReadBufferFromPocoSocket.cpp | 58 +++++++++---------- src/IO/ReadBufferFromPocoSocket.h | 2 - src/IO/WriteBufferFromPocoSocket.cpp | 58 +++++++++---------- src/IO/WriteBufferFromPocoSocket.h | 2 - 12 files changed, 154 insertions(+), 65 deletions(-) create mode 100644 src/Common/checkSSLError.h create mode 100644 src/Common/checkSSLReturnCode.cpp create mode 100644 src/Common/checkSSLReturnCode.h diff --git a/base/poco/Net/src/SocketImpl.cpp b/base/poco/Net/src/SocketImpl.cpp index 2aba413b322..484b8cfeec3 100644 --- a/base/poco/Net/src/SocketImpl.cpp +++ b/base/poco/Net/src/SocketImpl.cpp @@ -274,7 +274,9 @@ void SocketImpl::shutdown() int SocketImpl::sendBytes(const void* buffer, int length, int flags) { - if (_isBrokenTimeout) + bool blocking = _blocking && (flags & MSG_DONTWAIT) == 0; + + if (_isBrokenTimeout && blocking) { if (_sndTimeout.totalMicroseconds() != 0) { @@ -289,11 +291,13 @@ int SocketImpl::sendBytes(const void* buffer, int length, int flags) if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); rc = ::send(_sockfd, reinterpret_cast(buffer), length, flags); } - while (_blocking && rc < 0 && lastError() == POCO_EINTR); + while (blocking && rc < 0 && lastError() == POCO_EINTR); if (rc < 0) { int err = lastError(); - if (err == POCO_EAGAIN || err == POCO_ETIMEDOUT) + if ((err == POCO_EAGAIN || err == POCO_EWOULDBLOCK) && !blocking) + ; + else if (err == POCO_EAGAIN || err == POCO_ETIMEDOUT) throw TimeoutException(); else error(err); diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureSocketImpl.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureSocketImpl.h index 56c550decfe..49c12b6b45f 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureSocketImpl.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureSocketImpl.h @@ -183,6 +183,16 @@ namespace Net /// Returns true iff a reused session was negotiated during /// the handshake. + virtual void setBlocking(bool flag); + /// Sets the socket in blocking mode if flag is true, + /// disables blocking mode if flag is false. + + virtual bool getBlocking() const; + /// Returns the blocking mode of the socket. + /// This method will only work if the blocking modes of + /// the socket are changed via the setBlocking method! + + protected: void acceptSSL(); /// Assume per-object mutex is locked. diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureStreamSocketImpl.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureStreamSocketImpl.h index b41043769fe..99e2130d673 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureStreamSocketImpl.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SecureStreamSocketImpl.h @@ -201,6 +201,16 @@ namespace Net /// Returns true iff a reused session was negotiated during /// the handshake. + virtual void setBlocking(bool flag); + /// Sets the socket in blocking mode if flag is true, + /// disables blocking mode if flag is false. + + virtual bool getBlocking() const; + /// Returns the blocking mode of the socket. + /// This method will only work if the blocking modes of + /// the socket are changed via the setBlocking method! + + protected: void acceptSSL(); /// Performs a SSL server-side handshake. diff --git a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp index 9631c7a401a..efe25f65909 100644 --- a/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SecureSocketImpl.cpp @@ -629,5 +629,15 @@ bool SecureSocketImpl::sessionWasReused() return false; } +void SecureSocketImpl::setBlocking(bool flag) +{ + _pSocket->setBlocking(flag); +} + +bool SecureSocketImpl::getBlocking() const +{ + return _pSocket->getBlocking(); +} + } } // namespace Poco::Net diff --git a/base/poco/NetSSL_OpenSSL/src/SecureStreamSocketImpl.cpp b/base/poco/NetSSL_OpenSSL/src/SecureStreamSocketImpl.cpp index aa1a96e1585..c00dd43b2ed 100644 --- a/base/poco/NetSSL_OpenSSL/src/SecureStreamSocketImpl.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SecureStreamSocketImpl.cpp @@ -237,5 +237,15 @@ int SecureStreamSocketImpl::completeHandshake() return _impl.completeHandshake(); } +bool SecureStreamSocketImpl::getBlocking() const +{ + return _impl.getBlocking(); +} + +void SecureStreamSocketImpl::setBlocking(bool flag) +{ + _impl.setBlocking(flag); +} + } } // namespace Poco::Net diff --git a/src/Common/checkSSLError.h b/src/Common/checkSSLError.h new file mode 100644 index 00000000000..05bca9f8b5f --- /dev/null +++ b/src/Common/checkSSLError.h @@ -0,0 +1,8 @@ +// +// Created by Павел Круглов on 27/05/2023. +// + +#ifndef CLICKHOUSE_CHECKSSLERROR_H +#define CLICKHOUSE_CHECKSSLERROR_H + +#endif //CLICKHOUSE_CHECKSSLERROR_H diff --git a/src/Common/checkSSLReturnCode.cpp b/src/Common/checkSSLReturnCode.cpp new file mode 100644 index 00000000000..8916a25e19c --- /dev/null +++ b/src/Common/checkSSLReturnCode.cpp @@ -0,0 +1,29 @@ +#include +#include "config.h" + +#if USE_SSL +#include +#endif + +namespace DB +{ + +bool checkSSLWantRead(ssize_t res) +{ +#if USE_SSL + return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ; +#else + return false; +#endif +} + +bool checkSSLWantWrite(ssize_t res) +{ +#if USE_SSL + return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE; +#else + return false; +#endif +} + +} diff --git a/src/Common/checkSSLReturnCode.h b/src/Common/checkSSLReturnCode.h new file mode 100644 index 00000000000..f30564137aa --- /dev/null +++ b/src/Common/checkSSLReturnCode.h @@ -0,0 +1,12 @@ +#pragma once + +namespace DB +{ + +/// Check if ret is ERR_SSL_WANT_READ. +bool checkSSLWantRead(ssize_t ret); + +/// CHeck if ret is ERR_SSL_WANT_WRITE. +bool checkSSLWantWrite(ssize_t ret); + +} diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index d6790439683..ff72dc5386c 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -9,12 +9,7 @@ #include #include #include - -#include "config.h" - -#if USE_SSL -#include -#endif +#include namespace ProfileEvents { @@ -27,7 +22,6 @@ namespace CurrentMetrics extern const Metric NetworkReceive; } - namespace DB { namespace ErrorCodes @@ -38,14 +32,13 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - bool ReadBufferFromPocoSocket::nextImpl() { ssize_t bytes_read = 0; Stopwatch watch; SCOPE_EXIT({ - // / NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one + /// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds()); ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read); }); @@ -58,17 +51,35 @@ bool ReadBufferFromPocoSocket::nextImpl() if (internal_buffer.size() > INT_MAX) throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - bytes_read = readFromSocket(); - -#if USE_SSL - /// In case of non-blocking connect for secure socket receiveBytes can return ERR_SSL_WANT_READ, - /// in this case we should call receiveBytes again when socket is ready. - if (socket.secure()) + /// If async_callback is specified, set socket to non-blocking mode + /// and try to read data from it, if socket is not ready for reading, + /// run async_callback and try again later. + /// It is expected that file descriptor may be polled externally. + /// Note that send timeout is not checked here. External code should check it while polling. + if (async_callback) { - while (bytes_read == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ) - bytes_read = readFromSocket(); + socket.setBlocking(false); + SCOPE_EXIT(socket.setBlocking(true)); + bool secure = socket.secure(); + bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + + /// Check EAGAIN and ERR_SSL_WANT_READ/ERR_SSL_WANT_WRITE for secure socket (reading from secure socket can write too). + while (bytes_read < 0 && (errno == EAGAIN || (secure && (checkSSLWantRead(bytes_read) || checkSSLWantWrite(bytes_read))))) + { + /// In case of ERR_SSL_WANT_WRITE we should wait for socket to be ready for writing, otherwise - for reading. + if (secure && checkSSLWantWrite(bytes_read)) + async_callback(socket.impl()->sockfd(), socket.getSendTimeout(), AsyncEventTimeoutType::SEND, socket_description, AsyncTaskExecutor::Event::WRITE | AsyncTaskExecutor::Event::ERROR); + else + async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); + + /// Try to read again. + bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); + } + } + else + { + bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); } -#endif } catch (const Poco::Net::NetException & e) { @@ -96,17 +107,6 @@ bool ReadBufferFromPocoSocket::nextImpl() return true; } -ssize_t ReadBufferFromPocoSocket::readFromSocket() -{ - /// If async_callback is specified, and read will block, run async_callback and try again later. - /// It is expected that file descriptor may be polled externally. - /// Note that receive timeout is not checked here. External code should check it while polling. - while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR)) - async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); - - return socket.impl()->receiveBytes(internal_buffer.begin(), static_cast(internal_buffer.size())); -} - ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index 3c4bc424334..dab4ac86295 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -30,8 +30,6 @@ public: void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } private: - ssize_t readFromSocket(); - AsyncCallback async_callback; std::string socket_description; }; diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 6e7c67cc054..df1041f0056 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -10,12 +10,7 @@ #include #include #include - -#include "config.h" - -#if USE_SSL -#include -#endif +#include namespace ProfileEvents { @@ -28,7 +23,6 @@ namespace CurrentMetrics extern const Metric NetworkSend; } - namespace DB { @@ -40,7 +34,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - void WriteBufferFromPocoSocket::nextImpl() { if (!offset()) @@ -67,17 +60,36 @@ void WriteBufferFromPocoSocket::nextImpl() if (size > INT_MAX) throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer overflow"); - res = writeToSocket(pos, size); - -#if USE_SSL - /// In case of non-blocking connect for secure socket sendBytes can return ERR_SSL_WANT_WRITE, - /// in this case we should call sendBytes again when socket is ready. - if (socket.secure()) + /// If async_callback is specified, set socket to non-blocking mode + /// and try to write data to it, if socket is not ready for writing, + /// run async_callback and try again later. + /// It is expected that file descriptor may be polled externally. + /// Note that send timeout is not checked here. External code should check it while polling. + if (async_callback) { - while (res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE) - res = writeToSocket(pos, size); + socket.setBlocking(false); + /// Set socket to blocking mode at the end. + SCOPE_EXIT(socket.setBlocking(true)); + bool secure = socket.secure(); + res = socket.impl()->sendBytes(pos, static_cast(size)); + + /// Check EAGAIN and ERR_SSL_WANT_WRITE/ERR_SSL_WANT_READ for secure socket (writing to secure socket can read too). + while (res < 0 && (errno == EAGAIN || (secure && (checkSSLWantRead(res) || checkSSLWantWrite(res))))) + { + /// In case of ERR_SSL_WANT_READ we should wait for socket to be ready for reading, otherwise - for writing. + if (secure && checkSSLWantRead(res)) + async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), AsyncEventTimeoutType::RECEIVE, socket_description, AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); + else + async_callback(socket.impl()->sockfd(), socket.getSendTimeout(), AsyncEventTimeoutType::SEND, socket_description, AsyncTaskExecutor::Event::WRITE | AsyncTaskExecutor::Event::ERROR); + + /// Try to write again. + res = socket.impl()->sendBytes(pos, static_cast(size)); + } + } + else + { + res = socket.impl()->sendBytes(pos, static_cast(size)); } -#endif } catch (const Poco::Net::NetException & e) { @@ -104,18 +116,6 @@ void WriteBufferFromPocoSocket::nextImpl() } } -ssize_t WriteBufferFromPocoSocket::writeToSocket(char * data, size_t size) -{ - /// If async_callback is specified, and write will block, run async_callback and try again later. - /// It is expected that file descriptor may be polled externally. - /// Note that send timeout is not checked here. External code should check it while polling. - while (async_callback && !socket.poll(0, Poco::Net::Socket::SELECT_WRITE | Poco::Net::Socket::SELECT_ERROR)) - async_callback(socket.impl()->sockfd(), socket.getSendTimeout(), AsyncEventTimeoutType::SEND, socket_description, AsyncTaskExecutor::Event::WRITE | AsyncTaskExecutor::Event::ERROR); - - return socket.impl()->sendBytes(data, static_cast(size)); - -} - WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) : BufferWithOwnMemory(buf_size) , socket(socket_) diff --git a/src/IO/WriteBufferFromPocoSocket.h b/src/IO/WriteBufferFromPocoSocket.h index 0f03e816af5..ecb61020357 100644 --- a/src/IO/WriteBufferFromPocoSocket.h +++ b/src/IO/WriteBufferFromPocoSocket.h @@ -35,8 +35,6 @@ protected: Poco::Net::SocketAddress our_address; private: - ssize_t writeToSocket(char * data, size_t size); - AsyncCallback async_callback; std::string socket_description; }; From 86f85d7b3b621f2890596c096efe669d1baab196 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 26 May 2023 20:07:00 +0200 Subject: [PATCH 0734/2223] cope with finalize in d-tors --- src/IO/WriteBufferFromS3TaskTracker.cpp | 9 ++++++++- src/IO/WriteBufferFromS3TaskTracker.h | 2 +- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 7ae31044012..4abae90eeac 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -162,8 +162,15 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() for (auto & it : finished_futures) { + SCOPE_EXIT({ + /// According to basic exception safety TaskTracker has to be destroyed after exception + /// If it would be true than this SCOPE_EXIT is superfluous + /// However WriteBufferWithFinalizeCallback, WriteBufferFromFileDecorator do call finalize in d-tor + /// TaskTracker has to cope this until the issue with finalizing in d-tor is addressed in #50274 + futures.erase(it); + }); + it->get(); - futures.erase(it); } finished_futures.clear(); diff --git a/src/IO/WriteBufferFromS3TaskTracker.h b/src/IO/WriteBufferFromS3TaskTracker.h index c7dc562e78d..800e5239cc4 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.h +++ b/src/IO/WriteBufferFromS3TaskTracker.h @@ -36,7 +36,7 @@ public: /// There could be no finished tasks yet, so waitIfAny do nothing useful in that case /// the first exception is thrown if any task has failed void waitIfAny(); - + /// Well, waitAll waits all the tasks until they finish and collects their statuses void waitAll(); From 806b7fc18b0e526f39f6b9f04faa2ee90d4136ee Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Sat, 27 May 2023 17:39:21 +0800 Subject: [PATCH 0735/2223] fix datetime64 index querying --- src/Interpreters/convertFieldToType.cpp | 8 ++++++++ .../1_stateful/00178_query_datetime64_index.reference | 1 + tests/queries/1_stateful/00178_query_datetime64_index.sql | 8 ++++++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/1_stateful/00178_query_datetime64_index.reference create mode 100644 tests/queries/1_stateful/00178_query_datetime64_index.sql diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index dc61e748db6..9d82cade814 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -192,6 +192,14 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); } + else if(which_type.isDateTime64() && which_from_type.isDate()) { + const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); + return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); + } + else if(which_type.isDateTime64() && which_from_type.isDate()) { + const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); + return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); + } else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { if (which_type.isUInt8()) return convertNumericType(src, type); diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.reference b/tests/queries/1_stateful/00178_query_datetime64_index.reference new file mode 100644 index 00000000000..3d26cfc1d66 --- /dev/null +++ b/tests/queries/1_stateful/00178_query_datetime64_index.reference @@ -0,0 +1 @@ +2023-05-27 00:00:00.000 diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.sql b/tests/queries/1_stateful/00178_query_datetime64_index.sql new file mode 100644 index 00000000000..0e46401c2d3 --- /dev/null +++ b/tests/queries/1_stateful/00178_query_datetime64_index.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS datetime64_index_tbl; + +CREATE TABLE datetime64_index_tbl(ts DateTime64(3, 'UTC')) ENGINE=MergeTree ORDER BY ts; +INSERT INTO datetime64_index_tbl(ts) VALUES(toDateTime64('2023-05-27 00:00:00', 3, 'UTC')); + +SELECT ts from datetime64_index_tbl where ts < toDate('2023-05-28'); + +DROP TABLE datetime64_index_tbl; \ No newline at end of file From 41c599b86252d7219adc8e0adc455d93d5e7a5ee Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Sat, 27 May 2023 17:59:41 +0800 Subject: [PATCH 0736/2223] fix date32 --- src/Interpreters/convertFieldToType.cpp | 2 +- tests/queries/1_stateful/00178_query_datetime64_index.reference | 1 + tests/queries/1_stateful/00178_query_datetime64_index.sql | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 9d82cade814..9bea9f6851f 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -196,7 +196,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } - else if(which_type.isDateTime64() && which_from_type.isDate()) { + else if(which_type.isDateTime64() && which_from_type.isDate32()) { const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.reference b/tests/queries/1_stateful/00178_query_datetime64_index.reference index 3d26cfc1d66..f24667b36da 100644 --- a/tests/queries/1_stateful/00178_query_datetime64_index.reference +++ b/tests/queries/1_stateful/00178_query_datetime64_index.reference @@ -1 +1,2 @@ 2023-05-27 00:00:00.000 +2023-05-27 00:00:00.000 diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.sql b/tests/queries/1_stateful/00178_query_datetime64_index.sql index 0e46401c2d3..1e1d1561af7 100644 --- a/tests/queries/1_stateful/00178_query_datetime64_index.sql +++ b/tests/queries/1_stateful/00178_query_datetime64_index.sql @@ -4,5 +4,6 @@ CREATE TABLE datetime64_index_tbl(ts DateTime64(3, 'UTC')) ENGINE=MergeTree ORDE INSERT INTO datetime64_index_tbl(ts) VALUES(toDateTime64('2023-05-27 00:00:00', 3, 'UTC')); SELECT ts from datetime64_index_tbl where ts < toDate('2023-05-28'); +SELECT ts from datetime64_index_tbl where ts < toDate32('2023-05-28'); DROP TABLE datetime64_index_tbl; \ No newline at end of file From a8122861213071026430af636801587679b3e710 Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Sat, 27 May 2023 18:04:19 +0800 Subject: [PATCH 0737/2223] new line eof --- tests/queries/1_stateful/00178_query_datetime64_index.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.sql b/tests/queries/1_stateful/00178_query_datetime64_index.sql index 1e1d1561af7..68542bc0524 100644 --- a/tests/queries/1_stateful/00178_query_datetime64_index.sql +++ b/tests/queries/1_stateful/00178_query_datetime64_index.sql @@ -6,4 +6,4 @@ INSERT INTO datetime64_index_tbl(ts) VALUES(toDateTime64('2023-05-27 00:00:00', SELECT ts from datetime64_index_tbl where ts < toDate('2023-05-28'); SELECT ts from datetime64_index_tbl where ts < toDate32('2023-05-28'); -DROP TABLE datetime64_index_tbl; \ No newline at end of file +DROP TABLE datetime64_index_tbl; From 2e347007a2d220f332e86cd985b2b5a41290cfa8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 27 May 2023 12:19:11 +0200 Subject: [PATCH 0738/2223] Fix 02765_queries_with_subqueries_profile_events for Analyzer check Settings does not contains unchanged values, and the value is considered unchanged, if it matches the default. So simply use the shell variable instead of Settings. Signed-off-by: Azat Khuzhin --- ..._queries_with_subqueries_profile_events.reference | 12 ++++++------ .../02765_queries_with_subqueries_profile_events.sh | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference index 1d49b598382..d21f914f0dc 100644 --- a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference +++ b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.reference @@ -1,15 +1,15 @@ view allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries -1 1 0 1 2 3 +1 0 1 0 1 2 3 subquery allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries -1 0 1 0 2 2 +1 0 0 1 0 2 2 CSE allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries -1 0 1 0 2 2 +1 0 0 1 0 2 2 CSE_Multi allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries -1 0 1 0 2 2 +1 0 0 1 0 2 2 CTE allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries -1 0 1 0 2 2 +1 0 0 1 0 2 2 CTE_Multi allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries -1 0 1 0 4 4 +1 0 0 1 0 4 4 view allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries 1 1 1 0 1 3 4 subquery allow_experimental_analyzer InsertQuery SelectQuery InsertQueriesWithSubqueries SelectQueriesWithSubqueries QueriesWithSubqueries diff --git a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh index 4a7877a68b8..cded0b28409 100755 --- a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh +++ b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh @@ -21,7 +21,7 @@ for allow_experimental_analyzer in 0 1; do SYSTEM FLUSH LOGS; SELECT 1 view, - Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + $allow_experimental_analyzer allow_experimental_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, @@ -39,7 +39,7 @@ for allow_experimental_analyzer in 0 1; do SYSTEM FLUSH LOGS; SELECT 1 subquery, - Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + $allow_experimental_analyzer allow_experimental_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, @@ -56,7 +56,7 @@ for allow_experimental_analyzer in 0 1; do SYSTEM FLUSH LOGS; SELECT 1 CSE, - Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + $allow_experimental_analyzer allow_experimental_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, @@ -73,7 +73,7 @@ for allow_experimental_analyzer in 0 1; do SYSTEM FLUSH LOGS; SELECT 1 CSE_Multi, - Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + $allow_experimental_analyzer allow_experimental_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, @@ -90,7 +90,7 @@ for allow_experimental_analyzer in 0 1; do SYSTEM FLUSH LOGS; SELECT 1 CTE, - Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + $allow_experimental_analyzer allow_experimental_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, @@ -107,7 +107,7 @@ for allow_experimental_analyzer in 0 1; do SYSTEM FLUSH LOGS; SELECT 1 CTE_Multi, - Settings['allow_experimental_analyzer'] allow_experimental_analyzer, + $allow_experimental_analyzer allow_experimental_analyzer, ProfileEvents['InsertQuery'] InsertQuery, ProfileEvents['SelectQuery'] SelectQuery, ProfileEvents['InsertQueriesWithSubqueries'] InsertQueriesWithSubqueries, From 0e2156c77693ba0cfdd3c62b113f7f3a57ca2ffa Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 27 May 2023 14:35:37 +0200 Subject: [PATCH 0739/2223] Fix checksum calculation for backup entries (#50264) * Fix possible deadlock in calculating backup entries' checksums. * Fix data type used to store checksums. * Allow ReadBufferFromMemory::seek() to go to the end of the buffer (seek(0, SEEK_SET) must be allowed for an empty buffer) * Add tests. --------- Co-authored-by: Nikita Mikhaylov --- src/Backups/BackupEntryFromImmutableFile.cpp | 31 +- src/Backups/BackupEntryFromImmutableFile.h | 2 +- .../BackupEntryWithChecksumCalculation.cpp | 31 +- src/Backups/tests/gtest_backup_entries.cpp | 350 ++++++++++++++++++ src/IO/ReadBufferFromMemory.cpp | 4 +- 5 files changed, 400 insertions(+), 18 deletions(-) create mode 100644 src/Backups/tests/gtest_backup_entries.cpp diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index d066db0c6ae..93d555065ec 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -59,16 +59,31 @@ UInt64 BackupEntryFromImmutableFile::getSize() const UInt128 BackupEntryFromImmutableFile::getChecksum() const { - std::lock_guard lock{size_and_checksum_mutex}; - if (!checksum_adjusted) { - if (!checksum) - checksum = BackupEntryWithChecksumCalculation::getChecksum(); - else if (copy_encrypted) - checksum = combineChecksums(*checksum, disk->getEncryptedFileIV(file_path)); - checksum_adjusted = true; + std::lock_guard lock{size_and_checksum_mutex}; + if (checksum_adjusted) + return *checksum; + + if (checksum) + { + if (copy_encrypted) + checksum = combineChecksums(*checksum, disk->getEncryptedFileIV(file_path)); + checksum_adjusted = true; + return *checksum; + } + } + + auto calculated_checksum = BackupEntryWithChecksumCalculation::getChecksum(); + + { + std::lock_guard lock{size_and_checksum_mutex}; + if (!checksum_adjusted) + { + checksum = calculated_checksum; + checksum_adjusted = true; + } + return *checksum; } - return *checksum; } std::optional BackupEntryFromImmutableFile::getPartialChecksum(size_t prefix_length) const diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 850a86a3264..37bc6b43cd3 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -44,7 +44,7 @@ private: const DataSourceDescription data_source_description; const bool copy_encrypted; mutable std::optional file_size; - mutable std::optional checksum; + mutable std::optional checksum; mutable bool file_size_adjusted = false; mutable bool checksum_adjusted = false; mutable std::mutex size_and_checksum_mutex; diff --git a/src/Backups/BackupEntryWithChecksumCalculation.cpp b/src/Backups/BackupEntryWithChecksumCalculation.cpp index 1e634e6bb73..610b46238ba 100644 --- a/src/Backups/BackupEntryWithChecksumCalculation.cpp +++ b/src/Backups/BackupEntryWithChecksumCalculation.cpp @@ -8,15 +8,32 @@ namespace DB template UInt128 BackupEntryWithChecksumCalculation::getChecksum() const { - std::lock_guard lock{checksum_calculation_mutex}; - if (!calculated_checksum) { - auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(this->getSize())); - HashingReadBuffer hashing_read_buffer(*read_buffer); - hashing_read_buffer.ignoreAll(); - calculated_checksum = hashing_read_buffer.getHash(); + std::lock_guard lock{checksum_calculation_mutex}; + if (calculated_checksum) + return *calculated_checksum; + } + + size_t size = this->getSize(); + + { + std::lock_guard lock{checksum_calculation_mutex}; + if (!calculated_checksum) + { + if (size == 0) + { + calculated_checksum = 0; + } + else + { + auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(size)); + HashingReadBuffer hashing_read_buffer(*read_buffer); + hashing_read_buffer.ignoreAll(); + calculated_checksum = hashing_read_buffer.getHash(); + } + } + return *calculated_checksum; } - return *calculated_checksum; } template diff --git a/src/Backups/tests/gtest_backup_entries.cpp b/src/Backups/tests/gtest_backup_entries.cpp new file mode 100644 index 00000000000..3077bedad0e --- /dev/null +++ b/src/Backups/tests/gtest_backup_entries.cpp @@ -0,0 +1,350 @@ +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include + +using namespace DB; + + +class BackupEntriesTest : public ::testing::Test +{ +protected: + void SetUp() override + { + /// Make local disk. + temp_dir = std::make_unique(); + temp_dir->createDirectories(); + local_disk = std::make_shared("local_disk", temp_dir->path() + "/", 0); + + /// Make encrypted disk. + auto settings = std::make_unique(); + settings->wrapped_disk = local_disk; + settings->current_algorithm = FileEncryption::Algorithm::AES_128_CTR; + settings->keys[0] = "1234567890123456"; + settings->current_key_id = 0; + settings->disk_path = "encrypted/"; + encrypted_disk = std::make_shared("encrypted_disk", std::move(settings), true); + } + + void TearDown() override + { + encrypted_disk.reset(); + local_disk.reset(); + } + + static void writeFile(DiskPtr disk, const String & filepath) + { + auto buf = disk->writeFile(filepath, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); + writeString(std::string_view{"Some text"}, *buf); + buf->finalize(); + } + + static void writeEmptyFile(DiskPtr disk, const String & filepath) + { + auto buf = disk->writeFile(filepath, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); + buf->finalize(); + } + + static void appendFile(DiskPtr disk, const String & filepath) + { + auto buf = disk->writeFile(filepath, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); + writeString(std::string_view{"Appended"}, *buf); + buf->finalize(); + } + + static String getChecksum(const BackupEntryPtr & backup_entry) + { + return getHexUIntUppercase(backup_entry->getChecksum()); + } + + static const constexpr std::string_view NO_CHECKSUM = "no checksum"; + + static String getPartialChecksum(const BackupEntryPtr & backup_entry, size_t prefix_length) + { + auto partial_checksum = backup_entry->getPartialChecksum(prefix_length); + if (!partial_checksum) + return String{NO_CHECKSUM}; + return getHexUIntUppercase(*partial_checksum); + } + + static String readAll(const BackupEntryPtr & backup_entry) + { + auto in = backup_entry->getReadBuffer({}); + String str; + readStringUntilEOF(str, *in); + return str; + } + + std::unique_ptr temp_dir; + std::shared_ptr local_disk; + std::shared_ptr encrypted_disk; +}; + + +static const constexpr std::string_view ZERO_CHECKSUM = "00000000000000000000000000000000"; + +static const constexpr std::string_view SOME_TEXT_CHECKSUM = "28B5529750AC210952FFD366774363ED"; +static const constexpr std::string_view S_CHECKSUM = "C27395C39AFB5557BFE47661CC9EB86C"; +static const constexpr std::string_view SOME_TEX_CHECKSUM = "D00D9BE8D87919A165F14EDD31088A0E"; +static const constexpr std::string_view SOME_TEXT_APPENDED_CHECKSUM = "5A1F10F638DC7A226231F3FD927D1726"; + +static const constexpr std::string_view PRECALCULATED_CHECKSUM = "1122334455667788AABBCCDDAABBCCDD"; +static const constexpr UInt128 PRECALCULATED_CHECKSUM_UINT128 = (UInt128(0x1122334455667788) << 64) | 0xAABBCCDDAABBCCDD; +static const size_t PRECALCULATED_SIZE = 123; + +TEST_F(BackupEntriesTest, BackupEntryFromImmutableFile) +{ + writeFile(local_disk, "a.txt"); + + auto entry = std::make_shared(local_disk, "a.txt"); + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 8), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1000), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); + + writeEmptyFile(local_disk, "empty.txt"); + + auto empty_entry = std::make_shared(local_disk, "empty.txt"); + EXPECT_EQ(empty_entry->getSize(), 0); + EXPECT_EQ(getChecksum(empty_entry), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1000), ZERO_CHECKSUM); + EXPECT_EQ(readAll(empty_entry), ""); + + auto precalculated_entry = std::make_shared(local_disk, "a.txt", false, PRECALCULATED_SIZE, PRECALCULATED_CHECKSUM_UINT128); + EXPECT_EQ(precalculated_entry->getSize(), PRECALCULATED_SIZE); + + EXPECT_EQ(getChecksum(precalculated_entry), PRECALCULATED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, PRECALCULATED_SIZE - 1), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, PRECALCULATED_SIZE), PRECALCULATED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1000), PRECALCULATED_CHECKSUM); + EXPECT_EQ(readAll(precalculated_entry), "Some text"); +} + +TEST_F(BackupEntriesTest, BackupEntryFromAppendOnlyFile) +{ + writeFile(local_disk, "a.txt"); + + auto entry = std::make_shared(local_disk, "a.txt"); + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), S_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 8), SOME_TEX_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1000), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); + + appendFile(local_disk, "a.txt"); + + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), S_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 8), SOME_TEX_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1000), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); + + auto appended_entry = std::make_shared(local_disk, "a.txt"); + EXPECT_EQ(appended_entry->getSize(), 17); + EXPECT_EQ(getChecksum(appended_entry), SOME_TEXT_APPENDED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 1), S_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 8), SOME_TEX_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 22), SOME_TEXT_APPENDED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 1000), SOME_TEXT_APPENDED_CHECKSUM); + EXPECT_EQ(readAll(appended_entry), "Some textAppended"); + + writeEmptyFile(local_disk, "empty_appended.txt"); + + auto empty_entry = std::make_shared(local_disk, "empty_appended.txt"); + EXPECT_EQ(empty_entry->getSize(), 0); + EXPECT_EQ(getChecksum(empty_entry), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1000), ZERO_CHECKSUM); + EXPECT_EQ(readAll(empty_entry), ""); + + appendFile(local_disk, "empty_appended.txt"); + EXPECT_EQ(empty_entry->getSize(), 0); + EXPECT_EQ(getChecksum(empty_entry), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1000), ZERO_CHECKSUM); + EXPECT_EQ(readAll(empty_entry), ""); +} + +TEST_F(BackupEntriesTest, PartialChecksumBeforeFullChecksum) +{ + writeFile(local_disk, "a.txt"); + + auto entry = std::make_shared(local_disk, "a.txt"); + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); + + entry = std::make_shared(local_disk, "a.txt"); + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getPartialChecksum(entry, 1), S_CHECKSUM); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); +} + +TEST_F(BackupEntriesTest, BackupEntryFromSmallFile) +{ + writeFile(local_disk, "a.txt"); + auto entry = std::make_shared(local_disk, "a.txt"); + + local_disk->removeFile("a.txt"); + + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), S_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 8), SOME_TEX_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1000), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); +} + +TEST_F(BackupEntriesTest, DecryptedEntriesFromEncryptedDisk) +{ + { + writeFile(encrypted_disk, "a.txt"); + std::pair test_cases[] + = {{std::make_shared(encrypted_disk, "a.txt"), false}, + {std::make_shared(encrypted_disk, "a.txt"), true}, + {std::make_shared(encrypted_disk, "a.txt"), true}}; + for (const auto & [entry, partial_checksum_allowed] : test_cases) + { + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), partial_checksum_allowed ? S_CHECKSUM : NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 8), partial_checksum_allowed ? SOME_TEX_CHECKSUM : NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1000), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); + } + } + + { + writeEmptyFile(encrypted_disk, "empty.txt"); + BackupEntryPtr entries[] + = {std::make_shared(encrypted_disk, "empty.txt"), + std::make_shared(encrypted_disk, "empty.txt"), + std::make_shared(encrypted_disk, "empty.txt")}; + for (const auto & entry : entries) + { + EXPECT_EQ(entry->getSize(), 0); + EXPECT_EQ(getChecksum(entry), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), ZERO_CHECKSUM); + EXPECT_EQ(readAll(entry), ""); + } + } + + { + auto precalculated_entry = std::make_shared(encrypted_disk, "a.txt", false, PRECALCULATED_SIZE, PRECALCULATED_CHECKSUM_UINT128); + EXPECT_EQ(precalculated_entry->getSize(), PRECALCULATED_SIZE); + EXPECT_EQ(getChecksum(precalculated_entry), PRECALCULATED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, PRECALCULATED_SIZE), PRECALCULATED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1000), PRECALCULATED_CHECKSUM); + EXPECT_EQ(readAll(precalculated_entry), "Some text"); + } +} + +TEST_F(BackupEntriesTest, EncryptedEntriesFromEncryptedDisk) +{ + { + writeFile(encrypted_disk, "a.txt"); + BackupEntryPtr entries[] + = {std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true), + std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true), + std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true)}; + + auto encrypted_checksum = getChecksum(entries[0]); + EXPECT_NE(encrypted_checksum, NO_CHECKSUM); + EXPECT_NE(encrypted_checksum, ZERO_CHECKSUM); + EXPECT_NE(encrypted_checksum, SOME_TEXT_CHECKSUM); + + auto partial_checksum = getPartialChecksum(entries[1], 9); + EXPECT_NE(partial_checksum, NO_CHECKSUM); + EXPECT_NE(partial_checksum, ZERO_CHECKSUM); + EXPECT_NE(partial_checksum, SOME_TEXT_CHECKSUM); + EXPECT_NE(partial_checksum, encrypted_checksum); + + auto encrypted_data = readAll(entries[0]); + EXPECT_EQ(encrypted_data.size(), 9 + FileEncryption::Header::kSize); + + for (const auto & entry : entries) + { + EXPECT_EQ(entry->getSize(), 9 + FileEncryption::Header::kSize); + EXPECT_EQ(getChecksum(entry), encrypted_checksum); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + auto encrypted_checksum_9 = getPartialChecksum(entry, 9); + EXPECT_TRUE(encrypted_checksum_9 == NO_CHECKSUM || encrypted_checksum_9 == partial_checksum); + EXPECT_EQ(getPartialChecksum(entry, 9 + FileEncryption::Header::kSize), encrypted_checksum); + EXPECT_EQ(getPartialChecksum(entry, 1000), encrypted_checksum); + EXPECT_EQ(readAll(entry), encrypted_data); + } + } + + { + writeEmptyFile(encrypted_disk, "empty.txt"); + BackupEntryPtr entries[] + = {std::make_shared(encrypted_disk, "empty.txt", /* copy_encrypted= */ true), + std::make_shared(encrypted_disk, "empty.txt", /* copy_encrypted= */ true), + std::make_shared(encrypted_disk, "empty.txt", /* copy_encrypted= */ true)}; + for (const auto & entry : entries) + { + EXPECT_EQ(entry->getSize(), 0); + EXPECT_EQ(getChecksum(entry), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), ZERO_CHECKSUM); + EXPECT_EQ(readAll(entry), ""); + } + } + + { + auto precalculated_entry = std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true, PRECALCULATED_SIZE, PRECALCULATED_CHECKSUM_UINT128); + EXPECT_EQ(precalculated_entry->getSize(), PRECALCULATED_SIZE + FileEncryption::Header::kSize); + + auto encrypted_checksum = getChecksum(precalculated_entry); + EXPECT_NE(encrypted_checksum, NO_CHECKSUM); + EXPECT_NE(encrypted_checksum, ZERO_CHECKSUM); + EXPECT_NE(encrypted_checksum, SOME_TEXT_CHECKSUM); + EXPECT_NE(encrypted_checksum, PRECALCULATED_CHECKSUM); + + EXPECT_EQ(getPartialChecksum(precalculated_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, PRECALCULATED_SIZE), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, PRECALCULATED_SIZE + FileEncryption::Header::kSize), encrypted_checksum); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1000), encrypted_checksum); + + auto encrypted_data = readAll(precalculated_entry); + EXPECT_EQ(encrypted_data.size(), 9 + FileEncryption::Header::kSize); + } +} diff --git a/src/IO/ReadBufferFromMemory.cpp b/src/IO/ReadBufferFromMemory.cpp index ea0054f24d4..ede2c531e43 100644 --- a/src/IO/ReadBufferFromMemory.cpp +++ b/src/IO/ReadBufferFromMemory.cpp @@ -12,7 +12,7 @@ off_t ReadBufferFromMemory::seek(off_t offset, int whence) { if (whence == SEEK_SET) { - if (offset >= 0 && internal_buffer.begin() + offset < internal_buffer.end()) + if (offset >= 0 && internal_buffer.begin() + offset <= internal_buffer.end()) { pos = internal_buffer.begin() + offset; working_buffer = internal_buffer; /// We need to restore `working_buffer` in case the position was at EOF before this seek(). @@ -25,7 +25,7 @@ off_t ReadBufferFromMemory::seek(off_t offset, int whence) else if (whence == SEEK_CUR) { Position new_pos = pos + offset; - if (new_pos >= internal_buffer.begin() && new_pos < internal_buffer.end()) + if (new_pos >= internal_buffer.begin() && new_pos <= internal_buffer.end()) { pos = new_pos; working_buffer = internal_buffer; /// We need to restore `working_buffer` in case the position was at EOF before this seek(). From 93415789d7e69c2874171941f56eb92a6bc80bbf Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Sun, 28 May 2023 10:40:34 +0800 Subject: [PATCH 0740/2223] format --- src/Interpreters/convertFieldToType.cpp | 4 ++-- tests/queries/1_stateful/00178_query_datetime64_index.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 9bea9f6851f..ba02eefabc9 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -192,11 +192,11 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); } - else if(which_type.isDateTime64() && which_from_type.isDate()) { + else if (which_type.isDateTime64() && which_from_type.isDate()) { const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } - else if(which_type.isDateTime64() && which_from_type.isDate32()) { + else if (which_type.isDateTime64() && which_from_type.isDate32()) { const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.sql b/tests/queries/1_stateful/00178_query_datetime64_index.sql index 68542bc0524..a3fb594db73 100644 --- a/tests/queries/1_stateful/00178_query_datetime64_index.sql +++ b/tests/queries/1_stateful/00178_query_datetime64_index.sql @@ -3,7 +3,7 @@ DROP TABLE IF EXISTS datetime64_index_tbl; CREATE TABLE datetime64_index_tbl(ts DateTime64(3, 'UTC')) ENGINE=MergeTree ORDER BY ts; INSERT INTO datetime64_index_tbl(ts) VALUES(toDateTime64('2023-05-27 00:00:00', 3, 'UTC')); -SELECT ts from datetime64_index_tbl where ts < toDate('2023-05-28'); -SELECT ts from datetime64_index_tbl where ts < toDate32('2023-05-28'); +SELECT ts FROM datetime64_index_tbl WHERE ts < toDate('2023-05-28'); +SELECT ts FROM datetime64_index_tbl WHERE ts < toDate32('2023-05-28'); DROP TABLE datetime64_index_tbl; From 0d873d50a33d4837d4010be0385df23c8be9b204 Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Sun, 28 May 2023 10:49:36 +0800 Subject: [PATCH 0741/2223] format --- src/Interpreters/convertFieldToType.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index ba02eefabc9..79251d944c8 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -192,11 +192,13 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); } - else if (which_type.isDateTime64() && which_from_type.isDate()) { + else if (which_type.isDateTime64() && which_from_type.isDate()) + { const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } - else if (which_type.isDateTime64() && which_from_type.isDate32()) { + else if (which_type.isDateTime64() && which_from_type.isDate32()) + { const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } From 90a643771622803301eb1789dcfef0591b89a1fb Mon Sep 17 00:00:00 2001 From: helifu Date: Thu, 11 May 2023 16:02:15 +0800 Subject: [PATCH 0742/2223] Support custom 'order by' expression for system log tables --- src/Interpreters/SystemLog.cpp | 20 +++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index fca7eccf93a..89528b67d8a 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -152,19 +152,33 @@ std::shared_ptr createSystemLog( } else { - String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); + /// ENGINE expr is necessary. engine = "ENGINE = MergeTree"; + + /// PARTITION expr is not necessary. + String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); if (!partition_by.empty()) + { engine += " PARTITION BY (" + partition_by + ")"; + } + + /// TTL expr is not necessary. String ttl = config.getString(config_prefix + ".ttl", ""); if (!ttl.empty()) + { engine += " TTL " + ttl; + } - engine += " ORDER BY "; - engine += TSystemLog::getDefaultOrderBy(); + /// ORDER BY expr is necessary. + String order_by = config.getString(config_prefix + ".order_by", TSystemLog::getDefaultOrderBy()); + engine += " ORDER BY (" + order_by + ")"; + + /// STORAGE POLICY expr is not necessary. String storage_policy = config.getString(config_prefix + ".storage_policy", ""); if (!storage_policy.empty()) + { engine += " SETTINGS storage_policy = " + quoteString(storage_policy); + } } /// Validate engine definition syntax to prevent some configuration errors. From 03e11b6e165b9de21a30e37a699a996e0323aa54 Mon Sep 17 00:00:00 2001 From: helifu Date: Tue, 16 May 2023 14:00:20 +0800 Subject: [PATCH 0743/2223] Remove the duplicate pair of parentheses --- programs/server/config.xml | 10 ++++++++++ src/Common/SystemLogBase.h | 2 +- src/Interpreters/AsynchronousInsertLog.h | 2 +- src/Interpreters/AsynchronousMetricLog.h | 2 +- 4 files changed, 13 insertions(+), 3 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index c6ae47b3050..5b69d9f6283 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1137,6 +1137,16 @@ event_date + INTERVAL 30 DAY DELETE --> + + diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 8ac731c34f7..f8febd8b159 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -92,7 +92,7 @@ public: String getName() const override { return LogElement::name(); } - static const char * getDefaultOrderBy() { return "(event_date, event_time)"; } + static const char * getDefaultOrderBy() { return "event_date, event_time"; } protected: Poco::Logger * log; diff --git a/src/Interpreters/AsynchronousInsertLog.h b/src/Interpreters/AsynchronousInsertLog.h index f1e39db1ded..372d1cf5a1b 100644 --- a/src/Interpreters/AsynchronousInsertLog.h +++ b/src/Interpreters/AsynchronousInsertLog.h @@ -45,7 +45,7 @@ public: using SystemLog::SystemLog; /// This table is usually queried for fixed table name. - static const char * getDefaultOrderBy() { return "(database, table, event_date, event_time)"; } + static const char * getDefaultOrderBy() { return "database, table, event_date, event_time"; } }; } diff --git a/src/Interpreters/AsynchronousMetricLog.h b/src/Interpreters/AsynchronousMetricLog.h index 1937aa09dbd..3b0ce1bee04 100644 --- a/src/Interpreters/AsynchronousMetricLog.h +++ b/src/Interpreters/AsynchronousMetricLog.h @@ -49,7 +49,7 @@ public: void addValues(const AsynchronousMetricValues &); /// This table is usually queried for fixed metric name. - static const char * getDefaultOrderBy() { return "(metric, event_date, event_time)"; } + static const char * getDefaultOrderBy() { return "metric, event_date, event_time"; } }; } From 57e5a4aa63239258d43c475efe3b329e892420dd Mon Sep 17 00:00:00 2001 From: helifu Date: Tue, 16 May 2023 15:53:12 +0800 Subject: [PATCH 0744/2223] Add the 'order_by' description for system log tables --- .../settings.md | 68 ++++++++++--------- src/Interpreters/SystemLog.cpp | 15 ++-- 2 files changed, 48 insertions(+), 35 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 76ff8ea049e..fe6b0bfb060 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1341,12 +1341,13 @@ Queries are logged in the [system.part_log](../../operations/system-tables/part_ Use the following parameters to configure logging: -- `database` – Name of the database. -- `table` – Name of the system table. -- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `database` - Name of the database. +- `table` - Name of the system table. +- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. -- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. -- `storage_policy` – Name of storage policy to use for the table (optional) +- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. +- `storage_policy` - Name of storage policy to use for the table (optional) **Example** @@ -1417,12 +1418,13 @@ Queries are logged in the [system.query_log](../../operations/system-tables/quer Use the following parameters to configure logging: -- `database` – Name of the database. -- `table` – Name of the system table the queries will be logged in. -- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `database` - Name of the database. +- `table` - Name of the system table the queries will be logged in. +- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. -- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. -- `storage_policy` – Name of storage policy to use for the table (optional) +- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. +- `storage_policy` - Name of storage policy to use for the table (optional) If the table does not exist, ClickHouse will create it. If the structure of the query log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. @@ -1473,12 +1475,13 @@ Queries are logged in the [system.query_thread_log](../../operations/system-tabl Use the following parameters to configure logging: -- `database` – Name of the database. -- `table` – Name of the system table the queries will be logged in. -- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `database` - Name of the database. +- `table` - Name of the system table the queries will be logged in. +- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. -- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. -- `storage_policy` – Name of storage policy to use for the table (optional) +- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. +- `storage_policy` - Name of storage policy to use for the table (optional) If the table does not exist, ClickHouse will create it. If the structure of the query thread log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. @@ -1501,12 +1504,13 @@ Queries are logged in the [system.query_views_log](../../operations/system-table Use the following parameters to configure logging: -- `database` – Name of the database. -- `table` – Name of the system table the queries will be logged in. -- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `database` - Name of the database. +- `table` - Name of the system table the queries will be logged in. +- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. -- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. -- `storage_policy` – Name of storage policy to use for the table (optional) +- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. +- `storage_policy` - Name of storage policy to use for the table (optional) If the table does not exist, ClickHouse will create it. If the structure of the query views log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. @@ -1527,13 +1531,14 @@ Settings for the [text_log](../../operations/system-tables/text_log.md#system_ta Parameters: -- `level` — Maximum Message Level (by default `Trace`) which will be stored in a table. -- `database` — Database name. -- `table` — Table name. -- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `level` - Maximum Message Level (by default `Trace`) which will be stored in a table. +- `database` - Database name. +- `table` - Table name. +- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. -- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. -- `storage_policy` – Name of storage policy to use for the table (optional) +- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. +- `storage_policy` - Name of storage policy to use for the table (optional) **Example** ```xml @@ -1556,12 +1561,13 @@ Settings for the [trace_log](../../operations/system-tables/trace_log.md#system_ Parameters: -- `database` — Database for storing a table. -- `table` — Table name. -- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `database` - Database for storing a table. +- `table` - Table name. +- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` defined. -- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. -- `storage_policy` – Name of storage policy to use for the table (optional) +- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. +- `storage_policy` - Name of storage policy to use for the table (optional) The default server configuration file `config.xml` contains the following settings section: diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 89528b67d8a..fa2af9b7d0b 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -143,11 +143,18 @@ std::shared_ptr createSystemLog( "If 'engine' is specified for system table, PARTITION BY parameters should " "be specified directly inside 'engine' and 'partition_by' setting doesn't make sense"); if (config.has(config_prefix + ".ttl")) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "If 'engine' is specified for system table, " - "TTL parameters should be specified directly inside 'engine' and 'ttl' setting doesn't make sense"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "If 'engine' is specified for system table, TTL parameters should " + "be specified directly inside 'engine' and 'ttl' setting doesn't make sense"); + if (config.has(config_prefix + ".order_by")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "If 'engine' is specified for system table, ORDER BY parameters should " + "be specified directly inside 'engine' and 'order_by' setting doesn't make sense"); if (config.has(config_prefix + ".storage_policy")) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "If 'engine' is specified for system table, SETTINGS storage_policy = '...' " - "should be specified directly inside 'engine' and 'storage_policy' setting doesn't make sense"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "If 'engine' is specified for system table, SETTINGS storage_policy = '...' should " + "be specified directly inside 'engine' and 'storage_policy' setting doesn't make sense"); + engine = config.getString(config_prefix + ".engine"); } else From 59a815e4e6a8439e260988f5dda2def3dd1ac86b Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 17 May 2023 13:38:36 +0800 Subject: [PATCH 0745/2223] Update according to review comments --- .../server-configuration-parameters/settings.md | 12 ++++++------ src/Interpreters/SystemLog.cpp | 6 ------ 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index fe6b0bfb060..0befeb108a5 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1345,7 +1345,7 @@ Use the following parameters to configure logging: - `table` - Name of the system table. - `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. -- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. - `storage_policy` - Name of storage policy to use for the table (optional) @@ -1422,7 +1422,7 @@ Use the following parameters to configure logging: - `table` - Name of the system table the queries will be logged in. - `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. -- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. - `storage_policy` - Name of storage policy to use for the table (optional) @@ -1479,7 +1479,7 @@ Use the following parameters to configure logging: - `table` - Name of the system table the queries will be logged in. - `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. -- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. - `storage_policy` - Name of storage policy to use for the table (optional) @@ -1508,7 +1508,7 @@ Use the following parameters to configure logging: - `table` - Name of the system table the queries will be logged in. - `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. -- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. - `storage_policy` - Name of storage policy to use for the table (optional) @@ -1536,7 +1536,7 @@ Parameters: - `table` - Table name. - `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. -- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. - `storage_policy` - Name of storage policy to use for the table (optional) @@ -1565,7 +1565,7 @@ Parameters: - `table` - Table name. - `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. -- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. - `storage_policy` - Name of storage policy to use for the table (optional) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index fa2af9b7d0b..71fd46686c1 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -165,16 +165,12 @@ std::shared_ptr createSystemLog( /// PARTITION expr is not necessary. String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); if (!partition_by.empty()) - { engine += " PARTITION BY (" + partition_by + ")"; - } /// TTL expr is not necessary. String ttl = config.getString(config_prefix + ".ttl", ""); if (!ttl.empty()) - { engine += " TTL " + ttl; - } /// ORDER BY expr is necessary. String order_by = config.getString(config_prefix + ".order_by", TSystemLog::getDefaultOrderBy()); @@ -183,9 +179,7 @@ std::shared_ptr createSystemLog( /// STORAGE POLICY expr is not necessary. String storage_policy = config.getString(config_prefix + ".storage_policy", ""); if (!storage_policy.empty()) - { engine += " SETTINGS storage_policy = " + quoteString(storage_policy); - } } /// Validate engine definition syntax to prevent some configuration errors. From 4013ec982ff887e180328e7b1bbde280517cab22 Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 17 May 2023 16:00:21 +0800 Subject: [PATCH 0746/2223] Add test case for 'order_by' configuration --- .../integration/test_system_logs/__init__.py | 0 .../configs/config.d/config.xml | 55 +++++++++++++++++++ tests/integration/test_system_logs/test.py | 34 ++++++++++++ 3 files changed, 89 insertions(+) create mode 100644 tests/integration/test_system_logs/__init__.py create mode 100644 tests/integration/test_system_logs/configs/config.d/config.xml create mode 100644 tests/integration/test_system_logs/test.py diff --git a/tests/integration/test_system_logs/__init__.py b/tests/integration/test_system_logs/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_system_logs/configs/config.d/config.xml b/tests/integration/test_system_logs/configs/config.d/config.xml new file mode 100644 index 00000000000..46e60164ce6 --- /dev/null +++ b/tests/integration/test_system_logs/configs/config.d/config.xml @@ -0,0 +1,55 @@ + + + event_date, event_time, initial_query_id + + + event_date, event_time, query_id + + + diff --git a/tests/integration/test_system_logs/test.py b/tests/integration/test_system_logs/test.py new file mode 100644 index 00000000000..252b167a262 --- /dev/null +++ b/tests/integration/test_system_logs/test.py @@ -0,0 +1,34 @@ +# pylint: disable=line-too-long +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node_default", + main_configs=["configs/config.d/config.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_system_logs_order_by_expr(start_cluster): + node.query("SET log_query_threads = 1") + node.query("SELECT count() FROM system.tables") + node.query("SYSTEM FLUSH LOGS") + + # system.query_log + assert node.query("SELECT sorting_key FROM system.tables WHERE database='system' and name='query_log'") + == "event_date, event_time, initial_query_id\n" + # system.query_thread_log + assert node.query("SELECT sorting_key FROM system.tables WHERE database='system' and name='query_thread_log'") + == "event_date, event_time, query_id\n" From 6e8136e6f4f035dc2bd89234e818885515d92977 Mon Sep 17 00:00:00 2001 From: helifu Date: Thu, 18 May 2023 16:36:11 +0800 Subject: [PATCH 0747/2223] Add 'settings' expression for system log tables --- .../settings.md | 18 ++++++++++------ src/Interpreters/SystemLog.cpp | 21 ++++++++++++++++--- 2 files changed, 30 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 0befeb108a5..7ebf387f2ed 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1347,7 +1347,8 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional) +- `storage_policy` - Name of storage policy to use for the table (optional). +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). **Example** @@ -1424,7 +1425,8 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional) +- `storage_policy` - Name of storage policy to use for the table (optional). +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). If the table does not exist, ClickHouse will create it. If the structure of the query log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. @@ -1481,7 +1483,8 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional) +- `storage_policy` - Name of storage policy to use for the table (optional). +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). If the table does not exist, ClickHouse will create it. If the structure of the query thread log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. @@ -1510,7 +1513,8 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional) +- `storage_policy` - Name of storage policy to use for the table (optional). +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). If the table does not exist, ClickHouse will create it. If the structure of the query views log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. @@ -1538,7 +1542,8 @@ Parameters: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional) +- `storage_policy` - Name of storage policy to use for the table (optional). +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). **Example** ```xml @@ -1567,7 +1572,8 @@ Parameters: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional) +- `storage_policy` - Name of storage policy to use for the table (optional). +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). The default server configuration file `config.xml` contains the following settings section: diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 71fd46686c1..3fd0297f5b8 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -154,6 +154,10 @@ std::shared_ptr createSystemLog( throw Exception(ErrorCodes::BAD_ARGUMENTS, "If 'engine' is specified for system table, SETTINGS storage_policy = '...' should " "be specified directly inside 'engine' and 'storage_policy' setting doesn't make sense"); + if (config.has(config_prefix + ".settings")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "If 'engine' is specified for system table, SETTINGS parameters should " + "be specified directly inside 'engine' and 'settings' setting doesn't make sense"); engine = config.getString(config_prefix + ".engine"); } @@ -176,10 +180,21 @@ std::shared_ptr createSystemLog( String order_by = config.getString(config_prefix + ".order_by", TSystemLog::getDefaultOrderBy()); engine += " ORDER BY (" + order_by + ")"; - /// STORAGE POLICY expr is not necessary. + /// SETTINGS expr is not necessary. + /// https://clickhouse.com/docs/en/engines/table-engines/mergetree-family/mergetree#settings + /// + /// STORAGE POLICY expr is retained for backward compatible. String storage_policy = config.getString(config_prefix + ".storage_policy", ""); - if (!storage_policy.empty()) - engine += " SETTINGS storage_policy = " + quoteString(storage_policy); + String settings = config.getString(config_prefix + ".settings", ""); + if (!storage_policy.empty() || !settings.empty()) + { + engine += " SETTINGS"; + /// If 'storage_policy' is repeated, the 'settings' configuration is preferred. + if (!storage_policy.empty()) + engine += " storage_policy = " + quoteString(storage_policy); + if (!settings.empty()) + engine += (storage_policy.empty() ? " " : ", ") + settings; + } } /// Validate engine definition syntax to prevent some configuration errors. From 01158e783a45d9fec60c06389af82018369a2818 Mon Sep 17 00:00:00 2001 From: helifu Date: Thu, 18 May 2023 19:55:16 +0800 Subject: [PATCH 0748/2223] Add test case for 'engine','settings' configuration --- .../configs/config.d/config.xml | 55 -------------- .../configs/config.d/disks.xml | 28 +++++++ .../configs/system_logs_engine.xml | 5 ++ .../configs/system_logs_order_by.xml | 8 ++ .../configs/system_logs_settings.xml | 10 +++ tests/integration/test_system_logs/test.py | 34 --------- .../test_system_logs/test_system_logs.py | 73 +++++++++++++++++++ 7 files changed, 124 insertions(+), 89 deletions(-) delete mode 100644 tests/integration/test_system_logs/configs/config.d/config.xml create mode 100644 tests/integration/test_system_logs/configs/config.d/disks.xml create mode 100644 tests/integration/test_system_logs/configs/system_logs_engine.xml create mode 100644 tests/integration/test_system_logs/configs/system_logs_order_by.xml create mode 100644 tests/integration/test_system_logs/configs/system_logs_settings.xml delete mode 100644 tests/integration/test_system_logs/test.py create mode 100644 tests/integration/test_system_logs/test_system_logs.py diff --git a/tests/integration/test_system_logs/configs/config.d/config.xml b/tests/integration/test_system_logs/configs/config.d/config.xml deleted file mode 100644 index 46e60164ce6..00000000000 --- a/tests/integration/test_system_logs/configs/config.d/config.xml +++ /dev/null @@ -1,55 +0,0 @@ - - - event_date, event_time, initial_query_id - - - event_date, event_time, query_id - - - diff --git a/tests/integration/test_system_logs/configs/config.d/disks.xml b/tests/integration/test_system_logs/configs/config.d/disks.xml new file mode 100644 index 00000000000..90a1b110326 --- /dev/null +++ b/tests/integration/test_system_logs/configs/config.d/disks.xml @@ -0,0 +1,28 @@ + + + + + /var/lib/clickhouse1/ + + + /var/lib/clickhouse2/ + + + + + + + disk1 + + + + + + + disk2 + + + + + + \ No newline at end of file diff --git a/tests/integration/test_system_logs/configs/system_logs_engine.xml b/tests/integration/test_system_logs/configs/system_logs_engine.xml new file mode 100644 index 00000000000..8afb2b84c63 --- /dev/null +++ b/tests/integration/test_system_logs/configs/system_logs_engine.xml @@ -0,0 +1,5 @@ + + + Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day SETTINGS storage_policy='policy2', ttl_only_drop_parts=1 + + diff --git a/tests/integration/test_system_logs/configs/system_logs_order_by.xml b/tests/integration/test_system_logs/configs/system_logs_order_by.xml new file mode 100644 index 00000000000..1015f53c06c --- /dev/null +++ b/tests/integration/test_system_logs/configs/system_logs_order_by.xml @@ -0,0 +1,8 @@ + + + event_date, event_time, initial_query_id + + + event_date, event_time, query_id + + diff --git a/tests/integration/test_system_logs/configs/system_logs_settings.xml b/tests/integration/test_system_logs/configs/system_logs_settings.xml new file mode 100644 index 00000000000..a219f8b2fee --- /dev/null +++ b/tests/integration/test_system_logs/configs/system_logs_settings.xml @@ -0,0 +1,10 @@ + + + toYYYYMM(event_date) + event_date + INTERVAL 30 DAY DELETE + event_date, event_time, initial_query_id + 7500 + policy1 + storage_policy='policy2', ttl_only_drop_parts=1 + + diff --git a/tests/integration/test_system_logs/test.py b/tests/integration/test_system_logs/test.py deleted file mode 100644 index 252b167a262..00000000000 --- a/tests/integration/test_system_logs/test.py +++ /dev/null @@ -1,34 +0,0 @@ -# pylint: disable=line-too-long -# pylint: disable=unused-argument -# pylint: disable=redefined-outer-name - -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance( - "node_default", - main_configs=["configs/config.d/config.xml"], - stay_alive=True, -) - - -@pytest.fixture(scope="module", autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - -def test_system_logs_order_by_expr(start_cluster): - node.query("SET log_query_threads = 1") - node.query("SELECT count() FROM system.tables") - node.query("SYSTEM FLUSH LOGS") - - # system.query_log - assert node.query("SELECT sorting_key FROM system.tables WHERE database='system' and name='query_log'") - == "event_date, event_time, initial_query_id\n" - # system.query_thread_log - assert node.query("SELECT sorting_key FROM system.tables WHERE database='system' and name='query_thread_log'") - == "event_date, event_time, query_id\n" diff --git a/tests/integration/test_system_logs/test_system_logs.py b/tests/integration/test_system_logs/test_system_logs.py new file mode 100644 index 00000000000..1636e16a1a8 --- /dev/null +++ b/tests/integration/test_system_logs/test_system_logs.py @@ -0,0 +1,73 @@ +# pylint: disable=line-too-long +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", + main_configs=["configs/config_order_by.xml"], + stay_alive=True, +) + +node2 = cluster.add_instance( + "node2", + main_configs=[ + "configs/system_logs_engine.xml", + "configs/config.d/disks.xml", + ], + stay_alive=True, +) + +node3 = cluster.add_instance( + "node3", + main_configs=[ + "configs/system_logs_settings.xml", + "configs/config.d/disks.xml", + ], + stay_alive=True, +) + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_system_logs_order_by_expr(start_cluster): + node1.query("SET log_query_threads = 1") + node1.query("SELECT count() FROM system.tables") + node1.query("SYSTEM FLUSH LOGS") + + # Check 'sorting_key' of system.query_log. + assert node1.query("SELECT sorting_key FROM system.tables WHERE database='system' and name='query_log'") + == "event_date, event_time, initial_query_id\n" + + # Check 'sorting_key' of system.query_thread_log. + assert node1.query("SELECT sorting_key FROM system.tables WHERE database='system' and name='query_thread_log'") + == "event_date, event_time, query_id\n" + + +def test_system_logs_engine_expr(start_cluster): + node2.query("SET log_query_threads = 1") + node2.query("SELECT count() FROM system.tables") + node2.query("SYSTEM FLUSH LOGS") + + # Check 'engine_full' of system.query_log. + expected_result = "MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + toIntervalDay(30) SETTINGS storage_policy = 'policy2', ttl_only_drop_parts = 1" + assert expected_result in node2.query("SELECT engine_full FROM system.tables WHERE database='system' and name='query_log'") + + +def test_system_logs_settings_expr(start_cluster): + node3.query("SET log_query_threads = 1") + node3.query("SELECT count() FROM system.tables") + node3.query("SYSTEM FLUSH LOGS") + + # Check 'engine_full' of system.query_log. + expected_result = "MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time, initial_query_id) TTL event_date + toIntervalDay(30) SETTINGS storage_policy = 'policy1', storage_policy = 'policy2', ttl_only_drop_parts = 1" + assert expected_result in node3.query("SELECT engine_full FROM system.tables WHERE database='system' and name='query_log'") From 780e9df08df4dd07d736e607bfc6cc7bf402fe48 Mon Sep 17 00:00:00 2001 From: helifu Date: Mon, 22 May 2023 17:33:53 +0800 Subject: [PATCH 0749/2223] fix the config file name --- tests/integration/test_system_logs/test_system_logs.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_system_logs/test_system_logs.py b/tests/integration/test_system_logs/test_system_logs.py index 1636e16a1a8..7c5ef813dfa 100644 --- a/tests/integration/test_system_logs/test_system_logs.py +++ b/tests/integration/test_system_logs/test_system_logs.py @@ -9,7 +9,7 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", - main_configs=["configs/config_order_by.xml"], + main_configs=["configs/system_logs_order_by.xml"], stay_alive=True, ) From b2d7a48095e3fc3a36bebce00dd309c14239017e Mon Sep 17 00:00:00 2001 From: helifu Date: Wed, 24 May 2023 09:50:29 +0800 Subject: [PATCH 0750/2223] fix a style checking issue --- .../test_system_logs/test_system_logs.py | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_system_logs/test_system_logs.py b/tests/integration/test_system_logs/test_system_logs.py index 7c5ef813dfa..9e0d0afe76e 100644 --- a/tests/integration/test_system_logs/test_system_logs.py +++ b/tests/integration/test_system_logs/test_system_logs.py @@ -31,6 +31,7 @@ node3 = cluster.add_instance( stay_alive=True, ) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: @@ -39,18 +40,27 @@ def start_cluster(): finally: cluster.shutdown() + def test_system_logs_order_by_expr(start_cluster): node1.query("SET log_query_threads = 1") node1.query("SELECT count() FROM system.tables") node1.query("SYSTEM FLUSH LOGS") # Check 'sorting_key' of system.query_log. - assert node1.query("SELECT sorting_key FROM system.tables WHERE database='system' and name='query_log'") + assert ( + node1.query( + "SELECT sorting_key FROM system.tables WHERE database='system' and name='query_log'" + ) == "event_date, event_time, initial_query_id\n" + ) # Check 'sorting_key' of system.query_thread_log. - assert node1.query("SELECT sorting_key FROM system.tables WHERE database='system' and name='query_thread_log'") + assert ( + node1.query( + "SELECT sorting_key FROM system.tables WHERE database='system' and name='query_thread_log'" + ) == "event_date, event_time, query_id\n" + ) def test_system_logs_engine_expr(start_cluster): @@ -60,7 +70,9 @@ def test_system_logs_engine_expr(start_cluster): # Check 'engine_full' of system.query_log. expected_result = "MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + toIntervalDay(30) SETTINGS storage_policy = 'policy2', ttl_only_drop_parts = 1" - assert expected_result in node2.query("SELECT engine_full FROM system.tables WHERE database='system' and name='query_log'") + assert expected_result in node2.query( + "SELECT engine_full FROM system.tables WHERE database='system' and name='query_log'" + ) def test_system_logs_settings_expr(start_cluster): @@ -70,4 +82,6 @@ def test_system_logs_settings_expr(start_cluster): # Check 'engine_full' of system.query_log. expected_result = "MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time, initial_query_id) TTL event_date + toIntervalDay(30) SETTINGS storage_policy = 'policy1', storage_policy = 'policy2', ttl_only_drop_parts = 1" - assert expected_result in node3.query("SELECT engine_full FROM system.tables WHERE database='system' and name='query_log'") + assert expected_result in node3.query( + "SELECT engine_full FROM system.tables WHERE database='system' and name='query_log'" + ) From e98cd5d44c9afaaf4fe581186b7854f442aa3ccd Mon Sep 17 00:00:00 2001 From: helifu Date: Thu, 25 May 2023 16:58:58 +0800 Subject: [PATCH 0751/2223] Fix failed test cases test_system_logs/test_system_logs.py::test_system_logs_order_by_expr PASSED [ 33%] test_system_logs/test_system_logs.py::test_system_logs_engine_expr PASSED [ 66%] test_system_logs/test_system_logs.py::test_system_logs_settings_expr PASSED [100%] --- .../integration/test_system_logs/test_system_logs.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_system_logs/test_system_logs.py b/tests/integration/test_system_logs/test_system_logs.py index 9e0d0afe76e..dcbbe0db93b 100644 --- a/tests/integration/test_system_logs/test_system_logs.py +++ b/tests/integration/test_system_logs/test_system_logs.py @@ -9,7 +9,9 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", - main_configs=["configs/system_logs_order_by.xml"], + main_configs=[ + "configs/system_logs_order_by.xml" + ], stay_alive=True, ) @@ -69,8 +71,8 @@ def test_system_logs_engine_expr(start_cluster): node2.query("SYSTEM FLUSH LOGS") # Check 'engine_full' of system.query_log. - expected_result = "MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + toIntervalDay(30) SETTINGS storage_policy = 'policy2', ttl_only_drop_parts = 1" - assert expected_result in node2.query( + expected = "MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + toIntervalDay(30) SETTINGS storage_policy = \\'policy2\\', ttl_only_drop_parts = 1" + assert expected in node2.query( "SELECT engine_full FROM system.tables WHERE database='system' and name='query_log'" ) @@ -81,7 +83,7 @@ def test_system_logs_settings_expr(start_cluster): node3.query("SYSTEM FLUSH LOGS") # Check 'engine_full' of system.query_log. - expected_result = "MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time, initial_query_id) TTL event_date + toIntervalDay(30) SETTINGS storage_policy = 'policy1', storage_policy = 'policy2', ttl_only_drop_parts = 1" - assert expected_result in node3.query( + expected = "MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time, initial_query_id) TTL event_date + toIntervalDay(30) SETTINGS storage_policy = \\'policy1\\', storage_policy = \\'policy2\\', ttl_only_drop_parts = 1" + assert expected in node3.query( "SELECT engine_full FROM system.tables WHERE database='system' and name='query_log'" ) From 1d1d84497ef2e839e7332c4c0c9ab885b785e63b Mon Sep 17 00:00:00 2001 From: helifu Date: Thu, 25 May 2023 17:45:28 +0800 Subject: [PATCH 0752/2223] Fix a style checking issue --- tests/integration/test_system_logs/test_system_logs.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_system_logs/test_system_logs.py b/tests/integration/test_system_logs/test_system_logs.py index dcbbe0db93b..d6bca58cae9 100644 --- a/tests/integration/test_system_logs/test_system_logs.py +++ b/tests/integration/test_system_logs/test_system_logs.py @@ -9,9 +9,7 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", - main_configs=[ - "configs/system_logs_order_by.xml" - ], + main_configs=["configs/system_logs_order_by.xml"], stay_alive=True, ) From 6a64fa6c6c7e69189cf35e2dd3573468b0d7ce99 Mon Sep 17 00:00:00 2001 From: helifu Date: Fri, 26 May 2023 17:04:11 +0800 Subject: [PATCH 0753/2223] Node2 cannot be started due to a conflict with config.xml merge --- .../configs/{ => config.d}/system_logs_engine.xml | 0 .../configs/{ => config.d}/system_logs_order_by.xml | 0 .../configs/{ => config.d}/system_logs_settings.xml | 0 tests/integration/test_system_logs/configs/config.xml | 9 +++++++++ tests/integration/test_system_logs/test_system_logs.py | 10 +++++++--- 5 files changed, 16 insertions(+), 3 deletions(-) rename tests/integration/test_system_logs/configs/{ => config.d}/system_logs_engine.xml (100%) rename tests/integration/test_system_logs/configs/{ => config.d}/system_logs_order_by.xml (100%) rename tests/integration/test_system_logs/configs/{ => config.d}/system_logs_settings.xml (100%) create mode 100644 tests/integration/test_system_logs/configs/config.xml diff --git a/tests/integration/test_system_logs/configs/system_logs_engine.xml b/tests/integration/test_system_logs/configs/config.d/system_logs_engine.xml similarity index 100% rename from tests/integration/test_system_logs/configs/system_logs_engine.xml rename to tests/integration/test_system_logs/configs/config.d/system_logs_engine.xml diff --git a/tests/integration/test_system_logs/configs/system_logs_order_by.xml b/tests/integration/test_system_logs/configs/config.d/system_logs_order_by.xml similarity index 100% rename from tests/integration/test_system_logs/configs/system_logs_order_by.xml rename to tests/integration/test_system_logs/configs/config.d/system_logs_order_by.xml diff --git a/tests/integration/test_system_logs/configs/system_logs_settings.xml b/tests/integration/test_system_logs/configs/config.d/system_logs_settings.xml similarity index 100% rename from tests/integration/test_system_logs/configs/system_logs_settings.xml rename to tests/integration/test_system_logs/configs/config.d/system_logs_settings.xml diff --git a/tests/integration/test_system_logs/configs/config.xml b/tests/integration/test_system_logs/configs/config.xml new file mode 100644 index 00000000000..0b5d4673804 --- /dev/null +++ b/tests/integration/test_system_logs/configs/config.xml @@ -0,0 +1,9 @@ + + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + + diff --git a/tests/integration/test_system_logs/test_system_logs.py b/tests/integration/test_system_logs/test_system_logs.py index d6bca58cae9..b3070e00c25 100644 --- a/tests/integration/test_system_logs/test_system_logs.py +++ b/tests/integration/test_system_logs/test_system_logs.py @@ -9,14 +9,17 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", - main_configs=["configs/system_logs_order_by.xml"], + main_configs=[ + "configs/config.xml", + "configs/config.d/system_logs_order_by.xml"], stay_alive=True, ) node2 = cluster.add_instance( "node2", main_configs=[ - "configs/system_logs_engine.xml", + "configs/config.xml", + "configs/config.d/system_logs_engine.xml", "configs/config.d/disks.xml", ], stay_alive=True, @@ -25,7 +28,8 @@ node2 = cluster.add_instance( node3 = cluster.add_instance( "node3", main_configs=[ - "configs/system_logs_settings.xml", + "configs/config.xml", + "configs/config.d/system_logs_settings.xml", "configs/config.d/disks.xml", ], stay_alive=True, From 7c3fa3658d608ee98162e61cf728c950e8e8d453 Mon Sep 17 00:00:00 2001 From: helifu Date: Fri, 26 May 2023 17:30:07 +0800 Subject: [PATCH 0754/2223] fix a style checking issue --- tests/integration/test_system_logs/test_system_logs.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_system_logs/test_system_logs.py b/tests/integration/test_system_logs/test_system_logs.py index b3070e00c25..134a2990c45 100644 --- a/tests/integration/test_system_logs/test_system_logs.py +++ b/tests/integration/test_system_logs/test_system_logs.py @@ -9,9 +9,7 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", - main_configs=[ - "configs/config.xml", - "configs/config.d/system_logs_order_by.xml"], + main_configs=["configs/config.xml", "configs/config.d/system_logs_order_by.xml"], stay_alive=True, ) From e9ffc383da0e544cc01bbad7717dae3f566e95f5 Mon Sep 17 00:00:00 2001 From: helifu Date: Sat, 27 May 2023 22:56:00 +0800 Subject: [PATCH 0755/2223] Add input parameter 'base_config_dir' --- .../test_system_logs/configs/config.xml | 6 +++ .../test_system_logs/configs/users.xml | 38 +++++++++++++++++++ .../test_system_logs/test_system_logs.py | 7 ++-- 3 files changed, 48 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_system_logs/configs/users.xml diff --git a/tests/integration/test_system_logs/configs/config.xml b/tests/integration/test_system_logs/configs/config.xml index 0b5d4673804..e972f199568 100644 --- a/tests/integration/test_system_logs/configs/config.xml +++ b/tests/integration/test_system_logs/configs/config.xml @@ -6,4 +6,10 @@ 1000M 10 + + 8123 + 9000 + 127.0.0.1 + 9009 + 127.0.0.1 diff --git a/tests/integration/test_system_logs/configs/users.xml b/tests/integration/test_system_logs/configs/users.xml new file mode 100644 index 00000000000..1c6cd38a05c --- /dev/null +++ b/tests/integration/test_system_logs/configs/users.xml @@ -0,0 +1,38 @@ + + + + 10000000000 + 64999 + random + + + + 1 + + + + + + + + ::/0 + + default + + default + + + + + + + 3600 + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/tests/integration/test_system_logs/test_system_logs.py b/tests/integration/test_system_logs/test_system_logs.py index 134a2990c45..aac5ee53819 100644 --- a/tests/integration/test_system_logs/test_system_logs.py +++ b/tests/integration/test_system_logs/test_system_logs.py @@ -9,14 +9,15 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", - main_configs=["configs/config.xml", "configs/config.d/system_logs_order_by.xml"], + base_config_dir="configs", + main_configs=["configs/config.d/system_logs_order_by.xml"], stay_alive=True, ) node2 = cluster.add_instance( "node2", + base_config_dir="configs", main_configs=[ - "configs/config.xml", "configs/config.d/system_logs_engine.xml", "configs/config.d/disks.xml", ], @@ -25,8 +26,8 @@ node2 = cluster.add_instance( node3 = cluster.add_instance( "node3", + base_config_dir="configs", main_configs=[ - "configs/config.xml", "configs/config.d/system_logs_settings.xml", "configs/config.d/disks.xml", ], From ec82c657eb13b6998deefc94469a8f17c9183c4e Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 28 May 2023 06:04:23 +0000 Subject: [PATCH 0756/2223] Parallel merge of uniqExactIf states --- src/AggregateFunctions/AggregateFunctionIf.h | 7 +++++++ tests/performance/uniqExactIf.xml | 3 +++ 2 files changed, 10 insertions(+) create mode 100644 tests/performance/uniqExactIf.xml diff --git a/src/AggregateFunctions/AggregateFunctionIf.h b/src/AggregateFunctions/AggregateFunctionIf.h index 47f04dad9a4..cd7d7e27a25 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.h +++ b/src/AggregateFunctions/AggregateFunctionIf.h @@ -152,6 +152,13 @@ public: nested_func->merge(place, rhs, arena); } + bool isAbleToParallelizeMerge() const override { return nested_func->isAbleToParallelizeMerge(); } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override + { + nested_func->merge(place, rhs, thread_pool, arena); + } + void mergeBatch( size_t row_begin, size_t row_end, diff --git a/tests/performance/uniqExactIf.xml b/tests/performance/uniqExactIf.xml new file mode 100644 index 00000000000..24899660177 --- /dev/null +++ b/tests/performance/uniqExactIf.xml @@ -0,0 +1,3 @@ + + SELECT uniqExactIf(number, ) FROM numbers_mt(1e7) + From 65c34e73e7010bfb5c6298ed2a5bedcc434e7aa7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sun, 28 May 2023 15:45:40 +0200 Subject: [PATCH 0757/2223] Clean --- src/Common/checkSSLError.h | 8 -------- 1 file changed, 8 deletions(-) delete mode 100644 src/Common/checkSSLError.h diff --git a/src/Common/checkSSLError.h b/src/Common/checkSSLError.h deleted file mode 100644 index 05bca9f8b5f..00000000000 --- a/src/Common/checkSSLError.h +++ /dev/null @@ -1,8 +0,0 @@ -// -// Created by Павел Круглов on 27/05/2023. -// - -#ifndef CLICKHOUSE_CHECKSSLERROR_H -#define CLICKHOUSE_CHECKSSLERROR_H - -#endif //CLICKHOUSE_CHECKSSLERROR_H From 3aa795355f946e6638804124b03d0724bf1ba37b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sun, 28 May 2023 15:46:32 +0200 Subject: [PATCH 0758/2223] Fix build --- src/Common/checkSSLReturnCode.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/checkSSLReturnCode.h b/src/Common/checkSSLReturnCode.h index f30564137aa..77e51625daf 100644 --- a/src/Common/checkSSLReturnCode.h +++ b/src/Common/checkSSLReturnCode.h @@ -1,4 +1,5 @@ #pragma once +#include namespace DB { From 7ef20bbdcd2561e9569e4bfc0ae0a7a687a0dc73 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 28 May 2023 17:02:46 +0300 Subject: [PATCH 0759/2223] Function equals NaN fix --- src/Functions/FunctionsComparison.h | 7 +++++-- .../02769_nan_equality_comparison.reference | 4 ++++ .../02769_nan_equality_comparison.sql | 20 +++++++++++++++++++ 3 files changed, 29 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02769_nan_equality_comparison.reference create mode 100644 tests/queries/0_stateless/02769_nan_equality_comparison.sql diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index a9c8bbaa183..08bc350c1d4 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1230,8 +1230,11 @@ public: /// The case when arguments are the same (tautological comparison). Return constant. /// NOTE: Nullable types are special case. /// (BTW, this function use default implementation for Nullable, so Nullable types cannot be here. Check just in case.) - /// NOTE: We consider NaN comparison to be implementation specific (and in our implementation NaNs are sometimes equal sometimes not). - if (left_type->equals(*right_type) && !left_type->isNullable() && !isTuple(left_type) && col_left_untyped == col_right_untyped) + if (left_type->equals(*right_type) && + !left_type->isNullable() && + !isTuple(left_type) && + !WhichDataType(left_type).isFloat() && + col_left_untyped == col_right_untyped) { ColumnPtr result_column; diff --git a/tests/queries/0_stateless/02769_nan_equality_comparison.reference b/tests/queries/0_stateless/02769_nan_equality_comparison.reference new file mode 100644 index 00000000000..edeb7a37e34 --- /dev/null +++ b/tests/queries/0_stateless/02769_nan_equality_comparison.reference @@ -0,0 +1,4 @@ +nan 0 0 0 0 +nan 0 0 0 0 +nan nan 0 0 0 0 +-- diff --git a/tests/queries/0_stateless/02769_nan_equality_comparison.sql b/tests/queries/0_stateless/02769_nan_equality_comparison.sql new file mode 100644 index 00000000000..ae9082230bb --- /dev/null +++ b/tests/queries/0_stateless/02769_nan_equality_comparison.sql @@ -0,0 +1,20 @@ +SET compile_expressions = 1; +SET min_count_to_compile_expression = 0; + +SELECT nan AS value, value = value, value = materialize(value), materialize(value) = value, materialize(value) = materialize(value); +SELECT cast(nan, 'Float32') AS value, value = value, value = materialize(value), materialize(value) = value, materialize(value) = materialize(value); +SELECT nan AS lhs, cast(nan, 'Float32') AS rhs, lhs = rhs, lhs = materialize(rhs), materialize(lhs) = rhs, materialize(lhs) = materialize(rhs); + +SELECT '--'; + +CREATE TABLE test_table +( + id UInt32, + value UInt32 +) ENGINE = MergeTree ORDER BY id; +INSERT INTO test_table VALUES (76, 57); + +SELECT value FROM (SELECT stddevSamp(id) AS value FROM test_table) as subquery +WHERE ((value = value) AND (NOT (value = value))); + +DROP TABLE test_table; From 05783f99cdbee81d1a47a7b2772c41271a007ab0 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 28 May 2023 14:17:59 +0000 Subject: [PATCH 0760/2223] update test --- tests/performance/uniqExactIf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/uniqExactIf.xml b/tests/performance/uniqExactIf.xml index 24899660177..c79f9f000bb 100644 --- a/tests/performance/uniqExactIf.xml +++ b/tests/performance/uniqExactIf.xml @@ -1,3 +1,3 @@ - SELECT uniqExactIf(number, ) FROM numbers_mt(1e7) + SELECT uniqExactIf(number, ) FROM numbers_mt(1e6) From c970266084574269bb26bcfe3544e6ca102973b9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 28 May 2023 17:35:29 +0300 Subject: [PATCH 0761/2223] Updated tests --- .../02769_nan_equality_comparison.reference | 6 +++++ .../02769_nan_equality_comparison.sql | 22 +++++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/tests/queries/0_stateless/02769_nan_equality_comparison.reference b/tests/queries/0_stateless/02769_nan_equality_comparison.reference index edeb7a37e34..a8ba06cfce6 100644 --- a/tests/queries/0_stateless/02769_nan_equality_comparison.reference +++ b/tests/queries/0_stateless/02769_nan_equality_comparison.reference @@ -2,3 +2,9 @@ nan 0 0 0 0 nan 0 0 0 0 nan nan 0 0 0 0 -- +-- +nan 1 1 1 1 +nan 1 1 1 1 +nan nan 1 1 1 1 +-- +nan diff --git a/tests/queries/0_stateless/02769_nan_equality_comparison.sql b/tests/queries/0_stateless/02769_nan_equality_comparison.sql index ae9082230bb..6cce19a2204 100644 --- a/tests/queries/0_stateless/02769_nan_equality_comparison.sql +++ b/tests/queries/0_stateless/02769_nan_equality_comparison.sql @@ -18,3 +18,25 @@ SELECT value FROM (SELECT stddevSamp(id) AS value FROM test_table) as subquery WHERE ((value = value) AND (NOT (value = value))); DROP TABLE test_table; + +SELECT '--'; + +SELECT nan AS value, value != value, value != materialize(value), materialize(value) != value, materialize(value) != materialize(value); +SELECT cast(nan, 'Float32') AS value, value != value, value != materialize(value), materialize(value) != value, materialize(value) != materialize(value); +SELECT nan AS lhs, cast(nan, 'Float32') AS rhs, lhs != rhs, lhs != materialize(rhs), materialize(lhs) != rhs, materialize(lhs) != materialize(rhs); + +SELECT '--'; + +CREATE TABLE test_table +( + id UInt32, + value_1 UInt32, + value_2 Float32 +) ENGINE = MergeTree ORDER BY id; +INSERT INTO test_table VALUES (12000, 36, 77.94); + +SELECT value +FROM (SELECT (corr(value_1, value_1) OVER test_window) AS value FROM test_table WINDOW test_window AS (PARTITION BY value_2 ORDER BY id ASC)) as subquery +WHERE not (not (value <> value)); + +DROP TABLE test_table; From e0563b95f254326a74b06c940e955983b155ef7b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 28 May 2023 18:18:59 +0200 Subject: [PATCH 0762/2223] Fix 02767_into_outfile_extensions_msan under analyzer Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh index d1e7312b150..0c5767314d5 100755 --- a/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh +++ b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh @@ -6,6 +6,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) out="explain1.$CLICKHOUSE_TEST_UNIQUE_NAME.out" # only EXPLAIN triggers the problem under MSan -$CLICKHOUSE_CLIENT -q "explain select * from numbers(1) into outfile '$out'" +$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q "explain select * from numbers(1) into outfile '$out'" cat "$out" rm -f "$out" From e9840bc6e190270ad1e9e86c6cca45375418fe24 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 28 May 2023 21:05:17 +0300 Subject: [PATCH 0763/2223] JIT aggregation nullable key fix --- src/Interpreters/Aggregator.cpp | 3 +- ...jit_aggregation_nullable_key_fix.reference | 4 ++ ...02770_jit_aggregation_nullable_key_fix.sql | 39 +++++++++++++++++++ 3 files changed, 45 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02770_jit_aggregation_nullable_key_fix.reference create mode 100644 tests/queries/0_stateless/02770_jit_aggregation_nullable_key_fix.sql diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 7df36e8600a..c7d4b87694b 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2041,7 +2041,7 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena */ if (data.hasNullKeyData()) { - has_null_key_data = Method::one_key_nullable_optimization; + has_null_key_data = true; out_cols->key_columns[0]->insertDefault(); insertAggregatesIntoColumns(data.getNullKeyData(), out_cols->final_aggregate_columns, arena); data.hasNullKeyData() = false; @@ -2076,6 +2076,7 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena, has_null_key_data)); places.clear(); out_cols.reset(); + has_null_key_data = false; } } }); diff --git a/tests/queries/0_stateless/02770_jit_aggregation_nullable_key_fix.reference b/tests/queries/0_stateless/02770_jit_aggregation_nullable_key_fix.reference new file mode 100644 index 00000000000..7d604c076f4 --- /dev/null +++ b/tests/queries/0_stateless/02770_jit_aggregation_nullable_key_fix.reference @@ -0,0 +1,4 @@ +1048576 +65411 +1048576 +65411 diff --git a/tests/queries/0_stateless/02770_jit_aggregation_nullable_key_fix.sql b/tests/queries/0_stateless/02770_jit_aggregation_nullable_key_fix.sql new file mode 100644 index 00000000000..e4ce789f4da --- /dev/null +++ b/tests/queries/0_stateless/02770_jit_aggregation_nullable_key_fix.sql @@ -0,0 +1,39 @@ +SET compile_aggregate_expressions = 1; +SET min_count_to_compile_aggregate_expression = 0; +SET group_by_use_nulls = 0; + +SELECT count() FROM +( + SELECT + count([NULL, NULL]), + count([2147483646, -2147483647, 3, 3]), + uniqExact(if(number >= 1048577, number, NULL), NULL) + FROM numbers(1048577) + GROUP BY if(number >= 2., number, NULL) +); + +SELECT count() FROM +( + SELECT count() + FROM numbers(65411) + GROUP BY if(number < 1, NULL, number) +); + +SET group_by_use_nulls = 1; + +SELECT count() FROM +( + SELECT + count([NULL, NULL]), + count([2147483646, -2147483647, 3, 3]), + uniqExact(if(number >= 1048577, number, NULL), NULL) + FROM numbers(1048577) + GROUP BY if(number >= 2., number, NULL) +); + +SELECT count() FROM +( + SELECT count() + FROM numbers(65411) + GROUP BY if(number < 1, NULL, number) +); From 8574559e233b9319ffd6e15d317923fb1a84d22f Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sun, 28 May 2023 22:25:40 +0000 Subject: [PATCH 0764/2223] Disallow usage of --query and --queries-file --- programs/client/Client.cpp | 3 +++ programs/local/LocalServer.cpp | 6 ++--- ...ent_local_queries_file_parameter.reference | 4 ++++ ...523_client_local_queries_file_parameter.sh | 23 +++++++++++++------ .../02751_multiquery_with_argument.reference | 2 ++ .../02751_multiquery_with_argument.sh | 4 ++++ 6 files changed, 32 insertions(+), 10 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 5870327c3b5..32a07284d26 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1180,6 +1180,9 @@ void Client::processOptions(const OptionsDescription & options_description, void Client::processConfig() { + if (config().has("query") && config().has("queries-file")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Options '--query' and '--queries-file' cannot be specified at the same time"); + /// Batch mode is enabled if one of the following is true: /// - -q (--query) command line option is present. /// The value of the option is used as the text of query (or of multiple queries). diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e026f87279a..944a60d4e4c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -516,12 +516,12 @@ void LocalServer::updateLoggerLevel(const String & logs_level) void LocalServer::processConfig() { + if (config().has("query") && config().has("queries-file")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Options '--query' and '--queries-file' cannot be specified at the same time"); + delayed_interactive = config().has("interactive") && (config().has("query") || config().has("queries-file")); if (is_interactive && !delayed_interactive) { - if (config().has("query") && config().has("queries-file")) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specify either `query` or `queries-file` option"); - if (config().has("multiquery")) is_multiquery = true; } diff --git a/tests/queries/0_stateless/01523_client_local_queries_file_parameter.reference b/tests/queries/0_stateless/01523_client_local_queries_file_parameter.reference index 2a3eb745751..634bee3d288 100644 --- a/tests/queries/0_stateless/01523_client_local_queries_file_parameter.reference +++ b/tests/queries/0_stateless/01523_client_local_queries_file_parameter.reference @@ -4,3 +4,7 @@ 3 1 2 3 4 +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS diff --git a/tests/queries/0_stateless/01523_client_local_queries_file_parameter.sh b/tests/queries/0_stateless/01523_client_local_queries_file_parameter.sh index cf19efa9b7d..17dc04d7571 100755 --- a/tests/queries/0_stateless/01523_client_local_queries_file_parameter.sh +++ b/tests/queries/0_stateless/01523_client_local_queries_file_parameter.sh @@ -5,20 +5,29 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo "SELECT 1;" > 01523_client_local_queries_file_parameter_tmp.sql -$CLICKHOUSE_CLIENT --queries-file=01523_client_local_queries_file_parameter_tmp.sql 2>&1 +## Use process ID ($$) for uniqueness of file name +TEMP_SQL_FILE_NAME=$"01523_client_local_queries_file_parameter_tmp_$$.sql" +echo "SELECT 1;" > "$TEMP_SQL_FILE_NAME" +$CLICKHOUSE_CLIENT --queries-file="$TEMP_SQL_FILE_NAME" 2>&1 echo "CREATE TABLE 01523_test(value Int32) ENGINE=Log; INSERT INTO 01523_test VALUES (1), (2), (3); SELECT * FROM 01523_test; -DROP TABLE 01523_test;" > 01523_client_local_queries_file_parameter_tmp.sql -$CLICKHOUSE_CLIENT --queries-file=01523_client_local_queries_file_parameter_tmp.sql 2>&1 +DROP TABLE 01523_test;" > "$TEMP_SQL_FILE_NAME" +$CLICKHOUSE_CLIENT --queries-file="$TEMP_SQL_FILE_NAME" 2>&1 echo "CREATE TABLE 01523_test (a Int64, b Int64) ENGINE = File(CSV, stdin); SELECT a, b FROM 01523_test; -DROP TABLE 01523_test;" > 01523_client_local_queries_file_parameter_tmp.sql +DROP TABLE 01523_test;" > "$TEMP_SQL_FILE_NAME" -echo -e "1,2\n3,4" | $CLICKHOUSE_LOCAL --queries-file=01523_client_local_queries_file_parameter_tmp.sql 2>&1 +echo -e "1,2\n3,4" | $CLICKHOUSE_LOCAL --queries-file="$TEMP_SQL_FILE_NAME" 2>&1 -rm 01523_client_local_queries_file_parameter_tmp.sql +# Simultaneously passing --queries-file + --query is prohibited. +echo "SELECT 1;" > "$TEMP_SQL_FILE_NAME" +$CLICKHOUSE_LOCAL --queries-file="$TEMP_SQL_FILE_NAME" -q "SELECT 1;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_CLIENT --queries-file="$TEMP_SQL_FILE_NAME" -q "SELECT 2;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_LOCAL --queries-file="$TEMP_SQL_FILE_NAME" --query "SELECT 3;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_CLIENT --queries-file="$TEMP_SQL_FILE_NAME" --query "SELECT 4;" 2>&1 | grep -o 'BAD_ARGUMENTS' + +rm "$TEMP_SQL_FILE_NAME" diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.reference b/tests/queries/0_stateless/02751_multiquery_with_argument.reference index 33288ec5bcb..df9771b6bd3 100644 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.reference +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.reference @@ -8,6 +8,8 @@ Empty query BAD_ARGUMENTS BAD_ARGUMENTS BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS Bad arguments Bad arguments Bad arguments diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.sh b/tests/queries/0_stateless/02751_multiquery_with_argument.sh index 153004c752e..ce53ede3331 100755 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.sh +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.sh @@ -13,6 +13,10 @@ $CLICKHOUSE_LOCAL --multiquery "SELECT 200; S" 2>&1 | grep -o 'Syntax error' $CLICKHOUSE_LOCAL --multiquery "; SELECT 201;" 2>&1 | grep -o 'Empty query' $CLICKHOUSE_LOCAL --multiquery "; S; SELECT 202" 2>&1 | grep -o 'Empty query' +# Simultaneously passing --queries-file + --query (multiquery) is prohibited. +$CLICKHOUSE_LOCAL --queries-file "queries.csv" --multiquery "SELECT 250;" 2>&1 | grep -o 'BAD_ARGUMENTS' +$CLICKHOUSE_CLIENT --queries-file "queries.csv" --multiquery "SELECT 251;" 2>&1 | grep -o 'BAD_ARGUMENTS' + # Error expectation cases. # -n is prohibited $CLICKHOUSE_LOCAL -n "SELECT 301" 2>&1 | grep -o 'BAD_ARGUMENTS' From f616314f8b68b88d249d778b4eff8ea71a99953e Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 29 May 2023 02:22:13 +0000 Subject: [PATCH 0765/2223] fix typo --- tests/performance/uniqExactIf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/uniqExactIf.xml b/tests/performance/uniqExactIf.xml index c79f9f000bb..409be257737 100644 --- a/tests/performance/uniqExactIf.xml +++ b/tests/performance/uniqExactIf.xml @@ -1,3 +1,3 @@ - SELECT uniqExactIf(number, ) FROM numbers_mt(1e6) + SELECT uniqExactIf(number, 1) FROM numbers_mt(1e6) From 18219339959460393dfa567a44394be7bf897a10 Mon Sep 17 00:00:00 2001 From: helifu Date: Mon, 29 May 2023 15:12:16 +0800 Subject: [PATCH 0766/2223] Fix a typo --- tests/integration/test_storage_postgresql/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 0a3a01101d5..e9ddb0f73a5 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -183,7 +183,7 @@ def test_postgres_conversions(started_cluster): cursor.execute(f"DROP TABLE test_array_dimensions") -def test_non_default_scema(started_cluster): +def test_non_default_schema(started_cluster): node1.query("DROP TABLE IF EXISTS test_pg_table_schema") node1.query("DROP TABLE IF EXISTS test_pg_table_schema_with_dots") From f0ed55bac40e1e29d1d04447ebbbd95291143ec9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 May 2023 07:43:40 +0000 Subject: [PATCH 0767/2223] QPL: Add a comment about isal --- contrib/qpl-cmake/CMakeLists.txt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 21d665d12e4..4e6c66fe731 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -25,6 +25,9 @@ message(STATUS "Intel QPL version: ${QPL_VERSION}") # Generate 8 library targets: middle_layer_lib, isal, isal_asm, qplcore_px, qplcore_avx512, qplcore_sw_dispatcher, core_iaa, middle_layer_lib. # Output ch_contrib::qpl by linking with 8 library targets. +# The qpl submodule comes with its own version of isal. It contains code which does not exist in upstream isal. It would be nice to link +# only upstream isal (ch_contrib::isal) but at this point we can't. + include("${QPL_PROJECT_DIR}/cmake/CompileOptions.cmake") # check nasm compiler @@ -308,7 +311,7 @@ target_include_directories(middle_layer_lib target_compile_definitions(middle_layer_lib PUBLIC -DQPL_LIB) # [SUBDIR]c_api -file(GLOB_RECURSE QPL_C_API_SRC +file(GLOB_RECURSE QPL_C_API_SRC ${QPL_SRC_DIR}/c_api/*.c ${QPL_SRC_DIR}/c_api/*.cpp) From 503105d4b1f1fb66c7f440478a8e4492dd66280d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 May 2023 08:07:46 +0000 Subject: [PATCH 0768/2223] Fix 02415_all_new_functions_must_be_documented --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index f2aed15e367..b2c9073648e 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -559,6 +559,7 @@ positionCaseInsensitive positionCaseInsensitiveUTF8 positionUTF8 pow +prepareTrainEntropyLearnedHash proportionsZTest protocol queryID From 41d60f0be3209663fa8659fdb22fa7ea042fa5fe Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 May 2023 08:08:47 +0000 Subject: [PATCH 0769/2223] Fix style --- src/Functions/EntropyLearnedHash.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp index a80a8fe0228..854379bbb9d 100644 --- a/src/Functions/EntropyLearnedHash.cpp +++ b/src/Functions/EntropyLearnedHash.cpp @@ -219,7 +219,7 @@ public: static constexpr auto name = "prepareTrainEntropyLearnedHash"; static FunctionPtr create(ContextPtr context) { - if(!context->getSettings().allow_experimental_hash_functions) + if (!context->getSettings().allow_experimental_hash_functions) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Entropy-learned hashing is experimental. Set `allow_experimental_hash_functions` setting to enable it"); @@ -274,7 +274,7 @@ public: static constexpr auto name = "trainEntropyLearnedHash"; static FunctionPtr create(ContextPtr context) { - if(!context->getSettings().allow_experimental_hash_functions) + if (!context->getSettings().allow_experimental_hash_functions) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Entropy-learned hashing is experimental. Set `allow_experimental_hash_functions` setting to enable it"); return std::make_shared(context->getUserName()); @@ -325,7 +325,7 @@ public: static constexpr auto name = "entropyLearnedHash"; static FunctionPtr create(ContextPtr context) { - if(!context->getSettings().allow_experimental_hash_functions) + if (!context->getSettings().allow_experimental_hash_functions) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Entropy-learned hashing experimental. Set `allow_experimental_hash_functions` setting to enable it"); return std::make_shared(context->getUserName()); From 1ae1f1273a1b915264bbf3f53f2f255650c9304a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 29 May 2023 11:40:45 +0200 Subject: [PATCH 0770/2223] Avoid clang 15 crash --- src/Coordination/KeeperServer.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b07c2edc53a..45db9e85fa5 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -631,14 +631,12 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ request_for_session->digest = state_machine->getNodesDigest(); - using enum KeeperStateMachine::ZooKeeperLogSerializationVersion; - /// older versions of Keeper can send logs that are missing some fields size_t bytes_missing = 0; - if (serialization_version < WITH_TIME) + if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME) bytes_missing += sizeof(request_for_session->time); - if (serialization_version < WITH_ZXID_DIGEST) + if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_ZXID_DIGEST) bytes_missing += sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value); if (bytes_missing != 0) @@ -652,14 +650,14 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ size_t write_buffer_header_size = sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value); - if (serialization_version < WITH_TIME) + if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME) write_buffer_header_size += sizeof(request_for_session->time); auto * buffer_start = reinterpret_cast(entry_buf->data_begin() + entry_buf->size() - write_buffer_header_size); WriteBuffer write_buf(buffer_start, write_buffer_header_size); - if (serialization_version < WITH_TIME) + if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME) writeIntBinary(request_for_session->time, write_buf); writeIntBinary(request_for_session->zxid, write_buf); From bdb974ab3c6e4f425ecca35424ff934f13f59677 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 29 May 2023 11:21:44 +0000 Subject: [PATCH 0771/2223] Fix flaky `AsyncLoader.StaticPriorities` unit test --- src/Common/tests/gtest_async_loader.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index a5cdde881fd..083b5bd97a4 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -616,6 +616,15 @@ TEST(AsyncLoader, StaticPriorities) schedule += fmt::format("{}{}", self->name, self->pool()); }; + // Job DAG with priorities. After priority inheritance from H9, jobs D9 and E9 can be + // executed in undefined order (Tested further in DynamicPriorities) + // A0(9) -+-> B3 + // | + // `-> C4 + // | + // `-> D1(9) -. + // | +-> F0(9) --> G0(9) --> H9 + // `-> E2(9) -' std::vector jobs; jobs.push_back(makeLoadJob({}, 0, "A", job_func)); // 0 jobs.push_back(makeLoadJob({ jobs[0] }, 3, "B", job_func)); // 1 @@ -629,8 +638,7 @@ TEST(AsyncLoader, StaticPriorities) t.loader.start(); t.loader.wait(); - - ASSERT_EQ(schedule, "A9E9D9F9G9H9C4B3"); + ASSERT(schedule == "A9E9D9F9G9H9C4B3" || schedule == "A9D9E9F9G9H9C4B3" ); } TEST(AsyncLoader, SimplePrioritization) From 355f917daa800919d78a1b380f9a5366aefb4c6c Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 29 May 2023 13:38:14 +0200 Subject: [PATCH 0772/2223] Update src/Common/tests/gtest_async_loader.cpp --- src/Common/tests/gtest_async_loader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 083b5bd97a4..bfc305de9f8 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -638,7 +638,7 @@ TEST(AsyncLoader, StaticPriorities) t.loader.start(); t.loader.wait(); - ASSERT(schedule == "A9E9D9F9G9H9C4B3" || schedule == "A9D9E9F9G9H9C4B3" ); + ASSERT(schedule == "A9E9D9F9G9H9C4B3" || schedule == "A9D9E9F9G9H9C4B3"); } TEST(AsyncLoader, SimplePrioritization) From be62c8faba774c8c594a6d0368efba0c7a540576 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 May 2023 15:27:39 +0200 Subject: [PATCH 0773/2223] fix build --- src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 05e6dad157b..ba8c2c6385f 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -82,7 +82,7 @@ MergeTreeReadPool::MergeTreeReadPool( const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading; const auto avg_mark_bytes = std::max(total_compressed_bytes / total_marks, 1); /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. - const auto heuristic_min_marks = std::min(total_marks / threads_, min_bytes_per_task / avg_mark_bytes); + const auto heuristic_min_marks = std::min(total_marks / threads_, min_bytes_per_task / avg_mark_bytes); if (heuristic_min_marks > min_marks_for_concurrent_read) { min_marks_for_concurrent_read = heuristic_min_marks; From 87d6564b9bcb5509f7ac64c906aa2ed2b9f7984a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 May 2023 15:50:06 +0200 Subject: [PATCH 0774/2223] Update gtest_async_loader.cpp --- src/Common/tests/gtest_async_loader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index bfc305de9f8..6504f1c97a9 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -638,7 +638,7 @@ TEST(AsyncLoader, StaticPriorities) t.loader.start(); t.loader.wait(); - ASSERT(schedule == "A9E9D9F9G9H9C4B3" || schedule == "A9D9E9F9G9H9C4B3"); + ASSERT_TRUE(schedule == "A9E9D9F9G9H9C4B3" || schedule == "A9D9E9F9G9H9C4B3"); } TEST(AsyncLoader, SimplePrioritization) From 437880d4c15a5cc9e3a541db58939ee7c507e10c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 29 May 2023 15:52:50 +0200 Subject: [PATCH 0775/2223] Enable `enable_memory_bound_merging_of_aggregation_results` by default --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 607be1522db..10d94f8d218 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -143,7 +143,7 @@ class IColumn; M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ - M(Bool, enable_memory_bound_merging_of_aggregation_results, false, "Enable memory bound merging strategy for aggregation. Set it to true only if all nodes of your clusters have versions >= 22.12.", 0) \ + M(Bool, enable_memory_bound_merging_of_aggregation_results, true, "Enable memory bound merging strategy for aggregation.", 0) \ M(Bool, enable_positional_arguments, true, "Enable positional arguments in ORDER BY, GROUP BY and LIMIT BY", 0) \ M(Bool, enable_extended_results_for_datetime_functions, false, "Enable date functions like toLastDayOfMonth return Date32 results (instead of Date results) for Date32/DateTime64 arguments.", 0) \ \ From e10f951467390f5deb67b7fe12e38f546059ec9a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 29 May 2023 19:09:07 +0000 Subject: [PATCH 0776/2223] Fix IS (NOT) NULL operator priority --- src/Parsers/ExpressionListParsers.cpp | 66 +++++++++---------- .../02752_is_null_priority.reference | 16 +++++ .../0_stateless/02752_is_null_priority.sql | 1 + 3 files changed, 50 insertions(+), 33 deletions(-) create mode 100644 tests/queries/0_stateless/02752_is_null_priority.reference create mode 100644 tests/queries/0_stateless/02752_is_null_priority.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 1477f3d000d..8903b2b02b0 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2332,44 +2332,44 @@ const std::vector> ParserExpressionImpl::o {":", Operator("if", 3, 3, OperatorType::FinishIf)}, {"OR", Operator("or", 3, 2, OperatorType::Mergeable)}, {"AND", Operator("and", 4, 2, OperatorType::Mergeable)}, - {"BETWEEN", Operator("", 6, 0, OperatorType::StartBetween)}, - {"NOT BETWEEN", Operator("", 6, 0, OperatorType::StartNotBetween)}, - {"==", Operator("equals", 8, 2, OperatorType::Comparison)}, - {"!=", Operator("notEquals", 8, 2, OperatorType::Comparison)}, - {"<>", Operator("notEquals", 8, 2, OperatorType::Comparison)}, - {"<=", Operator("lessOrEquals", 8, 2, OperatorType::Comparison)}, - {">=", Operator("greaterOrEquals", 8, 2, OperatorType::Comparison)}, - {"<", Operator("less", 8, 2, OperatorType::Comparison)}, - {">", Operator("greater", 8, 2, OperatorType::Comparison)}, - {"=", Operator("equals", 8, 2, OperatorType::Comparison)}, - {"LIKE", Operator("like", 8, 2)}, - {"ILIKE", Operator("ilike", 8, 2)}, - {"NOT LIKE", Operator("notLike", 8, 2)}, - {"NOT ILIKE", Operator("notILike", 8, 2)}, - {"REGEXP", Operator("match", 8, 2)}, - {"IN", Operator("in", 8, 2)}, - {"NOT IN", Operator("notIn", 8, 2)}, - {"GLOBAL IN", Operator("globalIn", 8, 2)}, - {"GLOBAL NOT IN", Operator("globalNotIn", 8, 2)}, - {"||", Operator("concat", 9, 2, OperatorType::Mergeable)}, - {"+", Operator("plus", 10, 2)}, - {"-", Operator("minus", 10, 2)}, - {"*", Operator("multiply", 11, 2)}, - {"/", Operator("divide", 11, 2)}, - {"%", Operator("modulo", 11, 2)}, - {"MOD", Operator("modulo", 11, 2)}, - {"DIV", Operator("intDiv", 11, 2)}, - {".", Operator("tupleElement", 13, 2, OperatorType::TupleElement)}, - {"[", Operator("arrayElement", 13, 2, OperatorType::ArrayElement)}, - {"::", Operator("CAST", 13, 2, OperatorType::Cast)}, - {"IS NULL", Operator("isNull", 13, 1, OperatorType::IsNull)}, - {"IS NOT NULL", Operator("isNotNull", 13, 1, OperatorType::IsNull)}, + {"IS NULL", Operator("isNull", 6, 1, OperatorType::IsNull)}, + {"IS NOT NULL", Operator("isNotNull", 6, 1, OperatorType::IsNull)}, + {"BETWEEN", Operator("", 7, 0, OperatorType::StartBetween)}, + {"NOT BETWEEN", Operator("", 7, 0, OperatorType::StartNotBetween)}, + {"==", Operator("equals", 9, 2, OperatorType::Comparison)}, + {"!=", Operator("notEquals", 9, 2, OperatorType::Comparison)}, + {"<>", Operator("notEquals", 9, 2, OperatorType::Comparison)}, + {"<=", Operator("lessOrEquals", 9, 2, OperatorType::Comparison)}, + {">=", Operator("greaterOrEquals", 9, 2, OperatorType::Comparison)}, + {"<", Operator("less", 9, 2, OperatorType::Comparison)}, + {">", Operator("greater", 9, 2, OperatorType::Comparison)}, + {"=", Operator("equals", 9, 2, OperatorType::Comparison)}, + {"LIKE", Operator("like", 9, 2)}, + {"ILIKE", Operator("ilike", 9, 2)}, + {"NOT LIKE", Operator("notLike", 9, 2)}, + {"NOT ILIKE", Operator("notILike", 9, 2)}, + {"REGEXP", Operator("match", 9, 2)}, + {"IN", Operator("in", 9, 2)}, + {"NOT IN", Operator("notIn", 9, 2)}, + {"GLOBAL IN", Operator("globalIn", 9, 2)}, + {"GLOBAL NOT IN", Operator("globalNotIn", 9, 2)}, + {"||", Operator("concat", 10, 2, OperatorType::Mergeable)}, + {"+", Operator("plus", 11, 2)}, + {"-", Operator("minus", 11, 2)}, + {"*", Operator("multiply", 12, 2)}, + {"/", Operator("divide", 12, 2)}, + {"%", Operator("modulo", 12, 2)}, + {"MOD", Operator("modulo", 12, 2)}, + {"DIV", Operator("intDiv", 12, 2)}, + {".", Operator("tupleElement", 14, 2, OperatorType::TupleElement)}, + {"[", Operator("arrayElement", 14, 2, OperatorType::ArrayElement)}, + {"::", Operator("CAST", 14, 2, OperatorType::Cast)}, }; const std::vector> ParserExpressionImpl::unary_operators_table { {"NOT", Operator("not", 5, 1)}, - {"-", Operator("negate", 12, 1)} + {"-", Operator("negate", 13, 1)} }; const Operator ParserExpressionImpl::finish_between_operator("", 7, 0, OperatorType::FinishBetween); diff --git a/tests/queries/0_stateless/02752_is_null_priority.reference b/tests/queries/0_stateless/02752_is_null_priority.reference new file mode 100644 index 00000000000..54606882ddc --- /dev/null +++ b/tests/queries/0_stateless/02752_is_null_priority.reference @@ -0,0 +1,16 @@ +SelectWithUnionQuery (children 1) + ExpressionList (children 1) + SelectQuery (children 1) + ExpressionList (children 2) + Function isNull (children 1) + ExpressionList (children 1) + Function multiply (children 1) + ExpressionList (children 2) + Identifier a + Identifier b + Function isNotNull (children 1) + ExpressionList (children 1) + Function multiply (children 1) + ExpressionList (children 2) + Identifier a + Identifier b diff --git a/tests/queries/0_stateless/02752_is_null_priority.sql b/tests/queries/0_stateless/02752_is_null_priority.sql new file mode 100644 index 00000000000..a0a9741e752 --- /dev/null +++ b/tests/queries/0_stateless/02752_is_null_priority.sql @@ -0,0 +1 @@ +EXPLAIN AST SELECT a * b IS NULL, a * b IS NOT NULL; From 6c94632d47f9c8b123507da49da94b6b0cf727a0 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Mon, 29 May 2023 22:10:34 +0000 Subject: [PATCH 0777/2223] Deprive toStartOfWeek and toLastDayOfWeek functions of in-source documentation --- src/Functions/toCustomWeek.cpp | 26 ++----------------- ...new_functions_must_be_documented.reference | 2 ++ 2 files changed, 4 insertions(+), 24 deletions(-) diff --git a/src/Functions/toCustomWeek.cpp b/src/Functions/toCustomWeek.cpp index 0dbd60b3eff..98e7aaf1d6b 100644 --- a/src/Functions/toCustomWeek.cpp +++ b/src/Functions/toCustomWeek.cpp @@ -17,30 +17,8 @@ REGISTER_FUNCTION(ToCustomWeek) { factory.registerFunction(); factory.registerFunction(); - - factory.registerFunction(FunctionDocumentation{.description=R"( -Rounds a date or date with time down to the nearest Sunday or Monday. Returns the date. -Syntax: toStartOfWeek(t[, mode[, timezone]]) -The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. - -Example: -[example:typical] -)", - .examples{ - {"typical", "SELECT toStartOfWeek(today(), 1);", ""}}, - .categories{"Dates and Times"}}, FunctionFactory::CaseSensitive); - - factory.registerFunction(FunctionDocumentation{.description=R"( -Rounds a date or date with time up to the nearest Saturday or Sunday. Returns the date. -Syntax: toLastDayOfWeek(t[, mode[, timezone]]) -The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. - -Example: -[example:typical] -)", - .examples{ - {"typical", "SELECT toLastDayOfWeek(today(), 1);", ""}}, - .categories{"Dates and Times"}}, FunctionFactory::CaseSensitive); + factory.registerFunction(); + factory.registerFunction(); /// Compatibility aliases for mysql. factory.registerAlias("week", "toWeek", FunctionFactory::CaseInsensitive); diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index eae36604845..bf43288fe1d 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -784,6 +784,7 @@ toIntervalWeek toIntervalYear toJSONString toLastDayOfMonth +toLastDayOfWeek toLowCardinality toMinute toModifiedJulianDay @@ -815,6 +816,7 @@ toStartOfNanosecond toStartOfQuarter toStartOfSecond toStartOfTenMinutes +toStartOfWeek toStartOfYear toString toStringCutToZero From 9a8086a3775c89197fe29e609d8cda9eb32879b5 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Mon, 29 May 2023 22:12:44 +0000 Subject: [PATCH 0778/2223] Reorder toLastDayOf* functions in settings.md --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7cac9a30d5d..239b8960190 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3788,7 +3788,7 @@ Result: ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Enables or disables returning results of type: -- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). +- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) and [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday). - `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). Possible values: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 3b100729e4d..cda338cef75 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3790,7 +3790,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Включает или отключает возвращение результатов типа: -- `Date32` с расширенным диапазоном (по сравнению с типом `Date`) для функций [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) и [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). +- `Date32` с расширенным диапазоном (по сравнению с типом `Date`) для функций [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toLastDayOfWeek](../../sql-reference/functions/date-time-functions.md#tolastdayofweek) и [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday). - `DateTime64` с расширенным диапазоном (по сравнению с типом `DateTime`) для функций [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) и [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). Возможные значения: From d4cec1f0e0fa114f5e04dead6b90f51b3e292310 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 30 May 2023 01:30:01 +0200 Subject: [PATCH 0779/2223] fix client thread attachment + make better tests --- programs/client/Client.cpp | 1 + src/Client/ClientBase.cpp | 3 --- src/Client/Connection.cpp | 4 --- src/Common/ThreadStatus.h | 3 ++- src/DataTypes/DataTypeDateTime.h | 4 ++- src/Server/TCPHandler.cpp | 2 +- .../02681_timezone_setting.reference | 5 ---- .../0_stateless/02681_timezone_setting.sql | 11 -------- .../02737_timezone_setting.reference | 7 +++++ .../0_stateless/02737_timezone_setting.sql | 27 +++++++++++++++++++ 10 files changed, 41 insertions(+), 26 deletions(-) delete mode 100644 tests/queries/0_stateless/02681_timezone_setting.reference delete mode 100644 tests/queries/0_stateless/02681_timezone_setting.sql create mode 100644 tests/queries/0_stateless/02737_timezone_setting.reference create mode 100644 tests/queries/0_stateless/02737_timezone_setting.sql diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 6b34bdbc5bb..231b7fd6d61 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -326,6 +326,7 @@ try // All that just to set DB::CurrentThread::get().getGlobalContext() // which is required for client timezone (pushed from server) to work. auto thread_group = std::make_shared(); + thread_group->global_context = global_context; thread_status.attachToGroup(thread_group, false); } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index fad9494ba4b..562c11680a1 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -448,9 +448,7 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) /// output_format, do not output it. /// Also do not output too much data if we're fuzzing. if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) - { return; - } /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout)) @@ -902,7 +900,6 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa if (send_external_tables) sendExternalTables(parsed_query); - receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); break; diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 86585d805d9..457d90c5bd4 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -355,10 +355,6 @@ void Connection::receiveHello() nonce.emplace(read_nonce); } } -// else if (packet_type == Protocol::Server::TimezoneUpdate) -// { -// // skip this packet at hello, will receive and process it later -// } else if (packet_type == Protocol::Server::Exception) receiveException()->rethrow(); else diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 600dfc56d2b..400b55c2409 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -72,7 +72,8 @@ public: /// Set up at creation, no race when reading const ContextWeakPtr query_context; - const ContextWeakPtr global_context; + /// Cannot make it const -- we need to modify it in ch-client to process timezone from server + ContextWeakPtr global_context; const FatalErrorCallback fatal_error_callback; diff --git a/src/DataTypes/DataTypeDateTime.h b/src/DataTypes/DataTypeDateTime.h index 91a09ff7cb9..a4a05917ba5 100644 --- a/src/DataTypes/DataTypeDateTime.h +++ b/src/DataTypes/DataTypeDateTime.h @@ -21,7 +21,9 @@ namespace DB * all types with different time zones are equivalent and may be used interchangingly. * Time zone only affects parsing and displaying in text formats. * - * If time zone is not specified (example: DateTime without parameter), then default time zone is used. + * If time zone is not specified (example: DateTime without parameter), + * then `session_timezone` setting value is used. + * If `session_timezone` is not set (or empty string), server default time zone is used. * Default time zone is server time zone, if server is doing transformations * and if client is doing transformations, unless 'use_client_time_zone' setting is passed to client; * Server time zone is the time zone specified in 'timezone' parameter in configuration file, diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d57a1c93dd7..c41eace68ba 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1068,7 +1068,7 @@ void TCPHandler::sendTimezone() if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES) return; - const String & tz = query_context->getSettingsRef().session_timezone.toString(); + const String & tz = query_context->getSettingsRef().session_timezone.value; LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz); writeVarUInt(Protocol::Server::TimezoneUpdate, *out); diff --git a/tests/queries/0_stateless/02681_timezone_setting.reference b/tests/queries/0_stateless/02681_timezone_setting.reference deleted file mode 100644 index 8850d77ab03..00000000000 --- a/tests/queries/0_stateless/02681_timezone_setting.reference +++ /dev/null @@ -1,5 +0,0 @@ -2022-12-12 17:23:23.123 -2022-12-12 23:23:23.123 -2022-12-12 22:23:23.123 -Europe/Zurich Europe/Zurich -Pacific/Pitcairn Pacific/Pitcairn diff --git a/tests/queries/0_stateless/02681_timezone_setting.sql b/tests/queries/0_stateless/02681_timezone_setting.sql deleted file mode 100644 index f66e8d2b646..00000000000 --- a/tests/queries/0_stateless/02681_timezone_setting.sql +++ /dev/null @@ -1,11 +0,0 @@ -SET session_timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} - -SET session_timezone = 'Asia/Novosibirsk'; -SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); -SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'Europe/Zurich'; - -SET session_timezone = 'Asia/Manila'; -SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); - -SELECT timezone(), timezoneOf(now()) SETTINGS session_timezone = 'Europe/Zurich' FORMAT TSV; -SELECT timezone(), timezoneOf(now()) SETTINGS session_timezone = 'Pacific/Pitcairn' FORMAT TSV; diff --git a/tests/queries/0_stateless/02737_timezone_setting.reference b/tests/queries/0_stateless/02737_timezone_setting.reference new file mode 100644 index 00000000000..578aec4e316 --- /dev/null +++ b/tests/queries/0_stateless/02737_timezone_setting.reference @@ -0,0 +1,7 @@ +Pacific/Pitcairn Pacific/Pitcairn +Asia/Novosibirsk Asia/Novosibirsk +2022-12-12 17:23:23 +2022-12-13 07:23:23.123 +2002-12-12 23:23:23 2002-12-12 23:23:23 +2002-12-12 23:23:23.123 2002-12-12 23:23:23.123 +2000-01-01 01:00:00 diff --git a/tests/queries/0_stateless/02737_timezone_setting.sql b/tests/queries/0_stateless/02737_timezone_setting.sql new file mode 100644 index 00000000000..87eeec0779b --- /dev/null +++ b/tests/queries/0_stateless/02737_timezone_setting.sql @@ -0,0 +1,27 @@ +SET session_timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} + +SELECT timezone(), timezoneOf(now()) SETTINGS session_timezone = 'Pacific/Pitcairn'; + +SET session_timezone = 'Asia/Novosibirsk'; +SELECT timezone(), timezoneOf(now()); + +-- test simple queries +SELECT toDateTime(toDateTime('2022-12-12 23:23:23'), 'Europe/Zurich'); +SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver'; + +-- test proper serialization +SELECT toDateTime('2002-12-12 23:23:23'), toString(toDateTime('2002-12-12 23:23:23')) SETTINGS session_timezone = 'Asia/Phnom_Penh'; +SELECT toDateTime64('2002-12-12 23:23:23.123', 3), toString(toDateTime64('2002-12-12 23:23:23.123', 3)) SETTINGS session_timezone = 'Asia/Phnom_Penh'; + +-- Create a table and test that DateTimes are processed correctly on insert +SET session_timezone='Asia/Novosibirsk'; +CREATE TABLE test_tz_setting (d DateTime('UTC')) Engine=Memory AS SELECT toDateTime('2000-01-01 00:00:00'); +INSERT INTO test_tz_setting VALUES ('2000-01-01 01:00:00'); -- this is parsed using timezone from `d` column +INSERT INTO test_tz_setting VALUES (toDateTime('2000-01-02 02:00:00')); -- this is parsed using `session_timezone` + +-- Test parsing in WHERE filter, shall have the same logic as insert +SELECT d FROM test_tz_setting WHERE d == '2000-01-01 01:00:00'; -- 1 row expected +SELECT d FROM test_tz_setting WHERE d == toDateTime('2000-01-01 02:00:00'); -- 0 rows expected + +-- Cleanup table +DROP TABLE test_tz_setting SYNC; \ No newline at end of file From 954e76cb7a432081df8865a2988749fec3b950e4 Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Tue, 30 May 2023 09:38:50 +0800 Subject: [PATCH 0780/2223] use getScaleMultiplier --- src/Interpreters/convertFieldToType.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 79251d944c8..1ea9ecd7e00 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -194,13 +194,15 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } else if (which_type.isDateTime64() && which_from_type.isDate()) { - const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); - return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); + const DataTypeDateTime64 & data_type_date_time64 = static_cast(type); + const Int64 value = data_type_date_time64.getTimeZone().fromDayNum(DayNum(src.get())); + return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, data_type_date_time64.getScaleMultiplier()); } else if (which_type.isDateTime64() && which_from_type.isDate32()) { - const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); - return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); + const DataTypeDateTime64 & data_type_date_time64 = static_cast(type); + const Int64 value = data_type_date_time64.getTimeZone().fromDayNum(DayNum(src.get())); + return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, data_type_date_time64.getScaleMultiplier()); } else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { From a278d1304338b4243d34eb39372d8103eb1c368c Mon Sep 17 00:00:00 2001 From: sslouis Date: Tue, 30 May 2023 10:09:12 +0800 Subject: [PATCH 0781/2223] Update first_value.md:remove redundant 's' remove redundant 's' --- .../sql-reference/aggregate-functions/reference/first_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/first_value.md b/docs/en/sql-reference/aggregate-functions/reference/first_value.md index e163bd62a45..f343ca3f66c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/first_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/first_value.md @@ -29,7 +29,7 @@ select first_value(b) from test_data ### example2 The NULL value is ignored. ```sql -select first_value(b) ignore nulls sfrom test_data +select first_value(b) ignore nulls from test_data ``` ```text From ecea1ac090c9ac1749951b409f32e4911a9dc005 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Thu, 25 May 2023 14:25:08 +0200 Subject: [PATCH 0782/2223] Fix crash when Pool::Entry::disconnect() is called Many Pool::Entry objects can keep the same pointer to Pool::Connection. If Pool::Entry::disconnect() is called on one such object, Pool::removeConnection() is called to remove Pool::Connection from the pool, where connection->ref_count is cleared and connection->removed_from_pool is set. Next Pool::Entry::~Entry() calls decrementRefCount() with 1. const auto ref_count = data->ref_count.fetch_sub(1); where data->ref_count will be negative, since it was cleared 2. checks removed_from_pool and deletes Pool::Connection but there might be multiple Entry objects still keep pointer to this Pool::Connection Suggesting not to clear ref_count on disconnect() and delete Pool::Connection only on the last Pool::Entry is being destroyed. Fixes ea375ef9890f18be7038f66da2d731010ed4462f --- src/Common/mysqlxx/Pool.cpp | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index 6438d76cc3a..d10889d1f97 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -40,14 +40,10 @@ void Pool::Entry::decrementRefCount() { /// We were the last user of this thread, deinitialize it mysql_thread_end(); - } - else if (data->removed_from_pool) - { - /// data->ref_count == 0 in case we removed connection from pool (see Pool::removeConnection). - chassert(ref_count == 0); /// In Pool::Entry::disconnect() we remove connection from the list of pool's connections. /// So now we must deallocate the memory. - ::delete data; + if (data->removed_from_pool) + ::delete data; } } @@ -234,11 +230,8 @@ void Pool::removeConnection(Connection* connection) std::lock_guard lock(mutex); if (connection) { - if (connection->ref_count > 0) - { + if (!connection->removed_from_pool) connection->conn.disconnect(); - connection->ref_count = 0; - } connections.remove(connection); connection->removed_from_pool = true; } From fb6c4f280289215068f6e9c17a748b0c88c885fd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 May 2023 08:46:40 +0000 Subject: [PATCH 0783/2223] Fix msan issue, pt. II --- src/Functions/FunctionsHashing.h | 17 +++++++++++++---- .../0_stateless/02534_keyed_siphash.reference | 2 ++ .../queries/0_stateless/02534_keyed_siphash.sql | 2 ++ 3 files changed, 17 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index b8122ebac85..1fed80a66b8 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -92,16 +92,25 @@ namespace impl if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) { - const auto & key0col_data = key0col->getData(); - ret.key0 = key0col_data[0]; + if (!key0col->empty()) + { + const auto & key0col_data = key0col->getData(); + ret.key0 = key0col_data[0]; + } + else ret.key0 = 0; } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) { - const auto & key1col_data = key1col->getData(); - ret.key1 = key1col_data[0]; + if (!key1col->empty()) + { + const auto & key1col_data = key1col->getData(); + ret.key1 = key1col_data[0]; + } + else + ret.key1 = 0; } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index d4ab4a4e88f..a891d01e99d 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -196,3 +196,5 @@ E28DBDE7FE22E41C 1 1CE422FEE7BD8DE20000000000000000 7766709361750702608 +20AF99D3A87829E0 +12489502208762728797 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 09489be39c2..7ce50583cdb 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -276,3 +276,5 @@ SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000 -- Crashed with memory sanitizer SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); +SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); +SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); From 2d0694752212a60354986cc7e785f749fe79b230 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 May 2023 09:06:45 +0000 Subject: [PATCH 0784/2223] Better --- src/Functions/FunctionsHashing.h | 19 +++++-------------- 1 file changed, 5 insertions(+), 14 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 1fed80a66b8..d08a900c88a 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -92,25 +92,16 @@ namespace impl if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) { - if (!key0col->empty()) - { - const auto & key0col_data = key0col->getData(); - ret.key0 = key0col_data[0]; - } - else ret.key0 = 0; + const auto & key0col_data = key0col->getData(); + ret.key0 = key0col_data[0]; } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) { - if (!key1col->empty()) - { - const auto & key1col_data = key1col->getData(); - ret.key1 = key1col_data[0]; - } - else - ret.key1 = 0; + const auto & key1col_data = key1col->getData(); + ret.key1 = key1col_data[0]; } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); @@ -1450,7 +1441,7 @@ public: KeyType key{}; if constexpr (Keyed) - if (!arguments.empty()) + if (!arguments.empty() && input_rows_count != 0) key = Impl::parseKey(arguments[0]); /// The function supports arbitrary number of arguments of arbitrary types. From 100b4d0969a5682e464d1de7f58685f3f4f635f3 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 26 May 2023 20:07:00 +0200 Subject: [PATCH 0785/2223] cope with finalize in d-tors --- src/IO/WriteBufferFromS3TaskTracker.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/WriteBufferFromS3TaskTracker.h b/src/IO/WriteBufferFromS3TaskTracker.h index 800e5239cc4..c3f4628b946 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.h +++ b/src/IO/WriteBufferFromS3TaskTracker.h @@ -49,6 +49,8 @@ private: /// waitTilInflightShrink waits til the number of in-flight tasks beyond the limit `max_tasks_inflight`. void waitTilInflightShrink() TSA_NO_THREAD_SAFETY_ANALYSIS; + void collectFinishedFutures(bool propagate_exceptions) TSA_REQUIRES(mutex); + const bool is_async; ThreadPoolCallbackRunner scheduler; const size_t max_tasks_inflight; From 0e019c8e83aead4cd01fa21b0f46164d4fd4fb50 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 26 May 2023 22:18:55 +0200 Subject: [PATCH 0786/2223] turn off some d-tor finalize --- src/Compression/CompressedWriteBuffer.cpp | 1 + .../WriteBufferFromNuraftBuffer.cpp | 1 + .../IO/CachedOnDiskWriteBufferFromFile.cpp | 1 + .../IO/WriteBufferFromAzureBlobStorage.cpp | 1 + .../IO/WriteBufferWithFinalizeCallback.cpp | 17 +++++++++-------- src/IO/Archives/ZipArchiveWriter.cpp | 7 ++++++- src/IO/BrotliWriteBuffer.cpp | 1 + src/IO/Bzip2WriteBuffer.cpp | 1 + src/IO/ForkWriteBuffer.cpp | 1 + src/IO/LZMADeflatingWriteBuffer.cpp | 1 + src/IO/Lz4DeflatingWriteBuffer.cpp | 1 + src/IO/SnappyWriteBuffer.cpp | 1 + src/IO/WriteBufferFromEncryptedFile.cpp | 1 + src/IO/WriteBufferFromFile.cpp | 1 + src/IO/WriteBufferFromFileDecorator.cpp | 17 +++++++++-------- src/IO/WriteBufferFromFileDescriptor.cpp | 1 + src/IO/WriteBufferFromOStream.cpp | 1 + src/IO/WriteBufferFromPocoSocket.cpp | 1 + src/IO/WriteBufferFromVector.h | 1 + src/IO/ZlibDeflatingWriteBuffer.cpp | 1 + src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 1 + src/IO/ZstdDeflatingWriteBuffer.cpp | 1 + .../Cache/WriteBufferToFileSegment.cpp | 1 + src/Interpreters/TemporaryDataOnDisk.cpp | 1 + .../HTTP/WriteBufferFromHTTPServerResponse.cpp | 1 + src/Storages/HDFS/WriteBufferFromHDFS.cpp | 1 + 26 files changed, 47 insertions(+), 17 deletions(-) diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index cb2ee1140d0..c3152c0adba 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -59,6 +59,7 @@ void CompressedWriteBuffer::nextImpl() CompressedWriteBuffer::~CompressedWriteBuffer() { + //! finalize(); } diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp index c955d3fdbbe..5bed2da2978 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.cpp +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -53,6 +53,7 @@ nuraft::ptr WriteBufferFromNuraftBuffer::getBuffer() WriteBufferFromNuraftBuffer::~WriteBufferFromNuraftBuffer() { + //! try { finalize(); diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 9153af90312..2abf3ab5203 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -123,6 +123,7 @@ void FileSegmentRangeWriter::finalize() FileSegmentRangeWriter::~FileSegmentRangeWriter() { + //! try { if (!finalized) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index b5d296bd865..56b45f1dc22 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -38,6 +38,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage() { + //! finalize(); } diff --git a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp index 49e230b9dc3..63d7edf5437 100644 --- a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp @@ -16,14 +16,15 @@ WriteBufferWithFinalizeCallback::WriteBufferWithFinalizeCallback( WriteBufferWithFinalizeCallback::~WriteBufferWithFinalizeCallback() { - try - { - finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + //! +// try +// { +// finalize(); +// } +// catch (...) +// { +// tryLogCurrentException(__PRETTY_FUNCTION__); +// } } void WriteBufferWithFinalizeCallback::finalizeImpl() diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 088d83cd29e..57bb6452914 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -115,6 +115,7 @@ public: ~WriteBufferFromZipArchive() override { + //! try { finalize(); @@ -191,7 +192,11 @@ namespace explicit StreamFromWriteBuffer(std::unique_ptr write_buffer_) : write_buffer(std::move(write_buffer_)), start_offset(write_buffer->count()) {} - ~StreamFromWriteBuffer() { write_buffer->finalize(); } + ~StreamFromWriteBuffer() + { + //! + write_buffer->finalize(); + } static int closeFileFunc(void *, void * stream) { diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp index 47426d62a6e..0c434b4981e 100644 --- a/src/IO/BrotliWriteBuffer.cpp +++ b/src/IO/BrotliWriteBuffer.cpp @@ -44,6 +44,7 @@ BrotliWriteBuffer::BrotliWriteBuffer(std::unique_ptr out_, int comp BrotliWriteBuffer::~BrotliWriteBuffer() { + //! finalize(); } diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index 4b6bed70d35..e5aab177f3a 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -47,6 +47,7 @@ Bzip2WriteBuffer::Bzip2WriteBuffer(std::unique_ptr out_, int compre Bzip2WriteBuffer::~Bzip2WriteBuffer() { + //! finalize(); } diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index 8e11b9ff590..b3481203757 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -53,6 +53,7 @@ void ForkWriteBuffer::finalizeImpl() ForkWriteBuffer::~ForkWriteBuffer() { + //! finalize(); } diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index 30e247b1016..3e1c9dc58b9 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -46,6 +46,7 @@ LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() { + //! finalize(); } diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index c3a1b8282c3..2627643183d 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -42,6 +42,7 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer() { + //! finalize(); } diff --git a/src/IO/SnappyWriteBuffer.cpp b/src/IO/SnappyWriteBuffer.cpp index ca40d0656d1..3ea6f831d5b 100644 --- a/src/IO/SnappyWriteBuffer.cpp +++ b/src/IO/SnappyWriteBuffer.cpp @@ -22,6 +22,7 @@ SnappyWriteBuffer::SnappyWriteBuffer(std::unique_ptr out_, size_t b SnappyWriteBuffer::~SnappyWriteBuffer() { + //! finish(); } diff --git a/src/IO/WriteBufferFromEncryptedFile.cpp b/src/IO/WriteBufferFromEncryptedFile.cpp index 5bca0dc68d5..7efdbc43238 100644 --- a/src/IO/WriteBufferFromEncryptedFile.cpp +++ b/src/IO/WriteBufferFromEncryptedFile.cpp @@ -21,6 +21,7 @@ WriteBufferFromEncryptedFile::WriteBufferFromEncryptedFile( WriteBufferFromEncryptedFile::~WriteBufferFromEncryptedFile() { + //! finalize(); } diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index e58f1e3a60c..964f7415803 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -77,6 +77,7 @@ WriteBufferFromFile::~WriteBufferFromFile() if (fd < 0) return; + //! finalize(); int err = ::close(fd); /// Everything except for EBADF should be ignored in dtor, since all of diff --git a/src/IO/WriteBufferFromFileDecorator.cpp b/src/IO/WriteBufferFromFileDecorator.cpp index 4cc881f177f..82e38eac72d 100644 --- a/src/IO/WriteBufferFromFileDecorator.cpp +++ b/src/IO/WriteBufferFromFileDecorator.cpp @@ -30,14 +30,15 @@ void WriteBufferFromFileDecorator::finalizeImpl() WriteBufferFromFileDecorator::~WriteBufferFromFileDecorator() { - try - { - finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + //! +// try +// { +// finalize(); +// } +// catch (...) +// { +// tryLogCurrentException(__PRETTY_FUNCTION__); +// } /// It is not a mistake that swap is called here /// Swap has been called at constructor, it should be called at destructor diff --git a/src/IO/WriteBufferFromFileDescriptor.cpp b/src/IO/WriteBufferFromFileDescriptor.cpp index 135ff608967..0516171c051 100644 --- a/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/src/IO/WriteBufferFromFileDescriptor.cpp @@ -105,6 +105,7 @@ WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor( WriteBufferFromFileDescriptor::~WriteBufferFromFileDescriptor() { + //! finalize(); } diff --git a/src/IO/WriteBufferFromOStream.cpp b/src/IO/WriteBufferFromOStream.cpp index 2d0d5976f85..d55aa054174 100644 --- a/src/IO/WriteBufferFromOStream.cpp +++ b/src/IO/WriteBufferFromOStream.cpp @@ -40,6 +40,7 @@ WriteBufferFromOStream::WriteBufferFromOStream( WriteBufferFromOStream::~WriteBufferFromOStream() { + //! finalize(); } diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index 039110dfb62..a8b28445294 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -106,6 +106,7 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { + //! finalize(); } diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index 4b2a3581625..b0ff3183291 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -63,6 +63,7 @@ public: ~WriteBufferFromVector() override { + //! finalize(); } diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index 43bb0405555..4440254d15f 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -74,6 +74,7 @@ void ZlibDeflatingWriteBuffer::nextImpl() ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer() { + //! try { finalize(); diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index be739c0e654..05ee23158fd 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -87,6 +87,7 @@ void ZstdDeflatingAppendableWriteBuffer::nextImpl() ZstdDeflatingAppendableWriteBuffer::~ZstdDeflatingAppendableWriteBuffer() { + //! finalize(); } diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index c6d2ffc39f9..4ec3f5a3fce 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -33,6 +33,7 @@ ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer( ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() { + //! finalize(); } diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index 1eac87a804d..ed831350186 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -73,6 +73,7 @@ std::shared_ptr WriteBufferToFileSegment::getReadBufferImpl() WriteBufferToFileSegment::~WriteBufferToFileSegment() { + //! try { finalize(); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 69fef21dbab..14e0bff6966 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -184,6 +184,7 @@ struct TemporaryFileStream::OutputWriter ~OutputWriter() { + //! try { finalize(); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index c8015cfd185..03222fe7e0b 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -171,6 +171,7 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { + //! finalize(); } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index fad0447d2cf..48f35c378ba 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -147,6 +147,7 @@ void WriteBufferFromHDFS::finalizeImpl() WriteBufferFromHDFS::~WriteBufferFromHDFS() { + //! finalize(); } From fe3939287b476b7e0f2cec669dd9606f2fd0438f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 27 May 2023 01:02:48 +0200 Subject: [PATCH 0787/2223] add test, fix .gin_sid files --- src/IO/WriteBufferFromS3.cpp | 12 ++++++++++-- src/IO/WriteBufferFromS3TaskTracker.cpp | 15 ++++++++------- src/Storages/MergeTree/GinIndexStore.cpp | 7 +++++++ .../configs/config.d/storage_conf.xml | 19 +++++++++++++++++++ tests/integration/test_merge_tree_s3/test.py | 5 +++-- 5 files changed, 47 insertions(+), 11 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 954c996d929..4657a65f931 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -182,6 +182,14 @@ void WriteBufferFromS3::finalizeImpl() if (!is_prefinalized) preFinalize(); + if (std::uncaught_exception()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Detected buffer finalization when an exception is unwinding the stack." + " Do not call finalize buffer in destructors or when exception thrown." + " Details {}", + getLogDetails()); + chassert(offset() == 0); chassert(hidden_size == 0); @@ -521,7 +529,7 @@ void WriteBufferFromS3::completeMultipartUpload() if (multipart_tags.empty()) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::S3_ERROR, "Failed to complete multipart upload. No parts have uploaded"); for (size_t i = 0; i < multipart_tags.size(); ++i) @@ -529,7 +537,7 @@ void WriteBufferFromS3::completeMultipartUpload() const auto tag = multipart_tags.at(i); if (tag.empty()) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::S3_ERROR, "Failed to complete multipart upload. Part {} haven't been uploaded.", i); } diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 4abae90eeac..7cf5a89df86 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -162,15 +162,16 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() for (auto & it : finished_futures) { - SCOPE_EXIT({ - /// According to basic exception safety TaskTracker has to be destroyed after exception - /// If it would be true than this SCOPE_EXIT is superfluous - /// However WriteBufferWithFinalizeCallback, WriteBufferFromFileDecorator do call finalize in d-tor - /// TaskTracker has to cope this until the issue with finalizing in d-tor is addressed in #50274 - futures.erase(it); - }); +// SCOPE_EXIT({ +// /// According to basic exception safety TaskTracker has to be destroyed after exception +// /// If it would be true than this SCOPE_EXIT is superfluous +// /// However WriteBufferWithFinalizeCallback, WriteBufferFromFileDecorator do call finalize in d-tor +// /// TaskTracker has to cope this until the issue with finalizing in d-tor is addressed in #50274 +// futures.erase(it); +// }); it->get(); + futures.erase(it); } finished_futures.clear(); diff --git a/src/Storages/MergeTree/GinIndexStore.cpp b/src/Storages/MergeTree/GinIndexStore.cpp index 0904855755c..aa0c1fccbc3 100644 --- a/src/Storages/MergeTree/GinIndexStore.cpp +++ b/src/Storages/MergeTree/GinIndexStore.cpp @@ -166,6 +166,7 @@ UInt32 GinIndexStore::getNextSegmentIDRange(const String & file_name, size_t n) /// Write segment ID 1 writeVarUInt(1, *ostr); ostr->sync(); + ostr->finalize(); } /// Read id in file @@ -188,6 +189,7 @@ UInt32 GinIndexStore::getNextSegmentIDRange(const String & file_name, size_t n) writeVarUInt(result + n, *ostr); ostr->sync(); + ostr->finalize(); } return result; } @@ -317,8 +319,13 @@ void GinIndexStore::writeSegment() current_segment.segment_id = getNextSegmentID(); metadata_file_stream->sync(); + metadata_file_stream->finalize(); + dict_file_stream->sync(); + dict_file_stream->finalize(); + postings_file_stream->sync(); + postings_file_stream->finalize(); } GinIndexStoreDeserializer::GinIndexStoreDeserializer(const GinIndexStorePtr & store_) diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index cca80143548..504280e4bed 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -33,6 +33,18 @@ 20000 1 + + s3 + http://resolver:8083/root/data/ + minio + minio123 + true + 0 + 20000 + 20000 + 0 + 1 + local / @@ -128,6 +140,13 @@ + + +
+ broken_s3_always_multi_part +
+
+
diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index f87644a6876..4dcc5805294 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -930,8 +930,9 @@ def test_merge_canceled_by_drop(cluster, node_name): ) +@pytest.mark.parametrize("storage_policy", ["broken_s3_always_multi_part", "broken_s3"]) @pytest.mark.parametrize("node_name", ["node"]) -def test_merge_canceled_by_s3_errors(cluster, node_name): +def test_merge_canceled_by_s3_errors(cluster, node_name, storage_policy): node = cluster.instances[node_name] node.query("DROP TABLE IF EXISTS test_merge_canceled_by_s3_errors NO DELAY") node.query( @@ -939,7 +940,7 @@ def test_merge_canceled_by_s3_errors(cluster, node_name): " (key UInt32, value String)" " Engine=MergeTree() " " ORDER BY value " - " SETTINGS storage_policy='broken_s3'" + f" SETTINGS storage_policy='{storage_policy}'" ) node.query("SYSTEM STOP MERGES test_merge_canceled_by_s3_errors") node.query( From 99ad481505798a6da906e6b96d10b641db902f68 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 27 May 2023 11:47:44 +0200 Subject: [PATCH 0788/2223] fix sizes.log writes --- src/Common/FileChecker.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index a6e37654ff1..876bc4e641c 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -138,7 +138,7 @@ void FileChecker::save() const std::string tmp_files_info_path = parentPath(files_info_path) + "tmp_" + fileName(files_info_path); { - std::unique_ptr out = disk ? disk->writeFile(tmp_files_info_path) : std::make_unique(tmp_files_info_path); + std::unique_ptr out = disk ? disk->writeFile(tmp_files_info_path) : std::make_unique(tmp_files_info_path); /// So complex JSON structure - for compatibility with the old format. writeCString("{\"clickhouse\":{", *out); @@ -157,7 +157,9 @@ void FileChecker::save() const } writeCString("}}", *out); - out->next(); + + out->sync(); + out->finalize(); } if (disk) From d95e5b51af32db90de8f2194e83c0015dfd0f4cb Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 27 May 2023 12:14:07 +0200 Subject: [PATCH 0789/2223] mark all finalize calls in buffers d-tors with issue id --- src/Compression/CompressedWriteBuffer.cpp | 2 +- src/Coordination/WriteBufferFromNuraftBuffer.cpp | 2 +- src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp | 2 +- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- src/Disks/IO/WriteBufferWithFinalizeCallback.cpp | 2 +- src/IO/Archives/ZipArchiveWriter.cpp | 4 ++-- src/IO/BrotliWriteBuffer.cpp | 2 +- src/IO/Bzip2WriteBuffer.cpp | 2 +- src/IO/ForkWriteBuffer.cpp | 2 +- src/IO/LZMADeflatingWriteBuffer.cpp | 2 +- src/IO/Lz4DeflatingWriteBuffer.cpp | 2 +- src/IO/SnappyWriteBuffer.cpp | 2 +- src/IO/WriteBufferFromEncryptedFile.cpp | 2 +- src/IO/WriteBufferFromFile.cpp | 2 +- src/IO/WriteBufferFromFileDecorator.cpp | 2 +- src/IO/WriteBufferFromFileDescriptor.cpp | 2 +- src/IO/WriteBufferFromOStream.cpp | 2 +- src/IO/WriteBufferFromPocoSocket.cpp | 2 +- src/IO/WriteBufferFromVector.h | 2 +- src/IO/ZlibDeflatingWriteBuffer.cpp | 2 +- src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 2 +- src/IO/ZstdDeflatingWriteBuffer.cpp | 2 +- src/Interpreters/Cache/WriteBufferToFileSegment.cpp | 2 +- src/Interpreters/TemporaryDataOnDisk.cpp | 2 +- src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp | 2 +- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 2 +- 26 files changed, 27 insertions(+), 27 deletions(-) diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index c3152c0adba..22c5f235a27 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -59,7 +59,7 @@ void CompressedWriteBuffer::nextImpl() CompressedWriteBuffer::~CompressedWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp index 5bed2da2978..41b933717c6 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.cpp +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -53,7 +53,7 @@ nuraft::ptr WriteBufferFromNuraftBuffer::getBuffer() WriteBufferFromNuraftBuffer::~WriteBufferFromNuraftBuffer() { - //! + /// ! #50274 try { finalize(); diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 2abf3ab5203..8da9ce73ae3 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -123,7 +123,7 @@ void FileSegmentRangeWriter::finalize() FileSegmentRangeWriter::~FileSegmentRangeWriter() { - //! + /// ! #50274 try { if (!finalized) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 56b45f1dc22..8ac2c8ca831 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -38,7 +38,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage() { - //! + /// ! #50274 finalize(); } diff --git a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp index 63d7edf5437..47bf742fd5b 100644 --- a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp @@ -16,7 +16,7 @@ WriteBufferWithFinalizeCallback::WriteBufferWithFinalizeCallback( WriteBufferWithFinalizeCallback::~WriteBufferWithFinalizeCallback() { - //! + /// ! #50274 // try // { // finalize(); diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 57bb6452914..1c6ef18d372 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -115,7 +115,7 @@ public: ~WriteBufferFromZipArchive() override { - //! + /// ! #50274 try { finalize(); @@ -194,7 +194,7 @@ namespace ~StreamFromWriteBuffer() { - //! + /// ! #50274 write_buffer->finalize(); } diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp index 0c434b4981e..98786313b09 100644 --- a/src/IO/BrotliWriteBuffer.cpp +++ b/src/IO/BrotliWriteBuffer.cpp @@ -44,7 +44,7 @@ BrotliWriteBuffer::BrotliWriteBuffer(std::unique_ptr out_, int comp BrotliWriteBuffer::~BrotliWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index e5aab177f3a..c7159d4d002 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -47,7 +47,7 @@ Bzip2WriteBuffer::Bzip2WriteBuffer(std::unique_ptr out_, int compre Bzip2WriteBuffer::~Bzip2WriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index b3481203757..ea38f70745d 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -53,7 +53,7 @@ void ForkWriteBuffer::finalizeImpl() ForkWriteBuffer::~ForkWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index 3e1c9dc58b9..07597ce958b 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -46,7 +46,7 @@ LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 2627643183d..988d9861a9b 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -42,7 +42,7 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/SnappyWriteBuffer.cpp b/src/IO/SnappyWriteBuffer.cpp index 3ea6f831d5b..5203d8b053e 100644 --- a/src/IO/SnappyWriteBuffer.cpp +++ b/src/IO/SnappyWriteBuffer.cpp @@ -22,7 +22,7 @@ SnappyWriteBuffer::SnappyWriteBuffer(std::unique_ptr out_, size_t b SnappyWriteBuffer::~SnappyWriteBuffer() { - //! + /// ! #50274 finish(); } diff --git a/src/IO/WriteBufferFromEncryptedFile.cpp b/src/IO/WriteBufferFromEncryptedFile.cpp index 7efdbc43238..5831b3aee94 100644 --- a/src/IO/WriteBufferFromEncryptedFile.cpp +++ b/src/IO/WriteBufferFromEncryptedFile.cpp @@ -21,7 +21,7 @@ WriteBufferFromEncryptedFile::WriteBufferFromEncryptedFile( WriteBufferFromEncryptedFile::~WriteBufferFromEncryptedFile() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index 964f7415803..0777d596184 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -77,7 +77,7 @@ WriteBufferFromFile::~WriteBufferFromFile() if (fd < 0) return; - //! + /// ! #50274 finalize(); int err = ::close(fd); /// Everything except for EBADF should be ignored in dtor, since all of diff --git a/src/IO/WriteBufferFromFileDecorator.cpp b/src/IO/WriteBufferFromFileDecorator.cpp index 82e38eac72d..67a98d9e43f 100644 --- a/src/IO/WriteBufferFromFileDecorator.cpp +++ b/src/IO/WriteBufferFromFileDecorator.cpp @@ -30,7 +30,7 @@ void WriteBufferFromFileDecorator::finalizeImpl() WriteBufferFromFileDecorator::~WriteBufferFromFileDecorator() { - //! + /// ! #50274 // try // { // finalize(); diff --git a/src/IO/WriteBufferFromFileDescriptor.cpp b/src/IO/WriteBufferFromFileDescriptor.cpp index 0516171c051..03988c09f62 100644 --- a/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/src/IO/WriteBufferFromFileDescriptor.cpp @@ -105,7 +105,7 @@ WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor( WriteBufferFromFileDescriptor::~WriteBufferFromFileDescriptor() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromOStream.cpp b/src/IO/WriteBufferFromOStream.cpp index d55aa054174..25886b9d10d 100644 --- a/src/IO/WriteBufferFromOStream.cpp +++ b/src/IO/WriteBufferFromOStream.cpp @@ -40,7 +40,7 @@ WriteBufferFromOStream::WriteBufferFromOStream( WriteBufferFromOStream::~WriteBufferFromOStream() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index a8b28445294..a0dcf69e67d 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -106,7 +106,7 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index b0ff3183291..ab4f27a7bcb 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -63,7 +63,7 @@ public: ~WriteBufferFromVector() override { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index 4440254d15f..7ae8f636f69 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -74,7 +74,7 @@ void ZlibDeflatingWriteBuffer::nextImpl() ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer() { - //! + /// ! #50274 try { finalize(); diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index 05ee23158fd..8f66618f513 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -87,7 +87,7 @@ void ZstdDeflatingAppendableWriteBuffer::nextImpl() ZstdDeflatingAppendableWriteBuffer::~ZstdDeflatingAppendableWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index 4ec3f5a3fce..95e2e7d7716 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -33,7 +33,7 @@ ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer( ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() { - //! + /// ! #50274 finalize(); } diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index ed831350186..cbbf22078c1 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -73,7 +73,7 @@ std::shared_ptr WriteBufferToFileSegment::getReadBufferImpl() WriteBufferToFileSegment::~WriteBufferToFileSegment() { - //! + /// ! #50274 try { finalize(); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 14e0bff6966..2fc21e84be7 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -184,7 +184,7 @@ struct TemporaryFileStream::OutputWriter ~OutputWriter() { - //! + /// ! #50274 try { finalize(); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 03222fe7e0b..7d6710e76e6 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -171,7 +171,7 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { - //! + /// ! #50274 finalize(); } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 48f35c378ba..169cf845d3b 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -147,7 +147,7 @@ void WriteBufferFromHDFS::finalizeImpl() WriteBufferFromHDFS::~WriteBufferFromHDFS() { - //! + /// ! #50274 finalize(); } From a5dcd8dabbd2ba9fd796b0ccacf447fc75ecc97e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 27 May 2023 20:51:59 +0200 Subject: [PATCH 0790/2223] do not call finalize in d-tor MergedBlockOutputStream::Finalizer --- src/IO/WriteBufferFromS3.cpp | 7 +++++-- .../MergeTree/MergedBlockOutputStream.cpp | 17 +++++++++-------- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 4657a65f931..58d58e86ebe 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -109,8 +109,8 @@ void WriteBufferFromS3::nextImpl() if (is_prefinalized) throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot write to prefinalized buffer for S3, the file could have been created with PutObjectRequest"); + ErrorCodes::LOGICAL_ERROR, + "Cannot write to prefinalized buffer for S3, the file could have been created with PutObjectRequest"); /// Make sense to call waitIfAny before adding new async task to check if there is an exception /// The faster the exception is propagated the lesser time is spent for cancellation @@ -183,12 +183,15 @@ void WriteBufferFromS3::finalizeImpl() preFinalize(); if (std::uncaught_exception()) + { + tryLogCurrentException(__PRETTY_FUNCTION__); throw Exception( ErrorCodes::LOGICAL_ERROR, "Detected buffer finalization when an exception is unwinding the stack." " Do not call finalize buffer in destructors or when exception thrown." " Details {}", getLogDetails()); + } chassert(offset() == 0); chassert(hidden_size == 0); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index d97da5a0b50..7bff58fc411 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -121,14 +121,15 @@ void MergedBlockOutputStream::Finalizer::Impl::finish() MergedBlockOutputStream::Finalizer::~Finalizer() { - try - { - finish(); - } - catch (...) - { - tryLogCurrentException("MergedBlockOutputStream"); - } + /// ! #50274 +// try +// { +// finish(); +// } +// catch (...) +// { +// tryLogCurrentException("MergedBlockOutputStream"); +// } } MergedBlockOutputStream::Finalizer::Finalizer(Finalizer &&) noexcept = default; From 13dcb62ffb7abfe12bef0fb300e9c9e62610f754 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 28 May 2023 12:29:38 +0200 Subject: [PATCH 0791/2223] fix logs engins --- src/IO/WriteBufferFromS3.cpp | 8 +++++++- src/Storages/StorageLog.cpp | 3 +++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 58d58e86ebe..5bfc6b91946 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -253,7 +253,13 @@ WriteBufferFromS3::~WriteBufferFromS3() // That destructor could be call with finalized=false in case of exceptions if (!finalized) { - LOG_ERROR(log, "WriteBufferFromS3 is not finalized in destructor. It could be if an exception occurs. File is not written to S3. {}.", getLogDetails()); + LOG_INFO(log, + "WriteBufferFromS3 is not finalized in destructor. " + "It could be if an exception occurs. File is not written to S3. " + "{}. " + "Stack trace: {}", + getLogDetails(), + StackTrace().toString()); } task_tracker->safeWaitAll(); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index f698f1881fa..02dc4843660 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -341,7 +341,10 @@ private: void finalize() { compressed.next(); + compressed.finalize(); + plain->next(); + plain->finalize(); } }; From b13990efcc184c17356a14b67ec5057718f76e4c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 28 May 2023 15:32:10 +0200 Subject: [PATCH 0792/2223] fix build --- src/IO/WriteBufferFromS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 5bfc6b91946..640763f793c 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -182,7 +182,7 @@ void WriteBufferFromS3::finalizeImpl() if (!is_prefinalized) preFinalize(); - if (std::uncaught_exception()) + if (std::uncaught_exceptions()) { tryLogCurrentException(__PRETTY_FUNCTION__); throw Exception( From 0b4ea3e2e1d5d91d96bbb13965b00a607664879d Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 29 May 2023 00:37:45 +0200 Subject: [PATCH 0793/2223] remove reminder comments --- src/Compression/CompressedWriteBuffer.cpp | 1 - src/Coordination/WriteBufferFromNuraftBuffer.cpp | 1 - src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp | 1 - src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 1 - src/Disks/IO/WriteBufferWithFinalizeCallback.cpp | 14 -------------- src/Disks/IO/WriteBufferWithFinalizeCallback.h | 2 -- src/IO/Archives/ZipArchiveWriter.cpp | 2 -- src/IO/BrotliWriteBuffer.cpp | 1 - src/IO/Bzip2WriteBuffer.cpp | 1 - src/IO/ForkWriteBuffer.cpp | 1 - src/IO/LZMADeflatingWriteBuffer.cpp | 1 - src/IO/Lz4DeflatingWriteBuffer.cpp | 1 - src/IO/SnappyWriteBuffer.cpp | 1 - src/IO/WriteBufferFromEncryptedFile.cpp | 1 - src/IO/WriteBufferFromFile.cpp | 1 - src/IO/WriteBufferFromFileDecorator.cpp | 10 ---------- src/IO/WriteBufferFromFileDescriptor.cpp | 1 - src/IO/WriteBufferFromOStream.cpp | 1 - src/IO/WriteBufferFromPocoSocket.cpp | 1 - src/IO/WriteBufferFromS3.cpp | 4 ++-- src/IO/WriteBufferFromS3TaskTracker.cpp | 8 -------- src/IO/WriteBufferFromVector.h | 1 - src/IO/ZlibDeflatingWriteBuffer.cpp | 1 - src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 1 - src/IO/ZstdDeflatingWriteBuffer.cpp | 1 - .../Cache/WriteBufferToFileSegment.cpp | 1 - src/Interpreters/TemporaryDataOnDisk.cpp | 1 - .../HTTP/WriteBufferFromHTTPServerResponse.cpp | 1 - src/Storages/HDFS/WriteBufferFromHDFS.cpp | 1 - .../MergeTree/MergedBlockOutputStream.cpp | 15 ++------------- src/Storages/MergeTree/MergedBlockOutputStream.h | 3 ++- tests/integration/test_merge_tree_s3/test.py | 8 ++++---- 32 files changed, 10 insertions(+), 79 deletions(-) diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index 22c5f235a27..cb2ee1140d0 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -59,7 +59,6 @@ void CompressedWriteBuffer::nextImpl() CompressedWriteBuffer::~CompressedWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp index 41b933717c6..c955d3fdbbe 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.cpp +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -53,7 +53,6 @@ nuraft::ptr WriteBufferFromNuraftBuffer::getBuffer() WriteBufferFromNuraftBuffer::~WriteBufferFromNuraftBuffer() { - /// ! #50274 try { finalize(); diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 8da9ce73ae3..9153af90312 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -123,7 +123,6 @@ void FileSegmentRangeWriter::finalize() FileSegmentRangeWriter::~FileSegmentRangeWriter() { - /// ! #50274 try { if (!finalized) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 8ac2c8ca831..b5d296bd865 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -38,7 +38,6 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage() { - /// ! #50274 finalize(); } diff --git a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp index 47bf742fd5b..8703eae4913 100644 --- a/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.cpp @@ -13,20 +13,6 @@ WriteBufferWithFinalizeCallback::WriteBufferWithFinalizeCallback( { } - -WriteBufferWithFinalizeCallback::~WriteBufferWithFinalizeCallback() -{ - /// ! #50274 -// try -// { -// finalize(); -// } -// catch (...) -// { -// tryLogCurrentException(__PRETTY_FUNCTION__); -// } -} - void WriteBufferWithFinalizeCallback::finalizeImpl() { WriteBufferFromFileDecorator::finalizeImpl(); diff --git a/src/Disks/IO/WriteBufferWithFinalizeCallback.h b/src/Disks/IO/WriteBufferWithFinalizeCallback.h index 73c1b8d25d4..2798582c336 100644 --- a/src/Disks/IO/WriteBufferWithFinalizeCallback.h +++ b/src/Disks/IO/WriteBufferWithFinalizeCallback.h @@ -19,8 +19,6 @@ public: FinalizeCallback && create_callback_, const String & remote_path_); - ~WriteBufferWithFinalizeCallback() override; - String getFileName() const override { return remote_path; } private: diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 1c6ef18d372..ed4b3502b2f 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -115,7 +115,6 @@ public: ~WriteBufferFromZipArchive() override { - /// ! #50274 try { finalize(); @@ -194,7 +193,6 @@ namespace ~StreamFromWriteBuffer() { - /// ! #50274 write_buffer->finalize(); } diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp index 98786313b09..47426d62a6e 100644 --- a/src/IO/BrotliWriteBuffer.cpp +++ b/src/IO/BrotliWriteBuffer.cpp @@ -44,7 +44,6 @@ BrotliWriteBuffer::BrotliWriteBuffer(std::unique_ptr out_, int comp BrotliWriteBuffer::~BrotliWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index c7159d4d002..4b6bed70d35 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -47,7 +47,6 @@ Bzip2WriteBuffer::Bzip2WriteBuffer(std::unique_ptr out_, int compre Bzip2WriteBuffer::~Bzip2WriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/ForkWriteBuffer.cpp b/src/IO/ForkWriteBuffer.cpp index ea38f70745d..8e11b9ff590 100644 --- a/src/IO/ForkWriteBuffer.cpp +++ b/src/IO/ForkWriteBuffer.cpp @@ -53,7 +53,6 @@ void ForkWriteBuffer::finalizeImpl() ForkWriteBuffer::~ForkWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index 07597ce958b..30e247b1016 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -46,7 +46,6 @@ LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 988d9861a9b..c3a1b8282c3 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -42,7 +42,6 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/SnappyWriteBuffer.cpp b/src/IO/SnappyWriteBuffer.cpp index 5203d8b053e..ca40d0656d1 100644 --- a/src/IO/SnappyWriteBuffer.cpp +++ b/src/IO/SnappyWriteBuffer.cpp @@ -22,7 +22,6 @@ SnappyWriteBuffer::SnappyWriteBuffer(std::unique_ptr out_, size_t b SnappyWriteBuffer::~SnappyWriteBuffer() { - /// ! #50274 finish(); } diff --git a/src/IO/WriteBufferFromEncryptedFile.cpp b/src/IO/WriteBufferFromEncryptedFile.cpp index 5831b3aee94..5bca0dc68d5 100644 --- a/src/IO/WriteBufferFromEncryptedFile.cpp +++ b/src/IO/WriteBufferFromEncryptedFile.cpp @@ -21,7 +21,6 @@ WriteBufferFromEncryptedFile::WriteBufferFromEncryptedFile( WriteBufferFromEncryptedFile::~WriteBufferFromEncryptedFile() { - /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index 0777d596184..e58f1e3a60c 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -77,7 +77,6 @@ WriteBufferFromFile::~WriteBufferFromFile() if (fd < 0) return; - /// ! #50274 finalize(); int err = ::close(fd); /// Everything except for EBADF should be ignored in dtor, since all of diff --git a/src/IO/WriteBufferFromFileDecorator.cpp b/src/IO/WriteBufferFromFileDecorator.cpp index 67a98d9e43f..0e4e5e13a86 100644 --- a/src/IO/WriteBufferFromFileDecorator.cpp +++ b/src/IO/WriteBufferFromFileDecorator.cpp @@ -30,16 +30,6 @@ void WriteBufferFromFileDecorator::finalizeImpl() WriteBufferFromFileDecorator::~WriteBufferFromFileDecorator() { - /// ! #50274 -// try -// { -// finalize(); -// } -// catch (...) -// { -// tryLogCurrentException(__PRETTY_FUNCTION__); -// } - /// It is not a mistake that swap is called here /// Swap has been called at constructor, it should be called at destructor /// In oreder to provide valid buffer for impl's d-tor call diff --git a/src/IO/WriteBufferFromFileDescriptor.cpp b/src/IO/WriteBufferFromFileDescriptor.cpp index 03988c09f62..135ff608967 100644 --- a/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/src/IO/WriteBufferFromFileDescriptor.cpp @@ -105,7 +105,6 @@ WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor( WriteBufferFromFileDescriptor::~WriteBufferFromFileDescriptor() { - /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromOStream.cpp b/src/IO/WriteBufferFromOStream.cpp index 25886b9d10d..2d0d5976f85 100644 --- a/src/IO/WriteBufferFromOStream.cpp +++ b/src/IO/WriteBufferFromOStream.cpp @@ -40,7 +40,6 @@ WriteBufferFromOStream::WriteBufferFromOStream( WriteBufferFromOStream::~WriteBufferFromOStream() { - /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index a0dcf69e67d..039110dfb62 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -106,7 +106,6 @@ WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_ WriteBufferFromPocoSocket::~WriteBufferFromPocoSocket() { - /// ! #50274 finalize(); } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 640763f793c..44feb225471 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -538,7 +538,7 @@ void WriteBufferFromS3::completeMultipartUpload() if (multipart_tags.empty()) throw Exception( - ErrorCodes::S3_ERROR, + ErrorCodes::LOGICAL_ERROR, "Failed to complete multipart upload. No parts have uploaded"); for (size_t i = 0; i < multipart_tags.size(); ++i) @@ -546,7 +546,7 @@ void WriteBufferFromS3::completeMultipartUpload() const auto tag = multipart_tags.at(i); if (tag.empty()) throw Exception( - ErrorCodes::S3_ERROR, + ErrorCodes::LOGICAL_ERROR, "Failed to complete multipart upload. Part {} haven't been uploaded.", i); } diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 7cf5a89df86..7ae31044012 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -162,14 +162,6 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() for (auto & it : finished_futures) { -// SCOPE_EXIT({ -// /// According to basic exception safety TaskTracker has to be destroyed after exception -// /// If it would be true than this SCOPE_EXIT is superfluous -// /// However WriteBufferWithFinalizeCallback, WriteBufferFromFileDecorator do call finalize in d-tor -// /// TaskTracker has to cope this until the issue with finalizing in d-tor is addressed in #50274 -// futures.erase(it); -// }); - it->get(); futures.erase(it); } diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index ab4f27a7bcb..4b2a3581625 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -63,7 +63,6 @@ public: ~WriteBufferFromVector() override { - /// ! #50274 finalize(); } diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index 7ae8f636f69..43bb0405555 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -74,7 +74,6 @@ void ZlibDeflatingWriteBuffer::nextImpl() ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer() { - /// ! #50274 try { finalize(); diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index 8f66618f513..be739c0e654 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -87,7 +87,6 @@ void ZstdDeflatingAppendableWriteBuffer::nextImpl() ZstdDeflatingAppendableWriteBuffer::~ZstdDeflatingAppendableWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index 95e2e7d7716..c6d2ffc39f9 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -33,7 +33,6 @@ ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer( ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() { - /// ! #50274 finalize(); } diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index cbbf22078c1..1eac87a804d 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -73,7 +73,6 @@ std::shared_ptr WriteBufferToFileSegment::getReadBufferImpl() WriteBufferToFileSegment::~WriteBufferToFileSegment() { - /// ! #50274 try { finalize(); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 2fc21e84be7..69fef21dbab 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -184,7 +184,6 @@ struct TemporaryFileStream::OutputWriter ~OutputWriter() { - /// ! #50274 try { finalize(); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 7d6710e76e6..c8015cfd185 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -171,7 +171,6 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { - /// ! #50274 finalize(); } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 169cf845d3b..fad0447d2cf 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -147,7 +147,6 @@ void WriteBufferFromHDFS::finalizeImpl() WriteBufferFromHDFS::~WriteBufferFromHDFS() { - /// ! #50274 finalize(); } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 7bff58fc411..7e69f03e959 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -119,23 +119,12 @@ void MergedBlockOutputStream::Finalizer::Impl::finish() part->getDataPartStorage().removeFile(file_name); } -MergedBlockOutputStream::Finalizer::~Finalizer() -{ - /// ! #50274 -// try -// { -// finish(); -// } -// catch (...) -// { -// tryLogCurrentException("MergedBlockOutputStream"); -// } -} - MergedBlockOutputStream::Finalizer::Finalizer(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer & MergedBlockOutputStream::Finalizer::operator=(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer::Finalizer(std::unique_ptr impl_) : impl(std::move(impl_)) {} +MergedBlockOutputStream::Finalizer::~Finalizer() {} + void MergedBlockOutputStream::finalizePart( const MergeTreeMutableDataPartPtr & new_part, bool sync, diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index ad1bb584788..f3a5653a880 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -44,9 +44,10 @@ public: std::unique_ptr impl; explicit Finalizer(std::unique_ptr impl_); - ~Finalizer(); Finalizer(Finalizer &&) noexcept; Finalizer & operator=(Finalizer &&) noexcept; + ~Finalizer(); + void finish(); }; diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 4dcc5805294..626a71f006e 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -1049,8 +1049,8 @@ def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): " AND type!='QueryStart'" ).split() - assert int(memory_usage) < 1.1 * memory - assert int(memory_usage) > 0.9 * memory + assert int(memory_usage) < 1.2 * memory + assert int(memory_usage) > 0.8 * memory assert int(wait_inflight) > 10 * 1000 * 1000 @@ -1097,7 +1097,7 @@ def test_s3_disk_heavy_write_check_mem(cluster, node_name): " AND type!='QueryStart'" ) - assert int(result) < 1.1 * memory - assert int(result) > 0.9 * memory + assert int(result) < 1.2 * memory + assert int(result) > 0.8 * memory check_no_objects_after_drop(cluster, node_name=node_name) From 2d2b411c26fc4e64bce894a48684c144f3b689b1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 May 2023 12:21:33 +0200 Subject: [PATCH 0794/2223] fix build --- src/Storages/MergeTree/MergedBlockOutputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 7e69f03e959..c93ad135835 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -123,7 +123,7 @@ MergedBlockOutputStream::Finalizer::Finalizer(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer & MergedBlockOutputStream::Finalizer::operator=(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer::Finalizer(std::unique_ptr impl_) : impl(std::move(impl_)) {} -MergedBlockOutputStream::Finalizer::~Finalizer() {} +MergedBlockOutputStream::Finalizer::~Finalizer() = default; void MergedBlockOutputStream::finalizePart( const MergeTreeMutableDataPartPtr & new_part, From 0a128cec6179fbc97c9329538f3e28c2ca1567d1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 May 2023 12:28:58 +0200 Subject: [PATCH 0795/2223] remove tricky debug trap --- src/IO/WriteBufferFromS3.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 44feb225471..462cf2674c3 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -182,17 +182,6 @@ void WriteBufferFromS3::finalizeImpl() if (!is_prefinalized) preFinalize(); - if (std::uncaught_exceptions()) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Detected buffer finalization when an exception is unwinding the stack." - " Do not call finalize buffer in destructors or when exception thrown." - " Details {}", - getLogDetails()); - } - chassert(offset() == 0); chassert(hidden_size == 0); From fe82d2bbe24a98bf2192796f47a9b1e2b5d40b3a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 30 May 2023 12:58:37 +0200 Subject: [PATCH 0796/2223] Fix build --- src/Common/checkSSLReturnCode.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/checkSSLReturnCode.cpp b/src/Common/checkSSLReturnCode.cpp index 8916a25e19c..bc87af1d37d 100644 --- a/src/Common/checkSSLReturnCode.cpp +++ b/src/Common/checkSSLReturnCode.cpp @@ -8,7 +8,7 @@ namespace DB { -bool checkSSLWantRead(ssize_t res) +bool checkSSLWantRead([[maybe_unused]] ssize_t res) { #if USE_SSL return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ; @@ -17,7 +17,7 @@ bool checkSSLWantRead(ssize_t res) #endif } -bool checkSSLWantWrite(ssize_t res) +bool checkSSLWantWrite([[maybe_unused]] ssize_t res) { #if USE_SSL return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE; From fe944a55dc4f99bd5fd69da54427be83abe63ac9 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 May 2023 13:30:47 +0200 Subject: [PATCH 0797/2223] Updated to hold temporary_directory_lock till part is fetched, renamed and committed --- src/Storages/MergeTree/DataPartsExchange.cpp | 12 +++++----- src/Storages/MergeTree/DataPartsExchange.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 23 +++++++++++++++----- 3 files changed, 24 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 21fdda28767..357d48ae466 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -365,7 +365,7 @@ Fetcher::Fetcher(StorageReplicatedMergeTree & data_) , log(&Poco::Logger::get(data.getStorageID().getNameForLogs() + " (Fetcher)")) {} -MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( +std::pair Fetcher::fetchSelectedPart( const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const String & part_name, @@ -601,7 +601,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( return std::make_unique(full_path, std::min(DBMS_DEFAULT_BUFFER_SIZE, file_size)); }; - return downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix, disk, true, *in, output_buffer_getter, projections, throttler, sync); + return std::make_pair(downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix, disk, true, *in, output_buffer_getter, projections, throttler, sync), std::move(temporary_directory_lock)); } catch (const Exception & e) { @@ -667,11 +667,11 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( data.getRelativeDataPath(), part_name); - return downloadPartToMemory( + return std::make_pair(downloadPartToMemory( data_part_storage, part_name, MergeTreePartInfo::fromPartName(part_name, data.format_version), part_uuid, metadata_snapshot, context, *in, - projections, false, throttler); + projections, false, throttler), std::move(temporary_directory_lock)); } auto output_buffer_getter = [](IDataPartStorage & part_storage, const String & file_name, size_t file_size) @@ -679,10 +679,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( return part_storage.writeFile(file_name, std::min(file_size, DBMS_DEFAULT_BUFFER_SIZE), {}); }; - return downloadPartToDisk( + return std::make_pair(downloadPartToDisk( part_name, replica_path, to_detached, tmp_prefix, disk, false, *in, output_buffer_getter, - projections, throttler, sync); + projections, throttler, sync),std::move(temporary_directory_lock)); } MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 704c1b0a226..07939a660a8 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -66,7 +66,7 @@ public: explicit Fetcher(StorageReplicatedMergeTree & data_); /// Downloads a part to tmp_directory. If to_detached - downloads to the `detached` directory. - MergeTreeData::MutableDataPartPtr fetchSelectedPart( + std::pair fetchSelectedPart( const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const String & part_name, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0f79e9f8f19..56896f88423 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2344,16 +2344,19 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); + scope_guard part_temp_directory_lock; if (interserver_scheme != address.scheme) throw Exception(ErrorCodes::LOGICAL_ERROR, "Interserver schemas are different '{}' != '{}', can't fetch part from {}", interserver_scheme, address.scheme, address.host); - part_desc->res_part = fetcher.fetchSelectedPart( + auto [fetched_part, lock] = fetcher.fetchSelectedPart( metadata_snapshot, getContext(), part_desc->found_new_part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, false, TMP_PREFIX + "fetch_"); + part_desc->res_part = fetched_part; + part_temp_directory_lock = std::move(lock); /// TODO: check columns_version of fetched part @@ -2460,6 +2463,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr auto timeouts = getHTTPTimeouts(getContext()); auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); + scope_guard part_temp_directory_lock; auto get_part = [&, address, timeouts, credentials, interserver_scheme]() { @@ -2467,11 +2471,13 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr throw Exception(ErrorCodes::LOGICAL_ERROR, "Interserver schemes are different: '{}' != '{}', can't fetch part from {}", interserver_scheme, address.scheme, address.host); - return fetcher.fetchSelectedPart( + auto [fetched_part, lock] = fetcher.fetchSelectedPart( metadata_snapshot, getContext(), entry.new_part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, true); + part_temp_directory_lock = std::move(lock); + return fetched_part; }; part = get_part(); @@ -4170,14 +4176,14 @@ bool StorageReplicatedMergeTree::fetchPart( std::optional tagger_ptr; std::function get_part; MergeTreeData::HardlinkedFiles hardlinked_files; - scope_guard part_to_clone_lock; + scope_guard part_directory_lock; if (part_to_clone) { get_part = [&, part_to_clone]() { auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot, NO_TRANSACTION_PTR, &hardlinked_files, false, {}); - part_to_clone_lock = std::move(lock); + part_directory_lock = std::move(lock); return cloned_part; }; } @@ -4195,7 +4201,7 @@ bool StorageReplicatedMergeTree::fetchPart( throw Exception(ErrorCodes::INTERSERVER_SCHEME_DOESNT_MATCH, "Interserver schemes are different: " "'{}' != '{}', can't fetch part from {}", interserver_scheme, address.scheme, address.host); - return fetcher.fetchSelectedPart( + auto [fetched_part, lock] = fetcher.fetchSelectedPart( metadata_snapshot, getContext(), part_name, @@ -4212,6 +4218,8 @@ bool StorageReplicatedMergeTree::fetchPart( "", &tagger_ptr, try_fetch_shared); + part_directory_lock = std::move(lock); + return fetched_part; }; } @@ -4355,6 +4363,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( auto timeouts = getHTTPTimeouts(getContext()); auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); + scope_guard part_temp_directory_lock; get_part = [&, address, timeouts, interserver_scheme, credentials]() { @@ -4362,12 +4371,14 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( throw Exception(ErrorCodes::INTERSERVER_SCHEME_DOESNT_MATCH, "Interserver schemes are different: " "'{}' != '{}', can't fetch part from {}", interserver_scheme, address.scheme, address.host); - return fetcher.fetchSelectedPart( + auto [fetched_part, lock] = fetcher.fetchSelectedPart( metadata_snapshot, getContext(), part_name, zookeeper_name, source_replica_path, address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, replicated_fetches_throttler, false, "", nullptr, true, replaced_disk); + part_temp_directory_lock = std::move(lock); + return fetched_part; }; try From a69aa3f901d24a083f1be3d90eaa927138617f19 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Tue, 30 May 2023 13:39:32 +0200 Subject: [PATCH 0798/2223] Update README.md --- README.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index bbedea364fc..9c933540e01 100644 --- a/README.md +++ b/README.md @@ -23,10 +23,9 @@ curl https://clickhouse.com/ | sh ## Upcoming Events * [**v23.5 Release Webinar**](https://clickhouse.com/company/events/v23-5-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-05) - May 31 - 23.5 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. -* [**ClickHouse Meetup in Barcelona**](https://www.meetup.com/clickhouse-barcelona-user-group/events/292892669) - May 25 -* [**ClickHouse Meetup in London**](https://www.meetup.com/clickhouse-london-user-group/events/292892824) - May 25 +* [**ClickHouse Meetup in Bangalore**](https://www.meetup.com/clickhouse-bangalore-user-group/events/293740066/) - Jun 7 * [**ClickHouse Meetup in San Francisco**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/293426725/) - Jun 7 -* [**ClickHouse Meetup in Stockholm**](https://www.meetup.com/clickhouse-berlin-user-group/events/292892466) - Jun 13 + Also, keep an eye out for upcoming meetups in Amsterdam, Boston, NYC, Beijing, and Toronto. Somewhere else you want us to be? Please feel free to reach out to tyler clickhouse com. From 1cbce2196849dfb3dc8189a0f4c117b725c22fe3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 May 2023 11:46:23 +0000 Subject: [PATCH 0799/2223] Better, pt. II --- src/Functions/FunctionsHashing.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index d08a900c88a..14fad62bd85 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1426,6 +1426,9 @@ public: { auto col_to = ColumnVector::create(input_rows_count); + if (input_rows_count == 0) + return col_to; + typename ColumnVector::Container & vec_to = col_to->getData(); /// If using a "keyed" algorithm, the first argument is the key and @@ -1441,7 +1444,7 @@ public: KeyType key{}; if constexpr (Keyed) - if (!arguments.empty() && input_rows_count != 0) + if (!arguments.empty()) key = Impl::parseKey(arguments[0]); /// The function supports arbitrary number of arguments of arbitrary types. From bec8186b2f23aa0f7c4d0f714fce8e7f2b871fae Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 May 2023 13:59:31 +0200 Subject: [PATCH 0800/2223] do not fix the event list --- .../test_profile_events_s3/test.py | 62 ++----------------- 1 file changed, 6 insertions(+), 56 deletions(-) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index 5c56b5c05cb..10c9385f865 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -31,61 +31,8 @@ def cluster(): cluster.shutdown() -init_list = { - "ReadBufferFromS3Bytes": 0, - "ReadBufferFromS3Microseconds": 0, - "ReadBufferFromS3InitMicroseconds": 0, - "ReadBufferFromS3RequestsErrors": 0, - "WriteBufferFromS3Bytes": 0, - "WriteBufferFromS3Microseconds": 0, - "WriteBufferFromS3RequestsErrors": 0, - "S3ReadMicroseconds": 0, - "S3ReadRequestsCount": 0, - "S3ReadRequestsErrorsTotal": 0, - "S3ReadRequestsErrors503": 0, - "S3ReadRequestsRedirects": 0, - "S3WriteMicroseconds": 0, - "S3WriteRequestsCount": 0, - "S3WriteRequestsErrorsTotal": 0, - "S3WriteRequestsErrors503": 0, - "S3WriteRequestsRedirects": 0, - "DiskS3ReadMicroseconds": 0, - "DiskS3ReadRequestsCount": 0, - "DiskS3ReadRequestsErrorsTotal": 0, - "DiskS3ReadRequestsErrors503": 0, - "DiskS3ReadRequestsRedirects": 0, - "DiskS3WriteMicroseconds": 0, - "DiskS3WriteRequestsCount": 0, - "DiskS3WriteRequestsErrorsTotal": 0, - "DiskS3WriteRequestsErrors503": 0, - "DiskS3WriteRequestsRedirects": 0, - "S3DeleteObjects": 0, - "S3CopyObject": 0, - "S3ListObjects": 0, - "S3HeadObject": 0, - "S3CreateMultipartUpload": 0, - "S3UploadPartCopy": 0, - "S3UploadPart": 0, - "S3AbortMultipartUpload": 0, - "S3CompleteMultipartUpload": 0, - "S3PutObject": 0, - "S3GetObject": 0, - "DiskS3DeleteObjects": 0, - "DiskS3CopyObject": 0, - "DiskS3ListObjects": 0, - "DiskS3HeadObject": 0, - "DiskS3CreateMultipartUpload": 0, - "DiskS3UploadPartCopy": 0, - "DiskS3UploadPart": 0, - "DiskS3AbortMultipartUpload": 0, - "DiskS3CompleteMultipartUpload": 0, - "DiskS3PutObject": 0, - "DiskS3GetObject": 0, -} - - def get_s3_events(instance): - result = init_list.copy() + result = dict() events = instance.query( "SELECT event, value FROM system.events WHERE event LIKE '%S3%'" ).split("\n") @@ -130,7 +77,7 @@ def get_minio_stat(cluster): def get_query_stat(instance, hint): - result = init_list.copy() + result = dict() instance.query("SYSTEM FLUSH LOGS") events = instance.query( """ @@ -146,7 +93,10 @@ def get_query_stat(instance, hint): ev = event.split("\t") if len(ev) == 2: if "S3" in ev[0]: - result[ev[0]] += int(ev[1]) + if ev[0] in result: + result[ev[0]] += int(ev[1]) + else: + result[ev[0]] = int(ev[1]) return result From c25980bcf4ecae8aaec15b75421b3a187b410ab2 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 30 May 2023 12:02:44 +0000 Subject: [PATCH 0801/2223] Trying to fix toDateOrDefault() --- src/Functions/FunctionsConversion.h | 7 +++++++ .../01746_convert_type_with_default.reference | 4 ++++ .../0_stateless/01746_convert_type_with_default.sql | 10 ++++++++++ 3 files changed, 21 insertions(+) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 940585d6d57..d3676349318 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2882,6 +2882,13 @@ private: return true; } + if constexpr (IsDataTypeNumber && IsDataTypeDateOrDateTime) + { + result_column = ConvertImpl::execute( + arguments, result_type, input_rows_count); + return true; + } + return false; }); diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.reference b/tests/queries/0_stateless/01746_convert_type_with_default.reference index 9ebef9c4a8d..ec2a826982f 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.reference +++ b/tests/queries/0_stateless/01746_convert_type_with_default.reference @@ -20,5 +20,9 @@ 2 -1 -2 +2023-05-30 +2023-05-30 +2023-05-30 14:38:20 +2023-05-30 14:38:20 61f0c404-5cb3-11e7-907b-a6006ad3dba0 59f0c404-5cb3-11e7-907b-a6006ad3dba0 diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index 0881e911466..9d7873081e5 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -26,5 +26,15 @@ select toUInt256OrDefault('1xx', cast(2 as UInt256)); select toInt256OrDefault('-1', cast(-2 as Int256)); select toInt256OrDefault('-1xx', cast(-2 as Int256)); +select toDateOrDefault('2023-05-30'); +select toDateOrDefault(19507); + +select toDateTimeOrDefault('2023-05-30 14:38:20'); +select toDateTimeOrDefault(1685457500); + SELECT toUUIDOrDefault('61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); SELECT toUUIDOrDefault('-----61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); + + + + From c87e8c1f23bb88ac981ff59405e4f9a829e4fce6 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 30 May 2023 12:12:54 +0000 Subject: [PATCH 0802/2223] Fix bitShift* functions with both constant arguments --- src/Functions/FunctionBinaryArithmetic.h | 2 +- ...66_bitshift_with_const_arguments.reference | 7 ++++++ .../02766_bitshift_with_const_arguments.sql | 22 +++++++++++++++++++ 3 files changed, 30 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02766_bitshift_with_const_arguments.reference create mode 100644 tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index ec608883111..b205822aab5 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1741,7 +1741,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A OpImpl::template processString(in_vec.data(), col_left->getOffsets().data(), &value, out_vec, out_offsets, 1); } - return ColumnConst::create(std::move(col_res), col_left->size()); + return ColumnConst::create(std::move(col_res), col_left_const->size()); } else if (!col_left_const && !col_right_const && col_right) { diff --git a/tests/queries/0_stateless/02766_bitshift_with_const_arguments.reference b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.reference new file mode 100644 index 00000000000..9a05aeb71aa --- /dev/null +++ b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.reference @@ -0,0 +1,7 @@ +\0hdhdfhp +\0hdhdfhp +\0hdhdfhp +\0hdhdfhp +\0hdhdfhp +\0bdf +13000 diff --git a/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql new file mode 100644 index 00000000000..6b2961f0555 --- /dev/null +++ b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql @@ -0,0 +1,22 @@ +SELECT bitShiftLeft(if(number = NULL, '14342', '4242348'), 1) FROM numbers(1); +SELECT bitShiftLeft(if(number = NULL, '14342', '4242348'), 1) FROM numbers(3); +SELECT bitShiftLeft(if(materialize(NULL), '14342', '4242348'), 1) FROM numbers(1); +SELECT bitShiftLeft(if(materialize(1), '123', '123'), 1) from numbers(1); + + +-- The next queries are from fuzzer that found the bug: +DROP TABLE IF EXISTS t0; +DROP TABLE IF EXISTS t1; +CREATE TABLE t0 (vkey UInt32, pkey UInt32, c0 UInt32) engine = TinyLog; +CREATE TABLE t1 (vkey UInt32) ENGINE = AggregatingMergeTree ORDER BY vkey; +INSERT INTO t0 VALUES (15, 25000, 58); +SELECT ref_5.pkey AS c_2_c2392_6 FROM t0 AS ref_5 WHERE 'J[' < multiIf(ref_5.pkey IN ( SELECT 1 ), bitShiftLeft(multiIf(ref_5.c0 > NULL, '1', ')'), 40), NULL); +DROP TABLE t0; +DROP TABLE t1; + +DROP TABLE IF EXISTS t5; +CREATE TABLE t5 (vkey UInt32, pkey UInt32, c18 Float32, c19 UInt32) ENGINE = Log; +INSERT INTO t5 VALUES (3, 13000, 73.90, 83); +SELECT subq_0.pkey as c_1_c1193_15 FROM t5 AS subq_0 WHERE sipHash128(0, subq_0.c18, bitShiftRight(case when false then (sipHash128(subq_0.pkey, subq_0.c18, 'S')) else '1' end, 0)) is not null; +DROP TABLE t5; + From 391e61549d59d338d475732704023092e34faf4a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 30 May 2023 12:18:56 +0000 Subject: [PATCH 0803/2223] Fixes --- src/Parsers/ExpressionListParsers.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 8903b2b02b0..18a6de5b4f6 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2332,8 +2332,8 @@ const std::vector> ParserExpressionImpl::o {":", Operator("if", 3, 3, OperatorType::FinishIf)}, {"OR", Operator("or", 3, 2, OperatorType::Mergeable)}, {"AND", Operator("and", 4, 2, OperatorType::Mergeable)}, - {"IS NULL", Operator("isNull", 6, 1, OperatorType::IsNull)}, - {"IS NOT NULL", Operator("isNotNull", 6, 1, OperatorType::IsNull)}, + {"IS NULL", Operator("isNull", 6, 1, OperatorType::IsNull)}, + {"IS NOT NULL", Operator("isNotNull", 6, 1, OperatorType::IsNull)}, {"BETWEEN", Operator("", 7, 0, OperatorType::StartBetween)}, {"NOT BETWEEN", Operator("", 7, 0, OperatorType::StartNotBetween)}, {"==", Operator("equals", 9, 2, OperatorType::Comparison)}, @@ -2353,7 +2353,7 @@ const std::vector> ParserExpressionImpl::o {"NOT IN", Operator("notIn", 9, 2)}, {"GLOBAL IN", Operator("globalIn", 9, 2)}, {"GLOBAL NOT IN", Operator("globalNotIn", 9, 2)}, - {"||", Operator("concat", 10, 2, OperatorType::Mergeable)}, + {"||", Operator("concat", 10, 2, OperatorType::Mergeable)}, {"+", Operator("plus", 11, 2)}, {"-", Operator("minus", 11, 2)}, {"*", Operator("multiply", 12, 2)}, @@ -2372,7 +2372,7 @@ const std::vector> ParserExpressionImpl::u {"-", Operator("negate", 13, 1)} }; -const Operator ParserExpressionImpl::finish_between_operator("", 7, 0, OperatorType::FinishBetween); +const Operator ParserExpressionImpl::finish_between_operator("", 8, 0, OperatorType::FinishBetween); const std::array ParserExpressionImpl::overlapping_operators_to_skip { @@ -2392,6 +2392,7 @@ bool ParserExpressionImpl::parse(std::unique_ptr start, IParser::Pos & po { if (!layers.back()->parse(pos, expected, next)) break; + if (layers.back()->isFinished()) { if (layers.size() == 1) From 092b06f32eb7e4e8b0c4f379632d560810393299 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 30 May 2023 14:36:47 +0200 Subject: [PATCH 0804/2223] Fix special build --- src/Common/checkSSLReturnCode.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/checkSSLReturnCode.cpp b/src/Common/checkSSLReturnCode.cpp index bc87af1d37d..353c287813d 100644 --- a/src/Common/checkSSLReturnCode.cpp +++ b/src/Common/checkSSLReturnCode.cpp @@ -8,7 +8,7 @@ namespace DB { -bool checkSSLWantRead([[maybe_unused]] ssize_t res) +bool checkSSLWantRead([[maybe_unused]] ssize_t ret) { #if USE_SSL return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ; @@ -17,7 +17,7 @@ bool checkSSLWantRead([[maybe_unused]] ssize_t res) #endif } -bool checkSSLWantWrite([[maybe_unused]] ssize_t res) +bool checkSSLWantWrite([[maybe_unused]] ssize_t ret) { #if USE_SSL return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE; From a23af1b065e42f54a5c93ee96c3e4517d98611c2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 30 May 2023 13:16:25 +0000 Subject: [PATCH 0805/2223] Integrate test 02763_last_day_of_week into existing test 00941_to_custom_week --- .../0_stateless/00941_to_custom_week.reference | 10 ++++++++++ .../0_stateless/00941_to_custom_week.sql | 18 ++++++++++++++++++ .../02763_last_day_of_week.reference | 10 ---------- .../0_stateless/02763_last_day_of_week.sql | 17 ----------------- 4 files changed, 28 insertions(+), 27 deletions(-) delete mode 100644 tests/queries/0_stateless/02763_last_day_of_week.reference delete mode 100644 tests/queries/0_stateless/02763_last_day_of_week.sql diff --git a/tests/queries/0_stateless/00941_to_custom_week.reference b/tests/queries/0_stateless/00941_to_custom_week.reference index 6171bc3937f..1650f955989 100644 --- a/tests/queries/0_stateless/00941_to_custom_week.reference +++ b/tests/queries/0_stateless/00941_to_custom_week.reference @@ -71,3 +71,13 @@ 2019-01-01 2019-01-01 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2019-01-02 2019-01-02 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2019-01-03 2019-01-03 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 +2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-28 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-29 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-30 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2018-12-30 2018-12-30 2018-12-30 2018-12-30 +2018-12-31 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-01 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-02 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 +2019-01-03 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 diff --git a/tests/queries/0_stateless/00941_to_custom_week.sql b/tests/queries/0_stateless/00941_to_custom_week.sql index 4dd5d209306..58a81c03ef8 100644 --- a/tests/queries/0_stateless/00941_to_custom_week.sql +++ b/tests/queries/0_stateless/00941_to_custom_week.sql @@ -50,3 +50,21 @@ SELECT toStartOfWeek(x_t, 3) AS wt3 FROM numbers(10); +-- toLastDayOfWeek +WITH + toDate('2018-12-25') + number AS d, + toDate32(d) AS d32, + toDateTime(d) AS dt, + toDateTime64(d, 0) AS dt64 +SELECT + dt64, + toLastDayOfWeek(d) AS wd_sun, + toLastDayOfWeek(d32) AS wd32_sun, + toLastDayOfWeek(dt) AS wdt_sun, + toLastDayOfWeek(dt64) AS wdt64_sun, + toLastDayOfWeek(d, 1) AS wd_mon, + toLastDayOfWeek(d32, 1) AS wd32_mon, + toLastDayOfWeek(dt, 1) AS wdt_mon, + toLastDayOfWeek(dt64, 1) AS wdt64_mon +FROM numbers(10); + diff --git a/tests/queries/0_stateless/02763_last_day_of_week.reference b/tests/queries/0_stateless/02763_last_day_of_week.reference deleted file mode 100644 index 140ee9bfe24..00000000000 --- a/tests/queries/0_stateless/02763_last_day_of_week.reference +++ /dev/null @@ -1,10 +0,0 @@ -2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-28 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-29 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-30 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2018-12-30 2018-12-30 2018-12-30 2018-12-30 -2018-12-31 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 -2019-01-01 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 -2019-01-02 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 -2019-01-03 00:00:00 2019-01-05 2019-01-05 2019-01-05 2019-01-05 2019-01-06 2019-01-06 2019-01-06 2019-01-06 diff --git a/tests/queries/0_stateless/02763_last_day_of_week.sql b/tests/queries/0_stateless/02763_last_day_of_week.sql deleted file mode 100644 index 533ecfaa5b2..00000000000 --- a/tests/queries/0_stateless/02763_last_day_of_week.sql +++ /dev/null @@ -1,17 +0,0 @@ -WITH - toDate('2018-12-25') + number AS d, - toDate32(d) AS d32, - toDateTime(d) AS dt, - toDateTime64(d, 0) AS dt64 -SELECT - dt64, - toLastDayOfWeek(d) AS wd_0, - toLastDayOfWeek(d32) AS wd32_0, - toLastDayOfWeek(dt) AS wdt_0, - toLastDayOfWeek(dt64) AS wdt64_0, - toLastDayOfWeek(d, 3) AS wd_3, - toLastDayOfWeek(d32, 3) AS wd32_3, - toLastDayOfWeek(dt, 3) AS wdt_3, - toLastDayOfWeek(dt64, 3) AS wdt64_3 -FROM numbers(10); - From 21aba94909e2ab7fe357c30d694af8674f81dbd0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 May 2023 13:22:40 +0000 Subject: [PATCH 0806/2223] Add support for latest snapshot disk --- src/Coordination/KeeperContext.cpp | 11 ++ src/Coordination/KeeperContext.h | 2 + src/Coordination/KeeperSnapshotManager.cpp | 143 ++++++++++++++------- src/Coordination/KeeperSnapshotManager.h | 4 + src/Coordination/KeeperStateMachine.cpp | 8 +- 5 files changed, 119 insertions(+), 49 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 9d7e62ffae7..5ec86c827b8 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -43,6 +43,11 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) snapshot_storage = getSnapshotsPathFromConfig(config); + if (config.has("keeper_server.latest_snapshot_storage_disk")) + latest_snapshot_storage = config.getString("keeper_server.latest_snapshot_storage_disk"); + else + latest_snapshot_storage = snapshot_storage; + state_file_storage = getStatePathFromConfig(config); } @@ -107,6 +112,11 @@ void KeeperContext::setLogDisk(DiskPtr disk) latest_log_storage = std::move(disk); } +DiskPtr KeeperContext::getLatestSnapshotDisk() const +{ + return getDisk(latest_snapshot_storage); +} + DiskPtr KeeperContext::getSnapshotDisk() const { return getDisk(snapshot_storage); @@ -126,6 +136,7 @@ std::vector KeeperContext::getOldSnapshotDisks() const void KeeperContext::setSnapshotDisk(DiskPtr disk) { snapshot_storage = std::move(disk); + latest_snapshot_storage = snapshot_storage; } DiskPtr KeeperContext::getStateFileDisk() const diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 00607fd904e..b04afd2a57b 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -39,6 +39,7 @@ public: std::vector getOldLogDisks() const; void setLogDisk(DiskPtr disk); + DiskPtr getLatestSnapshotDisk() const; DiskPtr getSnapshotDisk() const; std::vector getOldSnapshotDisks() const; void setSnapshotDisk(DiskPtr disk); @@ -67,6 +68,7 @@ private: Storage log_storage; Storage latest_log_storage; Storage snapshot_storage; + Storage latest_snapshot_storage; Storage state_file_storage; std::vector old_log_disk_names; diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index c3371501976..4b9a34c07e9 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -16,6 +16,7 @@ #include #include #include +#include "Core/Field.h" #include @@ -31,6 +32,24 @@ namespace ErrorCodes namespace { + constexpr std::string_view tmp_prefix = "tmp_"; + + void moveFileBetweenDisks(DiskPtr disk_from, const std::string & path_from, DiskPtr disk_to, const std::string & path_to) + { + /// we use empty file with prefix tmp_ to detect incomplete copies + /// if a copy is complete we don't care from which disk we use the same file + /// so it's okay if a failure happens after removing of tmp file but before we remove + /// the snapshot from the source disk + auto from_path = fs::path(path_from); + auto tmp_snapshot_name = from_path.parent_path() / (std::string{tmp_prefix} + from_path.filename().string()); + { + disk_to->writeFile(tmp_snapshot_name); + } + disk_from->copyFile(from_path, *disk_to, path_to, {}); + disk_to->removeFile(tmp_snapshot_name); + disk_from->removeFile(path_from); + } + uint64_t getSnapshotPathUpToLogIdx(const String & snapshot_path) { std::filesystem::path path(snapshot_path); @@ -519,43 +538,48 @@ KeeperSnapshotManager::KeeperSnapshotManager( , storage_tick_time(storage_tick_time_) , keeper_context(keeper_context_) { - const auto load_snapshot_from_disk = [&](DiskPtr disk) + const auto load_snapshot_from_disk = [&](const auto & disk) { - std::unordered_set invalid_snapshots; - /// collect invalid snapshots + LOG_TRACE(log, "Reading from disk {}", disk->getName()); + std::unordered_map incomplete_files; + + const auto clean_incomplete_file = [&](const auto & file_path) + { + if (auto incomplete_it = incomplete_files.find(fs::path(file_path).filename()); incomplete_it != incomplete_files.end()) + { + LOG_TRACE(log, "Removing {} from {}", file_path, disk->getName()); + disk->removeFile(file_path); + disk->removeFile(incomplete_it->second); + incomplete_files.erase(incomplete_it); + return true; + } + + return false; + }; + + std::vector snapshot_files; for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { - const auto & name = it->name(); - if (name.empty()) - continue; - - if (startsWith(name, "tmp_")) + if (it->name().starts_with(tmp_prefix)) { - disk->removeFile(it->path()); - invalid_snapshots.insert(name.substr(4)); + incomplete_files.emplace(it->name().substr(tmp_prefix.size()), it->path()); continue; } + + if (clean_incomplete_file(it->path())) + continue; + + snapshot_files.push_back(it->path()); } - /// process snapshots - for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) + for (const auto & snapshot_file : snapshot_files) { - const auto & name = it->name(); - if (name.empty()) + if (clean_incomplete_file(fs::path(snapshot_file).filename())) continue; - /// Not snapshot file - if (!startsWith(name, "snapshot_")) - continue; - - if (invalid_snapshots.contains(name)) - { - disk->removeFile(it->path()); - continue; - } - - size_t snapshot_up_to = getSnapshotPathUpToLogIdx(name); - auto [_, inserted] = existing_snapshots.insert_or_assign(snapshot_up_to, SnapshotFileInfo{it->path(), disk}); + LOG_TRACE(log, "Found {} on {}", snapshot_file, disk->getName()); + size_t snapshot_up_to = getSnapshotPathUpToLogIdx(snapshot_file); + auto [_, inserted] = existing_snapshots.insert_or_assign(snapshot_up_to, SnapshotFileInfo{snapshot_file, disk}); if (!inserted) LOG_WARNING( @@ -564,6 +588,9 @@ KeeperSnapshotManager::KeeperSnapshotManager( snapshot_up_to, disk->getName()); } + + for (const auto & [name, path] : incomplete_files) + disk->removeFile(path); }; for (const auto & disk : keeper_context->getOldSnapshotDisks()) @@ -572,25 +599,12 @@ KeeperSnapshotManager::KeeperSnapshotManager( auto disk = getDisk(); load_snapshot_from_disk(disk); + auto latest_snapshot_disk = getLatestSnapshotDisk(); + if (latest_snapshot_disk != disk) + load_snapshot_from_disk(latest_snapshot_disk); + removeOutdatedSnapshotsIfNeeded(); - - /// move snapshots from old disks to new one - for (auto & [_, file_info] : existing_snapshots) - { - if (file_info.disk == disk) - continue; - - auto file_path = fs::path(file_info.path); - auto tmp_snapshot_path = file_path.parent_path() / ("tmp_" + file_path.filename().generic_string()); - - { - disk->writeFile(tmp_snapshot_path); - } - - file_info.disk->copyFile(file_info.path, *disk, file_info.path, {}); - disk->removeFile(tmp_snapshot_path); - file_info.disk = disk; - } + moveSnapshotsIfNeeded(); } SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) @@ -600,7 +614,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - auto disk = getDisk(); + auto disk = getLatestSnapshotDisk(); { disk->writeFile(tmp_snapshot_file_name); @@ -614,6 +628,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu existing_snapshots.emplace(up_to_log_idx, SnapshotFileInfo{snapshot_file_name, disk}); removeOutdatedSnapshotsIfNeeded(); + moveSnapshotsIfNeeded(); return {snapshot_file_name, disk}; } @@ -710,12 +725,47 @@ DiskPtr KeeperSnapshotManager::getDisk() const return keeper_context->getSnapshotDisk(); } +DiskPtr KeeperSnapshotManager::getLatestSnapshotDisk() const +{ + return keeper_context->getLatestSnapshotDisk(); +} + void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() { while (existing_snapshots.size() > snapshots_to_keep) removeSnapshot(existing_snapshots.begin()->first); } +void KeeperSnapshotManager::moveSnapshotsIfNeeded() +{ + /// move snapshots to correct disks + + auto disk = getDisk(); + auto latest_snapshot_disk = getLatestSnapshotDisk(); + auto latest_snapshot_idx = getLatestSnapshotIndex(); + + for (auto & [idx, file_info] : existing_snapshots) + { + if (idx == latest_snapshot_idx) + { + if (file_info.disk != latest_snapshot_disk) + { + moveFileBetweenDisks(file_info.disk, file_info.path, latest_snapshot_disk, file_info.path); + file_info.disk = latest_snapshot_disk; + } + } + else + { + if (file_info.disk != disk) + { + moveFileBetweenDisks(file_info.disk, file_info.path, disk, file_info.path); + file_info.disk = disk; + } + } + } + +} + void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) { auto itr = existing_snapshots.find(log_idx); @@ -732,7 +782,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - auto disk = getDisk(); + auto disk = getLatestSnapshotDisk(); { disk->writeFile(tmp_snapshot_file_name); } @@ -752,6 +802,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor existing_snapshots.emplace(up_to_log_idx, SnapshotFileInfo{snapshot_file_name, disk}); removeOutdatedSnapshotsIfNeeded(); + moveSnapshotsIfNeeded(); return {snapshot_file_name, disk}; } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 0afe582ef59..9bb287b9276 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -165,8 +165,10 @@ public: private: void removeOutdatedSnapshotsIfNeeded(); + void moveSnapshotsIfNeeded(); DiskPtr getDisk() const; + DiskPtr getLatestSnapshotDisk() const; /// Checks first 4 buffer bytes to became sure that snapshot compressed with /// ZSTD codec. @@ -184,6 +186,8 @@ private: size_t storage_tick_time; KeeperContextPtr keeper_context; + + Poco::Logger * log = &Poco::Logger::get("KeeperSnapshotManager"); }; /// Keeper create snapshots in background thread. KeeperStateMachine just create diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e7cd409f569..a8c0d8d1518 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -468,15 +468,17 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res { latest_snapshot_meta = snapshot->snapshot_meta; /// we rely on the fact that the snapshot disk cannot be changed during runtime - if (isLocalDisk(*keeper_context->getSnapshotDisk())) + if (isLocalDisk(*keeper_context->getLatestSnapshotDisk())) { - latest_snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot); + auto snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot); + latest_snapshot_info = std::move(snapshot_info); latest_snapshot_buf = nullptr; } else { auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(*snapshot); - latest_snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx()); + auto snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx()); + latest_snapshot_info = std::move(snapshot_info); latest_snapshot_buf = std::move(snapshot_buf); } From 91a3c881267d1d725db78da4ec8ed5d53ae230fb Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 30 May 2023 15:49:47 +0200 Subject: [PATCH 0807/2223] less logs in WriteBufferFromS3 --- src/IO/WriteBufferFromS3.cpp | 8 -------- src/IO/WriteBufferFromS3TaskTracker.cpp | 11 ----------- 2 files changed, 19 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 954c996d929..ffdc23b274b 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -195,18 +195,14 @@ void WriteBufferFromS3::finalizeImpl() if (request_settings.check_objects_after_upload) { - LOG_TRACE(log, "Checking object {} exists after upload", key); S3::checkObjectExists(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage, "Immediately after upload"); - LOG_TRACE(log, "Checking object {} has size as expected {}", key, total_size); size_t actual_size = S3::getObjectSize(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage); if (actual_size != total_size) throw Exception( ErrorCodes::S3_ERROR, "Object {} from bucket {} has unexpected size {} after upload, expected size {}, it's a bug in S3 or S3 API.", key, bucket, actual_size, total_size); - - LOG_TRACE(log, "Object {} exists after upload", key); } } @@ -286,8 +282,6 @@ void WriteBufferFromS3::reallocateFirstBuffer() WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); chassert(offset() == 0); - - LOG_TRACE(log, "Reallocated first buffer with size {}. {}", memory.size(), getLogDetails()); } void WriteBufferFromS3::detachBuffer() @@ -310,8 +304,6 @@ void WriteBufferFromS3::allocateFirstBuffer() const auto size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), max_first_buffer); memory = Memory(size); WriteBuffer::set(memory.data(), memory.size()); - - LOG_TRACE(log, "Allocated first buffer with size {}. {}", memory.size(), getLogDetails()); } void WriteBufferFromS3::allocateBuffer() diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 4abae90eeac..b023de16c98 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -36,8 +36,6 @@ ThreadPoolCallbackRunner WriteBufferFromS3::TaskTracker::syncRunner() void WriteBufferFromS3::TaskTracker::waitAll() { - LOG_TEST(log, "waitAll, in queue {}", futures.size()); - /// Exceptions are propagated for (auto & future : futures) { @@ -51,8 +49,6 @@ void WriteBufferFromS3::TaskTracker::waitAll() void WriteBufferFromS3::TaskTracker::safeWaitAll() { - LOG_TEST(log, "safeWaitAll, wait in queue {}", futures.size()); - for (auto & future : futures) { if (future.valid()) @@ -76,7 +72,6 @@ void WriteBufferFromS3::TaskTracker::safeWaitAll() void WriteBufferFromS3::TaskTracker::waitIfAny() { - LOG_TEST(log, "waitIfAny, in queue {}", futures.size()); if (futures.empty()) return; @@ -101,8 +96,6 @@ void WriteBufferFromS3::TaskTracker::waitIfAny() watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); - - LOG_TEST(log, "waitIfAny ended, in queue {}", futures.size()); } void WriteBufferFromS3::TaskTracker::add(Callback && func) @@ -147,8 +140,6 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() if (!max_tasks_inflight) return; - LOG_TEST(log, "waitTilInflightShrink, in queue {}", futures.size()); - Stopwatch watch; /// Alternative approach is to wait until at least futures.size() - max_tasks_inflight element are finished @@ -178,8 +169,6 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); - - LOG_TEST(log, "waitTilInflightShrink ended, in queue {}", futures.size()); } bool WriteBufferFromS3::TaskTracker::isAsync() const From 308db6784c21b034925981d812112a6eeba3ce46 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Tue, 30 May 2023 08:50:58 -0500 Subject: [PATCH 0808/2223] Update environmental-sensors.md --- .../getting-started/example-datasets/environmental-sensors.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/environmental-sensors.md b/docs/en/getting-started/example-datasets/environmental-sensors.md index 309a6dc6c0f..00e8dfa21c2 100644 --- a/docs/en/getting-started/example-datasets/environmental-sensors.md +++ b/docs/en/getting-started/example-datasets/environmental-sensors.md @@ -36,8 +36,8 @@ The data is in CSV files but uses a semi-colon for the delimiter. The rows look │ 7389 │ BMP180 │ 3735 │ 50.136 │ 11.062 │ 2019-06-01T00:00:06 │ 98905 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 12.1 │ │ 13199 │ BMP180 │ 6664 │ 52.514 │ 13.44 │ 2019-06-01T00:00:07 │ 101855.54 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 19.74 │ │ 12753 │ BMP180 │ 6440 │ 44.616 │ 2.032 │ 2019-06-01T00:00:07 │ 99475 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 17 │ -│ 16956 │ BMP180 │ 8594 │ 52.052 │ 8.354 │ 2019-06-01T00:00:08 │ 101322 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 17.2 │ -└───────────┴─────────────┴──────────┴────────┴───────┴─────────────────────┴──────────┴──────────┴───────────────────┴─────────────┘ +│ 16956 │ BMP180 │ 8594 │ 52.052 │ 8.354 │ 2019-06-01T00:00:08 │ 101322 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 17.2 │ +└───────────┴─────────────┴──────────┴────────┴────────┴─────────────────────┴───────────┴──────────┴───────────────────┴─────────────┘ ``` 2. We will use the following `MergeTree` table to store the data in ClickHouse: From 93cf34320c12d17554d047497928cac1c177937b Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 30 May 2023 16:10:52 +0200 Subject: [PATCH 0809/2223] Fixed convertFieldToType case of converting Date and Date32 to DateTime64 Field Also added a unit-test that clarifies implementation --- src/Interpreters/convertFieldToType.cpp | 16 +- .../tests/gtest_convertFieldToType.cpp | 185 ++++++++++++++++++ 2 files changed, 195 insertions(+), 6 deletions(-) create mode 100644 src/Interpreters/tests/gtest_convertFieldToType.cpp diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 1ea9ecd7e00..ff09175f898 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -194,15 +194,19 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } else if (which_type.isDateTime64() && which_from_type.isDate()) { - const DataTypeDateTime64 & data_type_date_time64 = static_cast(type); - const Int64 value = data_type_date_time64.getTimeZone().fromDayNum(DayNum(src.get())); - return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, data_type_date_time64.getScaleMultiplier()); + const auto & date_time64_type = static_cast(type); + const auto value = date_time64_type.getTimeZone().fromDayNum(DayNum(src.get())); + return DecimalField( + DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, date_time64_type.getScaleMultiplier()), + date_time64_type.getScale()); } else if (which_type.isDateTime64() && which_from_type.isDate32()) { - const DataTypeDateTime64 & data_type_date_time64 = static_cast(type); - const Int64 value = data_type_date_time64.getTimeZone().fromDayNum(DayNum(src.get())); - return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, data_type_date_time64.getScaleMultiplier()); + const auto & date_time64_type = static_cast(type); + const auto value = date_time64_type.getTimeZone().fromDayNum(ExtendedDayNum(static_cast(src.get()))); + return DecimalField( + DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, date_time64_type.getScaleMultiplier()), + date_time64_type.getScale()); } else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { diff --git a/src/Interpreters/tests/gtest_convertFieldToType.cpp b/src/Interpreters/tests/gtest_convertFieldToType.cpp new file mode 100644 index 00000000000..5421c192ac7 --- /dev/null +++ b/src/Interpreters/tests/gtest_convertFieldToType.cpp @@ -0,0 +1,185 @@ +#include +#include +#include +#include +#include +#include +#include + +#include +#include "base/Decimal.h" +#include "base/types.h" +#include "gtest/gtest.h" + +using namespace DB; + +struct ConvertFieldToTypeTestParams +{ + const char * from_type; // MUST NOT BE NULL + const Field from_value; + const char * to_type; // MUST NOT BE NULL + const std::optional expected_value; +}; + +std::ostream & operator << (std::ostream & ostr, const ConvertFieldToTypeTestParams & params) +{ + return ostr << "{" + << "\n\tfrom_type : " << params.from_type + << "\n\tfrom_value : " << params.from_value + << "\n\tto_type : " << params.to_type + << "\n\texpected : " << (params.expected_value ? *params.expected_value : Field()) + << "\n}"; +} + +class ConvertFieldToTypeTest : public ::testing::TestWithParam +{}; + +TEST_P(ConvertFieldToTypeTest, convert) +{ + const auto & params = GetParam(); + + ASSERT_NE(nullptr, params.from_type); + ASSERT_NE(nullptr, params.to_type); + + const auto & type_factory = DataTypeFactory::instance(); + const auto from_type = type_factory.get(params.from_type); + const auto to_type = type_factory.get(params.to_type); + + if (params.expected_value) + { + const auto result = convertFieldToType(params.from_value, *to_type, from_type.get()); + EXPECT_EQ(*params.expected_value, result); + } + else + { + EXPECT_ANY_THROW(convertFieldToType(params.from_value, *to_type, from_type.get())); + } +} + +// Basically nuber of seconds in a day, works for UTC here +const long long int Day = 24 * 60 * 60; + +// 123 is arbitrary value here + +INSTANTIATE_TEST_SUITE_P( + DateToDateTime64, + ConvertFieldToTypeTest, + ::testing::ValuesIn(std::initializer_list{ + // min value of Date + { + "Date", + Field(0), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(0), 0) + }, + // Max value of Date + { + "Date", + Field(std::numeric_limits::max()), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(std::numeric_limits::max() * Day), 0) + }, + // check that scale is respected + { + "Date", + Field(123), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(123 * Day), 0) + }, + { + "Date", + Field(1), + "DateTime64(1, 'UTC')", + DecimalField(DateTime64(Day * 10), 1) + }, + { + "Date", + Field(123), + "DateTime64(3, 'UTC')", + DecimalField(DateTime64(123 * Day * 1000), 3) + }, + { + "Date", + Field(123), + "DateTime64(6, 'UTC')", + DecimalField(DateTime64(123 * Day * 1'000'000), 6) + }, + }) +); + +INSTANTIATE_TEST_SUITE_P( + Date32ToDateTime64, + ConvertFieldToTypeTest, + ::testing::ValuesIn(std::initializer_list{ + // min value of Date32: 1st Jan 1900 (see DATE_LUT_MIN_YEAR) + { + "Date32", + Field(-25'567), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(-25'567 * Day), 0) + }, + // max value of Date32: 31 Dec 2299 (see DATE_LUT_MAX_YEAR) + { + "Date32", + Field(120'529), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(120'529 * Day), 0) + }, + // check that scale is respected + { + "Date32", + Field(123), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(123 * Day), 0) + }, + { + "Date32", + Field(123), + "DateTime64(1, 'UTC')", + DecimalField(DateTime64(123 * Day * 10), 1) + }, + { + "Date32", + Field(123), + "DateTime64(3, 'UTC')", + DecimalField(DateTime64(123 * Day * 1000), 3) + }, + { + "Date32", + Field(123), + "DateTime64(6, 'UTC')", + DecimalField(DateTime64(123 * Day * 1'000'000), 6) + } + }) + ); + +INSTANTIATE_TEST_SUITE_P( + DateTimeToDateTime64, + ConvertFieldToTypeTest, + ::testing::ValuesIn(std::initializer_list{ + { + "DateTime", + Field(1), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(1), 0) + }, + { + "DateTime", + Field(1), + "DateTime64(1, 'UTC')", + DecimalField(DateTime64(1'0), 1) + }, + { + "DateTime", + Field(123), + "DateTime64(3, 'UTC')", + DecimalField(DateTime64(123'000), 3) + }, + { + "DateTime", + Field(123), + "DateTime64(6, 'UTC')", + DecimalField(DateTime64(123'000'000), 6) + }, + }) +); From 1c904ecc8ef5886ec90221f52c02758a26a5b81e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 30 May 2023 16:25:17 +0200 Subject: [PATCH 0810/2223] Fix typo --- src/Common/checkSSLReturnCode.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/checkSSLReturnCode.cpp b/src/Common/checkSSLReturnCode.cpp index 353c287813d..ffb912da18a 100644 --- a/src/Common/checkSSLReturnCode.cpp +++ b/src/Common/checkSSLReturnCode.cpp @@ -11,7 +11,7 @@ namespace DB bool checkSSLWantRead([[maybe_unused]] ssize_t ret) { #if USE_SSL - return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ; + return ret == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_READ; #else return false; #endif @@ -20,7 +20,7 @@ bool checkSSLWantRead([[maybe_unused]] ssize_t ret) bool checkSSLWantWrite([[maybe_unused]] ssize_t ret) { #if USE_SSL - return res == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE; + return ret == Poco::Net::SecureStreamSocket::ERR_SSL_WANT_WRITE; #else return false; #endif From bcd57078cebbadd00b3f51fd17aba2f67a571996 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 30 May 2023 16:36:54 +0200 Subject: [PATCH 0811/2223] Update the runner version --- tests/ci/worker/ubuntu_ami_for_ci.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/ubuntu_ami_for_ci.sh b/tests/ci/worker/ubuntu_ami_for_ci.sh index f8d1645a54d..22e5ada2932 100644 --- a/tests/ci/worker/ubuntu_ami_for_ci.sh +++ b/tests/ci/worker/ubuntu_ami_for_ci.sh @@ -3,7 +3,7 @@ set -xeuo pipefail echo "Running prepare script" export DEBIAN_FRONTEND=noninteractive -export RUNNER_VERSION=2.298.2 +export RUNNER_VERSION=2.304.0 export RUNNER_HOME=/home/ubuntu/actions-runner deb_arch() { From 6c9b7a710c083041b25159109e8cf786c5875ba0 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 30 May 2023 14:46:44 +0000 Subject: [PATCH 0812/2223] Added more tests for toDateOrDefault/toDateTimeOrDefault --- .../01746_convert_type_with_default.reference | 8 ++++++++ .../01746_convert_type_with_default.sql | 14 +++++++++----- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.reference b/tests/queries/0_stateless/01746_convert_type_with_default.reference index ec2a826982f..235a88157c8 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.reference +++ b/tests/queries/0_stateless/01746_convert_type_with_default.reference @@ -20,8 +20,16 @@ 2 -1 -2 +1970-01-01 2023-05-30 2023-05-30 +2023-05-30 +1970-01-01 +2023-05-30 +2023-05-30 +1970-01-01 +2023-05-30 14:38:20 +2023-05-30 14:38:20 2023-05-30 14:38:20 2023-05-30 14:38:20 61f0c404-5cb3-11e7-907b-a6006ad3dba0 diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index 9d7873081e5..18b5ae60920 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -26,15 +26,19 @@ select toUInt256OrDefault('1xx', cast(2 as UInt256)); select toInt256OrDefault('-1', cast(-2 as Int256)); select toInt256OrDefault('-1xx', cast(-2 as Int256)); +select toDateOrDefault('2020-0x-02'); select toDateOrDefault('2023-05-30'); +select toDateOrDefault('2023-05-30', '2000-01-01'::Date); +select toDateOrDefault('2020-0x-02', '2023-05-30'::Date); +select toDateOrDefault(-1); select toDateOrDefault(19507); +select toDateOrDefault(19507, '2000-01-01'::Date); +select toDateOrDefault(-1, '2000-01-01'::Date); select toDateTimeOrDefault('2023-05-30 14:38:20'); +select toDateTimeOrDefault('2023-05-30 14:38:20', 'UTC'); +select toDateTimeOrDefault('s2023', 'UTC', '2023-05-30 14:38:20'::DateTime('UTC')); select toDateTimeOrDefault(1685457500); SELECT toUUIDOrDefault('61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); -SELECT toUUIDOrDefault('-----61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); - - - - +SELECT toUUIDOrDefault('-----61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); \ No newline at end of file From 060d0118d743a1c77e3c2ac692d6fc8c81c61900 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 30 May 2023 12:08:24 -0300 Subject: [PATCH 0813/2223] Update merge-tree-settings.md --- docs/en/operations/settings/merge-tree-settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 0b1207ee7b6..35f9e4b29a7 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -577,7 +577,7 @@ Default value: 20 **Usage** -The value of the `number_of_free_entries_in_pool_to_execute_mutation` setting should be less than the value of the [background_pool_size](/docs/en/operations/server-configuration-parameters/settings#background_pool_size) * [background_pool_size](/docs/en/operations/server-configuration-parameters/settings#background_merges_mutations_concurrency_ratio). Otherwise, ClickHouse throws an exception. +The value of the `number_of_free_entries_in_pool_to_execute_mutation` setting should be less than the value of the [background_pool_size](/docs/en/operations/server-configuration-parameters/settings#background_pool_size) * [background_merges_mutations_concurrency_ratio](/docs/en/operations/server-configuration-parameters/settings#background_merges_mutations_concurrency_ratio). Otherwise, ClickHouse throws an exception. ## max_part_loading_threads {#max-part-loading-threads} @@ -840,4 +840,4 @@ Possible values: - `Always` or `Never`. -Default value: `Never` \ No newline at end of file +Default value: `Never` From e91934bceaea3809feb0e2e52532b9eeb35ad7f2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 May 2023 16:02:14 +0000 Subject: [PATCH 0814/2223] Fixing style. --- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 8d2f2204e84..fa4fc2c82df 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -202,7 +202,7 @@ public: external_storage_holder->future_set = std::move(future_set); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "!!!!!!!!"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Set is already created for GLOBAL IN"); //prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } From cfa3fbf5b8718692f06aaacce6a5dab3ee38da87 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Tue, 30 May 2023 12:06:27 -0400 Subject: [PATCH 0815/2223] add .md extensions --- docs/en/operations/settings/merge-tree-settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 35f9e4b29a7..4122b4af40f 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -577,7 +577,7 @@ Default value: 20 **Usage** -The value of the `number_of_free_entries_in_pool_to_execute_mutation` setting should be less than the value of the [background_pool_size](/docs/en/operations/server-configuration-parameters/settings#background_pool_size) * [background_merges_mutations_concurrency_ratio](/docs/en/operations/server-configuration-parameters/settings#background_merges_mutations_concurrency_ratio). Otherwise, ClickHouse throws an exception. +The value of the `number_of_free_entries_in_pool_to_execute_mutation` setting should be less than the value of the [background_pool_size](/docs/en/operations/server-configuration-parameters/settings.md/#background_pool_size) * [background_merges_mutations_concurrency_ratio](/docs/en/operations/server-configuration-parameters/settings.md/#background_merges_mutations_concurrency_ratio). Otherwise, ClickHouse throws an exception. ## max_part_loading_threads {#max-part-loading-threads} From 40d658e467549d589116da449f2ac8edbfe54368 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 1 May 2023 15:09:26 +0000 Subject: [PATCH 0816/2223] Fix join_use_nulls in analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 81 +++- .../02722_matcher_join_use_nulls.reference | 430 ++++++++++++++++++ .../02722_matcher_join_use_nulls.sql.j2 | 119 +++++ 3 files changed, 612 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/02722_matcher_join_use_nulls.reference create mode 100644 tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index aa915e48d35..36a85739339 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1205,6 +1205,29 @@ private: static std::string rewriteAggregateFunctionNameIfNeeded(const std::string & aggregate_function_name, const ContextPtr & context); + static std::optional getColumnSideFromJoinTree(QueryTreeNodePtr & resolved_identifier, const JoinNode & join_node) + { + const auto * column_src = resolved_identifier->as().getColumnSource().get(); + + if (join_node.getLeftTableExpression().get() == column_src) + return JoinTableSide::Left; + if (join_node.getRightTableExpression().get() == column_src) + return JoinTableSide::Right; + return {}; + } + + static void convertJoinedColumnTypeToNullIfNeeded(QueryTreeNodePtr & resolved_identifier, const JoinKind & join_kind, std::optional resolved_side) + { + if (resolved_identifier->getNodeType() == QueryTreeNodeType::COLUMN && + (isFull(join_kind) || + (isLeft(join_kind) && resolved_side && *resolved_side == JoinTableSide::Right) || + (isRight(join_kind) && resolved_side && *resolved_side == JoinTableSide::Left))) + { + auto & resolved_column = resolved_identifier->as(); + resolved_column.setColumnType(makeNullableOrLowCardinalityNullable(resolved_column.getColumnType())); + } + } + /// Resolve identifier functions static QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier, ContextPtr context); @@ -2982,6 +3005,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo QueryTreeNodePtr resolved_identifier; JoinKind join_kind = from_join_node.getKind(); + bool join_use_nulls = scope.context->getSettingsRef().join_use_nulls; if (left_resolved_identifier && right_resolved_identifier) { @@ -3027,19 +3051,31 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo * * Otherwise we prefer column from left table. */ - if (identifier_path_part == right_column_source_alias) - return right_resolved_identifier; - else if (!left_column_source_alias.empty() && - right_column_source_alias.empty() && - identifier_path_part != left_column_source_alias) - return right_resolved_identifier; + bool column_resolved_using_right_alias = identifier_path_part == right_column_source_alias; + bool column_resolved_without_using_left_alias = !left_column_source_alias.empty() + && right_column_source_alias.empty() + && identifier_path_part != left_column_source_alias; + if (column_resolved_using_right_alias || column_resolved_without_using_left_alias) + { + resolved_side = JoinTableSide::Right; + resolved_identifier = right_resolved_identifier; + } + else + { + resolved_side = JoinTableSide::Left; + resolved_identifier = left_resolved_identifier; + } + } + else + { + resolved_side = JoinTableSide::Left; + resolved_identifier = left_resolved_identifier; } - - return left_resolved_identifier; } else if (scope.joins_count == 1 && scope.context->getSettingsRef().single_join_prefer_left_table) { - return left_resolved_identifier; + resolved_side = JoinTableSide::Left; + resolved_identifier = left_resolved_identifier; } else { @@ -3092,17 +3128,10 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo if (join_node_in_resolve_process || !resolved_identifier) return resolved_identifier; - bool join_use_nulls = scope.context->getSettingsRef().join_use_nulls; - - if (join_use_nulls && - resolved_identifier->getNodeType() == QueryTreeNodeType::COLUMN && - (isFull(join_kind) || - (isLeft(join_kind) && resolved_side && *resolved_side == JoinTableSide::Right) || - (isRight(join_kind) && resolved_side && *resolved_side == JoinTableSide::Left))) + if (join_use_nulls) { resolved_identifier = resolved_identifier->clone(); - auto & resolved_column = resolved_identifier->as(); - resolved_column.setColumnType(makeNullableOrLowCardinalityNullable(resolved_column.getColumnType())); + convertJoinedColumnTypeToNullIfNeeded(resolved_identifier, join_kind, resolved_side); } return resolved_identifier; @@ -4001,6 +4030,22 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I else matched_expression_nodes_with_names = resolveUnqualifiedMatcher(matcher_node, scope); + if (scope.context->getSettingsRef().join_use_nulls) + { + const auto * nearest_query_scope = scope.getNearestQueryScope(); + const QueryNode * nearest_scope_query_node = nearest_query_scope ? nearest_query_scope->scope_node->as() : nullptr; + const QueryTreeNodePtr & nearest_scope_join_tree = nearest_scope_query_node ? nearest_scope_query_node->getJoinTree() : nullptr; + const JoinNode * nearest_scope_join_node = nearest_scope_join_tree ? nearest_scope_join_tree->as() : nullptr; + if (nearest_scope_join_node) + { + for (auto & [node, node_name] : matched_expression_nodes_with_names) + { + auto join_identifier_side = getColumnSideFromJoinTree(node, *nearest_scope_join_node); + convertJoinedColumnTypeToNullIfNeeded(node, nearest_scope_join_node->getKind(), join_identifier_side); + } + } + } + std::unordered_map> strict_transformer_to_used_column_names; for (const auto & transformer : matcher_node_typed.getColumnTransformers().getNodes()) { diff --git a/tests/queries/0_stateless/02722_matcher_join_use_nulls.reference b/tests/queries/0_stateless/02722_matcher_join_use_nulls.reference new file mode 100644 index 00000000000..746d02dc381 --- /dev/null +++ b/tests/queries/0_stateless/02722_matcher_join_use_nulls.reference @@ -0,0 +1,430 @@ +-- { echoOn } + +SELECT '============ LEFT JOIN ============' FORMAT Null; +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 \N Int32 Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + LEFT JOIN t2 + ON t1.a = t2.a +) ORDER BY 1; +\N \N Nullable(UInt32) Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Int64 +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Int64 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM t1 + LEFT JOIN t2 + USING (a) +) ORDER BY 1; +1 \N Int64 Nullable(UInt32) +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 \N Int32 Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Int32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + LEFT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + LEFT JOIN t2 + ON t1.a = t2.key +) ORDER BY 1; +\N \N Nullable(UInt32) Nullable(UInt32) +SELECT '============ RIGHT JOIN ============' FORMAT Null; +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +\N 2 Nullable(Int32) UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +2 UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + RIGHT JOIN t2 + ON t1.a = t2.a +) ORDER BY 1; +2 2 UInt32 UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +2 Int64 +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +2 Int64 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +2 Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +2 UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM t1 + RIGHT JOIN t2 + USING (a) +) ORDER BY 1; +2 2 Int64 UInt32 +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +\N 2 Nullable(Int32) UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + RIGHT JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +2 UInt32 +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + RIGHT JOIN t2 + ON t1.a = t2.key +) ORDER BY 1; +2 2 UInt32 UInt32 +SELECT '============ FULL JOIN ============' FORMAT Null; +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 \N Nullable(Int32) Nullable(UInt32) +\N 2 Nullable(Int32) Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + ON t1.a = t2.a +) ORDER BY 1; +2 Nullable(UInt32) +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + FULL JOIN t2 + ON t1.a = t2.a +) ORDER BY 1; +2 2 Nullable(UInt32) Nullable(UInt32) +\N \N Nullable(UInt32) Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Nullable(Int64) +2 Nullable(Int64) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Nullable(Int64) +2 Nullable(Int64) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +1 Nullable(Int32) +2 Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as a) t2 + USING (a) +) ORDER BY 1; +2 Nullable(UInt32) +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM t1 + FULL JOIN t2 + USING (a) +) ORDER BY 1; +1 \N Nullable(Int64) Nullable(UInt32) +2 2 Nullable(Int64) Nullable(UInt32) +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 \N Nullable(Int32) Nullable(UInt32) +\N 2 Nullable(Int32) Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +1 Nullable(Int32) +\N Nullable(Int32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + FULL JOIN (SELECT 2 :: UInt32 as key) t2 + ON t1.a = t2.key +) ORDER BY 1; +2 Nullable(UInt32) +\N Nullable(UInt32) +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + FULL JOIN t2 + ON t1.a = t2.key +) ORDER BY 1; +2 2 Nullable(UInt32) Nullable(UInt32) +\N \N Nullable(UInt32) Nullable(UInt32) diff --git a/tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 b/tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 new file mode 100644 index 00000000000..25451a34867 --- /dev/null +++ b/tests/queries/0_stateless/02722_matcher_join_use_nulls.sql.j2 @@ -0,0 +1,119 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (a Int32) ENGINE = TinyLog; +CREATE TABLE t2 (a UInt32, key UInt32) ENGINE = TinyLog; + +INSERT INTO t1 VALUES (1); +INSERT INTO t2 VALUES (2, 2); + +SET join_use_nulls = 1; +SET allow_experimental_analyzer = 1; + +-- { echoOn } + +{% for KIND in ('LEFT', 'RIGHT', 'FULL') -%} + +SELECT '============ {{ KIND }} JOIN ============' FORMAT Null; + +{% for right_column_name in ['a', 'key'] -%} + +SELECT a, toTypeName(a) +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM t1 + {{ KIND }} JOIN t2 + ON t1.a = t2.{{ right_column_name }} +) ORDER BY 1; + +{% if right_column_name == 'a' -%} + +SELECT *, * APPLY toTypeName +FROM ( + SELECT a + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + USING (a) +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + USING (a) +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT t1.* + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + USING (a) +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT t2.* + FROM (SELECT 1 :: Int32 as a) t1 + {{ KIND }} JOIN (SELECT 2 :: UInt32 as {{ right_column_name }}) t2 + USING (a) +) ORDER BY 1; + +SELECT *, * APPLY toTypeName +FROM ( + SELECT * + FROM t1 + {{ KIND }} JOIN t2 + USING (a) +) ORDER BY 1; + +{% endif -%} + +{% endfor -%} +{% endfor -%} + +-- { echoOff } + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; From 12141bb6cdd42e8a2f9f985549d1c970dbb0295f Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 May 2023 16:43:32 +0000 Subject: [PATCH 0817/2223] Check can become nullable before applying join_use_nulls --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 36a85739339..8bacdfd7bd2 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1219,6 +1219,7 @@ private: static void convertJoinedColumnTypeToNullIfNeeded(QueryTreeNodePtr & resolved_identifier, const JoinKind & join_kind, std::optional resolved_side) { if (resolved_identifier->getNodeType() == QueryTreeNodeType::COLUMN && + JoinCommon::canBecomeNullable(resolved_identifier->getResultType()) && (isFull(join_kind) || (isLeft(join_kind) && resolved_side && *resolved_side == JoinTableSide::Right) || (isRight(join_kind) && resolved_side && *resolved_side == JoinTableSide::Left))) From 6fb836a4b207aaa4cf174e03b33e8dd41b671952 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 10 May 2023 13:22:57 +0200 Subject: [PATCH 0818/2223] Update broken_tests.txt --- tests/broken_tests.txt | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index faee1c5b295..02935712325 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -21,8 +21,6 @@ 01072_optimize_skip_unused_shards_const_expr_eval 01083_expressions_in_engine_arguments 01086_odbc_roundtrip -01142_join_lc_and_nullable_in_key -01142_merge_join_lc_and_nullable_in_key 01152_cross_replication 01155_rename_move_materialized_view 01173_transaction_control_queries @@ -39,8 +37,6 @@ 01319_optimize_skip_unused_shards_nesting 01353_low_cardinality_join_types 01455_shard_leaf_max_rows_bytes_to_read -01476_right_full_join_switch -01477_lc_in_merge_join_left_key 01487_distributed_in_not_default_db 01495_subqueries_in_with_statement 01504_rocksdb From d72789369329661b15fc0b1f1ae1d24b8b2b545f Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 15 May 2023 16:40:46 +0000 Subject: [PATCH 0819/2223] Handle function nodes in getColumnSideFromJoinTree --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 28 ++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 8bacdfd7bd2..637f8a03a76 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1205,8 +1205,34 @@ private: static std::string rewriteAggregateFunctionNameIfNeeded(const std::string & aggregate_function_name, const ContextPtr & context); - static std::optional getColumnSideFromJoinTree(QueryTreeNodePtr & resolved_identifier, const JoinNode & join_node) + static std::optional getColumnSideFromJoinTree(const QueryTreeNodePtr & resolved_identifier, const JoinNode & join_node) { + if (resolved_identifier->getNodeType() == QueryTreeNodeType::CONSTANT) + return {}; + + if (resolved_identifier->getNodeType() == QueryTreeNodeType::FUNCTION) + { + const auto & resolved_function = resolved_identifier->as(); + + const auto & argument_nodes = resolved_function.getArguments().getNodes(); + + std::optional result; + for (const auto & argument_node : argument_nodes) + { + auto table_side = getColumnSideFromJoinTree(argument_node, join_node); + if (table_side && result && *table_side != *result) + { + throw Exception(ErrorCodes::AMBIGUOUS_IDENTIFIER, + "Ambiguous identifier {}. In scope {}", + resolved_identifier->formatASTForErrorMessage(), + join_node.formatASTForErrorMessage()); + } + if (table_side) + result = *table_side; + } + return result; + } + const auto * column_src = resolved_identifier->as().getColumnSource().get(); if (join_node.getLeftTableExpression().get() == column_src) From c8bb1f64ad21dea5ba63fa8f2ea8434d90f9e823 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 30 May 2023 18:46:49 +0200 Subject: [PATCH 0820/2223] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 3 +++ tests/integration/test_lost_part/test.py | 12 ++++++++---- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e71f5217c2b..35f75880ced 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5469,6 +5469,7 @@ void StorageReplicatedMergeTree::alter( if (mutation_znode) { LOG_DEBUG(log, "Metadata changes applied. Will wait for data changes."); + merge_selecting_task->schedule(); waitMutation(*mutation_znode, query_context->getSettingsRef().alter_sync); LOG_DEBUG(log, "Data changes applied."); } @@ -6620,6 +6621,8 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte throw Coordination::Exception("Unable to create a mutation znode", rc); } + merge_selecting_task->schedule(); + waitMutation(mutation_entry.znode_name, query_context->getSettingsRef().mutations_sync); } diff --git a/tests/integration/test_lost_part/test.py b/tests/integration/test_lost_part/test.py index 44cd19fd1fb..0bc24268040 100644 --- a/tests/integration/test_lost_part/test.py +++ b/tests/integration/test_lost_part/test.py @@ -42,7 +42,8 @@ def test_lost_part_same_replica(start_cluster): for node in [node1, node2]: node.query( f"CREATE TABLE mt0 (id UInt64, date Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/t', '{node.name}') ORDER BY tuple() PARTITION BY date " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0" + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" ) node1.query("SYSTEM STOP MERGES mt0") @@ -109,7 +110,8 @@ def test_lost_part_other_replica(start_cluster): for node in [node1, node2]: node.query( f"CREATE TABLE mt1 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t1', '{node.name}') ORDER BY tuple() " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0" + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" ) node1.query("SYSTEM STOP MERGES mt1") @@ -178,7 +180,8 @@ def test_lost_part_mutation(start_cluster): for node in [node1, node2]: node.query( f"CREATE TABLE mt2 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t2', '{node.name}') ORDER BY tuple() " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0" + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" ) node1.query("SYSTEM STOP MERGES mt2") @@ -241,7 +244,8 @@ def test_lost_last_part(start_cluster): for node in [node1, node2]: node.query( f"CREATE TABLE mt3 (id UInt64, p String) ENGINE ReplicatedMergeTree('/clickhouse/tables/t3', '{node.name}') " - "ORDER BY tuple() PARTITION BY p SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0" + "ORDER BY tuple() PARTITION BY p SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" ) node1.query("SYSTEM STOP MERGES mt3") From 783b54624b7122a591e48f72d69a03f2c6e87978 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 30 May 2023 16:48:56 +0000 Subject: [PATCH 0821/2223] add comment to join_use_nulls in QueryAnalyzer::resolveMatcher --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 637f8a03a76..b2bfa648435 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4059,6 +4059,11 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I if (scope.context->getSettingsRef().join_use_nulls) { + /** If we are resolving matcher came from the result of JOIN and `join_use_nulls` is set, + * we need to convert joined column type to Nullable. + * We are taking the nearest JoinNode to check to which table column belongs, + * because for LEFT/RIGHT join, we convert only the corresponding side. + */ const auto * nearest_query_scope = scope.getNearestQueryScope(); const QueryNode * nearest_scope_query_node = nearest_query_scope ? nearest_query_scope->scope_node->as() : nullptr; const QueryTreeNodePtr & nearest_scope_join_tree = nearest_scope_query_node ? nearest_scope_query_node->getJoinTree() : nullptr; From 1a6517d4a0904908dbc09fc5537c1bf08a49a5c9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 30 May 2023 17:13:28 +0000 Subject: [PATCH 0822/2223] Fix priority of the operators after IS NULL --- src/Parsers/ExpressionListParsers.cpp | 12 ++++++++++-- .../0_stateless/02477_is_null_parser.reference | 4 ++-- tests/queries/0_stateless/02477_is_null_parser.sql | 4 ++-- 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 18a6de5b4f6..cd399531064 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2736,11 +2736,19 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po } } - layers.back()->pushOperator(op); - /// isNull & isNotNull are postfix unary operators if (op.type == OperatorType::IsNull) + { + ASTPtr function = makeASTFunction(op); + + if (!layers.back()->popLastNOperands(function->children[0]->children, 1)) + return Action::NONE; + + layers.back()->pushOperand(std::move(function)); return Action::OPERATOR; + } + + layers.back()->pushOperator(op); if (op.type == OperatorType::Cast) { diff --git a/tests/queries/0_stateless/02477_is_null_parser.reference b/tests/queries/0_stateless/02477_is_null_parser.reference index 57d96862011..2820f5ec2db 100644 --- a/tests/queries/0_stateless/02477_is_null_parser.reference +++ b/tests/queries/0_stateless/02477_is_null_parser.reference @@ -1,3 +1,3 @@ -SELECT (\'a\' IS NULL) + (\'b\' IS NOT NULL) -SELECT (\'a\' IS NULL) = 0 +SELECT ((1 IS NULL) + 1) IS NOT NULL +SELECT (1 IS NULL) = 0 SELECT CAST(1 IS NULL, \'Int32\') diff --git a/tests/queries/0_stateless/02477_is_null_parser.sql b/tests/queries/0_stateless/02477_is_null_parser.sql index b95a35fde21..f3ec0affd85 100644 --- a/tests/queries/0_stateless/02477_is_null_parser.sql +++ b/tests/queries/0_stateless/02477_is_null_parser.sql @@ -1,3 +1,3 @@ -EXPLAIN SYNTAX SELECT 'a' IS NULL + 'b' IS NOT NULL; -EXPLAIN SYNTAX SELECT 'a' IS NULL = 0; +EXPLAIN SYNTAX SELECT 1 IS NULL + 1 IS NOT NULL; +EXPLAIN SYNTAX SELECT 1 IS NULL = 0; EXPLAIN SYNTAX SELECT 1 IS NULL :: Int32; From 8d6244ed991e30129a5f66aea85842a2c45d8520 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 May 2023 17:10:10 +0000 Subject: [PATCH 0823/2223] Docs: Explain non-production / advanced install options in more detail Fixes: #49878 --- docs/en/getting-started/install.md | 41 +++++++++++++++++++++++++----- 1 file changed, 34 insertions(+), 7 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 3f6c2577c94..74d712d9f5e 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -28,21 +28,23 @@ The quickest and easiest way to get up and running with ClickHouse is to create For production installs of a specific release version see the [installation options](#available-installation-options) down below. ::: -On Linux and macOS: +On Linux, macOS and FreeBSD: -1. If you are just getting started and want to see what ClickHouse can do, the simplest way to download ClickHouse locally is to run the following command. It downloads a single binary for your operating system that can be used to run the ClickHouse server, clickhouse-client, clickhouse-local, -ClickHouse Keeper, and other tools: +1. If you are just getting started and want to see what ClickHouse can do, the simplest way to download ClickHouse locally is to run the + following command. It downloads a single binary for your operating system that can be used to run the ClickHouse server, + clickhouse-client, clickhouse-local, ClickHouse Keeper, and other tools: ```bash curl https://clickhouse.com/ | sh ``` 1. Run the following command to start the ClickHouse server: + ```bash ./clickhouse server ``` - The first time you run this script, the necessary files and folders are created in the current directory, then the server starts. + The first time you run this script, the necessary files and folders are created in the current directory, then the server starts. 1. Open a new terminal and use the **clickhouse-client** to connect to your service: @@ -330,7 +332,9 @@ For production environments, it’s recommended to use the latest `stable`-versi To run ClickHouse inside Docker follow the guide on [Docker Hub](https://hub.docker.com/r/clickhouse/clickhouse-server/). Those images use official `deb` packages inside. -### From Sources {#from-sources} +## Non-Production Deployments (Advanced) + +### Compile From Source {#from-sources} To manually compile ClickHouse, follow the instructions for [Linux](/docs/en/development/build.md) or [macOS](/docs/en/development/build-osx.md). @@ -346,8 +350,31 @@ You’ll need to create data and metadata folders manually and `chown` them for On Gentoo, you can just use `emerge clickhouse` to install ClickHouse from sources. -### From CI checks pre-built binaries -ClickHouse binaries are built for each [commit](/docs/en/development/build.md#you-dont-have-to-build-clickhouse). +### Install a CI-generated Binary + +ClickHouse's continuous integration (CI) infrastructure produces specialized builds for each commit, e.g. +[sanitized](https://github.com/google/sanitizers) builds, unoptimized (Debug) builds, cross-compiled builds etc. While such builds are +normally only useful during development, they can in certain situations also be interesting for users. + +:::note +Since ClickHouse's CI is evolving over time, the exact steps to download CI-generated builds may vary. +Also, CI may delete too old build artifacts, making them unavailable for download. +::: + +For example, to download a aarch64 binary for ClickHouse v23.4, follow these steps: + +- Find the GitHub pull request for release v23.4: [Release pull request for branch 23.4](https://github.com/ClickHouse/ClickHouse/pull/49238) +- Click "Commits", then click a commit similar to "Update autogenerated version to 23.4.2.1 and contributors" for the particular version you like to install. +- Click the red cross to open the list of CI checks. +- Click "Details" next to "ClickHouse Build Check" in the list, it will open a page similar to [this page](https://s3.amazonaws.com/clickhouse-test-reports/46793/b460eb70bf29b19eadd19a1f959b15d186705394/clickhouse_build_check/report.html) +- Find the rows with compiler = "clang-*-aarch64" - there are multiple rows. +- Download the artifacts for these builds. + +To download binaries for very old x86-64 systems without [SSE3](https://en.wikipedia.org/wiki/SSE3) support or old ARM systems without +[ARMv8.1-A](https://en.wikipedia.org/wiki/AArch64#ARMv8.1-A) support, open a GitHub pull request and find CI check "BuilderBinAmd64Compat", +respectively "BuilderBinAarch64V80Compat". Then click "Details", open the "Build" fold, scroll to the end, find message "Notice: Build URLs +https://s3.amazonaws.com/clickhouse/builds/PRs/.../.../binary_aarch64_v80compat/clickhouse". You can then click the link to download the +build. ## Launch {#launch} From e0759eff97912fb7b20a70b8a45ce39944711725 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 May 2023 17:28:48 +0000 Subject: [PATCH 0824/2223] Remove install scripts superseded by universal.sh --- docs/_includes/install/arm.sh | 6 ------ docs/_includes/install/freebsd.sh | 3 --- docs/_includes/install/mac-arm.sh | 3 --- docs/_includes/install/mac-x86.sh | 3 --- 4 files changed, 15 deletions(-) delete mode 100644 docs/_includes/install/arm.sh delete mode 100644 docs/_includes/install/freebsd.sh delete mode 100644 docs/_includes/install/mac-arm.sh delete mode 100644 docs/_includes/install/mac-x86.sh diff --git a/docs/_includes/install/arm.sh b/docs/_includes/install/arm.sh deleted file mode 100644 index c6b1f7d12f8..00000000000 --- a/docs/_includes/install/arm.sh +++ /dev/null @@ -1,6 +0,0 @@ -# ARM (AArch64) build works on Amazon Graviton, Oracle Cloud, Huawei Cloud ARM machines. -# The support for AArch64 is pre-production ready. - -wget 'https://builds.clickhouse.com/master/aarch64/clickhouse' -chmod a+x ./clickhouse -sudo ./clickhouse install diff --git a/docs/_includes/install/freebsd.sh b/docs/_includes/install/freebsd.sh deleted file mode 100644 index 2a715a1795f..00000000000 --- a/docs/_includes/install/freebsd.sh +++ /dev/null @@ -1,3 +0,0 @@ -fetch 'https://builds.clickhouse.com/master/freebsd/clickhouse' -chmod a+x ./clickhouse -su -m root -c './clickhouse install' diff --git a/docs/_includes/install/mac-arm.sh b/docs/_includes/install/mac-arm.sh deleted file mode 100644 index 5863d5b090d..00000000000 --- a/docs/_includes/install/mac-arm.sh +++ /dev/null @@ -1,3 +0,0 @@ -wget 'https://builds.clickhouse.com/master/macos-aarch64/clickhouse' -chmod a+x ./clickhouse -./clickhouse diff --git a/docs/_includes/install/mac-x86.sh b/docs/_includes/install/mac-x86.sh deleted file mode 100644 index 2a216b534b4..00000000000 --- a/docs/_includes/install/mac-x86.sh +++ /dev/null @@ -1,3 +0,0 @@ -wget 'https://builds.clickhouse.com/master/macos/clickhouse' -chmod a+x ./clickhouse -./clickhouse From 536c4a99c8edbe22dec31f7b830f8fb71f208020 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 29 May 2023 21:03:07 +0000 Subject: [PATCH 0825/2223] Fix clickhouse-local crashing when writing empty Arrow or Parquet output --- src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp | 2 +- .../0_stateless/01429_empty_arrow_and_parquet.reference | 3 +++ tests/queries/0_stateless/01429_empty_arrow_and_parquet.sh | 7 +++++++ 4 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index 5e7d3edb528..1e72c949b09 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -75,7 +75,7 @@ void ArrowBlockOutputFormat::finalizeImpl() { if (!writer) { - const Block & header = getPort(PortKind::Main).getHeader(); + Block header = materializeBlock(getPort(PortKind::Main).getHeader()); consume(Chunk(header.getColumns(), 0)); } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index e39271efa7c..031e499b545 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -105,7 +105,7 @@ void ParquetBlockOutputFormat::finalizeImpl() if (!file_writer) { - const Block & header = getPort(PortKind::Main).getHeader(); + Block header = materializeBlock(getPort(PortKind::Main).getHeader()); write(Chunk(header.getColumns(), 0), 1); } diff --git a/tests/queries/0_stateless/01429_empty_arrow_and_parquet.reference b/tests/queries/0_stateless/01429_empty_arrow_and_parquet.reference index 167851e057b..957d112aaf2 100644 --- a/tests/queries/0_stateless/01429_empty_arrow_and_parquet.reference +++ b/tests/queries/0_stateless/01429_empty_arrow_and_parquet.reference @@ -4,3 +4,6 @@ 2 3 3 +0 +0 +0 diff --git a/tests/queries/0_stateless/01429_empty_arrow_and_parquet.sh b/tests/queries/0_stateless/01429_empty_arrow_and_parquet.sh index 129f558ee2e..b89fc617851 100755 --- a/tests/queries/0_stateless/01429_empty_arrow_and_parquet.sh +++ b/tests/queries/0_stateless/01429_empty_arrow_and_parquet.sh @@ -2,6 +2,8 @@ # Tags: no-fasttest set -e +# Fail `A | B` if A fails. +set -o pipefail CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -25,3 +27,8 @@ ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test_empty_data" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test_empty_data" ${CLICKHOUSE_CLIENT} -q "DROP TABLE test_empty_data" + +# Test with clickhouse-local too. (It used to crash because header block had const columns.) +$CLICKHOUSE_LOCAL -q "select 1 as x where 0 format Arrow" | $CLICKHOUSE_LOCAL --input-format Arrow --structure="x Int8" -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select 1 as x where 0 format ArrowStream" | $CLICKHOUSE_LOCAL --input-format ArrowStream --structure="x Int8" -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select 1 as x where 0 format Parquet" | $CLICKHOUSE_LOCAL --input-format Parquet --structure="x Int8" -q "select count() from table" From d7002f8b94193f4a9c0dd1fcb154c55d63151d7f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 30 May 2023 17:46:21 +0000 Subject: [PATCH 0826/2223] Incorporate Dan's feedback --- docs/en/development/build.md | 17 +---------------- docs/en/getting-started/install.md | 16 +++++++++------- .../aspell-ignore/en/aspell-dict.txt | 6 ++++++ 3 files changed, 16 insertions(+), 23 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index c35e077a166..83a4550df88 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -43,7 +43,7 @@ sudo add-apt-repository -y ppa:ubuntu-toolchain-r/test For other Linux distribution - check the availability of LLVM's [prebuild packages](https://releases.llvm.org/download.html). As of April 2023, any version of Clang >= 15 will work. -GCC as a compiler is not supported +GCC as a compiler is not supported. To build with a specific Clang version: :::tip @@ -114,18 +114,3 @@ mkdir build cmake -S . -B build cmake --build build ``` - -## You Don’t Have to Build ClickHouse {#you-dont-have-to-build-clickhouse} - -ClickHouse is available in pre-built binaries and packages. Binaries are portable and can be run on any Linux flavour. - -The CI checks build the binaries on each commit to [ClickHouse](https://github.com/clickhouse/clickhouse/). To download them: - -1. Open the [commits list](https://github.com/ClickHouse/ClickHouse/commits/master) -1. Choose a **Merge pull request** commit that includes the new feature, or was added after the new feature -1. Click the status symbol (yellow dot, red x, green check) to open the CI check list -1. Scroll through the list until you find **ClickHouse build check x/x artifact groups are OK** -1. Click **Details** -1. Find the type of package for your operating system that you need and download the files. - -![build artifact check](images/find-build-artifact.png) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 74d712d9f5e..d44dc861888 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -46,7 +46,7 @@ On Linux, macOS and FreeBSD: The first time you run this script, the necessary files and folders are created in the current directory, then the server starts. -1. Open a new terminal and use the **clickhouse-client** to connect to your service: +1. Open a new terminal and use the **./clickhouse client** to connect to your service: ```bash ./clickhouse client @@ -352,9 +352,10 @@ On Gentoo, you can just use `emerge clickhouse` to install ClickHouse from sourc ### Install a CI-generated Binary -ClickHouse's continuous integration (CI) infrastructure produces specialized builds for each commit, e.g. -[sanitized](https://github.com/google/sanitizers) builds, unoptimized (Debug) builds, cross-compiled builds etc. While such builds are -normally only useful during development, they can in certain situations also be interesting for users. +ClickHouse's continuous integration (CI) infrastructure produces specialized builds for each commit in the [ClickHouse +repository](https://github.com/clickhouse/clickhouse/), e.g. [sanitized](https://github.com/google/sanitizers) builds, unoptimized (Debug) +builds, cross-compiled builds etc. While such builds are normally only useful during development, they can in certain situations also be +interesting for users. :::note Since ClickHouse's CI is evolving over time, the exact steps to download CI-generated builds may vary. @@ -365,14 +366,15 @@ For example, to download a aarch64 binary for ClickHouse v23.4, follow these ste - Find the GitHub pull request for release v23.4: [Release pull request for branch 23.4](https://github.com/ClickHouse/ClickHouse/pull/49238) - Click "Commits", then click a commit similar to "Update autogenerated version to 23.4.2.1 and contributors" for the particular version you like to install. -- Click the red cross to open the list of CI checks. +- Click the green check / yellow dot / red cross to open the list of CI checks. - Click "Details" next to "ClickHouse Build Check" in the list, it will open a page similar to [this page](https://s3.amazonaws.com/clickhouse-test-reports/46793/b460eb70bf29b19eadd19a1f959b15d186705394/clickhouse_build_check/report.html) - Find the rows with compiler = "clang-*-aarch64" - there are multiple rows. - Download the artifacts for these builds. To download binaries for very old x86-64 systems without [SSE3](https://en.wikipedia.org/wiki/SSE3) support or old ARM systems without -[ARMv8.1-A](https://en.wikipedia.org/wiki/AArch64#ARMv8.1-A) support, open a GitHub pull request and find CI check "BuilderBinAmd64Compat", -respectively "BuilderBinAarch64V80Compat". Then click "Details", open the "Build" fold, scroll to the end, find message "Notice: Build URLs +[ARMv8.1-A](https://en.wikipedia.org/wiki/AArch64#ARMv8.1-A) support, open a [pull +request](https://github.com/ClickHouse/ClickHouse/commits/master) and find CI check "BuilderBinAmd64Compat", respectively +"BuilderBinAarch64V80Compat". Then click "Details", open the "Build" fold, scroll to the end, find message "Notice: Build URLs https://s3.amazonaws.com/clickhouse/builds/PRs/.../.../binary_aarch64_v80compat/clickhouse". You can then click the link to download the build. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 65844a4946e..ded7a4643a9 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2,6 +2,7 @@ personal_ws-1.1 en 543 AArch ACLs AMQP +ARMv ASLR ASan Actian @@ -12,6 +13,8 @@ AvroConfluent BSON BSONEachRow Bool +BuilderBinAarch +BuilderBinAmd CCTOOLS CLion CMake @@ -27,6 +30,7 @@ ClickHouse's ClickableSquare CodeBlock CodeLLDB +Compat Config ConnectionDetails Contrib @@ -213,6 +217,7 @@ anonymized ansi async atomicity +autogenerated autogeneration autostart avro @@ -539,6 +544,7 @@ uint unary unencrypted unixodbc +unoptimized url userspace userver From c7403284260992c296fbb34782bd1007a18ba28f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 May 2023 18:51:18 +0000 Subject: [PATCH 0827/2223] Fix more tests. --- src/Interpreters/ActionsVisitor.cpp | 11 +++++++---- src/Planner/CollectSets.cpp | 2 ++ src/Processors/QueryPlan/ReadFromMergeTree.cpp | 13 +++++++++++-- 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 23c6867a868..142b6f73b75 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1420,11 +1420,14 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } - // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; - if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) + if (!data.getContext()->isGlobalContext()) { - external_table_set = tmp_table->future_set; - // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; + // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; + if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) + { + external_table_set = tmp_table->future_set; + // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; + } } } diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 19dc569ff0c..913cf1c6ee3 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -90,6 +90,8 @@ public: in_second_argument_node_type == QueryTreeNodeType::UNION) { auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + if (sets.getFuture(set_key)) + return; auto subquery_options = select_query_options.subquery(); Planner subquery_planner( diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 367d3dcb525..8483df797ef 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1236,6 +1236,15 @@ static void buildIndexes( if (!indexes->use_skip_indexes) return; + const SelectQueryInfo * info = &query_info; + std::optional info_copy; + if (settings.allow_experimental_analyzer) + { + info_copy.emplace(query_info); + info_copy->filter_actions_dag = filter_actions_dag; + info = &*info_copy; + } + UsefulSkipIndexes skip_indexes; using Key = std::pair; std::map merged; @@ -1249,14 +1258,14 @@ static void buildIndexes( if (inserted) { skip_indexes.merged_indices.emplace_back(); - skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot); } skip_indexes.merged_indices[it->second].addIndex(index_helper); } else { - auto condition = index_helper->createIndexCondition(query_info, context); + auto condition = index_helper->createIndexCondition(*info, context); if (!condition->alwaysUnknownOrTrue()) skip_indexes.useful_indices.emplace_back(index_helper, condition); } From 7fcafaabd515b9453b3ce7b5f5453e386d70ad67 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 May 2023 20:56:48 +0200 Subject: [PATCH 0828/2223] Fix --- .../IO/AsynchronousBoundedReadBuffer.cpp | 2 +- .../02770_async_buffer_ignore.reference | 1 + .../0_stateless/02770_async_buffer_ignore.sh | 29 +++++++++++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02770_async_buffer_ignore.reference create mode 100755 tests/queries/0_stateless/02770_async_buffer_ignore.sh diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index aed4c2f82f2..4aff69505a1 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -298,7 +298,7 @@ off_t AsynchronousBoundedReadBuffer::seek(off_t offset, int whence) * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. * Note: we read in range [file_offset_of_buffer_end, read_until_position). */ - if (read_until_position && new_pos < *read_until_position + if (file_offset_of_buffer_end && read_until_position && new_pos < *read_until_position && new_pos > file_offset_of_buffer_end && new_pos < file_offset_of_buffer_end + read_settings.remote_read_min_bytes_for_seek) { diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.reference b/tests/queries/0_stateless/02770_async_buffer_ignore.reference new file mode 100644 index 00000000000..9a9d6d6febf --- /dev/null +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.reference @@ -0,0 +1 @@ +4 66446 66446 diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh new file mode 100755 index 00000000000..f8ccce2b5bb --- /dev/null +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test_optimize_exception" +${CLICKHOUSE_CLIENT} -nm --query " +DROP TABLE IF EXISTS test_s3; + +CREATE TABLE test_s3 (a UInt64, b UInt64) +ENGINE = MergeTree ORDER BY a +SETTINGS storage_policy = 's3', min_bytes_for_wide_part = 0; + +INSERT INTO test_s3 SELECT number, number FROM numbers(1000000); +" +query="SELECT sum(b) FROM test_s3 WHERE a >= 100000 AND a <= 102000" +query_id=$(${CLICKHOUSE_CLIENT} --query "select queryID() from ($query) limit 1" 2>&1) +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -nm --query " +SELECT + ProfileEvents['S3ReadRequestsCount'], + ProfileEvents['ReadBufferFromS3Bytes'], + ProfileEvents['ReadCompressedBytes'] +FROM system.query_log +WHERE type = 'QueryFinish' + AND current_database = currentDatabase() + AND query_id='$query_id'; +" From 07467639a21ecad0e7744da898a6ff2682241f65 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 30 May 2023 21:00:15 +0200 Subject: [PATCH 0829/2223] Update 02770_async_buffer_ignore.sh --- tests/queries/0_stateless/02770_async_buffer_ignore.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh index f8ccce2b5bb..779d5b6729e 100755 --- a/tests/queries/0_stateless/02770_async_buffer_ignore.sh +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -10,7 +10,7 @@ DROP TABLE IF EXISTS test_s3; CREATE TABLE test_s3 (a UInt64, b UInt64) ENGINE = MergeTree ORDER BY a -SETTINGS storage_policy = 's3', min_bytes_for_wide_part = 0; +SETTINGS disk = 's3_disk', min_bytes_for_wide_part = 0; INSERT INTO test_s3 SELECT number, number FROM numbers(1000000); " From daaae3f573c03be49c9e015c249642034113374d Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 30 May 2023 19:11:26 +0000 Subject: [PATCH 0830/2223] Add toString() to fix time zone error --- tests/queries/0_stateless/01746_convert_type_with_default.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index 18b5ae60920..40e4798721b 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -37,7 +37,7 @@ select toDateOrDefault(-1, '2000-01-01'::Date); select toDateTimeOrDefault('2023-05-30 14:38:20'); select toDateTimeOrDefault('2023-05-30 14:38:20', 'UTC'); -select toDateTimeOrDefault('s2023', 'UTC', '2023-05-30 14:38:20'::DateTime('UTC')); +select toString(toDateTimeOrDefault('s2023', 'Asia/Istanbul', '2023-05-30 14:38:20'::DateTime('Asia/Istanbul')), 'Asia/Istanbul'); select toDateTimeOrDefault(1685457500); SELECT toUUIDOrDefault('61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); From f15769ee6c9a7923a8e7651caf965c81e054ba0f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 May 2023 21:29:08 +0200 Subject: [PATCH 0831/2223] Fix --- src/Interpreters/Cache/FileCache.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 1908a4ce895..65dca790183 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -592,7 +592,6 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) std::unordered_map to_delete; size_t freeable_space = 0, freeable_count = 0; - size_t removed_size = 0; auto iterate_func = [&](LockedKey & locked_key, FileSegmentMetadataPtr segment_metadata) { chassert(segment_metadata->file_segment->assertCorrectness()); @@ -659,8 +658,8 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) && freeable_count == 0 && main_priority->getElementsCount(cache_lock) == main_priority->getElementsLimit()); LOG_TEST( - log, "Overflow: {}, size: {}, ready to remove: {}, current cache size: {}/{}, elements: {}/{}, while reserving for {}:{}", - is_overflow, size, removed_size, + log, "Overflow: {}, size: {}, ready to remove: {} ({} in number), current cache size: {}/{}, elements: {}/{}, while reserving for {}:{}", + is_overflow, size, freeable_space, freeable_count, main_priority->getSize(cache_lock), main_priority->getSizeLimit(), main_priority->getElementsCount(cache_lock), main_priority->getElementsLimit(), file_segment.key(), file_segment.offset()); From d4efbbfbd3ca954b83391c60afffe760cc602361 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 30 May 2023 19:32:24 +0000 Subject: [PATCH 0832/2223] Allow to skip empty files in file/s3/url/hdfs table functions --- src/Core/Settings.h | 4 + src/Storages/HDFS/StorageHDFS.cpp | 113 +++++++++++------- src/Storages/HDFS/StorageHDFS.h | 23 +++- src/Storages/HDFS/StorageHDFSCluster.cpp | 2 +- src/Storages/StorageFile.cpp | 59 ++++++--- src/Storages/StorageS3.cpp | 12 +- src/Storages/StorageURL.cpp | 43 ++++--- src/Storages/StorageURL.h | 2 +- tests/integration/test_storage_hdfs/test.py | 50 ++++++++ tests/integration/test_storage_s3/test.py | 55 +++++++++ .../02771_skip_empty_files.reference | 7 ++ .../0_stateless/02771_skip_empty_files.sh | 24 ++++ 12 files changed, 307 insertions(+), 87 deletions(-) create mode 100644 tests/queries/0_stateless/02771_skip_empty_files.reference create mode 100755 tests/queries/0_stateless/02771_skip_empty_files.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 607be1522db..534cb629aa8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -91,6 +91,7 @@ class IColumn; M(UInt64, s3_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ + M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ @@ -99,6 +100,7 @@ class IColumn; M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ + M(Bool, hdfs_skip_empty_files, false, "Allow to skip empty files in hdfs table engine", 0) \ M(UInt64, hsts_max_age, 0, "Expired time for hsts. 0 means disable HSTS.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, false, "Whether to use the cache of uncompressed blocks.", 0) \ @@ -602,6 +604,8 @@ class IColumn; M(Bool, engine_file_empty_if_not_exists, false, "Allows to select data from a file engine table without file", 0) \ M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \ M(Bool, engine_file_allow_create_multiple_files, false, "Enables or disables creating a new file on each insert in file engine tables if format has suffix.", 0) \ + M(Bool, engine_file_skip_empty_files, false, "Allows to skip empty files in file table engine", 0) \ + M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in url table engine", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(Bool, database_replicated_enforce_synchronous_settings, false, "Enforces synchronous waiting for some queries (see also database_atomic_wait_for_drop_and_detach_synchronously, mutation_sync, alter_sync). Not recommended to enable these settings.", 0) \ diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 19c0840149b..08114ed3cba 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -66,7 +66,7 @@ namespace /* Recursive directory listing with matched paths as a result. * Have the same method in StorageFile. */ - Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match, std::unordered_map * last_mod_times) + std::vector LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match) { const size_t first_glob = for_match.find_first_of("*?{"); @@ -88,7 +88,7 @@ namespace throw Exception( ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", prefix_without_globs, String(hdfsGetLastError())); } - Strings result; + std::vector result; if (!ls.file_info && ls.length > 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); for (int i = 0; i < ls.length; ++i) @@ -102,17 +102,15 @@ namespace if (!is_directory && !looking_for_directory) { if (re2::RE2::FullMatch(file_name, matcher)) - { - result.push_back(String(ls.file_info[i].mName)); - if (last_mod_times) - (*last_mod_times)[result.back()] = ls.file_info[i].mLastMod; - } + result.emplace_back( + String(ls.file_info[i].mName), + StorageHDFS::PathInfo{ls.file_info[i].mLastMod, static_cast(ls.file_info[i].mSize)}); } else if (is_directory && looking_for_directory) { if (re2::RE2::FullMatch(file_name, matcher)) { - Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash), last_mod_times); + std::vector result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash)); /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } @@ -135,12 +133,20 @@ namespace throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage HDFS requires valid URL to be set"); } - std::vector getPathsList(const String & path_from_uri, const String & uri_without_path, ContextPtr context, std::unordered_map * last_mod_times = nullptr) + std::vector getPathsList(const String & path_from_uri, const String & uri_without_path, ContextPtr context) { HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - return LSWithRegexpMatching("/", fs, path_from_uri, last_mod_times); + return LSWithRegexpMatching("/", fs, path_from_uri); + } + + size_t getFileSize(const String & path_from_uri, const String & uri_without_path, ContextPtr context) + { + HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); + HDFSFSPtr fs = createHDFSFS(builder.get()); + auto * info = hdfsGetPathInfo(fs.get(), path_from_uri.data()); + return info->mSize; } } @@ -199,9 +205,8 @@ ColumnsDescription StorageHDFS::getTableStructureFromData( ContextPtr ctx) { const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); - std::unordered_map last_mod_time; - auto paths = getPathsList(path_from_uri, uri, ctx, &last_mod_time); - if (paths.empty() && !FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format)) + auto paths_with_info = getPathsList(path_from_uri, uri, ctx); + if (paths_with_info.empty() && !FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format)) throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file, because there are no files in HDFS with provided path." @@ -209,14 +214,27 @@ ColumnsDescription StorageHDFS::getTableStructureFromData( std::optional columns_from_cache; if (ctx->getSettingsRef().schema_inference_use_cache_for_hdfs) - columns_from_cache = tryGetColumnsFromCache(paths, path_from_uri, last_mod_time, format, ctx); + columns_from_cache = tryGetColumnsFromCache(paths_with_info, path_from_uri, format, ctx); - ReadBufferIterator read_buffer_iterator = [&, my_uri_without_path = uri_without_path, it = paths.begin()](ColumnsDescription &) mutable -> std::unique_ptr + ReadBufferIterator read_buffer_iterator + = [&, my_uri_without_path = uri_without_path, it = paths_with_info.begin(), first = true]( + ColumnsDescription & columns) mutable -> std::unique_ptr { - if (it == paths.end()) + if (it == paths_with_info.end()) + { + if (first) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because all files are empty. " + "You must specify table structure manually", format); return nullptr; - auto compression = chooseCompressionMethod(*it, compression_method); - auto impl = std::make_unique(my_uri_without_path, *it++, ctx->getGlobalContext()->getConfigRef(), ctx->getReadSettings()); + } + + auto path_with_info = *it++; + if (ctx->getSettingsRef().hdfs_skip_empty_files && path_with_info.info && path_with_info.info->size == 0) + return read_buffer_iterator(columns); + + auto compression = chooseCompressionMethod(path_with_info.path, compression_method); + auto impl = std::make_unique(my_uri_without_path, path_with_info.path, ctx->getGlobalContext()->getConfigRef(), ctx->getReadSettings()); const Int64 zstd_window_log_max = ctx->getSettingsRef().zstd_window_log_max; return wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); }; @@ -225,10 +243,10 @@ ColumnsDescription StorageHDFS::getTableStructureFromData( if (columns_from_cache) columns = *columns_from_cache; else - columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, paths.size() > 1, ctx); + columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, paths_with_info.size() > 1, ctx); if (ctx->getSettingsRef().schema_inference_use_cache_for_hdfs) - addColumnsToCache(paths, path_from_uri, columns, format, ctx); + addColumnsToCache(paths_with_info, path_from_uri, columns, format, ctx); return columns; } @@ -241,11 +259,11 @@ public: const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); uris = getPathsList(path_from_uri, uri_without_path, context_); for (auto & elem : uris) - elem = uri_without_path + elem; + elem.path = uri_without_path + elem.path; uris_iter = uris.begin(); } - String next() + StorageHDFS::PathWithInfo next() { std::lock_guard lock(mutex); if (uris_iter != uris.end()) @@ -258,8 +276,8 @@ public: } private: std::mutex mutex; - Strings uris; - Strings::iterator uris_iter; + std::vector uris; + std::vector::iterator uris_iter; }; class HDFSSource::URISIterator::Impl @@ -279,14 +297,14 @@ public: uris_iter = uris.begin(); } - String next() + StorageHDFS::PathWithInfo next() { std::lock_guard lock(mutex); if (uris_iter == uris.end()) - return ""; + return {"", {}}; auto key = *uris_iter; ++uris_iter; - return key; + return {key, {}}; } private: @@ -298,7 +316,7 @@ private: HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(ContextPtr context_, const String & uri) : pimpl(std::make_shared(context_, uri)) {} -String HDFSSource::DisclosedGlobIterator::next() +StorageHDFS::PathWithInfo HDFSSource::DisclosedGlobIterator::next() { return pimpl->next(); } @@ -308,7 +326,7 @@ HDFSSource::URISIterator::URISIterator(const std::vector & uris_, Contex { } -String HDFSSource::URISIterator::next() +StorageHDFS::PathWithInfo HDFSSource::URISIterator::next() { return pimpl->next(); } @@ -343,12 +361,21 @@ HDFSSource::HDFSSource( bool HDFSSource::initialize() { - current_path = (*file_iterator)(); - if (current_path.empty()) + auto path_with_info = (*file_iterator)(); + if (path_with_info.path.empty()) return false; + current_path = path_with_info.path; const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_path); + if (getContext()->getSettingsRef().hdfs_skip_empty_files) + { + auto file_size = path_with_info.info ? path_with_info.info->size : getFileSize(path_from_uri, uri_without_path, getContext()); + /// If file is empty and hdfs_skip_empty_files=1, skip it and go to the next file. + if (file_size == 0) + return initialize(); + } + auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); auto impl = std::make_unique( uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); @@ -553,8 +580,8 @@ Pipe StorageHDFS::read( if (distributed_processing) { iterator_wrapper = std::make_shared( - [callback = context_->getReadTaskCallback()]() -> String { - return callback(); + [callback = context_->getReadTaskCallback()]() -> StorageHDFS::PathWithInfo { + return StorageHDFS::PathWithInfo{callback(), std::nullopt}; }); } else if (is_path_with_globs) @@ -761,24 +788,22 @@ SchemaCache & StorageHDFS::getSchemaCache(const ContextPtr & ctx) } std::optional StorageHDFS::tryGetColumnsFromCache( - const Strings & paths, + const std::vector & paths_with_info, const String & uri_without_path, - std::unordered_map & last_mod_time, const String & format_name, const ContextPtr & ctx) { auto & schema_cache = getSchemaCache(ctx); - for (const auto & path : paths) + for (const auto & path_with_info : paths_with_info) { auto get_last_mod_time = [&]() -> std::optional { - auto it = last_mod_time.find(path); - if (it == last_mod_time.end()) - return std::nullopt; - return it->second; + if (path_with_info.info) + return path_with_info.info->last_mod_time; + return std::nullopt; }; - String url = fs::path(uri_without_path) / path; + String url = fs::path(uri_without_path) / path_with_info.path; auto cache_key = getKeyForSchemaCache(url, format_name, {}, ctx); auto columns = schema_cache.tryGet(cache_key, get_last_mod_time); if (columns) @@ -789,7 +814,7 @@ std::optional StorageHDFS::tryGetColumnsFromCache( } void StorageHDFS::addColumnsToCache( - const Strings & paths, + const std::vector & paths_with_info, const String & uri_without_path, const ColumnsDescription & columns, const String & format_name, @@ -797,8 +822,8 @@ void StorageHDFS::addColumnsToCache( { auto & schema_cache = getSchemaCache(ctx); Strings sources; - sources.reserve(paths.size()); - std::transform(paths.begin(), paths.end(), std::back_inserter(sources), [&](const String & path){ return fs::path(uri_without_path) / path; }); + sources.reserve(paths_with_info.size()); + std::transform(paths_with_info.begin(), paths_with_info.end(), std::back_inserter(sources), [&](const PathWithInfo & path_with_info){ return fs::path(uri_without_path) / path_with_info.path; }); auto cache_keys = getKeysForSchemaCache(sources, format_name, {}, ctx); schema_cache.addMany(cache_keys, columns); } diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index b123834e981..87ad5aee6a3 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -18,6 +18,18 @@ namespace DB class StorageHDFS final : public IStorage, WithContext { public: + struct PathInfo + { + time_t last_mod_time; + size_t size; + }; + + struct PathWithInfo + { + String path; + std::optional info; + }; + StorageHDFS( const String & uri_, const StorageID & table_id_, @@ -72,14 +84,13 @@ protected: private: static std::optional tryGetColumnsFromCache( - const Strings & paths, + const std::vector & paths_with_info, const String & uri_without_path, - std::unordered_map & last_mod_time, const String & format_name, const ContextPtr & ctx); static void addColumnsToCache( - const Strings & paths, + const std::vector & paths, const String & uri_without_path, const ColumnsDescription & columns, const String & format_name, @@ -105,7 +116,7 @@ public: { public: DisclosedGlobIterator(ContextPtr context_, const String & uri_); - String next(); + StorageHDFS::PathWithInfo next(); private: class Impl; /// shared_ptr to have copy constructor @@ -116,14 +127,14 @@ public: { public: URISIterator(const std::vector & uris_, ContextPtr context); - String next(); + StorageHDFS::PathWithInfo next(); private: class Impl; /// shared_ptr to have copy constructor std::shared_ptr pimpl; }; - using IteratorWrapper = std::function; + using IteratorWrapper = std::function; using StorageHDFSPtr = std::shared_ptr; static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 46e67b623e2..b98459aeee3 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -79,7 +79,7 @@ void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(ASTPtr, const ContextPtr & context) const { auto iterator = std::make_shared(context, uri); - auto callback = std::make_shared([iter = std::move(iterator)]() mutable -> String { return iter->next(); }); + auto callback = std::make_shared>([iter = std::move(iterator)]() mutable -> String { return iter->next().path; }); return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 647f9511052..a20a4e63ba6 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -257,35 +257,40 @@ std::unique_ptr selectReadBuffer( return res; } -std::unique_ptr createReadBuffer( - const String & current_path, - bool use_table_fd, - const String & storage_name, - int table_fd, - const String & compression_method, - ContextPtr context) +struct stat getFileStat(const String & current_path, bool use_table_fd, int table_fd, const String & storage_name) { - CompressionMethod method; - struct stat file_stat{}; - if (use_table_fd) { /// Check if file descriptor allows random reads (and reading it twice). if (0 != fstat(table_fd, &file_stat)) throwFromErrno("Cannot stat table file descriptor, inside " + storage_name, ErrorCodes::CANNOT_STAT); - - method = chooseCompressionMethod("", compression_method); } else { /// Check if file descriptor allows random reads (and reading it twice). if (0 != stat(current_path.c_str(), &file_stat)) throwFromErrno("Cannot stat file " + current_path, ErrorCodes::CANNOT_STAT); - - method = chooseCompressionMethod(current_path, compression_method); } + return file_stat; +} + +std::unique_ptr createReadBuffer( + const String & current_path, + const struct stat & file_stat, + bool use_table_fd, + int table_fd, + const String & compression_method, + ContextPtr context) +{ + CompressionMethod method; + + if (use_table_fd) + method = chooseCompressionMethod("", compression_method); + else + method = chooseCompressionMethod(current_path, compression_method); + std::unique_ptr nested_buffer = selectReadBuffer(current_path, use_table_fd, table_fd, file_stat, context); /// For clickhouse-local and clickhouse-client add progress callback to display progress bar. @@ -355,7 +360,8 @@ ColumnsDescription StorageFile::getTableStructureFromFileDescriptor(ContextPtr c { /// We will use PeekableReadBuffer to create a checkpoint, so we need a place /// where we can store the original read buffer. - read_buffer_from_fd = createReadBuffer("", true, getName(), table_fd, compression_method, context); + auto file_stat = getFileStat("", true, table_fd, getName()); + read_buffer_from_fd = createReadBuffer("", file_stat, true, table_fd, compression_method, context); auto read_buf = std::make_unique(*read_buffer_from_fd); read_buf->setCheckpoint(); return read_buf; @@ -396,12 +402,24 @@ ColumnsDescription StorageFile::getTableStructureFromFile( if (context->getSettingsRef().schema_inference_use_cache_for_file) columns_from_cache = tryGetColumnsFromCache(paths, format, format_settings, context); - ReadBufferIterator read_buffer_iterator = [&, it = paths.begin()](ColumnsDescription &) mutable -> std::unique_ptr + ReadBufferIterator read_buffer_iterator = [&, it = paths.begin(), first = true](ColumnsDescription & columns) mutable -> std::unique_ptr { if (it == paths.end()) + { + if (first) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because all files are empty. You must specify table structure manually", + format); return nullptr; + } - return createReadBuffer(*it++, false, "File", -1, compression_method, context); + auto path = *it++; + auto file_stat = getFileStat(path, false, -1, "File"); + if (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) + return read_buffer_iterator(columns); + + first = false; + return createReadBuffer(path, file_stat, false, -1, compression_method, context); }; ColumnsDescription columns; @@ -628,7 +646,12 @@ public: } if (!read_buf) - read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context); + { + auto file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName()); + if (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) + continue; + read_buf = createReadBuffer(current_path, file_stat, storage->use_table_fd, storage->table_fd, storage->compression_method, context); + } const Settings & settings = context->getSettingsRef(); chassert(!storage->paths.empty()); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2a2192d9cfe..9c4791020f2 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -573,6 +573,11 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() return {}; size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); + + /// If object is empty and s3_skip_empty_files=1, skip it and go to the next key. + if (getContext()->getSettingsRef().s3_skip_empty_files && object_size == 0) + return createReader(); + auto compression_method = chooseCompressionMethod(current_key, compression_hint); InputFormatPtr input_format; @@ -1456,7 +1461,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( ReadBufferIterator read_buffer_iterator = [&, first = true](ColumnsDescription & cached_columns) mutable -> std::unique_ptr { - auto [key, _] = (*file_iterator)(); + auto [key, info] = (*file_iterator)(); if (key.empty()) { @@ -1464,11 +1469,14 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file, because there are no files with provided path " - "in S3. You must specify table structure manually", configuration.format); + "in S3 or all files are empty. You must specify table structure manually", configuration.format); return nullptr; } + if (ctx->getSettingsRef().s3_skip_empty_files && info->size == 0) + return read_buffer_iterator(cached_columns); + /// S3 file iterator could get new keys after new iteration, check them in schema cache. if (ctx->getSettingsRef().schema_inference_use_cache_for_s3 && read_keys.size() > prev_read_keys_size) { diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index da8f6a151b2..706ce481a24 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -49,6 +49,7 @@ namespace ErrorCodes extern const int NETWORK_ERROR; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } static constexpr auto bad_arguments_error_message = "Storage URL requires 1-4 arguments: " @@ -242,15 +243,16 @@ StorageURLSource::StorageURLSource( auto headers = getHeaders(headers_); /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. - initialize = [=, this](const FailoverOptions & uri_options) + initialize = [=, this]() { - if (uri_options.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); + const auto current_uri_options = (*uri_iterator)(); + if (current_uri_options.empty()) + return false; - auto first_option = uri_options.begin(); + auto first_option = current_uri_options.begin(); auto [actual_uri, buf_factory] = getFirstAvailableURIAndReadBuffer( first_option, - uri_options.end(), + current_uri_options.end(), context, params, http_method, @@ -259,7 +261,11 @@ StorageURLSource::StorageURLSource( credentials, headers, glob_url, - uri_options.size() == 1); + current_uri_options.size() == 1); + + /// If file is empty and engine_url_skip_empty_files=1, skip it and go to the next file. + if (context->getSettingsRef().engine_url_skip_empty_files && buf_factory->getFileSize() == 0) + return initialize(); curr_uri = actual_uri; @@ -292,6 +298,7 @@ StorageURLSource::StorageURLSource( pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); reader = std::make_unique(*pipeline); + return true; }; } @@ -306,14 +313,8 @@ Chunk StorageURLSource::generate() break; } - if (!reader) - { - auto current_uri = (*uri_iterator)(); - if (current_uri.empty()) - return {}; - - initialize(current_uri); - } + if (!reader && !initialize()) + return {}; Chunk chunk; if (reader->pull(chunk)) @@ -592,10 +593,16 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( if (context->getSettingsRef().schema_inference_use_cache_for_url) columns_from_cache = tryGetColumnsFromCache(urls_to_check, headers, credentials, format, format_settings, context); - ReadBufferIterator read_buffer_iterator = [&, it = urls_to_check.cbegin()](ColumnsDescription &) mutable -> std::unique_ptr + ReadBufferIterator read_buffer_iterator = [&, it = urls_to_check.cbegin(), first = true](ColumnsDescription & columns) mutable -> std::unique_ptr { if (it == urls_to_check.cend()) + { + if (first) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because all files are empty. " + "You must specify table structure manually", format); return nullptr; + } auto [_, buf_factory] = StorageURLSource::getFirstAvailableURIAndReadBuffer( it, @@ -609,7 +616,13 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( headers, false, false); + ++it; + + if (context->getSettingsRef().engine_url_skip_empty_files && buf_factory->getFileSize() == 0) + return read_buffer_iterator(columns); + + first = false; return wrapReadBufferWithCompressionMethod( buf_factory->getReader(), compression_method, diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index d53b72105e4..4cd4b66e69a 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -197,7 +197,7 @@ public: bool delay_initialization); private: - using InitializeFunc = std::function; + using InitializeFunc = std::function; InitializeFunc initialize; String name; diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index edf5344e887..5ac1d3bea6f 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -816,6 +816,56 @@ def test_hdfsCluster_unset_skip_unavailable_shards(started_cluster): ) +def test_skip_empty_files(started_cluster): + node = started_cluster.instances["node1"] + + node.query( + f"insert into function hdfs('hdfs://hdfs1:9000/skip_empty_files1.parquet', TSVRaw) select * from numbers(0) settings hdfs_truncate_on_insert=1" + ) + + node.query( + f"insert into function hdfs('hdfs://hdfs1:9000/skip_empty_files2.parquet') select * from numbers(1) settings hdfs_truncate_on_insert=1" + ) + + node.query_and_get_error( + f"select * from hdfs('hdfs://hdfs1:9000/skip_empty_files1.parquet') settings hdfs_skip_empty_files=0" + ) + + node.query_and_get_error( + f"select * from hdfs('hdfs://hdfs1:9000/skip_empty_files1.parquet', auto, 'number UINt64') settings hdfs_skip_empty_files=0" + ) + + node.query_and_get_error( + f"select * from hdfs('hdfs://hdfs1:9000/skip_empty_files1.parquet') settings hdfs_skip_empty_files=1" + ) + + res = node.query( + f"select * from hdfs('hdfs://hdfs1:9000/skip_empty_files1.parquet', auto, 'number UInt64') settings hdfs_skip_empty_files=1" + ) + + assert len(res) == 0 + + node.query_and_get_error( + f"select * from hdfs('hdfs://hdfs1:9000/skip_empty_files*.parquet') settings hdfs_skip_empty_files=0" + ) + + node.query_and_get_error( + f"select * from hdfs('hdfs://hdfs1:9000/skip_empty_files*.parquet', auto, 'number UInt64') settings hdfs_skip_empty_files=0" + ) + + res = node.query( + f"select * from hdfs('hdfs://hdfs1:9000/skip_empty_files*.parquet') settings hdfs_skip_empty_files=1" + ) + + assert int(res) == 0 + + res = node.query( + f"select * from hdfs('hdfs://hdfs1:9000/skip_empty_files*.parquet', auto, 'number UInt64') settings hdfs_skip_empty_files=1" + ) + + assert int(res) == 0 + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index d9ac70f51ad..516c8ed152a 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1713,3 +1713,58 @@ def test_s3_list_objects_failure(started_cluster): assert ei.value.returncode == 243 assert "Could not list objects" in ei.value.stderr + + +def test_skip_empty_files(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files1.parquet', TSVRaw) select * from numbers(0) settings s3_truncate_on_insert=1" + ) + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files2.parquet') select * from numbers(1) settings s3_truncate_on_insert=1" + ) + def test(engine, setting): + instance.query_and_get_error( + f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files1.parquet') settings {setting}=0" + ) + + instance.query_and_get_error( + f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files1.parquet', auto, 'number UINt64') settings {setting}=0" + ) + + instance.query_and_get_error( + f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files1.parquet') settings {setting}=1" + ) + + res = instance.query( + f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files1.parquet', auto, 'number UInt64') settings {setting}=1" + ) + + assert len(res) == 0 + + instance.query_and_get_error( + f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{1,2}}.parquet') settings {setting}=0" + ) + + instance.query_and_get_error( + f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{1,2}}.parquet', auto, 'number UInt64') settings {setting}=0" + ) + + res = instance.query( + f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{1,2}}.parquet') settings {setting}=1" + ) + + assert int(res) == 0 + + res = instance.query( + f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{1,2}}.parquet', auto, 'number UInt64') settings {setting}=1" + ) + + assert int(res) == 0 + + test("s3", "s3_skip_empty_files") + test("url", "engine_url_skip_empty_files") diff --git a/tests/queries/0_stateless/02771_skip_empty_files.reference b/tests/queries/0_stateless/02771_skip_empty_files.reference new file mode 100644 index 00000000000..83f2e99acd0 --- /dev/null +++ b/tests/queries/0_stateless/02771_skip_empty_files.reference @@ -0,0 +1,7 @@ +1 +1 +1 +1 +1 +0 +0 diff --git a/tests/queries/0_stateless/02771_skip_empty_files.sh b/tests/queries/0_stateless/02771_skip_empty_files.sh new file mode 100755 index 00000000000..99f43d7868a --- /dev/null +++ b/tests/queries/0_stateless/02771_skip_empty_files.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +FILE_PREFIX=$CLICKHOUSE_TEST_UNIQUE_NAME +touch $FILE_PREFIX-1.parquet +$CLICKHOUSE_LOCAL -q "select * from numbers(1) format Parquet" > $FILE_PREFIX-2.parquet +$CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-1.parquet') settings engine_file_skip_empty_files=0" 2>&1 | grep -c "CANNOT_EXTRACT_TABLE_STRUCTURE" +$CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-1.parquet', auto, 'number UInt64') settings engine_file_skip_empty_files=0" 2>&1 | grep -c "Exception" +$CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-1.parquet') settings engine_file_skip_empty_files=1" 2>&1 | grep -c "Exception" +$CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-1.parquet', auto, 'number UInt64') settings engine_file_skip_empty_files=1" +$CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-*.parquet') settings engine_file_skip_empty_files=0" 2>&1 | grep -c "Exception" +$CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-*.parquet', auto, 'number UInt64') settings engine_file_skip_empty_files=0" 2>&1 | grep -c "Exception" +$CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-*.parquet') settings engine_file_skip_empty_files=1" +$CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-*.parquet', auto, 'number UInt64') settings engine_file_skip_empty_files=1" + + + + + + +rm $FILE_PREFIX-* From 38634cc5c5221a6ec646fc11dff34deda7c6b7d2 Mon Sep 17 00:00:00 2001 From: tpanetti Date: Wed, 24 May 2023 13:49:18 +0100 Subject: [PATCH 0833/2223] Convert Clickhouse Types to MySQL types in Compatibility mode This changes MySQL compatibility mode to display MySQL compatible types --- src/DataTypes/DataTypeAggregateFunction.h | 1 + src/DataTypes/DataTypeArray.h | 4 + src/DataTypes/DataTypeDate.h | 1 + src/DataTypes/DataTypeDate32.h | 1 + src/DataTypes/DataTypeDateTime.h | 1 + src/DataTypes/DataTypeDateTime64.h | 1 + src/DataTypes/DataTypeEnum.cpp | 24 ++++ src/DataTypes/DataTypeEnum.h | 3 + src/DataTypes/DataTypeFixedString.h | 1 + src/DataTypes/DataTypeFunction.h | 1 + src/DataTypes/DataTypeIPv4andIPv6.h | 4 + src/DataTypes/DataTypeInterval.h | 1 + src/DataTypes/DataTypeLowCardinality.h | 2 + src/DataTypes/DataTypeMap.h | 1 + src/DataTypes/DataTypeNothing.h | 2 + src/DataTypes/DataTypeNullable.h | 1 + src/DataTypes/DataTypeNumberBase.cpp | 17 +++ src/DataTypes/DataTypeNumberBase.h | 3 + src/DataTypes/DataTypeObject.h | 1 + src/DataTypes/DataTypeSet.h | 2 + src/DataTypes/DataTypeString.h | 3 + src/DataTypes/DataTypeTuple.h | 1 + src/DataTypes/DataTypeUUID.h | 2 + src/DataTypes/DataTypesDecimal.h | 3 + src/DataTypes/IDataType.h | 10 ++ src/Storages/System/StorageSystemColumns.cpp | 15 ++- .../02740_show_columns_mysql_compatibility.sh | 116 ++++++++++++++++++ 27 files changed, 221 insertions(+), 1 deletion(-) create mode 100755 tests/queries/0_stateless/02740_show_columns_mysql_compatibility.sh diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index 2d712d9c686..697be13652c 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -45,6 +45,7 @@ public: String doGetName() const override; String getNameWithoutVersion() const; const char * getFamilyName() const override { return "AggregateFunction"; } + const char * getMySQLName() const override { return "text"; } TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; } Array getParameters() const { return parameters; } diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 033a657c845..35462df9a4e 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -30,6 +30,10 @@ public: { return "Array"; } + const char * getMySQLName() const override + { + return "string"; + } bool canBeInsideNullable() const override { diff --git a/src/DataTypes/DataTypeDate.h b/src/DataTypes/DataTypeDate.h index 2f17207cc07..33bcb6123ff 100644 --- a/src/DataTypes/DataTypeDate.h +++ b/src/DataTypes/DataTypeDate.h @@ -13,6 +13,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Date; } const char * getFamilyName() const override { return family_name; } + const char * getMySQLName() const override { return "date"; } bool canBeUsedAsVersion() const override { return true; } bool canBeInsideNullable() const override { return true; } diff --git a/src/DataTypes/DataTypeDate32.h b/src/DataTypes/DataTypeDate32.h index 9160b62dc15..56315f46e8c 100644 --- a/src/DataTypes/DataTypeDate32.h +++ b/src/DataTypes/DataTypeDate32.h @@ -13,6 +13,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Date32; } const char * getFamilyName() const override { return family_name; } + const char * getMySQLName() const override { return "date"; } Field getDefault() const override { diff --git a/src/DataTypes/DataTypeDateTime.h b/src/DataTypes/DataTypeDateTime.h index 91a09ff7cb9..c868f92c311 100644 --- a/src/DataTypes/DataTypeDateTime.h +++ b/src/DataTypes/DataTypeDateTime.h @@ -36,6 +36,7 @@ public: static constexpr auto family_name = "DateTime"; const char * getFamilyName() const override { return family_name; } + const char * getMySQLName() const override { return "datetime"; } String doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::DateTime; } diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index aaa99485040..8d317bb9430 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -28,6 +28,7 @@ public: DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info); const char * getFamilyName() const override { return family_name; } + const char * getMySQLName() const override { return "datetime"; } std::string doGetName() const override; TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index 3c3ac2ae4e2..bfed4d4d5a2 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -36,6 +36,29 @@ const char * DataTypeEnum::getFamilyName() const return EnumName::value; } +template +std::string DataTypeEnum::generateMySQLName(const Values & values) +{ + WriteBufferFromOwnString out; + + writeString("enum", out); + writeChar('(', out); + + auto first = true; + for (const auto & name_and_value : values) + { + if (!first) + writeString(", ", out); + + first = false; + + writeQuotedString(name_and_value.first, out); + } + + writeChar(')', out); + + return out.str(); +} template std::string DataTypeEnum::generateName(const Values & values) @@ -67,6 +90,7 @@ template DataTypeEnum::DataTypeEnum(const Values & values_) : EnumValues(values_) , type_name(generateName(this->getValues())) + , my_sql_type_name(generateMySQLName(this->getValues())) { } diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index 2f607fc2aa6..c6e523adf96 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -45,13 +45,16 @@ public: private: std::string type_name; + std::string my_sql_type_name; static std::string generateName(const Values & values); + static std::string generateMySQLName(const Values & values); public: explicit DataTypeEnum(const Values & values_); std::string doGetName() const override { return type_name; } const char * getFamilyName() const override; + const char * getMySQLName() const override { return my_sql_type_name.c_str(); } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeFixedString.h b/src/DataTypes/DataTypeFixedString.h index 8d114121c1a..eb09914ec9c 100644 --- a/src/DataTypes/DataTypeFixedString.h +++ b/src/DataTypes/DataTypeFixedString.h @@ -42,6 +42,7 @@ public: TypeIndex getTypeId() const override { return type_id; } const char * getFamilyName() const override { return "FixedString"; } + const char * getMySQLName() const override { return "text"; } size_t getN() const { diff --git a/src/DataTypes/DataTypeFunction.h b/src/DataTypes/DataTypeFunction.h index 888bcb6a775..f3423796126 100644 --- a/src/DataTypes/DataTypeFunction.h +++ b/src/DataTypes/DataTypeFunction.h @@ -24,6 +24,7 @@ public: std::string doGetName() const override; const char * getFamilyName() const override { return "Function"; } + const char * getMySQLName() const override { return "text"; } TypeIndex getTypeId() const override { return TypeIndex::Function; } const DataTypes & getArgumentTypes() const diff --git a/src/DataTypes/DataTypeIPv4andIPv6.h b/src/DataTypes/DataTypeIPv4andIPv6.h index ad70bdae933..8f7fe79793b 100644 --- a/src/DataTypes/DataTypeIPv4andIPv6.h +++ b/src/DataTypes/DataTypeIPv4andIPv6.h @@ -19,6 +19,8 @@ public: static constexpr auto type_id = TypeToTypeIndex; const char * getFamilyName() const override { return TypeName.data(); } + const char * getMySQLName() const override { return "text"; } + TypeIndex getTypeId() const override { return type_id; } Field getDefault() const override { return IPv4{}; } @@ -59,6 +61,8 @@ public: static constexpr auto type_id = TypeToTypeIndex; const char * getFamilyName() const override { return TypeName.data(); } + const char * getMySQLName() const override { return "text"; } + TypeIndex getTypeId() const override { return type_id; } Field getDefault() const override { return IPv6{}; } diff --git a/src/DataTypes/DataTypeInterval.h b/src/DataTypes/DataTypeInterval.h index 05abe1d9b24..69a56e8aadd 100644 --- a/src/DataTypes/DataTypeInterval.h +++ b/src/DataTypes/DataTypeInterval.h @@ -26,6 +26,7 @@ public: std::string doGetName() const override { return fmt::format("Interval{}", kind.toString()); } const char * getFamilyName() const override { return "Interval"; } + const char * getMySQLName() const override { return "text"; } TypeIndex getTypeId() const override { return TypeIndex::Interval; } bool equals(const IDataType & rhs) const override; diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index d301a0f5443..6fd4344311c 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -22,6 +22,8 @@ public: return "LowCardinality(" + dictionary_type->getName() + ")"; } const char * getFamilyName() const override { return "LowCardinality"; } + const char * getMySQLName() const override { return "text"; } + TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; } MutableColumnPtr createColumn() const override; diff --git a/src/DataTypes/DataTypeMap.h b/src/DataTypes/DataTypeMap.h index 4712f6bbdef..526dc321f44 100644 --- a/src/DataTypes/DataTypeMap.h +++ b/src/DataTypes/DataTypeMap.h @@ -30,6 +30,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Map; } std::string doGetName() const override; const char * getFamilyName() const override { return "Map"; } + const char * getMySQLName() const override { return "json"; } bool canBeInsideNullable() const override { return false; } diff --git a/src/DataTypes/DataTypeNothing.h b/src/DataTypes/DataTypeNothing.h index c7d12388de9..fdef6026603 100644 --- a/src/DataTypes/DataTypeNothing.h +++ b/src/DataTypes/DataTypeNothing.h @@ -16,6 +16,8 @@ public: static constexpr bool is_parametric = false; const char * getFamilyName() const override { return "Nothing"; } + const char * getMySQLName() const override { return "text"; } + TypeIndex getTypeId() const override { return TypeIndex::Nothing; } MutableColumnPtr createColumn() const override; diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index 06d46fb15ed..64b201d32b2 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -16,6 +16,7 @@ public: explicit DataTypeNullable(const DataTypePtr & nested_data_type_); std::string doGetName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } const char * getFamilyName() const override { return "Nullable"; } + const char * getMySQLName() const override { return nested_data_type->getMySQLName(); } TypeIndex getTypeId() const override { return TypeIndex::Nullable; } MutableColumnPtr createColumn() const override; diff --git a/src/DataTypes/DataTypeNumberBase.cpp b/src/DataTypes/DataTypeNumberBase.cpp index f668a4c522e..cd5e73ac4a1 100644 --- a/src/DataTypes/DataTypeNumberBase.cpp +++ b/src/DataTypes/DataTypeNumberBase.cpp @@ -30,6 +30,23 @@ bool DataTypeNumberBase::isValueRepresentedByUnsignedInteger() const return is_integer && is_unsigned_v; } +template +const std::map DataTypeNumberBase::mysqlTypeMap = { + {"UInt8", "tinyint unsigned"}, + {"UInt16", "smallint unsigned"}, + {"UInt32", "mediumint unsigned"}, + {"UInt64", "bigint unsigned"}, + {"UInt128", "bigint unsigned"}, + {"UInt256", "bigint unsigned"}, + {"Int8", "tinyint"}, + {"Int16", "smallint"}, + {"Int32", "int"}, + {"Int64", "bigint"}, + {"Int128", "bigint"}, + {"Int256", "bigint"}, + {"Float32", "float"}, + {"Float64", "double"}, +}; /// Explicit template instantiations - to avoid code bloat in headers. template class DataTypeNumberBase; diff --git a/src/DataTypes/DataTypeNumberBase.h b/src/DataTypes/DataTypeNumberBase.h index 3a5b11c5124..b5c963cf245 100644 --- a/src/DataTypes/DataTypeNumberBase.h +++ b/src/DataTypes/DataTypeNumberBase.h @@ -20,11 +20,14 @@ public: static constexpr bool is_parametric = false; static constexpr auto family_name = TypeName; static constexpr auto type_id = TypeToTypeIndex; + // Create a map from the name of the type to the name of the type in MySQL. + static const std::map mysqlTypeMap; using FieldType = T; using ColumnType = ColumnVector; const char * getFamilyName() const override { return TypeName.data(); } + const char * getMySQLName() const override { return mysqlTypeMap.at(TypeName.data()).c_str(); } TypeIndex getTypeId() const override { return TypeToTypeIndex; } Field getDefault() const override; diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 937a9091371..8a2c36abcd7 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -23,6 +23,7 @@ public: DataTypeObject(const String & schema_format_, bool is_nullable_); const char * getFamilyName() const override { return "Object"; } + const char * getMySQLName() const override { return "json"; } String doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::Object; } diff --git a/src/DataTypes/DataTypeSet.h b/src/DataTypes/DataTypeSet.h index 7ddfeb9fe30..bdad638b5d5 100644 --- a/src/DataTypes/DataTypeSet.h +++ b/src/DataTypes/DataTypeSet.h @@ -15,6 +15,8 @@ class DataTypeSet final : public IDataTypeDummy public: static constexpr bool is_parametric = true; const char * getFamilyName() const override { return "Set"; } + const char * getMySQLName() const override { return "text"; } + TypeIndex getTypeId() const override { return TypeIndex::Set; } bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); } bool isParametric() const override { return true; } diff --git a/src/DataTypes/DataTypeString.h b/src/DataTypes/DataTypeString.h index 5f3bde43a13..3ac739fe68c 100644 --- a/src/DataTypes/DataTypeString.h +++ b/src/DataTypes/DataTypeString.h @@ -21,6 +21,9 @@ public: return "String"; } + // FIXME: string can contain arbitrary bytes, not only UTF-8 sequences + const char * getMySQLName() const override { return "text"; } + TypeIndex getTypeId() const override { return type_id; } MutableColumnPtr createColumn() const override; diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 152f21015f5..d264cc97f60 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -33,6 +33,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Tuple; } std::string doGetName() const override; const char * getFamilyName() const override { return "Tuple"; } + const char * getMySQLName() const override { return "json"; } bool canBeInsideNullable() const override { return false; } bool supportsSparseSerialization() const override { return true; } diff --git a/src/DataTypes/DataTypeUUID.h b/src/DataTypes/DataTypeUUID.h index af9f1f35ca5..4d54db42b45 100644 --- a/src/DataTypes/DataTypeUUID.h +++ b/src/DataTypes/DataTypeUUID.h @@ -18,6 +18,8 @@ public: static constexpr auto type_id = TypeIndex::UUID; const char * getFamilyName() const override { return "UUID"; } + const char * getMySQLName() const override { return "char"; } + TypeIndex getTypeId() const override { return type_id; } Field getDefault() const override; diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index 583f7ea804a..5c9405cb060 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -37,8 +37,11 @@ public: using Base::Base; static constexpr auto family_name = "Decimal"; + static constexpr auto mysql_name = "decimal"; const char * getFamilyName() const override { return family_name; } + const char * getMySQLName() const override { return mysql_name; } + std::string doGetName() const override; TypeIndex getTypeId() const override { return TypeToTypeIndex; } bool canBePromoted() const override { return true; } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 7cc18fea00c..2bed18897ce 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -71,10 +71,19 @@ public: return doGetName(); } + /// MySQL equivalent Name of data type (examples: UInt64, Array(String)). + String getMySQLTypeName() const + { + if (custom_name) + return custom_name->getName(); + else + return doGetMySQLName(); + } DataTypePtr getPtr() const { return shared_from_this(); } /// Name of data type family (example: FixedString, Array). virtual const char * getFamilyName() const = 0; + virtual const char * getMySQLName() const = 0; /// Data type id. It's used for runtime type checks. virtual TypeIndex getTypeId() const = 0; @@ -126,6 +135,7 @@ public: protected: virtual String doGetName() const { return getFamilyName(); } + virtual String doGetMySQLName() const { return getMySQLName(); } virtual SerializationPtr doGetDefaultSerialization() const = 0; public: diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 18e7d269795..f391a392dbb 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -74,6 +74,7 @@ public: : ISource(header_) , columns_mask(std::move(columns_mask_)), max_block_size(max_block_size_) , databases(std::move(databases_)), tables(std::move(tables_)), storages(std::move(storages_)) + , clientInfo(context->getClientInfo()) , total_tables(tables->size()), access(context->getAccess()) , query_id(context->getCurrentQueryId()), lock_acquire_timeout(context->getSettingsRef().lock_acquire_timeout) { @@ -129,6 +130,17 @@ protected: bool check_access_for_columns = check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name); + auto get_type_name = [this](const IDataType& type) -> std::string + { + if (clientInfo.interface == DB::ClientInfo::Interface::MYSQL) + { + return type.getMySQLTypeName(); + } + else + { + return type.getName(); + } + }; size_t position = 0; for (const auto & column : columns) { @@ -146,7 +158,7 @@ protected: if (columns_mask[src_index++]) res_columns[res_index++]->insert(column.name); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(column.type->getName()); + res_columns[res_index++]->insert(get_type_name(*column.type)); if (columns_mask[src_index++]) res_columns[res_index++]->insert(position); @@ -281,6 +293,7 @@ private: ColumnPtr databases; ColumnPtr tables; Storages storages; + ClientInfo clientInfo; size_t db_table_num = 0; size_t total_tables; std::shared_ptr access; diff --git a/tests/queries/0_stateless/02740_show_columns_mysql_compatibility.sh b/tests/queries/0_stateless/02740_show_columns_mysql_compatibility.sh new file mode 100755 index 00000000000..7f828d35679 --- /dev/null +++ b/tests/queries/0_stateless/02740_show_columns_mysql_compatibility.sh @@ -0,0 +1,116 @@ +#!/bin/bash + +# This script tests the MySQL compatibility of the SHOW COLUMNS command in ClickHouse +USER="default" +PASSWORD="" +HOST="127.0.0.1" +PORT=9004 + +# First run the clickhouse test to create the ClickHouse Tables + +echo "Drop tables if they exist" +${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS tab" +${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde" +${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde.tab" + +echo "Create tab table " +${CLICKHOUSE_LOCAL} --query " + CREATE TABLE tab + ( + uint64 UInt64, + int32 Nullable(Int32), + float32 Float32, + float64 Float64, + decimal_value Decimal(10, 2), + boolean_value UInt8, -- Use 0 for false, 1 for true + string_value String, + fixed_string_value FixedString(10), + date_value Date, + date32_value Date32, + datetime_value DateTime, + datetime64_value DateTime64(3), + json_value String, -- Store JSON as a string + uuid_value UUID, + enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), + low_cardinality LowCardinality(String), + array_value Array(Int32), + map_value Map(String, Int32), + tuple_value Tuple(Int32, String), + nullable_value Nullable(Int32), + ipv4_value IPv4, + ipv6_value IPv6, + nested Nested + ( + nested_int Int32, + nested_string String + ) + ) ENGINE = MergeTree + ORDER BY uint64; + " + + +echo "Create pseudo-random database name" +${CLICKHOUSE_LOCAL} --query "CREATE DATABASE database_123456789abcde;" + +echo "Create tab duplicate table" +${CLICKHOUSE_LOCAL} --query " + CREATE TABLE database_123456789abcde.tab + ( + uint64 UInt64, + int32 Nullable(Int32), + float32 Float32, + float64 Float64, + decimal_value Decimal(10, 2), + boolean_value UInt8, -- Use 0 for false, 1 for true + string_value String, + fixed_string_value FixedString(10), + date_value Date, + date32_value Date32, + datetime_value DateTime, + datetime64_value DateTime64(3), + json_value String, -- Store JSON as a string + uuid_value UUID, + enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), + low_cardinality LowCardinality(String), + array_value Array(Int32), + map_value Map(String, Int32), + tuple_value Tuple(Int32, String), + nullable_value Nullable(Int32), + ipv4_value IPv4, + ipv6_value IPv6, + nested Nested + ( + nested_int Int32, + nested_string String + ) + ) ENGINE = MergeTree + ORDER BY uint64; + " + +# Write sql to temp file +TEMP_FILE=$(mktemp) + +cat < $TEMP_FILE +SHOW COLUMNS FROM tab; +SHOW EXTENDED COLUMNS FROM tab; +SHOW FULL COLUMNS FROM tab; +SHOW COLUMNS FROM tab LIKE '%int%'; +SHOW COLUMNS FROM tab NOT LIKE '%int%'; +SHOW COLUMNS FROM tab ILIKE '%INT%'; +SHOW COLUMNS FROM tab NOT ILIKE '%INT%'; +SHOW COLUMNS FROM tab WHERE field LIKE '%int%'; +SHOW COLUMNS FROM tab LIMIT 1; +SHOW COLUMNS FROM tab; +SHOW COLUMNS FROM tab FROM database_123456789abcde; +SHOW COLUMNS FROM database_123456789abcde.tab; +DROP DATABASE database_123456789abcde; +DROP TABLE tab; +EOT + +# Now run the MySQL test script on the ClickHouse DB +echo "Run MySQL test" +mysql --user="$USER" --password="$PASSWORD" --host="$HOST" --port="$PORT" < $TEMP_FILE + +# Clean up the temp file +rm $TEMP_FILE + From bd5a1ae2b97b66361e5b958811a6055f8f5cd2ae Mon Sep 17 00:00:00 2001 From: tpanetti Date: Tue, 30 May 2023 13:32:33 -0700 Subject: [PATCH 0834/2223] Revert "Change SHOW COLUMNS query to display MySQL types in MySQL Compatibility mode" This reverts commit ddbad79c5e67518acebbacaad5be0cad3967ac67. --- .../InterpreterShowColumnsQuery.cpp | 76 +------ .../InterpreterShowColumnsQuery.h | 1 - ...show_columns_mysql_compatibility.reference | 213 ------------------ .../02726_show_columns_mysql_compatibility.sh | 115 ---------- 4 files changed, 3 insertions(+), 402 deletions(-) delete mode 100644 tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference delete mode 100755 tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 0ad93e37b58..c86d3c753c4 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -42,11 +42,9 @@ SELECT if (default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra )"; - rewritten_query += getMySQLQuery(); - } - else { - rewritten_query += "SELECT name AS field, type AS type, startsWith(type, 'Nullable') AS null, trim(concatWithSeparator(' ', if(is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, if(default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra "; - } + // TODO Interpret query.extended. It is supposed to show internal/virtual columns. Need to fetch virtual column names, see + // IStorage::getVirtuals(). We can't easily do that via SQL. + if (query.full) { /// "Full" mode is mostly for MySQL compat @@ -90,74 +88,6 @@ WHERE return rewritten_query; } -String InterpreterShowColumnsQuery::getMySQLQuery() -{ - String mysql_specific_query; - - mysql_specific_query = R"(SELECT name AS field, - CASE - WHEN startsWith(type, 'Nullable') THEN - CASE - WHEN substring(type, 10, length(type) - 10) IN ('UInt8', 'Int8') THEN 'tinyint' - WHEN substring(type, 10, length(type) - 10) IN ('UInt16', 'Int16') THEN 'smallint' - WHEN substring(type, 10, length(type) - 10) IN ('UInt32', 'Int32') THEN 'int' - WHEN substring(type, 10, length(type) - 10) IN ('UInt64', 'Int64', 'UInt128', 'Int128', 'UInt256', 'Int256') THEN 'bigint' - WHEN substring(type, 10, length(type) - 10) = 'Float32' THEN 'float' - WHEN substring(type, 10, length(type) - 10) = 'Float64' THEN 'double' - WHEN substring(type, 10, length(type) - 10) LIKE 'Decimal%' THEN 'decimal' - WHEN substring(type, 10, length(type) - 10) = 'Boolean' THEN 'tinyint' - WHEN substring(type, 10, length(type) - 10) = 'String' THEN 'text' - WHEN substring(type, 10, length(type) - 10) LIKE 'FixedString%' THEN 'text' - WHEN substring(type, 10, length(type) - 10) LIKE 'Date%' THEN 'date' - WHEN substring(type, 10, length(type) - 10) LIKE 'DateTime%' THEN 'datetime' - WHEN substring(type, 10, length(type) - 10) = 'JSON' THEN 'json' - WHEN substring(type, 10, length(type) - 10) = 'UUID' THEN 'binary' - WHEN substring(type, 10, length(type) - 10) LIKE 'Enum%' THEN 'enum' - WHEN substring(type, 10, length(type) - 10) LIKE 'LowCardinality%' THEN 'text' - WHEN substring(type, 10, length(type) - 10) LIKE 'Array%' THEN 'json' - WHEN substring(type, 10, length(type) - 10) LIKE 'Map%' THEN 'json' - WHEN substring(type, 10, length(type) - 10) IN ('SimpleAggregateFunction', 'AggregateFunction') THEN 'text' - WHEN substring(type, 10, length(type) - 10) = 'Nested' THEN 'json' - WHEN substring(type, 10, length(type) - 10) LIKE 'Tuple%' THEN 'json' - WHEN substring(type, 10, length(type) - 10) LIKE 'IPv%' THEN 'text' - WHEN substring(type, 10, length(type) - 10) IN ('Expression', 'Set', 'Nothing', 'Interval') THEN 'text' - ELSE substring(type, 10, length(type) - 10) - END - ELSE - CASE - WHEN type IN ('UInt8', 'Int8') THEN 'tinyint' - WHEN type IN ('UInt16', 'Int16') THEN 'smallint' - WHEN type IN ('UInt32', 'Int32') THEN 'int' - WHEN type IN ('UInt64', 'Int64', 'UInt128', 'Int128', 'UInt256', 'Int256') THEN 'bigint' - WHEN type = 'Float32' THEN 'float' - WHEN type = 'Float64' THEN 'double' - WHEN type LIKE 'Decimal%' THEN 'decimal' - WHEN type = 'Boolean' THEN 'tinyint' - WHEN type = 'String' THEN 'text' - WHEN type LIKE 'FixedString%' THEN 'text' - WHEN type LIKE 'Date%' THEN 'date' - WHEN type LIKE 'DateTime%' THEN 'datetime' - WHEN type = 'JSON' THEN 'json' - WHEN type = 'UUID' THEN 'binary' - WHEN type LIKE 'Enum%' THEN 'enum' - WHEN type LIKE 'LowCardinality%' THEN 'text' - WHEN type LIKE 'Array%' THEN 'json' - WHEN type LIKE 'Map%' THEN 'json' - WHEN type IN ('SimpleAggregateFunction', 'AggregateFunction') THEN 'text' - WHEN type = 'Nested' THEN 'json' - WHEN type LIKE 'Tuple%' THEN 'json' - WHEN type LIKE 'IPv%' THEN 'text' - WHEN type IN ('Expression', 'Set', 'Nothing', 'Interval') THEN 'text' - ELSE type - END - END AS type, - startsWith(type, 'Nullable') AS null, - trim(concatWithSeparator(' ', if(is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, - if(default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, - '' AS extra )"; - - return mysql_specific_query.str(); -} BlockIO InterpreterShowColumnsQuery::execute() { diff --git a/src/Interpreters/InterpreterShowColumnsQuery.h b/src/Interpreters/InterpreterShowColumnsQuery.h index b843a163978..ee6dcabd97b 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.h +++ b/src/Interpreters/InterpreterShowColumnsQuery.h @@ -26,7 +26,6 @@ private: ASTPtr query_ptr; String getRewrittenQuery(); - String getMySQLQuery(); }; diff --git a/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference deleted file mode 100644 index c9ad94a34c4..00000000000 --- a/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.reference +++ /dev/null @@ -1,213 +0,0 @@ -Drop tables if they exist -Create tab table -Create pseudo-random database name -Create tab duplicate table -Run MySQL test -field type null key default extra -array_value json 0 NULL -boolean_value tinyint 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value date 0 NULL -datetime_value date 0 NULL -decimal_value decimal 0 NULL -enum_value enum 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -int32 int 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value text 0 NULL -low_cardinality text 0 NULL -map_value json 0 NULL -nested.nested_int json 0 NULL -nested.nested_string json 0 NULL -nullable_value int 0 NULL -string_value text 0 NULL -tuple_value json 0 NULL -uint64 bigint 0 PRI SOR NULL -uuid_value binary 0 NULL -field type null key default extra -array_value json 0 NULL -boolean_value tinyint 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value date 0 NULL -datetime_value date 0 NULL -decimal_value decimal 0 NULL -enum_value enum 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -int32 int 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value text 0 NULL -low_cardinality text 0 NULL -map_value json 0 NULL -nested.nested_int json 0 NULL -nested.nested_string json 0 NULL -nullable_value int 0 NULL -string_value text 0 NULL -tuple_value json 0 NULL -uint64 bigint 0 PRI SOR NULL -uuid_value binary 0 NULL -field type null key default extra collation comment privileges -array_value json 0 NULL NULL -boolean_value tinyint 0 NULL NULL -date32_value date 0 NULL NULL -date_value date 0 NULL NULL -datetime64_value date 0 NULL NULL -datetime_value date 0 NULL NULL -decimal_value decimal 0 NULL NULL -enum_value enum 0 NULL NULL -fixed_string_value text 0 NULL NULL -float32 float 0 NULL NULL -float64 double 0 NULL NULL -int32 int 0 NULL NULL -ipv4_value text 0 NULL NULL -ipv6_value text 0 NULL NULL -json_value text 0 NULL NULL -low_cardinality text 0 NULL NULL -map_value json 0 NULL NULL -nested.nested_int json 0 NULL NULL -nested.nested_string json 0 NULL NULL -nullable_value int 0 NULL NULL -string_value text 0 NULL NULL -tuple_value json 0 NULL NULL -uint64 bigint 0 PRI SOR NULL NULL -uuid_value binary 0 NULL NULL -field type null key default extra -int32 int 0 NULL -nested.nested_int json 0 NULL -uint64 bigint 0 PRI SOR NULL -field type null key default extra -array_value json 0 NULL -boolean_value tinyint 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value date 0 NULL -datetime_value date 0 NULL -decimal_value decimal 0 NULL -enum_value enum 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value text 0 NULL -low_cardinality text 0 NULL -map_value json 0 NULL -nested.nested_string json 0 NULL -nullable_value int 0 NULL -string_value text 0 NULL -tuple_value json 0 NULL -uuid_value binary 0 NULL -field type null key default extra -int32 int 0 NULL -nested.nested_int json 0 NULL -uint64 bigint 0 PRI SOR NULL -field type null key default extra -array_value json 0 NULL -boolean_value tinyint 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value date 0 NULL -datetime_value date 0 NULL -decimal_value decimal 0 NULL -enum_value enum 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value text 0 NULL -low_cardinality text 0 NULL -map_value json 0 NULL -nested.nested_string json 0 NULL -nullable_value int 0 NULL -string_value text 0 NULL -tuple_value json 0 NULL -uuid_value binary 0 NULL -field type null key default extra -int32 int 0 NULL -nested.nested_int json 0 NULL -uint64 bigint 0 PRI SOR NULL -field type null key default extra -array_value json 0 NULL -field type null key default extra -array_value json 0 NULL -boolean_value tinyint 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value date 0 NULL -datetime_value date 0 NULL -decimal_value decimal 0 NULL -enum_value enum 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -int32 int 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value text 0 NULL -low_cardinality text 0 NULL -map_value json 0 NULL -nested.nested_int json 0 NULL -nested.nested_string json 0 NULL -nullable_value int 0 NULL -string_value text 0 NULL -tuple_value json 0 NULL -uint64 bigint 0 PRI SOR NULL -uuid_value binary 0 NULL -field type null key default extra -array_value json 0 NULL -boolean_value tinyint 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value date 0 NULL -datetime_value date 0 NULL -decimal_value decimal 0 NULL -enum_value enum 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -int32 int 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value text 0 NULL -low_cardinality text 0 NULL -map_value json 0 NULL -nested.nested_int json 0 NULL -nested.nested_string json 0 NULL -nullable_value int 0 NULL -string_value text 0 NULL -tuple_value json 0 NULL -uint64 bigint 0 PRI SOR NULL -uuid_value binary 0 NULL -field type null key default extra -array_value json 0 NULL -boolean_value tinyint 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value date 0 NULL -datetime_value date 0 NULL -decimal_value decimal 0 NULL -enum_value enum 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -int32 int 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value text 0 NULL -low_cardinality text 0 NULL -map_value json 0 NULL -nested.nested_int json 0 NULL -nested.nested_string json 0 NULL -nullable_value int 0 NULL -string_value text 0 NULL -tuple_value json 0 NULL -uint64 bigint 0 PRI SOR NULL -uuid_value binary 0 NULL diff --git a/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh b/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh deleted file mode 100755 index 5324496edd3..00000000000 --- a/tests/queries/0_stateless/02726_show_columns_mysql_compatibility.sh +++ /dev/null @@ -1,115 +0,0 @@ -#!/bin/bash - -# This script tests the MySQL compatibility of the SHOW COLUMNS command in ClickHouse -USER="default" -PASSWORD="" -HOST="127.0.0.1" -PORT=9004 - -# First run the clickhouse test to create the ClickHouse Tables - -echo "Drop tables if they exist" -${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS tab" -${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde" -${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde.tab" - -echo "Create tab table " -${CLICKHOUSE_LOCAL} --query " - CREATE TABLE tab - ( - uint64 UInt64, - int32 Nullable(Int32), - float32 Float32, - float64 Float64, - decimal_value Decimal(10, 2), - boolean_value UInt8, -- Use 0 for false, 1 for true - string_value String, - fixed_string_value FixedString(10), - date_value Date, - date32_value Date32, - datetime_value DateTime, - datetime64_value DateTime64(3), - json_value String, -- Store JSON as a string - uuid_value UUID, - enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), - low_cardinality LowCardinality(String), - array_value Array(Int32), - map_value Map(String, Int32), - tuple_value Tuple(Int32, String), - nullable_value Nullable(Int32), - ipv4_value IPv4, - ipv6_value IPv6, - nested Nested - ( - nested_int Int32, - nested_string String - ) - ) ENGINE = MergeTree - ORDER BY uint64; - " - - -echo "Create pseudo-random database name" -${CLICKHOUSE_LOCAL} --query "CREATE DATABASE database_123456789abcde;" - -echo "Create tab duplicate table" -${CLICKHOUSE_LOCAL} --query " - CREATE TABLE database_123456789abcde.tab - ( - uint64 UInt64, - int32 Nullable(Int32), - float32 Float32, - float64 Float64, - decimal_value Decimal(10, 2), - boolean_value UInt8, -- Use 0 for false, 1 for true - string_value String, - fixed_string_value FixedString(10), - date_value Date, - date32_value Date32, - datetime_value DateTime, - datetime64_value DateTime64(3), - json_value String, -- Store JSON as a string - uuid_value UUID, - enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), - low_cardinality LowCardinality(String), - array_value Array(Int32), - map_value Map(String, Int32), - tuple_value Tuple(Int32, String), - nullable_value Nullable(Int32), - ipv4_value IPv4, - ipv6_value IPv6, - nested Nested - ( - nested_int Int32, - nested_string String - ) - ) ENGINE = MergeTree - ORDER BY uint64; - " - -# Write sql to temp file -TEMP_FILE=$(mktemp) - -cat < $TEMP_FILE -SHOW COLUMNS FROM tab; -SHOW EXTENDED COLUMNS FROM tab; -SHOW FULL COLUMNS FROM tab; -SHOW COLUMNS FROM tab LIKE '%int%'; -SHOW COLUMNS FROM tab NOT LIKE '%int%'; -SHOW COLUMNS FROM tab ILIKE '%INT%'; -SHOW COLUMNS FROM tab NOT ILIKE '%INT%'; -SHOW COLUMNS FROM tab WHERE field LIKE '%int%'; -SHOW COLUMNS FROM tab LIMIT 1; -SHOW COLUMNS FROM tab; -SHOW COLUMNS FROM tab FROM database_123456789abcde; -SHOW COLUMNS FROM database_123456789abcde.tab; -DROP DATABASE database_123456789abcde; -DROP TABLE tab; -EOT - -# Now run the MySQL test script on the ClickHouse DB -echo "Run MySQL test" -mysql --user="$USER" --password="$PASSWORD" --host="$HOST" --port="$PORT" < $TEMP_FILE - -# Clean up the temp file -rm $TEMP_FILE From a2c7c843464be20f48ecde46306b00992b27ce4c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 30 May 2023 23:37:27 +0200 Subject: [PATCH 0835/2223] Update 02770_async_buffer_ignore.sh --- tests/queries/0_stateless/02770_async_buffer_ignore.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh index 779d5b6729e..ca8909a8d0d 100755 --- a/tests/queries/0_stateless/02770_async_buffer_ignore.sh +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -4,7 +4,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test_optimize_exception" ${CLICKHOUSE_CLIENT} -nm --query " DROP TABLE IF EXISTS test_s3; From db3a05091185284cdc60685c5ed077845f710303 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 31 May 2023 00:22:12 +0200 Subject: [PATCH 0836/2223] Seems like buildx needs explicit install since some time --- tests/ci/worker/ubuntu_ami_for_ci.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/ubuntu_ami_for_ci.sh b/tests/ci/worker/ubuntu_ami_for_ci.sh index 22e5ada2932..2bb8f01535f 100644 --- a/tests/ci/worker/ubuntu_ami_for_ci.sh +++ b/tests/ci/worker/ubuntu_ami_for_ci.sh @@ -56,7 +56,7 @@ echo "deb [arch=$(deb_arch) signed-by=/usr/share/keyrings/docker-archive-keyring apt-get update -apt-get install --yes --no-install-recommends docker-ce docker-ce-cli containerd.io +apt-get install --yes --no-install-recommends docker-ce docker-buildx-plugin docker-ce-cli containerd.io usermod -aG docker ubuntu From dbd3766f5f662e3338c30fd5408e4818598b9660 Mon Sep 17 00:00:00 2001 From: pufit Date: Sat, 27 May 2023 01:33:07 -0400 Subject: [PATCH 0837/2223] Specify roles in users.xml --- src/Access/UsersConfigAccessStorage.cpp | 159 +++++++++++++----- src/Parsers/Access/ParserGrantQuery.h | 2 +- .../configs/another_user.xml | 25 +++ .../test_user_grants_from_config/test.py | 18 +- 4 files changed, 163 insertions(+), 41 deletions(-) diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index df0e4584709..187258d0fcd 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -13,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -52,11 +54,64 @@ namespace UUID generateID(const IAccessEntity & entity) { return generateID(entity.getType(), entity.getName()); } + template + void parseGrant(T & entity, const String & string_query, const std::unordered_set & allowed_role_ids) + { + ParserGrantQuery parser; + parser.setParseWithoutGrantees(); + + String error_message; + const char * pos = string_query.data(); + auto ast = tryParseQuery(parser, pos, pos + string_query.size(), error_message, false, "", false, 0, 0); + + if (!ast) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse grant query. Error: {}", error_message); + + auto & query = ast->as(); + + if (query.roles && query.is_revoke) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Roles can't be revoked in config file"); + + if (!query.cluster.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can't grant on cluster using config file"); + + if (query.grantees) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "You can't specify grantees in query using config file"); + + for (auto & element : query.access_rights_elements) + { + if (query.is_revoke) + entity.access.revoke(element); + else + entity.access.grant(element); + } + + if (query.roles) + { + std::vector roles_to_grant; + roles_to_grant.reserve(query.roles->size()); + + for (const auto & role_name : query.roles->names) + { + auto role_id = generateID(AccessEntityType::ROLE, role_name); + if (!allowed_role_ids.contains(role_id)) + throw Exception(ErrorCodes::THERE_IS_NO_PROFILE, "Role {} was not found", role_name); + + roles_to_grant.push_back(role_id); + } + + if (query.admin_option) + entity.granted_roles.grantWithAdminOption(roles_to_grant); + else + entity.granted_roles.grant(roles_to_grant); + } + } UserPtr parseUser( const Poco::Util::AbstractConfiguration & config, const String & user_name, const std::unordered_set & allowed_profile_ids, + const std::unordered_set & allowed_role_ids, bool allow_no_password, bool allow_plaintext_password) { @@ -241,37 +296,8 @@ namespace if (grant_queries) { - ParserGrantQuery parser; - parser.parseWithoutGrantees(); - for (const auto & string_query : *grant_queries) - { - String error_message; - const char * pos = string_query.data(); - auto ast = tryParseQuery(parser, pos, pos + string_query.size(), error_message, false, "", false, 0, 0); - - if (!ast) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse grant query. Error: {}", error_message); - - auto & query = ast->as(); - - if (query.roles) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Roles can't be granted in config file"); - - if (!query.cluster.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can't grant on cluster using config file"); - - if (query.grantees) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "You can't specify grantees in query using config file"); - - for (auto & element : query.access_rights_elements) - { - if (query.is_revoke) - user->access.revoke(element); - else - user->access.grant(element); - } - } + parseGrant(*user, string_query, allowed_role_ids); } else { @@ -321,6 +347,7 @@ namespace std::vector parseUsers( const Poco::Util::AbstractConfiguration & config, const std::unordered_set & allowed_profile_ids, + const std::unordered_set & allowed_role_ids, bool allow_no_password, bool allow_plaintext_password) { @@ -333,7 +360,7 @@ namespace { try { - users.push_back(parseUser(config, user_name, allowed_profile_ids, allow_no_password, allow_plaintext_password)); + users.push_back(parseUser(config, user_name, allowed_profile_ids, allowed_role_ids, allow_no_password, allow_plaintext_password)); } catch (Exception & e) { @@ -345,6 +372,55 @@ namespace return users; } + RolePtr parseRole( + const Poco::Util::AbstractConfiguration & config, + const String & role_name, + const std::unordered_set & allowed_role_ids) + { + auto role = std::make_shared(); + role->setName(role_name); + String role_config = "roles." + role_name; + + const auto grants_config = role_config + ".grants"; + if (config.has(grants_config)) + { + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(grants_config, keys); + for (const auto & key : keys) + { + const auto query = config.getString(grants_config + "." + key); + parseGrant(*role, query, allowed_role_ids); + } + } + + return role; + } + + std::vector parseRoles( + const Poco::Util::AbstractConfiguration & config, + const std::unordered_set & allowed_role_ids) + { + Poco::Util::AbstractConfiguration::Keys role_names; + config.keys("roles", role_names); + + std::vector roles; + roles.reserve(role_names.size()); + for (const auto & role_name : role_names) + { + try + { + roles.push_back(parseRole(config, role_name, allowed_role_ids)); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while parsing roles '{}' in users configuration file", role_name)); + throw; + } + } + + return roles; + } + QuotaPtr parseQuota(const Poco::Util::AbstractConfiguration & config, const String & quota_name, const std::vector & user_ids) { @@ -635,14 +711,16 @@ namespace return profiles; } - - std::unordered_set getAllowedSettingsProfileIDs(const Poco::Util::AbstractConfiguration & config) + std::unordered_set getAllowedIDs( + const Poco::Util::AbstractConfiguration & config, + const String & configuration_key, + const AccessEntityType type) { - Poco::Util::AbstractConfiguration::Keys profile_names; - config.keys("profiles", profile_names); + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(configuration_key, keys); std::unordered_set ids; - for (const auto & profile_name : profile_names) - ids.emplace(generateID(AccessEntityType::SETTINGS_PROFILE, profile_name)); + for (const auto & key : keys) + ids.emplace(generateID(type, key)); return ids; } } @@ -693,12 +771,13 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu { try { - auto allowed_profile_ids = getAllowedSettingsProfileIDs(config); + auto allowed_profile_ids = getAllowedIDs(config, "profiles", AccessEntityType::SETTINGS_PROFILE); + auto allowed_role_ids = getAllowedIDs(config, "roles", AccessEntityType::ROLE); bool no_password_allowed = access_control.isNoPasswordAllowed(); bool plaintext_password_allowed = access_control.isPlaintextPasswordAllowed(); std::vector> all_entities; - for (const auto & entity : parseUsers(config, allowed_profile_ids, no_password_allowed, plaintext_password_allowed)) + for (const auto & entity : parseUsers(config, allowed_profile_ids, allowed_role_ids, no_password_allowed, plaintext_password_allowed)) all_entities.emplace_back(generateID(*entity), entity); for (const auto & entity : parseQuotas(config)) all_entities.emplace_back(generateID(*entity), entity); @@ -706,6 +785,8 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu all_entities.emplace_back(generateID(*entity), entity); for (const auto & entity : parseSettingsProfiles(config, allowed_profile_ids, access_control)) all_entities.emplace_back(generateID(*entity), entity); + for (const auto & entity : parseRoles(config, allowed_role_ids)) + all_entities.emplace_back(generateID(*entity), entity); memory_storage.setAll(all_entities); } catch (Exception & e) diff --git a/src/Parsers/Access/ParserGrantQuery.h b/src/Parsers/Access/ParserGrantQuery.h index 58c2be433d5..0ecfef916f5 100644 --- a/src/Parsers/Access/ParserGrantQuery.h +++ b/src/Parsers/Access/ParserGrantQuery.h @@ -14,7 +14,7 @@ class ParserGrantQuery : public IParserBase public: ParserGrantQuery & useAttachMode(bool attach_mode_ = true) { attach_mode = attach_mode_; return *this; } - ParserGrantQuery & parseWithoutGrantees(bool allow_no_grantees_ = true) { allow_no_grantees = allow_no_grantees_; return *this; } + ParserGrantQuery & setParseWithoutGrantees(bool allow_no_grantees_ = true) { allow_no_grantees = allow_no_grantees_; return *this; } protected: const char * getName() const override { return "GRANT or REVOKE query"; } diff --git a/tests/integration/test_user_grants_from_config/configs/another_user.xml b/tests/integration/test_user_grants_from_config/configs/another_user.xml index 16c026e81d0..0b0b2473142 100644 --- a/tests/integration/test_user_grants_from_config/configs/another_user.xml +++ b/tests/integration/test_user_grants_from_config/configs/another_user.xml @@ -14,5 +14,30 @@ REVOKE CREATE ON system.* + + + + ::/0 + + default + default + + GRANT admin_role + + + + + + GRANT SHOW ON *.* + REVOKE SHOW ON system.* + GRANT CREATE ON *.* WITH GRANT OPTION + + + + + GRANT ALL ON *.* WITH GRANT OPTION + + + diff --git a/tests/integration/test_user_grants_from_config/test.py b/tests/integration/test_user_grants_from_config/test.py index a4d5c0c904b..d2bd0b0facd 100644 --- a/tests/integration/test_user_grants_from_config/test.py +++ b/tests/integration/test_user_grants_from_config/test.py @@ -42,7 +42,7 @@ def test_allow_read_from_system_tables(): ) -def test_grants_from_config(): +def test_user_grants_from_config(): assert node.query("SHOW GRANTS FOR another") == TSV( [ "GRANT SHOW ON *.* TO another", @@ -51,3 +51,19 @@ def test_grants_from_config(): "REVOKE CREATE DATABASE, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY ON system.* FROM another", ] ) + + assert node.query("SHOW GRANTS FOR admin_user") == TSV( + [ + "GRANT admin_role TO admin_user", + ] + ) + + +def test_role_grants_from_config(): + assert node.query("SHOW GRANTS FOR test_role") == TSV( + [ + "GRANT SHOW ON *.* TO test_role", + "GRANT CREATE ON *.* TO test_role WITH GRANT OPTION", + "REVOKE SHOW ON system.* FROM test_role", + ] + ) From a01e8644c5cad786cc9b1a34a2e1e1faf93d9399 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Tue, 30 May 2023 21:09:56 -0400 Subject: [PATCH 0838/2223] Settings update (#50350) * sync system.server_settings with doc * typos * update DNS cache setting * add server settings from system tbl * catch up * catch up * Trigger Build --------- Co-authored-by: Nikita Mikhaylov --- .../settings.md | 1061 ++++++++++------- 1 file changed, 631 insertions(+), 430 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 7ebf387f2ed..82be1c10dcc 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -7,6 +7,600 @@ description: This section contains descriptions of server settings that cannot b # Server Settings +## allow_use_jemalloc_memory + +Allows to use jemalloc memory. + +Type: Bool + +Default: 1 + + +## asynchronous_heavy_metrics_update_period_s + +Period in seconds for updating asynchronous metrics. + +Type: UInt32 + +Default: 120 + + +## asynchronous_metrics_update_period_s + +Period in seconds for updating asynchronous metrics. + +Type: UInt32 + +Default: 1 + + +## background_buffer_flush_schedule_pool_size + +The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in the background. + +Type: UInt64 + +Default: 16 + + +## background_common_pool_size + +The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for \*MergeTree-engine tables in a background. + +Type: UInt64 + +Default: 8 + +## background_distributed_schedule_pool_size + +The maximum number of threads that will be used for executing distributed sends. + +Type: UInt64 + +Default: 16 + + +## background_fetches_pool_size + +The maximum number of threads that will be used for fetching data parts from another replica for \*MergeTree-engine tables in a background. + +Type: UInt64 + +Default: 8 + +## background_merges_mutations_concurrency_ratio + +Sets a ratio between the number of threads and the number of background merges and mutations that can be executed concurrently. For example, if the ratio equals to 2 and +`background_pool_size` is set to 16 then ClickHouse can execute 32 background merges concurrently. This is possible, because background operations could be suspended and postponed. This is needed to give small merges more execution priority. You can only increase this ratio at runtime. To lower it you have to restart the server. +The same as for `background_pool_size` setting `background_merges_mutations_concurrency_ratio` could be applied from the `default` profile for backward compatibility. + +Type: Float + +Default: 2 + +## background_merges_mutations_scheduling_policy + +The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. + +## background_merges_mutations_scheduling_policy + +Algorithm used to select next merge or mutation to be executed by background thread pool. Policy may be changed at runtime without server restart. +Could be applied from the `default` profile for backward compatibility. + +Possible values: + +- "round_robin" — Every concurrent merge and mutation is executed in round-robin order to ensure starvation-free operation. Smaller merges are completed faster than bigger ones just because they have fewer blocks to merge. +- "shortest_task_first" — Always execute smaller merge or mutation. Merges and mutations are assigned priorities based on their resulting size. Merges with smaller sizes are strictly preferred over bigger ones. This policy ensures the fastest possible merge of small parts but can lead to indefinite starvation of big merges in partitions heavily overloaded by INSERTs. + +Type: String + +Default: round_robin + +## background_message_broker_schedule_pool_size + +The maximum number of threads that will be used for executing background operations for message streaming. + +Type: UInt64 + +Default: 16 + +## background_move_pool_size + +The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background. + +Type: UInt64 + +Default: 8 + +## background_pool_size + +Sets the number of threads performing background merges and mutations for tables with MergeTree engines. You can only increase the number of threads at runtime. To lower the number of threads you have to restart the server. By adjusting this setting, you manage CPU and disk load. Smaller pool size utilizes less CPU and disk resources, but background processes advance slower which might eventually impact query performance. + +Before changing it, please also take a look at related MergeTree settings, such as `number_of_free_entries_in_pool_to_lower_max_size_of_merge` and `number_of_free_entries_in_pool_to_execute_mutation`. + +Type: UInt64 + +Default: 16 + +## background_schedule_pool_size + +The maximum number of threads that will be used for constantly executing some lightweight periodic operations for replicated tables, Kafka streaming, and DNS cache updates. + +Type: UInt64 + +Default: 128 + +## backup_threads + +The maximum number of threads to execute BACKUP requests. + +Type: UInt64 + +Default: 16 + + +## backups_io_thread_pool_queue_size + +The maximum number of jobs that can be scheduled on the Backups IO Thread pool. It is recommended to keep this queue unlimited (0) due to the current S3 backup logic. + +Type: UInt64 + +Default: 0 + +## cache_size_to_ram_max_ratio + +Set cache size to RAM max ratio. Allows lowering the cache size on low-memory systems. + +Type: Double + +Default: 0.5 + +## concurrent_threads_soft_limit_num + +The maximum number of query processing threads, excluding threads for retrieving data from remote servers, allowed to run all queries. This is not a hard limit. In case if the limit is reached the query will still get at least one thread to run. Query can upscale to desired number of threads during execution if more threads become available. + +Zero means Unlimited. + +Type: UInt64 + +Default: 0 + +## concurrent_threads_soft_limit_ratio_to_cores + +Same as concurrent_threads_soft_limit_num, but with ratio to cores. + +Type: UInt64 + +Default: 0 + + +## default_database + +Default database name. + +Type: String + +Default: default + +## disable_internal_dns_cache + +Disables the internal DNS cache. Recommended for operating ClickHouse in systems +with frequently changing infrastructure such as Kubernetes. + +Type: Bool + +Default: 0 + +## dns_cache_update_period + +Internal DNS cache update period in seconds. + +Type: Int32 + +Default: 15 + + +## dns_max_consecutive_failures + +Max connection failures before dropping host from ClickHouse DNS cache + +Type: UInt32 + +Default: 1024 + + +## index_mark_cache_size + +Size of cache for index marks. Zero means disabled. + +Type: UInt64 + +Default: 0 + + +## index_uncompressed_cache_size + +Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled. + +Type: UInt64 + +Default: 0 + + +## io_thread_pool_queue_size + +Queue size for IO thread pool. Zero means unlimited. + +Type: UInt64 + +Default: 10000 + +## mark_cache_policy + +Mark cache policy name. + +Type: String + +Default: SLRU + +## mark_cache_size + +Size of cache for marks (index of MergeTree family of tables). + +Type: UInt64 + +Default: 5368709120 + +## max_backup_bandwidth_for_server + +The maximum read speed in bytes per second for all backups on server. Zero means unlimited. + +Type: UInt64 + +Default: 0 + +## max_backups_io_thread_pool_free_size + +If the number of **idle** threads in the Backups IO Thread pool exceeds `max_backup_io_thread_pool_free_size`, ClickHouse will release resources occupied by idling threads and decrease the pool size. Threads can be created again if necessary. + +Type: UInt64 + +Default: 0 + +## max_backups_io_thread_pool_size + +The maximum number of threads that would be used for IO operations for BACKUP queries + +Type: UInt64 + +Default: 1000 + +## max_concurrent_insert_queries + +Limit on total number of concurrent insert queries. Zero means Unlimited. + +:::note +These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. +::: + +Type: UInt64 + +Default: 0 + +## max_concurrent_queries + +Limit on total number of concurrently executed queries. Zero means Unlimited. Note that limits on insert and select queries, and on the maximum number of queries for users must also be considered. See also max_concurrent_insert_queries, max_concurrent_select_queries, max_concurrent_queries_for_all_users. Zero means unlimited. + +:::note +These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. +::: + +Type: UInt64 + +Default: 0 + +## max_concurrent_select_queries + +Limit on total number of concurrently select queries. Zero means Unlimited. + +:::note +These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. +::: + +Type: UInt64 + +Default: 0 + +## max_connections + +Max server connections. + +Type: Int32 + +Default: 1024 + +## max_io_thread_pool_free_size + +Max free size for IO thread pool. + +Type: UInt64 + +Default: 0 + +## max_io_thread_pool_size + +The maximum number of threads that would be used for IO operations + +Type: UInt64 + +Default: 100 + +## max_local_read_bandwidth_for_server + +The maximum speed of local reads in bytes per second. Zero means unlimited. + +Type: UInt64 + +Default: 0 + +## max_local_write_bandwidth_for_server + +The maximum speed of local writes in bytes per second. Zero means unlimited. + +Type: UInt64 + +Default: 0 + +## max_partition_size_to_drop + +Restriction on dropping partitions. + +If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_partition_size_to_drop` (in bytes), you can’t drop a partition using a [DROP PARTITION](../../sql-reference/statements/alter/partition.md#drop-partitionpart) query. +This setting does not require a restart of the Clickhouse server to apply. Another way to disable the restriction is to create the `/flags/force_drop_table` file. +Default value: 50 GB. +The value 0 means that you can drop partitions without any restrictions. + +:::note +This limitation does not restrict drop table and truncate table, see [max_table_size_to_drop](#max-table-size-to-drop) +::: + +## max_remote_read_network_bandwidth_for_server + +The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited. + +Type: UInt64 + +Default: 0 + +## max_remote_write_network_bandwidth_for_server + +The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited. + +Type: UInt64 + +Default: 0 + +## max_server_memory_usage + +Limit on total memory usage. Zero means Unlimited. + +The default `max_server_memory_usage` value is calculated as `memory_amount * max_server_memory_usage_to_ram_ratio`. + +Type: UInt64 + +Default: 0 + +## max_server_memory_usage_to_ram_ratio + +Same as max_server_memory_usage but in a ratio to physical RAM. Allows lowering the memory usage on low-memory systems. Zero means unlimited. + +On hosts with low RAM and swap, you possibly need setting `max_server_memory_usage_to_ram_ratio` larger than 1. + +Type: Double + +Default: 0.9 + +## max_table_size_to_drop + +Restriction on deleting tables. + +If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_table_size_to_drop` (in bytes), you can’t delete it using a [DROP](../../sql-reference/statements/drop.md) query or [TRUNCATE](../../sql-reference/statements/truncate.md) query. + +This setting does not require a restart of the Clickhouse server to apply. Another way to disable the restriction is to create the `/flags/force_drop_table` file. + +Default value: 50 GB. +The value 0 means that you can delete all tables without any restrictions. +**Example** +``` xml +0 +``` + +## max_temporary_data_on_disk_size + +The maximum amount of storage that could be used for external aggregation, joins or sorting. +Queries that exceed this limit will fail with an exception. Zero means unlimited. + +Also see `max_temporary_data_on_disk_size_for_user` and `max_temporary_data_on_disk_size_for_query`. + +Type: UInt64 + +Default: 0 + +## max_thread_pool_free_size + +If the number of **idle** threads in the Global Thread pool is greater than `max_thread_pool_free_size`, then ClickHouse releases resources occupied by some threads and the pool size is decreased. Threads can be created again if necessary. + +Type: UInt64 + +Default: 1000 + +## max_thread_pool_size + +The maximum number of threads that could be allocated from the OS and used for query execution and background operations. + +Type: UInt64 + +Default: 10000 + +## mmap_cache_size + +Sets the cache size (in bytes) for mapped files. This setting allows avoiding frequent open/close calls (which are very expensive due to consequent page faults), and to reuse mappings from several threads and queries. The setting value is the number of mapped regions (usually equal to the number of mapped files). The amount of data in mapped files can be monitored in the tables system.metrics and system.metric_log with the `MMappedFiles` and `MMappedFileBytes` metrics. Also, in system.asynchronous_metrics and system.asynchronous_metrics_log by the `MMapCacheCells` metric, and in system.events, system.processes, system.query_log, system.query_thread_log, system.query_views_log by the `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses` events. + +Note that the amount of data in mapped files does not consume memory directly and is not accounted for in query or server memory usage — because this memory can be discarded similar to the OS page cache. The cache is dropped (the files are closed) automatically on the removal of old parts in tables of the MergeTree family, also it can be dropped manually by the `SYSTEM DROP MMAP CACHE` query. + +Type: UInt64 + +Default: 1000 + +## restore_threads + +The maximum number of threads to execute RESTORE requests. + +Type: UInt64 + +Default: 16 + +## show_addresses_in_stack_traces + +If it is set true will show addresses in stack traces + +Type: Bool + +Default: 1 + +## shutdown_wait_unfinished_queries + +If set true ClickHouse will wait for running queries finish before shutdown. + +Type: Bool + +Default: 0 + +## temporary_data_in_cache + +With this option, temporary data will be stored in the cache for the particular disk. +In this section, you should specify the disk name with the type `cache`. +In that case, the cache and temporary data will share the same space, and the disk cache can be evicted to create temporary data. + +:::note +Only one option can be used to configure temporary data storage: `tmp_path` ,`tmp_policy`, `temporary_data_in_cache`. +::: + +**Example** + +Both the cache for `local_disk`, and temporary data will be stored in `/tiny_local_cache` on the filesystem, managed by `tiny_local_cache`. + +```xml + + + + + local + /local_disk/ + + + + + cache + local_disk + /tiny_local_cache/ + 10M + 1M + 1 + 0 + + + + + + + tiny_local_cache + + +``` + +Type: String + +Default: + +## thread_pool_queue_size + +TThe maximum number of jobs that can be scheduled on the Global Thread pool. Increasing queue size leads to larger memory usage. It is recommended to keep this value equal to `max_thread_pool_size`. Zero means unlimited. + +Type: UInt64 + +Default: 10000 + +## tmp_policy + +Policy for storage with temporary data. Also see the MergeTree Table Engine documentation. + +:::note +- Only one option can be used to configure temporary data storage: `tmp_path` ,`tmp_policy`, `temporary_data_in_cache`. +- `move_factor`, `keep_free_space_bytes`,`max_data_part_size_bytes` and are ignored. +- Policy should have exactly *one volume* with *local* disks. +::: + +**Example** + +When `/disk1` is full, temporary data will be stored on `/disk2`. + +```xml + + + + + /disk1/ + + + /disk2/ + + + + + + + +
+ disk1 + disk2 +
+
+
+ +
+
+ + + tmp_two_disks + +
+``` +Type: String + +Default: + +## uncompressed_cache_policy + +Uncompressed cache policy name. + +Type: String + +Default: SLRU + + +## uncompressed_cache_size + +Cache size (in bytes) for uncompressed data used by table engines from the MergeTree family. Zero means disabled. + +There is one shared cache for the server. Memory is allocated on demand. The cache is used if the option use_uncompressed_cache is enabled. + +The uncompressed cache is advantageous for very short queries in individual cases. + +Type: UInt64 + +Default: 0 + ## builtin_dictionaries_reload_interval {#builtin-dictionaries-reload-interval} The interval in seconds before reloading built-in dictionaries. @@ -224,18 +818,6 @@ Sets scheduling period of the task. Zero means "never". Default value: `86400` (1 day). -## default_database {#default-database} - -The default database. - -To get a list of databases, use the [SHOW DATABASES](../../sql-reference/statements/show.md#show-databases) query. - -**Example** - -``` xml -default -``` - ## default_profile {#default-profile} Default settings profile. @@ -695,157 +1277,6 @@ For more information, see the section [Creating replicated tables](../../engines ``` -## mark_cache_size {#server-mark-cache-size} - -Approximate size (in bytes) of the cache of marks used by table engines of the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family. - -The cache is shared for the server and memory is allocated as needed. - -**Example** - -``` xml -5368709120 -``` - -## max_server_memory_usage {#max_server_memory_usage} - -Limits total RAM usage by the ClickHouse server. - -Possible values: - -- Positive integer. -- 0 — Auto. - -Default value: `0`. - -**Additional Info** - -The default `max_server_memory_usage` value is calculated as `memory_amount * max_server_memory_usage_to_ram_ratio`. - -**See also** - -- [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) -- [max_server_memory_usage_to_ram_ratio](#max_server_memory_usage_to_ram_ratio) - -## max_server_memory_usage_to_ram_ratio {#max_server_memory_usage_to_ram_ratio} - -Defines the fraction of total physical RAM amount, available to the ClickHouse server. If the server tries to utilize more, the memory is cut down to the appropriate amount. - -Possible values: - -- Positive double. -- 0 — The ClickHouse server can use all available RAM. - -Default value: `0.9`. - -**Usage** - -On hosts with low RAM and swap, you possibly need setting `max_server_memory_usage_to_ram_ratio` larger than 1. - -**Example** - -``` xml -0.9 -``` - -**See Also** - -- [max_server_memory_usage](#max_server_memory_usage) - -## concurrent_threads_soft_limit_num {#concurrent_threads_soft_limit_num} -The maximum number of query processing threads, excluding threads for retrieving data from remote servers, allowed to run all queries. This is not a hard limit. In case if the limit is reached the query will still get at least one thread to run. Query can upscale to desired number of threads during execution if more threads become available. - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `0`. - -**See Also** - -- [Concurrency Control](/docs/en/development/architecture.md#concurrency-control) - -## concurrent_threads_soft_limit_ratio_to_cores {#concurrent_threads_soft_limit_ratio_to_cores} -The maximum number of query processing threads as multiple of number of logical cores. -More details: [concurrent_threads_soft_limit_num](#concurrent_threads_soft_limit_num). - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `0`. - -**Example** - -``` xml -3 -``` - -## max_concurrent_queries {#max-concurrent-queries} - -The maximum number of simultaneously processed queries. -Note that other limits also apply: [max_concurrent_insert_queries](#max-concurrent-insert-queries), [max_concurrent_select_queries](#max-concurrent-select-queries), [max_concurrent_queries_for_user](#max-concurrent-queries-for-user), [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users). - -:::note -These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. -::: - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `100`. - -**Example** - -``` xml -200 -``` - -## max_concurrent_insert_queries {#max-concurrent-insert-queries} - -The maximum number of simultaneously processed `INSERT` queries. - -:::note -These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. -::: - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `0`. - -**Example** - -``` xml -100 -``` - -## max_concurrent_select_queries {#max-concurrent-select-queries} - -The maximum number of simultaneously processed `SELECT` queries. - -:::note -These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. -::: - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `0`. - -**Example** - -``` xml -100 -``` ## max_concurrent_queries_for_user {#max-concurrent-queries-for-user} @@ -889,16 +1320,6 @@ Default value: `0`. - [max_concurrent_queries](#max-concurrent-queries) -## max_connections {#max-connections} - -The maximum number of inbound connections. - -**Example** - -``` xml -4096 -``` - ## max_open_files {#max-open-files} The maximum number of open files. @@ -1128,91 +1549,8 @@ Default value: `0.5`. - [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) - [merges_mutations_memory_usage_soft_limit](#merges_mutations_memory_usage_soft_limit) -## background_merges_mutations_scheduling_policy {#background_merges_mutations_scheduling_policy} -Algorithm used to select next merge or mutation to be executed by background thread pool. Policy may be changed at runtime without server restart. -Could be applied from the `default` profile for backward compatibility. -Possible values: - -- "round_robin" — Every concurrent merge and mutation is executed in round-robin order to ensure starvation-free operation. Smaller merges are completed faster than bigger ones just because they have fewer blocks to merge. -- "shortest_task_first" — Always execute smaller merge or mutation. Merges and mutations are assigned priorities based on their resulting size. Merges with smaller sizes are strictly preferred over bigger ones. This policy ensures the fastest possible merge of small parts but can lead to indefinite starvation of big merges in partitions heavily overloaded by INSERTs. - -Default value: "round_robin". - -**Example** - -```xml -shortest_task_first -``` - -## background_move_pool_size {#background_move_pool_size} - -Sets the number of threads performing background moves for tables with MergeTree engines. Could be increased at runtime and could be applied at server startup from the `default` profile for backward compatibility. - -Possible values: - -- Any positive integer. - -Default value: 8. - -**Example** - -```xml -36 -``` - -## background_fetches_pool_size {#background_fetches_pool_size} - -Sets the number of threads performing background fetches for tables with ReplicatedMergeTree engines. Could be increased at runtime. - -Possible values: - -- Any positive integer. - -Default value: 8. - -**Example** - -```xml -36 -``` - -## background_common_pool_size {#background_common_pool_size} - -Sets the number of threads performing background non-specialized operations like cleaning the filesystem etc. for tables with MergeTree engines. Could be increased at runtime. - -Possible values: - -- Any positive integer. - -Default value: 8. - -**Example** - -```xml -36 -``` - -## background_buffer_flush_schedule_pool_size {#background_buffer_flush_schedule_pool_size} - -Sets the number of threads performing background flush in [Buffer](../../engines/table-engines/special/buffer.md)-engine tables. - -Possible values: - -- Any positive integer. - -Default value: 16. - -## background_schedule_pool_size {#background_schedule_pool_size} - -Sets the number of threads performing background tasks for [replicated](../../engines/table-engines/mergetree-family/replication.md) tables, [Kafka](../../engines/table-engines/integrations/kafka.md) streaming, [DNS cache updates](../../operations/server-configuration-parameters/settings.md/#server-settings-dns-cache-update-period). - -Possible values: - -- Any positive integer. - -Default value: 128. ## merge_tree {#server_configuration_parameters-merge_tree} @@ -1341,14 +1679,14 @@ Queries are logged in the [system.part_log](../../operations/system-tables/part_ Use the following parameters to configure logging: -- `database` - Name of the database. -- `table` - Name of the system table. -- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `database` – Name of the database. +- `table` – Name of the system table. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. -- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional). -- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). +- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `storage_policy` – Name of storage policy to use for the table (optional) +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). **Example** @@ -1419,14 +1757,14 @@ Queries are logged in the [system.query_log](../../operations/system-tables/quer Use the following parameters to configure logging: -- `database` - Name of the database. -- `table` - Name of the system table the queries will be logged in. -- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `database` – Name of the database. +- `table` – Name of the system table the queries will be logged in. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. -- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional). -- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). +- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `storage_policy` – Name of storage policy to use for the table (optional) +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). If the table does not exist, ClickHouse will create it. If the structure of the query log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. @@ -1477,14 +1815,14 @@ Queries are logged in the [system.query_thread_log](../../operations/system-tabl Use the following parameters to configure logging: -- `database` - Name of the database. -- `table` - Name of the system table the queries will be logged in. -- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `database` – Name of the database. +- `table` – Name of the system table the queries will be logged in. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. -- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional). -- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). +- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `storage_policy` – Name of storage policy to use for the table (optional) +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). If the table does not exist, ClickHouse will create it. If the structure of the query thread log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. @@ -1507,14 +1845,14 @@ Queries are logged in the [system.query_views_log](../../operations/system-table Use the following parameters to configure logging: -- `database` - Name of the database. -- `table` - Name of the system table the queries will be logged in. -- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `database` – Name of the database. +- `table` – Name of the system table the queries will be logged in. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. -- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional). -- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). +- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `storage_policy` – Name of storage policy to use for the table (optional) +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). If the table does not exist, ClickHouse will create it. If the structure of the query views log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. @@ -1535,15 +1873,15 @@ Settings for the [text_log](../../operations/system-tables/text_log.md#system_ta Parameters: -- `level` - Maximum Message Level (by default `Trace`) which will be stored in a table. -- `database` - Database name. -- `table` - Table name. -- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `level` — Maximum Message Level (by default `Trace`) which will be stored in a table. +- `database` — Database name. +- `table` — Table name. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. -- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional). -- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). +- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `storage_policy` – Name of storage policy to use for the table (optional) +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). **Example** ```xml @@ -1566,14 +1904,14 @@ Settings for the [trace_log](../../operations/system-tables/trace_log.md#system_ Parameters: -- `database` - Database for storing a table. -- `table` - Table name. -- `partition_by` - [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `database` — Database for storing a table. +- `table` — Table name. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` or `order_by` defined. -- `flush_interval_milliseconds` - Interval for flushing data from the buffer in memory to the table. -- `storage_policy` - Name of storage policy to use for the table (optional). -- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree#settings) that control the behavior of the MergeTree (optional). +- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `storage_policy` – Name of storage policy to use for the table (optional) +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). The default server configuration file `config.xml` contains the following settings section: @@ -1714,115 +2052,6 @@ Path on the local filesystem to store temporary data for processing large querie /var/lib/clickhouse/tmp/ ``` -## tmp_policy {#tmp-policy} - -Alternatively, a policy from [storage_configuration](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) can be used to store temporary files. - -:::note -- Only one option can be used to configure temporary data storage: `tmp_path` ,`tmp_policy`, `temporary_data_in_cache`. -- `move_factor`, `keep_free_space_bytes`,`max_data_part_size_bytes` and are ignored. -- Policy should have exactly *one volume* with *local* disks. -::: - -**Example** - -```xml - - - - /disk1/ - - - /disk2/ - - - - - - -
- disk1 - disk2 -
-
-
-
-
- - tmp_two_disks -
- -``` - -When `/disk1` is full, temporary data will be stored on `/disk2`. - -## temporary_data_in_cache {#temporary-data-in-cache} - -With this option, temporary data will be stored in the cache for the particular disk. -In this section, you should specify the disk name with the type `cache`. -In that case, the cache and temporary data will share the same space, and the disk cache can be evicted to create temporary data. - -:::note -- Only one option can be used to configure temporary data storage: `tmp_path` ,`tmp_policy`, `temporary_data_in_cache`. -::: - -**Example** - -```xml - - - - - local - /local_disk/ - - - - cache - local_disk - /tiny_local_cache/ - 10M - 1M - 1 - 0 - - - - - tiny_local_cache - -``` - -Cache for `local_disk` and temporary data will be stored in `/tiny_local_cache` on the filesystem, managed by `tiny_local_cache`. - -## max_temporary_data_on_disk_size {#max_temporary_data_on_disk_size} - -Limit the amount of disk space consumed by temporary files in `tmp_path` for the server. -Queries that exceed this limit will fail with an exception. - -Default value: `0`. - -**See also** - -- [max_temporary_data_on_disk_size_for_user](../../operations/settings/query-complexity.md#settings_max_temporary_data_on_disk_size_for_user) -- [max_temporary_data_on_disk_size_for_query](../../operations/settings/query-complexity.md#settings_max_temporary_data_on_disk_size_for_query) -- [tmp_path](#tmp-path) -- [tmp_policy](#tmp-policy) -- [max_server_memory_usage](#max_server_memory_usage) - -## uncompressed_cache_size {#server-settings-uncompressed_cache_size} - -Cache size (in bytes) for uncompressed data used by table engines from the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). - -There is one shared cache for the server. Memory is allocated on demand. The cache is used if the option [use_uncompressed_cache](../../operations/settings/settings.md#setting-use_uncompressed_cache) is enabled. - -The uncompressed cache is advantageous for very short queries in individual cases. - -**Example** - -``` xml -8589934592 -``` ## user_files_path {#server_configuration_parameters-user_files_path} @@ -1953,24 +2182,6 @@ Data part headers already stored with this setting can't be restored to their pr **Default value:** 0. -## disable_internal_dns_cache {#server-settings-disable-internal-dns-cache} - -Disables the internal DNS cache. Recommended for operating ClickHouse in systems -with frequently changing infrastructure such as Kubernetes. - -**Default value:** 0. - -## dns_cache_update_period {#server-settings-dns-cache-update-period} - -The period of updating IP addresses stored in the ClickHouse internal DNS cache (in seconds). -The update is performed asynchronously, in a separate system thread. - -**Default value**: 15. - -**See also** - -- [background_schedule_pool_size](../../operations/settings/settings.md#background_schedule_pool_size) - ## distributed_ddl {#server-settings-distributed_ddl} Manage executing [distributed ddl queries](../../sql-reference/distributed-ddl.md) (CREATE, DROP, ALTER, RENAME) on cluster. @@ -2100,16 +2311,6 @@ Possible values: Default value: `0`. -## mmap_cache_size {#mmap-cache-size} - -Sets the cache size (in bytes) for mapped files. This setting allows to avoid frequent open/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/close calls (which are very expensive due to consequent page faults) and to reuse mappings from several threads and queries. The setting value is the number of mapped regions (usually equal to the number of mapped files). The amount of data in mapped files can be monitored in [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) system tables by the `MMappedFiles` and `MMappedFileBytes` metrics, in [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) by the `MMapCacheCells` metric, and also in [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) by the `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses` events. Note that the amount of data in mapped files does not consume memory directly and is not accounted in query or server memory usage — because this memory can be discarded similar to OS page cache. The cache is dropped (the files are closed) automatically on the removal of old parts in tables of the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family, also it can be dropped manually by the `SYSTEM DROP MMAP CACHE` query. - -Possible values: - -- Positive integer. - -Default value: `1000`. - ## compiled_expression_cache_size {#compiled-expression-cache-size} Sets the cache size (in bytes) for [compiled expressions](../../operations/caches.md). From 28ee307e75490840c85b16e934e6789e7569707a Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 06:21:23 +0000 Subject: [PATCH 0839/2223] Integrate toLastDayOfWeek() to 01921_datatype_date32 test --- tests/queries/0_stateless/01921_datatype_date32.reference | 6 ++++++ tests/queries/0_stateless/01921_datatype_date32.sql | 2 ++ 2 files changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/01921_datatype_date32.reference b/tests/queries/0_stateless/01921_datatype_date32.reference index 14079b906cb..cc3dba12069 100644 --- a/tests/queries/0_stateless/01921_datatype_date32.reference +++ b/tests/queries/0_stateless/01921_datatype_date32.reference @@ -84,6 +84,12 @@ 2120-07-05 2120-07-26 2021-06-20 +-------toLastDayOfWeek--------- +2079-06-12 +2079-06-12 +2120-07-11 +2120-08-01 +2021-06-26 -------toStartOfMonth--------- 2079-06-07 2079-06-07 diff --git a/tests/queries/0_stateless/01921_datatype_date32.sql b/tests/queries/0_stateless/01921_datatype_date32.sql index 8b65f82825f..717afc483aa 100644 --- a/tests/queries/0_stateless/01921_datatype_date32.sql +++ b/tests/queries/0_stateless/01921_datatype_date32.sql @@ -36,6 +36,8 @@ select '-------toYearWeek---------'; select toYearWeek(x1) from t1; select '-------toStartOfWeek---------'; select toStartOfWeek(x1) from t1; +select '-------toLastDayOfWeek---------'; +select toLastDayOfWeek(x1) from t1; select '-------toStartOfMonth---------'; select toStartOfMonth(x1) from t1; select '-------toStartOfQuarter---------'; From dfd7d8b59eda5f2f1b7230641a5b3b9409f5eda6 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 07:19:12 +0000 Subject: [PATCH 0840/2223] Align toStartOfWeek() and toLastDayOfWeek() test cases in 01921_datatype_date32.sql test --- .../00941_to_custom_week.reference | 20 +++++++++---------- .../0_stateless/00941_to_custom_week.sql | 20 +++++++++++++------ 2 files changed, 24 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/00941_to_custom_week.reference b/tests/queries/0_stateless/00941_to_custom_week.reference index 1650f955989..48be1dbb072 100644 --- a/tests/queries/0_stateless/00941_to_custom_week.reference +++ b/tests/queries/0_stateless/00941_to_custom_week.reference @@ -61,16 +61,16 @@ 2017-01-09 00:00:00 2 3 201702 201703 2017-01-10 00:00:00 2 3 201702 201703 2017-01-11 00:00:00 2 3 201702 201703 -2018-12-25 2018-12-25 00:00:00 2018-12-23 2018-12-23 2018-12-24 2018-12-24 -2018-12-26 2018-12-26 00:00:00 2018-12-23 2018-12-23 2018-12-24 2018-12-24 -2018-12-27 2018-12-27 00:00:00 2018-12-23 2018-12-23 2018-12-24 2018-12-24 -2018-12-28 2018-12-28 00:00:00 2018-12-23 2018-12-23 2018-12-24 2018-12-24 -2018-12-29 2018-12-29 00:00:00 2018-12-23 2018-12-23 2018-12-24 2018-12-24 -2018-12-30 2018-12-30 00:00:00 2018-12-30 2018-12-30 2018-12-24 2018-12-24 -2018-12-31 2018-12-31 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 -2019-01-01 2019-01-01 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 -2019-01-02 2019-01-02 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 -2019-01-03 2019-01-03 00:00:00 2018-12-30 2018-12-30 2018-12-31 2018-12-31 +2018-12-25 00:00:00 2018-12-23 2018-12-23 2018-12-23 2018-12-23 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-26 00:00:00 2018-12-23 2018-12-23 2018-12-23 2018-12-23 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-27 00:00:00 2018-12-23 2018-12-23 2018-12-23 2018-12-23 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-28 00:00:00 2018-12-23 2018-12-23 2018-12-23 2018-12-23 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-29 00:00:00 2018-12-23 2018-12-23 2018-12-23 2018-12-23 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-30 00:00:00 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-24 2018-12-24 2018-12-24 2018-12-24 +2018-12-31 00:00:00 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2018-12-31 2018-12-31 +2019-01-01 00:00:00 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2018-12-31 2018-12-31 +2019-01-02 00:00:00 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2018-12-31 2018-12-31 +2019-01-03 00:00:00 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-31 2018-12-31 2018-12-31 2018-12-31 2018-12-25 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-26 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 2018-12-27 00:00:00 2018-12-29 2018-12-29 2018-12-29 2018-12-29 2018-12-30 2018-12-30 2018-12-30 2018-12-30 diff --git a/tests/queries/0_stateless/00941_to_custom_week.sql b/tests/queries/0_stateless/00941_to_custom_week.sql index 58a81c03ef8..3281ed23fe7 100644 --- a/tests/queries/0_stateless/00941_to_custom_week.sql +++ b/tests/queries/0_stateless/00941_to_custom_week.sql @@ -41,13 +41,21 @@ SELECT toDateTime(toDate('2016-12-22') + number, 'Asia/Istanbul' ) AS d, FROM numbers(21); -- toStartOfWeek +WITH + toDate('2018-12-25') + number AS d, + toDate32(d) AS d32, + toDateTime(d) AS dt, + toDateTime64(d, 0) AS dt64 SELECT - toDate('2018-12-25') + number AS x, - toDateTime(x) AS x_t, - toStartOfWeek(x) AS w0, - toStartOfWeek(x_t) AS wt0, - toStartOfWeek(x, 3) AS w3, - toStartOfWeek(x_t, 3) AS wt3 + dt64, + toStartOfWeek(d) AS wd_sun, + toStartOfWeek(d32) AS wd32_sun, + toStartOfWeek(dt) AS wdt_sun, + toStartOfWeek(dt64) AS wdt64_sun, + toStartOfWeek(d, 1) AS wd_mon, + toStartOfWeek(d32, 1) AS wd32_mon, + toStartOfWeek(dt, 1) AS wdt_mon, + toStartOfWeek(dt64, 1) AS wdt64_mon FROM numbers(10); -- toLastDayOfWeek From 65586c50f500dd5daa51de60087c54acfdf5f914 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Wed, 31 May 2023 00:21:13 -0700 Subject: [PATCH 0841/2223] Minor improvements in CGroup awareness. 1. Support CGroup2 in getMemoryAmountOrZero(). 2. Report CFS period and quota in asynchronous metric log. --- base/base/getMemoryAmount.cpp | 22 +++++++++--- src/Common/AsynchronousMetrics.cpp | 56 ++++++++++++++++++++++++++++-- src/Common/AsynchronousMetrics.h | 3 ++ 3 files changed, 75 insertions(+), 6 deletions(-) diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 9e1d2ac3279..6a5470a0549 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -28,14 +28,28 @@ uint64_t getMemoryAmountOrZero() #if defined(OS_LINUX) // Try to lookup at the Cgroup limit - std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes"); - if (cgroup_limit.is_open()) + + // v2 + std::ifstream cgroupv2_limit("/sys/fs/cgroup/memory.max"); + if (cgroupv2_limit.is_open()) { - uint64_t memory_limit = 0; // in case of read error - cgroup_limit >> memory_limit; + uint64_t memory_limit = 0; + cgroupv2_limit >> memory_limit; if (memory_limit > 0 && memory_limit < memory_amount) memory_amount = memory_limit; } + else + { + // v1 + std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes"); + if (cgroup_limit.is_open()) + { + uint64_t memory_limit = 0; // in case of read error + cgroup_limit >> memory_limit; + if (memory_limit > 0 && memory_limit < memory_amount) + memory_amount = memory_limit; + } + } #endif return memory_amount; diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index ac2180103c5..e1e99a3f7c7 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -69,13 +69,23 @@ AsynchronousMetrics::AsynchronousMetrics( /// CGroups v2 openFileIfExists("/sys/fs/cgroup/memory.max", cgroupmem_limit_in_bytes); - openFileIfExists("/sys/fs/cgroup/memory.current", cgroupmem_usage_in_bytes); + if (cgroupmem_limit_in_bytes) + { + openFileIfExists("/sys/fs/cgroup/memory.current", cgroupmem_usage_in_bytes); + } + openFileIfExists("/sys/fs/cgroup/cpu.max", cgroupcpu_max); /// CGroups v1 if (!cgroupmem_limit_in_bytes) + { openFileIfExists("/sys/fs/cgroup/memory/memory.limit_in_bytes", cgroupmem_limit_in_bytes); - if (!cgroupmem_usage_in_bytes) openFileIfExists("/sys/fs/cgroup/memory/memory.usage_in_bytes", cgroupmem_usage_in_bytes); + } + if (!cgroupcpu_max) + { + openFileIfExists("/sys/fs/cgroup/cpu/cpu.cfs_period_us", cgroupcpu_cfs_period); + openFileIfExists("/sys/fs/cgroup/cpu/cpu.cfs_quota_us", cgroupcpu_cfs_quota); + } openSensors(); openBlockDevices(); @@ -926,6 +936,48 @@ void AsynchronousMetrics::update(TimePoint update_time) tryLogCurrentException(__PRETTY_FUNCTION__); } } + + if (cgroupcpu_max) + { + try { + cgroupcpu_max->rewind(); + + uint64_t quota = 0; + uint64_t period = 0; + + readText(quota, *cgroupcpu_max); + skipWhitespaceIfAny(*cgroupcpu_max); + readText(period, *cgroupcpu_max); + + new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; + new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup."}; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + else if (cgroupcpu_cfs_quota && cgroupcpu_cfs_period) + { + try { + cgroupcpu_cfs_quota->rewind(); + cgroupcpu_cfs_period->rewind(); + + uint64_t quota = 0; + uint64_t period = 0; + + tryReadText(quota, *cgroupcpu_cfs_quota); + tryReadText(period, *cgroupcpu_cfs_period); + + new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; + new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup."}; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + if (meminfo) { try diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index d104b872f52..e3b5142553b 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -110,6 +110,9 @@ private: std::optional cgroupmem_limit_in_bytes; std::optional cgroupmem_usage_in_bytes; + std::optional cgroupcpu_cfs_period; + std::optional cgroupcpu_cfs_quota; + std::optional cgroupcpu_max; std::vector> thermal; From 45e2fb4b3eefca48b27e90d23147660d3da46e39 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Wed, 31 May 2023 10:58:05 +0200 Subject: [PATCH 0842/2223] Update README.md changing release webinar date --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 9c933540e01..9561458ba37 100644 --- a/README.md +++ b/README.md @@ -22,7 +22,7 @@ curl https://clickhouse.com/ | sh ## Upcoming Events -* [**v23.5 Release Webinar**](https://clickhouse.com/company/events/v23-5-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-05) - May 31 - 23.5 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. +* [**v23.5 Release Webinar**](https://clickhouse.com/company/events/v23-5-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-05) - Jun 8 - 23.5 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. * [**ClickHouse Meetup in Bangalore**](https://www.meetup.com/clickhouse-bangalore-user-group/events/293740066/) - Jun 7 * [**ClickHouse Meetup in San Francisco**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/293426725/) - Jun 7 From cd4c4e316991ad2d13cd41ee8feea3e6cc65322f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 May 2023 11:57:51 +0200 Subject: [PATCH 0843/2223] Update 02770_async_buffer_ignore.sh --- tests/queries/0_stateless/02770_async_buffer_ignore.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh index ca8909a8d0d..22b432db178 100755 --- a/tests/queries/0_stateless/02770_async_buffer_ignore.sh +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 8ce56dfe4616dcaf2638ffc8f1ef7c35718fea06 Mon Sep 17 00:00:00 2001 From: lihaibo42 Date: Wed, 31 May 2023 17:53:00 +0800 Subject: [PATCH 0844/2223] Link boost::context library to clickhouse_common_io --- src/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 622e18d4ff7..e1359a5a8aa 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -346,6 +346,7 @@ target_link_libraries(clickhouse_common_io PUBLIC boost::program_options boost::system + boost::context ch_contrib::cityhash ch_contrib::re2 ch_contrib::re2_st From e6e420da5517e61f2940cc9b349a62ed192e9822 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 31 May 2023 13:00:55 +0200 Subject: [PATCH 0845/2223] Add no-fasttest tag --- tests/queries/0_stateless/02771_skip_empty_files.sh | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02771_skip_empty_files.sh b/tests/queries/0_stateless/02771_skip_empty_files.sh index 99f43d7868a..2d1dc205dcd 100755 --- a/tests/queries/0_stateless/02771_skip_empty_files.sh +++ b/tests/queries/0_stateless/02771_skip_empty_files.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -16,9 +17,4 @@ $CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-*.parquet', auto, 'number $CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-*.parquet') settings engine_file_skip_empty_files=1" $CLICKHOUSE_LOCAL -q "select * from file('$FILE_PREFIX-*.parquet', auto, 'number UInt64') settings engine_file_skip_empty_files=1" - - - - - rm $FILE_PREFIX-* From 08d38878326fb9a2793e708008268a6eff000749 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 31 May 2023 07:04:29 -0400 Subject: [PATCH 0846/2223] Add re-creation for cherry-pick PRs (#50373) * Add recreation for cherry-pick PRs. Small PR comment fix. * Automatic style fix --------- Co-authored-by: robot-clickhouse Co-authored-by: Nikita Mikhaylov --- tests/ci/cherry_pick.py | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 2fa562a1386..d36315151aa 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -70,9 +70,12 @@ This pull-request will be merged automatically as it reaches the mergeable state ### If the PR was closed and then reopened -If it stuck, check {pr_url} for `{label_backports_created}` and delete it if \ +If it stuck, check {pr_url} for `{backport_created_label}` and delete it if \ necessary. Manually merging will do nothing, since `{label_backports_created}` \ prevents the original PR {pr_url} from being processed. + +If you want to recreate the PR: delete the `{label_cherrypick}` label and delete this branch. +You may also need to delete the `{label_backports_created}` label from the original PR. """ BACKPORT_DESCRIPTION = """This pull-request is a last step of an automated \ backporting. @@ -82,7 +85,13 @@ close it. """ REMOTE = "" - def __init__(self, name: str, pr: PullRequest, repo: Repository): + def __init__( + self, + name: str, + pr: PullRequest, + repo: Repository, + backport_created_label: str = Labels.BACKPORTS_CREATED, + ): self.name = name self.pr = pr self.repo = repo @@ -93,6 +102,8 @@ close it. self.backport_pr = None # type: Optional[PullRequest] self._backported = False + self.backport_created_label = backport_created_label + self.git_prefix = ( # All commits to cherrypick are done as robot-clickhouse "git -c user.email=robot-clickhouse@users.noreply.github.com " "-c user.name=robot-clickhouse -c commit.gpgsign=false" @@ -226,7 +237,8 @@ close it. body=self.CHERRYPICK_DESCRIPTION.format( pr_number=self.pr.number, pr_url=self.pr.html_url, - label_backports_created=Labels.BACKPORTS_CREATED, + backport_created_label=self.backport_created_label, + label_cherrypick=Labels.CHERRYPICK, ), base=self.backport_branch, head=self.cherrypick_branch, @@ -459,11 +471,12 @@ class Backport: pr_labels = [label.name for label in pr.labels] if self.must_create_backport_label in pr_labels: branches = [ - ReleaseBranch(br, pr, self.repo) for br in self.release_branches + ReleaseBranch(br, pr, self.repo, self.backport_created_label) + for br in self.release_branches ] # type: List[ReleaseBranch] else: branches = [ - ReleaseBranch(br, pr, self.repo) + ReleaseBranch(br, pr, self.repo, self.backport_created_label) for br in [ label.split("-", 1)[0][1:] # v21.8-must-backport for label in pr_labels @@ -492,6 +505,7 @@ class Backport: ) bp_cp_prs = self.gh.get_pulls_from_search( query=f"type:pr repo:{self._repo_name} {query_suffix}", + label=f"{Labels.BACKPORT},{Labels.CHERRYPICK}", ) for br in branches: br.pop_prs(bp_cp_prs) From 6d45d0c37404f4d3a7bd03c92e6a2f04adef7beb Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 31 May 2023 13:11:10 +0200 Subject: [PATCH 0847/2223] Use fingerprints instead of key IDs in encrypted disks (#49882) * Use fingerprints instead of key IDs to find keys in encrypted disks. Always use little endian in the headers of encryption files. * Add tests. * Fix copying binary files to test containers. * Fix ownership for copied files in test containers. * Add comments after review. --------- Co-authored-by: Nikita Mikhaylov --- src/Backups/tests/gtest_backup_entries.cpp | 11 +- src/Disks/DiskEncrypted.cpp | 251 +++++++++++----- src/Disks/DiskEncryptedTransaction.cpp | 43 +-- src/Disks/DiskEncryptedTransaction.h | 8 +- src/Disks/tests/gtest_disk_encrypted.cpp | 8 +- src/IO/FileEncryptionCommon.cpp | 108 +++++-- src/IO/FileEncryptionCommon.h | 42 ++- src/IO/tests/gtest_file_encryption.cpp | 3 +- tests/integration/helpers/cluster.py | 5 +- .../old_versions/version_1be/__marks.mrk | Bin 0 -> 96 bytes .../old_versions/version_1be/data.bin | Bin 0 -> 99 bytes .../old_versions/version_1be/id.bin | Bin 0 -> 102 bytes .../old_versions/version_1be/sizes.json | Bin 0 -> 162 bytes .../old_versions/version_1le/__marks.mrk | Bin 0 -> 96 bytes .../old_versions/version_1le/data.bin | Bin 0 -> 99 bytes .../old_versions/version_1le/id.bin | Bin 0 -> 102 bytes .../old_versions/version_1le/sizes.json | Bin 0 -> 162 bytes .../old_versions/version_2/__marks.mrk | Bin 0 -> 96 bytes .../old_versions/version_2/data.bin | Bin 0 -> 99 bytes .../old_versions/version_2/id.bin | Bin 0 -> 102 bytes .../old_versions/version_2/sizes.json | Bin 0 -> 162 bytes tests/integration/test_encrypted_disk/test.py | 284 ++++++++++++++---- 22 files changed, 543 insertions(+), 220 deletions(-) create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1be/__marks.mrk create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1be/data.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1be/id.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1be/sizes.json create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1le/__marks.mrk create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1le/data.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1le/id.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_1le/sizes.json create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_2/__marks.mrk create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_2/data.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_2/id.bin create mode 100644 tests/integration/test_encrypted_disk/old_versions/version_2/sizes.json diff --git a/src/Backups/tests/gtest_backup_entries.cpp b/src/Backups/tests/gtest_backup_entries.cpp index 3077bedad0e..ca603d20787 100644 --- a/src/Backups/tests/gtest_backup_entries.cpp +++ b/src/Backups/tests/gtest_backup_entries.cpp @@ -29,10 +29,15 @@ protected: /// Make encrypted disk. auto settings = std::make_unique(); settings->wrapped_disk = local_disk; - settings->current_algorithm = FileEncryption::Algorithm::AES_128_CTR; - settings->keys[0] = "1234567890123456"; - settings->current_key_id = 0; settings->disk_path = "encrypted/"; + + settings->current_algorithm = FileEncryption::Algorithm::AES_128_CTR; + String key = "1234567890123456"; + UInt128 fingerprint = FileEncryption::calculateKeyFingerprint(key); + settings->all_keys[fingerprint] = key; + settings->current_key = key; + settings->current_key_fingerprint = fingerprint; + encrypted_disk = std::make_shared("encrypted_disk", std::move(settings), true); } diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 2415b432e01..6b515b100c9 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -19,7 +19,6 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int INCORRECT_DISK_INDEX; - extern const int DATA_ENCRYPTION_ERROR; extern const int NOT_IMPLEMENTED; } @@ -42,87 +41,201 @@ namespace } } + /// Reads encryption keys from the configuration. + void getKeysFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, + std::map & out_keys_by_id, Strings & out_keys_without_id) + { + Strings config_keys; + config.keys(config_prefix, config_keys); + + for (const std::string & config_key : config_keys) + { + String key; + std::optional key_id; + + if ((config_key == "key") || config_key.starts_with("key[")) + { + String key_path = config_prefix + "." + config_key; + key = config.getString(key_path); + String key_id_path = key_path + "[@id]"; + if (config.has(key_id_path)) + key_id = config.getUInt64(key_id_path); + } + else if ((config_key == "key_hex") || config_key.starts_with("key_hex[")) + { + String key_path = config_prefix + "." + config_key; + key = unhexKey(config.getString(key_path)); + String key_id_path = key_path + "[@id]"; + if (config.has(key_id_path)) + key_id = config.getUInt64(key_id_path); + } + else + continue; + + if (key_id) + { + if (!out_keys_by_id.contains(*key_id)) + out_keys_by_id[*key_id] = key; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple keys specified for same ID {}", *key_id); + } + else + out_keys_without_id.push_back(key); + } + + if (out_keys_by_id.empty() && out_keys_without_id.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No encryption keys found"); + + if (out_keys_by_id.empty() && (out_keys_without_id.size() == 1)) + { + out_keys_by_id[0] = out_keys_without_id.front(); + out_keys_without_id.clear(); + } + } + + /// Reads the current encryption key from the configuration. + String getCurrentKeyFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, + const std::map & keys_by_id, const Strings & keys_without_id) + { + String key_path = config_prefix + ".current_key"; + String key_hex_path = config_prefix + ".current_key_hex"; + String key_id_path = config_prefix + ".current_key_id"; + + if (config.has(key_path) + config.has(key_hex_path) + config.has(key_id_path) > 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The current key is specified multiple times"); + + auto check_current_key_found = [&](const String & current_key_) + { + for (const auto & [_, key] : keys_by_id) + { + if (key == current_key_) + return; + } + for (const auto & key : keys_without_id) + { + if (key == current_key_) + return; + } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The current key is not found in keys"); + }; + + if (config.has(key_path)) + { + String current_key = config.getString(key_path); + check_current_key_found(current_key); + return current_key; + } + else if (config.has(key_hex_path)) + { + String current_key = unhexKey(config.getString(key_hex_path)); + check_current_key_found(current_key); + return current_key; + } + else if (config.has(key_id_path)) + { + UInt64 current_key_id = config.getUInt64(key_id_path); + auto it = keys_by_id.find(current_key_id); + if (it == keys_by_id.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found a key with the current ID {}", current_key_id); + return it->second; + } + else if (keys_by_id.size() == 1 && keys_without_id.empty() && keys_by_id.begin()->first == 0) + { + /// There is only a single key defined with id=0, so we can choose it as current. + return keys_by_id.begin()->second; + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The current key is not specified"); + } + } + + /// Reads the current encryption algorithm from the configuration. + Algorithm getCurrentAlgorithmFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) + { + String path = config_prefix + ".algorithm"; + if (!config.has(path)) + return DEFAULT_ENCRYPTION_ALGORITHM; + return parseAlgorithmFromString(config.getString(path)); + } + + /// Reads the name of a wrapped disk & the path on the wrapped disk and then finds that disk in a disk map. + void getDiskAndPathFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const DisksMap & map, + DiskPtr & out_disk, String & out_path) + { + String disk_name = config.getString(config_prefix + ".disk", ""); + if (disk_name.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Name of the wrapped disk must not be empty. Encrypted disk is a wrapper over another disk"); + + auto disk_it = map.find(disk_name); + if (disk_it == map.end()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "The wrapped disk must have been announced earlier. No disk with name {}", disk_name); + + out_disk = disk_it->second; + + out_path = config.getString(config_prefix + ".path", ""); + if (!out_path.empty() && (out_path.back() != '/')) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk path must ends with '/', but '{}' doesn't.", quoteString(out_path)); + } + + /// Parses the settings of an ecnrypted disk from the configuration. std::unique_ptr parseDiskEncryptedSettings( - const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const DisksMap & map) + const String & disk_name, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + const DisksMap & disk_map) { try { auto res = std::make_unique(); - res->current_algorithm = DEFAULT_ENCRYPTION_ALGORITHM; - if (config.has(config_prefix + ".algorithm")) - parseFromString(res->current_algorithm, config.getString(config_prefix + ".algorithm")); - Strings config_keys; - config.keys(config_prefix, config_keys); - for (const std::string & config_key : config_keys) + std::map keys_by_id; + Strings keys_without_id; + getKeysFromConfig(config, config_prefix, keys_by_id, keys_without_id); + + for (const auto & [key_id, key] : keys_by_id) { - String key; - UInt64 key_id; + auto fingerprint = calculateKeyFingerprint(key); + res->all_keys[fingerprint] = key; - if ((config_key == "key") || config_key.starts_with("key[")) - { - key = config.getString(config_prefix + "." + config_key, ""); - key_id = config.getUInt64(config_prefix + "." + config_key + "[@id]", 0); - } - else if ((config_key == "key_hex") || config_key.starts_with("key_hex[")) - { - key = unhexKey(config.getString(config_prefix + "." + config_key, "")); - key_id = config.getUInt64(config_prefix + "." + config_key + "[@id]", 0); - } - else - continue; - - if (res->keys.contains(key_id)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple keys have the same ID {}", key_id); - res->keys[key_id] = key; + /// Version 1 used key fingerprints based on the key id. + /// We have to add such fingerprints to the map too to support reading files encrypted by version 1. + auto v1_fingerprint = calculateV1KeyFingerprint(key, key_id); + res->all_keys[v1_fingerprint] = key; } - if (res->keys.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "No keys, an encrypted disk needs keys to work"); - - if (!config.has(config_prefix + ".current_key_id")) + for (const auto & key : keys_without_id) { - /// In case of multiple keys, current_key_id is mandatory - if (res->keys.size() > 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "There are multiple keys in config. current_key_id is required"); - - /// If there is only one key with non zero ID, curren_key_id should be defined. - if (res->keys.size() == 1 && !res->keys.contains(0)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config has one key with non zero id. сurrent_key_id is required"); + auto fingerprint = calculateKeyFingerprint(key); + res->all_keys[fingerprint] = key; } - res->current_key_id = config.getUInt64(config_prefix + ".current_key_id", 0); - if (!res->keys.contains(res->current_key_id)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found a key with the current ID {}", res->current_key_id); - FileEncryption::checkKeySize(res->current_algorithm, res->keys[res->current_key_id].size()); + String current_key = getCurrentKeyFromConfig(config, config_prefix, keys_by_id, keys_without_id); + res->current_key = current_key; + res->current_key_fingerprint = calculateKeyFingerprint(current_key); - String wrapped_disk_name = config.getString(config_prefix + ".disk", ""); - if (wrapped_disk_name.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Name of the wrapped disk must not be empty. Encrypted disk is a wrapper over another disk"); + res->current_algorithm = getCurrentAlgorithmFromConfig(config, config_prefix); - auto wrapped_disk_it = map.find(wrapped_disk_name); - if (wrapped_disk_it == map.end()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "The wrapped disk must have been announced earlier. No disk with name {}", - wrapped_disk_name); - res->wrapped_disk = wrapped_disk_it->second; + FileEncryption::checkKeySize(res->current_key.size(), res->current_algorithm); - res->disk_path = config.getString(config_prefix + ".path", ""); - if (!res->disk_path.empty() && (res->disk_path.back() != '/')) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk path must ends with '/', but '{}' doesn't.", quoteString(res->disk_path)); + DiskPtr wrapped_disk; + String disk_path; + getDiskAndPathFromConfig(config, config_prefix, disk_map, wrapped_disk, disk_path); + res->wrapped_disk = wrapped_disk; + res->disk_path = disk_path; return res; } catch (Exception & e) { - e.addMessage("Disk " + name); + e.addMessage("Disk " + disk_name); throw; } } + /// Reads the header of an encrypted file. FileEncryption::Header readHeader(ReadBufferFromFileBase & read_buffer) { try @@ -138,24 +251,6 @@ namespace } } - String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings) - { - auto it = settings.keys.find(header.key_id); - if (it == settings.keys.end()) - throw Exception( - ErrorCodes::DATA_ENCRYPTION_ERROR, - "Not found a key with ID {} required to decipher file {}", - header.key_id, - quoteString(path)); - - String key = it->second; - if (calculateKeyHash(key) != header.key_hash) - throw Exception( - ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key with ID {}, could not decipher file {}", header.key_id, quoteString(path)); - - return key; - } - bool inline isSameDiskType(const IDisk & one, const IDisk & another) { return typeid(one) == typeid(another); @@ -225,7 +320,7 @@ void DiskEncrypted::copy(const String & from_path, const std::shared_ptr { auto from_settings = current_settings.get(); auto to_settings = to_disk_enc->current_settings.get(); - if (from_settings->keys == to_settings->keys) + if (from_settings->all_keys == to_settings->all_keys) { /// Keys are the same so we can simply copy the encrypted file. auto wrapped_from_path = wrappedPath(from_path); @@ -252,7 +347,7 @@ void DiskEncrypted::copyDirectoryContent(const String & from_dir, const std::sha { auto from_settings = current_settings.get(); auto to_settings = to_disk_enc->current_settings.get(); - if (from_settings->keys == to_settings->keys) + if (from_settings->all_keys == to_settings->all_keys) { /// Keys are the same so we can simply copy the encrypted file. auto wrapped_from_path = wrappedPath(from_dir); @@ -293,7 +388,7 @@ std::unique_ptr DiskEncrypted::readFile( } auto encryption_settings = current_settings.get(); FileEncryption::Header header = readHeader(*buffer); - String key = getKey(path, header, *encryption_settings); + String key = encryption_settings->findKeyByFingerprint(header.key_fingerprint, path); return std::make_unique(settings.local_fs_buffer_size, std::move(buffer), key, header); } diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 4a613374ccf..40df94b309a 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -38,39 +38,21 @@ FileEncryption::Header readHeader(ReadBufferFromFileBase & read_buffer) } } -String getCurrentKey(const String & path, const DiskEncryptedSettings & settings) +} + +String DiskEncryptedSettings::findKeyByFingerprint(UInt128 key_fingerprint, const String & path_for_logs) const { - auto it = settings.keys.find(settings.current_key_id); - if (it == settings.keys.end()) + auto it = all_keys.find(key_fingerprint); + if (it == all_keys.end()) + { throw Exception( ErrorCodes::DATA_ENCRYPTION_ERROR, - "Not found a key with the current ID {} required to cipher file {}", - settings.current_key_id, - quoteString(path)); - + "Not found an encryption key required to decipher file {}", + quoteString(path_for_logs)); + } return it->second; } -String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings) -{ - auto it = settings.keys.find(header.key_id); - if (it == settings.keys.end()) - throw Exception( - ErrorCodes::DATA_ENCRYPTION_ERROR, - "Not found a key with ID {} required to decipher file {}", - header.key_id, - quoteString(path)); - - String key = it->second; - if (FileEncryption::calculateKeyHash(key) != header.key_hash) - throw Exception( - ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key with ID {}, could not decipher file {}", header.key_id, quoteString(path)); - - return key; -} - -} - void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path) { auto wrapped_from_path = wrappedPath(from_file_path); @@ -98,16 +80,15 @@ std::unique_ptr DiskEncryptedTransaction::writeFile( // /// Append mode: we continue to use the same header. auto read_buffer = delegate_disk->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize)); header = readHeader(*read_buffer); - key = getKey(path, header, current_settings); + key = current_settings.findKeyByFingerprint(header.key_fingerprint, path); } } if (!old_file_size) { /// Rewrite mode: we generate a new header. - key = getCurrentKey(path, current_settings); header.algorithm = current_settings.current_algorithm; - header.key_id = current_settings.current_key_id; - header.key_hash = FileEncryption::calculateKeyHash(key); + key = current_settings.current_key; + header.key_fingerprint = current_settings.current_key_fingerprint; header.init_vector = FileEncryption::InitVector::random(); } auto buffer = delegate_transaction->writeFile(wrapped_path, buf_size, mode, settings, autocommit); diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index bae3f2c728c..04cc63f1671 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -18,9 +18,13 @@ struct DiskEncryptedSettings { DiskPtr wrapped_disk; String disk_path; - std::unordered_map keys; - UInt64 current_key_id; + String current_key; + UInt128 current_key_fingerprint; FileEncryption::Algorithm current_algorithm; + std::unordered_map all_keys; + + /// Returns an encryption key found by its fingerprint. + String findKeyByFingerprint(UInt128 key_fingerprint, const String & path_for_logs) const; }; diff --git a/src/Disks/tests/gtest_disk_encrypted.cpp b/src/Disks/tests/gtest_disk_encrypted.cpp index 80a10e8680b..ee9e284d409 100644 --- a/src/Disks/tests/gtest_disk_encrypted.cpp +++ b/src/Disks/tests/gtest_disk_encrypted.cpp @@ -37,8 +37,10 @@ protected: auto settings = std::make_unique(); settings->wrapped_disk = local_disk; settings->current_algorithm = algorithm; - settings->keys[0] = key; - settings->current_key_id = 0; + auto fingerprint = FileEncryption::calculateKeyFingerprint(key); + settings->all_keys[fingerprint] = key; + settings->current_key = key; + settings->current_key_fingerprint = fingerprint; settings->disk_path = path; encrypted_disk = std::make_shared("encrypted_disk", std::move(settings), true); } @@ -255,7 +257,7 @@ TEST_F(DiskEncryptedTest, RandomIV) String bina = getBinaryRepresentation(getDirectory() + "a.txt"); String binb = getBinaryRepresentation(getDirectory() + "b.txt"); - constexpr size_t iv_offset = 16; + constexpr size_t iv_offset = 23; /// See the description of the format in the comment for FileEncryption::Header. constexpr size_t iv_size = FileEncryption::InitVector::kSize; EXPECT_EQ(bina.substr(0, iv_offset), binb.substr(0, iv_offset)); /// Part of the header before IV is the same. EXPECT_NE(bina.substr(iv_offset, iv_size), binb.substr(iv_offset, iv_size)); /// IV differs. diff --git a/src/IO/FileEncryptionCommon.cpp b/src/IO/FileEncryptionCommon.cpp index 4ac4d289b32..5529c813c40 100644 --- a/src/IO/FileEncryptionCommon.cpp +++ b/src/IO/FileEncryptionCommon.cpp @@ -34,6 +34,7 @@ namespace case Algorithm::AES_128_CTR: return EVP_aes_128_ctr(); case Algorithm::AES_192_CTR: return EVP_aes_192_ctr(); case Algorithm::AES_256_CTR: return EVP_aes_256_ctr(); + case Algorithm::MAX: break; } throw Exception( ErrorCodes::BAD_ARGUMENTS, @@ -187,10 +188,14 @@ namespace return plaintext_size; } - constexpr const char kHeaderSignature[] = "ENC"; - constexpr const UInt16 kHeaderCurrentVersion = 1; -} + constexpr const std::string_view kHeaderSignature = "ENC"; + UInt128 calculateV1KeyFingerprint(UInt8 small_key_hash, UInt64 key_id) + { + /// In the version 1 we stored {key_id, very_small_hash(key)} instead of a fingerprint. + return static_cast(key_id) | (static_cast(small_key_hash) << 64); + } +} String toString(Algorithm algorithm) { @@ -199,6 +204,7 @@ String toString(Algorithm algorithm) case Algorithm::AES_128_CTR: return "aes_128_ctr"; case Algorithm::AES_192_CTR: return "aes_192_ctr"; case Algorithm::AES_256_CTR: return "aes_256_ctr"; + case Algorithm::MAX: break; } throw Exception( ErrorCodes::BAD_ARGUMENTS, @@ -206,14 +212,14 @@ String toString(Algorithm algorithm) static_cast(algorithm)); } -void parseFromString(Algorithm & algorithm, const String & str) +Algorithm parseAlgorithmFromString(const String & str) { if (boost::iequals(str, "aes_128_ctr")) - algorithm = Algorithm::AES_128_CTR; + return Algorithm::AES_128_CTR; else if (boost::iequals(str, "aes_192_ctr")) - algorithm = Algorithm::AES_192_CTR; + return Algorithm::AES_192_CTR; else if (boost::iequals(str, "aes_256_ctr")) - algorithm = Algorithm::AES_256_CTR; + return Algorithm::AES_256_CTR; else throw Exception( ErrorCodes::BAD_ARGUMENTS, @@ -221,7 +227,7 @@ void parseFromString(Algorithm & algorithm, const String & str) str); } -void checkKeySize(Algorithm algorithm, size_t key_size) { checkKeySize(getCipher(algorithm), key_size); } +void checkKeySize(size_t key_size, Algorithm algorithm) { checkKeySize(getCipher(algorithm), key_size); } String InitVector::toString() const @@ -364,54 +370,92 @@ void Encryptor::decrypt(const char * data, size_t size, char * out) void Header::read(ReadBuffer & in) { - constexpr size_t header_signature_size = std::size(kHeaderSignature) - 1; - char signature[std::size(kHeaderSignature)] = {}; - in.readStrict(signature, header_signature_size); - if (strcmp(signature, kHeaderSignature) != 0) + char signature[kHeaderSignature.length()]; + in.readStrict(signature, kHeaderSignature.length()); + if (memcmp(signature, kHeaderSignature.data(), kHeaderSignature.length()) != 0) throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong signature, this is not an encrypted file"); - UInt16 version; - readPODBinary(version, in); - if (version != kHeaderCurrentVersion) + /// The endianness of how the header is written. + /// Starting from version 2 the header is always in little endian. + std::endian endian = std::endian::little; + + readBinaryLittleEndian(version, in); + + if (version == 0x0100ULL) + { + /// Version 1 could write the header of an encrypted file in either little-endian or big-endian. + /// So now if we read the version as little-endian and it's 256 that means two things: the version is actually 1 and the whole header is in big endian. + endian = std::endian::big; + version = 1; + } + + if (version < 1 || version > kCurrentVersion) throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Version {} of the header is not supported", version); UInt16 algorithm_u16; readPODBinary(algorithm_u16, in); + if (std::endian::native != endian) + algorithm_u16 = std::byteswap(algorithm_u16); + if (algorithm_u16 >= static_cast(Algorithm::MAX)) + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Algorithm {} is not supported", algorithm_u16); algorithm = static_cast(algorithm_u16); - readPODBinary(key_id, in); - readPODBinary(key_hash, in); + size_t bytes_to_skip = kSize - kHeaderSignature.length() - sizeof(version) - sizeof(algorithm_u16) - InitVector::kSize; + + if (version < 2) + { + UInt64 key_id; + UInt8 small_key_hash; + readPODBinary(key_id, in); + readPODBinary(small_key_hash, in); + bytes_to_skip -= sizeof(key_id) + sizeof(small_key_hash); + if (std::endian::native != endian) + key_id = std::byteswap(key_id); + key_fingerprint = calculateV1KeyFingerprint(small_key_hash, key_id); + } + else + { + readBinaryLittleEndian(key_fingerprint, in); + bytes_to_skip -= sizeof(key_fingerprint); + } + init_vector.read(in); - constexpr size_t reserved_size = kSize - header_signature_size - sizeof(version) - sizeof(algorithm_u16) - sizeof(key_id) - sizeof(key_hash) - InitVector::kSize; - static_assert(reserved_size < kSize); - in.ignore(reserved_size); + chassert(bytes_to_skip < kSize); + in.ignore(bytes_to_skip); } void Header::write(WriteBuffer & out) const { - constexpr size_t header_signature_size = std::size(kHeaderSignature) - 1; - out.write(kHeaderSignature, header_signature_size); + writeString(kHeaderSignature, out); - UInt16 version = kHeaderCurrentVersion; - writePODBinary(version, out); + writeBinaryLittleEndian(version, out); UInt16 algorithm_u16 = static_cast(algorithm); - writePODBinary(algorithm_u16, out); + writeBinaryLittleEndian(algorithm_u16, out); + + writeBinaryLittleEndian(key_fingerprint, out); - writePODBinary(key_id, out); - writePODBinary(key_hash, out); init_vector.write(out); - constexpr size_t reserved_size = kSize - header_signature_size - sizeof(version) - sizeof(algorithm_u16) - sizeof(key_id) - sizeof(key_hash) - InitVector::kSize; + constexpr size_t reserved_size = kSize - kHeaderSignature.length() - sizeof(version) - sizeof(algorithm_u16) - sizeof(key_fingerprint) - InitVector::kSize; static_assert(reserved_size < kSize); - char reserved_zero_bytes[reserved_size] = {}; - out.write(reserved_zero_bytes, reserved_size); + char zero_bytes[reserved_size] = {}; + out.write(zero_bytes, reserved_size); } -UInt8 calculateKeyHash(const String & key) +UInt128 calculateKeyFingerprint(const String & key) { - return static_cast(sipHash64(key.data(), key.size())) & 0x0F; + const UInt64 seed0 = 0x4368456E63727970ULL; // ChEncryp + const UInt64 seed1 = 0x7465644469736B46ULL; // tedDiskF + return sipHash128Keyed(seed0, seed1, key.data(), key.size()); +} + +UInt128 calculateV1KeyFingerprint(const String & key, UInt64 key_id) +{ + /// In the version 1 we stored {key_id, very_small_hash(key)} instead of a fingerprint. + UInt8 small_key_hash = sipHash64(key.data(), key.size()) & 0x0F; + return calculateV1KeyFingerprint(small_key_hash, key_id); } } diff --git a/src/IO/FileEncryptionCommon.h b/src/IO/FileEncryptionCommon.h index efc0194da52..87aa1194273 100644 --- a/src/IO/FileEncryptionCommon.h +++ b/src/IO/FileEncryptionCommon.h @@ -23,13 +23,14 @@ enum class Algorithm AES_128_CTR, /// Size of key is 16 bytes. AES_192_CTR, /// Size of key is 24 bytes. AES_256_CTR, /// Size of key is 32 bytes. + MAX }; String toString(Algorithm algorithm); -void parseFromString(Algorithm & algorithm, const String & str); +Algorithm parseAlgorithmFromString(const String & str); /// Throws an exception if a specified key size doesn't correspond a specified encryption algorithm. -void checkKeySize(Algorithm algorithm, size_t key_size); +void checkKeySize(size_t key_size, Algorithm algorithm); /// Initialization vector. Its size is always 16 bytes. @@ -103,15 +104,34 @@ private: /// File header which is stored at the beginning of encrypted files. +/// +/// The format of that header is following: +/// +--------+------+--------------------------------------------------------------------------+ +/// | offset | size | description | +/// +--------+------+--------------------------------------------------------------------------+ +/// | 0 | 3 | 'E', 'N', 'C' (file's signature) | +/// | 3 | 2 | version of this header (1..2) | +/// | 5 | 2 | encryption algorithm (0..2, 0=AES_128_CTR, 1=AES_192_CTR, 2=AES_256_CTR) | +/// | 7 | 16 | fingerprint of encryption key (SipHash) | +/// | 23 | 16 | initialization vector (randomly generated) | +/// | 39 | 25 | reserved for future use | +/// +--------+------+--------------------------------------------------------------------------+ +/// struct Header { + /// Versions: + /// 1 - Initial version + /// 2 - The header of an encrypted file contains the fingerprint of a used encryption key instead of a pair {key_id, very_small_hash(key)}. + /// The header is always stored in little endian. + static constexpr const UInt16 kCurrentVersion = 2; + + UInt16 version = kCurrentVersion; + + /// Encryption algorithm. Algorithm algorithm = Algorithm::AES_128_CTR; - /// Identifier of the key to encrypt or decrypt this file. - UInt64 key_id = 0; - - /// Hash of the key to encrypt or decrypt this file. - UInt8 key_hash = 0; + /// Fingerprint of a key. + UInt128 key_fingerprint = 0; InitVector init_vector; @@ -122,9 +142,11 @@ struct Header void write(WriteBuffer & out) const; }; -/// Calculates the hash of a passed key. -/// 1 byte is enough because this hash is used only for the first check. -UInt8 calculateKeyHash(const String & key); +/// Calculates the fingerprint of a passed encryption key. +UInt128 calculateKeyFingerprint(const String & key); + +/// Calculates kind of the fingerprint of a passed encryption key & key ID as it was implemented in version 1. +UInt128 calculateV1KeyFingerprint(const String & key, UInt64 key_id); } } diff --git a/src/IO/tests/gtest_file_encryption.cpp b/src/IO/tests/gtest_file_encryption.cpp index 6a090ff0810..2b3d7ce81c5 100644 --- a/src/IO/tests/gtest_file_encryption.cpp +++ b/src/IO/tests/gtest_file_encryption.cpp @@ -226,8 +226,7 @@ TEST(FileEncryptionPositionUpdateTest, Decryption) String key = "1234567812345678"; FileEncryption::Header header; header.algorithm = Algorithm::AES_128_CTR; - header.key_id = 1; - header.key_hash = calculateKeyHash(key); + header.key_fingerprint = calculateKeyFingerprint(key); header.init_vector = InitVector::random(); auto lwb = std::make_unique(tmp_path); diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 950663cb429..f57ebf40e54 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1963,9 +1963,9 @@ class ClickHouseCluster: return output def copy_file_to_container(self, container_id, local_path, dest_path): - with open(local_path, "r") as fdata: + with open(local_path, "rb") as fdata: data = fdata.read() - encodedBytes = base64.b64encode(data.encode("utf-8")) + encodedBytes = base64.b64encode(data) encodedStr = str(encodedBytes, "utf-8") self.exec_in_container( container_id, @@ -1974,7 +1974,6 @@ class ClickHouseCluster: "-c", "echo {} | base64 --decode > {}".format(encodedStr, dest_path), ], - user="root", ) def wait_for_url( diff --git a/tests/integration/test_encrypted_disk/old_versions/version_1be/__marks.mrk b/tests/integration/test_encrypted_disk/old_versions/version_1be/__marks.mrk new file mode 100644 index 0000000000000000000000000000000000000000..88ad4b6cf3a1b43e3a27eff0696429c94bee8597 GIT binary patch literal 96 zcmZ?ub7o*F)A02 literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_1be/data.bin b/tests/integration/test_encrypted_disk/old_versions/version_1be/data.bin new file mode 100644 index 0000000000000000000000000000000000000000..f82db60e077eec4aec1ff5e7885e71e4fb67587c GIT binary patch literal 99 zcmZ?ub7o*n;quo=<- literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_1be/id.bin b/tests/integration/test_encrypted_disk/old_versions/version_1be/id.bin new file mode 100644 index 0000000000000000000000000000000000000000..0c2426599a7cdafed1b0835df4962a6b4be8b0d2 GIT binary patch literal 102 zcmZ?ub7o*1(U%=%e;^La?0M$MLunY#jeu5XR2z7 YJrWis`8aqU?pvBM-}Zcv!kO9M0LYjcNB{r; literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_1be/sizes.json b/tests/integration/test_encrypted_disk/old_versions/version_1be/sizes.json new file mode 100644 index 0000000000000000000000000000000000000000..6d610f2da837e016e0dd58162e3fa47b3fdd8723 GIT binary patch literal 162 zcmZ?ub7o*(<1jvv&|4{XdzVvOGW)@xdOrrw;qjl6pl*Dbh_wq@spu#6KD7ua3}on$qCxAl-@ l_Vn!wPt8i^(JEOzCwNxJMXuhz7YoltPj!>n7vP&`4FHC@I$i(( literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_1le/__marks.mrk b/tests/integration/test_encrypted_disk/old_versions/version_1le/__marks.mrk new file mode 100644 index 0000000000000000000000000000000000000000..919c95234e5c54930c013a7f7f602b8a82d40841 GIT binary patch literal 96 zcmZ?ub7o{TRIz3A{p@GOcv~$@NmHm+R`0m20NkJvON8<9y_ m|1Z5Bz;phefdB6v(eAx#0#kB11CA=HobP^ZT^9CnZZ80`Q8xDg literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_2/__marks.mrk b/tests/integration/test_encrypted_disk/old_versions/version_2/__marks.mrk new file mode 100644 index 0000000000000000000000000000000000000000..e85676068f7e6ebe4952560368177d9e87987834 GIT binary patch literal 96 zcmZ?ub7o>-V3^Y8qto>FLP|yPKkMk--(;PlWBJu1r|n273|JHCH~j@r33jj{EcuQo h=ldzYWq4Js8YDihd0xh_KPBk}-{vE;FTS5M0RXtmAgurZ literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/old_versions/version_2/data.bin b/tests/integration/test_encrypted_disk/old_versions/version_2/data.bin new file mode 100644 index 0000000000000000000000000000000000000000..b0b5e06a28009caccea335a822ccae153c432902 GIT binary patch literal 99 zcmZ?ub7o>-V3^Y8qto>FLP|yPKkMk--(>H;TzD?zhE~IueU93t?-V3^Y8qto>FLP|yPKkMk--(=k*J-Rk77E*mA^(d`dd*)G~66}D#>VZt6 n_SLQDubuKL4;Aq~InCh8nW`(@UduI>7Ajm%Ymk(EGx;-V3^Y8qto>FLP|yPKkMk--(>r(e!mW4uVz^qyi_D;+M2IGCD?)9(#c#4 zVrtY(^m|UdpT6?Ks#O7(CO>%Mf77C0pwZU-c!b-Gd8MhGw{`a1KJ$FGW}(;0*D3aY y8zy{XoSm7pbNSjGDs44hF*$6tMpc3ucAK_0__1@XivP~0@bL|UZ^+TM<|F{?^gwX{ literal 0 HcmV?d00001 diff --git a/tests/integration/test_encrypted_disk/test.py b/tests/integration/test_encrypted_disk/test.py index 66ff073f02b..9f5415f4bea 100644 --- a/tests/integration/test_encrypted_disk/test.py +++ b/tests/integration/test_encrypted_disk/test.py @@ -1,9 +1,11 @@ import pytest +import os.path from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException import os.path from helpers.test_tools import assert_eq_with_retry +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) FIRST_PART_NAME = "all_1_1_0" @@ -170,53 +172,62 @@ def test_optimize_table(policy, encrypted_disk): assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" -# Test adding encryption key on the fly. -def test_add_key(): - def make_storage_policy_with_keys(policy_name, keys): - node.exec_in_container( - [ - "bash", - "-c", - """cat > /etc/clickhouse-server/config.d/storage_policy_{policy_name}.xml << EOF +def make_storage_policy_with_keys( + policy_name, keys, check_system_storage_policies=False +): + if check_system_storage_policies: + node.query("SELECT policy_name FROM system.storage_policies") + + node.exec_in_container( + [ + "bash", + "-c", + """cat > /etc/clickhouse-server/config.d/storage_policy_{policy_name}.xml << EOF - - - <{policy_name}_disk> - encrypted - disk_local - {policy_name}_dir/ - {keys} - - - - <{policy_name}> - -
- {policy_name}_disk -
-
- -
-
+ + + <{policy_name}_disk> + encrypted + disk_local + {policy_name}_dir/ + {keys} + + + + <{policy_name}> + +
+ {policy_name}_disk +
+
+ +
+
EOF""".format( - policy_name=policy_name, keys=keys - ), - ] + policy_name=policy_name, keys=keys + ), + ] + ) + + node.query("SYSTEM RELOAD CONFIG") + + if check_system_storage_policies: + assert_eq_with_retry( + node, + f"SELECT policy_name FROM system.storage_policies WHERE policy_name='{policy_name}'", + policy_name, ) - node.query("SYSTEM RELOAD CONFIG") + + +# Test adding encryption key on the fly. +def test_add_keys(): + keys = "firstfirstfirstf" + make_storage_policy_with_keys( + "encrypted_policy_multikeys", keys, check_system_storage_policies=True + ) # Add some data to an encrypted disk. - node.query("SELECT policy_name FROM system.storage_policies") - make_storage_policy_with_keys( - "encrypted_policy_multikeys", "firstfirstfirstf" - ) - assert_eq_with_retry( - node, - "SELECT policy_name FROM system.storage_policies WHERE policy_name='encrypted_policy_multikeys'", - "encrypted_policy_multikeys", - ) - node.query( """ CREATE TABLE encrypted_test ( @@ -233,31 +244,39 @@ EOF""".format( assert node.query(select_query) == "(0,'data'),(1,'data')" # Add a second key and start using it. - make_storage_policy_with_keys( - "encrypted_policy_multikeys", + keys = """ + firstfirstfirstf + secondsecondseco + secondsecondseco """ - firstfirstfirstf - secondsecondseco - 1 - """, - ) + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + node.query("INSERT INTO encrypted_test VALUES (2,'data'),(3,'data')") # Now "(0,'data'),(1,'data')" is encrypted with the first key and "(2,'data'),(3,'data')" is encrypted with the second key. # All data are accessible. assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" - # Try to replace the first key with something wrong, and check that "(0,'data'),(1,'data')" cannot be read. - make_storage_policy_with_keys( - "encrypted_policy_multikeys", - """ - wrongwrongwrongw + # Keys can be reordered. + keys = """ secondsecondseco + firstfirstfirstf 1 - """, - ) + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) - expected_error = "Wrong key" + # All data are still accessible. + assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" + + # Try to replace the first key with something wrong, and check that "(0,'data'),(1,'data')" cannot be read. + keys = """ + secondsecondseco + wrongwrongwrongw + secondsecondseco + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + expected_error = "Not found an encryption key required to decipher" assert expected_error in node.query_and_get_error(select_query) # Detach the part encrypted with the wrong key and check that another part containing "(2,'data'),(3,'data')" still can be read. @@ -265,6 +284,159 @@ EOF""".format( assert node.query(select_query) == "(2,'data'),(3,'data')" +# Test adding encryption key on the fly. +def test_add_keys_with_id(): + keys = "firstfirstfirstf" + make_storage_policy_with_keys( + "encrypted_policy_multikeys", keys, check_system_storage_policies=True + ) + + # Add some data to an encrypted disk. + node.query( + """ + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='encrypted_policy_multikeys' + """ + ) + + node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')") + select_query = "SELECT * FROM encrypted_test ORDER BY id FORMAT Values" + assert node.query(select_query) == "(0,'data'),(1,'data')" + + # Add a second key and start using it. + keys = """ + firstfirstfirstf + secondsecondseco + 1 + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + node.query("INSERT INTO encrypted_test VALUES (2,'data'),(3,'data')") + + # Now "(0,'data'),(1,'data')" is encrypted with the first key and "(2,'data'),(3,'data')" is encrypted with the second key. + # All data are accessible. + assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" + + # Keys can be reordered. + keys = """ + secondsecondseco + firstfirstfirstf + 1 + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + # All data are still accessible. + assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" + + # Try to replace the first key with something wrong, and check that "(0,'data'),(1,'data')" cannot be read. + keys = """ + secondsecondseco + wrongwrongwrongw + 1 + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + expected_error = "Not found an encryption key required to decipher" + assert expected_error in node.query_and_get_error(select_query) + + # Detach the part encrypted with the wrong key and check that another part containing "(2,'data'),(3,'data')" still can be read. + node.query("ALTER TABLE encrypted_test DETACH PART '{}'".format(FIRST_PART_NAME)) + assert node.query(select_query) == "(2,'data'),(3,'data')" + + +# Test appending of encrypted files. +def test_log_family(): + keys = "firstfirstfirstf" + make_storage_policy_with_keys( + "encrypted_policy_multikeys", keys, check_system_storage_policies=True + ) + + # Add some data to an encrypted disk. + node.query( + """ + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=Log + SETTINGS storage_policy='encrypted_policy_multikeys' + """ + ) + + node.query("INSERT INTO encrypted_test VALUES (0,'data'),(1,'data')") + select_query = "SELECT * FROM encrypted_test ORDER BY id FORMAT Values" + assert node.query(select_query) == "(0,'data'),(1,'data')" + + # Add a second key and start using it. + keys = """ + firstfirstfirstf + secondsecondseco + secondsecondseco + """ + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + node.query("INSERT INTO encrypted_test VALUES (2,'data'),(3,'data')") + assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" + + # Everything is still encrypted with the first key (because the Log engine appends files), so the second key can be removed. + keys = "firstfirstfirstf" + make_storage_policy_with_keys("encrypted_policy_multikeys", keys) + + assert node.query(select_query) == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" + + +@pytest.mark.parametrize( + "old_version", + ["version_1le", "version_1be", "version_2"], +) +def test_migration_from_old_version(old_version): + keys = """ + first_key_first_ + second_key_secon + third_key_third_ + 3 + """ + make_storage_policy_with_keys( + "migration_from_old_version", keys, check_system_storage_policies=True + ) + + # Create a table without data. + node.query( + """ + CREATE TABLE encrypted_test ( + id Int64, + data String + ) ENGINE=Log + SETTINGS storage_policy='migration_from_old_version' + """ + ) + + # Copy table's data from an old version. + data_path = node.query( + "SELECT data_paths[1] FROM system.tables WHERE table = 'encrypted_test'" + ).splitlines()[0] + node.query("DETACH TABLE encrypted_test") + + old_version_dir = os.path.join(SCRIPT_DIR, "old_versions", old_version) + for file_name in os.listdir(old_version_dir): + src_path = os.path.join(old_version_dir, file_name) + dest_path = os.path.join(data_path, file_name) + node.copy_file_to_container(src_path, dest_path) + + node.query("ATTACH TABLE encrypted_test") + + # We can read from encrypted disk after migration. + select_query = "SELECT * FROM encrypted_test ORDER BY id FORMAT Values" + assert node.query(select_query) == "(0,'ab'),(1,'cdefg')" + + # We can append files on encrypted disk after migration. + node.query("INSERT INTO encrypted_test VALUES (2,'xyz')") + assert node.query(select_query) == "(0,'ab'),(1,'cdefg'),(2,'xyz')" + + def test_read_in_order(): node.query( "CREATE TABLE encrypted_test(`a` UInt64, `b` String(150)) ENGINE = MergeTree() ORDER BY (a, b) SETTINGS storage_policy='encrypted_policy'" From 444ce60aeb4cda9ac2e5c61bbc0ae03cc76cd2b2 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 31 May 2023 11:50:25 +0000 Subject: [PATCH 0848/2223] Add tests with explicit cast --- .../01746_convert_type_with_default.reference | 47 +++++++++++++----- .../01746_convert_type_with_default.sql | 48 ++++++++++++++++--- 2 files changed, 77 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.reference b/tests/queries/0_stateless/01746_convert_type_with_default.reference index 235a88157c8..892a12434b9 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.reference +++ b/tests/queries/0_stateless/01746_convert_type_with_default.reference @@ -20,17 +20,40 @@ 2 -1 -2 -1970-01-01 -2023-05-30 -2023-05-30 -2023-05-30 -1970-01-01 -2023-05-30 -2023-05-30 -1970-01-01 -2023-05-30 14:38:20 -2023-05-30 14:38:20 -2023-05-30 14:38:20 -2023-05-30 14:38:20 61f0c404-5cb3-11e7-907b-a6006ad3dba0 59f0c404-5cb3-11e7-907b-a6006ad3dba0 +1970-01-01 +2023-05-30 +2023-05-30 +2023-05-30 +1970-01-01 +1970-01-20 +1970-01-20 +1970-01-20 +1970-01-20 +1970-01-20 +1970-01-20 +1970-01-20 +1970-01-20 +1970-01-20 +1970-01-20 +1970-01-20 +1970-01-20 +2023-05-30 +1970-01-01 +2023-05-30 14:38:20 +2023-05-30 14:38:20 +2023-05-30 14:38:20 +2023-05-30 14:38:20 +1970-01-01 00:00:19 +1970-01-01 00:00:19 +1970-01-01 00:00:19 +1970-01-01 00:00:19 +1970-01-01 00:00:19 +1970-01-01 00:00:19 +1970-01-01 00:00:19 +1970-01-01 00:00:19 +1970-01-01 00:00:19 +1970-01-01 00:00:19 +1970-01-01 00:00:19 +1970-01-01 00:00:19 diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index 40e4798721b..9fdd92491a7 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -26,19 +26,55 @@ select toUInt256OrDefault('1xx', cast(2 as UInt256)); select toInt256OrDefault('-1', cast(-2 as Int256)); select toInt256OrDefault('-1xx', cast(-2 as Int256)); -select toDateOrDefault('2020-0x-02'); +SELECT toUUIDOrDefault('61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); +SELECT toUUIDOrDefault('-----61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); + +select toDateOrDefault('1xxx'); select toDateOrDefault('2023-05-30'); select toDateOrDefault('2023-05-30', '2000-01-01'::Date); -select toDateOrDefault('2020-0x-02', '2023-05-30'::Date); +select toDateOrDefault('1xx', '2023-05-30'::Date); select toDateOrDefault(-1); -select toDateOrDefault(19507); + +select toDateOrDefault(cast(19 as Int8)); +select toDateOrDefault(cast(19 as UInt8)); + +select toDateOrDefault(cast(19 as Int16)); +select toDateOrDefault(cast(19 as UInt16)); + +select toDateOrDefault(cast(19 as Int32)); +select toDateOrDefault(cast(19 as UInt32)); + +select toDateOrDefault(cast(19 as Int64)); +select toDateOrDefault(cast(19 as UInt64)); + +select toDateOrDefault(cast(19 as Int128)); +select toDateOrDefault(cast(19 as UInt128)); + +select toDateOrDefault(cast(19 as Int256)); +select toDateOrDefault(cast(19 as UInt256)); + select toDateOrDefault(19507, '2000-01-01'::Date); select toDateOrDefault(-1, '2000-01-01'::Date); select toDateTimeOrDefault('2023-05-30 14:38:20'); select toDateTimeOrDefault('2023-05-30 14:38:20', 'UTC'); -select toString(toDateTimeOrDefault('s2023', 'Asia/Istanbul', '2023-05-30 14:38:20'::DateTime('Asia/Istanbul')), 'Asia/Istanbul'); +select toString(toDateTimeOrDefault('1xxx', 'Asia/Istanbul', '2023-05-30 14:38:20'::DateTime('Asia/Istanbul')), 'Asia/Istanbul'); select toDateTimeOrDefault(1685457500); -SELECT toUUIDOrDefault('61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); -SELECT toUUIDOrDefault('-----61f0c404-5cb3-11e7-907b-a6006ad3dba0', cast('59f0c404-5cb3-11e7-907b-a6006ad3dba0' as UUID)); \ No newline at end of file +select toDateTimeOrDefault(cast(19 as Int8)); +select toDateTimeOrDefault(cast(19 as UInt8)); + +select toDateTimeOrDefault(cast(19 as Int16)); +select toDateTimeOrDefault(cast(19 as UInt16)); + +select toDateTimeOrDefault(cast(19 as Int32)); +select toDateTimeOrDefault(cast(19 as UInt32)); + +select toDateTimeOrDefault(cast(19 as Int64)); +select toDateTimeOrDefault(cast(19 as UInt64)); + +select toDateTimeOrDefault(cast(19 as Int128)); +select toDateTimeOrDefault(cast(19 as UInt128)); + +select toDateTimeOrDefault(cast(19 as Int256)); +select toDateTimeOrDefault(cast(19 as UInt256)); \ No newline at end of file From 74dc37cf610746a8814e3cc3195c44f2f3926650 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 May 2023 14:15:28 +0200 Subject: [PATCH 0849/2223] Fix assertion --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index e244f61ae2e..dc4a2599d1d 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -925,16 +925,16 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() LOG_TEST( log, - "Read {} bytes, read type {}, position: {}, offset: {}, segment end: {}", - size, toString(read_type), implementation_buffer->getPosition(), - implementation_buffer->getFileOffsetOfBufferEnd(), file_segment.range().right); + "Read {} bytes, read type {}, file offset: {}, impl offset: {}/{}, segment: {}", + size, toString(read_type), file_offset_of_buffer_end, + implementation_buffer->getFileOffsetOfBufferEnd(), read_until_position, file_segment.range().toString()); if (read_type == ReadType::CACHED) { ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); - chassert(file_offset_of_buffer_end + size <= file_segment.range().size()); + chassert(file_offset_of_buffer_end + size - 1 <= file_segment.range().right); } else { From 9b8159e08135ccd65e2b4dd5c3000b89c6d09350 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 31 May 2023 14:20:28 +0200 Subject: [PATCH 0850/2223] Update 02770_async_buffer_ignore.sh --- tests/queries/0_stateless/02770_async_buffer_ignore.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02770_async_buffer_ignore.sh b/tests/queries/0_stateless/02770_async_buffer_ignore.sh index 22b432db178..6eaf655324d 100755 --- a/tests/queries/0_stateless/02770_async_buffer_ignore.sh +++ b/tests/queries/0_stateless/02770_async_buffer_ignore.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 4d4112ff536f819514973dfd0cb8274cf044bb3e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 31 May 2023 15:26:56 +0300 Subject: [PATCH 0851/2223] Revert "less logs in WriteBufferFromS3" (#50390) --- src/IO/WriteBufferFromS3.cpp | 8 ++++++++ src/IO/WriteBufferFromS3TaskTracker.cpp | 11 +++++++++++ 2 files changed, 19 insertions(+) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 6992c3ea4ac..462cf2674c3 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -195,14 +195,18 @@ void WriteBufferFromS3::finalizeImpl() if (request_settings.check_objects_after_upload) { + LOG_TRACE(log, "Checking object {} exists after upload", key); S3::checkObjectExists(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage, "Immediately after upload"); + LOG_TRACE(log, "Checking object {} has size as expected {}", key, total_size); size_t actual_size = S3::getObjectSize(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage); if (actual_size != total_size) throw Exception( ErrorCodes::S3_ERROR, "Object {} from bucket {} has unexpected size {} after upload, expected size {}, it's a bug in S3 or S3 API.", key, bucket, actual_size, total_size); + + LOG_TRACE(log, "Object {} exists after upload", key); } } @@ -288,6 +292,8 @@ void WriteBufferFromS3::reallocateFirstBuffer() WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); chassert(offset() == 0); + + LOG_TRACE(log, "Reallocated first buffer with size {}. {}", memory.size(), getLogDetails()); } void WriteBufferFromS3::detachBuffer() @@ -310,6 +316,8 @@ void WriteBufferFromS3::allocateFirstBuffer() const auto size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), max_first_buffer); memory = Memory(size); WriteBuffer::set(memory.data(), memory.size()); + + LOG_TRACE(log, "Allocated first buffer with size {}. {}", memory.size(), getLogDetails()); } void WriteBufferFromS3::allocateBuffer() diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index c10af5d0672..7ae31044012 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -36,6 +36,8 @@ ThreadPoolCallbackRunner WriteBufferFromS3::TaskTracker::syncRunner() void WriteBufferFromS3::TaskTracker::waitAll() { + LOG_TEST(log, "waitAll, in queue {}", futures.size()); + /// Exceptions are propagated for (auto & future : futures) { @@ -49,6 +51,8 @@ void WriteBufferFromS3::TaskTracker::waitAll() void WriteBufferFromS3::TaskTracker::safeWaitAll() { + LOG_TEST(log, "safeWaitAll, wait in queue {}", futures.size()); + for (auto & future : futures) { if (future.valid()) @@ -72,6 +76,7 @@ void WriteBufferFromS3::TaskTracker::safeWaitAll() void WriteBufferFromS3::TaskTracker::waitIfAny() { + LOG_TEST(log, "waitIfAny, in queue {}", futures.size()); if (futures.empty()) return; @@ -96,6 +101,8 @@ void WriteBufferFromS3::TaskTracker::waitIfAny() watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); + + LOG_TEST(log, "waitIfAny ended, in queue {}", futures.size()); } void WriteBufferFromS3::TaskTracker::add(Callback && func) @@ -140,6 +147,8 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() if (!max_tasks_inflight) return; + LOG_TEST(log, "waitTilInflightShrink, in queue {}", futures.size()); + Stopwatch watch; /// Alternative approach is to wait until at least futures.size() - max_tasks_inflight element are finished @@ -162,6 +171,8 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); + + LOG_TEST(log, "waitTilInflightShrink ended, in queue {}", futures.size()); } bool WriteBufferFromS3::TaskTracker::isAsync() const From 7d077f6130c61b529b13403b8a744220002f2ef0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 May 2023 14:30:27 +0200 Subject: [PATCH 0852/2223] Add one more assertion --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index dc4a2599d1d..5cb9d3bbf6f 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -934,7 +934,9 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); - chassert(file_offset_of_buffer_end + size - 1 <= file_segment.range().right); + [[maybe_unused]] size_t new_file_offset = file_offset_of_buffer_end + size; + chassert(new_file_offset - 1 <= file_segment.range().right); + chassert(new_file_offset <= file_segment.getCurrentWriteOffset(true)); } else { From a59effcc88314cfd8e4c7972c67d2dffde2114bb Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 May 2023 14:40:13 +0200 Subject: [PATCH 0853/2223] Minor improvements --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 52 +++++++++---------- 1 file changed, 25 insertions(+), 27 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 5cb9d3bbf6f..202914a0774 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -51,8 +51,8 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( std::optional read_until_position_, std::shared_ptr cache_log_) : ReadBufferFromFileBase(use_external_buffer_ ? 0 : settings_.remote_fs_buffer_size, nullptr, 0, file_size_) -#ifndef NDEBUG - , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile(" + source_file_path_ + ")")) +#ifdef ABORT_ON_LOGICAL_ERROR + , log(&Poco::Logger::get(fmt::format("CachedOnDiskReadBufferFromFile({})", cache_key_))) #else , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile")) #endif @@ -75,6 +75,9 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog( const FileSegment::Range & file_segment_range, CachedOnDiskReadBufferFromFile::ReadType type) { + if (!cache_log) + return; + FilesystemCacheLogElement elem { .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), @@ -104,8 +107,7 @@ void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog( break; } - if (cache_log) - cache_log->add(elem); + cache_log->add(elem); } void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) @@ -411,7 +413,7 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegment & file_segme { case ReadType::CACHED: { -#ifndef NDEBUG +#ifdef ABORT_ON_LOGICAL_ERROR size_t file_size = getFileSizeFromReadBuffer(*read_buffer_for_file_segment); if (file_size == 0 || range.left + file_size <= file_offset_of_buffer_end) throw Exception( @@ -456,7 +458,7 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegment & file_segme { read_buffer_for_file_segment->seek(file_offset_of_buffer_end, SEEK_SET); - assert(read_buffer_for_file_segment->getFileOffsetOfBufferEnd() == file_offset_of_buffer_end); + chassert(read_buffer_for_file_segment->getFileOffsetOfBufferEnd() == file_offset_of_buffer_end); } const auto current_write_offset = file_segment.getCurrentWriteOffset(false); @@ -887,28 +889,24 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (!result) { - auto debug_check = [&]() +#ifdef ABORT_ON_LOGICAL_ERROR + if (read_type == ReadType::CACHED) { - if (read_type == ReadType::CACHED) + size_t cache_file_size = getFileSizeFromReadBuffer(*implementation_buffer); + if (cache_file_size == 0) { - size_t cache_file_size = getFileSizeFromReadBuffer(*implementation_buffer); - if (cache_file_size == 0) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Attempt to read from an empty cache file: {} (just before actual read)", - cache_file_size); - } + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Attempt to read from an empty cache file: {} (just before actual read)", + cache_file_size); } - else - { - chassert(file_offset_of_buffer_end == static_cast(implementation_buffer->getFileOffsetOfBufferEnd())); - } - chassert(!implementation_buffer->hasPendingData()); - return true; - }; - - chassert(debug_check()); + } + else + { + chassert(file_offset_of_buffer_end == static_cast(implementation_buffer->getFileOffsetOfBufferEnd())); + } + chassert(!implementation_buffer->hasPendingData()); +#endif Stopwatch watch(CLOCK_MONOTONIC); @@ -1093,8 +1091,8 @@ off_t CachedOnDiskReadBufferFromFile::seek(off_t offset, int whence) if (file_offset_of_buffer_end - working_buffer.size() <= new_pos && new_pos <= file_offset_of_buffer_end) { pos = working_buffer.end() - file_offset_of_buffer_end + new_pos; - assert(pos >= working_buffer.begin()); - assert(pos <= working_buffer.end()); + chassert(pos >= working_buffer.begin()); + chassert(pos <= working_buffer.end()); return new_pos; } } From dbae50b6db6f885daef4d97f00d3ef81b6104741 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 May 2023 14:50:59 +0200 Subject: [PATCH 0854/2223] Better logging --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 202914a0774..a60f5dffa96 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -365,8 +365,8 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegment & file_s else { LOG_TRACE( - log, - "Bypassing cache because file segment state is `PARTIALLY_DOWNLOADED_NO_CONTINUATION` and downloaded part already used"); + log, "Bypassing cache because file segment state is " + "`PARTIALLY_DOWNLOADED_NO_CONTINUATION` and downloaded part already used"); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; return getRemoteReadBuffer(file_segment, read_type); } @@ -466,8 +466,8 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegment & file_segme { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Buffer's offsets mismatch. Cached buffer offset: {}, current_write_offset: {}, implementation buffer position: {}, " - "implementation buffer end position: {}, file segment info: {}", + "Buffer's offsets mismatch. Cached buffer offset: {}, current_write_offset: {}, " + "implementation buffer position: {}, implementation buffer end position: {}, file segment info: {}", file_offset_of_buffer_end, current_write_offset, read_buffer_for_file_segment->getPosition(), @@ -932,9 +932,18 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); - [[maybe_unused]] size_t new_file_offset = file_offset_of_buffer_end + size; +#ifdef ABORT_ON_LOGICAL_ERROR + const size_t new_file_offset = file_offset_of_buffer_end + size; chassert(new_file_offset - 1 <= file_segment.range().right); - chassert(new_file_offset <= file_segment.getCurrentWriteOffset(true)); + const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true); + if (new_file_offset > file_segment_write_offset) + { + LOG_TRACE( + log, "Read {} bytes, file offset: {}, segment: {}, segment write offset: {}", + size, file_offset_of_buffer_end, file_segment.range().toString(), file_segment_write_offset); + chassert(false); + } +#endif } else { From f74d7474d2a95b7fb4856c5a0e541855b5cd7fdd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 31 May 2023 15:04:41 +0200 Subject: [PATCH 0855/2223] fix poll timeout in MaterializedMySQL --- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index aaf6b00dc4f..a01ab2a15a8 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -262,9 +262,13 @@ void MaterializedMySQLSyncThread::synchronization() try { - BinlogEventPtr binlog_event = client.readOneBinlogEvent(std::max(UInt64(1), max_flush_time - watch.elapsedMilliseconds())); - if (binlog_event) - onEvent(buffers, binlog_event, metadata); + UInt64 elapsed_ms = watch.elapsedMilliseconds(); + if (elapsed_ms < max_flush_time) + { + BinlogEventPtr binlog_event = client.readOneBinlogEvent(max_flush_time - elapsed_ms); + if (binlog_event) + onEvent(buffers, binlog_event, metadata); + } } catch (const Exception & e) { From 2efebee5a37b25b26898705ecd833a2c3091eaf9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 31 May 2023 16:24:36 +0300 Subject: [PATCH 0856/2223] Compare functions NaN update test (#50366) Co-authored-by: Nikita Mikhaylov --- ...e => 02769_compare_functions_nan.reference} | 1 + ...son.sql => 02769_compare_functions_nan.sql} | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+) rename tests/queries/0_stateless/{02769_nan_equality_comparison.reference => 02769_compare_functions_nan.reference} (96%) rename tests/queries/0_stateless/{02769_nan_equality_comparison.sql => 02769_compare_functions_nan.sql} (78%) diff --git a/tests/queries/0_stateless/02769_nan_equality_comparison.reference b/tests/queries/0_stateless/02769_compare_functions_nan.reference similarity index 96% rename from tests/queries/0_stateless/02769_nan_equality_comparison.reference rename to tests/queries/0_stateless/02769_compare_functions_nan.reference index a8ba06cfce6..81f0ee6da73 100644 --- a/tests/queries/0_stateless/02769_nan_equality_comparison.reference +++ b/tests/queries/0_stateless/02769_compare_functions_nan.reference @@ -8,3 +8,4 @@ nan 1 1 1 1 nan nan 1 1 1 1 -- nan +-- diff --git a/tests/queries/0_stateless/02769_nan_equality_comparison.sql b/tests/queries/0_stateless/02769_compare_functions_nan.sql similarity index 78% rename from tests/queries/0_stateless/02769_nan_equality_comparison.sql rename to tests/queries/0_stateless/02769_compare_functions_nan.sql index 6cce19a2204..1e1a9df9ce2 100644 --- a/tests/queries/0_stateless/02769_nan_equality_comparison.sql +++ b/tests/queries/0_stateless/02769_compare_functions_nan.sql @@ -7,11 +7,13 @@ SELECT nan AS lhs, cast(nan, 'Float32') AS rhs, lhs = rhs, lhs = materialize(rhs SELECT '--'; +DROP TABLE IF EXISTS test_table; CREATE TABLE test_table ( id UInt32, value UInt32 ) ENGINE = MergeTree ORDER BY id; + INSERT INTO test_table VALUES (76, 57); SELECT value FROM (SELECT stddevSamp(id) AS value FROM test_table) as subquery @@ -33,6 +35,7 @@ CREATE TABLE test_table value_1 UInt32, value_2 Float32 ) ENGINE = MergeTree ORDER BY id; + INSERT INTO test_table VALUES (12000, 36, 77.94); SELECT value @@ -40,3 +43,18 @@ FROM (SELECT (corr(value_1, value_1) OVER test_window) AS value FROM test_table WHERE not (not (value <> value)); DROP TABLE test_table; + +SELECT '--'; + +CREATE TABLE test_table +( + id Float32, + value Float32 +) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_table VALUES (-10.75, 95.57); + +SELECT * FROM (SELECT corr(id, id) as corr_value FROM test_table GROUP BY value) AS subquery LEFT ANTI JOIN test_table ON (subquery.corr_value = test_table.id) +WHERE (test_table.id >= test_table.id) AND (NOT (test_table.id >= test_table.id)); + +DROP TABLE test_table; From a96ee7411b0fdd28d9a77d127f74848b889a73f6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 31 May 2023 14:28:36 +0000 Subject: [PATCH 0857/2223] Bump From cf9954f17a6363c9bd4211d4797cd33772c3a54f Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 14:55:51 +0000 Subject: [PATCH 0858/2223] Remove pointless duplication of some queries in 00189_time_zones_long test --- .../00189_time_zones_long.reference | 16 --------------- .../0_stateless/00189_time_zones_long.sql | 20 ++++--------------- 2 files changed, 4 insertions(+), 32 deletions(-) diff --git a/tests/queries/0_stateless/00189_time_zones_long.reference b/tests/queries/0_stateless/00189_time_zones_long.reference index d41c925bbe5..11aebd71417 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.reference +++ b/tests/queries/0_stateless/00189_time_zones_long.reference @@ -16,10 +16,6 @@ toMonday 2014-12-29 2014-12-22 2014-12-29 -2014-12-29 -2014-12-29 -2014-12-29 -2014-12-29 toStartOfMonth 2014-12-01 2014-12-01 @@ -27,10 +23,6 @@ toStartOfMonth 2014-12-01 2014-12-01 2014-12-01 -2014-12-01 -2014-12-01 -2014-12-01 -2014-12-01 toStartOfQuarter 2014-07-01 2014-07-01 @@ -38,10 +30,6 @@ toStartOfQuarter 2014-10-01 2014-07-01 2014-07-01 -2014-07-01 -2014-07-01 -2014-07-01 -2014-07-01 toStartOfYear 2014-01-01 2014-01-01 @@ -49,10 +37,6 @@ toStartOfYear 2014-01-01 2014-01-01 2014-01-01 -2014-01-01 -2014-01-01 -2014-01-01 -2014-01-01 toTime 1970-01-02 11:00:00 1970-01-02 12:00:00 1970-01-02 10:00:00 1970-01-02 11:00:00 diff --git a/tests/queries/0_stateless/00189_time_zones_long.sql b/tests/queries/0_stateless/00189_time_zones_long.sql index 5760f6c0447..8c8de50abe7 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.sql +++ b/tests/queries/0_stateless/00189_time_zones_long.sql @@ -33,10 +33,7 @@ SELECT toMonday(toDateTime(1419800400), 'Europe/Paris'); SELECT toMonday(toDateTime(1419800400), 'Europe/London'); SELECT toMonday(toDateTime(1419800400), 'Asia/Tokyo'); SELECT toMonday(toDateTime(1419800400), 'Pacific/Pitcairn'); -SELECT toMonday(toDate(16433)); -SELECT toMonday(toDate(16433)); -SELECT toMonday(toDate(16433)); -SELECT toMonday(toDate(16433)); +SELECT toMonday(toDate(16433), 'Asia/Istanbul'); -- { serverError 43 } SELECT toMonday(toDate(16433)); /* toStartOfMonth */ @@ -47,10 +44,7 @@ SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Paris'); SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/London'); SELECT toStartOfMonth(toDateTime(1419800400), 'Asia/Tokyo'); SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn'); -SELECT toStartOfMonth(toDate(16433)); -SELECT toStartOfMonth(toDate(16433)); -SELECT toStartOfMonth(toDate(16433)); -SELECT toStartOfMonth(toDate(16433)); +SELECT toStartOfMonth(toDate(16433), 'Asia/Istanbul'); -- { serverError 43 } SELECT toStartOfMonth(toDate(16433)); /* toStartOfQuarter */ @@ -61,10 +55,7 @@ SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Paris'); SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/London'); SELECT toStartOfQuarter(toDateTime(1412106600), 'Asia/Tokyo'); SELECT toStartOfQuarter(toDateTime(1412106600), 'Pacific/Pitcairn'); -SELECT toStartOfQuarter(toDate(16343)); -SELECT toStartOfQuarter(toDate(16343)); -SELECT toStartOfQuarter(toDate(16343)); -SELECT toStartOfQuarter(toDate(16343)); +SELECT toStartOfQuarter(toDate(16343), 'Asia/Istanbul'); -- { serverError 43 } SELECT toStartOfQuarter(toDate(16343)); /* toStartOfYear */ @@ -75,10 +66,7 @@ SELECT toStartOfYear(toDateTime(1419800400), 'Europe/Paris'); SELECT toStartOfYear(toDateTime(1419800400), 'Europe/London'); SELECT toStartOfYear(toDateTime(1419800400), 'Asia/Tokyo'); SELECT toStartOfYear(toDateTime(1419800400), 'Pacific/Pitcairn'); -SELECT toStartOfYear(toDate(16433)); -SELECT toStartOfYear(toDate(16433)); -SELECT toStartOfYear(toDate(16433)); -SELECT toStartOfYear(toDate(16433)); +SELECT toStartOfYear(toDate(16433), 'Asia/Istanbul'); -- { serverError 43 } SELECT toStartOfYear(toDate(16433)); /* toTime */ From 570778dee0cd06ff74558e12ef55ebc117a4b305 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 15:08:25 +0000 Subject: [PATCH 0859/2223] Integrated toStartOfWeekand and toLastDayOfWeek in 00189_time_zones_long --- .../00189_time_zones_long.reference | 28 ++++++++++++ .../0_stateless/00189_time_zones_long.sql | 44 +++++++++++++++++++ 2 files changed, 72 insertions(+) diff --git a/tests/queries/0_stateless/00189_time_zones_long.reference b/tests/queries/0_stateless/00189_time_zones_long.reference index 11aebd71417..a4287217a19 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.reference +++ b/tests/queries/0_stateless/00189_time_zones_long.reference @@ -16,6 +16,34 @@ toMonday 2014-12-29 2014-12-22 2014-12-29 +toStartOfWeek (Sunday) +2014-12-28 +2014-12-28 +2014-12-28 +2014-12-28 +2014-12-28 +2014-12-28 +toStartOfWeek (Monday) +2014-12-22 +2014-12-22 +2014-12-22 +2014-12-29 +2014-12-22 +2014-12-29 +toLastDayOfWeek (Sunday) +2015-01-03 +2015-01-03 +2015-01-03 +2015-01-03 +2015-01-03 +2015-01-03 +toLastDayOfWeek (Monday) +2014-12-28 +2014-12-28 +2014-12-28 +2015-01-04 +2014-12-28 +2015-01-04 toStartOfMonth 2014-12-01 2014-12-01 diff --git a/tests/queries/0_stateless/00189_time_zones_long.sql b/tests/queries/0_stateless/00189_time_zones_long.sql index 8c8de50abe7..4785bee1482 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.sql +++ b/tests/queries/0_stateless/00189_time_zones_long.sql @@ -36,6 +36,50 @@ SELECT toMonday(toDateTime(1419800400), 'Pacific/Pitcairn'); SELECT toMonday(toDate(16433), 'Asia/Istanbul'); -- { serverError 43 } SELECT toMonday(toDate(16433)); +/* toStartOfWeek (Sunday) */ + +SELECT 'toStartOfWeek (Sunday)'; +SELECT toStartOfWeek(toDateTime(1419800400), 0, 'Asia/Istanbul'); +SELECT toStartOfWeek(toDateTime(1419800400), 0, 'Europe/Paris'); +SELECT toStartOfWeek(toDateTime(1419800400), 0, 'Europe/London'); +SELECT toStartOfWeek(toDateTime(1419800400), 0, 'Asia/Tokyo'); +SELECT toStartOfWeek(toDateTime(1419800400), 0, 'Pacific/Pitcairn'); +SELECT toStartOfWeek(toDate(16433), 0, 'Asia/Istanbul'); -- { serverError 43 } +SELECT toStartOfWeek(toDate(16433), 0); + +/* toStartOfWeek (Monday) */ + +SELECT 'toStartOfWeek (Monday)'; +SELECT toStartOfWeek(toDateTime(1419800400), 1, 'Asia/Istanbul'); +SELECT toStartOfWeek(toDateTime(1419800400), 1, 'Europe/Paris'); +SELECT toStartOfWeek(toDateTime(1419800400), 1, 'Europe/London'); +SELECT toStartOfWeek(toDateTime(1419800400), 1, 'Asia/Tokyo'); +SELECT toStartOfWeek(toDateTime(1419800400), 1, 'Pacific/Pitcairn'); +SELECT toStartOfWeek(toDate(16433), 1, 'Asia/Istanbul'); -- { serverError 43 } +SELECT toStartOfWeek(toDate(16433), 1); + +/* toLastDayOfWeek (Sunday) */ + +SELECT 'toLastDayOfWeek (Sunday)'; +SELECT toLastDayOfWeek(toDateTime(1419800400), 0, 'Asia/Istanbul'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 0, 'Europe/Paris'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 0, 'Europe/London'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 0, 'Asia/Tokyo'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 0, 'Pacific/Pitcairn'); +SELECT toLastDayOfWeek(toDate(16433), 0, 'Asia/Istanbul'); -- { serverError 43 } +SELECT toLastDayOfWeek(toDate(16433), 0); + +/* toLastDayOfWeek (Monday) */ + +SELECT 'toLastDayOfWeek (Monday)'; +SELECT toLastDayOfWeek(toDateTime(1419800400), 1, 'Asia/Istanbul'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 1, 'Europe/Paris'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 1, 'Europe/London'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 1, 'Asia/Tokyo'); +SELECT toLastDayOfWeek(toDateTime(1419800400), 1, 'Pacific/Pitcairn'); +SELECT toLastDayOfWeek(toDate(16433), 1, 'Asia/Istanbul'); -- { serverError 43 } +SELECT toLastDayOfWeek(toDate(16433), 1); + /* toStartOfMonth */ SELECT 'toStartOfMonth'; From a96a845e232c05601119df156972fba783696b21 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 31 May 2023 17:12:49 +0200 Subject: [PATCH 0860/2223] Add comment to prepare-ci-ami.sh, add a success at the end --- tests/ci/worker/{ubuntu_ami_for_ci.sh => prepare-ci-ami.sh} | 5 +++++ 1 file changed, 5 insertions(+) rename tests/ci/worker/{ubuntu_ami_for_ci.sh => prepare-ci-ami.sh} (93%) diff --git a/tests/ci/worker/ubuntu_ami_for_ci.sh b/tests/ci/worker/prepare-ci-ami.sh similarity index 93% rename from tests/ci/worker/ubuntu_ami_for_ci.sh rename to tests/ci/worker/prepare-ci-ami.sh index 2bb8f01535f..fca13266f09 100644 --- a/tests/ci/worker/ubuntu_ami_for_ci.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# The script is downloaded the AWS image builder Task Orchestrator and Executor (AWSTOE) +# We can't use `user data script` because cloud-init does not check the exit code set -xeuo pipefail echo "Running prepare script" @@ -121,3 +123,6 @@ gpg --verify /tmp/amazon-cloudwatch-agent.deb.sig dpkg -i /tmp/amazon-cloudwatch-agent.deb aws ssm get-parameter --region us-east-1 --name AmazonCloudWatch-github-runners --query 'Parameter.Value' --output text > /opt/aws/amazon-cloudwatch-agent/etc/amazon-cloudwatch-agent.json systemctl enable amazon-cloudwatch-agent.service + +# The following line is used in aws TOE check. +touch /var/tmp/clickhouse-ci-ami.success From 0d4ed32baca8eb2d897bcfd66eed0d04781af166 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 31 May 2023 11:25:33 -0400 Subject: [PATCH 0861/2223] better exception message --- src/Functions/FunctionsCodingIP.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsCodingIP.h b/src/Functions/FunctionsCodingIP.h index bd53fa7e043..9d090abb736 100644 --- a/src/Functions/FunctionsCodingIP.h +++ b/src/Functions/FunctionsCodingIP.h @@ -341,7 +341,11 @@ ColumnPtr convertIPv6ToIPv4(ColumnPtr column, const PaddedPODArray * null { if constexpr (exception_mode == IPStringToNumExceptionMode::Throw) { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", column->getName()); + char addr[IPV6_MAX_TEXT_LENGTH + 1] {}; + char * paddr = addr; + formatIPv6(src, paddr); + + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 {} in column {} is not in IPv4 mapping block", addr, column->getName()); } else if constexpr (exception_mode == IPStringToNumExceptionMode::Default) { From 3af7e0a6fa21d570f78fcf9366c299e3199d2b77 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 31 May 2023 11:26:58 -0400 Subject: [PATCH 0862/2223] better exception message --- src/Functions/FunctionsConversion.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 4d4efc84df1..6d22fb661c3 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -310,7 +310,13 @@ struct ConvertImpl const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); if (!matchIPv6Subnet(src, ip4_cidr, 96)) - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", named_from.column->getName()); + { + char addr[IPV6_MAX_TEXT_LENGTH + 1] {}; + char * paddr = addr; + formatIPv6(src, paddr); + + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 {} in column {} is not in IPv4 mapping block", addr, named_from.column->getName()); + } uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); if constexpr (std::endian::native == std::endian::little) From bed7443181eb543d8ebd4b6a47e85f9d400931df Mon Sep 17 00:00:00 2001 From: rfraposa Date: Wed, 31 May 2023 09:31:46 -0600 Subject: [PATCH 0863/2223] Fixes --- .../example-datasets/reddit-comments.md | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/docs/en/getting-started/example-datasets/reddit-comments.md b/docs/en/getting-started/example-datasets/reddit-comments.md index e1e372746c9..aa5c474103d 100644 --- a/docs/en/getting-started/example-datasets/reddit-comments.md +++ b/docs/en/getting-started/example-datasets/reddit-comments.md @@ -5,7 +5,7 @@ sidebar_label: Reddit comments # Reddit comments dataset -This dataset contains publicly-available comments on Reddit that go back to December, 2005, to March, 2023, and contains over 7B rows of data. The raw data is in JSON format in compressed `.zst` files and the rows look like the following: +This dataset contains publicly-available comments on Reddit that go back to December, 2005, to March, 2023, and contains over 14B rows of data. The raw data is in JSON format in compressed files and the rows look like the following: ```json {"controversiality":0,"body":"A look at Vietnam and Mexico exposes the myth of market liberalisation.","subreddit_id":"t5_6","link_id":"t3_17863","stickied":false,"subreddit":"reddit.com","score":2,"ups":2,"author_flair_css_class":null,"created_utc":1134365188,"author_flair_text":null,"author":"frjo","id":"c13","edited":false,"parent_id":"t3_17863","gilded":0,"distinguished":null,"retrieved_on":1473738411} @@ -75,18 +75,6 @@ The names of the files in S3 start with `RC_YYYY-MM` where `YYYY-MM` goes from ` 2. We are going to start with one month of data, but if you want to simply insert every row - skip ahead to step 8 below. The following file has 86M records from December, 2017: -```sql -INSERT INTO reddit - SELECT * - FROM s3Cluster( - 'default', - 'https://clickhouse-public-datasets.s3.eu-central-1.amazonaws.com/reddit/original/RC_2017-12.xz', - 'JSONEachRow' - ); -``` - -If you do not have a cluster, use `s3` instead of `s3Cluster`: - ```sql INSERT INTO reddit SELECT * @@ -94,6 +82,7 @@ INSERT INTO reddit 'https://clickhouse-public-datasets.s3.eu-central-1.amazonaws.com/reddit/original/RC_2017-12.xz', 'JSONEachRow' ); + ``` 3. It will take a while depending on your resources, but when it's done verify it worked: @@ -211,6 +200,18 @@ FROM s3Cluster( SETTINGS zstd_window_log_max = 31; ``` +If you do not have a cluster, use `s3` instead of `s3Cluster`: + +```sql +INSERT INTO reddit +SELECT * +FROM s3( + 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC*', + 'JSONEachRow' + ) +SETTINGS zstd_window_log_max = 31; +``` + The response looks like: ```response From aedd3afb8aa6127d24a7a84146be113f88936bed Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Wed, 31 May 2023 18:20:58 +0200 Subject: [PATCH 0864/2223] fix hung in unit tests (#50391) * fix hung in unit tests * Update gtest_writebuffer_s3.cpp * Update gtest_writebuffer_s3.cpp --------- Co-authored-by: Alexander Tokmakov --- src/IO/tests/gtest_writebuffer_s3.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index bc16af7f779..cd38291fb31 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -609,9 +609,16 @@ protected: test_with_pool = GetParam(); client = MockS3::Client::CreateClient(bucket); if (test_with_pool) + { + /// Do not block the main thread awaiting the others task. + /// This test use the only one thread at all + getSettings().s3_max_inflight_parts_for_one_file = 0; async_policy = std::make_unique(); + } else + { async_policy = std::make_unique(); + } } }; From 653da5f00219c088af66d97a8f1ea3e35e798268 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 31 May 2023 18:23:21 +0200 Subject: [PATCH 0865/2223] Reduce the unzip verbosity --- tests/ci/worker/prepare-ci-ami.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index fca13266f09..57ab149237a 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -102,7 +102,7 @@ chown -R ubuntu:ubuntu $RUNNER_HOME cd /home/ubuntu curl "https://awscli.amazonaws.com/awscli-exe-linux-$(uname -m).zip" -o "awscliv2.zip" -unzip awscliv2.zip +unzip -q awscliv2.zip ./aws/install rm -rf /home/ubuntu/awscliv2.zip /home/ubuntu/aws From adf71e706eab3f0bd2e2fc23aba13d6a0b9fb782 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 16:34:55 +0000 Subject: [PATCH 0866/2223] Intergate week related functions to 01472_toStartOfInterval_disallow_empty_tz_field test --- ...rtOfInterval_disallow_empty_tz_field.reference | 5 +++++ ..._toStartOfInterval_disallow_empty_tz_field.sql | 15 +++++++++++++++ 2 files changed, 20 insertions(+) diff --git a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference b/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference index c5218102aa4..83a1ac4f193 100644 --- a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference +++ b/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference @@ -1,4 +1,9 @@ 2017-12-31 00:00:00 +2017-12-25 +2017-12-31 +2017-12-25 +2018-01-06 +2017-12-31 2017-12-01 2017-10-01 2017-01-01 diff --git a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql b/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql index c2bdac2b279..47e81653b25 100644 --- a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql +++ b/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql @@ -1,6 +1,21 @@ SELECT toStartOfDay(toDateTime('2017-12-31 00:00:00', 'UTC'), ''); -- {serverError 43} SELECT toStartOfDay(toDateTime('2017-12-31 03:45:00', 'UTC'), 'UTC'); -- success +SELECT toMonday(toDateTime('2017-12-31 00:00:00', 'UTC'), ''); -- {serverError 43} +SELECT toMonday(toDateTime('2017-12-31 00:00:00', 'UTC'), 'UTC'); -- success + +SELECT toStartOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 0, ''); -- {serverError 43} +SELECT toStartOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 0, 'UTC'); -- success + +SELECT toStartOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 1, ''); -- {serverError 43} +SELECT toStartOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 1, 'UTC'); -- success + +SELECT toLastDayOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 0, ''); -- {serverError 43} +SELECT toLastDayOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 0, 'UTC'); -- success + +SELECT toLastDayOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 1, ''); -- {serverError 43} +SELECT toLastDayOfWeek(toDateTime('2017-12-31 00:00:00', 'UTC'), 1, 'UTC'); -- success + SELECT toStartOfMonth(toDateTime('2017-12-31 00:00:00', 'UTC'), ''); -- {serverError 43} SELECT toStartOfMonth(toDateTime('2017-12-31 00:00:00', 'UTC'), 'UTC'); -- success From 3936c4dc52f24b477cfb5bf4821bb17626bd69df Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 31 May 2023 16:41:26 +0000 Subject: [PATCH 0867/2223] Try to fix fast tests (add timezone) --- .../01746_convert_type_with_default.sql | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index 9fdd92491a7..75e1510f330 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -58,23 +58,23 @@ select toDateOrDefault(-1, '2000-01-01'::Date); select toDateTimeOrDefault('2023-05-30 14:38:20'); select toDateTimeOrDefault('2023-05-30 14:38:20', 'UTC'); -select toString(toDateTimeOrDefault('1xxx', 'Asia/Istanbul', '2023-05-30 14:38:20'::DateTime('Asia/Istanbul')), 'Asia/Istanbul'); -select toDateTimeOrDefault(1685457500); +select toDateTimeOrDefault('1xxx', 'UTC', '2023-05-30 14:38:20'::DateTime('UTC')); +select toDateTimeOrDefault(1685457500, 'UTC'); -select toDateTimeOrDefault(cast(19 as Int8)); -select toDateTimeOrDefault(cast(19 as UInt8)); +select toDateTimeOrDefault(cast(19 as Int8), 'UTC'); +select toDateTimeOrDefault(cast(19 as UInt8), 'UTC'); -select toDateTimeOrDefault(cast(19 as Int16)); -select toDateTimeOrDefault(cast(19 as UInt16)); +select toDateTimeOrDefault(cast(19 as Int16), 'UTC'); +select toDateTimeOrDefault(cast(19 as UInt16), 'UTC'); -select toDateTimeOrDefault(cast(19 as Int32)); -select toDateTimeOrDefault(cast(19 as UInt32)); +select toDateTimeOrDefault(cast(19 as Int32), 'UTC'); +select toDateTimeOrDefault(cast(19 as UInt32), 'UTC'); -select toDateTimeOrDefault(cast(19 as Int64)); -select toDateTimeOrDefault(cast(19 as UInt64)); +select toDateTimeOrDefault(cast(19 as Int64), 'UTC'); +select toDateTimeOrDefault(cast(19 as UInt64), 'UTC'); -select toDateTimeOrDefault(cast(19 as Int128)); -select toDateTimeOrDefault(cast(19 as UInt128)); +select toDateTimeOrDefault(cast(19 as Int128), 'UTC'); +select toDateTimeOrDefault(cast(19 as UInt128), 'UTC'); -select toDateTimeOrDefault(cast(19 as Int256)); -select toDateTimeOrDefault(cast(19 as UInt256)); \ No newline at end of file +select toDateTimeOrDefault(cast(19 as Int256), 'UTC'); +select toDateTimeOrDefault(cast(19 as UInt256), 'UTC'); \ No newline at end of file From 8c1f579c481747c1257c53adf674493aae35b2c1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 16:41:30 +0000 Subject: [PATCH 0868/2223] Add google-protobuf submodule --- .gitmodules | 3 +++ contrib/google-protobuf | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/google-protobuf diff --git a/.gitmodules b/.gitmodules index f0984fec4db..30777a42a9d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -344,3 +344,6 @@ [submodule "contrib/isa-l"] path = contrib/isa-l url = https://github.com/ClickHouse/isa-l.git +[submodule "contrib/google-protobuf"] + path = contrib/google-protobuf + url = https://github.com/ClickHouse/google-protobuf.git diff --git a/contrib/google-protobuf b/contrib/google-protobuf new file mode 160000 index 00000000000..315ffb5be89 --- /dev/null +++ b/contrib/google-protobuf @@ -0,0 +1 @@ +Subproject commit 315ffb5be89460f2857387d20aefc59b76b8bdc3 From 8c3e256caedbbf1dae3bd52cdddbd1b1a315e8ee Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 16:42:56 +0000 Subject: [PATCH 0869/2223] Switch protobuf to v3.18.x --- contrib/google-protobuf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/google-protobuf b/contrib/google-protobuf index 315ffb5be89..3b3d8fe1913 160000 --- a/contrib/google-protobuf +++ b/contrib/google-protobuf @@ -1 +1 @@ -Subproject commit 315ffb5be89460f2857387d20aefc59b76b8bdc3 +Subproject commit 3b3d8fe191314ea903ea6b072f0e73ef18e15faa From 7d8c1ff3cca5a59749b839e7fe23dc1e3bd9cac8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 16:43:36 +0000 Subject: [PATCH 0870/2223] Move protobuf entry in .gitmodules --- .gitmodules | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index 30777a42a9d..d28f205b65c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -39,6 +39,9 @@ path = contrib/protobuf url = https://github.com/ClickHouse/protobuf branch = v3.13.0.1 +[submodule "contrib/google-protobuf"] + path = contrib/google-protobuf + url = https://github.com/ClickHouse/google-protobuf.git [submodule "contrib/boost"] path = contrib/boost url = https://github.com/ClickHouse/boost @@ -344,6 +347,3 @@ [submodule "contrib/isa-l"] path = contrib/isa-l url = https://github.com/ClickHouse/isa-l.git -[submodule "contrib/google-protobuf"] - path = contrib/google-protobuf - url = https://github.com/ClickHouse/google-protobuf.git From ab6fe946bd4554d08f70808ee4c13699e9862069 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 16:47:26 +0000 Subject: [PATCH 0871/2223] Switch build to google-protobuf-cmake --- contrib/CMakeLists.txt | 4 +- contrib/google-protobuf-cmake/CMakeLists.txt | 329 ++++++++++++++++++ .../protobuf_generate.cmake | 198 +++++++++++ 3 files changed, 529 insertions(+), 2 deletions(-) create mode 100644 contrib/google-protobuf-cmake/CMakeLists.txt create mode 100644 contrib/google-protobuf-cmake/protobuf_generate.cmake diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 020fe1e1c5a..4a4ff9982ea 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -88,7 +88,7 @@ add_contrib (thrift-cmake thrift) # parquet/arrow/orc add_contrib (arrow-cmake arrow) # requires: snappy, thrift, double-conversion add_contrib (avro-cmake avro) # requires: snappy -add_contrib (protobuf-cmake protobuf) +add_contrib (google-protobuf-cmake google-protobuf) add_contrib (openldap-cmake openldap) add_contrib (grpc-cmake grpc) add_contrib (msgpack-c-cmake msgpack-c) @@ -156,7 +156,7 @@ add_contrib (libgsasl-cmake libgsasl) # requires krb5 add_contrib (librdkafka-cmake librdkafka) # requires: libgsasl add_contrib (nats-io-cmake nats-io) add_contrib (isa-l-cmake isa-l) -add_contrib (libhdfs3-cmake libhdfs3) # requires: protobuf, krb5, isa-l +add_contrib (libhdfs3-cmake libhdfs3) # requires: google-protobuf, krb5, isa-l add_contrib (hive-metastore-cmake hive-metastore) # requires: thrift/avro/arrow/libhdfs3 add_contrib (cppkafka-cmake cppkafka) add_contrib (libpqxx-cmake libpqxx) diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt new file mode 100644 index 00000000000..e2d38acb51d --- /dev/null +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -0,0 +1,329 @@ +option(ENABLE_PROTOBUF "Enable protobuf" ${ENABLE_LIBRARIES}) + +if(NOT ENABLE_PROTOBUF) + message(STATUS "Not using protobuf") + return() +endif() + +set(Protobuf_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/google-protobuf/src") +if(OS_FREEBSD AND SANITIZE STREQUAL "address") + # ../contrib/protobuf/src/google/protobuf/arena_impl.h:45:10: fatal error: 'sanitizer/asan_interface.h' file not found + # #include + if(LLVM_INCLUDE_DIRS) + set(Protobuf_INCLUDE_DIR "${Protobuf_INCLUDE_DIR}" ${LLVM_INCLUDE_DIRS}) + else() + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't use protobuf on FreeBSD with address sanitizer without LLVM") + return() + endif() +endif() + +set(protobuf_source_dir "${ClickHouse_SOURCE_DIR}/contrib/google-protobuf") +set(protobuf_binary_dir "${ClickHouse_BINARY_DIR}/contrib/google-protobuf") + + +add_definitions(-DGOOGLE_PROTOBUF_CMAKE_BUILD) + +add_definitions(-DHAVE_PTHREAD) +add_definitions(-DHAVE_ZLIB) + +include_directories( + ${protobuf_binary_dir} + ${protobuf_source_dir}/src) + +set(libprotobuf_lite_files + ${protobuf_source_dir}/src/google/protobuf/any_lite.cc + ${protobuf_source_dir}/src/google/protobuf/arena.cc + ${protobuf_source_dir}/src/google/protobuf/arenastring.cc + ${protobuf_source_dir}/src/google/protobuf/extension_set.cc + ${protobuf_source_dir}/src/google/protobuf/field_access_listener.cc + ${protobuf_source_dir}/src/google/protobuf/generated_enum_util.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven_lite.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_util.cc + ${protobuf_source_dir}/src/google/protobuf/implicit_weak_message.cc + ${protobuf_source_dir}/src/google/protobuf/io/coded_stream.cc + ${protobuf_source_dir}/src/google/protobuf/io/io_win32.cc + ${protobuf_source_dir}/src/google/protobuf/io/strtod.cc + ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream.cc + ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl.cc + ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl_lite.cc + ${protobuf_source_dir}/src/google/protobuf/map.cc + ${protobuf_source_dir}/src/google/protobuf/message_lite.cc + ${protobuf_source_dir}/src/google/protobuf/parse_context.cc + ${protobuf_source_dir}/src/google/protobuf/repeated_field.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/bytestream.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/common.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/int128.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/status.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/statusor.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/stringpiece.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/stringprintf.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/structurally_valid.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/strutil.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/time.cc + ${protobuf_source_dir}/src/google/protobuf/wire_format_lite.cc +) + +add_library(_libprotobuf-lite ${libprotobuf_lite_files}) +target_link_libraries(_libprotobuf-lite pthread) +if(${CMAKE_SYSTEM_NAME} STREQUAL "Android") + target_link_libraries(_libprotobuf-lite log) +endif() +target_include_directories(_libprotobuf-lite SYSTEM PUBLIC ${protobuf_source_dir}/src) +add_library(protobuf::libprotobuf-lite ALIAS _libprotobuf-lite) + + +set(libprotobuf_files + ${protobuf_source_dir}/src/google/protobuf/any.cc + ${protobuf_source_dir}/src/google/protobuf/any.pb.cc + ${protobuf_source_dir}/src/google/protobuf/api.pb.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/importer.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/parser.cc + ${protobuf_source_dir}/src/google/protobuf/descriptor.cc + ${protobuf_source_dir}/src/google/protobuf/descriptor.pb.cc + ${protobuf_source_dir}/src/google/protobuf/descriptor_database.cc + ${protobuf_source_dir}/src/google/protobuf/duration.pb.cc + ${protobuf_source_dir}/src/google/protobuf/dynamic_message.cc + ${protobuf_source_dir}/src/google/protobuf/empty.pb.cc + ${protobuf_source_dir}/src/google/protobuf/extension_set_heavy.cc + ${protobuf_source_dir}/src/google/protobuf/field_mask.pb.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_reflection.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven.cc + ${protobuf_source_dir}/src/google/protobuf/io/gzip_stream.cc + ${protobuf_source_dir}/src/google/protobuf/io/printer.cc + ${protobuf_source_dir}/src/google/protobuf/io/tokenizer.cc + ${protobuf_source_dir}/src/google/protobuf/map_field.cc + ${protobuf_source_dir}/src/google/protobuf/message.cc + ${protobuf_source_dir}/src/google/protobuf/reflection_ops.cc + ${protobuf_source_dir}/src/google/protobuf/service.cc + ${protobuf_source_dir}/src/google/protobuf/source_context.pb.cc + ${protobuf_source_dir}/src/google/protobuf/struct.pb.cc + ${protobuf_source_dir}/src/google/protobuf/stubs/substitute.cc + ${protobuf_source_dir}/src/google/protobuf/text_format.cc + ${protobuf_source_dir}/src/google/protobuf/timestamp.pb.cc + ${protobuf_source_dir}/src/google/protobuf/type.pb.cc + ${protobuf_source_dir}/src/google/protobuf/unknown_field_set.cc + ${protobuf_source_dir}/src/google/protobuf/util/delimited_message_util.cc + ${protobuf_source_dir}/src/google/protobuf/util/field_comparator.cc + ${protobuf_source_dir}/src/google/protobuf/util/field_mask_util.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/datapiece.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/default_value_objectwriter.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/error_listener.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/field_mask_utility.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/json_escaping.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/json_objectwriter.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/json_stream_parser.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/object_writer.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/proto_writer.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/protostream_objectsource.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/protostream_objectwriter.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/type_info.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/type_info_test_helper.cc + ${protobuf_source_dir}/src/google/protobuf/util/internal/utility.cc + ${protobuf_source_dir}/src/google/protobuf/util/json_util.cc + ${protobuf_source_dir}/src/google/protobuf/util/message_differencer.cc + ${protobuf_source_dir}/src/google/protobuf/util/time_util.cc + ${protobuf_source_dir}/src/google/protobuf/util/type_resolver_util.cc + ${protobuf_source_dir}/src/google/protobuf/wire_format.cc + ${protobuf_source_dir}/src/google/protobuf/wrappers.pb.cc +) + +add_library(_libprotobuf ${libprotobuf_lite_files} ${libprotobuf_files}) +if (ENABLE_FUZZING) + target_compile_options(_libprotobuf PRIVATE "-fsanitize-recover=all") +endif() +target_link_libraries(_libprotobuf pthread) +target_link_libraries(_libprotobuf ch_contrib::zlib) +if(${CMAKE_SYSTEM_NAME} STREQUAL "Android") + target_link_libraries(_libprotobuf log) +endif() +target_include_directories(_libprotobuf SYSTEM PUBLIC ${protobuf_source_dir}/src) +add_library(protobuf::libprotobuf ALIAS _libprotobuf) + + +set(libprotoc_files + ${protobuf_source_dir}/src/google/protobuf/compiler/code_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/command_line_interface.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_extension.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_file.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_padding_optimizer.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_parse_function_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_service.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_string_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_doc_comment.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_field_base.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_reflection_class.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_source_generator_base.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_wrapper_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_context.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_doc_comment.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_extension.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_extension_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_file.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_generator_factory.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_kotlin_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_map_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_builder.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_builder_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_name_resolver.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_primitive_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_service.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_shared_code_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_string_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_string_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/js/js_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/js/well_known_types_embed.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_extension.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_file.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_oneof.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/php/php_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.pb.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/python/python_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/ruby/ruby_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/subprocess.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/zip_writer.cc +) + +add_library(_libprotoc ${libprotoc_files}) +target_link_libraries(_libprotoc _libprotobuf) +add_library(protobuf::libprotoc ALIAS _libprotoc) + +set(protoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/main.cc) + +if (CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME + AND CMAKE_HOST_SYSTEM_PROCESSOR STREQUAL CMAKE_SYSTEM_PROCESSOR) + + add_executable(protoc ${protoc_files}) + target_link_libraries(protoc _libprotoc _libprotobuf pthread) + add_executable(protobuf::protoc ALIAS protoc) + + if (ENABLE_FUZZING) + # `protoc` will be built with sanitizer and it could fail during ClickHouse build + # It easily reproduces in oss-fuzz building pipeline + # To avoid this we can try to build `protoc` without any sanitizer with option `-fno-sanitize=all`, but + # it this case we will face with linker errors, because libcxx still will be built with sanitizer + # So, we can simply suppress all of these failures with a combination this flag and an environment variable + # export MSAN_OPTIONS=exit_code=0 + target_compile_options(protoc PRIVATE "-fsanitize-recover=all") + endif() +else () + # Build 'protoc' for host arch + set (PROTOC_BUILD_DIR "${protobuf_binary_dir}/build") + + if (NOT EXISTS "${PROTOC_BUILD_DIR}/protoc") + + # This is quite ugly but I cannot make dependencies work propery. + + execute_process( + COMMAND mkdir -p ${PROTOC_BUILD_DIR} + COMMAND_ECHO STDOUT) + + execute_process( + COMMAND ${CMAKE_COMMAND} + "-G${CMAKE_GENERATOR}" + "-DCMAKE_MAKE_PROGRAM=${CMAKE_MAKE_PROGRAM}" + "-DCMAKE_C_COMPILER=${CMAKE_C_COMPILER}" + "-DCMAKE_CXX_COMPILER=${CMAKE_CXX_COMPILER}" + "-Dprotobuf_BUILD_TESTS=0" + "-Dprotobuf_BUILD_CONFORMANCE=0" + "-Dprotobuf_BUILD_EXAMPLES=0" + "-Dprotobuf_BUILD_PROTOC_BINARIES=1" + "${protobuf_source_dir}/cmake" + WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" + COMMAND_ECHO STDOUT) + + execute_process( + COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" + COMMAND_ECHO STDOUT) + endif () + +# add_custom_command ( +# OUTPUT ${PROTOC_BUILD_DIR} +# COMMAND mkdir -p ${PROTOC_BUILD_DIR}) +# +# add_custom_command ( +# OUTPUT "${PROTOC_BUILD_DIR}/CMakeCache.txt" +# +# COMMAND ${CMAKE_COMMAND} +# -G"${CMAKE_GENERATOR}" +# -DCMAKE_MAKE_PROGRAM="${CMAKE_MAKE_PROGRAM}" +# -DCMAKE_C_COMPILER="${CMAKE_C_COMPILER}" +# -DCMAKE_CXX_COMPILER="${CMAKE_CXX_COMPILER}" +# -Dprotobuf_BUILD_TESTS=0 +# -Dprotobuf_BUILD_CONFORMANCE=0 +# -Dprotobuf_BUILD_EXAMPLES=0 +# -Dprotobuf_BUILD_PROTOC_BINARIES=1 +# "${protobuf_source_dir}/cmake" +# +# DEPENDS "${PROTOC_BUILD_DIR}" +# WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" +# COMMENT "Configuring 'protoc' for host architecture." +# USES_TERMINAL) +# +# add_custom_command ( +# OUTPUT "${PROTOC_BUILD_DIR}/protoc" +# COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" +# DEPENDS "${PROTOC_BUILD_DIR}/CMakeCache.txt" +# COMMENT "Building 'protoc' for host architecture." +# USES_TERMINAL) +# +# add_custom_target (protoc-host DEPENDS "${PROTOC_BUILD_DIR}/protoc") + + add_executable(protoc IMPORTED GLOBAL) + set_target_properties (protoc PROPERTIES IMPORTED_LOCATION "${PROTOC_BUILD_DIR}/protoc") + add_dependencies(protoc "${PROTOC_BUILD_DIR}/protoc") +endif () + +include("${ClickHouse_SOURCE_DIR}/contrib/google-protobuf-cmake/protobuf_generate.cmake") + +add_library(_protobuf INTERFACE) +target_link_libraries(_protobuf INTERFACE _libprotobuf) +target_include_directories(_protobuf INTERFACE "${Protobuf_INCLUDE_DIR}") +add_library(ch_contrib::protobuf ALIAS _protobuf) + +add_library(_protoc INTERFACE) +target_link_libraries(_protoc INTERFACE _libprotoc _libprotobuf) +target_include_directories(_protoc INTERFACE "${Protobuf_INCLUDE_DIR}") +add_library(ch_contrib::protoc ALIAS _protoc) diff --git a/contrib/google-protobuf-cmake/protobuf_generate.cmake b/contrib/google-protobuf-cmake/protobuf_generate.cmake new file mode 100644 index 00000000000..3e30b4e40fd --- /dev/null +++ b/contrib/google-protobuf-cmake/protobuf_generate.cmake @@ -0,0 +1,198 @@ +# The code in this file was copied from https://github.com/Kitware/CMake/blob/master/Modules/FindProtobuf.cmake + +#[[ +Add custom commands to process ``.proto`` files to C++:: + +protobuf_generate_cpp ( + [DESCRIPTORS ] [EXPORT_MACRO ] [...]) + +``SRCS`` + Variable to define with autogenerated source files +``HDRS`` + Variable to define with autogenerated header files +``DESCRIPTORS`` + Variable to define with autogenerated descriptor files, if requested. +``EXPORT_MACRO`` + is a macro which should expand to ``__declspec(dllexport)`` or + ``__declspec(dllimport)`` depending on what is being compiled. +``ARGN`` + ``.proto`` files +#]] + +function(PROTOBUF_GENERATE_CPP SRCS HDRS) + cmake_parse_arguments(protobuf_generate_cpp "" "EXPORT_MACRO;DESCRIPTORS" "" ${ARGN}) + + set(_proto_files "${protobuf_generate_cpp_UNPARSED_ARGUMENTS}") + if(NOT _proto_files) + message(SEND_ERROR "Error: PROTOBUF_GENERATE_CPP() called without any proto files") + return() + endif() + + if(PROTOBUF_GENERATE_CPP_APPEND_PATH) + set(_append_arg APPEND_PATH) + endif() + + if(protobuf_generate_cpp_DESCRIPTORS) + set(_descriptors DESCRIPTORS) + endif() + + if(DEFINED PROTOBUF_IMPORT_DIRS AND NOT DEFINED Protobuf_IMPORT_DIRS) + set(Protobuf_IMPORT_DIRS "${PROTOBUF_IMPORT_DIRS}") + endif() + + if(DEFINED Protobuf_IMPORT_DIRS) + set(_import_arg IMPORT_DIRS ${Protobuf_IMPORT_DIRS}) + endif() + + set(_outvar) + protobuf_generate(${_append_arg} ${_descriptors} LANGUAGE cpp EXPORT_MACRO ${protobuf_generate_cpp_EXPORT_MACRO} OUT_VAR _outvar ${_import_arg} PROTOS ${_proto_files}) + + set(${SRCS}) + set(${HDRS}) + if(protobuf_generate_cpp_DESCRIPTORS) + set(${protobuf_generate_cpp_DESCRIPTORS}) + endif() + + foreach(_file ${_outvar}) + if(_file MATCHES "cc$") + list(APPEND ${SRCS} ${_file}) + elseif(_file MATCHES "desc$") + list(APPEND ${protobuf_generate_cpp_DESCRIPTORS} ${_file}) + else() + list(APPEND ${HDRS} ${_file}) + endif() + endforeach() + set(${SRCS} ${${SRCS}} PARENT_SCOPE) + set(${HDRS} ${${HDRS}} PARENT_SCOPE) + if(protobuf_generate_cpp_DESCRIPTORS) + set(${protobuf_generate_cpp_DESCRIPTORS} "${${protobuf_generate_cpp_DESCRIPTORS}}" PARENT_SCOPE) + endif() +endfunction() + +# By default have PROTOBUF_GENERATE_CPP macro pass -I to protoc +# for each directory where a proto file is referenced. +if(NOT DEFINED PROTOBUF_GENERATE_CPP_APPEND_PATH) + set(PROTOBUF_GENERATE_CPP_APPEND_PATH TRUE) +endif() + +function(protobuf_generate) + set(_options APPEND_PATH DESCRIPTORS) + set(_singleargs LANGUAGE OUT_VAR EXPORT_MACRO PROTOC_OUT_DIR) + if(COMMAND target_sources) + list(APPEND _singleargs TARGET) + endif() + set(_multiargs PROTOS IMPORT_DIRS GENERATE_EXTENSIONS) + + cmake_parse_arguments(protobuf_generate "${_options}" "${_singleargs}" "${_multiargs}" "${ARGN}") + + if(NOT protobuf_generate_PROTOS AND NOT protobuf_generate_TARGET) + message(SEND_ERROR "Error: protobuf_generate called without any targets or source files") + return() + endif() + + if(NOT protobuf_generate_OUT_VAR AND NOT protobuf_generate_TARGET) + message(SEND_ERROR "Error: protobuf_generate called without a target or output variable") + return() + endif() + + if(NOT protobuf_generate_LANGUAGE) + set(protobuf_generate_LANGUAGE cpp) + endif() + string(TOLOWER ${protobuf_generate_LANGUAGE} protobuf_generate_LANGUAGE) + + if(NOT protobuf_generate_PROTOC_OUT_DIR) + set(protobuf_generate_PROTOC_OUT_DIR ${CMAKE_CURRENT_BINARY_DIR}) + endif() + + if(protobuf_generate_EXPORT_MACRO AND protobuf_generate_LANGUAGE STREQUAL cpp) + set(_dll_export_decl "dllexport_decl=${protobuf_generate_EXPORT_MACRO}:") + endif() + + if(NOT protobuf_generate_GENERATE_EXTENSIONS) + if(protobuf_generate_LANGUAGE STREQUAL cpp) + set(protobuf_generate_GENERATE_EXTENSIONS .pb.h .pb.cc) + elseif(protobuf_generate_LANGUAGE STREQUAL python) + set(protobuf_generate_GENERATE_EXTENSIONS _pb2.py) + else() + message(SEND_ERROR "Error: protobuf_generate given unknown Language ${LANGUAGE}, please provide a value for GENERATE_EXTENSIONS") + return() + endif() + endif() + + if(protobuf_generate_TARGET) + get_target_property(_source_list ${protobuf_generate_TARGET} SOURCES) + foreach(_file ${_source_list}) + if(_file MATCHES "proto$") + list(APPEND protobuf_generate_PROTOS ${_file}) + endif() + endforeach() + endif() + + if(NOT protobuf_generate_PROTOS) + message(SEND_ERROR "Error: protobuf_generate could not find any .proto files") + return() + endif() + + if(protobuf_generate_APPEND_PATH) + # Create an include path for each file specified + foreach(_file ${protobuf_generate_PROTOS}) + get_filename_component(_abs_file ${_file} ABSOLUTE) + get_filename_component(_abs_path ${_abs_file} PATH) + list(FIND _protobuf_include_path ${_abs_path} _contains_already) + if(${_contains_already} EQUAL -1) + list(APPEND _protobuf_include_path -I ${_abs_path}) + endif() + endforeach() + else() + set(_protobuf_include_path -I ${CMAKE_CURRENT_SOURCE_DIR}) + endif() + + foreach(DIR ${protobuf_generate_IMPORT_DIRS}) + get_filename_component(ABS_PATH ${DIR} ABSOLUTE) + list(FIND _protobuf_include_path ${ABS_PATH} _contains_already) + if(${_contains_already} EQUAL -1) + list(APPEND _protobuf_include_path -I ${ABS_PATH}) + endif() + endforeach() + + set(_generated_srcs_all) + foreach(_proto ${protobuf_generate_PROTOS}) + get_filename_component(_abs_file ${_proto} ABSOLUTE) + get_filename_component(_abs_dir ${_abs_file} DIRECTORY) + get_filename_component(_basename ${_proto} NAME_WE) + file(RELATIVE_PATH _rel_dir ${CMAKE_CURRENT_SOURCE_DIR} ${_abs_dir}) + + set(_possible_rel_dir) + if (NOT protobuf_generate_APPEND_PATH) + set(_possible_rel_dir ${_rel_dir}/) + endif() + + set(_generated_srcs) + foreach(_ext ${protobuf_generate_GENERATE_EXTENSIONS}) + list(APPEND _generated_srcs "${protobuf_generate_PROTOC_OUT_DIR}/${_possible_rel_dir}${_basename}${_ext}") + endforeach() + + if(protobuf_generate_DESCRIPTORS AND protobuf_generate_LANGUAGE STREQUAL cpp) + set(_descriptor_file "${CMAKE_CURRENT_BINARY_DIR}/${_basename}.desc") + set(_dll_desc_out "--descriptor_set_out=${_descriptor_file}") + list(APPEND _generated_srcs ${_descriptor_file}) + endif() + list(APPEND _generated_srcs_all ${_generated_srcs}) + + add_custom_command( + OUTPUT ${_generated_srcs} + COMMAND $ + ARGS --${protobuf_generate_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_PROTOC_OUT_DIR} ${_dll_desc_out} ${_protobuf_include_path} ${_abs_file} + DEPENDS ${_abs_file} protoc + COMMENT "Running ${protobuf_generate_LANGUAGE} protocol buffer compiler on ${_proto}" + VERBATIM) + endforeach() + + set_source_files_properties(${_generated_srcs_all} PROPERTIES GENERATED TRUE) + if(protobuf_generate_OUT_VAR) + set(${protobuf_generate_OUT_VAR} ${_generated_srcs_all} PARENT_SCOPE) + endif() + if(protobuf_generate_TARGET) + target_sources(${protobuf_generate_TARGET} PRIVATE ${_generated_srcs_all}) + endif() +endfunction() From 9d4c8c30ab7dc00f0f5f0930177856c933a767f5 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 31 May 2023 19:57:37 +0300 Subject: [PATCH 0872/2223] Rename 01472_toStartOfInterval_disallow_empty_tz_field test to 01472_toBoundsOfInterval_disallow_empty_tz_field --- ...=> 01472_toBoundsOfInterval_disallow_empty_tz_field.reference} | 0 ...d.sql => 01472_toBoundsOfInterval_disallow_empty_tz_field.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01472_toStartOfInterval_disallow_empty_tz_field.reference => 01472_toBoundsOfInterval_disallow_empty_tz_field.reference} (100%) rename tests/queries/0_stateless/{01472_toStartOfInterval_disallow_empty_tz_field.sql => 01472_toBoundsOfInterval_disallow_empty_tz_field.sql} (100%) diff --git a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference b/tests/queries/0_stateless/01472_toBoundsOfInterval_disallow_empty_tz_field.reference similarity index 100% rename from tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.reference rename to tests/queries/0_stateless/01472_toBoundsOfInterval_disallow_empty_tz_field.reference diff --git a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql b/tests/queries/0_stateless/01472_toBoundsOfInterval_disallow_empty_tz_field.sql similarity index 100% rename from tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql rename to tests/queries/0_stateless/01472_toBoundsOfInterval_disallow_empty_tz_field.sql From 495580918a285ff46ddb3fb91f3b66885b6e2138 Mon Sep 17 00:00:00 2001 From: alekar Date: Wed, 31 May 2023 10:00:19 -0700 Subject: [PATCH 0873/2223] Update base/base/getMemoryAmount.cpp Co-authored-by: Sergei Trifonov --- base/base/getMemoryAmount.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 6a5470a0549..8a2fcd092d2 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -29,7 +29,7 @@ uint64_t getMemoryAmountOrZero() #if defined(OS_LINUX) // Try to lookup at the Cgroup limit - // v2 + // CGroups v2 std::ifstream cgroupv2_limit("/sys/fs/cgroup/memory.max"); if (cgroupv2_limit.is_open()) { From cb85e5a01ea1e823229c312279c67b12a5d6d3db Mon Sep 17 00:00:00 2001 From: alekar Date: Wed, 31 May 2023 10:00:43 -0700 Subject: [PATCH 0874/2223] Update base/base/getMemoryAmount.cpp Co-authored-by: Sergei Trifonov --- base/base/getMemoryAmount.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 8a2fcd092d2..a46e964c5a3 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -40,7 +40,7 @@ uint64_t getMemoryAmountOrZero() } else { - // v1 + // CGroups v1 std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes"); if (cgroup_limit.is_open()) { From 57c88e664c856d44eb144416947d30ef19f5d073 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:01:00 +0000 Subject: [PATCH 0875/2223] Remove old protobuf --- .gitmodules | 4 - contrib/protobuf | 1 - contrib/protobuf-cmake/CMakeLists.txt | 329 ------------------ .../protobuf-cmake/protobuf_generate.cmake | 198 ----------- 4 files changed, 532 deletions(-) delete mode 160000 contrib/protobuf delete mode 100644 contrib/protobuf-cmake/CMakeLists.txt delete mode 100644 contrib/protobuf-cmake/protobuf_generate.cmake diff --git a/.gitmodules b/.gitmodules index d28f205b65c..e28d8257465 100644 --- a/.gitmodules +++ b/.gitmodules @@ -35,10 +35,6 @@ [submodule "contrib/unixodbc"] path = contrib/unixodbc url = https://github.com/ClickHouse/UnixODBC -[submodule "contrib/protobuf"] - path = contrib/protobuf - url = https://github.com/ClickHouse/protobuf - branch = v3.13.0.1 [submodule "contrib/google-protobuf"] path = contrib/google-protobuf url = https://github.com/ClickHouse/google-protobuf.git diff --git a/contrib/protobuf b/contrib/protobuf deleted file mode 160000 index 6bb70196c53..00000000000 --- a/contrib/protobuf +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 6bb70196c5360268d9f021bb7936fb0b551724c2 diff --git a/contrib/protobuf-cmake/CMakeLists.txt b/contrib/protobuf-cmake/CMakeLists.txt deleted file mode 100644 index 5e22136fc1f..00000000000 --- a/contrib/protobuf-cmake/CMakeLists.txt +++ /dev/null @@ -1,329 +0,0 @@ -option(ENABLE_PROTOBUF "Enable protobuf" ${ENABLE_LIBRARIES}) - -if(NOT ENABLE_PROTOBUF) - message(STATUS "Not using protobuf") - return() -endif() - -set(Protobuf_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") -if(OS_FREEBSD AND SANITIZE STREQUAL "address") - # ../contrib/protobuf/src/google/protobuf/arena_impl.h:45:10: fatal error: 'sanitizer/asan_interface.h' file not found - # #include - if(LLVM_INCLUDE_DIRS) - set(Protobuf_INCLUDE_DIR "${Protobuf_INCLUDE_DIR}" ${LLVM_INCLUDE_DIRS}) - else() - message(${RECONFIGURE_MESSAGE_LEVEL} "Can't use protobuf on FreeBSD with address sanitizer without LLVM") - return() - endif() -endif() - -set(protobuf_source_dir "${ClickHouse_SOURCE_DIR}/contrib/protobuf") -set(protobuf_binary_dir "${ClickHouse_BINARY_DIR}/contrib/protobuf") - - -add_definitions(-DGOOGLE_PROTOBUF_CMAKE_BUILD) - -add_definitions(-DHAVE_PTHREAD) -add_definitions(-DHAVE_ZLIB) - -include_directories( - ${protobuf_binary_dir} - ${protobuf_source_dir}/src) - -set(libprotobuf_lite_files - ${protobuf_source_dir}/src/google/protobuf/any_lite.cc - ${protobuf_source_dir}/src/google/protobuf/arena.cc - ${protobuf_source_dir}/src/google/protobuf/arenastring.cc - ${protobuf_source_dir}/src/google/protobuf/extension_set.cc - ${protobuf_source_dir}/src/google/protobuf/field_access_listener.cc - ${protobuf_source_dir}/src/google/protobuf/generated_enum_util.cc - ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven_lite.cc - ${protobuf_source_dir}/src/google/protobuf/generated_message_util.cc - ${protobuf_source_dir}/src/google/protobuf/implicit_weak_message.cc - ${protobuf_source_dir}/src/google/protobuf/io/coded_stream.cc - ${protobuf_source_dir}/src/google/protobuf/io/io_win32.cc - ${protobuf_source_dir}/src/google/protobuf/io/strtod.cc - ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream.cc - ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl.cc - ${protobuf_source_dir}/src/google/protobuf/io/zero_copy_stream_impl_lite.cc - ${protobuf_source_dir}/src/google/protobuf/map.cc - ${protobuf_source_dir}/src/google/protobuf/message_lite.cc - ${protobuf_source_dir}/src/google/protobuf/parse_context.cc - ${protobuf_source_dir}/src/google/protobuf/repeated_field.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/bytestream.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/common.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/int128.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/status.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/statusor.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/stringpiece.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/stringprintf.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/structurally_valid.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/strutil.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/time.cc - ${protobuf_source_dir}/src/google/protobuf/wire_format_lite.cc -) - -add_library(_libprotobuf-lite ${libprotobuf_lite_files}) -target_link_libraries(_libprotobuf-lite pthread) -if(${CMAKE_SYSTEM_NAME} STREQUAL "Android") - target_link_libraries(_libprotobuf-lite log) -endif() -target_include_directories(_libprotobuf-lite SYSTEM PUBLIC ${protobuf_source_dir}/src) -add_library(protobuf::libprotobuf-lite ALIAS _libprotobuf-lite) - - -set(libprotobuf_files - ${protobuf_source_dir}/src/google/protobuf/any.cc - ${protobuf_source_dir}/src/google/protobuf/any.pb.cc - ${protobuf_source_dir}/src/google/protobuf/api.pb.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/importer.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/parser.cc - ${protobuf_source_dir}/src/google/protobuf/descriptor.cc - ${protobuf_source_dir}/src/google/protobuf/descriptor.pb.cc - ${protobuf_source_dir}/src/google/protobuf/descriptor_database.cc - ${protobuf_source_dir}/src/google/protobuf/duration.pb.cc - ${protobuf_source_dir}/src/google/protobuf/dynamic_message.cc - ${protobuf_source_dir}/src/google/protobuf/empty.pb.cc - ${protobuf_source_dir}/src/google/protobuf/extension_set_heavy.cc - ${protobuf_source_dir}/src/google/protobuf/field_mask.pb.cc - ${protobuf_source_dir}/src/google/protobuf/generated_message_reflection.cc - ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven.cc - ${protobuf_source_dir}/src/google/protobuf/io/gzip_stream.cc - ${protobuf_source_dir}/src/google/protobuf/io/printer.cc - ${protobuf_source_dir}/src/google/protobuf/io/tokenizer.cc - ${protobuf_source_dir}/src/google/protobuf/map_field.cc - ${protobuf_source_dir}/src/google/protobuf/message.cc - ${protobuf_source_dir}/src/google/protobuf/reflection_ops.cc - ${protobuf_source_dir}/src/google/protobuf/service.cc - ${protobuf_source_dir}/src/google/protobuf/source_context.pb.cc - ${protobuf_source_dir}/src/google/protobuf/struct.pb.cc - ${protobuf_source_dir}/src/google/protobuf/stubs/substitute.cc - ${protobuf_source_dir}/src/google/protobuf/text_format.cc - ${protobuf_source_dir}/src/google/protobuf/timestamp.pb.cc - ${protobuf_source_dir}/src/google/protobuf/type.pb.cc - ${protobuf_source_dir}/src/google/protobuf/unknown_field_set.cc - ${protobuf_source_dir}/src/google/protobuf/util/delimited_message_util.cc - ${protobuf_source_dir}/src/google/protobuf/util/field_comparator.cc - ${protobuf_source_dir}/src/google/protobuf/util/field_mask_util.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/datapiece.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/default_value_objectwriter.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/error_listener.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/field_mask_utility.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/json_escaping.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/json_objectwriter.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/json_stream_parser.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/object_writer.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/proto_writer.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/protostream_objectsource.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/protostream_objectwriter.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/type_info.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/type_info_test_helper.cc - ${protobuf_source_dir}/src/google/protobuf/util/internal/utility.cc - ${protobuf_source_dir}/src/google/protobuf/util/json_util.cc - ${protobuf_source_dir}/src/google/protobuf/util/message_differencer.cc - ${protobuf_source_dir}/src/google/protobuf/util/time_util.cc - ${protobuf_source_dir}/src/google/protobuf/util/type_resolver_util.cc - ${protobuf_source_dir}/src/google/protobuf/wire_format.cc - ${protobuf_source_dir}/src/google/protobuf/wrappers.pb.cc -) - -add_library(_libprotobuf ${libprotobuf_lite_files} ${libprotobuf_files}) -if (ENABLE_FUZZING) - target_compile_options(_libprotobuf PRIVATE "-fsanitize-recover=all") -endif() -target_link_libraries(_libprotobuf pthread) -target_link_libraries(_libprotobuf ch_contrib::zlib) -if(${CMAKE_SYSTEM_NAME} STREQUAL "Android") - target_link_libraries(_libprotobuf log) -endif() -target_include_directories(_libprotobuf SYSTEM PUBLIC ${protobuf_source_dir}/src) -add_library(protobuf::libprotobuf ALIAS _libprotobuf) - - -set(libprotoc_files - ${protobuf_source_dir}/src/google/protobuf/compiler/code_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/command_line_interface.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_extension.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_file.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_padding_optimizer.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_parse_function_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_service.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_string_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_doc_comment.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_field_base.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_reflection_class.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_source_generator_base.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_wrapper_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_context.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_doc_comment.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_extension.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_extension_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_file.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_generator_factory.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_kotlin_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_map_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_builder.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_builder_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_name_resolver.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_primitive_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_service.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_shared_code_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_string_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_string_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/js/js_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/js/well_known_types_embed.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_extension.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_file.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_oneof.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/php/php_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.pb.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/python/python_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/ruby/ruby_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/subprocess.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/zip_writer.cc -) - -add_library(_libprotoc ${libprotoc_files}) -target_link_libraries(_libprotoc _libprotobuf) -add_library(protobuf::libprotoc ALIAS _libprotoc) - -set(protoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/main.cc) - -if (CMAKE_HOST_SYSTEM_NAME STREQUAL CMAKE_SYSTEM_NAME - AND CMAKE_HOST_SYSTEM_PROCESSOR STREQUAL CMAKE_SYSTEM_PROCESSOR) - - add_executable(protoc ${protoc_files}) - target_link_libraries(protoc _libprotoc _libprotobuf pthread) - add_executable(protobuf::protoc ALIAS protoc) - - if (ENABLE_FUZZING) - # `protoc` will be built with sanitizer and it could fail during ClickHouse build - # It easily reproduces in oss-fuzz building pipeline - # To avoid this we can try to build `protoc` without any sanitizer with option `-fno-sanitize=all`, but - # it this case we will face with linker errors, because libcxx still will be built with sanitizer - # So, we can simply suppress all of these failures with a combination this flag and an environment variable - # export MSAN_OPTIONS=exit_code=0 - target_compile_options(protoc PRIVATE "-fsanitize-recover=all") - endif() -else () - # Build 'protoc' for host arch - set (PROTOC_BUILD_DIR "${protobuf_binary_dir}/build") - - if (NOT EXISTS "${PROTOC_BUILD_DIR}/protoc") - - # This is quite ugly but I cannot make dependencies work propery. - - execute_process( - COMMAND mkdir -p ${PROTOC_BUILD_DIR} - COMMAND_ECHO STDOUT) - - execute_process( - COMMAND ${CMAKE_COMMAND} - "-G${CMAKE_GENERATOR}" - "-DCMAKE_MAKE_PROGRAM=${CMAKE_MAKE_PROGRAM}" - "-DCMAKE_C_COMPILER=${CMAKE_C_COMPILER}" - "-DCMAKE_CXX_COMPILER=${CMAKE_CXX_COMPILER}" - "-Dprotobuf_BUILD_TESTS=0" - "-Dprotobuf_BUILD_CONFORMANCE=0" - "-Dprotobuf_BUILD_EXAMPLES=0" - "-Dprotobuf_BUILD_PROTOC_BINARIES=1" - "${protobuf_source_dir}/cmake" - WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" - COMMAND_ECHO STDOUT) - - execute_process( - COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" - COMMAND_ECHO STDOUT) - endif () - -# add_custom_command ( -# OUTPUT ${PROTOC_BUILD_DIR} -# COMMAND mkdir -p ${PROTOC_BUILD_DIR}) -# -# add_custom_command ( -# OUTPUT "${PROTOC_BUILD_DIR}/CMakeCache.txt" -# -# COMMAND ${CMAKE_COMMAND} -# -G"${CMAKE_GENERATOR}" -# -DCMAKE_MAKE_PROGRAM="${CMAKE_MAKE_PROGRAM}" -# -DCMAKE_C_COMPILER="${CMAKE_C_COMPILER}" -# -DCMAKE_CXX_COMPILER="${CMAKE_CXX_COMPILER}" -# -Dprotobuf_BUILD_TESTS=0 -# -Dprotobuf_BUILD_CONFORMANCE=0 -# -Dprotobuf_BUILD_EXAMPLES=0 -# -Dprotobuf_BUILD_PROTOC_BINARIES=1 -# "${protobuf_source_dir}/cmake" -# -# DEPENDS "${PROTOC_BUILD_DIR}" -# WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" -# COMMENT "Configuring 'protoc' for host architecture." -# USES_TERMINAL) -# -# add_custom_command ( -# OUTPUT "${PROTOC_BUILD_DIR}/protoc" -# COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" -# DEPENDS "${PROTOC_BUILD_DIR}/CMakeCache.txt" -# COMMENT "Building 'protoc' for host architecture." -# USES_TERMINAL) -# -# add_custom_target (protoc-host DEPENDS "${PROTOC_BUILD_DIR}/protoc") - - add_executable(protoc IMPORTED GLOBAL) - set_target_properties (protoc PROPERTIES IMPORTED_LOCATION "${PROTOC_BUILD_DIR}/protoc") - add_dependencies(protoc "${PROTOC_BUILD_DIR}/protoc") -endif () - -include("${ClickHouse_SOURCE_DIR}/contrib/protobuf-cmake/protobuf_generate.cmake") - -add_library(_protobuf INTERFACE) -target_link_libraries(_protobuf INTERFACE _libprotobuf) -target_include_directories(_protobuf INTERFACE "${Protobuf_INCLUDE_DIR}") -add_library(ch_contrib::protobuf ALIAS _protobuf) - -add_library(_protoc INTERFACE) -target_link_libraries(_protoc INTERFACE _libprotoc _libprotobuf) -target_include_directories(_protoc INTERFACE "${Protobuf_INCLUDE_DIR}") -add_library(ch_contrib::protoc ALIAS _protoc) diff --git a/contrib/protobuf-cmake/protobuf_generate.cmake b/contrib/protobuf-cmake/protobuf_generate.cmake deleted file mode 100644 index 3e30b4e40fd..00000000000 --- a/contrib/protobuf-cmake/protobuf_generate.cmake +++ /dev/null @@ -1,198 +0,0 @@ -# The code in this file was copied from https://github.com/Kitware/CMake/blob/master/Modules/FindProtobuf.cmake - -#[[ -Add custom commands to process ``.proto`` files to C++:: - -protobuf_generate_cpp ( - [DESCRIPTORS ] [EXPORT_MACRO ] [...]) - -``SRCS`` - Variable to define with autogenerated source files -``HDRS`` - Variable to define with autogenerated header files -``DESCRIPTORS`` - Variable to define with autogenerated descriptor files, if requested. -``EXPORT_MACRO`` - is a macro which should expand to ``__declspec(dllexport)`` or - ``__declspec(dllimport)`` depending on what is being compiled. -``ARGN`` - ``.proto`` files -#]] - -function(PROTOBUF_GENERATE_CPP SRCS HDRS) - cmake_parse_arguments(protobuf_generate_cpp "" "EXPORT_MACRO;DESCRIPTORS" "" ${ARGN}) - - set(_proto_files "${protobuf_generate_cpp_UNPARSED_ARGUMENTS}") - if(NOT _proto_files) - message(SEND_ERROR "Error: PROTOBUF_GENERATE_CPP() called without any proto files") - return() - endif() - - if(PROTOBUF_GENERATE_CPP_APPEND_PATH) - set(_append_arg APPEND_PATH) - endif() - - if(protobuf_generate_cpp_DESCRIPTORS) - set(_descriptors DESCRIPTORS) - endif() - - if(DEFINED PROTOBUF_IMPORT_DIRS AND NOT DEFINED Protobuf_IMPORT_DIRS) - set(Protobuf_IMPORT_DIRS "${PROTOBUF_IMPORT_DIRS}") - endif() - - if(DEFINED Protobuf_IMPORT_DIRS) - set(_import_arg IMPORT_DIRS ${Protobuf_IMPORT_DIRS}) - endif() - - set(_outvar) - protobuf_generate(${_append_arg} ${_descriptors} LANGUAGE cpp EXPORT_MACRO ${protobuf_generate_cpp_EXPORT_MACRO} OUT_VAR _outvar ${_import_arg} PROTOS ${_proto_files}) - - set(${SRCS}) - set(${HDRS}) - if(protobuf_generate_cpp_DESCRIPTORS) - set(${protobuf_generate_cpp_DESCRIPTORS}) - endif() - - foreach(_file ${_outvar}) - if(_file MATCHES "cc$") - list(APPEND ${SRCS} ${_file}) - elseif(_file MATCHES "desc$") - list(APPEND ${protobuf_generate_cpp_DESCRIPTORS} ${_file}) - else() - list(APPEND ${HDRS} ${_file}) - endif() - endforeach() - set(${SRCS} ${${SRCS}} PARENT_SCOPE) - set(${HDRS} ${${HDRS}} PARENT_SCOPE) - if(protobuf_generate_cpp_DESCRIPTORS) - set(${protobuf_generate_cpp_DESCRIPTORS} "${${protobuf_generate_cpp_DESCRIPTORS}}" PARENT_SCOPE) - endif() -endfunction() - -# By default have PROTOBUF_GENERATE_CPP macro pass -I to protoc -# for each directory where a proto file is referenced. -if(NOT DEFINED PROTOBUF_GENERATE_CPP_APPEND_PATH) - set(PROTOBUF_GENERATE_CPP_APPEND_PATH TRUE) -endif() - -function(protobuf_generate) - set(_options APPEND_PATH DESCRIPTORS) - set(_singleargs LANGUAGE OUT_VAR EXPORT_MACRO PROTOC_OUT_DIR) - if(COMMAND target_sources) - list(APPEND _singleargs TARGET) - endif() - set(_multiargs PROTOS IMPORT_DIRS GENERATE_EXTENSIONS) - - cmake_parse_arguments(protobuf_generate "${_options}" "${_singleargs}" "${_multiargs}" "${ARGN}") - - if(NOT protobuf_generate_PROTOS AND NOT protobuf_generate_TARGET) - message(SEND_ERROR "Error: protobuf_generate called without any targets or source files") - return() - endif() - - if(NOT protobuf_generate_OUT_VAR AND NOT protobuf_generate_TARGET) - message(SEND_ERROR "Error: protobuf_generate called without a target or output variable") - return() - endif() - - if(NOT protobuf_generate_LANGUAGE) - set(protobuf_generate_LANGUAGE cpp) - endif() - string(TOLOWER ${protobuf_generate_LANGUAGE} protobuf_generate_LANGUAGE) - - if(NOT protobuf_generate_PROTOC_OUT_DIR) - set(protobuf_generate_PROTOC_OUT_DIR ${CMAKE_CURRENT_BINARY_DIR}) - endif() - - if(protobuf_generate_EXPORT_MACRO AND protobuf_generate_LANGUAGE STREQUAL cpp) - set(_dll_export_decl "dllexport_decl=${protobuf_generate_EXPORT_MACRO}:") - endif() - - if(NOT protobuf_generate_GENERATE_EXTENSIONS) - if(protobuf_generate_LANGUAGE STREQUAL cpp) - set(protobuf_generate_GENERATE_EXTENSIONS .pb.h .pb.cc) - elseif(protobuf_generate_LANGUAGE STREQUAL python) - set(protobuf_generate_GENERATE_EXTENSIONS _pb2.py) - else() - message(SEND_ERROR "Error: protobuf_generate given unknown Language ${LANGUAGE}, please provide a value for GENERATE_EXTENSIONS") - return() - endif() - endif() - - if(protobuf_generate_TARGET) - get_target_property(_source_list ${protobuf_generate_TARGET} SOURCES) - foreach(_file ${_source_list}) - if(_file MATCHES "proto$") - list(APPEND protobuf_generate_PROTOS ${_file}) - endif() - endforeach() - endif() - - if(NOT protobuf_generate_PROTOS) - message(SEND_ERROR "Error: protobuf_generate could not find any .proto files") - return() - endif() - - if(protobuf_generate_APPEND_PATH) - # Create an include path for each file specified - foreach(_file ${protobuf_generate_PROTOS}) - get_filename_component(_abs_file ${_file} ABSOLUTE) - get_filename_component(_abs_path ${_abs_file} PATH) - list(FIND _protobuf_include_path ${_abs_path} _contains_already) - if(${_contains_already} EQUAL -1) - list(APPEND _protobuf_include_path -I ${_abs_path}) - endif() - endforeach() - else() - set(_protobuf_include_path -I ${CMAKE_CURRENT_SOURCE_DIR}) - endif() - - foreach(DIR ${protobuf_generate_IMPORT_DIRS}) - get_filename_component(ABS_PATH ${DIR} ABSOLUTE) - list(FIND _protobuf_include_path ${ABS_PATH} _contains_already) - if(${_contains_already} EQUAL -1) - list(APPEND _protobuf_include_path -I ${ABS_PATH}) - endif() - endforeach() - - set(_generated_srcs_all) - foreach(_proto ${protobuf_generate_PROTOS}) - get_filename_component(_abs_file ${_proto} ABSOLUTE) - get_filename_component(_abs_dir ${_abs_file} DIRECTORY) - get_filename_component(_basename ${_proto} NAME_WE) - file(RELATIVE_PATH _rel_dir ${CMAKE_CURRENT_SOURCE_DIR} ${_abs_dir}) - - set(_possible_rel_dir) - if (NOT protobuf_generate_APPEND_PATH) - set(_possible_rel_dir ${_rel_dir}/) - endif() - - set(_generated_srcs) - foreach(_ext ${protobuf_generate_GENERATE_EXTENSIONS}) - list(APPEND _generated_srcs "${protobuf_generate_PROTOC_OUT_DIR}/${_possible_rel_dir}${_basename}${_ext}") - endforeach() - - if(protobuf_generate_DESCRIPTORS AND protobuf_generate_LANGUAGE STREQUAL cpp) - set(_descriptor_file "${CMAKE_CURRENT_BINARY_DIR}/${_basename}.desc") - set(_dll_desc_out "--descriptor_set_out=${_descriptor_file}") - list(APPEND _generated_srcs ${_descriptor_file}) - endif() - list(APPEND _generated_srcs_all ${_generated_srcs}) - - add_custom_command( - OUTPUT ${_generated_srcs} - COMMAND $ - ARGS --${protobuf_generate_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_PROTOC_OUT_DIR} ${_dll_desc_out} ${_protobuf_include_path} ${_abs_file} - DEPENDS ${_abs_file} protoc - COMMENT "Running ${protobuf_generate_LANGUAGE} protocol buffer compiler on ${_proto}" - VERBATIM) - endforeach() - - set_source_files_properties(${_generated_srcs_all} PROPERTIES GENERATED TRUE) - if(protobuf_generate_OUT_VAR) - set(${protobuf_generate_OUT_VAR} ${_generated_srcs_all} PARENT_SCOPE) - endif() - if(protobuf_generate_TARGET) - target_sources(${protobuf_generate_TARGET} PRIVATE ${_generated_srcs_all}) - endif() -endfunction() From f57c5105f6318573107ab6ae0d01fdccb757cf73 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:02:05 +0000 Subject: [PATCH 0876/2223] Remove file deleted upstream from build description --- contrib/google-protobuf-cmake/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index e2d38acb51d..2d72ac90f49 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -35,7 +35,6 @@ set(libprotobuf_lite_files ${protobuf_source_dir}/src/google/protobuf/arena.cc ${protobuf_source_dir}/src/google/protobuf/arenastring.cc ${protobuf_source_dir}/src/google/protobuf/extension_set.cc - ${protobuf_source_dir}/src/google/protobuf/field_access_listener.cc ${protobuf_source_dir}/src/google/protobuf/generated_enum_util.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven_lite.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_util.cc From 2e2f98ced69f0c6205292cee1290cab6795914c5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:26:31 +0000 Subject: [PATCH 0877/2223] Add new source file --- contrib/google-protobuf-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index 2d72ac90f49..8afb86b25dd 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -85,6 +85,7 @@ set(libprotobuf_files ${protobuf_source_dir}/src/google/protobuf/empty.pb.cc ${protobuf_source_dir}/src/google/protobuf/extension_set_heavy.cc ${protobuf_source_dir}/src/google/protobuf/field_mask.pb.cc + ${protobuf_source_dir}/src/google/protobuf/generated_message_bases.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_reflection.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven.cc ${protobuf_source_dir}/src/google/protobuf/io/gzip_stream.cc From 0b62be649f9974a4433897b39fb8a59e9e7f30f2 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 31 May 2023 17:52:29 +0000 Subject: [PATCH 0878/2223] Add docs, fix style --- .../table-engines/integrations/hdfs.md | 6 ++ .../engines/table-engines/integrations/s3.md | 6 ++ docs/en/engines/table-engines/special/file.md | 8 ++ docs/en/engines/table-engines/special/url.md | 4 + docs/en/operations/settings/settings.md | 92 ++++++++++++++----- docs/en/sql-reference/table-functions/file.md | 12 ++- docs/en/sql-reference/table-functions/hdfs.md | 6 ++ docs/en/sql-reference/table-functions/s3.md | 6 ++ docs/en/sql-reference/table-functions/url.md | 4 + src/Storages/StorageURL.cpp | 4 - tests/integration/test_storage_s3/test.py | 20 ++-- 11 files changed, 130 insertions(+), 38 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index b9db0fae68f..b37ccb00ba6 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -233,6 +233,12 @@ libhdfs3 support HDFS namenode HA. - `_path` — Path to the file. - `_file` — Name of the file. +## Storage Settings {#storage-settings} + +- [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. +- [hdfs_create_multiple_files](/docs/en/operations/settings/settings.md#hdfs_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. +- [hdfs_skip_empty_files](/docs/en/operations/settings/settings.md#hdfs_skip_empty_files) - allows to skip empty files while reading. Disabled by default. + **See Also** - [Virtual columns](../../../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 595bc0c344f..e8d0ab6255d 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -127,6 +127,12 @@ CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = S3('https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/{some,another}_folder/*', 'CSV'); ``` +## Storage Settings {#storage-settings} + +- [s3_truncate_on_insert](/docs/en/operations/settings/settings.md#s3-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. +- [s3_create_multiple_files](/docs/en/operations/settings/settings.md#s3_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. +- [s3_skip_empty_files](/docs/en/operations/settings/settings.md#s3_skip_empty_files) - allows to skip empty files while reading. Disabled by default. + ## S3-related Settings {#settings} The following settings can be set before query execution or placed into configuration file. diff --git a/docs/en/engines/table-engines/special/file.md b/docs/en/engines/table-engines/special/file.md index 9c4e87487b4..cf325961b6a 100644 --- a/docs/en/engines/table-engines/special/file.md +++ b/docs/en/engines/table-engines/special/file.md @@ -92,3 +92,11 @@ $ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64 `PARTITION BY` — Optional. It is possible to create separate files by partitioning the data on a partition key. In most cases, you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression). For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. + +## Settings {#settings} + +- [engine_file_empty_if_not_exists](/docs/en/operations/settings/settings.md#engine-file-emptyif-not-exists) - allows to select empty data from a file that doesn't exist. Disabled by default. +- [engine_file_truncate_on_insert](/docs/en/operations/settings/settings.md#engine-file-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. +- [engine_file_allow_create_multiple_files](/docs/en/operations/settings/settings.md#engine_file_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. +- [engine_file_skip_empty_files](/docs/en/operations/settings/settings.md#engine_file_skip_empty_files) - allows to skip empty files while reading. Disabled by default. +- [storage_file_read_method](/docs/en/operations/settings/settings.md#engine-file-emptyif-not-exists) - method of reading data from storage file, one of: read, pread, mmap (only for clickhouse-local). Default value: `pread` for clickhouse-server, `mmap` for clickhouse-local. diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index a4530767e11..26d4975954f 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -102,3 +102,7 @@ SELECT * FROM url_engine_table `PARTITION BY` — Optional. It is possible to create separate files by partitioning the data on a partition key. In most cases, you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression). For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. + +## Storage Settings {#storage-settings} + +- [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4f3b4e43358..ac3e624387e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3195,27 +3195,7 @@ Possible values: Default value: `0`. -## s3_truncate_on_insert - -Enables or disables truncate before inserts in s3 engine tables. If disabled, an exception will be thrown on insert attempts if an S3 object already exists. - -Possible values: -- 0 — `INSERT` query appends new data to the end of the file. -- 1 — `INSERT` query replaces existing content of the file with the new data. - -Default value: `0`. - -## hdfs_truncate_on_insert - -Enables or disables truncation before an insert in hdfs engine tables. If disabled, an exception will be thrown on an attempt to insert if a file in HDFS already exists. - -Possible values: -- 0 — `INSERT` query appends new data to the end of the file. -- 1 — `INSERT` query replaces existing content of the file with the new data. - -Default value: `0`. - -## engine_file_allow_create_multiple_files +## engine_file_allow_create_multiple_files {#engine_file_allow_create_multiple_files} Enables or disables creating a new file on each insert in file engine tables if the format has the suffix (`JSON`, `ORC`, `Parquet`, etc.). If enabled, on each insert a new file will be created with a name following this pattern: @@ -3227,7 +3207,33 @@ Possible values: Default value: `0`. -## s3_create_new_file_on_insert +## engine_file_skip_empty_files {#engine_file_skip_empty_files} + +Enables or disables skipping empty files in [File](../../engines/table-engines/special/file.md) engine tables. + +Possible values: +- 0 — `SELECT` throws an exception if empty file is not compatible with requested format. +- 1 — `SELECT` returns empty result for empty file. + +Default value: `0`. + +## storage_file_read_method {#storage_file_read_method} + +Method of reading data from storage file, one of: `read`, `pread`, `mmap`. The mmap method does not apply to clickhouse-server (it's intended for clickhouse-local). + +Default value: `pread` for clickhouse-server, `mmap` for clickhouse-local. + +## s3_truncate_on_insert {#s3_truncate_on_insert} + +Enables or disables truncate before inserts in s3 engine tables. If disabled, an exception will be thrown on insert attempts if an S3 object already exists. + +Possible values: +- 0 — `INSERT` query appends new data to the end of the file. +- 1 — `INSERT` query replaces existing content of the file with the new data. + +Default value: `0`. + +## s3_create_new_file_on_insert {#s3_create_new_file_on_insert} Enables or disables creating a new file on each insert in s3 engine tables. If enabled, on each insert a new S3 object will be created with the key, similar to this pattern: @@ -3239,7 +3245,27 @@ Possible values: Default value: `0`. -## hdfs_create_new_file_on_insert +## s3_skip_empty_files {#s3_skip_empty_files} + +Enables or disables skipping empty files in [S3](../../engines/table-engines/special/s3.md) engine tables. + +Possible values: +- 0 — `SELECT` throws an exception if empty file is not compatible with requested format. +- 1 — `SELECT` returns empty result for empty file. + +Default value: `0`. + +## hdfs_truncate_on_insert {#hdfs_truncate_on_insert} + +Enables or disables truncation before an insert in hdfs engine tables. If disabled, an exception will be thrown on an attempt to insert if a file in HDFS already exists. + +Possible values: +- 0 — `INSERT` query appends new data to the end of the file. +- 1 — `INSERT` query replaces existing content of the file with the new data. + +Default value: `0`. + +## hdfs_create_new_file_on_insert {#hdfs_create_new_file_on_insert Enables or disables creating a new file on each insert in HDFS engine tables. If enabled, on each insert a new HDFS file will be created with the name, similar to this pattern: @@ -3251,6 +3277,26 @@ Possible values: Default value: `0`. +## hdfs_skip_empty_files {#hdfs_skip_empty_files} + +Enables or disables skipping empty files in [HDFS](../../engines/table-engines/special/hdfs.md) engine tables. + +Possible values: +- 0 — `SELECT` throws an exception if empty file is not compatible with requested format. +- 1 — `SELECT` returns empty result for empty file. + +Default value: `0`. + +## engine_url_skip_empty_files {#engine_url_skip_empty_files} + +Enables or disables skipping empty files in [URL](../../engines/table-engines/special/url.md) engine tables. + +Possible values: +- 0 — `SELECT` throws an exception if empty file is not compatible with requested format. +- 1 — `SELECT` returns empty result for empty file. + +Default value: `0`. + ## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously} Adds a modifier `SYNC` to all `DROP` and `DETACH` queries. diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 28c2dc9f1f3..b1903c990b1 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -18,7 +18,7 @@ file(path [,format] [,structure] [,compression]) **Parameters** -- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-user_files_path). Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. - `format` — The [format](/docs/en/interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format: `'column1_name column1_type, column2_name column2_type, ...'`. - `compression` — The existing compression type when used in a `SELECT` query, or the desired compression type when used in an `INSERT` query. The supported compression types are `gz`, `br`, `xz`, `zst`, `lz4`, and `bz2`. @@ -196,6 +196,16 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 - `_path` — Path to the file. - `_file` — Name of the file. +## Settings + +- [engine_file_empty_if_not_exists](/docs/en/operations/settings/settings.md#engine-file-emptyif-not-exists) - allows to select empty data from a file that doesn't exist. Disabled by default. +- [engine_file_truncate_on_insert](/docs/en/operations/settings/settings.md#engine-file-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. +- [engine_file_allow_create_multiple_files](/docs/en/operations/settings/settings.md#engine_file_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. +- [engine_file_skip_empty_files](/docs/en/operations/settings/settings.md#engine_file_skip_empty_files) - allows to skip empty files while reading. Disabled by default. +- [storage_file_read_method](/docs/en/operations/settings/settings.md#engine-file-emptyif-not-exists) - method of reading data from storage file, one of: read, pread, mmap (only for clickhouse-local). Default value: `pread` for clickhouse-server, `mmap` for clickhouse-local. + + + **See Also** - [Virtual columns](/docs/en/engines/table-engines/index.md#table_engines-virtual_columns) diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 6ba24211131..1b52e786de4 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -97,6 +97,12 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin - `_path` — Path to the file. - `_file` — Name of the file. +## Storage Settings {#storage-settings} + +- [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. +- [hdfs_create_multiple_files](/docs/en/operations/settings/settings.md#hdfs_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. +- [hdfs_skip_empty_files](/docs/en/operations/settings/settings.md#hdfs_skip_empty_files) - allows to skip empty files while reading. Disabled by default. + **See Also** - [Virtual columns](../../engines/table-engines/index.md#table_engines-virtual_columns) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index a9ddc286ec5..7068c208022 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -202,6 +202,12 @@ FROM s3( LIMIT 5; ``` +## Storage Settings {#storage-settings} + +- [s3_truncate_on_insert](/docs/en/operations/settings/settings.md#s3-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. +- [s3_create_multiple_files](/docs/en/operations/settings/settings.md#s3_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. +- [s3_skip_empty_files](/docs/en/operations/settings/settings.md#s3_skip_empty_files) - allows to skip empty files while reading. Disabled by default. + **See Also** - [S3 engine](../../engines/table-engines/integrations/s3.md) diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index f157a850a12..ac4162c15de 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -53,6 +53,10 @@ Character `|` inside patterns is used to specify failover addresses. They are it - `_path` — Path to the `URL`. - `_file` — Resource name of the `URL`. +## Storage Settings {#storage-settings} + +- [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. + **See Also** - [Virtual columns](/docs/en/engines/table-engines/index.md#table_engines-virtual_columns) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 706ce481a24..e882138bf0d 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -14,10 +14,8 @@ #include #include -#include #include #include -#include #include #include @@ -29,7 +27,6 @@ #include #include #include -#include #include #include @@ -48,7 +45,6 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NETWORK_ERROR; extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 516c8ed152a..174cbad1de4 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1719,7 +1719,6 @@ def test_skip_empty_files(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] - instance.query( f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files1.parquet', TSVRaw) select * from numbers(0) settings s3_truncate_on_insert=1" ) @@ -1727,43 +1726,44 @@ def test_skip_empty_files(started_cluster): instance.query( f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files2.parquet') select * from numbers(1) settings s3_truncate_on_insert=1" ) + def test(engine, setting): instance.query_and_get_error( f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files1.parquet') settings {setting}=0" ) - + instance.query_and_get_error( f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files1.parquet', auto, 'number UINt64') settings {setting}=0" ) - + instance.query_and_get_error( f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files1.parquet') settings {setting}=1" ) - + res = instance.query( f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files1.parquet', auto, 'number UInt64') settings {setting}=1" ) - + assert len(res) == 0 instance.query_and_get_error( f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{1,2}}.parquet') settings {setting}=0" ) - + instance.query_and_get_error( f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{1,2}}.parquet', auto, 'number UInt64') settings {setting}=0" ) - + res = instance.query( f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{1,2}}.parquet') settings {setting}=1" ) - + assert int(res) == 0 - + res = instance.query( f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{1,2}}.parquet', auto, 'number UInt64') settings {setting}=1" ) - + assert int(res) == 0 test("s3", "s3_skip_empty_files") From 95cbc11d7b440bb22a3b9118685213cffd41862c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:54:31 +0000 Subject: [PATCH 0879/2223] Remove c-ares submodule --- .gitmodules | 3 --- contrib/c-ares | 1 - 2 files changed, 4 deletions(-) delete mode 160000 contrib/c-ares diff --git a/.gitmodules b/.gitmodules index f0984fec4db..0915280e08c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -268,9 +268,6 @@ [submodule "contrib/vectorscan"] path = contrib/vectorscan url = https://github.com/VectorCamp/vectorscan.git -[submodule "contrib/c-ares"] - path = contrib/c-ares - url = https://github.com/ClickHouse/c-ares [submodule "contrib/llvm-project"] path = contrib/llvm-project url = https://github.com/ClickHouse/llvm-project diff --git a/contrib/c-ares b/contrib/c-ares deleted file mode 160000 index afee6748b0b..00000000000 --- a/contrib/c-ares +++ /dev/null @@ -1 +0,0 @@ -Subproject commit afee6748b0b99acf4509d42fa37ac8422262f91b From 3a1e07d4c18c276ba70fbb803ffa5a6e89d78c4b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:54:47 +0000 Subject: [PATCH 0880/2223] Add submodule c-ares --- .gitmodules | 3 +++ contrib/c-ares | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/c-ares diff --git a/.gitmodules b/.gitmodules index 0915280e08c..67f181414e5 100644 --- a/.gitmodules +++ b/.gitmodules @@ -341,3 +341,6 @@ [submodule "contrib/isa-l"] path = contrib/isa-l url = https://github.com/ClickHouse/isa-l.git +[submodule "contrib/c-ares"] + path = contrib/c-ares + url = https://github.com/c-ares/c-ares.git diff --git a/contrib/c-ares b/contrib/c-ares new file mode 160000 index 00000000000..903a613f9f7 --- /dev/null +++ b/contrib/c-ares @@ -0,0 +1 @@ +Subproject commit 903a613f9f72e1509bb04cd57b5f320fe7e4d39e From 5eb592acc57d4fdda62907935b5468bf390defc9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 17:57:50 +0000 Subject: [PATCH 0881/2223] Bump libxml2 from 2.10.3 to 2.10.4 CVE-2023-28484 CVE-2023-29469 --- contrib/libxml2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libxml2 b/contrib/libxml2 index f507d167f17..223cb03a5d2 160000 --- a/contrib/libxml2 +++ b/contrib/libxml2 @@ -1 +1 @@ -Subproject commit f507d167f1755b7eaea09fb1a44d29aab828b6d1 +Subproject commit 223cb03a5d27b1b2393b266a8657443d046139d6 From eeab9fb3d4a3be717159795d608b8776430e1c7b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 18:08:20 +0000 Subject: [PATCH 0882/2223] Add new file --- contrib/c-ares-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/c-ares-cmake/CMakeLists.txt b/contrib/c-ares-cmake/CMakeLists.txt index 63e145bec18..86ab6f90260 100644 --- a/contrib/c-ares-cmake/CMakeLists.txt +++ b/contrib/c-ares-cmake/CMakeLists.txt @@ -48,6 +48,7 @@ SET(SRCS "${LIBRARY_DIR}/src/lib/ares_platform.c" "${LIBRARY_DIR}/src/lib/ares_process.c" "${LIBRARY_DIR}/src/lib/ares_query.c" + "${LIBRARY_DIR}/src/lib/ares_rand.c" "${LIBRARY_DIR}/src/lib/ares_search.c" "${LIBRARY_DIR}/src/lib/ares_send.c" "${LIBRARY_DIR}/src/lib/ares_strcasecmp.c" From 2bf867c1cf325e237aae63f96003bd9bf37a0b59 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 18:13:18 +0000 Subject: [PATCH 0883/2223] Bump to v1.19.1 --- contrib/c-ares | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/c-ares b/contrib/c-ares index 903a613f9f7..6360e96b5cf 160000 --- a/contrib/c-ares +++ b/contrib/c-ares @@ -1 +1 @@ -Subproject commit 903a613f9f72e1509bb04cd57b5f320fe7e4d39e +Subproject commit 6360e96b5cf8e5980c887ce58ef727e53d77243a From 5b32c026b91f02ab0a1c22ab5c77c16681233d99 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 May 2023 18:48:41 +0000 Subject: [PATCH 0884/2223] Fix CVE-2022-2469 --- contrib/libgsasl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libgsasl b/contrib/libgsasl index f4e7bf0bb06..0fb79e7609a 160000 --- a/contrib/libgsasl +++ b/contrib/libgsasl @@ -1 +1 @@ -Subproject commit f4e7bf0bb068030d57266f87ccac4c8c012fb5c4 +Subproject commit 0fb79e7609ae5a5e015a41d24bcbadd48f8f5469 From c9626314f7e7b128eccfd5fca7e04c9ff1fc45c5 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 31 May 2023 19:22:44 +0000 Subject: [PATCH 0885/2223] Better --- contrib/capnproto | 2 +- src/Formats/CapnProtoSerializer.cpp | 1028 ++++++++--------- ...lumnsStructureToQueryWithClusterEngine.cpp | 52 + ...ColumnsStructureToQueryWithClusterEngine.h | 14 + 4 files changed, 529 insertions(+), 567 deletions(-) create mode 100644 src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp create mode 100644 src/Storages/addColumnsStructureToQueryWithClusterEngine.h diff --git a/contrib/capnproto b/contrib/capnproto index dc8b50b9997..976209a6d18 160000 --- a/contrib/capnproto +++ b/contrib/capnproto @@ -1 +1 @@ -Subproject commit dc8b50b999777bcb23c89bb5907c785c3f654441 +Subproject commit 976209a6d18074804f60d18ef99b6a809d27dadf diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index 91e207a1846..e36f5fa4947 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -94,9 +94,21 @@ namespace std::vector> field_builders; }; + template + std::unique_ptr initStructBuilder(ParentBuilder & parent_builder, UInt32 offset_or_index, const capnp::_::StructSize & struct_size, size_t elements, const capnp::StructSchema & schema) + { + capnp::DynamicStruct::Builder builder_impl; + if constexpr (std::is_same_v) + builder_impl = capnp::DynamicStruct::Builder(schema, parent_builder.getBuilderImpl().getPointerField(offset_or_index).initStruct(struct_size)); + else + builder_impl = capnp::DynamicStruct::Builder(schema, parent_builder.getBuilderImpl().getStructElement(offset_or_index)); + return std::make_unique(std::move(builder_impl), elements); + } + class ICapnProtoSerializer { public: + /// Write row as struct field. virtual void writeRow( const ColumnPtr & column, std::unique_ptr & builder, @@ -104,6 +116,7 @@ namespace UInt32 slot_offset, size_t row_num) = 0; + /// Write row as list element. virtual void writeRow( const ColumnPtr & column, std::unique_ptr & builder, @@ -111,8 +124,10 @@ namespace UInt32 array_index, size_t row_num) = 0; + /// Read row from struct field at slot_offset. virtual void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) = 0; + /// Read row from list element at array_index. virtual void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) = 0; virtual ~ICapnProtoSerializer() = default; @@ -124,32 +139,32 @@ namespace public: void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - CapnProtoNumericType value = static_cast(assert_cast &>(*column).getElement(row_num)); - builder_impl.setDataField(slot_offset, value); + parent_struct_builder.getBuilderImpl().setDataField(slot_offset, getValue(column, row_num)); } void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto & builder_impl = parent_list_builder.getBuilderImpl(); - CapnProtoNumericType value = static_cast(assert_cast &>(*column).getElement(row_num)); - builder_impl.setDataElement(array_index, value); + parent_list_builder.getBuilderImpl().setDataElement(array_index, getValue(column, row_num)); } void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - CapnProtoNumericType value = reader_impl.getDataField(slot_offset); - if constexpr (convert_to_bool_on_read) - assert_cast(column).insertValue(static_cast(value)); - else - assert_cast &>(column).insertValue(static_cast(value)); + insertValue(column, parent_struct_reader.getReaderImpl().getDataField(slot_offset)); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - CapnProtoNumericType value = reader_impl.getDataElement(array_index); + insertValue(column, parent_list_reader.getReaderImpl().getDataElement(array_index)); + } + + private: + CapnProtoNumericType getValue(const ColumnPtr & column, size_t row_num) + { + return static_cast(assert_cast &>(*column).getElement(row_num)); + } + + void insertValue(IColumn & column, CapnProtoNumericType value) + { if constexpr (convert_to_bool_on_read) assert_cast(column).insertValue(static_cast(value)); else @@ -191,29 +206,32 @@ namespace public: void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - CapnProtoFloatType value = static_cast(assert_cast &>(*column).getElement(row_num)); - builder_impl.setDataField(slot_offset, value); + parent_struct_builder.getBuilderImpl().setDataField(slot_offset, getValue(column, row_num)); } void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto & builder_impl = parent_list_builder.getBuilderImpl(); - CapnProtoFloatType value = static_cast(assert_cast &>(*column).getElement(row_num)); - builder_impl.setDataElement(array_index, value); + parent_list_builder.getBuilderImpl().setDataElement(array_index, getValue(column, row_num)); } void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - CapnProtoFloatType value = reader_impl.getDataField(slot_offset); - assert_cast &>(column).insertValue(static_cast(value)); + insertValue(column, parent_struct_reader.getReaderImpl().getDataField(slot_offset)); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - CapnProtoFloatType value = reader_impl.getDataElement(array_index); + insertValue(column, parent_list_reader.getReaderImpl().getDataElement(array_index)); + } + + private: + CapnProtoFloatType getValue(const ColumnPtr & column, size_t row_num) + { + return static_cast(assert_cast &>(*column).getElement(row_num)); + } + + void insertValue(IColumn & column, CapnProtoFloatType value) + { assert_cast &>(column).insertValue(static_cast(value)); } }; @@ -298,57 +316,41 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - EnumType enum_value = assert_cast &>(*column).getElement(row_num); - UInt16 capnp_value; - if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) - capnp_value = static_cast(enum_value); - else - capnp_value = ch_to_capnp_values[enum_value]; - - builder_impl.setDataField(slot_offset, capnp_value); + parent_struct_builder.getBuilderImpl().setDataField(slot_offset, getValue(column, row_num)); } void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto & builder_impl = parent_list_builder.getBuilderImpl(); - EnumType enum_value = assert_cast &>(*column).getElement(row_num); - UInt16 capnp_value; - if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) - capnp_value = static_cast(enum_value); - else - capnp_value = ch_to_capnp_values[enum_value]; - - builder_impl.setDataElement(array_index, capnp_value); + parent_list_builder.getBuilderImpl().setDataElement(array_index, getValue(column, row_num)); } void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - UInt16 capnp_value = reader_impl.getDataField(slot_offset); - EnumType value; - if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) - value = static_cast(capnp_value); - else - value = capnp_to_ch_values[capnp_value]; - - assert_cast &>(column).insertValue(value); + insertValue(column, parent_struct_reader.getReaderImpl().getDataField(slot_offset)); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - UInt16 capnp_value = reader_impl.getDataElement(array_index); - EnumType value; - if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) - value = static_cast(capnp_value); - else - value = capnp_to_ch_values[capnp_value]; - - assert_cast &>(column).insertValue(value); + insertValue(column, parent_list_reader.getReaderImpl().getDataElement(array_index)); } private: + UInt16 getValue(const ColumnPtr & column, size_t row_num) + { + EnumType enum_value = assert_cast &>(*column).getElement(row_num); + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + return static_cast(enum_value); + return ch_to_capnp_values[enum_value]; + } + + void insertValue(IColumn & column, UInt16 capnp_enum_value) + { + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + assert_cast &>(column).insertValue(static_cast(capnp_enum_value)); + else + assert_cast &>(column).insertValue(capnp_to_ch_values[capnp_enum_value]); + } + DataTypePtr data_type; capnp::EnumSchema enum_schema; const FormatSettings::CapnProtoEnumComparingMode enum_comparing_mode; @@ -367,29 +369,32 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - UInt16 value = assert_cast(*column).getElement(row_num); - builder_impl.setDataField(slot_offset, value); + parent_struct_builder.getBuilderImpl().setDataField(slot_offset, getValue(column, row_num)); } void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto & builder_impl = parent_list_builder.getBuilderImpl(); - UInt16 value = assert_cast(*column).getElement(row_num); - builder_impl.setDataElement(array_index, value); + parent_list_builder.getBuilderImpl().setDataElement(array_index, getValue(column, row_num)); } void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - UInt16 value = reader_impl.getDataField(slot_offset); - assert_cast(column).insertValue(value); + insertValue(column, parent_struct_reader.getReaderImpl().getDataField(slot_offset)); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - UInt16 value = reader_impl.getDataElement(array_index); + insertValue(column, parent_list_reader.getReaderImpl().getDataElement(array_index)); + } + + private: + UInt16 getValue(const ColumnPtr & column, size_t row_num) + { + return assert_cast(*column).getElement(row_num); + } + + void insertValue(IColumn & column, UInt16 value) + { assert_cast(column).insertValue(value); } }; @@ -405,29 +410,32 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - Int32 value = assert_cast(*column).getElement(row_num); - builder_impl.setDataField(slot_offset, value); + parent_struct_builder.getBuilderImpl().setDataField(slot_offset, getValue(column, row_num)); } void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto & builder_impl = parent_list_builder.getBuilderImpl(); - Int32 value = assert_cast(*column).getElement(row_num); - builder_impl.setDataElement(array_index, value); + parent_list_builder.getBuilderImpl().setDataElement(array_index, getValue(column, row_num)); } void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - Int32 value = reader_impl.getDataField(slot_offset); - assert_cast(column).insertValue(value); + insertValue(column, parent_struct_reader.getReaderImpl().getDataField(slot_offset)); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - Int32 value = reader_impl.getDataElement(array_index); + insertValue(column, parent_list_reader.getReaderImpl().getDataElement(array_index)); + } + + private: + Int32 getValue(const ColumnPtr & column, size_t row_num) + { + return assert_cast(*column).getElement(row_num); + } + + void insertValue(IColumn & column, Int32 value) + { assert_cast(column).insertValue(value); } }; @@ -443,29 +451,32 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - UInt32 value = assert_cast(*column).getElement(row_num); - builder_impl.setDataField(slot_offset, value); + parent_struct_builder.getBuilderImpl().setDataField(slot_offset, getValue(column, row_num)); } void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto & builder_impl = parent_list_builder.getBuilderImpl(); - UInt32 value = assert_cast(*column).getElement(row_num); - builder_impl.setDataElement(array_index, value); + parent_list_builder.getBuilderImpl().setDataElement(array_index, getValue(column, row_num)); } void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - UInt32 value = reader_impl.getDataField(slot_offset); - assert_cast(column).insertValue(value); + insertValue(column, parent_struct_reader.getReaderImpl().getDataField(slot_offset)); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - UInt32 value = reader_impl.getDataElement(array_index); + insertValue(column, parent_list_reader.getReaderImpl().getDataElement(array_index)); + } + + private: + UInt32 getValue(const ColumnPtr & column, size_t row_num) + { + return assert_cast(*column).getElement(row_num); + } + + void insertValue(IColumn & column, UInt32 value) + { assert_cast(column).insertValue(value); } }; @@ -481,29 +492,32 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - Int64 value = assert_cast(*column).getElement(row_num); - builder_impl.setDataField(slot_offset, value); + parent_struct_builder.getBuilderImpl().setDataField(slot_offset, getValue(column, row_num)); } void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto & builder_impl = parent_list_builder.getBuilderImpl(); - Int64 value = assert_cast(*column).getElement(row_num); - builder_impl.setDataElement(array_index, value); + parent_list_builder.getBuilderImpl().setDataElement(array_index, getValue(column, row_num)); } void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - Int64 value = reader_impl.getDataField(slot_offset); - assert_cast(column).insertValue(value); + insertValue(column, parent_struct_reader.getReaderImpl().getDataField(slot_offset)); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - Int64 value = reader_impl.getDataElement(array_index); + insertValue(column, parent_list_reader.getReaderImpl().getDataElement(array_index)); + } + + private: + Int64 getValue(const ColumnPtr & column, size_t row_num) + { + return assert_cast(*column).getElement(row_num); + } + + void insertValue(IColumn & column, Int64 value) + { assert_cast(column).insertValue(value); } }; @@ -523,275 +537,36 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - DecimalType value = assert_cast &>(*column).getElement(row_num); - builder_impl.setDataField(slot_offset, value); + parent_struct_builder.getBuilderImpl().setDataField(slot_offset, getValue(column, row_num)); } void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto & builder_impl = parent_list_builder.getBuilderImpl(); - DecimalType value = assert_cast &>(*column).getElement(row_num); - builder_impl.setDataElement(array_index, value); + parent_list_builder.getBuilderImpl().setDataElement(array_index, getValue(column, row_num)); } void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - NativeType value = reader_impl.getDataField(slot_offset); - assert_cast &>(column).insertValue(value); + insertValue(column, parent_struct_reader.getReaderImpl().getDataField(slot_offset)); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - NativeType value = reader_impl.getDataElement(array_index); + insertValue(column, parent_list_reader.getReaderImpl().getDataElement(array_index)); + } + + private: + NativeType getValue(const ColumnPtr & column, size_t row_num) + { + return assert_cast &>(*column).getElement(row_num); + } + + void insertValue(IColumn & column, NativeType value) + { assert_cast &>(column).insertValue(value); } }; - template - class CapnProtoFixedSizeRawDataSerializer : public ICapnProtoSerializer - { - private: - static constexpr size_t value_size = sizeof(T); - - public: - CapnProtoFixedSizeRawDataSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) - { - if (!capnp_type.isData()) - throwCannotConvert(data_type, column_name, capnp_type); - } - - void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override - { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - auto data = column->getDataAt(row_num); - capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); - builder_impl.getPointerField(slot_offset).template setBlob(value); - } - - void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_struct_builder, UInt32 array_index, size_t row_num) override - { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - auto data = column->getDataAt(row_num); - capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); - builder_impl.getPointerElement(array_index).setBlob(value); - } - - void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override - { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - capnp::Data::Reader value = reader_impl.getPointerField(slot_offset).template getBlob(nullptr, 0); - if (value.size() != value_size) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); - - column.insertData(reinterpret_cast(value.begin()), value.size()); - } - - void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override - { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - capnp::Data::Reader value = reader_impl.getPointerElement(array_index).getBlob(nullptr, 0); - if (value.size() != value_size) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); - - column.insertData(reinterpret_cast(value.begin()), value.size()); - } - - private: - DataTypePtr data_type; - }; - - template - class CapnProtoStringSerializer : public ICapnProtoSerializer - { - public: - CapnProtoStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) - { - if (!capnp_type.isData() && !capnp_type.isText()) - throwCannotConvert(data_type, column_name, capnp_type); - } - - void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override - { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - auto data = column->getDataAt(row_num); - if constexpr (is_binary) - { - capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); - builder_impl.getPointerField(slot_offset).setBlob(value); - } - else - { - capnp::Text::Reader value = capnp::Text::Reader(data.data, data.size); - builder_impl.getPointerField(slot_offset).setBlob(value); - } - } - - void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_struct_builder, UInt32 array_index, size_t row_num) override - { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - auto data = column->getDataAt(row_num); - if constexpr (is_binary) - { - capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); - builder_impl.getPointerElement(array_index).setBlob(value); - } - else - { - capnp::Text::Reader value = capnp::Text::Reader(data.data, data.size); - builder_impl.getPointerElement(array_index).setBlob(value); - } - } - - void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override - { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - if constexpr (is_binary) - { - capnp::Data::Reader value = reader_impl.getPointerField(slot_offset).getBlob(nullptr, 0); - column.insertData(reinterpret_cast(value.begin()), value.size()); - } - else - { - capnp::Text::Reader value = reader_impl.getPointerField(slot_offset).getBlob(nullptr, 0); - column.insertData(reinterpret_cast(value.begin()), value.size()); - } - } - - void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override - { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - if constexpr (is_binary) - { - capnp::Data::Reader value = reader_impl.getPointerElement(array_index).getBlob(nullptr, 0); - column.insertData(reinterpret_cast(value.begin()), value.size()); - } - else - { - capnp::Text::Reader value = reader_impl.getPointerElement(array_index).getBlob(nullptr, 0); - column.insertData(reinterpret_cast(value.begin()), value.size()); - } - } - }; - - template - class CapnProtoFixedStringSerializer : public ICapnProtoSerializer - { - public: - CapnProtoFixedStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type_) : capnp_type(capnp_type_) - { - if (!capnp_type.isData() && !capnp_type.isText()) - throwCannotConvert(data_type, column_name, capnp_type); - } - - void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override - { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - auto data = column->getDataAt(row_num); - if constexpr (is_binary) - { - capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); - builder_impl.getPointerField(slot_offset).setBlob(value); - } - else - { - if (data.data[data.size - 1] == 0) - { - capnp::Text::Reader value = capnp::Text::Reader(data.data, data.size); - builder_impl.getPointerField(slot_offset).setBlob(value); - } - else - { - /// In TEXT type data should be null-terminated, but ClickHouse FixedString data could not be. - /// To make data null-terminated we should copy it to temporary String object and use it in capnp::Text::Reader. - /// Note that capnp::Text::Reader works only with pointer to the data and it's size, so we should - /// guarantee that new String object life time is longer than capnp::Text::Reader life time. - tmp_string = data.toString(); - capnp::Text::Reader value = capnp::Text::Reader(tmp_string.data(), tmp_string.size()); - builder_impl.getPointerField(slot_offset).setBlob(value); - } - } - } - - void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_struct_builder, UInt32 array_index, size_t row_num) override - { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - auto data = column->getDataAt(row_num); - if constexpr (is_binary) - { - capnp::Data::Reader value = capnp::Data::Reader(reinterpret_cast(data.data), data.size); - builder_impl.getPointerElement(array_index).setBlob(value); - } - else - { - if (data.data[data.size - 1] == 0) - { - capnp::Text::Reader value = capnp::Text::Reader(data.data, data.size); - builder_impl.getPointerElement(array_index).setBlob(value); - } - else - { - /// In TEXT type data should be null-terminated, but ClickHouse FixedString data could not be. - /// To make data null-terminated we should copy it to temporary String object and use it in capnp::Text::Reader. - /// Note that capnp::Text::Reader works only with pointer to the data and it's size, so we should - /// guarantee that new String object life time is longer than capnp::Text::Reader life time. - tmp_string = data.toString(); - capnp::Text::Reader value = capnp::Text::Reader(tmp_string.data(), tmp_string.size()); - builder_impl.getPointerElement(array_index).setBlob(value); - } - } - } - - void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override - { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - auto & fixed_string_column = assert_cast(column); - if constexpr (is_binary) - { - capnp::Data::Reader value = reader_impl.getPointerField(slot_offset).getBlob(nullptr, 0); - if (value.size() > fixed_string_column.getN()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); - - fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); - } - else - { - capnp::Text::Reader value = reader_impl.getPointerField(slot_offset).getBlob(nullptr, 0); - if (value.size() > fixed_string_column.getN()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); - - fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); - } - } - - void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override - { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - auto & fixed_string_column = assert_cast(column); - if constexpr (is_binary) - { - capnp::Data::Reader value = reader_impl.getPointerElement(array_index).getBlob(nullptr, 0); - if (value.size() > fixed_string_column.getN()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); - - fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); - } - else - { - capnp::Text::Reader value = reader_impl.getPointerElement(array_index).getBlob(nullptr, 0); - if (value.size() > fixed_string_column.getN()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); - - fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); - } - } - - private: - String tmp_string; - capnp::Type capnp_type; - }; class CapnProtoIPv4Serializer : public ICapnProtoSerializer { @@ -804,33 +579,204 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - auto & builder_impl = parent_struct_builder.getBuilderImpl(); - UInt32 value = assert_cast(*column).getElement(row_num); - builder_impl.setDataField(slot_offset, value); + parent_struct_builder.getBuilderImpl().setDataField(slot_offset, getValue(column, row_num)); } void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - auto & builder_impl = parent_list_builder.getBuilderImpl(); - UInt32 value = assert_cast(*column).getElement(row_num); - builder_impl.setDataElement(array_index, value); + parent_list_builder.getBuilderImpl().setDataElement(array_index, getValue(column, row_num)); } void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - UInt32 value = reader_impl.getDataField(slot_offset); - assert_cast(column).insertValue(IPv4(value)); + insertValue(column, parent_struct_reader.getReaderImpl().getDataField(slot_offset)); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - UInt32 value = reader_impl.getDataElement(array_index); + insertValue(column, parent_list_reader.getReaderImpl().getDataElement(array_index)); + } + + private: + UInt32 getValue(const ColumnPtr & column, size_t row_num) + { + return assert_cast(*column).getElement(row_num); + } + + void insertValue(IColumn & column, UInt32 value) + { assert_cast(column).insertValue(IPv4(value)); } }; + template + class CapnProtoFixedSizeRawDataSerializer : public ICapnProtoSerializer + { + private: + static constexpr size_t expected_value_size = sizeof(T); + + public: + CapnProtoFixedSizeRawDataSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override + { + parent_struct_builder.getBuilderImpl().getPointerField(slot_offset).setBlob(getData(column, row_num)); + } + + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_struct_builder, UInt32 array_index, size_t row_num) override + { + parent_struct_builder.getBuilderImpl().getPointerElement(array_index).setBlob(getData(column, row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + insertData(column, parent_struct_reader.getReaderImpl().getPointerField(slot_offset).getBlob(nullptr, 0)); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + insertData(column, parent_list_reader.getReaderImpl().getPointerElement(array_index).getBlob(nullptr, 0)); + } + + private: + capnp::Data::Reader getData(const ColumnPtr & column, size_t row_num) + { + auto data = column->getDataAt(row_num); + return capnp::Data::Reader(reinterpret_cast(data.data), data.size); + } + + void insertData(IColumn & column, capnp::Data::Reader data) + { + if (data.size() != expected_value_size) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), data.size()); + + column.insertData(reinterpret_cast(data.begin()), data.size()); + } + + DataTypePtr data_type; + }; + + template + class CapnProtoStringSerializer : public ICapnProtoSerializer + { + public: + CapnProtoStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isData() && !capnp_type.isText()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override + { + parent_struct_builder.getBuilderImpl().getPointerField(slot_offset).setBlob(getData(column, row_num)); + } + + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_struct_builder, UInt32 array_index, size_t row_num) override + { + parent_struct_builder.getBuilderImpl().getPointerElement(array_index).setBlob(getData(column, row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + insertData(column, parent_struct_reader.getReaderImpl().getPointerField(slot_offset).getBlob(nullptr, 0)); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + insertData(column, parent_list_reader.getReaderImpl().getPointerElement(array_index).getBlob(nullptr, 0)); + } + + private: + using Reader = typename CapnpType::Reader; + + CapnpType::Reader getData(const ColumnPtr & column, size_t row_num) + { + auto data = column->getDataAt(row_num); + if constexpr (std::is_same_v) + return Reader(reinterpret_cast(data.data), data.size); + else + return Reader(data.data, data.size); + } + + void insertData(IColumn & column, Reader data) + { + column.insertData(reinterpret_cast(data.begin()), data.size()); + } + }; + + template + class CapnProtoFixedStringSerializer : public ICapnProtoSerializer + { + private: + + public: + CapnProtoFixedStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type_) : capnp_type(capnp_type_) + { + if (!capnp_type.isData() && !capnp_type.isText()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override + { + parent_struct_builder.getBuilderImpl().getPointerField(slot_offset).setBlob(getData(column, row_num)); + } + + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_struct_builder, UInt32 array_index, size_t row_num) override + { + parent_struct_builder.getBuilderImpl().getPointerElement(array_index).setBlob(getData(column, row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + insertData(column, parent_struct_reader.getReaderImpl().getPointerField(slot_offset).getBlob(nullptr, 0)); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + insertData(column, parent_list_reader.getReaderImpl().getPointerElement(array_index).getBlob(nullptr, 0)); + } + + private: + using Reader = typename CapnpType::Reader; + + CapnpType::Reader getData(const ColumnPtr & column, size_t row_num) + { + auto data = column->getDataAt(row_num); + if constexpr (std::is_same_v) + { + return Reader(reinterpret_cast(data.data), data.size); + } + else + { + if (data.data[data.size - 1] == 0) + return Reader(data.data, data.size); + + /// In TEXT type data should be null-terminated, but ClickHouse FixedString data could not be. + /// To make data null-terminated we should copy it to temporary String object and use it in capnp::Text::Reader. + /// Note that capnp::Text::Reader works only with pointer to the data and it's size, so we should + /// guarantee that new String object life time is longer than capnp::Text::Reader life time. + tmp_string = data.toString(); + return Reader(tmp_string.data(), tmp_string.size()); + } + } + + void insertData(IColumn & column, Reader data) + { + auto & fixed_string_column = assert_cast(column); + if (data.size() > fixed_string_column.getN()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", data.size(), fixed_string_column.getN()); + + fixed_string_column.insertData(reinterpret_cast(data.begin()), data.size()); + } + + String tmp_string; + capnp::Type capnp_type; + }; + std::unique_ptr createSerializer(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings); class CapnProtoLowCardinalitySerializer : public ICapnProtoSerializer @@ -843,37 +789,43 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - const auto & low_cardinality_column = assert_cast(*column); - size_t index = low_cardinality_column.getIndexAt(row_num); - const auto & dict_column = low_cardinality_column.getDictionary().getNestedColumn(); - nested_serializer->writeRow(dict_column, field_builder, parent_struct_builder, slot_offset, index); + writeRowImpl(column, field_builder, parent_struct_builder, slot_offset, row_num); } void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - const auto & low_cardinality_column = assert_cast(*column); - size_t index = low_cardinality_column.getIndexAt(row_num); - const auto & dict_column = low_cardinality_column.getDictionary().getNestedColumn(); - nested_serializer->writeRow(dict_column, field_builder, parent_list_builder, array_index, index); + writeRowImpl(column, field_builder, parent_list_builder, array_index, row_num); } void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - auto & low_cardinality_column = assert_cast(column); - auto tmp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); - nested_serializer->readRow(*tmp_column, parent_struct_reader, slot_offset); - low_cardinality_column.insertFromFullColumn(*tmp_column, 0); + readRowImpl(column, parent_struct_reader, slot_offset); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - auto & low_cardinality_column = assert_cast(column); - auto tmp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); - nested_serializer->readRow(*tmp_column, parent_list_reader, array_index); - low_cardinality_column.insertFromFullColumn(*tmp_column, 0); + readRowImpl(column, parent_list_reader, array_index); } private: + template + void writeRowImpl(const ColumnPtr & column, std::unique_ptr & field_builder, ParentBuilder & parent_builder, UInt32 offset_or_index, size_t row_num) + { + const auto & low_cardinality_column = assert_cast(*column); + size_t index = low_cardinality_column.getIndexAt(row_num); + const auto & dict_column = low_cardinality_column.getDictionary().getNestedColumn(); + nested_serializer->writeRow(dict_column, field_builder, parent_builder, offset_or_index, index); + } + + template + void readRowImpl(IColumn & column, const ParentReader & parent_reader, UInt32 offset_or_index) + { + auto & low_cardinality_column = assert_cast(column); + auto tmp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); + nested_serializer->readRow(*tmp_column, parent_reader, offset_or_index); + low_cardinality_column.insertFromFullColumn(*tmp_column, 0); + } + std::unique_ptr nested_serializer; }; @@ -938,38 +890,32 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - if (!field_builder) - { - auto builder_impl = parent_struct_builder.getBuilderImpl(); - auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getPointerField(slot_offset).initStruct(struct_size)); - field_builder = std::make_unique(std::move(struct_builder_impl), 1); - } - - auto & struct_builder = assert_cast(*field_builder); - - const auto & nullable_column = assert_cast(*column); - if (nullable_column.isNullAt(row_num)) - { - auto struct_builder_impl = struct_builder.impl.getBuilderImpl(); - struct_builder_impl.setDataField(discriminant_offset, null_discriminant); - struct_builder_impl.setDataField(nested_slot_offset, capnp::Void()); - } - else - { - const auto & nested_column = nullable_column.getNestedColumnPtr(); - struct_builder.impl.getBuilderImpl().setDataField(discriminant_offset, nested_discriminant); - nested_serializer->writeRow(nested_column, struct_builder.field_builders[0], struct_builder.impl, nested_slot_offset, row_num); - } + writeRowImpl(column, field_builder, parent_struct_builder, slot_offset, row_num); } void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override + { + writeRowImpl(column, field_builder, parent_list_builder, array_index, row_num); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, parent_struct_reader.getReaderImpl().getPointerField(slot_offset).getStruct(nullptr)); + readRowImpl(column, struct_reader); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, parent_list_reader.getReaderImpl().getStructElement(array_index)); + readRowImpl(column, struct_reader); + } + + private: + template + void writeRowImpl(const ColumnPtr & column, std::unique_ptr & field_builder, ParentBuilder & parent_builder, UInt32 offset_or_index, size_t row_num) { if (!field_builder) - { - auto builder_impl = parent_list_builder.getBuilderImpl(); - auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getStructElement(array_index)); - field_builder = std::make_unique(std::move(struct_builder_impl), 1); - } + field_builder = initStructBuilder(parent_builder, offset_or_index, struct_size, 1, struct_schema); auto & struct_builder = assert_cast(*field_builder); @@ -988,12 +934,9 @@ namespace } } - void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + void readRowImpl(IColumn & column, capnp::DynamicStruct::Reader & struct_reader) { auto & nullable_column = assert_cast(column); - auto reader_impl = parent_struct_reader.getReaderImpl(); - auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getPointerField(slot_offset).getStruct(nullptr)); - auto discriminant = struct_reader.getReaderImpl().getDataField(discriminant_offset); if (discriminant == null_discriminant) @@ -1006,25 +949,7 @@ namespace } } - void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override - { - auto & nullable_column = assert_cast(column); - auto reader_impl = parent_list_reader.getReaderImpl(); - auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getStructElement(array_index)); - auto discriminant = struct_reader.getReaderImpl().getDataField(discriminant_offset); - - if (discriminant == null_discriminant) - nullable_column.insertDefault(); - else - { - auto & nested_column = nullable_column.getNestedColumn(); - nested_serializer->readRow(nested_column, struct_reader, nested_slot_offset); - nullable_column.getNullMapData().push_back(0); - } - } - - private: std::unique_ptr nested_serializer; capnp::StructSchema struct_schema; capnp::_::StructSize struct_size; @@ -1058,29 +983,29 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - const auto * array_column = assert_cast(column.get()); - const auto & nested_column = array_column->getDataPtr(); - const auto & offsets = array_column->getOffsets(); - auto offset = offsets[row_num - 1]; - UInt32 size = static_cast(offsets[row_num] - offset); - - if (!field_builder) - { - auto builder_impl = parent_struct_builder.getBuilderImpl(); - capnp::DynamicList::Builder list_builder_impl; - if (element_is_struct) - list_builder_impl = capnp::DynamicList::Builder(list_schema, builder_impl.getPointerField(slot_offset).initStructList(size, element_struct_size)); - else - list_builder_impl = capnp::DynamicList::Builder(list_schema, builder_impl.getPointerField(slot_offset).initList(element_size, size)); - field_builder = std::make_unique(std::move(list_builder_impl), size); - } - - auto & list_builder = assert_cast(*field_builder); - for (UInt32 i = 0; i != size; ++i) - nested_serializer->writeRow(nested_column, list_builder.nested_builders[i], list_builder.impl, i, offset + i); + writeRowImpl(column, field_builder, parent_struct_builder, slot_offset, row_num); } void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override + { + writeRowImpl(column, field_builder, parent_list_builder, array_index, row_num); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + auto list_reader = capnp::DynamicList::Reader(list_schema, parent_struct_reader.getReaderImpl().getPointerField(slot_offset).getList(element_size, nullptr)); + readRowImpl(column, list_reader); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + auto list_reader = capnp::DynamicList::Reader(list_schema, parent_list_reader.getReaderImpl().getPointerElement(array_index).getList(element_size, nullptr)); + readRowImpl(column, list_reader); + } + + private: + template + void writeRowImpl(const ColumnPtr & column, std::unique_ptr & field_builder, ParentBuilder & parent_builder, UInt32 offset_or_index, size_t row_num) { const auto * array_column = assert_cast(column.get()); const auto & nested_column = array_column->getDataPtr(); @@ -1089,25 +1014,32 @@ namespace UInt32 size = static_cast(offsets[row_num] - offset); if (!field_builder) - { - auto builder_impl = parent_list_builder.getBuilderImpl(); - capnp::DynamicList::Builder list_builder_impl; - if (element_is_struct) - list_builder_impl = capnp::DynamicList::Builder(list_schema, builder_impl.getPointerElement(array_index).initStructList(size, element_struct_size)); - else - list_builder_impl = capnp::DynamicList::Builder(list_schema, builder_impl.getPointerElement(array_index).initList(element_size, size)); - field_builder = std::make_unique(std::move(list_builder_impl), size); - } + field_builder = std::make_unique(capnp::DynamicList::Builder(list_schema, initListBuilder(parent_builder, offset_or_index, size)), size); auto & list_builder = assert_cast(*field_builder); for (UInt32 i = 0; i != size; ++i) nested_serializer->writeRow(nested_column, list_builder.nested_builders[i], list_builder.impl, i, offset + i); } - void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + template + capnp::_::ListBuilder initListBuilder(ParentBuilder & parent_builder, UInt32 offset_or_index, UInt32 size) + { + if (element_is_struct) + { + if constexpr (std::is_same_v) + return parent_builder.getBuilderImpl().getPointerField(offset_or_index).initStructList(size, element_struct_size); + else + return parent_builder.getBuilderImpl().getPointerElement(offset_or_index).initStructList(size, element_struct_size); + } + + if constexpr (std::is_same_v) + return parent_builder.getBuilderImpl().getPointerField(offset_or_index).initList(element_size, size); + else + return parent_builder.getBuilderImpl().getPointerElement(offset_or_index).initList(element_size, size); + } + + void readRowImpl(IColumn & column, const capnp::DynamicList::Reader & list_reader) { - const auto & reader_impl = parent_struct_reader.getReaderImpl(); - auto list_reader = capnp::DynamicList::Reader(list_schema, reader_impl.getPointerField(slot_offset).getList(element_size, nullptr)); UInt32 size = list_reader.size(); auto & column_array = assert_cast(column); auto & offsets = column_array.getOffsets(); @@ -1118,21 +1050,6 @@ namespace nested_serializer->readRow(nested_column, list_reader, i); } - void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override - { - const auto & reader_impl = parent_list_reader.getReaderImpl(); - auto list_reader = capnp::DynamicList::Reader(list_schema, reader_impl.getPointerElement(array_index).getList(element_size, nullptr)); - UInt32 size = list_reader.size(); - auto & column_array = assert_cast(column); - auto & offsets = column_array.getOffsets(); - offsets.push_back(offsets.back() + list_reader.size()); - - auto & nested_column = column_array.getData(); - for (UInt32 i = 0; i != size; ++i) - nested_serializer->readRow(nested_column, list_reader, i); - } - - private: capnp::ListSchema list_schema; std::unique_ptr nested_serializer; capnp::ElementSize element_size; @@ -1219,49 +1136,44 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - if (!field_builder) - { - auto builder_impl = parent_struct_builder.getBuilderImpl(); - auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getPointerField(slot_offset).initStruct(struct_size)); - field_builder = std::make_unique(std::move(struct_builder_impl), 1); - } - - auto & struct_builder = assert_cast(*field_builder); - const auto & entries_column = assert_cast(column.get())->getNestedColumnPtr(); - nested_serializer->writeRow(entries_column, struct_builder.field_builders[0], struct_builder.impl, entries_slot_offset, row_num); + writeRowImpl(column, field_builder, parent_struct_builder, slot_offset, row_num); } void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override + { + writeRowImpl(column, field_builder, parent_list_builder, array_index, row_num); + } + + void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + { + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, parent_struct_reader.getReaderImpl().getPointerField(slot_offset).getStruct(nullptr)); + readRowImpl(column, struct_reader); + } + + void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override + { + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, parent_list_reader.getReaderImpl().getStructElement(array_index)); + readRowImpl(column, struct_reader); + } + + private: + template + void writeRowImpl(const ColumnPtr & column, std::unique_ptr & field_builder, ParentBuilder & parent_builder, UInt32 offset_or_index, size_t row_num) { if (!field_builder) - { - auto builder_impl = parent_list_builder.getBuilderImpl(); - auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getStructElement(array_index)); - field_builder = std::make_unique(std::move(struct_builder_impl), 1); - } + field_builder = initStructBuilder(parent_builder, offset_or_index, struct_size, 1, struct_schema); auto & struct_builder = assert_cast(*field_builder); const auto & entries_column = assert_cast(column.get())->getNestedColumnPtr(); nested_serializer->writeRow(entries_column, struct_builder.field_builders[0], struct_builder.impl, entries_slot_offset, row_num); } - void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override + void readRowImpl(IColumn & column, const capnp::DynamicStruct::Reader & struct_reader) { - auto reader_impl = parent_struct_reader.getReaderImpl(); - auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getPointerField(slot_offset).getStruct(nullptr)); auto & entries_column = assert_cast(column).getNestedColumn(); nested_serializer->readRow(entries_column, struct_reader, entries_slot_offset); } - void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override - { - auto reader_impl = parent_list_reader.getReaderImpl(); - auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getStructElement(array_index)); - auto & entries_column = assert_cast(column).getNestedColumn(); - nested_serializer->readRow(entries_column, struct_reader, entries_slot_offset); - } - - private: std::unique_ptr nested_serializer; capnp::StructSchema struct_schema; capnp::_::StructSize struct_size; @@ -1332,48 +1244,15 @@ namespace void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) override { - if (!field_builder) - { - auto builder_impl = parent_struct_builder.getBuilderImpl(); - auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getPointerField(slot_offset).initStruct(struct_size)); - field_builder = std::make_unique(std::move(struct_builder_impl), fields_count); - } - - auto & struct_builder = assert_cast(*field_builder); - if (const auto * tuple_column = typeid_cast(column.get())) - { - const auto & columns = tuple_column->getColumns(); - for (size_t i = 0; i != columns.size(); ++i) - fields_serializers[i]->writeRow(columns[i], struct_builder.field_builders[fields_indexes[i]], struct_builder.impl, fields_offsets[i], row_num); - } - else - { - fields_serializers[0]->writeRow(column, struct_builder.field_builders[fields_indexes[0]], struct_builder.impl, fields_offsets[0], row_num); - } + writeRowImpl(column, field_builder, parent_struct_builder, slot_offset, row_num); } void writeRow(const ColumnPtr & column, std::unique_ptr & field_builder, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { - if (!field_builder) - { - auto builder_impl = parent_list_builder.getBuilderImpl(); - auto struct_builder_impl = capnp::DynamicStruct::Builder(struct_schema, builder_impl.getStructElement(array_index)); - field_builder = std::make_unique(std::move(struct_builder_impl), fields_count); - } - - auto & struct_builder = assert_cast(*field_builder); - if (const auto * tuple_column = typeid_cast(column.get())) - { - const auto & columns = tuple_column->getColumns(); - for (size_t i = 0; i != columns.size(); ++i) - fields_serializers[i]->writeRow(columns[i], struct_builder.field_builders[fields_indexes[i]], struct_builder.impl, fields_offsets[i], row_num); - } - else - { - fields_serializers[0]->writeRow(column, struct_builder.field_builders[fields_indexes[0]], struct_builder.impl, fields_offsets[0], row_num); - } + writeRowImpl(column, field_builder, parent_list_builder, array_index, row_num); } + /// Method for writing root struct. void writeRow(const Columns & columns, StructBuilder & struct_builder, size_t row_num) { for (size_t i = 0; i != columns.size(); ++i) @@ -1382,30 +1261,17 @@ namespace void readRow(IColumn & column, const capnp::DynamicStruct::Reader & parent_struct_reader, UInt32 slot_offset) override { - auto reader_impl = parent_struct_reader.getReaderImpl(); - auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getPointerField(slot_offset).getStruct(nullptr)); - if (auto * tuple_column = typeid_cast(&column)) - { - for (size_t i = 0; i != tuple_column->tupleSize(); ++i) - fields_serializers[i]->readRow(tuple_column->getColumn(i), struct_reader, fields_offsets[i]); - } - else - fields_serializers[0]->readRow(column, struct_reader, fields_offsets[0]); + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, parent_struct_reader.getReaderImpl().getPointerField(slot_offset).getStruct(nullptr)); + readRowImpl(column, struct_reader); } void readRow(IColumn & column, const capnp::DynamicList::Reader & parent_list_reader, UInt32 array_index) override { - auto reader_impl = parent_list_reader.getReaderImpl(); - auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, reader_impl.getStructElement(array_index)); - if (auto * tuple_column = typeid_cast(&column)) - { - for (size_t i = 0; i != tuple_column->tupleSize(); ++i) - fields_serializers[i]->readRow(tuple_column->getColumn(i), struct_reader, fields_offsets[i]); - } - else - fields_serializers[0]->readRow(column, struct_reader, fields_offsets[0]); + auto struct_reader = capnp::DynamicStruct::Reader(struct_schema, parent_list_reader.getReaderImpl().getStructElement(array_index)); + readRowImpl(column, struct_reader); } + /// Method for reading from root struct. void readRow(MutableColumns & columns, const capnp::DynamicStruct::Reader & reader) { for (size_t i = 0; i != columns.size(); ++i) @@ -1435,6 +1301,36 @@ namespace } } + template + void writeRowImpl(const ColumnPtr & column, std::unique_ptr & field_builder, ParentBuilder & parent_builder, UInt32 offset_or_index, size_t row_num) + { + if (!field_builder) + field_builder = initStructBuilder(parent_builder, offset_or_index, struct_size, fields_count, struct_schema); + + auto & struct_builder = assert_cast(*field_builder); + if (const auto * tuple_column = typeid_cast(column.get())) + { + const auto & columns = tuple_column->getColumns(); + for (size_t i = 0; i != columns.size(); ++i) + fields_serializers[i]->writeRow(columns[i], struct_builder.field_builders[fields_indexes[i]], struct_builder.impl, fields_offsets[i], row_num); + } + else + { + fields_serializers[0]->writeRow(column, struct_builder.field_builders[fields_indexes[0]], struct_builder.impl, fields_offsets[0], row_num); + } + } + + void readRowImpl(IColumn & column, const capnp::DynamicStruct::Reader & struct_reader) + { + if (auto * tuple_column = typeid_cast(&column)) + { + for (size_t i = 0; i != tuple_column->tupleSize(); ++i) + fields_serializers[i]->readRow(tuple_column->getColumn(i), struct_reader, fields_offsets[i]); + } + else + fields_serializers[0]->readRow(column, struct_reader, fields_offsets[0]); + } + capnp::StructSchema struct_schema; capnp::_::StructSize struct_size; size_t fields_count; @@ -1515,12 +1411,12 @@ namespace return std::make_unique>(type, name, capnp_type, settings.enum_comparing_mode); case TypeIndex::String: if (capnp_type.isData()) - return std::make_unique>(type, name, capnp_type); - return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); case TypeIndex::FixedString: if (capnp_type.isData()) - return std::make_unique>(type, name, capnp_type); - return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); case TypeIndex::LowCardinality: return std::make_unique(type, name, capnp_type, settings); case TypeIndex::Nullable: diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp new file mode 100644 index 00000000000..106161ae620 --- /dev/null +++ b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query) +{ + auto * select_query = query->as(); + if (!select_query || !select_query->tables()) + return nullptr; + + auto * tables = select_query->tables()->as(); + auto * table_expression = tables->children[0]->as()->table_expression->as(); + if (!table_expression->table_function) + return nullptr; + + auto * table_function = table_expression->table_function->as(); + return table_function->arguments->as(); +} + +void addColumnsStructureToQueryWithClusterEngine(ASTPtr & query, const String & structure, size_t max_arguments, const String & function_name) +{ + ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); + if (!expression_list) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function {}, got '{}'", function_name, queryToString(query)); + auto structure_literal = std::make_shared(structure); + + if (expression_list->children.size() < 2 || expression_list->children.size() > max_arguments) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 2 to {} arguments in {} table functions, got {}", + function_name, max_arguments, expression_list->children.size()); + + if (expression_list->children.size() == 2 || expression_list->children.size() == max_arguments - 1) + { + auto format_literal = std::make_shared("auto"); + expression_list->children.push_back(format_literal); + } + + expression_list->children.push_back(structure_literal); +} + +} diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.h b/src/Storages/addColumnsStructureToQueryWithClusterEngine.h new file mode 100644 index 00000000000..5939f3f43aa --- /dev/null +++ b/src/Storages/addColumnsStructureToQueryWithClusterEngine.h @@ -0,0 +1,14 @@ +#pragma once + +#include +#include + +namespace DB +{ + +ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query); + +/// Add structure argument for queries with s3Cluster/hdfsCluster table function. +void addColumnsStructureToQueryWithClusterEngine(ASTPtr & query, const String & structure, size_t max_arguments, const String & function_name); + +} From f2e076a4431e1515d46b039ef248ddd5fd33de81 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 31 May 2023 19:33:32 +0000 Subject: [PATCH 0886/2223] Implement big-endian support for the deterministic reservoir sampler --- .../ReservoirSamplerDeterministic.h | 14 ++--- src/Common/TransformEndianness.hpp | 54 +++++++++++++++++++ src/IO/ReadHelpers.h | 22 +------- src/IO/WriteHelpers.h | 22 +------- 4 files changed, 65 insertions(+), 47 deletions(-) create mode 100644 src/Common/TransformEndianness.hpp diff --git a/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/src/AggregateFunctions/ReservoirSamplerDeterministic.h index bde33260f5a..9dea821e839 100644 --- a/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -157,8 +157,8 @@ public: void read(DB::ReadBuffer & buf) { size_t size = 0; - DB::readIntBinary(size, buf); - DB::readIntBinary(total_values, buf); + readBinaryLittleEndian(size, buf); + readBinaryLittleEndian(total_values, buf); /// Compatibility with old versions. if (size > total_values) @@ -171,16 +171,16 @@ public: samples.resize(size); for (size_t i = 0; i < size; ++i) - DB::readPODBinary(samples[i], buf); + readBinaryLittleEndian(samples[i], buf); sorted = false; } void write(DB::WriteBuffer & buf) const { - size_t size = samples.size(); - DB::writeIntBinary(size, buf); - DB::writeIntBinary(total_values, buf); + const auto size = samples.size(); + writeBinaryLittleEndian(size, buf); + writeBinaryLittleEndian(total_values, buf); for (size_t i = 0; i < size; ++i) { @@ -195,7 +195,7 @@ public: memset(&elem, 0, sizeof(elem)); elem = samples[i]; - DB::writePODBinary(elem, buf); + writeBinaryLittleEndian(elem, buf); } } diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp new file mode 100644 index 00000000000..17cf441d17f --- /dev/null +++ b/src/Common/TransformEndianness.hpp @@ -0,0 +1,54 @@ +#pragma once + +#include +#include + +#include + +namespace DB +{ +template + requires is_big_int_v +inline void transformEndianness(T & x) +{ + if constexpr (std::endian::native != endian) + { + std::ranges::transform(x.items, std::begin(x.items), [](auto& item) { return std::byteswap(item); }); + std::ranges::reverse(x.items); + } +} + +template + requires is_decimal || std::is_floating_point_v +inline void transformEndianness(T & value) +{ + if constexpr (std::endian::native != endian) + { + auto * start = reinterpret_cast(&value); + std::reverse(start, start + sizeof(T)); + } +} + +template + requires std::is_integral_v && (sizeof(T) <= 8) +inline void transformEndianness(T & value) +{ + if constexpr (endian != std::endian::native) + value = std::byteswap(value); +} + +template + requires std::is_scoped_enum_v +inline void transformEndianness(T & x) +{ + using UnderlyingType = std::underlying_type_t; + transformEndianness(reinterpret_cast(x)); +} + +template +inline void transformEndianness(std::pair & pair) +{ + transformEndianness(pair.first); + transformEndianness(pair.second); +} +} diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 32338552b66..c42e992c807 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -1098,30 +1099,11 @@ inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf) inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); } inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); } - template -requires is_arithmetic_v && (sizeof(T) <= 8) inline void readBinaryEndian(T & x, ReadBuffer & buf) { readPODBinary(x, buf); - if constexpr (std::endian::native != endian) - x = std::byteswap(x); -} - -template -requires is_big_int_v -inline void readBinaryEndian(T & x, ReadBuffer & buf) -{ - if constexpr (std::endian::native == endian) - { - for (size_t i = 0; i != std::size(x.items); ++i) - readBinaryEndian(x.items[i], buf); - } - else - { - for (size_t i = 0; i != std::size(x.items); ++i) - readBinaryEndian(x.items[std::size(x.items) - i - 1], buf); - } + transformEndianness(x); } template diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index cdbc952690c..26c999cb761 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -1172,32 +1173,13 @@ inline void writeNullTerminatedString(const String & s, WriteBuffer & buffer) buffer.write(s.c_str(), s.size() + 1); } - template -requires is_arithmetic_v && (sizeof(T) <= 8) inline void writeBinaryEndian(T x, WriteBuffer & buf) { - if constexpr (std::endian::native != endian) - x = std::byteswap(x); + transformEndianness(x); writePODBinary(x, buf); } -template -requires is_big_int_v -inline void writeBinaryEndian(const T & x, WriteBuffer & buf) -{ - if constexpr (std::endian::native == endian) - { - for (size_t i = 0; i != std::size(x.items); ++i) - writeBinaryEndian(x.items[i], buf); - } - else - { - for (size_t i = 0; i != std::size(x.items); ++i) - writeBinaryEndian(x.items[std::size(x.items) - i - 1], buf); - } -} - template inline void writeBinaryLittleEndian(T x, WriteBuffer & buf) { From 4b46486491a80ab7a0f2bd62d6c6e4fb606aa429 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 31 May 2023 21:52:58 +0200 Subject: [PATCH 0887/2223] Clean up --- ...lumnsStructureToQueryWithClusterEngine.cpp | 52 ------------------- 1 file changed, 52 deletions(-) delete mode 100644 src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp deleted file mode 100644 index 106161ae620..00000000000 --- a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp +++ /dev/null @@ -1,52 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query) -{ - auto * select_query = query->as(); - if (!select_query || !select_query->tables()) - return nullptr; - - auto * tables = select_query->tables()->as(); - auto * table_expression = tables->children[0]->as()->table_expression->as(); - if (!table_expression->table_function) - return nullptr; - - auto * table_function = table_expression->table_function->as(); - return table_function->arguments->as(); -} - -void addColumnsStructureToQueryWithClusterEngine(ASTPtr & query, const String & structure, size_t max_arguments, const String & function_name) -{ - ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); - if (!expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function {}, got '{}'", function_name, queryToString(query)); - auto structure_literal = std::make_shared(structure); - - if (expression_list->children.size() < 2 || expression_list->children.size() > max_arguments) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 2 to {} arguments in {} table functions, got {}", - function_name, max_arguments, expression_list->children.size()); - - if (expression_list->children.size() == 2 || expression_list->children.size() == max_arguments - 1) - { - auto format_literal = std::make_shared("auto"); - expression_list->children.push_back(format_literal); - } - - expression_list->children.push_back(structure_literal); -} - -} From 4987c4baaa0c10ed1114155a0d35db3953f34ab9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 31 May 2023 21:53:21 +0200 Subject: [PATCH 0888/2223] Clean up --- .../addColumnsStructureToQueryWithClusterEngine.h | 14 -------------- 1 file changed, 14 deletions(-) delete mode 100644 src/Storages/addColumnsStructureToQueryWithClusterEngine.h diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.h b/src/Storages/addColumnsStructureToQueryWithClusterEngine.h deleted file mode 100644 index 5939f3f43aa..00000000000 --- a/src/Storages/addColumnsStructureToQueryWithClusterEngine.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query); - -/// Add structure argument for queries with s3Cluster/hdfsCluster table function. -void addColumnsStructureToQueryWithClusterEngine(ASTPtr & query, const String & structure, size_t max_arguments, const String & function_name); - -} From 3543d95980d5426fdc496bb3cafb572b8a62cf71 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 31 May 2023 22:10:33 +0200 Subject: [PATCH 0889/2223] Parallel replicas always skip unavailable ones (#50293) --- programs/server/config.xml | 5 ++ src/Interpreters/InterpreterSelectQuery.cpp | 6 +++ src/QueryPipeline/RemoteQueryExecutor.cpp | 39 ++++++++++----- src/QueryPipeline/RemoteQueryExecutor.h | 4 +- .../ParallelReplicasReadingCoordinator.cpp | 49 ++++++++++++++++--- .../ParallelReplicasReadingCoordinator.h | 7 ++- ...llel_replicas_unavailable_shards.reference | 2 + ...9_parallel_replicas_unavailable_shards.sql | 15 ++++++ 8 files changed, 105 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.reference create mode 100644 tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql diff --git a/programs/server/config.xml b/programs/server/config.xml index 5b69d9f6283..d18b4cb2ac9 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -909,6 +909,11 @@ 127.0.0.10 9000 + + + 127.0.0.11 + 1234 + diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a4ea474e4e9..d2be48dafb3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -468,6 +468,12 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } + /// Set skip_unavailable_shards to true only if it wasn't disabled explicitly + if (settings.allow_experimental_parallel_reading_from_replicas > 0 && !settings.skip_unavailable_shards && !settings.isChanged("skip_unavailable_shards")) + { + context->setSetting("skip_unavailable_shards", true); + } + /// Check support for JOIN for parallel replicas with custom key if (joined_tables.tablesCount() > 1 && !settings.parallel_replicas_custom_key.value.empty()) { diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 991876eccbd..3f9f945fd45 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -47,8 +47,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( QueryProcessingStage::Enum stage_, std::optional extension_) : header(header_), query(query_), context(context_), scalars(scalars_) , external_tables(external_tables_), stage(stage_) - , task_iterator(extension_ ? extension_->task_iterator : nullptr) - , parallel_reading_coordinator(extension_ ? extension_->parallel_reading_coordinator : nullptr) + , extension(extension_) {} RemoteQueryExecutor::RemoteQueryExecutor( @@ -90,8 +89,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( QueryProcessingStage::Enum stage_, std::optional extension_) : header(header_), query(query_), context(context_) , scalars(scalars_), external_tables(external_tables_), stage(stage_) - , task_iterator(extension_ ? extension_->task_iterator : nullptr) - , parallel_reading_coordinator(extension_ ? extension_->parallel_reading_coordinator : nullptr) + , extension(extension_) { create_connections = [this, connections_, throttler, extension_](AsyncCallback) mutable { auto res = std::make_unique(std::move(connections_), context->getSettingsRef(), throttler); @@ -108,8 +106,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( QueryProcessingStage::Enum stage_, std::optional extension_) : header(header_), query(query_), context(context_) , scalars(scalars_), external_tables(external_tables_), stage(stage_) - , task_iterator(extension_ ? extension_->task_iterator : nullptr) - , parallel_reading_coordinator(extension_ ? extension_->parallel_reading_coordinator : nullptr) + , extension(extension_) { create_connections = [this, pool, throttler, extension_](AsyncCallback async_callback)->std::unique_ptr { @@ -247,6 +244,13 @@ void RemoteQueryExecutor::sendQueryUnlocked(ClientInfo::QueryKind query_kind, As finished = true; sent_query = true; + /// We need to tell the coordinator not to wait for this replica. + if (extension && extension->parallel_reading_coordinator) + { + chassert(extension->replica_info); + extension->parallel_reading_coordinator->markReplicaAsUnavailable(extension->replica_info->number_of_current_replica); + } + return; } @@ -360,7 +364,18 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync() read_context->resume(); if (needToSkipUnavailableShard()) + { + /// We need to tell the coordinator not to wait for this replica. + /// But at this point it may lead to an incomplete result set, because + /// this replica committed to read some part of there data and then died. + if (extension && extension->parallel_reading_coordinator) + { + chassert(extension->parallel_reading_coordinator); + extension->parallel_reading_coordinator->markReplicaAsUnavailable(extension->replica_info->number_of_current_replica); + } + return ReadResult(Block()); + } /// Check if packet is not ready yet. if (read_context->isInProgress()) @@ -524,30 +539,30 @@ bool RemoteQueryExecutor::setPartUUIDs(const std::vector & uuids) void RemoteQueryExecutor::processReadTaskRequest() { - if (!task_iterator) + if (!extension || !extension->task_iterator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Distributed task iterator is not initialized"); ProfileEvents::increment(ProfileEvents::ReadTaskRequestsReceived); - auto response = (*task_iterator)(); + auto response = (*extension->task_iterator)(); connections->sendReadTaskResponse(response); } void RemoteQueryExecutor::processMergeTreeReadTaskRequest(ParallelReadRequest request) { - if (!parallel_reading_coordinator) + if (!extension || !extension->parallel_reading_coordinator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized"); ProfileEvents::increment(ProfileEvents::MergeTreeReadTaskRequestsReceived); - auto response = parallel_reading_coordinator->handleRequest(std::move(request)); + auto response = extension->parallel_reading_coordinator->handleRequest(std::move(request)); connections->sendMergeTreeReadTaskResponse(response); } void RemoteQueryExecutor::processMergeTreeInitialReadAnnounecement(InitialAllRangesAnnouncement announcement) { - if (!parallel_reading_coordinator) + if (!extension || !extension->parallel_reading_coordinator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized"); - parallel_reading_coordinator->handleInitialAllRangesAnnouncement(announcement); + extension->parallel_reading_coordinator->handleInitialAllRangesAnnouncement(announcement); } void RemoteQueryExecutor::finish() diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 96dc5510bf4..a843ce520de 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -212,11 +212,11 @@ private: /// Temporary tables needed to be sent to remote servers Tables external_tables; QueryProcessingStage::Enum stage; + + std::optional extension; /// Initiator identifier for distributed task processing std::shared_ptr task_iterator; - std::shared_ptr parallel_reading_coordinator; - /// This is needed only for parallel reading from replicas, because /// we create a RemoteQueryExecutor per replica and have to store additional info /// about the number of the current replica or the count of replicas at all. diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index ee38cecb9c4..bb044d15ba2 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -19,6 +19,7 @@ #include "Storages/MergeTree/RequestResponse.h" #include #include +#include #include namespace DB @@ -61,18 +62,22 @@ public: { size_t number_of_requests{0}; size_t sum_marks{0}; + bool is_unavailable{false}; }; using Stats = std::vector; static String toString(Stats stats) { String result = "Statistics: "; + std::vector stats_by_replica; for (size_t i = 0; i < stats.size(); ++i) - result += fmt::format("-- replica {}, requests: {} marks: {} ", i, stats[i].number_of_requests, stats[i].sum_marks); + stats_by_replica.push_back(fmt::format("replica {}{} - {{requests: {} marks: {}}}", i, stats[i].is_unavailable ? " is unavailable" : "", stats[i].number_of_requests, stats[i].sum_marks)); + result += fmt::format("{}", fmt::join(stats_by_replica, "; ")); return result; } Stats stats; - size_t replicas_count; + size_t replicas_count{0}; + size_t unavailable_replicas_count{0}; explicit ImplInterface(size_t replicas_count_) : stats{replicas_count_} @@ -82,6 +87,7 @@ public: virtual ~ImplInterface() = default; virtual ParallelReadResponse handleRequest(ParallelReadRequest request) = 0; virtual void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) = 0; + virtual void markReplicaAsUnavailable(size_t replica_number) = 0; }; using Parts = std::set; @@ -128,6 +134,7 @@ public: ParallelReadResponse handleRequest(ParallelReadRequest request) override; void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) override; + void markReplicaAsUnavailable(size_t replica_number) override; void updateReadingState(const InitialAllRangesAnnouncement & announcement); void finalizeReadingState(); @@ -199,6 +206,17 @@ void DefaultCoordinator::updateReadingState(const InitialAllRangesAnnouncement & } } +void DefaultCoordinator::markReplicaAsUnavailable(size_t replica_number) +{ + LOG_DEBUG(log, "Replica number {} is unavailable", replica_number); + + ++unavailable_replicas_count; + stats[replica_number].is_unavailable = true; + + if (sent_initial_requests == replicas_count - unavailable_replicas_count) + finalizeReadingState(); +} + void DefaultCoordinator::finalizeReadingState() { /// Clear all the delayed queue @@ -345,12 +363,23 @@ public: ParallelReadResponse handleRequest([[ maybe_unused ]] ParallelReadRequest request) override; void handleInitialAllRangesAnnouncement([[ maybe_unused ]] InitialAllRangesAnnouncement announcement) override; + void markReplicaAsUnavailable(size_t replica_number) override; Parts all_parts_to_read; Poco::Logger * log = &Poco::Logger::get(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator")); }; +template +void InOrderCoordinator::markReplicaAsUnavailable(size_t replica_number) +{ + LOG_DEBUG(log, "Replica number {} is unavailable", replica_number); + + stats[replica_number].is_unavailable = true; + ++unavailable_replicas_count; + + /// There is nothing to do else. +} template void InOrderCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) @@ -388,7 +417,6 @@ void InOrderCoordinator::handleInitialAllRangesAnnouncement(InitialAllRang } } - template ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest request) { @@ -486,7 +514,7 @@ void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(Init if (!pimpl) { - setMode(announcement.mode); + mode = announcement.mode; initialize(); } @@ -500,16 +528,23 @@ ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelR if (!pimpl) { - setMode(request.mode); + mode = request.mode; initialize(); } return pimpl->handleRequest(std::move(request)); } -void ParallelReplicasReadingCoordinator::setMode(CoordinationMode mode_) +void ParallelReplicasReadingCoordinator::markReplicaAsUnavailable(size_t replica_number) { - mode = mode_; + std::lock_guard lock(mutex); + + if (!pimpl) + { + initialize(); + } + + return pimpl->markReplicaAsUnavailable(replica_number); } void ParallelReplicasReadingCoordinator::initialize() diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index 0f41d24a9c6..ad8229be2d0 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -18,10 +18,15 @@ public: explicit ParallelReplicasReadingCoordinator(size_t replicas_count_); ~ParallelReplicasReadingCoordinator(); - void setMode(CoordinationMode mode); void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement); ParallelReadResponse handleRequest(ParallelReadRequest request); + /// Called when some replica is unavailable and we skipped it. + /// This is needed to "finalize" reading state e.g. spread all the marks using + /// consistent hashing, because otherwise coordinator will continue working in + /// "pending" state waiting for the unavailable replica to send the announcement. + void markReplicaAsUnavailable(size_t replica_number); + private: void initialize(); diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.reference b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.reference new file mode 100644 index 00000000000..af81158ecae --- /dev/null +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.reference @@ -0,0 +1,2 @@ +10 +1 diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql new file mode 100644 index 00000000000..ecc243b9c89 --- /dev/null +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test_parallel_replicas_unavailable_shards; +CREATE TABLE test_parallel_replicas_unavailable_shards (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10); + +SYSTEM FLUSH LOGS; + +SET skip_unavailable_shards=1, allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=11, use_hedged_requests=0, 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(*); + +SYSTEM FLUSH LOGS; + +SELECT count() > 0 FROM system.text_log WHERE yesterday() <= event_date AND message LIKE '%Replica number 10 is unavailable%'; + +DROP TABLE test_parallel_replicas_unavailable_shards; From da09823ecebe67d2d6983ac58ee38fe51d5d9e7f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 31 May 2023 23:14:39 +0300 Subject: [PATCH 0890/2223] Compile aggregate expressions enable by default --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 607be1522db..6fed70a9303 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -135,7 +135,7 @@ class IColumn; M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \ M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ - M(Bool, compile_aggregate_expressions, false, "Compile aggregate functions to native code. This feature has a bug and should not be used.", 0) \ + M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code. This feature has a bug and should not be used.", 0) \ M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \ M(Bool, compile_sort_description, true, "Compile sort description to native code.", 0) \ M(UInt64, min_count_to_compile_sort_description, 3, "The number of identical sort descriptions before they are JIT-compiled", 0) \ From 8b34a30455fd42928e5c89c503a25cf5d02ccff8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 31 May 2023 22:14:57 +0200 Subject: [PATCH 0891/2223] Fix style --- src/Formats/CapnProtoSerializer.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index e36f5fa4947..e99db23bb5e 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -29,7 +29,6 @@ namespace DB namespace ErrorCodes { extern const int THERE_IS_NO_COLUMN; - extern const int LOGICAL_ERROR; extern const int CAPN_PROTO_BAD_CAST; extern const int INCORRECT_DATA; extern const int ILLEGAL_COLUMN; @@ -371,7 +370,7 @@ namespace { parent_struct_builder.getBuilderImpl().setDataField(slot_offset, getValue(column, row_num)); } - + void writeRow(const ColumnPtr & column, std::unique_ptr &, capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) override { parent_list_builder.getBuilderImpl().setDataElement(array_index, getValue(column, row_num)); From 4ba08a5cbc960cace3dfcf32b3497855b3ffe6fd Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 31 May 2023 23:21:39 +0200 Subject: [PATCH 0892/2223] remove unused import --- src/DataTypes/Serializations/SerializationDate.h | 2 +- src/DataTypes/Serializations/SerializationDate32.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationDate.h b/src/DataTypes/Serializations/SerializationDate.h index 4d6a6fa36ec..f751b06fba6 100644 --- a/src/DataTypes/Serializations/SerializationDate.h +++ b/src/DataTypes/Serializations/SerializationDate.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationDate32.h b/src/DataTypes/Serializations/SerializationDate32.h index 6b6e5442240..49560fb6c7d 100644 --- a/src/DataTypes/Serializations/SerializationDate32.h +++ b/src/DataTypes/Serializations/SerializationDate32.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { From 4c92bc7aadf354b713d6b8f3c24728d6172f1867 Mon Sep 17 00:00:00 2001 From: tpanetti Date: Wed, 31 May 2023 15:30:26 -0700 Subject: [PATCH 0893/2223] Fix incompatible ClickHouse -> MySQL types for compability mode This adjusts specific incompatible ClickHouse types to a format that can be read and interpreted by MySQL (Ex: Int128 -> text) --- src/DataTypes/DataTypeArray.h | 2 +- src/DataTypes/DataTypeLowCardinality.cpp | 3 +- src/DataTypes/DataTypeLowCardinality.h | 4 +- src/DataTypes/DataTypeNumberBase.cpp | 8 +- src/DataTypes/DataTypeString.h | 2 +- ...show_columns_mysql_compatibility.reference | 229 ++++++++++++++++++ ...02775_show_columns_mysql_compatibility.sh} | 23 +- 7 files changed, 256 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference rename tests/queries/0_stateless/{02740_show_columns_mysql_compatibility.sh => 02775_show_columns_mysql_compatibility.sh} (80%) diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 35462df9a4e..b031f411975 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -32,7 +32,7 @@ public: } const char * getMySQLName() const override { - return "string"; + return "text"; } bool canBeInsideNullable() const override diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index 8293455cabc..b1c32317015 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -28,7 +28,8 @@ namespace ErrorCodes } DataTypeLowCardinality::DataTypeLowCardinality(DataTypePtr dictionary_type_) - : dictionary_type(std::move(dictionary_type_)) + : dictionary_type(std::move(dictionary_type_)), + mysql_name(dictionary_type->getMySQLName()) { auto inner_type = dictionary_type; if (dictionary_type->isNullable()) diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index 6fd4344311c..bcc39f58ff7 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -11,6 +11,8 @@ class DataTypeLowCardinality : public IDataType { private: DataTypePtr dictionary_type; + std::string mysql_name; + public: explicit DataTypeLowCardinality(DataTypePtr dictionary_type_); @@ -22,7 +24,7 @@ public: return "LowCardinality(" + dictionary_type->getName() + ")"; } const char * getFamilyName() const override { return "LowCardinality"; } - const char * getMySQLName() const override { return "text"; } + const char * getMySQLName() const override { return mysql_name.c_str(); } TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; } diff --git a/src/DataTypes/DataTypeNumberBase.cpp b/src/DataTypes/DataTypeNumberBase.cpp index cd5e73ac4a1..7d200de7996 100644 --- a/src/DataTypes/DataTypeNumberBase.cpp +++ b/src/DataTypes/DataTypeNumberBase.cpp @@ -36,14 +36,14 @@ const std::map DataTypeNumberBase::mysqlTypeMap = { {"UInt16", "smallint unsigned"}, {"UInt32", "mediumint unsigned"}, {"UInt64", "bigint unsigned"}, - {"UInt128", "bigint unsigned"}, - {"UInt256", "bigint unsigned"}, + {"UInt128", "text"}, + {"UInt256", "text"}, {"Int8", "tinyint"}, {"Int16", "smallint"}, {"Int32", "int"}, {"Int64", "bigint"}, - {"Int128", "bigint"}, - {"Int256", "bigint"}, + {"Int128", "text"}, + {"Int256", "text"}, {"Float32", "float"}, {"Float64", "double"}, }; diff --git a/src/DataTypes/DataTypeString.h b/src/DataTypes/DataTypeString.h index 3ac739fe68c..bddfb4ae287 100644 --- a/src/DataTypes/DataTypeString.h +++ b/src/DataTypes/DataTypeString.h @@ -22,7 +22,7 @@ public: } // FIXME: string can contain arbitrary bytes, not only UTF-8 sequences - const char * getMySQLName() const override { return "text"; } + const char * getMySQLName() const override { return "blob"; } TypeIndex getTypeId() const override { return type_id; } diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference new file mode 100644 index 00000000000..96e542611c6 --- /dev/null +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference @@ -0,0 +1,229 @@ +Drop tables if they exist +Create tab table +Create pseudo-random database name +Create tab duplicate table +Run MySQL test +field type null key default extra +aggregate_function text 0 NULL +array_value text 0 NULL +boolean_value tinyint unsigned 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value datetime 0 NULL +datetime_value datetime 0 NULL +decimal_value decimal 0 NULL +enum_value enum('apple', 'banana', 'orange') 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value json 0 NULL +low_cardinality blob 0 NULL +low_cardinality_date datetime 0 NULL +map_value json 0 NULL +nested.nested_int text 0 NULL +nested.nested_string text 0 NULL +nullable_value int 0 NULL +string_value blob 0 NULL +tuple_value json 0 NULL +uint64 bigint unsigned 0 PRI SOR NULL +uuid_value char 0 NULL +field type null key default extra +aggregate_function text 0 NULL +array_value text 0 NULL +boolean_value tinyint unsigned 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value datetime 0 NULL +datetime_value datetime 0 NULL +decimal_value decimal 0 NULL +enum_value enum('apple', 'banana', 'orange') 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value json 0 NULL +low_cardinality blob 0 NULL +low_cardinality_date datetime 0 NULL +map_value json 0 NULL +nested.nested_int text 0 NULL +nested.nested_string text 0 NULL +nullable_value int 0 NULL +string_value blob 0 NULL +tuple_value json 0 NULL +uint64 bigint unsigned 0 PRI SOR NULL +uuid_value char 0 NULL +field type null key default extra collation comment privileges +aggregate_function text 0 NULL NULL +array_value text 0 NULL NULL +boolean_value tinyint unsigned 0 NULL NULL +date32_value date 0 NULL NULL +date_value date 0 NULL NULL +datetime64_value datetime 0 NULL NULL +datetime_value datetime 0 NULL NULL +decimal_value decimal 0 NULL NULL +enum_value enum('apple', 'banana', 'orange') 0 NULL NULL +fixed_string_value text 0 NULL NULL +float32 float 0 NULL NULL +float64 double 0 NULL NULL +int32 int 0 NULL NULL +ipv4_value text 0 NULL NULL +ipv6_value text 0 NULL NULL +json_value json 0 NULL NULL +low_cardinality blob 0 NULL NULL +low_cardinality_date datetime 0 NULL NULL +map_value json 0 NULL NULL +nested.nested_int text 0 NULL NULL +nested.nested_string text 0 NULL NULL +nullable_value int 0 NULL NULL +string_value blob 0 NULL NULL +tuple_value json 0 NULL NULL +uint64 bigint unsigned 0 PRI SOR NULL NULL +uuid_value char 0 NULL NULL +field type null key default extra +int32 int 0 NULL +nested.nested_int text 0 NULL +uint64 bigint unsigned 0 PRI SOR NULL +field type null key default extra +aggregate_function text 0 NULL +array_value text 0 NULL +boolean_value tinyint unsigned 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value datetime 0 NULL +datetime_value datetime 0 NULL +decimal_value decimal 0 NULL +enum_value enum('apple', 'banana', 'orange') 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value json 0 NULL +low_cardinality blob 0 NULL +low_cardinality_date datetime 0 NULL +map_value json 0 NULL +nested.nested_string text 0 NULL +nullable_value int 0 NULL +string_value blob 0 NULL +tuple_value json 0 NULL +uuid_value char 0 NULL +field type null key default extra +int32 int 0 NULL +nested.nested_int text 0 NULL +uint64 bigint unsigned 0 PRI SOR NULL +field type null key default extra +aggregate_function text 0 NULL +array_value text 0 NULL +boolean_value tinyint unsigned 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value datetime 0 NULL +datetime_value datetime 0 NULL +decimal_value decimal 0 NULL +enum_value enum('apple', 'banana', 'orange') 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value json 0 NULL +low_cardinality blob 0 NULL +low_cardinality_date datetime 0 NULL +map_value json 0 NULL +nested.nested_string text 0 NULL +nullable_value int 0 NULL +string_value blob 0 NULL +tuple_value json 0 NULL +uuid_value char 0 NULL +field type null key default extra +int32 int 0 NULL +nested.nested_int text 0 NULL +uint64 bigint unsigned 0 PRI SOR NULL +field type null key default extra +aggregate_function text 0 NULL +field type null key default extra +aggregate_function text 0 NULL +array_value text 0 NULL +boolean_value tinyint unsigned 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value datetime 0 NULL +datetime_value datetime 0 NULL +decimal_value decimal 0 NULL +enum_value enum('apple', 'banana', 'orange') 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value json 0 NULL +low_cardinality blob 0 NULL +low_cardinality_date datetime 0 NULL +map_value json 0 NULL +nested.nested_int text 0 NULL +nested.nested_string text 0 NULL +nullable_value int 0 NULL +string_value blob 0 NULL +tuple_value json 0 NULL +uint64 bigint unsigned 0 PRI SOR NULL +uuid_value char 0 NULL +field type null key default extra +aggregate_function text 0 NULL +array_value text 0 NULL +boolean_value tinyint unsigned 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value datetime 0 NULL +datetime_value datetime 0 NULL +decimal_value decimal 0 NULL +enum_value enum('apple', 'banana', 'orange') 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value json 0 NULL +low_cardinality blob 0 NULL +low_cardinality_date datetime 0 NULL +map_value json 0 NULL +nested.nested_int text 0 NULL +nested.nested_string text 0 NULL +nullable_value int 0 NULL +string_value blob 0 NULL +tuple_value json 0 NULL +uint64 bigint unsigned 0 PRI SOR NULL +uuid_value char 0 NULL +field type null key default extra +aggregate_function text 0 NULL +array_value text 0 NULL +boolean_value tinyint unsigned 0 NULL +date32_value date 0 NULL +date_value date 0 NULL +datetime64_value datetime 0 NULL +datetime_value datetime 0 NULL +decimal_value decimal 0 NULL +enum_value enum('apple', 'banana', 'orange') 0 NULL +fixed_string_value text 0 NULL +float32 float 0 NULL +float64 double 0 NULL +int32 int 0 NULL +ipv4_value text 0 NULL +ipv6_value text 0 NULL +json_value json 0 NULL +low_cardinality blob 0 NULL +low_cardinality_date datetime 0 NULL +map_value json 0 NULL +nested.nested_int text 0 NULL +nested.nested_string text 0 NULL +nullable_value int 0 NULL +string_value blob 0 NULL +tuple_value json 0 NULL +uint64 bigint unsigned 0 PRI SOR NULL +uuid_value char 0 NULL diff --git a/tests/queries/0_stateless/02740_show_columns_mysql_compatibility.sh b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh similarity index 80% rename from tests/queries/0_stateless/02740_show_columns_mysql_compatibility.sh rename to tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh index 7f828d35679..a446c6e817e 100755 --- a/tests/queries/0_stateless/02740_show_columns_mysql_compatibility.sh +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh @@ -13,8 +13,11 @@ ${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS tab" ${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde" ${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde.tab" +#${CLICKHOUSE_LOCAL} --query "SET allow_suspicious_low_cardinality_types = 1;" echo "Create tab table " -${CLICKHOUSE_LOCAL} --query " +${CLICKHOUSE_LOCAL} -n -q " + SET allow_suspicious_low_cardinality_types=1; + SET allow_experimental_object_type =1; CREATE TABLE tab ( uint64 UInt64, @@ -22,17 +25,19 @@ ${CLICKHOUSE_LOCAL} --query " float32 Float32, float64 Float64, decimal_value Decimal(10, 2), - boolean_value UInt8, -- Use 0 for false, 1 for true + boolean_value UInt8, string_value String, fixed_string_value FixedString(10), date_value Date, date32_value Date32, datetime_value DateTime, datetime64_value DateTime64(3), - json_value String, -- Store JSON as a string + json_value JSON, uuid_value UUID, enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), low_cardinality LowCardinality(String), + low_cardinality_date LowCardinality(DateTime), + aggregate_function AggregateFunction(sum, Int32), array_value Array(Int32), map_value Map(String, Int32), tuple_value Tuple(Int32, String), @@ -53,7 +58,9 @@ echo "Create pseudo-random database name" ${CLICKHOUSE_LOCAL} --query "CREATE DATABASE database_123456789abcde;" echo "Create tab duplicate table" -${CLICKHOUSE_LOCAL} --query " +${CLICKHOUSE_LOCAL} -n -q " + SET allow_suspicious_low_cardinality_types=1; + SET allow_experimental_object_type =1; CREATE TABLE database_123456789abcde.tab ( uint64 UInt64, @@ -61,17 +68,19 @@ ${CLICKHOUSE_LOCAL} --query " float32 Float32, float64 Float64, decimal_value Decimal(10, 2), - boolean_value UInt8, -- Use 0 for false, 1 for true + boolean_value UInt8, string_value String, fixed_string_value FixedString(10), date_value Date, date32_value Date32, datetime_value DateTime, datetime64_value DateTime64(3), - json_value String, -- Store JSON as a string + json_value JSON, uuid_value UUID, enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), low_cardinality LowCardinality(String), + low_cardinality_date LowCardinality(DateTime), + aggregate_function AggregateFunction(sum, Int32), array_value Array(Int32), map_value Map(String, Int32), tuple_value Tuple(Int32, String), @@ -109,7 +118,7 @@ EOT # Now run the MySQL test script on the ClickHouse DB echo "Run MySQL test" -mysql --user="$USER" --password="$PASSWORD" --host="$HOST" --port="$PORT" < $TEMP_FILE +${MYSQL_CLIENT} --user="$USER" --password="$PASSWORD" --host="$HOST" --port="$PORT" < $TEMP_FILE # Clean up the temp file rm $TEMP_FILE From 801a4d574294b76a7f660bdf2e698d534f861ab3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 31 May 2023 20:08:22 -0300 Subject: [PATCH 0894/2223] test for #42610 --- ...10_view_dictionary_check_refresh.reference | 4 ++ .../01910_view_dictionary_check_refresh.sql | 54 +++++++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 tests/queries/0_stateless/01910_view_dictionary_check_refresh.reference create mode 100644 tests/queries/0_stateless/01910_view_dictionary_check_refresh.sql diff --git a/tests/queries/0_stateless/01910_view_dictionary_check_refresh.reference b/tests/queries/0_stateless/01910_view_dictionary_check_refresh.reference new file mode 100644 index 00000000000..c1be003ebef --- /dev/null +++ b/tests/queries/0_stateless/01910_view_dictionary_check_refresh.reference @@ -0,0 +1,4 @@ +view 1 2022-10-20 first +dict 1 2022-10-20 first +view 1 2022-10-21 second +dict 1 2022-10-21 second diff --git a/tests/queries/0_stateless/01910_view_dictionary_check_refresh.sql b/tests/queries/0_stateless/01910_view_dictionary_check_refresh.sql new file mode 100644 index 00000000000..b36a378d827 --- /dev/null +++ b/tests/queries/0_stateless/01910_view_dictionary_check_refresh.sql @@ -0,0 +1,54 @@ +-- Tags: long + +DROP DICTIONARY IF EXISTS TestTblDict; +DROP VIEW IF EXISTS TestTbl_view; +DROP TABLE IF EXISTS TestTbl; + +CREATE TABLE TestTbl +( + `id` UInt16, + `dt` Date, + `val` String +) +ENGINE = MergeTree +PARTITION BY dt +ORDER BY (id); + +CREATE VIEW TestTbl_view +AS +SELECT * +FROM TestTbl +WHERE dt = ( SELECT max(dt) FROM TestTbl ); + +CREATE DICTIONARY IF NOT EXISTS TestTblDict +( + `id` UInt16, + `dt` Date, + `val` String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE TestTbl_view DB currentDatabase())) +LIFETIME(1) +LAYOUT(COMPLEX_KEY_HASHED()); + +select 'view' src,* FROM TestTbl_view; +select 'dict' src,* FROM TestTblDict ; + +insert into TestTbl values(1, '2022-10-20', 'first'); + +SELECT sleep(3) from numbers(4) settings max_block_size= 1 format Null; + +select 'view' src,* FROM TestTbl_view; +select 'dict' src,* FROM TestTblDict ; + +insert into TestTbl values(1, '2022-10-21', 'second'); + +SELECT sleep(3) from numbers(4) settings max_block_size= 1 format Null; + +select 'view' src,* FROM TestTbl_view; +select 'dict' src,* FROM TestTblDict ; + +DROP DICTIONARY IF EXISTS TestTblDict; +DROP VIEW IF EXISTS TestTbl_view; +DROP TABLE IF EXISTS TestTbl; + From ad19d2142f86d9b205d3635685e092e8d61eebe4 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 31 May 2023 20:17:54 -0300 Subject: [PATCH 0895/2223] ping ci From 0aa30ef129ff0cc920c2e8c2880625d4e1dc6c1e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 31 May 2023 21:23:01 +0200 Subject: [PATCH 0896/2223] reuse s3_mocks, rewrite test test_paranoid_check_in_logs --- tests/integration/helpers/mock_servers.py | 26 ++++ .../integration/helpers/s3_mocks/__init__.py | 0 .../s3_mocks/broken_s3.py | 122 ++++++++++++++---- .../configs/storage_conf.xml | 14 +- .../test_checking_s3_blobs_paranoid/test.py | 31 +++-- tests/integration/test_merge_tree_s3/test.py | 111 ++++------------ 6 files changed, 175 insertions(+), 129 deletions(-) create mode 100644 tests/integration/helpers/s3_mocks/__init__.py rename tests/integration/{test_merge_tree_s3 => helpers}/s3_mocks/broken_s3.py (64%) diff --git a/tests/integration/helpers/mock_servers.py b/tests/integration/helpers/mock_servers.py index ea1f4e07a9f..e4655ffeeaf 100644 --- a/tests/integration/helpers/mock_servers.py +++ b/tests/integration/helpers/mock_servers.py @@ -1,6 +1,7 @@ import logging import os import time +import importlib # Starts simple HTTP servers written in Python. @@ -65,3 +66,28 @@ def start_mock_servers(cluster, script_dir, mocks, timeout=100): attempt += 1 logging.info(f"Mock {server_names_with_desc} started") + + +# The same as start_mock_servers, but +# import servers from central directory tests/integration/helpers +# and return the control instance +def start_s3_mock(cluster, mock_name, port, timeout=100): + script_dir = os.path.join(os.path.dirname(__file__), "s3_mocks") + registered_servers = [ + mock + for mock in os.listdir(script_dir) + if os.path.isfile(os.path.join(script_dir, mock)) + ] + + file_name = mock_name + ".py" + if file_name not in registered_servers: + raise KeyError( + f"Can't run s3 mock `{mock_name}`. No file `{file_name}` in directory `{script_dir}`" + ) + + start_mock_servers(cluster, script_dir, [(file_name, "resolver", port)], timeout) + + fmt = importlib.import_module("." + mock_name, "helpers.s3_mocks") + control = getattr(fmt, "MockControl")(cluster, "resolver", port) + + return control diff --git a/tests/integration/helpers/s3_mocks/__init__.py b/tests/integration/helpers/s3_mocks/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_merge_tree_s3/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py similarity index 64% rename from tests/integration/test_merge_tree_s3/s3_mocks/broken_s3.py rename to tests/integration/helpers/s3_mocks/broken_s3.py index 3adf3ba0047..026a3c6f515 100644 --- a/tests/integration/test_merge_tree_s3/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -12,7 +12,75 @@ UPSTREAM_HOST = "minio1" UPSTREAM_PORT = 9001 -class ServerRuntime: +class MockControl: + def __init__(self, cluster, container, port): + self._cluster = cluster + self._container = container + self._port = port + + def reset(self): + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + f"http://localhost:{self._port}/mock_settings/reset", + ], + nothrow=True, + ) + assert response == "OK" + + def setup_fail_upload(self, part_length): + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + f"http://localhost:{self._port}/mock_settings/error_at_put?when_length_bigger={part_length}", + ], + nothrow=True, + ) + assert response == "OK" + + def setup_fake_upload(self, part_length): + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + f"http://localhost:{self._port}/mock_settings/fake_put?when_length_bigger={part_length}", + ], + nothrow=True, + ) + assert response == "OK" + + def setup_slow_answers( + self, minimal_length=0, timeout=None, probability=None, count=None + ): + url = ( + f"http://localhost:{self._port}/" + f"mock_settings/slow_put" + f"?minimal_length={minimal_length}" + ) + + if timeout is not None: + url += f"&timeout={timeout}" + + if probability is not None: + url += f"&probability={probability}" + + if count is not None: + url += f"&count={count}" + + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + ["curl", "-s", url], + nothrow=True, + ) + assert response == "OK" + + +class _ServerRuntime: class SlowPut: def __init__( self, probability_=None, timeout_=None, minimal_length_=None, count_=None @@ -34,11 +102,11 @@ class ServerRuntime: if content_length > self.minimal_length: if self.count > 0: if ( - runtime.slow_put.probability == 1 - or random.random() <= runtime.slow_put.probability + _runtime.slow_put.probability == 1 + or random.random() <= _runtime.slow_put.probability ): self.count -= 1 - return runtime.slow_put.timeout + return _runtime.slow_put.timeout return None def __init__(self): @@ -65,10 +133,10 @@ class ServerRuntime: self.slow_put = None -runtime = ServerRuntime() +_runtime = _ServerRuntime() -def and_then(value, func): +def _and_then(value, func): assert callable(func) return None if value is None else func(value) @@ -153,28 +221,28 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "error_at_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - runtime.error_at_put_when_length_bigger = int( + _runtime.error_at_put_when_length_bigger = int( params.get("when_length_bigger", [1024 * 1024])[0] ) return self._ok() if path[1] == "fake_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - runtime.fake_put_when_length_bigger = int( + _runtime.fake_put_when_length_bigger = int( params.get("when_length_bigger", [1024 * 1024])[0] ) return self._ok() if path[1] == "slow_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - runtime.slow_put = ServerRuntime.SlowPut( - minimal_length_=and_then(params.get("minimal_length", [None])[0], int), - probability_=and_then(params.get("probability", [None])[0], float), - timeout_=and_then(params.get("timeout", [None])[0], float), - count_=and_then(params.get("count", [None])[0], int), + _runtime.slow_put = _ServerRuntime.SlowPut( + minimal_length_=_and_then(params.get("minimal_length", [None])[0], int), + probability_=_and_then(params.get("probability", [None])[0], float), + timeout_=_and_then(params.get("timeout", [None])[0], float), + count_=_and_then(params.get("count", [None])[0], int), ) - self.log_message("set slow put %s", runtime.slow_put) + self.log_message("set slow put %s", _runtime.slow_put) return self._ok() if path[1] == "reset": - runtime.reset() + _runtime.reset() return self._ok() return self._error("_mock_settings: wrong command") @@ -191,14 +259,14 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): def do_PUT(self): content_length = int(self.headers.get("Content-Length", 0)) - if runtime.slow_put is not None: - timeout = runtime.slow_put.get_timeout(content_length) + if _runtime.slow_put is not None: + timeout = _runtime.slow_put.get_timeout(content_length) if timeout is not None: self.log_message("slow put %s", timeout) time.sleep(timeout) - if runtime.error_at_put_when_length_bigger is not None: - if content_length > runtime.error_at_put_when_length_bigger: + if _runtime.error_at_put_when_length_bigger is not None: + if content_length > _runtime.error_at_put_when_length_bigger: return self._error( '' "" @@ -211,9 +279,10 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): parts = urllib.parse.urlsplit(self.path) params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) upload_id = params.get("uploadId", [None])[0] - if runtime.fake_put_when_length_bigger is not None and upload_id is not None: - if content_length > runtime.fake_put_when_length_bigger: - runtime.register_fake_upload(upload_id, parts.path) + if _runtime.fake_put_when_length_bigger is not None: + if content_length > _runtime.fake_put_when_length_bigger: + if upload_id is not None: + _runtime.register_fake_upload(upload_id, parts.path) return self._fake_put_ok() return self._redirect() @@ -223,7 +292,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) upload_id = params.get("uploadId", [None])[0] - if runtime.is_fake_upload(upload_id, parts.path): + if _runtime.is_fake_upload(upload_id, parts.path): return self._fake_post_ok(parts.path) return self._redirect() @@ -235,9 +304,10 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self._redirect() -class ThreadedHTTPServer(socketserver.ThreadingMixIn, http.server.HTTPServer): +class _ThreadedHTTPServer(socketserver.ThreadingMixIn, http.server.HTTPServer): """Handle requests in a separate thread.""" -httpd = ThreadedHTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) -httpd.serve_forever() +if __name__ == "__main__": + httpd = _ThreadedHTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) + httpd.serve_forever() diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml index 733205ce3e1..b77e72d808b 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml @@ -7,25 +7,25 @@ - + s3 - http://minio1:9001/root/data/ + http://resolver:8083/root/data/ minio minio123 - + - +
- s3 + broken_s3
-
+
- s3 + broken_s3 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index adb56b1899c..042d57a0c43 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -1,12 +1,10 @@ #!/usr/bin/env python3 import logging -import os -import time - +import pytest from helpers.cluster import ClickHouseCluster -import pytest +from helpers.mock_servers import start_s3_mock @pytest.fixture(scope="module") @@ -32,12 +30,23 @@ def cluster(): cluster.shutdown() -def test_paranoid_check_in_logs(cluster): +@pytest.fixture(scope="module") +def init_broken_s3(cluster): + yield start_s3_mock(cluster, "broken_s3", "8083") + + +@pytest.fixture(scope="function") +def broken_s3(init_broken_s3): + init_broken_s3.reset() + yield init_broken_s3 + + +def test_upload_after_check_works(cluster, broken_s3): node = cluster.instances["node"] node.query( """ - CREATE TABLE s3_failover_test ( + CREATE TABLE s3_upload_after_check_works ( id Int64, data String ) ENGINE=MergeTree() @@ -45,8 +54,12 @@ def test_paranoid_check_in_logs(cluster): """ ) - node.query("INSERT INTO s3_failover_test VALUES (1, 'Hello')") + broken_s3.setup_fake_upload(1) - assert node.contains_in_log("exists after upload") + error = node.query_and_get_error( + "INSERT INTO s3_upload_after_check_works VALUES (1, 'Hello')" + ) - assert node.query("SELECT * FROM s3_failover_test ORDER BY id") == "1\tHello\n" + assert "Code: 499" in error, error + assert "Immediately after upload" in error, error + assert "suddenly disappeared" in error, error diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 626a71f006e..7730bfcf7b2 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -4,12 +4,11 @@ import os import pytest from helpers.cluster import ClickHouseCluster -from helpers.mock_servers import start_mock_servers +from helpers.mock_servers import start_s3_mock, start_mock_servers from helpers.utility import generate_values, replace_config, SafeThread from helpers.wait_for_helpers import wait_for_delete_inactive_parts from helpers.wait_for_helpers import wait_for_delete_empty_parts from helpers.wait_for_helpers import wait_for_merges -from helpers.test_tools import assert_eq_with_retry SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -94,6 +93,17 @@ def create_table(node, table_name, **additional_settings): node.query(create_table_statement) +@pytest.fixture(scope="module") +def init_broken_s3(cluster): + yield start_s3_mock(cluster, "broken_s3", "8083") + + +@pytest.fixture(scope="function") +def broken_s3(init_broken_s3): + init_broken_s3.reset() + yield init_broken_s3 + + def run_s3_mocks(cluster): script_dir = os.path.join(os.path.dirname(__file__), "s3_mocks") start_mock_servers( @@ -102,7 +112,6 @@ def run_s3_mocks(cluster): [ ("unstable_proxy.py", "resolver", "8081"), ("no_delete_objects.py", "resolver", "8082"), - ("broken_s3.py", "resolver", "8083"), ], ) @@ -142,80 +151,6 @@ def clear_minio(cluster): yield -class BrokenS3: - @staticmethod - def reset(cluster): - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - [ - "curl", - "-s", - f"http://localhost:8083/mock_settings/reset", - ], - nothrow=True, - ) - assert response == "OK" - - @staticmethod - def setup_fail_upload(cluster, part_length): - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - [ - "curl", - "-s", - f"http://localhost:8083/mock_settings/error_at_put?when_length_bigger={part_length}", - ], - nothrow=True, - ) - assert response == "OK" - - @staticmethod - def setup_fake_upload(cluster, part_length): - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - [ - "curl", - "-s", - f"http://localhost:8083/mock_settings/fake_put?when_length_bigger={part_length}", - ], - nothrow=True, - ) - assert response == "OK" - - @staticmethod - def setup_slow_answers( - cluster, minimal_length=0, timeout=None, probability=None, count=None - ): - url = ( - f"http://localhost:8083/" - f"mock_settings/slow_put" - f"?minimal_length={minimal_length}" - ) - - if timeout is not None: - url += f"&timeout={timeout}" - - if probability is not None: - url += f"&probability={probability}" - - if count is not None: - url += f"&count={count}" - - response = cluster.exec_in_container( - cluster.get_container_id("resolver"), - ["curl", "-s", url], - nothrow=True, - ) - assert response == "OK" - - -@pytest.fixture(autouse=True, scope="function") -def reset_broken_s3(cluster): - BrokenS3.reset(cluster) - - yield - - def check_no_objects_after_drop(cluster, table_name="s3_test", node_name="node"): node = cluster.instances[node_name] node.query(f"DROP TABLE IF EXISTS {table_name} SYNC") @@ -932,7 +867,7 @@ def test_merge_canceled_by_drop(cluster, node_name): @pytest.mark.parametrize("storage_policy", ["broken_s3_always_multi_part", "broken_s3"]) @pytest.mark.parametrize("node_name", ["node"]) -def test_merge_canceled_by_s3_errors(cluster, node_name, storage_policy): +def test_merge_canceled_by_s3_errors(cluster, broken_s3, node_name, storage_policy): node = cluster.instances[node_name] node.query("DROP TABLE IF EXISTS test_merge_canceled_by_s3_errors NO DELAY") node.query( @@ -952,7 +887,7 @@ def test_merge_canceled_by_s3_errors(cluster, node_name, storage_policy): min_key = node.query("SELECT min(key) FROM test_merge_canceled_by_s3_errors") assert int(min_key) == 0, min_key - BrokenS3.setup_fail_upload(cluster, 50000) + broken_s3.setup_fail_upload(50000) node.query("SYSTEM START MERGES test_merge_canceled_by_s3_errors") @@ -969,7 +904,7 @@ def test_merge_canceled_by_s3_errors(cluster, node_name, storage_policy): @pytest.mark.parametrize("node_name", ["node"]) -def test_merge_canceled_by_s3_errors_when_move(cluster, node_name): +def test_merge_canceled_by_s3_errors_when_move(cluster, broken_s3, node_name): node = cluster.instances[node_name] settings = { "storage_policy": "external_broken_s3", @@ -995,7 +930,7 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, node_name): settings={"materialize_ttl_after_modify": 0}, ) - BrokenS3.setup_fail_upload(cluster, 10000) + broken_s3.setup_fail_upload(10000) node.query("SYSTEM START MERGES merge_canceled_by_s3_errors_when_move") @@ -1015,7 +950,9 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, node_name): @pytest.mark.parametrize( "in_flight_memory", [(10, 245918115), (5, 156786752), (1, 106426187)] ) -def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): +def test_s3_engine_heavy_write_check_mem( + cluster, broken_s3, node_name, in_flight_memory +): in_flight = in_flight_memory[0] memory = in_flight_memory[1] @@ -1029,8 +966,8 @@ def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): " ENGINE S3('http://resolver:8083/root/data/test-upload.csv', 'minio', 'minio123', 'CSV')", ) - BrokenS3.setup_fake_upload(cluster, 1000) - BrokenS3.setup_slow_answers(cluster, 10 * 1024 * 1024, timeout=15, count=10) + broken_s3.setup_fake_upload(1000) + broken_s3.setup_slow_answers(10 * 1024 * 1024, timeout=15, count=10) query_id = f"INSERT_INTO_S3_ENGINE_QUERY_ID_{in_flight}" node.query( @@ -1058,7 +995,7 @@ def test_s3_engine_heavy_write_check_mem(cluster, node_name, in_flight_memory): @pytest.mark.parametrize("node_name", ["node"]) -def test_s3_disk_heavy_write_check_mem(cluster, node_name): +def test_s3_disk_heavy_write_check_mem(cluster, broken_s3, node_name): memory = 2279055040 node = cluster.instances[node_name] @@ -1075,8 +1012,8 @@ def test_s3_disk_heavy_write_check_mem(cluster, node_name): ) node.query("SYSTEM STOP MERGES s3_test") - BrokenS3.setup_fake_upload(cluster, 1000) - BrokenS3.setup_slow_answers(cluster, 10 * 1024 * 1024, timeout=10, count=50) + broken_s3.setup_fake_upload(1000) + broken_s3.setup_slow_answers(10 * 1024 * 1024, timeout=10, count=50) query_id = f"INSERT_INTO_S3_DISK_QUERY_ID" node.query( From c54abdb4f8ad952f0402d10807fe65fb78a5a0b3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Jun 2023 01:32:03 +0200 Subject: [PATCH 0897/2223] Support clang-17 --- src/IO/WriteBufferFromS3TaskTracker.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 4abae90eeac..01455fc7fa3 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -121,17 +121,17 @@ void WriteBufferFromS3::TaskTracker::add(Callback && func) /// preallocation for the second issue FinishedList pre_allocated_finished {future_placeholder}; - Callback func_with_notification = [&, func=std::move(func), pre_allocated_finished=std::move(pre_allocated_finished)] () mutable + Callback func_with_notification = [&, my_func = std::move(func), my_pre_allocated_finished = std::move(pre_allocated_finished)]() mutable { SCOPE_EXIT({ DENY_ALLOCATIONS_IN_SCOPE; std::lock_guard lock(mutex); - finished_futures.splice(finished_futures.end(), pre_allocated_finished); + finished_futures.splice(finished_futures.end(), my_pre_allocated_finished); has_finished.notify_one(); }); - func(); + my_func(); }; /// this move is nothrow From 5dbce62ec365f8d4801faef4ef5a0cd8eb117615 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 31 May 2023 17:37:20 +0800 Subject: [PATCH 0898/2223] fixed. without apply prepareRightBlock will cause mismatch block structure --- src/Interpreters/GraceHashJoin.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index f54ee9d85c7..0dd61ff2793 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -623,6 +623,7 @@ Block GraceHashJoin::prepareRightBlock(const Block & block) void GraceHashJoin::addJoinedBlockImpl(Block block) { + block = prepareRightBlock(block); Buckets buckets_snapshot = getCurrentBuckets(); size_t bucket_index = current_bucket->idx; Block current_block; From 6334b6d5698f81429f5ae1351172ae89f5d7d2ea Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Wed, 31 May 2023 17:54:30 -0700 Subject: [PATCH 0899/2223] Address case where cpu cgroup is set to max. --- src/Common/AsynchronousMetrics.cpp | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 168dd3f0c4a..a4cb18249b6 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -945,12 +945,22 @@ void AsynchronousMetrics::update(TimePoint update_time) uint64_t quota = 0; uint64_t period = 0; - readText(quota, *cgroupcpu_max); - skipWhitespaceIfAny(*cgroupcpu_max); - readText(period, *cgroupcpu_max); + std::string line; + readText(line, *cgroupcpu_max); + + auto space = line.find_first_of(" "); + + if (line.rfind("max", 0) == std::string::npos) + { + auto field1 = line.substr(0, space); + quota = std::stoull(field1); + } + + auto field2 = line.substr(space + 1); + period = std::stoull(field2); new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; - new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup."}; + new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; } catch (...) { @@ -970,7 +980,7 @@ void AsynchronousMetrics::update(TimePoint update_time) tryReadText(period, *cgroupcpu_cfs_period); new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; - new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup."}; + new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; } catch (...) { From 956c399b2af27e6d70c2e4894657f010efb2594d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Jun 2023 03:04:29 +0200 Subject: [PATCH 0900/2223] Remove useless code --- src/Common/ProgressIndication.cpp | 5 +++-- src/Databases/DatabaseFactory.cpp | 1 - src/Storages/StorageDistributed.cpp | 7 ------- src/Storages/StorageS3.h | 1 - 4 files changed, 3 insertions(+), 11 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index df8778eb0d1..61b60060430 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -9,12 +9,13 @@ #include "Common/formatReadable.h" #include #include -#include "IO/WriteBufferFromString.h" -#include +#include +#include /// http://en.wikipedia.org/wiki/ANSI_escape_code #define CLEAR_TO_END_OF_LINE "\033[K" + namespace DB { diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index f346205ea79..e1c8afa52c0 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index fc552a5ab6d..bcf6f68d00d 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1137,13 +1137,6 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata auto cluster = getCluster(); const auto & settings = local_context->getSettingsRef(); - /// Ban an attempt to make async insert into the table belonging to DatabaseMemory - if (!storage_policy && !owned_cluster && !settings.insert_distributed_sync && !settings.insert_shard_id) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage {} must have own data directory to enable asynchronous inserts", - getName()); - } - auto shard_num = cluster->getLocalShardCount() + cluster->getRemoteShardCount(); /// If sharding key is not specified, then you can only write to a shard containing only one shard diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 12573ab513f..fae8d8eb3cb 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -31,7 +31,6 @@ namespace DB { class PullingPipelineExecutor; -class StorageS3SequentialSource; class NamedCollection; class StorageS3Source : public ISource, WithContext From fb86fe8f9dfb4fb96cca399b4967959424c8e959 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Jun 2023 03:08:05 +0200 Subject: [PATCH 0901/2223] Remove useless code --- src/Storages/StorageS3.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index fae8d8eb3cb..bca804a7f53 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -247,11 +247,6 @@ public: String getPath() const { return url.key; } - void appendToPath(const String & suffix) - { - url = S3::URI{std::filesystem::path(url.uri.toString()) / suffix}; - } - bool update(ContextPtr context); void connect(ContextPtr context); From c9d0d217f5d180f5c78286b1b819534dd8cc1b59 Mon Sep 17 00:00:00 2001 From: frinkr Date: Wed, 31 May 2023 19:53:06 +0800 Subject: [PATCH 0902/2223] fix Keeper deadlock on exception when preprocessing requests --- src/Coordination/KeeperStateMachine.cpp | 11 ++++++++++- src/Coordination/KeeperStateMachine.h | 2 ++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 6635c74149a..f787cc8722e 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -272,7 +272,8 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req } catch (...) { - rollbackRequest(request_for_session, true); + tryLogCurrentException(__PRETTY_FUNCTION__); + rollbackRequestNoLock(request_for_session, true); throw; } @@ -411,6 +412,14 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession storage->rollbackRequest(request_for_session.zxid, allow_missing); } +void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing) +{ + if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) + return; + + storage->rollbackRequest(request_for_session.zxid, allow_missing); +} + nuraft::ptr KeeperStateMachine::last_snapshot() { /// Just return the latest snapshot. diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index fbd97fd8631..f6d81d23056 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -68,6 +68,8 @@ public: // (can happen in case of exception during preprocessing) void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); + void rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); + uint64_t last_commit_index() override { return last_committed_idx; } /// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state. From 3d0f33664390607258eecd6982e3d787347c298d Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Mon, 29 May 2023 14:31:54 +0800 Subject: [PATCH 0903/2223] Optimize predicate with toYYYYMM converter Like toYear(PR #50062), the predicate with toYYYYMM could also be optimized with its preimage. For example, we could transform the predicate "toYYYYMM(c) = 202305" to "c >= '2023-05-01' AND c <= '2023-05-31'", so that the invocation of toYYYYMM is avoided. The similar transformation also applies to other comparisons. This commit extends OptimizeDateFilterInPlaceVisitor by having it recognize the toYYYYMM converter in the WHERE/PREWHERE clauses and replace it with the above tranformed predicate in-place at the AST level. --- .../OptimizeDateFilterVisitor.cpp | 33 +++++++-- ..._date_filter_predicate_optimizer.reference | 69 +++++++++++++++++++ .../02764_date_filter_predicate_optimizer.sql | 23 +++++++ 3 files changed, 120 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/OptimizeDateFilterVisitor.cpp b/src/Interpreters/OptimizeDateFilterVisitor.cpp index 58e1b3335f9..aec2dec19c8 100644 --- a/src/Interpreters/OptimizeDateFilterVisitor.cpp +++ b/src/Interpreters/OptimizeDateFilterVisitor.cpp @@ -10,14 +10,37 @@ namespace DB { -ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 year) +ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 compare_to) { const DateLUTImpl & date_lut = DateLUT::instance(); - if (converter != "toYear") return {}; + String start_date; + String end_date; - String start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); - String end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); + if (converter == "toYear") + { + UInt64 year = compare_to; + start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); + end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); + } + else if (converter == "toYYYYMM") + { + UInt64 year = compare_to / 100; + UInt64 month = compare_to % 100; + + if (month == 0 || month > 12) return {}; + + static constexpr UInt8 days_of_month[] = {31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; + + bool leap_year = (year & 3) == 0 && (year % 100 || (year % 400 == 0 && year)); + + start_date = date_lut.dateToString(date_lut.makeDayNum(year, month, 1)); + end_date = date_lut.dateToString(date_lut.makeDayNum(year, month, days_of_month[month - 1] + (leap_year && month == 2))); + } + else + { + return {}; + } if (comparator == "equals") { @@ -82,7 +105,7 @@ bool rewritePredicateInPlace(ASTFunction & function, ASTPtr & ast) { if (const auto * func = function.arguments->children[i]->as(); func) { - if (func->name == "toYear") + if (func->name == "toYear" || func->name == "toYYYYMM") { func_id = i; } diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference index e5c608ddc1a..54704fb3b3e 100644 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference @@ -37,3 +37,72 @@ WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) A SELECT value1 FROM t WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) +SELECT value1 +FROM t +WHERE ((date1 >= \'1900-02-01\') AND (date1 <= \'1900-02-28\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1992-02-01\') AND (date1 <= \'1992-02-29\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'2000-02-01\') AND (date1 <= \'2000-02-29\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (toYYYYMM(date1) = 199300) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-01-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-02-01\') AND (date1 <= \'1993-02-28\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-03-01\') AND (date1 <= \'1993-03-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-04-01\') AND (date1 <= \'1993-04-30\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-05-01\') AND (date1 <= \'1993-05-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-06-01\') AND (date1 <= \'1993-06-30\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-07-01\') AND (date1 <= \'1993-07-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-08-01\') AND (date1 <= \'1993-08-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-09-01\') AND (date1 <= \'1993-09-30\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-10-01\') AND (date1 <= \'1993-10-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-11-01\') AND (date1 <= \'1993-11-30\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1993-12-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (toYYYYMM(date1) = 199313) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 < \'1992-03-01\') OR (date1 > \'1992-03-31\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 < \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 > \'1992-03-31\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 <= \'1992-03-31\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE (date1 >= \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM t +WHERE ((date1 >= \'1992-03-01\') OR ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\'))) AND ((id >= 1) AND (id <= 3)) diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql index 563468d4f82..a04273bbb18 100644 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql @@ -13,5 +13,28 @@ EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM t WHERE year1 = 1993 A EXPLAIN SYNTAX SELECT value1 FROM t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; EXPLAIN SYNTAX SELECT value1 FROM t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; EXPLAIN SYNTAX SELECT value1 FROM t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 190002 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199202 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 200002 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199301 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199302 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199303 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199304 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199305 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199306 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199307 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199308 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199309 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199310 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199311 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3; DROP TABLE t; From bcbce15cde19eb1ec3705a572451812cb6e18c2c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Jun 2023 08:51:44 +0200 Subject: [PATCH 0904/2223] Mark the builds without results as pending --- tests/ci/build_report_check.py | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 82c04ce82c5..1362f3c8934 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -23,6 +23,7 @@ from get_robot_token import get_best_robot_token from pr_info import NeedsDataType, PRInfo from commit_status_helper import ( RerunHelper, + format_description, get_commit, post_commit_status, update_mergeable_check, @@ -269,14 +270,20 @@ def main(): if build_result.status == "success": ok_groups += 1 - if ok_groups == 0 or some_builds_are_missing: - summary_status = "error" + # Check if there are no builds at all, do not override bad status + if summary_status == "success": + if some_builds_are_missing: + summary_status = "pending" + elif ok_groups == 0: + summary_status = "error" addition = "" if some_builds_are_missing: - addition = f"({len(build_reports)} of {required_builds} builds are OK)" + addition = f" ({len(build_reports)} of {required_builds} builds are OK)" - description = f"{ok_groups}/{total_groups} artifact groups are OK {addition}" + description = format_description( + f"{ok_groups}/{total_groups} artifact groups are OK{addition}" + ) post_commit_status( commit, summary_status, url, description, build_check_name, pr_info From 08d98329b07ff772812999059a45af03352be030 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 10:12:09 +0200 Subject: [PATCH 0905/2223] Revert "Add SQL functions for Entropy Learned Hashing" --- .../sql-reference/functions/hash-functions.md | 71 ---- src/Functions/EntropyLearnedHash.cpp | 395 ------------------ ...new_functions_must_be_documented.reference | 3 - .../02734_entropy_learned_hashing.reference | 18 - .../02734_entropy_learned_hashing.sql | 30 -- 5 files changed, 517 deletions(-) delete mode 100644 src/Functions/EntropyLearnedHash.cpp delete mode 100644 tests/queries/0_stateless/02734_entropy_learned_hashing.reference delete mode 100644 tests/queries/0_stateless/02734_entropy_learned_hashing.sql diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 89afcca3799..8dfa03ceaf2 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -560,77 +560,6 @@ Result: └───────────────────────────┘ ``` -## Entropy-learned hashing (experimental) - -Entropy-learned hashing is disabled by default, to enable: `SET allow_experimental_hash_functions=1`. - -Entropy-learned hashing is not a standalone hash function like `metroHash64`, `cityHash64`, `sipHash64` etc. Instead, it aims to preprocess -the data to be hashed in a way that a standalone hash function can be computed more efficiently while not compromising the hash quality, -i.e. the randomness of the hashes. For that, entropy-based hashing chooses a subset of the bytes in a training data set of Strings which has -the same randomness (entropy) as the original Strings. For example, if the Strings are in average 100 bytes long, and we pick a subset of 5 -bytes, then a hash function will be 95% less expensive to evaluate. For details of the method, refer to [Entropy-Learned Hashing: Constant -Time Hashing with Controllable Uniformity](https://doi.org/10.1145/3514221.3517894). - -Entropy-learned hashing has two phases: - -1. A training phase on a representative but typically small set of Strings to be hashed. Training consists of two steps: - - - Function `prepareTrainEntropyLearnedHash(data, id)` caches the training data in a global state under a given `id`. It returns dummy - value `0` on every row. - - Function `trainEntropyLearnedHash(id)` computes a minimal partial sub-key of the training data stored stored under `id` in the global - state. The cached training data in the global state is replaced by the partial key. Dummy value `0` is returned on every row. - -2. An evaluation phase where hashes are computed using the previously calculated partial sub-keys. Function `entropyLearnedHash(data, id)` - hashes `data` using the partial subkey stored as `id`. CityHash64 is used as hash function. - -The reason that the training phase comprises two steps is that ClickHouse processes data at chunk granularity but entropy-learned hashing -needs to process the entire training set at once. - -Since functions `prepareTrainEntropyLearnedHash()` and `trainEntropyLearnedHash()` access global state, they should not be called in -parallel with the same `id`. - -**Syntax** - -``` sql -prepareTrainEntropyLearnedHash(data, id); -trainEntropyLearnedHash(id); -entropyLearnedHash(data, id); -``` - -**Example** - -```sql -SET allow_experimental_hash_functions=1; -CREATE TABLE tab (col String) ENGINE=Memory; -INSERT INTO tab VALUES ('aa'), ('ba'), ('ca'); - -SELECT prepareTrainEntropyLearnedHash(col, 'id1') AS prepared FROM tab; -SELECT trainEntropyLearnedHash('id1') AS trained FROM tab; -SELECT entropyLearnedHash(col, 'id1') as hashes FROM tab; -``` - -Result: - -``` response -┌─prepared─┐ -│ 0 │ -│ 0 │ -│ 0 │ -└──────────┘ - -┌─trained─┐ -│ 0 │ -│ 0 │ -│ 0 │ -└─────────┘ - -┌───────────────hashes─┐ -│ 2603192927274642682 │ -│ 4947675599669400333 │ -│ 10783339242466472992 │ -└──────────────────────┘ -``` - ## metroHash64 Produces a 64-bit [MetroHash](http://www.jandrewrogers.com/2015/05/27/metrohash/) hash value. diff --git a/src/Functions/EntropyLearnedHash.cpp b/src/Functions/EntropyLearnedHash.cpp deleted file mode 100644 index 854379bbb9d..00000000000 --- a/src/Functions/EntropyLearnedHash.cpp +++ /dev/null @@ -1,395 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -/// Implementation of entropy-learned hashing: https://doi.org/10.1145/3514221.3517894 -/// If you change something in this file, please don't deviate too much from the pseudocode in the paper! - -/// TODOs for future work: -/// - allow to specify an arbitrary hash function (currently always CityHash is used) -/// - allow function chaining a la entropyLearnedHash(trainEntropyLearnedHash()) -/// - support more datatypes for data (besides String) - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; - extern const int SUPPORT_IS_DISABLED; -} - -namespace -{ - -using PartialKeyPositions = std::vector; -using Entropies = std::vector; - -void getPartialKey(std::string_view key, const PartialKeyPositions & partial_key_positions, String & result) -{ - result.clear(); - result.reserve(partial_key_positions.size()); - - for (auto partial_key_position : partial_key_positions) - if (partial_key_position < key.size()) - result.push_back(key[partial_key_position]); -} - -bool allPartialKeysAreUnique(const std::vector & keys, const PartialKeyPositions & partial_key_positions) -{ - std::unordered_set unique_partial_keys; - unique_partial_keys.reserve(keys.size()); - String partial_key; - - for (const auto & key : keys) - { - getPartialKey(key, partial_key_positions, partial_key); - if (!unique_partial_keys.insert(partial_key).second) - return false; - } - - return true; -} - -// NextByte returns position of byte which adds the most entropy and the new entropy -std::pair nextByte(const std::vector & keys, size_t max_len, PartialKeyPositions & partial_key_positions) -{ - size_t min_collisions = std::numeric_limits::max(); - size_t best_position = 0; - - std::unordered_map count_table; - count_table.reserve(keys.size()); - - String partial_key; - - for (size_t i = 0; i < max_len; ++i) - { - count_table.clear(); - - partial_key_positions.push_back(i); - size_t collisions = 0; - for (const auto & key : keys) - { - getPartialKey(key, partial_key_positions, partial_key); - collisions += count_table[partial_key]++; - } - - if (collisions < min_collisions) - { - min_collisions = collisions; - best_position = i; - } - partial_key_positions.pop_back(); - } - - return {best_position, min_collisions}; -} - -std::pair chooseBytes(const std::vector & train_data) -{ - if (train_data.size() <= 1) - return {}; - - PartialKeyPositions partial_key_positions; - Entropies entropies; - - size_t max_len = 0; /// length of the longest key in training data - for (const auto & key : train_data) - max_len = std::max(max_len, key.size()); - - while (!allPartialKeysAreUnique(train_data, partial_key_positions)) - { - auto [new_position, new_entropy] = nextByte(train_data, max_len, partial_key_positions); - if (!entropies.empty() && new_entropy == entropies.back()) - break; - partial_key_positions.push_back(new_position); - entropies.push_back(new_entropy); - } - return {partial_key_positions, entropies}; -} - -/// Contains global state to convey information between SQL functions -/// - prepareTrainEntropyLearnedHash(), -/// - trainEntropyLearnedHash() and -/// - entropyLearnedHash(). -/// -/// The reason this machinery is necessary is that ClickHouse processes data in chunks of unpredictable size, yet the training step of -/// entropy-learned hashing needs to process *all* training data in one go. The downside is that the training step becomes quite expensive :-( -class EntropyLearnedHashGlobalState -{ -public: - static EntropyLearnedHashGlobalState & instance() - { - static EntropyLearnedHashGlobalState instance; - return instance; - } - - /// Called by prepareTrainEntropyLearnedHash() - void cacheTrainingSample(const String & user_name, const String & id, IColumn::MutablePtr column) - { - std::lock_guard lock(mutex); - auto & ids_for_user = global_state[user_name]; - auto & training_samples_for_id = ids_for_user[id].training_samples; - training_samples_for_id.push_back(std::move(column)); - } - - void train(const String & user_name, const String & id) - { - std::lock_guard lock(mutex); - auto & ids_for_user = global_state[user_name]; - auto & training_samples = ids_for_user[id].training_samples; - - if (training_samples.empty()) - return; - - auto & concatenated_training_sample = training_samples[0]; - for (size_t i = 1; i < training_samples.size(); ++i) - { - auto & other_training_sample = training_samples[i]; - concatenated_training_sample->insertRangeFrom(*other_training_sample, 0, other_training_sample->size()); - } - - const ColumnString * concatenated_training_sample_string = checkAndGetColumn(*concatenated_training_sample); - if (!concatenated_training_sample_string) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column"); - - const size_t num_rows = concatenated_training_sample_string->size(); - std::vector training_data; - for (size_t i = 0; i < num_rows; ++i) - { - std::string_view string_view = concatenated_training_sample_string->getDataAt(i).toView(); - training_data.emplace_back(string_view); - } - - PartialKeyPositions partial_key_positions = chooseBytes(training_data).first; - - ids_for_user[id].partial_key_positions = partial_key_positions; - training_samples.clear(); - } - - const PartialKeyPositions & getPartialKeyPositions(const String & user_name, const String & id) const - { - std::lock_guard lock(mutex); - auto it_user = global_state.find(user_name); - if (it_user == global_state.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Id {} not registered for user in entropy learned hashing", id); - auto it_id = it_user->second.find(id); - if (it_id == it_user->second.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Id {} not registered for user in entropy learned hashing", id); - return it_id->second.partial_key_positions; - } - -private: - mutable std::mutex mutex; - - /// The state. - struct ColumnsAndPartialKeyPositions - { - /// Caches training data chunks. Filled by prepareTrainEntropyLearnedHash(), cleared by trainEntropyLearnedHash(). - MutableColumns training_samples; - /// The result of the training phase. Filled by trainEntropyLearnedHash(). - PartialKeyPositions partial_key_positions; - }; - - /// Maps a state id to the state. - using IdToColumnsAndPartialKeyPositions = std::map; - - /// Maps the user name to a state id. As a result, the state id is unique at user scope. - using UserNameToId = std::map; - - UserNameToId global_state TSA_GUARDED_BY(mutex); -}; - -} - - -/// Copies all chunks of the training sample column into the global state under a given id. -class FunctionPrepareTrainEntropyLearnedHash : public IFunction -{ -public: - static constexpr auto name = "prepareTrainEntropyLearnedHash"; - static FunctionPtr create(ContextPtr context) - { - if (!context->getSettings().allow_experimental_hash_functions) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Entropy-learned hashing is experimental. Set `allow_experimental_hash_functions` setting to enable it"); - - return std::make_shared(context->getUserName()); - } - explicit FunctionPrepareTrainEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} - - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 2; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - FunctionArgumentDescriptors args{ - {"data", &isString, nullptr, "String"}, - {"id", &isString, nullptr, "String"} - }; - - validateFunctionArgumentTypes(*this, arguments, args); - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override - { - const IColumn * id_col = arguments[1].column.get(); - const ColumnConst * id_col_const = checkAndGetColumn(id_col); - const String id = id_col_const->getValue(); - - IColumn::Ptr data_col = arguments[0].column; - IColumn::MutablePtr data_col_mutable = IColumn::mutate(data_col); - - auto & global_state = EntropyLearnedHashGlobalState::instance(); - global_state.cacheTrainingSample(user_name, id, std::move(data_col_mutable)); - - const size_t num_rows = data_col->size(); - return result_type->createColumnConst(num_rows, 0u); /// dummy output - } -private: - const String user_name; -}; - - -/// 1. Concatenates the training samples of a given id in the global state. -/// 2. Computes the partial key positions from the concatenated training samples and stores that in the global state. -/// 3. clear()-s the training samples in the global state. -class FunctionTrainEntropyLearnedHash : public IFunction -{ -public: - static constexpr auto name = "trainEntropyLearnedHash"; - static FunctionPtr create(ContextPtr context) - { - if (!context->getSettings().allow_experimental_hash_functions) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Entropy-learned hashing is experimental. Set `allow_experimental_hash_functions` setting to enable it"); - return std::make_shared(context->getUserName()); - } - explicit FunctionTrainEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} - - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 1; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - bool useDefaultImplementationForConstants() const override { return false; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - FunctionArgumentDescriptors args{ - {"id", &isString, nullptr, "String"} - }; - - validateFunctionArgumentTypes(*this, arguments, args); - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override - { - const IColumn * id_col = arguments[0].column.get(); - const ColumnConst * id_col_const = checkAndGetColumn(id_col); - if (!id_col_const) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - arguments.begin()->column->getName(), getName()); - - auto & global_state = EntropyLearnedHashGlobalState::instance(); - - const String id = id_col_const->getValue(); - global_state.train(user_name, id); - - const size_t num_rows = id_col->size(); - return result_type->createColumnConst(num_rows, 0u); /// dummy output - } -private: - const String user_name; -}; - - -/// Hashes input strings using partial key positions stored in the global state. -class FunctionEntropyLearnedHash : public IFunction -{ -public: - static constexpr auto name = "entropyLearnedHash"; - static FunctionPtr create(ContextPtr context) - { - if (!context->getSettings().allow_experimental_hash_functions) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Entropy-learned hashing experimental. Set `allow_experimental_hash_functions` setting to enable it"); - return std::make_shared(context->getUserName()); - } - explicit FunctionEntropyLearnedHash(const String & user_name_) : IFunction(), user_name(user_name_) {} - - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 2; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - FunctionArgumentDescriptors args{ - {"data", &isString, nullptr, "String"}, - {"id", &isString, nullptr, "String"} - }; - - validateFunctionArgumentTypes(*this, arguments, args); - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override - { - const IColumn * id_col = arguments.back().column.get(); - const ColumnConst * id_col_const = checkAndGetColumn(id_col); - const String id = id_col_const->getValue(); - - const auto & global_state = EntropyLearnedHashGlobalState::instance(); - const auto & partial_key_positions = global_state.getPartialKeyPositions(user_name, id); - - const auto * data_col = arguments[0].column.get(); - if (const auto * col_data_string = checkAndGetColumn(data_col)) - { - const size_t num_rows = col_data_string->size(); - auto col_res = ColumnUInt64::create(num_rows); - - auto & col_res_vec = col_res->getData(); - String partial_key; - for (size_t i = 0; i < num_rows; ++i) - { - std::string_view string_ref = col_data_string->getDataAt(i).toView(); - getPartialKey(string_ref, partial_key_positions, partial_key); - col_res_vec[i] = CityHash_v1_0_2::CityHash64(partial_key.data(), partial_key.size()); - } - - return col_res; - } - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - arguments.begin()->column->getName(), getName()); - } -private: - const String user_name; -}; - -REGISTER_FUNCTION(EntropyLearnedHash) -{ - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); -} - -} diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index b2c9073648e..7ab26982402 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -266,7 +266,6 @@ encodeURLComponent encodeURLFormComponent encodeXMLComponent endsWith -entropyLearnedHash equals erf erfc @@ -559,7 +558,6 @@ positionCaseInsensitive positionCaseInsensitiveUTF8 positionUTF8 pow -prepareTrainEntropyLearnedHash proportionsZTest protocol queryID @@ -864,7 +862,6 @@ toYear toYearWeek today tokens -trainEntropyLearnedHash transactionID transactionLatestSnapshot transactionOldestSnapshot diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference b/tests/queries/0_stateless/02734_entropy_learned_hashing.reference deleted file mode 100644 index f558e3cd444..00000000000 --- a/tests/queries/0_stateless/02734_entropy_learned_hashing.reference +++ /dev/null @@ -1,18 +0,0 @@ -0 -0 -0 -0 -0 -0 -2603192927274642682 -4947675599669400333 -10783339242466472992 -0 -0 -0 -0 -0 -0 -2603192927274642682 -4947675599669400333 -10783339242466472992 diff --git a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql b/tests/queries/0_stateless/02734_entropy_learned_hashing.sql deleted file mode 100644 index ae829fa03c3..00000000000 --- a/tests/queries/0_stateless/02734_entropy_learned_hashing.sql +++ /dev/null @@ -1,30 +0,0 @@ --- Tags: no-parallel --- no-parallel because entropy-learned hash uses global state - -SET allow_experimental_hash_functions = 1; - --- no commonalities between keys -DROP TABLE IF EXISTS tbl1; -CREATE TABLE tbl1 (x String) ENGINE=Memory; -INSERT INTO tbl1 VALUES ('a'), ('b'), ('c'); -SELECT prepareTrainEntropyLearnedHash(x, 'id1') FROM tbl1; -SELECT trainEntropyLearnedHash('id1') FROM tbl1; -SELECT entropyLearnedHash(x, 'id1') FROM tbl1; - --- with commonalities between keys -DROP TABLE IF EXISTS tbl2; -CREATE TABLE tbl2 (x String) ENGINE=Memory; -INSERT INTO tbl2 VALUES ('aa'), ('ba'), ('ca'); -SELECT prepareTrainEntropyLearnedHash(x, 'id2') FROM tbl2; -SELECT trainEntropyLearnedHash('id2') FROM tbl2; -SELECT entropyLearnedHash(x, 'id2') FROM tbl2; - --- negative tests -SELECT prepareTrainEntropyLearnedHash(x, 1) FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT prepareTrainEntropyLearnedHash(1, 'id1') FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT trainEntropyLearnedHash(1) FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT entropyLearnedHash(1, 'id1') FROM tbl1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT entropyLearnedHash(x, 'non-existing id') FROM tbl1; -- { serverError BAD_ARGUMENTS } - -DROP TABLE tbl1; -DROP TABLE tbl2; From 86e97f5f5c80ae036655cf201dd50feac3563ad5 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 1 Jun 2023 03:19:23 -0600 Subject: [PATCH 0906/2223] Update reddit-comments.md --- .../example-datasets/reddit-comments.md | 616 ++++++++++-------- 1 file changed, 350 insertions(+), 266 deletions(-) diff --git a/docs/en/getting-started/example-datasets/reddit-comments.md b/docs/en/getting-started/example-datasets/reddit-comments.md index aa5c474103d..02961a07730 100644 --- a/docs/en/getting-started/example-datasets/reddit-comments.md +++ b/docs/en/getting-started/example-datasets/reddit-comments.md @@ -18,7 +18,7 @@ This dataset contains publicly-available comments on Reddit that go back to Dece A shoutout to Percona for the [motivation behind ingesting this dataset](https://www.percona.com/blog/big-data-set-reddit-comments-analyzing-clickhouse/), which we have downloaded and stored in an S3 bucket. :::note -The following commands were executed on ClickHouse Cloud. To run this on your own cluster, replace `default` in the `s3Cluster` function call with the name of your cluster. If you do not have a cluster, then replace the `s3Cluster` function with the `s3` function. +The following commands were executed on a Production instance of ClickHouse Cloud with the minimum memory set to 720GB. To run this on your own cluster, replace `default` in the `s3Cluster` function call with the name of your cluster. If you do not have a cluster, then replace the `s3Cluster` function with the `s3` function. ::: 1. Let's create a table for the Reddit data: @@ -187,14 +187,28 @@ LIMIT 10; TRUNCATE TABLE reddit; ``` -8. This is a fun dataset and it looks like we can find some great information, so let's go ahead and insert the entire dataset from 2005 to 2023. When you're ready, run this command to insert all the rows. (It takes a while - up to 17 hours!) +8. This is a fun dataset and it looks like we can find some great information, so let's go ahead and insert the entire dataset from 2005 to 2023. For practical reasons, it works well to insert the data by years starting with... + + +```sql +INSERT INTO reddit + SELECT * + FROM s3Cluster( + 'default', + 'https://clickhouse-public-datasets.s3.eu-central-1.amazonaws.com/reddit/original/RC_2005*', + 'JSONEachRow' + ) + SETTINGS zstd_window_log_max = 31; +``` + +...and ending with: ```sql INSERT INTO reddit SELECT * FROM s3Cluster( 'default', - 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC*', + 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC_2023*', 'JSONEachRow' ) SETTINGS zstd_window_log_max = 31; @@ -206,19 +220,48 @@ If you do not have a cluster, use `s3` instead of `s3Cluster`: INSERT INTO reddit SELECT * FROM s3( - 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC*', + 'https://clickhouse-public-datasets.s3.amazonaws.com/reddit/original/RC_2005*', 'JSONEachRow' ) SETTINGS zstd_window_log_max = 31; ``` -The response looks like: +8. To verify it worked, here are the number of rows per year (as of February, 2023): -```response -0 rows in set. Elapsed: 61187.839 sec. Processed 6.74 billion rows, 2.06 TB (110.17 thousand rows/s., 33.68 MB/s.) +```sql +SELECT + toYear(created_utc) AS year, + formatReadableQuantity(count()) +FROM reddit +GROUP BY year; ``` -8. Let's see how many rows were inserted and how much disk space the table is using: +```response + +┌─year─┬─formatReadableQuantity(count())─┐ +│ 2005 │ 1.07 thousand │ +│ 2006 │ 417.18 thousand │ +│ 2007 │ 2.46 million │ +│ 2008 │ 7.24 million │ +│ 2009 │ 18.86 million │ +│ 2010 │ 42.93 million │ +│ 2011 │ 28.91 million │ +│ 2012 │ 260.31 million │ +│ 2013 │ 402.21 million │ +│ 2014 │ 531.80 million │ +│ 2015 │ 667.76 million │ +│ 2016 │ 799.90 million │ +│ 2017 │ 972.86 million │ +│ 2018 │ 1.24 billion │ +│ 2019 │ 1.66 billion │ +│ 2020 │ 2.16 billion │ +│ 2021 │ 2.59 billion │ +│ 2022 │ 2.82 billion │ +│ 2023 │ 474.86 million │ +└──────┴─────────────────────────────────┘ +``` + +9. Let's see how many rows were inserted and how much disk space the table is using: ```sql @@ -228,17 +271,17 @@ SELECT formatReadableSize(sum(bytes)) AS disk_size, formatReadableSize(sum(data_uncompressed_bytes)) AS uncompressed_size FROM system.parts -WHERE (table = 'reddit') AND active +WHERE (table = 'reddit') AND active; ``` Notice the compression of disk storage is about 1/3 of the uncompressed size: ```response -┌──────count─┬─formatReadableQuantity(sum(rows))─┬─disk_size──┬─uncompressed_size─┐ -│ 6739503568 │ 6.74 billion │ 501.10 GiB │ 1.51 TiB │ -└────────────┴───────────────────────────────────┴────────────┴───────────────────┘ +┌───────count─┬─formatReadableQuantity(sum(rows))─┬─disk_size─┬─uncompressed_size─┐ +│ 14688534662 │ 14.69 billion │ 1.03 TiB │ 3.26 TiB │ +└─────────────┴───────────────────────────────────┴───────────┴───────────────────┘ -1 row in set. Elapsed: 0.010 sec. +1 row in set. Elapsed: 0.005 sec. ``` 9. The following query shows how many comments, authors and subreddits we have for each month: @@ -257,185 +300,216 @@ GROUP BY firstOfMonth ORDER BY firstOfMonth ASC; ``` -This is a substantial query that has to process all 6.74 billion rows, but we still get an impressive response time (about 3 minutes): +This is a substantial query that has to process all 14.69 billion rows, but we still get an impressive response time (about 48 seconds): ```response -┌─firstOfMonth─┬─────────c─┬─bar_count─────────────────┬─authors─┬─bar_authors───────────────┬─subreddits─┬─bar_subreddits────────────┐ -│ 2005-12-01 │ 1075 │ │ 394 │ │ 1 │ │ -│ 2006-01-01 │ 3666 │ │ 791 │ │ 2 │ │ -│ 2006-02-01 │ 9095 │ │ 1464 │ │ 18 │ │ -│ 2006-03-01 │ 13859 │ │ 1958 │ │ 15 │ │ -│ 2006-04-01 │ 19090 │ │ 2334 │ │ 21 │ │ -│ 2006-05-01 │ 26859 │ │ 2698 │ │ 21 │ │ -│ 2006-06-01 │ 29163 │ │ 3043 │ │ 19 │ │ -│ 2006-07-01 │ 37031 │ │ 3532 │ │ 22 │ │ -│ 2006-08-01 │ 50559 │ │ 4750 │ │ 24 │ │ -│ 2006-09-01 │ 50675 │ │ 4908 │ │ 21 │ │ -│ 2006-10-01 │ 54148 │ │ 5654 │ │ 31 │ │ -│ 2006-11-01 │ 62021 │ │ 6490 │ │ 23 │ │ -│ 2006-12-01 │ 61018 │ │ 6707 │ │ 24 │ │ -│ 2007-01-01 │ 81341 │ │ 7931 │ │ 23 │ │ -│ 2007-02-01 │ 95634 │ │ 9020 │ │ 21 │ │ -│ 2007-03-01 │ 112444 │ │ 10842 │ │ 23 │ │ -│ 2007-04-01 │ 126773 │ │ 10701 │ │ 26 │ │ -│ 2007-05-01 │ 170097 │ │ 11365 │ │ 25 │ │ -│ 2007-06-01 │ 178800 │ │ 11267 │ │ 22 │ │ -│ 2007-07-01 │ 203319 │ │ 12482 │ │ 25 │ │ -│ 2007-08-01 │ 225111 │ │ 14124 │ │ 30 │ │ -│ 2007-09-01 │ 259497 │ ▏ │ 15416 │ │ 33 │ │ -│ 2007-10-01 │ 274170 │ ▏ │ 15302 │ │ 36 │ │ -│ 2007-11-01 │ 372983 │ ▏ │ 15134 │ │ 43 │ │ -│ 2007-12-01 │ 363390 │ ▏ │ 15915 │ │ 31 │ │ -│ 2008-01-01 │ 452990 │ ▏ │ 18857 │ │ 126 │ │ -│ 2008-02-01 │ 441768 │ ▏ │ 18266 │ │ 173 │ │ -│ 2008-03-01 │ 463728 │ ▏ │ 18947 │ │ 292 │ │ -│ 2008-04-01 │ 468317 │ ▏ │ 18590 │ │ 323 │ │ -│ 2008-05-01 │ 536380 │ ▎ │ 20861 │ │ 375 │ │ -│ 2008-06-01 │ 577684 │ ▎ │ 22557 │ │ 575 │ ▏ │ -│ 2008-07-01 │ 592610 │ ▎ │ 23123 │ │ 657 │ ▏ │ -│ 2008-08-01 │ 595959 │ ▎ │ 23729 │ │ 707 │ ▏ │ -│ 2008-09-01 │ 680892 │ ▎ │ 26374 │ ▏ │ 801 │ ▏ │ -│ 2008-10-01 │ 789874 │ ▍ │ 28970 │ ▏ │ 893 │ ▏ │ -│ 2008-11-01 │ 792310 │ ▍ │ 30272 │ ▏ │ 1024 │ ▎ │ -│ 2008-12-01 │ 850359 │ ▍ │ 34073 │ ▏ │ 1103 │ ▎ │ -│ 2009-01-01 │ 1051649 │ ▌ │ 38978 │ ▏ │ 1316 │ ▎ │ -│ 2009-02-01 │ 944711 │ ▍ │ 43390 │ ▏ │ 1132 │ ▎ │ -│ 2009-03-01 │ 1048643 │ ▌ │ 46516 │ ▏ │ 1203 │ ▎ │ -│ 2009-04-01 │ 1094599 │ ▌ │ 48284 │ ▏ │ 1334 │ ▎ │ -│ 2009-05-01 │ 1201257 │ ▌ │ 52512 │ ▎ │ 1395 │ ▎ │ -│ 2009-06-01 │ 1258750 │ ▋ │ 57728 │ ▎ │ 1473 │ ▎ │ -│ 2009-07-01 │ 1470290 │ ▋ │ 60098 │ ▎ │ 1686 │ ▍ │ -│ 2009-08-01 │ 1750688 │ ▉ │ 67347 │ ▎ │ 1777 │ ▍ │ -│ 2009-09-01 │ 2032276 │ █ │ 78051 │ ▍ │ 1784 │ ▍ │ -│ 2009-10-01 │ 2242017 │ █ │ 93409 │ ▍ │ 2071 │ ▌ │ -│ 2009-11-01 │ 2207444 │ █ │ 95940 │ ▍ │ 2141 │ ▌ │ -│ 2009-12-01 │ 2560510 │ █▎ │ 104239 │ ▌ │ 2141 │ ▌ │ -│ 2010-01-01 │ 2884096 │ █▍ │ 114314 │ ▌ │ 2313 │ ▌ │ -│ 2010-02-01 │ 2687779 │ █▎ │ 115683 │ ▌ │ 2522 │ ▋ │ -│ 2010-03-01 │ 3228254 │ █▌ │ 125775 │ ▋ │ 2890 │ ▋ │ -│ 2010-04-01 │ 3209898 │ █▌ │ 128936 │ ▋ │ 3170 │ ▊ │ -│ 2010-05-01 │ 3267363 │ █▋ │ 131851 │ ▋ │ 3166 │ ▊ │ -│ 2010-06-01 │ 3532867 │ █▊ │ 139522 │ ▋ │ 3301 │ ▊ │ -│ 2010-07-01 │ 4032737 │ ██ │ 153451 │ ▊ │ 3662 │ ▉ │ -│ 2010-08-01 │ 4247982 │ ██ │ 164071 │ ▊ │ 3653 │ ▉ │ -│ 2010-09-01 │ 4704069 │ ██▎ │ 186613 │ ▉ │ 4009 │ █ │ -│ 2010-10-01 │ 5032368 │ ██▌ │ 203800 │ █ │ 4154 │ █ │ -│ 2010-11-01 │ 5689002 │ ██▊ │ 226134 │ █▏ │ 4383 │ █ │ -│ 2010-12-01 │ 5972642 │ ██▉ │ 245824 │ █▏ │ 4692 │ █▏ │ -│ 2011-01-01 │ 6603329 │ ███▎ │ 270025 │ █▎ │ 5141 │ █▎ │ -│ 2011-02-01 │ 6363114 │ ███▏ │ 277593 │ █▍ │ 5202 │ █▎ │ -│ 2011-03-01 │ 7556165 │ ███▊ │ 314748 │ █▌ │ 5445 │ █▎ │ -│ 2011-04-01 │ 7571398 │ ███▊ │ 329920 │ █▋ │ 6128 │ █▌ │ -│ 2011-05-01 │ 8803949 │ ████▍ │ 365013 │ █▊ │ 6834 │ █▋ │ -│ 2011-06-01 │ 9766511 │ ████▉ │ 393945 │ █▉ │ 7519 │ █▉ │ -│ 2011-07-01 │ 10557466 │ █████▎ │ 424235 │ ██ │ 8293 │ ██ │ -│ 2011-08-01 │ 12316144 │ ██████▏ │ 475326 │ ██▍ │ 9657 │ ██▍ │ -│ 2011-09-01 │ 12150412 │ ██████ │ 503142 │ ██▌ │ 10278 │ ██▌ │ -│ 2011-10-01 │ 13470278 │ ██████▋ │ 548801 │ ██▋ │ 10922 │ ██▋ │ -│ 2011-11-01 │ 13621533 │ ██████▊ │ 574435 │ ██▊ │ 11572 │ ██▉ │ -│ 2011-12-01 │ 14509469 │ ███████▎ │ 622849 │ ███ │ 12335 │ ███ │ -│ 2012-01-01 │ 16350205 │ ████████▏ │ 696110 │ ███▍ │ 14281 │ ███▌ │ -│ 2012-02-01 │ 16015695 │ ████████ │ 722892 │ ███▌ │ 14949 │ ███▋ │ -│ 2012-03-01 │ 17881943 │ ████████▉ │ 789664 │ ███▉ │ 15795 │ ███▉ │ -│ 2012-04-01 │ 19044534 │ █████████▌ │ 842491 │ ████▏ │ 16440 │ ████ │ -│ 2012-05-01 │ 20388260 │ ██████████▏ │ 886176 │ ████▍ │ 16974 │ ████▏ │ -│ 2012-06-01 │ 21897913 │ ██████████▉ │ 946798 │ ████▋ │ 17952 │ ████▍ │ -│ 2012-07-01 │ 24087517 │ ████████████ │ 1018636 │ █████ │ 19069 │ ████▊ │ -│ 2012-08-01 │ 25703326 │ ████████████▊ │ 1094445 │ █████▍ │ 20553 │ █████▏ │ -│ 2012-09-01 │ 23419524 │ ███████████▋ │ 1088491 │ █████▍ │ 20831 │ █████▏ │ -│ 2012-10-01 │ 24788236 │ ████████████▍ │ 1131885 │ █████▋ │ 21868 │ █████▍ │ -│ 2012-11-01 │ 24648302 │ ████████████▎ │ 1167608 │ █████▊ │ 21791 │ █████▍ │ -│ 2012-12-01 │ 26080276 │ █████████████ │ 1218402 │ ██████ │ 22622 │ █████▋ │ -│ 2013-01-01 │ 30365867 │ ███████████████▏ │ 1341703 │ ██████▋ │ 24696 │ ██████▏ │ -│ 2013-02-01 │ 27213960 │ █████████████▌ │ 1304756 │ ██████▌ │ 24514 │ ██████▏ │ -│ 2013-03-01 │ 30771274 │ ███████████████▍ │ 1391703 │ ██████▉ │ 25730 │ ██████▍ │ -│ 2013-04-01 │ 33259557 │ ████████████████▋ │ 1485971 │ ███████▍ │ 27294 │ ██████▊ │ -│ 2013-05-01 │ 33126225 │ ████████████████▌ │ 1506473 │ ███████▌ │ 27299 │ ██████▊ │ -│ 2013-06-01 │ 32648247 │ ████████████████▎ │ 1506650 │ ███████▌ │ 27450 │ ██████▊ │ -│ 2013-07-01 │ 34922133 │ █████████████████▍ │ 1561771 │ ███████▊ │ 28294 │ ███████ │ -│ 2013-08-01 │ 34766579 │ █████████████████▍ │ 1589781 │ ███████▉ │ 28943 │ ███████▏ │ -│ 2013-09-01 │ 31990369 │ ███████████████▉ │ 1570342 │ ███████▊ │ 29408 │ ███████▎ │ -│ 2013-10-01 │ 35940040 │ █████████████████▉ │ 1683770 │ ████████▍ │ 30273 │ ███████▌ │ -│ 2013-11-01 │ 37396497 │ ██████████████████▋ │ 1757467 │ ████████▊ │ 31173 │ ███████▊ │ -│ 2013-12-01 │ 39810216 │ ███████████████████▉ │ 1846204 │ █████████▏ │ 32326 │ ████████ │ -│ 2014-01-01 │ 42420655 │ █████████████████████▏ │ 1927229 │ █████████▋ │ 35603 │ ████████▉ │ -│ 2014-02-01 │ 38703362 │ ███████████████████▎ │ 1874067 │ █████████▎ │ 37007 │ █████████▎ │ -│ 2014-03-01 │ 42459956 │ █████████████████████▏ │ 1959888 │ █████████▊ │ 37948 │ █████████▍ │ -│ 2014-04-01 │ 42440735 │ █████████████████████▏ │ 1951369 │ █████████▊ │ 38362 │ █████████▌ │ -│ 2014-05-01 │ 42514094 │ █████████████████████▎ │ 1970197 │ █████████▊ │ 39078 │ █████████▊ │ -│ 2014-06-01 │ 41990650 │ ████████████████████▉ │ 1943850 │ █████████▋ │ 38268 │ █████████▌ │ -│ 2014-07-01 │ 46868899 │ ███████████████████████▍ │ 2059346 │ ██████████▎ │ 40634 │ ██████████▏ │ -│ 2014-08-01 │ 46990813 │ ███████████████████████▍ │ 2117335 │ ██████████▌ │ 41764 │ ██████████▍ │ -│ 2014-09-01 │ 44992201 │ ██████████████████████▍ │ 2124708 │ ██████████▌ │ 41890 │ ██████████▍ │ -│ 2014-10-01 │ 47497520 │ ███████████████████████▋ │ 2206535 │ ███████████ │ 43109 │ ██████████▊ │ -│ 2014-11-01 │ 46118074 │ ███████████████████████ │ 2239747 │ ███████████▏ │ 43718 │ ██████████▉ │ -│ 2014-12-01 │ 48807699 │ ████████████████████████▍ │ 2372945 │ ███████████▊ │ 43823 │ ██████████▉ │ -│ 2015-01-01 │ 53851542 │ █████████████████████████ │ 2499536 │ ████████████▍ │ 47172 │ ███████████▊ │ -│ 2015-02-01 │ 48342747 │ ████████████████████████▏ │ 2448496 │ ████████████▏ │ 47229 │ ███████████▊ │ -│ 2015-03-01 │ 54564441 │ █████████████████████████ │ 2550534 │ ████████████▊ │ 48156 │ ████████████ │ -│ 2015-04-01 │ 55005780 │ █████████████████████████ │ 2609443 │ █████████████ │ 49865 │ ████████████▍ │ -│ 2015-05-01 │ 54504410 │ █████████████████████████ │ 2585535 │ ████████████▉ │ 50137 │ ████████████▌ │ -│ 2015-06-01 │ 54258492 │ █████████████████████████ │ 2595129 │ ████████████▉ │ 49598 │ ████████████▍ │ -│ 2015-07-01 │ 58451788 │ █████████████████████████ │ 2720026 │ █████████████▌ │ 55022 │ █████████████▊ │ -│ 2015-08-01 │ 58075327 │ █████████████████████████ │ 2743994 │ █████████████▋ │ 55302 │ █████████████▊ │ -│ 2015-09-01 │ 55574825 │ █████████████████████████ │ 2672793 │ █████████████▎ │ 53960 │ █████████████▍ │ -│ 2015-10-01 │ 59494045 │ █████████████████████████ │ 2816426 │ ██████████████ │ 70210 │ █████████████████▌ │ -│ 2015-11-01 │ 57117500 │ █████████████████████████ │ 2847146 │ ██████████████▏ │ 71363 │ █████████████████▊ │ -│ 2015-12-01 │ 58523312 │ █████████████████████████ │ 2854840 │ ██████████████▎ │ 94559 │ ███████████████████████▋ │ -│ 2016-01-01 │ 61991732 │ █████████████████████████ │ 2920366 │ ██████████████▌ │ 108438 │ █████████████████████████ │ -│ 2016-02-01 │ 59189875 │ █████████████████████████ │ 2854683 │ ██████████████▎ │ 109916 │ █████████████████████████ │ -│ 2016-03-01 │ 63918864 │ █████████████████████████ │ 2969542 │ ██████████████▊ │ 84787 │ █████████████████████▏ │ -│ 2016-04-01 │ 64271256 │ █████████████████████████ │ 2999086 │ ██████████████▉ │ 61647 │ ███████████████▍ │ -│ 2016-05-01 │ 65212004 │ █████████████████████████ │ 3034674 │ ███████████████▏ │ 67465 │ ████████████████▊ │ -│ 2016-06-01 │ 65867743 │ █████████████████████████ │ 3057604 │ ███████████████▎ │ 75170 │ ██████████████████▊ │ -│ 2016-07-01 │ 66974735 │ █████████████████████████ │ 3199374 │ ███████████████▉ │ 77732 │ ███████████████████▍ │ -│ 2016-08-01 │ 69654819 │ █████████████████████████ │ 3239957 │ ████████████████▏ │ 63080 │ ███████████████▊ │ -│ 2016-09-01 │ 67024973 │ █████████████████████████ │ 3190864 │ ███████████████▉ │ 62324 │ ███████████████▌ │ -│ 2016-10-01 │ 71826553 │ █████████████████████████ │ 3284340 │ ████████████████▍ │ 62549 │ ███████████████▋ │ -│ 2016-11-01 │ 71022319 │ █████████████████████████ │ 3300822 │ ████████████████▌ │ 69718 │ █████████████████▍ │ -│ 2016-12-01 │ 72942967 │ █████████████████████████ │ 3430324 │ █████████████████▏ │ 71705 │ █████████████████▉ │ -│ 2017-01-01 │ 78946585 │ █████████████████████████ │ 3572093 │ █████████████████▊ │ 78198 │ ███████████████████▌ │ -│ 2017-02-01 │ 70609487 │ █████████████████████████ │ 3421115 │ █████████████████ │ 69823 │ █████████████████▍ │ -│ 2017-03-01 │ 79723106 │ █████████████████████████ │ 3638122 │ ██████████████████▏ │ 73865 │ ██████████████████▍ │ -│ 2017-04-01 │ 77478009 │ █████████████████████████ │ 3620591 │ ██████████████████ │ 74387 │ ██████████████████▌ │ -│ 2017-05-01 │ 79810360 │ █████████████████████████ │ 3650820 │ ██████████████████▎ │ 74356 │ ██████████████████▌ │ -│ 2017-06-01 │ 79901711 │ █████████████████████████ │ 3737614 │ ██████████████████▋ │ 72114 │ ██████████████████ │ -│ 2017-07-01 │ 81798725 │ █████████████████████████ │ 3872330 │ ███████████████████▎ │ 76052 │ ███████████████████ │ -│ 2017-08-01 │ 84658503 │ █████████████████████████ │ 3960093 │ ███████████████████▊ │ 77798 │ ███████████████████▍ │ -│ 2017-09-01 │ 83165192 │ █████████████████████████ │ 3880501 │ ███████████████████▍ │ 78402 │ ███████████████████▌ │ -│ 2017-10-01 │ 85828912 │ █████████████████████████ │ 3980335 │ ███████████████████▉ │ 80685 │ ████████████████████▏ │ -│ 2017-11-01 │ 84965681 │ █████████████████████████ │ 4026749 │ ████████████████████▏ │ 82659 │ ████████████████████▋ │ -│ 2017-12-01 │ 85973810 │ █████████████████████████ │ 4196354 │ ████████████████████▉ │ 91984 │ ██████████████████████▉ │ -│ 2018-01-01 │ 91558594 │ █████████████████████████ │ 4364443 │ █████████████████████▊ │ 102577 │ █████████████████████████ │ -│ 2018-02-01 │ 86467179 │ █████████████████████████ │ 4277899 │ █████████████████████▍ │ 104610 │ █████████████████████████ │ -│ 2018-03-01 │ 96490262 │ █████████████████████████ │ 4422470 │ ██████████████████████ │ 112559 │ █████████████████████████ │ -│ 2018-04-01 │ 98101232 │ █████████████████████████ │ 4572434 │ ██████████████████████▊ │ 105284 │ █████████████████████████ │ -│ 2018-05-01 │ 100109100 │ █████████████████████████ │ 4698908 │ ███████████████████████▍ │ 103910 │ █████████████████████████ │ -│ 2018-06-01 │ 100009462 │ █████████████████████████ │ 4697426 │ ███████████████████████▍ │ 101107 │ █████████████████████████ │ -│ 2018-07-01 │ 108151359 │ █████████████████████████ │ 5099492 │ █████████████████████████ │ 106184 │ █████████████████████████ │ -│ 2018-08-01 │ 107330940 │ █████████████████████████ │ 5084082 │ █████████████████████████ │ 109985 │ █████████████████████████ │ -│ 2018-09-01 │ 104473929 │ █████████████████████████ │ 5011953 │ █████████████████████████ │ 109710 │ █████████████████████████ │ -│ 2018-10-01 │ 112346556 │ █████████████████████████ │ 5320405 │ █████████████████████████ │ 112533 │ █████████████████████████ │ -│ 2018-11-01 │ 112573001 │ █████████████████████████ │ 5353282 │ █████████████████████████ │ 112211 │ █████████████████████████ │ -│ 2018-12-01 │ 121953600 │ █████████████████████████ │ 5611543 │ █████████████████████████ │ 118291 │ █████████████████████████ │ -│ 2019-01-01 │ 129386587 │ █████████████████████████ │ 6016687 │ █████████████████████████ │ 125725 │ █████████████████████████ │ -│ 2019-02-01 │ 120645639 │ █████████████████████████ │ 5974488 │ █████████████████████████ │ 125420 │ █████████████████████████ │ -│ 2019-03-01 │ 137650471 │ █████████████████████████ │ 6410197 │ █████████████████████████ │ 135924 │ █████████████████████████ │ -│ 2019-04-01 │ 138473643 │ █████████████████████████ │ 6416384 │ █████████████████████████ │ 139844 │ █████████████████████████ │ -│ 2019-05-01 │ 142463421 │ █████████████████████████ │ 6574836 │ █████████████████████████ │ 142012 │ █████████████████████████ │ -│ 2019-06-01 │ 134172939 │ █████████████████████████ │ 6601267 │ █████████████████████████ │ 140997 │ █████████████████████████ │ -│ 2019-07-01 │ 145965083 │ █████████████████████████ │ 6901822 │ █████████████████████████ │ 147802 │ █████████████████████████ │ -│ 2019-08-01 │ 146854393 │ █████████████████████████ │ 6993882 │ █████████████████████████ │ 151888 │ █████████████████████████ │ -│ 2019-09-01 │ 137540219 │ █████████████████████████ │ 7001362 │ █████████████████████████ │ 148839 │ █████████████████████████ │ -│ 2019-10-01 │ 129771456 │ █████████████████████████ │ 6825690 │ █████████████████████████ │ 144453 │ █████████████████████████ │ -│ 2019-11-01 │ 107990259 │ █████████████████████████ │ 6368286 │ █████████████████████████ │ 141768 │ █████████████████████████ │ -│ 2019-12-01 │ 112895934 │ █████████████████████████ │ 6640902 │ █████████████████████████ │ 148277 │ █████████████████████████ │ -│ 2020-01-01 │ 54354879 │ █████████████████████████ │ 4782339 │ ███████████████████████▉ │ 111658 │ █████████████████████████ │ -│ 2020-02-01 │ 22696923 │ ███████████▎ │ 3135175 │ ███████████████▋ │ 79521 │ ███████████████████▉ │ -│ 2020-03-01 │ 3466677 │ █▋ │ 987960 │ ████▉ │ 40901 │ ██████████▏ │ -└──────────────┴───────────┴───────────────────────────┴─────────┴───────────────────────────┴────────────┴───────────────────────────┘ +┌─firstOfMonth─┬─────────c─┬─bar_count─────────────────┬──authors─┬─bar_authors───────────────┬─subreddits─┬─bar_subreddits────────────┐ +│ 2005-12-01 │ 1075 │ │ 394 │ │ 1 │ │ +│ 2006-01-01 │ 3666 │ │ 791 │ │ 2 │ │ +│ 2006-02-01 │ 9095 │ │ 1464 │ │ 18 │ │ +│ 2006-03-01 │ 13859 │ │ 1958 │ │ 15 │ │ +│ 2006-04-01 │ 19090 │ │ 2334 │ │ 21 │ │ +│ 2006-05-01 │ 26859 │ │ 2698 │ │ 21 │ │ +│ 2006-06-01 │ 29163 │ │ 3043 │ │ 19 │ │ +│ 2006-07-01 │ 37031 │ │ 3532 │ │ 22 │ │ +│ 2006-08-01 │ 50559 │ │ 4750 │ │ 24 │ │ +│ 2006-09-01 │ 50675 │ │ 4908 │ │ 21 │ │ +│ 2006-10-01 │ 54148 │ │ 5654 │ │ 31 │ │ +│ 2006-11-01 │ 62021 │ │ 6490 │ │ 23 │ │ +│ 2006-12-01 │ 61018 │ │ 6707 │ │ 24 │ │ +│ 2007-01-01 │ 81341 │ │ 7931 │ │ 23 │ │ +│ 2007-02-01 │ 95634 │ │ 9020 │ │ 21 │ │ +│ 2007-03-01 │ 112444 │ │ 10842 │ │ 23 │ │ +│ 2007-04-01 │ 126773 │ │ 10701 │ │ 26 │ │ +│ 2007-05-01 │ 170097 │ │ 11365 │ │ 25 │ │ +│ 2007-06-01 │ 178800 │ │ 11267 │ │ 22 │ │ +│ 2007-07-01 │ 203319 │ │ 12482 │ │ 25 │ │ +│ 2007-08-01 │ 225111 │ │ 14124 │ │ 30 │ │ +│ 2007-09-01 │ 259497 │ ▏ │ 15416 │ │ 33 │ │ +│ 2007-10-01 │ 274170 │ ▏ │ 15302 │ │ 36 │ │ +│ 2007-11-01 │ 372983 │ ▏ │ 15134 │ │ 43 │ │ +│ 2007-12-01 │ 363390 │ ▏ │ 15915 │ │ 31 │ │ +│ 2008-01-01 │ 452990 │ ▏ │ 18857 │ │ 126 │ │ +│ 2008-02-01 │ 441768 │ ▏ │ 18266 │ │ 173 │ │ +│ 2008-03-01 │ 463728 │ ▏ │ 18947 │ │ 292 │ │ +│ 2008-04-01 │ 468317 │ ▏ │ 18590 │ │ 323 │ │ +│ 2008-05-01 │ 536380 │ ▎ │ 20861 │ │ 375 │ │ +│ 2008-06-01 │ 577684 │ ▎ │ 22557 │ │ 575 │ ▏ │ +│ 2008-07-01 │ 592610 │ ▎ │ 23123 │ │ 657 │ ▏ │ +│ 2008-08-01 │ 595959 │ ▎ │ 23729 │ │ 707 │ ▏ │ +│ 2008-09-01 │ 680892 │ ▎ │ 26374 │ ▏ │ 801 │ ▏ │ +│ 2008-10-01 │ 789874 │ ▍ │ 28970 │ ▏ │ 893 │ ▏ │ +│ 2008-11-01 │ 792310 │ ▍ │ 30272 │ ▏ │ 1024 │ ▎ │ +│ 2008-12-01 │ 850359 │ ▍ │ 34073 │ ▏ │ 1103 │ ▎ │ +│ 2009-01-01 │ 1051649 │ ▌ │ 38978 │ ▏ │ 1316 │ ▎ │ +│ 2009-02-01 │ 944711 │ ▍ │ 43390 │ ▏ │ 1132 │ ▎ │ +│ 2009-03-01 │ 1048643 │ ▌ │ 46516 │ ▏ │ 1203 │ ▎ │ +│ 2009-04-01 │ 1094599 │ ▌ │ 48284 │ ▏ │ 1334 │ ▎ │ +│ 2009-05-01 │ 1201257 │ ▌ │ 52512 │ ▎ │ 1395 │ ▎ │ +│ 2009-06-01 │ 1258750 │ ▋ │ 57728 │ ▎ │ 1473 │ ▎ │ +│ 2009-07-01 │ 1470290 │ ▋ │ 60098 │ ▎ │ 1686 │ ▍ │ +│ 2009-08-01 │ 1750688 │ ▉ │ 67347 │ ▎ │ 1777 │ ▍ │ +│ 2009-09-01 │ 2032276 │ █ │ 78051 │ ▍ │ 1784 │ ▍ │ +│ 2009-10-01 │ 2242017 │ █ │ 93409 │ ▍ │ 2071 │ ▌ │ +│ 2009-11-01 │ 2207444 │ █ │ 95940 │ ▍ │ 2141 │ ▌ │ +│ 2009-12-01 │ 2560510 │ █▎ │ 104239 │ ▌ │ 2141 │ ▌ │ +│ 2010-01-01 │ 2884096 │ █▍ │ 114314 │ ▌ │ 2313 │ ▌ │ +│ 2010-02-01 │ 2687779 │ █▎ │ 115683 │ ▌ │ 2522 │ ▋ │ +│ 2010-03-01 │ 3228254 │ █▌ │ 125775 │ ▋ │ 2890 │ ▋ │ +│ 2010-04-01 │ 3209898 │ █▌ │ 128936 │ ▋ │ 3170 │ ▊ │ +│ 2010-05-01 │ 3267363 │ █▋ │ 131851 │ ▋ │ 3166 │ ▊ │ +│ 2010-06-01 │ 3532867 │ █▊ │ 139522 │ ▋ │ 3301 │ ▊ │ +│ 2010-07-01 │ 806612 │ ▍ │ 76486 │ ▍ │ 1955 │ ▍ │ +│ 2010-08-01 │ 4247982 │ ██ │ 164071 │ ▊ │ 3653 │ ▉ │ +│ 2010-09-01 │ 4704069 │ ██▎ │ 186613 │ ▉ │ 4009 │ █ │ +│ 2010-10-01 │ 5032368 │ ██▌ │ 203800 │ █ │ 4154 │ █ │ +│ 2010-11-01 │ 5689002 │ ██▊ │ 226134 │ █▏ │ 4383 │ █ │ +│ 2010-12-01 │ 3642690 │ █▊ │ 196847 │ ▉ │ 3914 │ ▉ │ +│ 2011-01-01 │ 3924540 │ █▉ │ 215057 │ █ │ 4240 │ █ │ +│ 2011-02-01 │ 3859131 │ █▉ │ 223485 │ █ │ 4371 │ █ │ +│ 2011-03-01 │ 2877996 │ █▍ │ 208607 │ █ │ 3870 │ ▉ │ +│ 2011-04-01 │ 3859131 │ █▉ │ 248931 │ █▏ │ 4881 │ █▏ │ +│ 2011-06-01 │ 3859131 │ █▉ │ 267197 │ █▎ │ 5255 │ █▎ │ +│ 2011-08-01 │ 2943405 │ █▍ │ 259428 │ █▎ │ 5806 │ █▍ │ +│ 2011-10-01 │ 3859131 │ █▉ │ 327342 │ █▋ │ 6958 │ █▋ │ +│ 2011-12-01 │ 3728313 │ █▊ │ 354817 │ █▊ │ 7713 │ █▉ │ +│ 2012-01-01 │ 16350205 │ ████████▏ │ 696110 │ ███▍ │ 14281 │ ███▌ │ +│ 2012-02-01 │ 16015695 │ ████████ │ 722892 │ ███▌ │ 14949 │ ███▋ │ +│ 2012-03-01 │ 17881943 │ ████████▉ │ 789664 │ ███▉ │ 15795 │ ███▉ │ +│ 2012-04-01 │ 19044534 │ █████████▌ │ 842491 │ ████▏ │ 16440 │ ████ │ +│ 2012-05-01 │ 20388260 │ ██████████▏ │ 886176 │ ████▍ │ 16974 │ ████▏ │ +│ 2012-06-01 │ 21897913 │ ██████████▉ │ 946798 │ ████▋ │ 17952 │ ████▍ │ +│ 2012-07-01 │ 24087517 │ ████████████ │ 1018636 │ █████ │ 19069 │ ████▊ │ +│ 2012-08-01 │ 25703326 │ ████████████▊ │ 1094445 │ █████▍ │ 20553 │ █████▏ │ +│ 2012-09-01 │ 23419524 │ ███████████▋ │ 1088491 │ █████▍ │ 20831 │ █████▏ │ +│ 2012-10-01 │ 24788236 │ ████████████▍ │ 1131885 │ █████▋ │ 21868 │ █████▍ │ +│ 2012-11-01 │ 24648302 │ ████████████▎ │ 1167608 │ █████▊ │ 21791 │ █████▍ │ +│ 2012-12-01 │ 26080276 │ █████████████ │ 1218402 │ ██████ │ 22622 │ █████▋ │ +│ 2013-01-01 │ 30365867 │ ███████████████▏ │ 1341703 │ ██████▋ │ 24696 │ ██████▏ │ +│ 2013-02-01 │ 27213960 │ █████████████▌ │ 1304756 │ ██████▌ │ 24514 │ ██████▏ │ +│ 2013-03-01 │ 30771274 │ ███████████████▍ │ 1391703 │ ██████▉ │ 25730 │ ██████▍ │ +│ 2013-04-01 │ 33259557 │ ████████████████▋ │ 1485971 │ ███████▍ │ 27294 │ ██████▊ │ +│ 2013-05-01 │ 33126225 │ ████████████████▌ │ 1506473 │ ███████▌ │ 27299 │ ██████▊ │ +│ 2013-06-01 │ 32648247 │ ████████████████▎ │ 1506650 │ ███████▌ │ 27450 │ ██████▊ │ +│ 2013-07-01 │ 34922133 │ █████████████████▍ │ 1561771 │ ███████▊ │ 28294 │ ███████ │ +│ 2013-08-01 │ 34766579 │ █████████████████▍ │ 1589781 │ ███████▉ │ 28943 │ ███████▏ │ +│ 2013-09-01 │ 31990369 │ ███████████████▉ │ 1570342 │ ███████▊ │ 29408 │ ███████▎ │ +│ 2013-10-01 │ 35940040 │ █████████████████▉ │ 1683770 │ ████████▍ │ 30273 │ ███████▌ │ +│ 2013-11-01 │ 37396497 │ ██████████████████▋ │ 1757467 │ ████████▊ │ 31173 │ ███████▊ │ +│ 2013-12-01 │ 39810216 │ ███████████████████▉ │ 1846204 │ █████████▏ │ 32326 │ ████████ │ +│ 2014-01-01 │ 42420655 │ █████████████████████▏ │ 1927229 │ █████████▋ │ 35603 │ ████████▉ │ +│ 2014-02-01 │ 38703362 │ ███████████████████▎ │ 1874067 │ █████████▎ │ 37007 │ █████████▎ │ +│ 2014-03-01 │ 42459956 │ █████████████████████▏ │ 1959888 │ █████████▊ │ 37948 │ █████████▍ │ +│ 2014-04-01 │ 42440735 │ █████████████████████▏ │ 1951369 │ █████████▊ │ 38362 │ █████████▌ │ +│ 2014-05-01 │ 42514094 │ █████████████████████▎ │ 1970197 │ █████████▊ │ 39078 │ █████████▊ │ +│ 2014-06-01 │ 41990650 │ ████████████████████▉ │ 1943850 │ █████████▋ │ 38268 │ █████████▌ │ +│ 2014-07-01 │ 46868899 │ ███████████████████████▍ │ 2059346 │ ██████████▎ │ 40634 │ ██████████▏ │ +│ 2014-08-01 │ 46990813 │ ███████████████████████▍ │ 2117335 │ ██████████▌ │ 41764 │ ██████████▍ │ +│ 2014-09-01 │ 44992201 │ ██████████████████████▍ │ 2124708 │ ██████████▌ │ 41890 │ ██████████▍ │ +│ 2014-10-01 │ 47497520 │ ███████████████████████▋ │ 2206535 │ ███████████ │ 43109 │ ██████████▊ │ +│ 2014-11-01 │ 46118074 │ ███████████████████████ │ 2239747 │ ███████████▏ │ 43718 │ ██████████▉ │ +│ 2014-12-01 │ 48807699 │ ████████████████████████▍ │ 2372945 │ ███████████▊ │ 43823 │ ██████████▉ │ +│ 2015-01-01 │ 53851542 │ █████████████████████████ │ 2499536 │ ████████████▍ │ 47172 │ ███████████▊ │ +│ 2015-02-01 │ 48342747 │ ████████████████████████▏ │ 2448496 │ ████████████▏ │ 47229 │ ███████████▊ │ +│ 2015-03-01 │ 54564441 │ █████████████████████████ │ 2550534 │ ████████████▊ │ 48156 │ ████████████ │ +│ 2015-04-01 │ 55005780 │ █████████████████████████ │ 2609443 │ █████████████ │ 49865 │ ████████████▍ │ +│ 2015-05-01 │ 54504410 │ █████████████████████████ │ 2585535 │ ████████████▉ │ 50137 │ ████████████▌ │ +│ 2015-06-01 │ 54258492 │ █████████████████████████ │ 2595129 │ ████████████▉ │ 49598 │ ████████████▍ │ +│ 2015-07-01 │ 58451788 │ █████████████████████████ │ 2720026 │ █████████████▌ │ 55022 │ █████████████▊ │ +│ 2015-08-01 │ 58075327 │ █████████████████████████ │ 2743994 │ █████████████▋ │ 55302 │ █████████████▊ │ +│ 2015-09-01 │ 55574825 │ █████████████████████████ │ 2672793 │ █████████████▎ │ 53960 │ █████████████▍ │ +│ 2015-10-01 │ 59494045 │ █████████████████████████ │ 2816426 │ ██████████████ │ 70210 │ █████████████████▌ │ +│ 2015-11-01 │ 57117500 │ █████████████████████████ │ 2847146 │ ██████████████▏ │ 71363 │ █████████████████▊ │ +│ 2015-12-01 │ 58523312 │ █████████████████████████ │ 2854840 │ ██████████████▎ │ 94559 │ ███████████████████████▋ │ +│ 2016-01-01 │ 61991732 │ █████████████████████████ │ 2920366 │ ██████████████▌ │ 108438 │ █████████████████████████ │ +│ 2016-02-01 │ 59189875 │ █████████████████████████ │ 2854683 │ ██████████████▎ │ 109916 │ █████████████████████████ │ +│ 2016-03-01 │ 63918864 │ █████████████████████████ │ 2969542 │ ██████████████▊ │ 84787 │ █████████████████████▏ │ +│ 2016-04-01 │ 64271256 │ █████████████████████████ │ 2999086 │ ██████████████▉ │ 61647 │ ███████████████▍ │ +│ 2016-05-01 │ 65212004 │ █████████████████████████ │ 3034674 │ ███████████████▏ │ 67465 │ ████████████████▊ │ +│ 2016-06-01 │ 65867743 │ █████████████████████████ │ 3057604 │ ███████████████▎ │ 75170 │ ██████████████████▊ │ +│ 2016-07-01 │ 66974735 │ █████████████████████████ │ 3199374 │ ███████████████▉ │ 77732 │ ███████████████████▍ │ +│ 2016-08-01 │ 69654819 │ █████████████████████████ │ 3239957 │ ████████████████▏ │ 63080 │ ███████████████▊ │ +│ 2016-09-01 │ 67024973 │ █████████████████████████ │ 3190864 │ ███████████████▉ │ 62324 │ ███████████████▌ │ +│ 2016-10-01 │ 71826553 │ █████████████████████████ │ 3284340 │ ████████████████▍ │ 62549 │ ███████████████▋ │ +│ 2016-11-01 │ 71022319 │ █████████████████████████ │ 3300822 │ ████████████████▌ │ 69718 │ █████████████████▍ │ +│ 2016-12-01 │ 72942967 │ █████████████████████████ │ 3430324 │ █████████████████▏ │ 71705 │ █████████████████▉ │ +│ 2017-01-01 │ 78946585 │ █████████████████████████ │ 3572093 │ █████████████████▊ │ 78198 │ ███████████████████▌ │ +│ 2017-02-01 │ 70609487 │ █████████████████████████ │ 3421115 │ █████████████████ │ 69823 │ █████████████████▍ │ +│ 2017-03-01 │ 79723106 │ █████████████████████████ │ 3638122 │ ██████████████████▏ │ 73865 │ ██████████████████▍ │ +│ 2017-04-01 │ 77478009 │ █████████████████████████ │ 3620591 │ ██████████████████ │ 74387 │ ██████████████████▌ │ +│ 2017-05-01 │ 79810360 │ █████████████████████████ │ 3650820 │ ██████████████████▎ │ 74356 │ ██████████████████▌ │ +│ 2017-06-01 │ 79901711 │ █████████████████████████ │ 3737614 │ ██████████████████▋ │ 72114 │ ██████████████████ │ +│ 2017-07-01 │ 81798725 │ █████████████████████████ │ 3872330 │ ███████████████████▎ │ 76052 │ ███████████████████ │ +│ 2017-08-01 │ 84658503 │ █████████████████████████ │ 3960093 │ ███████████████████▊ │ 77798 │ ███████████████████▍ │ +│ 2017-09-01 │ 83165192 │ █████████████████████████ │ 3880501 │ ███████████████████▍ │ 78402 │ ███████████████████▌ │ +│ 2017-10-01 │ 85828912 │ █████████████████████████ │ 3980335 │ ███████████████████▉ │ 80685 │ ████████████████████▏ │ +│ 2017-11-01 │ 84965681 │ █████████████████████████ │ 4026749 │ ████████████████████▏ │ 82659 │ ████████████████████▋ │ +│ 2017-12-01 │ 85973810 │ █████████████████████████ │ 4196354 │ ████████████████████▉ │ 91984 │ ██████████████████████▉ │ +│ 2018-01-01 │ 91558594 │ █████████████████████████ │ 4364443 │ █████████████████████▊ │ 102577 │ █████████████████████████ │ +│ 2018-02-01 │ 86467179 │ █████████████████████████ │ 4277899 │ █████████████████████▍ │ 104610 │ █████████████████████████ │ +│ 2018-03-01 │ 96490262 │ █████████████████████████ │ 4422470 │ ██████████████████████ │ 112559 │ █████████████████████████ │ +│ 2018-04-01 │ 98101232 │ █████████████████████████ │ 4572434 │ ██████████████████████▊ │ 105284 │ █████████████████████████ │ +│ 2018-05-01 │ 100109100 │ █████████████████████████ │ 4698908 │ ███████████████████████▍ │ 103910 │ █████████████████████████ │ +│ 2018-06-01 │ 100009462 │ █████████████████████████ │ 4697426 │ ███████████████████████▍ │ 101107 │ █████████████████████████ │ +│ 2018-07-01 │ 108151359 │ █████████████████████████ │ 5099492 │ █████████████████████████ │ 106184 │ █████████████████████████ │ +│ 2018-08-01 │ 107330940 │ █████████████████████████ │ 5084082 │ █████████████████████████ │ 109985 │ █████████████████████████ │ +│ 2018-09-01 │ 104473929 │ █████████████████████████ │ 5011953 │ █████████████████████████ │ 109710 │ █████████████████████████ │ +│ 2018-10-01 │ 112346556 │ █████████████████████████ │ 5320405 │ █████████████████████████ │ 112533 │ █████████████████████████ │ +│ 2018-11-01 │ 112573001 │ █████████████████████████ │ 5353282 │ █████████████████████████ │ 112211 │ █████████████████████████ │ +│ 2018-12-01 │ 121953600 │ █████████████████████████ │ 5611543 │ █████████████████████████ │ 118291 │ █████████████████████████ │ +│ 2019-01-01 │ 129386587 │ █████████████████████████ │ 6016687 │ █████████████████████████ │ 125725 │ █████████████████████████ │ +│ 2019-02-01 │ 120645639 │ █████████████████████████ │ 5974488 │ █████████████████████████ │ 125420 │ █████████████████████████ │ +│ 2019-03-01 │ 137650471 │ █████████████████████████ │ 6410197 │ █████████████████████████ │ 135924 │ █████████████████████████ │ +│ 2019-04-01 │ 138473643 │ █████████████████████████ │ 6416384 │ █████████████████████████ │ 139844 │ █████████████████████████ │ +│ 2019-05-01 │ 142463421 │ █████████████████████████ │ 6574836 │ █████████████████████████ │ 142012 │ █████████████████████████ │ +│ 2019-06-01 │ 134172939 │ █████████████████████████ │ 6601267 │ █████████████████████████ │ 140997 │ █████████████████████████ │ +│ 2019-07-01 │ 145965083 │ █████████████████████████ │ 6901822 │ █████████████████████████ │ 147802 │ █████████████████████████ │ +│ 2019-08-01 │ 146854393 │ █████████████████████████ │ 6993882 │ █████████████████████████ │ 151888 │ █████████████████████████ │ +│ 2019-09-01 │ 137540219 │ █████████████████████████ │ 7001362 │ █████████████████████████ │ 148839 │ █████████████████████████ │ +│ 2019-10-01 │ 145909884 │ █████████████████████████ │ 7160126 │ █████████████████████████ │ 152075 │ █████████████████████████ │ +│ 2019-11-01 │ 138512489 │ █████████████████████████ │ 7098723 │ █████████████████████████ │ 164597 │ █████████████████████████ │ +│ 2019-12-01 │ 146012313 │ █████████████████████████ │ 7438261 │ █████████████████████████ │ 166966 │ █████████████████████████ │ +│ 2020-01-01 │ 153498208 │ █████████████████████████ │ 7703548 │ █████████████████████████ │ 174390 │ █████████████████████████ │ +│ 2020-02-01 │ 148386817 │ █████████████████████████ │ 7582031 │ █████████████████████████ │ 170257 │ █████████████████████████ │ +│ 2020-03-01 │ 166266315 │ █████████████████████████ │ 8339049 │ █████████████████████████ │ 192460 │ █████████████████████████ │ +│ 2020-04-01 │ 178511581 │ █████████████████████████ │ 8991649 │ █████████████████████████ │ 202334 │ █████████████████████████ │ +│ 2020-05-01 │ 189993779 │ █████████████████████████ │ 9331358 │ █████████████████████████ │ 217357 │ █████████████████████████ │ +│ 2020-06-01 │ 187914434 │ █████████████████████████ │ 9085003 │ █████████████████████████ │ 223362 │ █████████████████████████ │ +│ 2020-07-01 │ 194244994 │ █████████████████████████ │ 9321706 │ █████████████████████████ │ 228222 │ █████████████████████████ │ +│ 2020-08-01 │ 196099301 │ █████████████████████████ │ 9368408 │ █████████████████████████ │ 230251 │ █████████████████████████ │ +│ 2020-09-01 │ 182549761 │ █████████████████████████ │ 9271571 │ █████████████████████████ │ 227889 │ █████████████████████████ │ +│ 2020-10-01 │ 186583890 │ █████████████████████████ │ 9396112 │ █████████████████████████ │ 233715 │ █████████████████████████ │ +│ 2020-11-01 │ 186083723 │ █████████████████████████ │ 9623053 │ █████████████████████████ │ 234963 │ █████████████████████████ │ +│ 2020-12-01 │ 191317162 │ █████████████████████████ │ 9898168 │ █████████████████████████ │ 249115 │ █████████████████████████ │ +│ 2021-01-01 │ 210496207 │ █████████████████████████ │ 10503943 │ █████████████████████████ │ 259805 │ █████████████████████████ │ +│ 2021-02-01 │ 193510365 │ █████████████████████████ │ 10215033 │ █████████████████████████ │ 253656 │ █████████████████████████ │ +│ 2021-03-01 │ 207454415 │ █████████████████████████ │ 10365629 │ █████████████████████████ │ 267263 │ █████████████████████████ │ +│ 2021-04-01 │ 204573086 │ █████████████████████████ │ 10391984 │ █████████████████████████ │ 270543 │ █████████████████████████ │ +│ 2021-05-01 │ 217655366 │ █████████████████████████ │ 10648130 │ █████████████████████████ │ 288555 │ █████████████████████████ │ +│ 2021-06-01 │ 208027069 │ █████████████████████████ │ 10397311 │ █████████████████████████ │ 291520 │ █████████████████████████ │ +│ 2021-07-01 │ 210955954 │ █████████████████████████ │ 10063967 │ █████████████████████████ │ 252061 │ █████████████████████████ │ +│ 2021-08-01 │ 225681244 │ █████████████████████████ │ 10383556 │ █████████████████████████ │ 254569 │ █████████████████████████ │ +│ 2021-09-01 │ 220086513 │ █████████████████████████ │ 10298344 │ █████████████████████████ │ 256826 │ █████████████████████████ │ +│ 2021-10-01 │ 227527379 │ █████████████████████████ │ 10729882 │ █████████████████████████ │ 283328 │ █████████████████████████ │ +│ 2021-11-01 │ 228289963 │ █████████████████████████ │ 10995197 │ █████████████████████████ │ 302386 │ █████████████████████████ │ +│ 2021-12-01 │ 235807471 │ █████████████████████████ │ 11312798 │ █████████████████████████ │ 313876 │ █████████████████████████ │ +│ 2022-01-01 │ 256766679 │ █████████████████████████ │ 12074520 │ █████████████████████████ │ 340407 │ █████████████████████████ │ +│ 2022-02-01 │ 219927645 │ █████████████████████████ │ 10846045 │ █████████████████████████ │ 293236 │ █████████████████████████ │ +│ 2022-03-01 │ 236554668 │ █████████████████████████ │ 11330285 │ █████████████████████████ │ 302387 │ █████████████████████████ │ +│ 2022-04-01 │ 231188077 │ █████████████████████████ │ 11697995 │ █████████████████████████ │ 316303 │ █████████████████████████ │ +│ 2022-05-01 │ 230492108 │ █████████████████████████ │ 11448584 │ █████████████████████████ │ 323725 │ █████████████████████████ │ +│ 2022-06-01 │ 218842949 │ █████████████████████████ │ 11400399 │ █████████████████████████ │ 324846 │ █████████████████████████ │ +│ 2022-07-01 │ 242504279 │ █████████████████████████ │ 12049204 │ █████████████████████████ │ 335621 │ █████████████████████████ │ +│ 2022-08-01 │ 247215325 │ █████████████████████████ │ 12189276 │ █████████████████████████ │ 337873 │ █████████████████████████ │ +│ 2022-09-01 │ 234131223 │ █████████████████████████ │ 11674079 │ █████████████████████████ │ 326325 │ █████████████████████████ │ +│ 2022-10-01 │ 237365072 │ █████████████████████████ │ 11804508 │ █████████████████████████ │ 336063 │ █████████████████████████ │ +│ 2022-11-01 │ 229478878 │ █████████████████████████ │ 11543020 │ █████████████████████████ │ 323122 │ █████████████████████████ │ +│ 2022-12-01 │ 238862690 │ █████████████████████████ │ 11967451 │ █████████████████████████ │ 331668 │ █████████████████████████ │ +│ 2023-01-01 │ 253577512 │ █████████████████████████ │ 12264087 │ █████████████████████████ │ 332711 │ █████████████████████████ │ +│ 2023-02-01 │ 221285501 │ █████████████████████████ │ 11537091 │ █████████████████████████ │ 317879 │ █████████████████████████ │ +└──────────────┴───────────┴───────────────────────────┴──────────┴───────────────────────────┴────────────┴───────────────────────────┘ -172 rows in set. Elapsed: 184.809 sec. Processed 6.74 billion rows, 89.56 GB (36.47 million rows/s., 484.62 MB/s.) +203 rows in set. Elapsed: 48.492 sec. Processed 14.69 billion rows, 213.35 GB (302.91 million rows/s., 4.40 GB/s.) ``` 10. Here are the top 10 subreddits of 2022: @@ -451,23 +525,21 @@ ORDER BY count DESC LIMIT 10; ``` -The response is: - ```response -┌─subreddit────────┬───count─┐ -│ AskReddit │ 3858203 │ -│ politics │ 1356782 │ -│ memes │ 1249120 │ -│ nfl │ 883667 │ -│ worldnews │ 866065 │ -│ teenagers │ 777095 │ -│ AmItheAsshole │ 752720 │ -│ dankmemes │ 657932 │ -│ nba │ 514184 │ -│ unpopularopinion │ 473649 │ -└──────────────────┴─────────┘ +┌─subreddit──────┬────count─┐ +│ AskReddit │ 72312060 │ +│ AmItheAsshole │ 25323210 │ +│ teenagers │ 22355960 │ +│ worldnews │ 17797707 │ +│ FreeKarma4U │ 15652274 │ +│ FreeKarma4You │ 14929055 │ +│ wallstreetbets │ 14235271 │ +│ politics │ 12511136 │ +│ memes │ 11610792 │ +│ nba │ 11586571 │ +└────────────────┴──────────┘ -10 rows in set. Elapsed: 27.824 sec. Processed 6.74 billion rows, 53.26 GB (242.22 million rows/s., 1.91 GB/s.) +10 rows in set. Elapsed: 5.956 sec. Processed 14.69 billion rows, 126.19 GB (2.47 billion rows/s., 21.19 GB/s.) ``` 11. Let's see which subreddits had the biggest increase in commnents from 2018 to 2019: @@ -503,62 +575,62 @@ It looks like memes and teenagers were busy on Reddit in 2019: ```response ┌─subreddit────────────┬─────diff─┐ -│ memes │ 15368369 │ -│ AskReddit │ 14663662 │ -│ teenagers │ 12266991 │ -│ AmItheAsshole │ 11561538 │ -│ dankmemes │ 11305158 │ -│ unpopularopinion │ 6332772 │ -│ PewdiepieSubmissions │ 5930818 │ -│ Market76 │ 5014668 │ -│ relationship_advice │ 3776383 │ -│ freefolk │ 3169236 │ -│ Minecraft │ 3160241 │ -│ classicwow │ 2907056 │ -│ Animemes │ 2673398 │ -│ gameofthrones │ 2402835 │ -│ PublicFreakout │ 2267605 │ -│ ShitPostCrusaders │ 2207266 │ -│ RoastMe │ 2195715 │ -│ gonewild │ 2148649 │ -│ AnthemTheGame │ 1803818 │ -│ entitledparents │ 1706270 │ -│ MortalKombat │ 1679508 │ -│ Cringetopia │ 1620555 │ -│ pokemon │ 1615266 │ -│ HistoryMemes │ 1608289 │ -│ Brawlstars │ 1574977 │ -│ iamatotalpieceofshit │ 1558315 │ -│ trashy │ 1518549 │ -│ ChapoTrapHouse │ 1505748 │ -│ Pikabu │ 1501001 │ -│ Showerthoughts │ 1475101 │ -│ cursedcomments │ 1465607 │ -│ ukpolitics │ 1386043 │ -│ wallstreetbets │ 1384431 │ -│ interestingasfuck │ 1378900 │ -│ wholesomememes │ 1353333 │ -│ AskOuija │ 1233263 │ -│ borderlands3 │ 1197192 │ -│ aww │ 1168257 │ -│ insanepeoplefacebook │ 1155473 │ -│ FortniteCompetitive │ 1122778 │ -│ EpicSeven │ 1117380 │ -│ FreeKarma4U │ 1116423 │ -│ YangForPresidentHQ │ 1086700 │ -│ SquaredCircle │ 1044089 │ -│ MurderedByWords │ 1042511 │ -│ AskMen │ 1024434 │ -│ thedivision │ 1016634 │ -│ barstoolsports │ 985032 │ -│ nfl │ 978340 │ -│ BattlefieldV │ 971408 │ +│ AskReddit │ 18765909 │ +│ memes │ 16496996 │ +│ teenagers │ 13071715 │ +│ AmItheAsshole │ 12312663 │ +│ dankmemes │ 12016716 │ +│ unpopularopinion │ 6809935 │ +│ PewdiepieSubmissions │ 6330844 │ +│ Market76 │ 5213690 │ +│ relationship_advice │ 4060717 │ +│ Minecraft │ 3328659 │ +│ freefolk │ 3227970 │ +│ classicwow │ 3063133 │ +│ Animemes │ 2866876 │ +│ gonewild │ 2457680 │ +│ PublicFreakout │ 2452288 │ +│ gameofthrones │ 2411661 │ +│ RoastMe │ 2378781 │ +│ ShitPostCrusaders │ 2345414 │ +│ AnthemTheGame │ 1813152 │ +│ nfl │ 1804407 │ +│ Showerthoughts │ 1797968 │ +│ Cringetopia │ 1764034 │ +│ pokemon │ 1763269 │ +│ entitledparents │ 1744852 │ +│ HistoryMemes │ 1721645 │ +│ MortalKombat │ 1718184 │ +│ trashy │ 1684357 │ +│ ChapoTrapHouse │ 1675363 │ +│ Brawlstars │ 1663763 │ +│ iamatotalpieceofshit │ 1647381 │ +│ ukpolitics │ 1599204 │ +│ cursedcomments │ 1590781 │ +│ Pikabu │ 1578597 │ +│ wallstreetbets │ 1535225 │ +│ AskOuija │ 1533214 │ +│ interestingasfuck │ 1528910 │ +│ aww │ 1439008 │ +│ wholesomememes │ 1436566 │ +│ SquaredCircle │ 1432172 │ +│ insanepeoplefacebook │ 1290686 │ +│ borderlands3 │ 1274462 │ +│ FreeKarma4U │ 1217769 │ +│ YangForPresidentHQ │ 1186918 │ +│ FortniteCompetitive │ 1184508 │ +│ AskMen │ 1180820 │ +│ EpicSeven │ 1172061 │ +│ MurderedByWords │ 1112476 │ +│ politics │ 1084087 │ +│ barstoolsports │ 1068020 │ +│ BattlefieldV │ 1053878 │ └──────────────────────┴──────────┘ -50 rows in set. Elapsed: 65.954 sec. Processed 13.48 billion rows, 79.67 GB (204.37 million rows/s., 1.21 GB/s.) +50 rows in set. Elapsed: 10.680 sec. Processed 29.38 billion rows, 198.67 GB (2.75 billion rows/s., 18.60 GB/s.) ``` -12. One more query: let's compare ClickHouse mentions to other technologies like Snowflake and Postgres. This query is a big one because it has to search all the comments three times for a substring, and unfortunately ClickHouse user are obviously not very active on Reddit yet: +12. One more query: let's compare ClickHouse mentions to other technologies like Snowflake and Postgres. This query is a big one because it has to search all 14.69 billion comments three times for a substring, but the performance is actually quite impressive. (Unfortunately ClickHouse users are not very active on Reddit yet): ```sql SELECT @@ -572,7 +644,7 @@ ORDER BY quarter ASC; ``` ```response -┌────Quarter─┬─clickhouse─┬─snowflake─┬─postgres─┐ +┌────quarter─┬─clickhouse─┬─snowflake─┬─postgres─┐ │ 2005-10-01 │ 0 │ 0 │ 0 │ │ 2006-01-01 │ 0 │ 2 │ 23 │ │ 2006-04-01 │ 0 │ 2 │ 24 │ @@ -592,12 +664,12 @@ ORDER BY quarter ASC; │ 2009-10-01 │ 0 │ 633 │ 589 │ │ 2010-01-01 │ 0 │ 555 │ 501 │ │ 2010-04-01 │ 0 │ 587 │ 469 │ -│ 2010-07-01 │ 0 │ 770 │ 821 │ -│ 2010-10-01 │ 0 │ 1480 │ 550 │ -│ 2011-01-01 │ 0 │ 1482 │ 568 │ -│ 2011-04-01 │ 0 │ 1558 │ 406 │ -│ 2011-07-01 │ 0 │ 2163 │ 628 │ -│ 2011-10-01 │ 0 │ 4064 │ 566 │ +│ 2010-07-01 │ 0 │ 601 │ 696 │ +│ 2010-10-01 │ 0 │ 1246 │ 505 │ +│ 2011-01-01 │ 0 │ 758 │ 247 │ +│ 2011-04-01 │ 0 │ 537 │ 113 │ +│ 2011-07-01 │ 0 │ 173 │ 64 │ +│ 2011-10-01 │ 0 │ 649 │ 96 │ │ 2012-01-01 │ 0 │ 4621 │ 662 │ │ 2012-04-01 │ 0 │ 5737 │ 785 │ │ 2012-07-01 │ 0 │ 6097 │ 1127 │ @@ -629,9 +701,21 @@ ORDER BY quarter ASC; │ 2019-01-01 │ 14 │ 80250 │ 4305 │ │ 2019-04-01 │ 30 │ 70307 │ 3872 │ │ 2019-07-01 │ 33 │ 77149 │ 4164 │ -│ 2019-10-01 │ 13 │ 76746 │ 3541 │ -│ 2020-01-01 │ 16 │ 54475 │ 846 │ +│ 2019-10-01 │ 22 │ 113011 │ 4369 │ +│ 2020-01-01 │ 34 │ 238273 │ 5133 │ +│ 2020-04-01 │ 52 │ 454467 │ 6100 │ +│ 2020-07-01 │ 37 │ 406623 │ 5507 │ +│ 2020-10-01 │ 49 │ 212143 │ 5385 │ +│ 2021-01-01 │ 56 │ 151262 │ 5749 │ +│ 2021-04-01 │ 71 │ 119928 │ 6039 │ +│ 2021-07-01 │ 53 │ 110342 │ 5765 │ +│ 2021-10-01 │ 92 │ 121144 │ 6401 │ +│ 2022-01-01 │ 93 │ 107512 │ 6772 │ +│ 2022-04-01 │ 120 │ 91560 │ 6687 │ +│ 2022-07-01 │ 183 │ 99764 │ 7377 │ +│ 2022-10-01 │ 123 │ 99447 │ 7052 │ +│ 2023-01-01 │ 126 │ 58733 │ 4891 │ └────────────┴────────────┴───────────┴──────────┘ -58 rows in set. Elapsed: 2663.751 sec. Processed 6.74 billion rows, 1.21 TB (2.53 million rows/s., 454.37 MB/s.) +70 rows in set. Elapsed: 325.835 sec. Processed 14.69 billion rows, 2.57 TB (45.08 million rows/s., 7.87 GB/s.) ``` \ No newline at end of file From 6a5ae6558a1340d2d42f350879ccc88498dbb78e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Jun 2023 12:58:03 +0200 Subject: [PATCH 0907/2223] Add comment for usage of prepare-ci-ami.sh --- tests/ci/worker/prepare-ci-ami.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index 57ab149237a..20e7e3fd53e 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -1,6 +1,10 @@ #!/usr/bin/env bash # The script is downloaded the AWS image builder Task Orchestrator and Executor (AWSTOE) # We can't use `user data script` because cloud-init does not check the exit code +# The script is downloaded in the component named ci-infrastructure-prepare in us-east-1 +# The link there must be adjusted to a particular RAW link, e.g. +# https://github.com/ClickHouse/ClickHouse/raw/653da5f00219c088af66d97a8f1ea3e35e798268/tests/ci/worker/prepare-ci-ami.sh + set -xeuo pipefail echo "Running prepare script" From 9a2b3afb7e35382eb7f5755cdf8e778c75bf8f88 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Jun 2023 15:02:32 +0300 Subject: [PATCH 0908/2223] Revert "Fix msan issue in keyed siphash" --- docs/en/sql-reference/functions/hash-functions.md | 8 ++++---- src/Functions/FunctionsHashing.h | 13 ++----------- .../0_stateless/02534_keyed_siphash.reference | 4 ---- tests/queries/0_stateless/02534_keyed_siphash.sql | 6 ------ 4 files changed, 6 insertions(+), 25 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 89afcca3799..663b8468a94 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -51,7 +51,7 @@ Calculates the MD5 from a string and returns the resulting set of bytes as Fixed If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 {#hash_functions-siphash64} +## sipHash64 (#hash_functions-siphash64) Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. @@ -63,9 +63,9 @@ This is a cryptographic hash function. It works at least three times faster than The function [interprets](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. It then combines the hashes by the following algorithm: -1. The first and the second hash value are concatenated to an array which is hashed. -2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. -3. This calculation is repeated for all remaining hash values of the original input. +1. The first and the second hash value are concatenated to an array which is hashed. +2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. +3. This calculation is repeated for all remaining hash values of the original input. **Arguments** diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 14fad62bd85..3de757bfa3f 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -91,18 +91,12 @@ namespace impl throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64"); if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) - { - const auto & key0col_data = key0col->getData(); - ret.key0 = key0col_data[0]; - } + ret.key0 = key0col->get64(0); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) - { - const auto & key1col_data = key1col->getData(); - ret.key1 = key1col_data[0]; - } + ret.key1 = key1col->get64(0); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); @@ -1426,9 +1420,6 @@ public: { auto col_to = ColumnVector::create(input_rows_count); - if (input_rows_count == 0) - return col_to; - typename ColumnVector::Container & vec_to = col_to->getData(); /// If using a "keyed" algorithm, the first argument is the key and diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index a891d01e99d..3606b9a41db 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -194,7 +194,3 @@ E28DBDE7FE22E41C 1 E28DBDE7FE22E41C 1 -1CE422FEE7BD8DE20000000000000000 -7766709361750702608 -20AF99D3A87829E0 -12489502208762728797 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 7ce50583cdb..9c914f586f0 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -272,9 +272,3 @@ select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; select hex(sipHash64Keyed()); SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; - --- Crashed with memory sanitizer -SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); -SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); -SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); -SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); From 68d46c81b99726a1e0467fc21d02a5311b1e49ca Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 11 Apr 2023 09:53:48 +0000 Subject: [PATCH 0909/2223] Do not store blocks in hash join if nothing is inseted --- src/Interpreters/HashJoin.cpp | 55 +++++++++++++++++++++++------------ 1 file changed, 37 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 146b57049a6..92597f87f4b 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -543,16 +543,20 @@ namespace template struct Inserter { - static ALWAYS_INLINE void insertOne(const HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, + static ALWAYS_INLINE bool insertOne(const HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); if (emplace_result.isInserted() || join.anyTakeLastRow()) + { new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); + return true; + } + return false; } - static ALWAYS_INLINE void insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + static ALWAYS_INLINE bool insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -563,9 +567,10 @@ namespace /// The first element of the list is stored in the value of the hash table, the rest in the pool. emplace_result.getMapped().insert({stored_block, i}, pool); } + return true; } - static ALWAYS_INLINE void insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, + static ALWAYS_INLINE bool insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, const IColumn & asof_column) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -575,6 +580,7 @@ namespace if (emplace_result.isInserted()) time_series_map = new (time_series_map) typename Map::mapped_type(createAsofRowRef(asof_type, join.getAsofInequality())); (*time_series_map)->insert(asof_column, stored_block, i); + return true; } }; @@ -582,7 +588,7 @@ namespace template size_t NO_INLINE insertFromBlockImplTypeCase( HashJoin & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) { [[maybe_unused]] constexpr bool mapped_one = std::is_same_v; constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; @@ -593,6 +599,7 @@ namespace auto key_getter = createKeyGetter(key_columns, key_sizes); + is_inserted = false; for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) @@ -603,11 +610,11 @@ namespace continue; if constexpr (is_asof_join) - Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); + is_inserted |= Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); else if constexpr (mapped_one) - Inserter::insertOne(join, map, key_getter, stored_block, i, pool); + is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); else - Inserter::insertAll(join, map, key_getter, stored_block, i, pool); + is_inserted |= Inserter::insertAll(join, map, key_getter, stored_block, i, pool); } return map.getBufferSizeInCells(); } @@ -616,32 +623,37 @@ namespace template size_t insertFromBlockImplType( HashJoin & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) { if (null_map) return insertFromBlockImplTypeCase( - join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool); + join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); else return insertFromBlockImplTypeCase( - join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool); + join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); } template size_t insertFromBlockImpl( HashJoin & join, HashJoin::Type type, Maps & maps, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) { switch (type) { - case HashJoin::Type::EMPTY: return 0; - case HashJoin::Type::CROSS: return 0; /// Do nothing. We have already saved block, and it is enough. + case HashJoin::Type::EMPTY: + [[fallthrough]]; + case HashJoin::Type::CROSS: + /// Do nothing. We will only save block, and it is enough + is_inserted = true; + return 0; #define M(TYPE) \ case HashJoin::Type::TYPE: \ return insertFromBlockImplType>::Type>(\ - join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool); \ + join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \ break; + APPLY_FOR_JOIN_VARIANTS(M) #undef M } @@ -816,6 +828,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block_, bool check_limits) } } + bool is_inserted = false; if (kind != JoinKind::Cross) { joinDispatch(kind, strictness, data->maps[onexpr_idx], [&](auto kind_, auto strictness_, auto & map) @@ -824,28 +837,34 @@ bool HashJoin::addJoinedBlock(const Block & source_block_, bool check_limits) *this, data->type, map, rows, key_columns, key_sizes[onexpr_idx], stored_block, null_map, /// If mask is false constant, rows are added to hashmap anyway. It's not a happy-flow, so this case is not optimized join_mask_col.getData(), - data->pool); + data->pool, is_inserted); if (multiple_disjuncts) used_flags.reinit(stored_block); - else + else if (is_inserted) /// Number of buckets + 1 value from zero storage used_flags.reinit(size + 1); }); } - if (!multiple_disjuncts && save_nullmap) + if (!multiple_disjuncts && save_nullmap && is_inserted) { data->blocks_nullmaps_allocated_size += null_map_holder->allocatedBytes(); data->blocks_nullmaps.emplace_back(stored_block, null_map_holder); } - if (!multiple_disjuncts && not_joined_map) + if (!multiple_disjuncts && not_joined_map && is_inserted) { data->blocks_nullmaps_allocated_size += not_joined_map->allocatedBytes(); data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); } + if (!multiple_disjuncts && !is_inserted) + { + data->blocks_allocated_size -= stored_block->allocatedBytes(); + data->blocks.pop_back(); + } + if (!check_limits) return true; From 62f950ddaff7178fc479e2ccab236be39567e0a7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 May 2023 12:15:28 +0000 Subject: [PATCH 0910/2223] Keep blocks with nulls for right and full join --- src/Interpreters/HashJoin.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 92597f87f4b..436ecd382cd 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -603,7 +603,12 @@ namespace for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) + { + /// nulls are not inserted into hash table, + /// keep them for RIGHT and FULL joins + is_inserted = true; continue; + } /// Check condition for right table from ON section if (join_mask && !(*join_mask)[i]) @@ -861,6 +866,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block_, bool check_limits) if (!multiple_disjuncts && !is_inserted) { + LOG_TRACE(log, "Skipping inserting block with {} rows", rows); data->blocks_allocated_size -= stored_block->allocatedBytes(); data->blocks.pop_back(); } From 0f0958f82d5048c0aa3b28bdf9c5ce655e853219 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 4 May 2023 14:08:03 +0000 Subject: [PATCH 0911/2223] Add test 02725_any_join_single_row --- .../02725_any_join_single_row.reference | 3 +++ .../0_stateless/02725_any_join_single_row.sql | 26 +++++++++++++++++++ 2 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/02725_any_join_single_row.reference create mode 100644 tests/queries/0_stateless/02725_any_join_single_row.sql diff --git a/tests/queries/0_stateless/02725_any_join_single_row.reference b/tests/queries/0_stateless/02725_any_join_single_row.reference new file mode 100644 index 00000000000..5d748fc6dbb --- /dev/null +++ b/tests/queries/0_stateless/02725_any_join_single_row.reference @@ -0,0 +1,3 @@ +Join(ANY, LEFT, key) 0 1 +Join(ANY, LEFT, key) 1 1 +Join(ANY, LEFT, key) 1 1 diff --git a/tests/queries/0_stateless/02725_any_join_single_row.sql b/tests/queries/0_stateless/02725_any_join_single_row.sql new file mode 100644 index 00000000000..5e5c959c278 --- /dev/null +++ b/tests/queries/0_stateless/02725_any_join_single_row.sql @@ -0,0 +1,26 @@ +CREATE TABLE join_test +( + `key` UInt64, + `value` UInt64 +) +ENGINE = Join(ANY, LEFT, key); + +CREATE TEMPORARY TABLE initial_table_size AS + SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); + +SELECT engine_full, total_rows, total_bytes < 100_000 FROM initial_table_size; + +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); + + +CREATE TEMPORARY TABLE one_row_table_size AS + SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); + +SELECT engine_full, total_rows, total_bytes < 2 * (SELECT total_bytes FROM initial_table_size) FROM one_row_table_size; + +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(10_000); + +SELECT engine_full, total_rows, total_bytes == (SELECT total_bytes FROM one_row_table_size) FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); From 0b4d29ff98916d3ab1f2ea24f4254897b18351cf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Jun 2023 12:19:00 +0000 Subject: [PATCH 0912/2223] Add tests for disks --- .../integration/test_keeper_disks/__init__.py | 0 .../configs/enable_keeper.xml | 53 ++++ tests/integration/test_keeper_disks/test.py | 260 ++++++++++++++++++ 3 files changed, 313 insertions(+) create mode 100644 tests/integration/test_keeper_disks/__init__.py create mode 100644 tests/integration/test_keeper_disks/configs/enable_keeper.xml create mode 100644 tests/integration/test_keeper_disks/test.py diff --git a/tests/integration/test_keeper_disks/__init__.py b/tests/integration/test_keeper_disks/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_disks/configs/enable_keeper.xml b/tests/integration/test_keeper_disks/configs/enable_keeper.xml new file mode 100644 index 00000000000..5814979229c --- /dev/null +++ b/tests/integration/test_keeper_disks/configs/enable_keeper.xml @@ -0,0 +1,53 @@ + + + + + local + /var/lib/clickhouse/coordination/logs/ + + + s3_plain + http://minio1:9001/root/logs/ + minio + minio123 + + + local + /var/lib/clickhouse/coordination/snapshots/ + + + s3_plain + http://minio1:9001/root/snapshots/ + minio + minio123 + + + + + + false + 9181 + 1 + false + + + 5000 + 10000 + trace + 10 + 10 + 1 + 3 + + + + + + + 1 + node + 9234 + + + + \ No newline at end of file diff --git a/tests/integration/test_keeper_disks/test.py b/tests/integration/test_keeper_disks/test.py new file mode 100644 index 00000000000..e978df18bab --- /dev/null +++ b/tests/integration/test_keeper_disks/test.py @@ -0,0 +1,260 @@ +#!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +from minio.deleteobjects import DeleteObject + +import os + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", main_configs=["configs/enable_keeper.xml"], stay_alive=True, with_minio=True +) + +from kazoo.client import KazooClient, KazooState + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def stop_zk(zk): + try: + if zk: + zk.stop() + zk.close() + except: + pass + + +def stop_clickhouse(cluster, cleanup_disks): + node.stop_clickhouse() + + if not cleanup_disks: + return + + node.exec_in_container(["rm", "-rf", "/var/lib/clickhouse/coordination/logs"]) + node.exec_in_container( + ["rm", "-rf", "/var/lib/clickhouse/coordination/snapshots"] + ) + + s3_objects = list_s3_objects(cluster, prefix="") + if len(s3_objects) == 0: + return + + assert ( + len( + list( + cluster.minio_client.remove_objects( + cluster.minio_bucket, + [DeleteObject(obj) for obj in s3_objects], + ) + ) + ) + == 0 + ) + + +def setup_storage(cluster, storage_config, cleanup_disks): + stop_clickhouse(cluster, cleanup_disks) + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), + "/etc/clickhouse-server/config.d/enable_keeper.xml", + ) + node.replace_in_config( + "/etc/clickhouse-server/config.d/enable_keeper.xml", + "", + storage_config, + ) + node.start_clickhouse() + keeper_utils.wait_until_connected(cluster, node) + + +def setup_local_storage(cluster): + setup_storage( + cluster, + "log_local<\\/log_storage_disk>" + "snapshot_local<\\/snapshot_storage_disk>", + cleanup_disks=True, + ) + + +def list_s3_objects(cluster, prefix=""): + minio = cluster.minio_client + prefix_len = len(prefix) + return [ + obj.object_name[prefix_len:] + for obj in minio.list_objects( + cluster.minio_bucket, prefix=prefix, recursive=True + ) + ] + + +def get_local_files(path): + files = node.exec_in_container(["ls", path]).strip().split("\n") + files.sort() + return files + + +def get_local_logs(): + return get_local_files("/var/lib/clickhouse/coordination/logs") + + +def get_local_snapshots(): + return get_local_files("/var/lib/clickhouse/coordination/snapshots") + + +def test_logs_with_disks(started_cluster): + setup_local_storage(started_cluster) + + node_zk = get_fake_zk("node") + try: + node_zk.create("/test") + for _ in range(30): + node_zk.create("/test/somenode", b"somedata", sequence=True) + + stop_zk(node_zk) + + previous_log_files = get_local_logs() + + setup_storage( + started_cluster, + "log_s3_plain<\\/log_storage_disk>" + "log_local<\\/latest_log_storage_disk>" + "snapshot_local<\\/snapshot_storage_disk>", + cleanup_disks=False, + ) + + # all but the latest log should be on S3 + s3_log_files = list_s3_objects(started_cluster, "logs/") + assert set(s3_log_files) == set(previous_log_files[:-1]) + local_log_files = get_local_logs() + assert len(local_log_files) == 1 + assert local_log_files[0] == previous_log_files[-1] + + previous_log_files = s3_log_files + local_log_files + + node_zk = get_fake_zk("node") + + for _ in range(30): + node_zk.create("/test/somenode", b"somedata", sequence=True) + + stop_zk(node_zk) + + log_files = list_s3_objects(started_cluster, "logs/") + local_log_files = get_local_logs() + assert len(local_log_files) == 1 + + log_files.extend(local_log_files) + assert set(log_files) != previous_log_files + + previous_log_files = log_files + + setup_storage( + started_cluster, + "log_s3_plain<\\/old_log_storage_disk>" + "log_local<\\/log_storage_disk>" + "snapshot_local<\\/snapshot_storage_disk>", + cleanup_disks=False, + ) + + local_log_files = get_local_logs() + assert set(local_log_files) == set(previous_log_files) + + node_zk = get_fake_zk("node") + + for child in node_zk.get_children("/test"): + assert node_zk.get(f"/test/{child}")[0] == b"somedata" + + finally: + stop_zk(node_zk) + + +def test_snapshots_with_disks(started_cluster): + setup_local_storage(started_cluster) + + node_zk = get_fake_zk("node") + try: + node_zk.create("/test2") + for _ in range(30): + node_zk.create("/test2/somenode", b"somedata", sequence=True) + + stop_zk(node_zk) + + snapshot_idx = keeper_utils.send_4lw_cmd(cluster, node, "csnp") + node.wait_for_log_line( + f"Created persistent snapshot {snapshot_idx}", look_behind_lines=1000 + ) + + previous_snapshot_files = get_local_snapshots() + + setup_storage( + started_cluster, + "snapshot_s3_plain<\\/snapshot_storage_disk>" + "snapshot_local<\\/latest_snapshot_storage_disk>" + "log_local<\\/log_storage_disk>", + cleanup_disks=False, + ) + + ## all but the latest log should be on S3 + s3_snapshot_files = list_s3_objects(started_cluster, "snapshots/") + assert set(s3_snapshot_files) == set(previous_snapshot_files[:-1]) + local_snapshot_files = get_local_snapshots() + assert len(local_snapshot_files) == 1 + assert local_snapshot_files[0] == previous_snapshot_files[-1] + + previous_snapshot_files = s3_snapshot_files + local_snapshot_files + + node_zk = get_fake_zk("node") + + for _ in range(30): + node_zk.create("/test2/somenode", b"somedata", sequence=True) + + stop_zk(node_zk) + + snapshot_idx = keeper_utils.send_4lw_cmd(cluster, node, "csnp") + node.wait_for_log_line( + f"Created persistent snapshot {snapshot_idx}", look_behind_lines=1000 + ) + + snapshot_files = list_s3_objects(started_cluster, "snapshots/") + local_snapshot_files = get_local_snapshots() + assert len(local_snapshot_files) == 1 + + snapshot_files.extend(local_snapshot_files) + + previous_snapshot_files = snapshot_files + + setup_storage( + started_cluster, + "snapshot_s3_plain<\\/old_snapshot_storage_disk>" + "snapshot_local<\\/snapshot_storage_disk>" + "log_local<\\/log_storage_disk>", + cleanup_disks=False, + ) + + local_snapshot_files = get_local_snapshots() + assert set(local_snapshot_files) == set(previous_snapshot_files) + + node_zk = get_fake_zk("node") + + for child in node_zk.get_children("/test2"): + assert node_zk.get(f"/test2/{child}")[0] == b"somedata" + + finally: + stop_zk(node_zk) From a9eb2c3c4eb8354753e611c6eed71fa9b672077c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Jun 2023 12:31:06 +0000 Subject: [PATCH 0913/2223] use multiple disks for jepsne --- .../resources/keeper_config.xml | 22 +++++++++++++++++++ .../src/jepsen/clickhouse/constants.clj | 2 ++ .../src/jepsen/clickhouse/keeper/utils.clj | 6 +++-- 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/tests/jepsen.clickhouse/resources/keeper_config.xml b/tests/jepsen.clickhouse/resources/keeper_config.xml index 2ab747fbd71..52f2a0dbdc2 100644 --- a/tests/jepsen.clickhouse/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse/resources/keeper_config.xml @@ -8,6 +8,28 @@ never + + + + + local + /var/lib/clickhouse/coordination/logs/ + + + local + /var/lib/clickhouse/coordination/latest_log/ + + + local + /var/lib/clickhouse/coordination/snapshots/ + + + local + /var/lib/clickhouse/coordination/latest_snapshot/ + + + + 9181 {id} diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj index 887fc04265c..2a9a7ef6d27 100644 --- a/tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj @@ -14,7 +14,9 @@ (def coordination-data-dir (str data-dir "/coordination")) (def coordination-snapshots-dir (str coordination-data-dir "/snapshots")) +(def coordination-latest-snapshot-dir (str coordination-data-dir "/latest_snapshot")) (def coordination-logs-dir (str coordination-data-dir "/logs")) +(def coordination-latest_log-dir (str coordination-data-dir "/latest_log")) (def stderr-file (str logs-dir "/stderr.log")) diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj index b882af77758..869313079d0 100644 --- a/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj @@ -146,6 +146,8 @@ :-- :--logger.log (str logs-dir "/clickhouse-keeper.log") :--logger.errorlog (str logs-dir "/clickhouse-keeper.err.log") - :--keeper_server.snapshot_storage_path coordination-snapshots-dir - :--keeper_server.log_storage_path coordination-logs-dir + :--keeper_server.snapshot_storage_disk "snapshot_local" + :--keeper_server.latest_snapshot_storage_disk "latest_snapshot_local" + :--keeper_server.log_storage_disk "log_local" + :--keeper_server.latest_log_storage_disk "latest_log_local" :--path coordination-data-dir)) From 56fa98cb77c30cba3f93dfcd5b04c33dd5b1ec68 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 09:23:39 +0000 Subject: [PATCH 0914/2223] Rename variables for better readability --- src/Functions/FunctionsHashing.h | 71 ++++++++++++++++---------------- 1 file changed, 36 insertions(+), 35 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 3de757bfa3f..70adc7eba0f 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1073,42 +1073,43 @@ private: size_t size = vec_from.size(); for (size_t i = 0; i < size; ++i) { - ToType h; + ToType hash; if constexpr (Impl::use_int_hash_for_pods) { if constexpr (std::is_same_v) - h = IntHash64Impl::apply(bit_cast(vec_from[i])); + hash = IntHash64Impl::apply(bit_cast(vec_from[i])); else - h = IntHash32Impl::apply(bit_cast(vec_from[i])); + hash = IntHash32Impl::apply(bit_cast(vec_from[i])); } else { if constexpr (std::is_same_v) - h = JavaHashImpl::apply(vec_from[i]); + hash = JavaHashImpl::apply(vec_from[i]); else { - FromType v = vec_from[i]; + FromType value = vec_from[i]; if constexpr (std::endian::native == std::endian::big) { - FromType tmp_v; - reverseMemcpy(&tmp_v, &v, sizeof(v)); - v = tmp_v; + FromType value_reversed; + reverseMemcpy(&value_reversed, &value, sizeof(value)); + value = value_reversed; } - h = apply(key, reinterpret_cast(&v), sizeof(v)); + hash = apply(key, reinterpret_cast(&value), sizeof(value)); } } if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else if (auto col_from_const = checkAndGetColumnConst(column)) { auto value = col_from_const->template getValue(); ToType hash; + if constexpr (std::is_same_v) hash = IntHash64Impl::apply(bit_cast(value)); else @@ -1139,45 +1140,45 @@ private: size_t size = vec_from.size(); for (size_t i = 0; i < size; ++i) { - ToType h; + ToType hash; if constexpr (std::endian::native == std::endian::little) { - h = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); + hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); } else { char tmp_buffer[sizeof(vec_from[i])]; reverseMemcpy(tmp_buffer, &vec_from[i], sizeof(vec_from[i])); - h = apply(key, reinterpret_cast(tmp_buffer), sizeof(vec_from[i])); + hash = apply(key, reinterpret_cast(tmp_buffer), sizeof(vec_from[i])); } if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else if (auto col_from_const = checkAndGetColumnConst(column)) { auto value = col_from_const->template getValue(); - ToType h; + ToType hash; if constexpr (std::endian::native == std::endian::little) { - h = apply(key, reinterpret_cast(&value), sizeof(value)); + hash = apply(key, reinterpret_cast(&value), sizeof(value)); } else { char tmp_buffer[sizeof(value)]; reverseMemcpy(tmp_buffer, &value, sizeof(value)); - h = apply(key, reinterpret_cast(tmp_buffer), sizeof(value)); + hash = apply(key, reinterpret_cast(tmp_buffer), sizeof(value)); } size_t size = vec_to.size(); if constexpr (first) - vec_to.assign(size, h); + vec_to.assign(size, hash); else { for (size_t i = 0; i < size; ++i) - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else @@ -1191,11 +1192,11 @@ private: for (size_t i = 0, size = column->size(); i < size; ++i) { StringRef bytes = column->getDataAt(i); - const ToType h = apply(key, bytes.data, bytes.size); + const ToType hash = apply(key, bytes.data, bytes.size); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } @@ -1211,14 +1212,14 @@ private: ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - const ToType h = apply(key, + const ToType hash = apply(key, reinterpret_cast(&data[current_offset]), offsets[i] - current_offset - 1); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); current_offset = offsets[i]; } @@ -1231,11 +1232,11 @@ private: for (size_t i = 0; i < size; ++i) { - const ToType h = apply(key, reinterpret_cast(&data[i * n]), n); + const ToType hash = apply(key, reinterpret_cast(&data[i * n]), n); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column)) @@ -1283,16 +1284,16 @@ private: { ColumnArray::Offset next_offset = offsets[i]; - ToType h; + ToType hash; if constexpr (std::is_same_v) - h = IntHash64Impl::apply(next_offset - current_offset); + hash = IntHash64Impl::apply(next_offset - current_offset); else - h = IntHash32Impl::apply(next_offset - current_offset); + hash = IntHash32Impl::apply(next_offset - current_offset); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); for (size_t j = current_offset; j < next_offset; ++j) vec_to[i] = combineHashes(key, vec_to[i], vec_temp[j]); From 93ba75b370c6ece34fe8480440193b4775b3d105 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 12:23:04 +0000 Subject: [PATCH 0915/2223] Remove parentheses from single-line if/for --- src/Functions/FunctionsHashing.h | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 70adc7eba0f..9896adca7f3 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1119,10 +1119,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) vec_to[i] = combineHashes(key, vec_to[i], hash); - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1142,9 +1140,7 @@ private: { ToType hash; if constexpr (std::endian::native == std::endian::little) - { hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); - } else { char tmp_buffer[sizeof(vec_from[i])]; @@ -1163,9 +1159,7 @@ private: ToType hash; if constexpr (std::endian::native == std::endian::little) - { hash = apply(key, reinterpret_cast(&value), sizeof(value)); - } else { char tmp_buffer[sizeof(value)]; @@ -1176,10 +1170,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) vec_to[i] = combineHashes(key, vec_to[i], hash); - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1246,16 +1238,10 @@ private: const size_t size = vec_to.size(); if constexpr (first) - { vec_to.assign(size, hash); - } else - { for (size_t i = 0; i < size; ++i) - { vec_to[i] = combineHashes(key, vec_to[i], hash); - } - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", From 3f08e3e03f7f61a7e70a5ed89ed66f09f3002f52 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 1 Jun 2023 14:53:07 +0200 Subject: [PATCH 0916/2223] make filter push down through cross join --- .../Optimizations/filterPushDown.cpp | 6 ++--- .../01763_filter_push_down_bugs.reference | 22 +++++++++++++++++++ .../01763_filter_push_down_bugs.sql | 16 ++++++++++++++ 3 files changed, 41 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 37bc894339f..db29038999b 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -272,7 +272,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { /// If totals step has HAVING expression, skip it for now. /// TODO: - /// We can merge HAVING expression with current filer. + /// We can merge HAVING expression with current filter. /// Also, we can push down part of HAVING which depend only on aggregation keys. if (totals_having->getActions()) return 0; @@ -323,9 +323,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin(); - /// Only inner and left(/right) join are supported. Other types may generate default values for left table keys. + /// Only inner, cross and left(/right) join are supported. Other types may generate default values for left table keys. /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (table_join.kind() != JoinKind::Inner && table_join.kind() != kind) + if (table_join.kind() != JoinKind::Inner && table_join.kind() != JoinKind::Cross && table_join.kind() != kind) return 0; bool is_left = kind == JoinKind::Left; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 5aa2e645509..eb4e88a1f81 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -6,3 +6,25 @@ String1_0 String2_0 String3_0 String4_0 1 String1_0 String2_0 String3_0 String4_0 1 1 [0,1,2] 1 +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + Join (JOIN FillRightFirst) + Filter (( + Before JOIN)) + ReadFromMergeTree (default.t1) + Indexes: + PrimaryKey + Keys: + id + Condition: (id in 1-element set) + Parts: 0/19 + Granules: 0/1204 + Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) + Filter (WHERE) + ReadFromMergeTree (default.t2) + Indexes: + PrimaryKey + Keys: + delete_time + Condition: (delete_time in [1685397601, +Inf)) + Parts: 0/1 + Granules: 0/1 diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 1058bf75144..917c350dadb 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -38,6 +38,22 @@ DROP TABLE IF EXISTS Test; select x, y from (select [0, 1, 2] as y, 1 as a, 2 as b) array join y as x where a = 1 and b = 2 and (x = 1 or x != 1) and x = 1; +DROP TABLE IF EXISTS t; create table t(a UInt8) engine=MergeTree order by a; insert into t select * from numbers(2); select a from t t1 join t t2 on t1.a = t2.a where t1.a; +DROP TABLE IF EXISTS t; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id AS SELECT number, toDateTime(number + 1600000000) from numbers(10000000) settings min_insert_block_size_rows=100000; +CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time AS SELECT toDateTime(number + 1610000000) from numbers(100); + +EXPLAIN indexes=1 SELECT id, delete_time FROM t1 + CROSS JOIN ( + SELECT delete_time + FROM t2 WHERE delete_time > '2023-05-30 00:00:00' +) AS d WHERE create_time < delete_time AND id IN (10000001); + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; From e8442b7a88467aa68a87c0abfa3983131c7b8c75 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Jun 2023 12:54:00 +0000 Subject: [PATCH 0917/2223] Add finalize calls --- src/Coordination/Changelog.cpp | 9 ++++++++- src/Coordination/KeeperSnapshotManager.cpp | 6 ++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index bcce9982e04..97f2b437377 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -43,7 +43,8 @@ void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr descrip auto from_path = fs::path(description->path); auto tmp_changelog_name = from_path.parent_path() / (std::string{tmp_prefix} + from_path.filename().string()); { - disk_to->writeFile(tmp_changelog_name); + auto buf = disk_to->writeFile(tmp_changelog_name); + buf->finalize(); } disk_from->copyFile(from_path, *disk_to, path_to, {}); disk_to->removeFile(tmp_changelog_name); @@ -342,9 +343,15 @@ private: } if (log_file_settings.compress_logs) + { compressed_buffer.reset(); + } else + { + chassert(file_buf); + file_buf->finalize(); file_buf.reset(); + } } WriteBuffer & getBuffer() diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 4b9a34c07e9..8dd6d27c379 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -43,7 +43,8 @@ namespace auto from_path = fs::path(path_from); auto tmp_snapshot_name = from_path.parent_path() / (std::string{tmp_prefix} + from_path.filename().string()); { - disk_to->writeFile(tmp_snapshot_name); + auto buf = disk_to->writeFile(tmp_snapshot_name); + buf->finalize(); } disk_from->copyFile(from_path, *disk_to, path_to, {}); disk_to->removeFile(tmp_snapshot_name); @@ -784,7 +785,8 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor auto disk = getLatestSnapshotDisk(); { - disk->writeFile(tmp_snapshot_file_name); + auto buf = disk->writeFile(tmp_snapshot_file_name); + buf->finalize(); } auto writer = disk->writeFile(snapshot_file_name); From 03628bde422823922d1fdd52531d34212270edae Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 12:54:34 +0000 Subject: [PATCH 0918/2223] Fix hashing of const integer values --- src/Functions/FunctionsHashing.h | 26 ++++++++++++++++--- .../0_stateless/02534_keyed_siphash.reference | 3 +++ .../0_stateless/02534_keyed_siphash.sql | 9 +++++++ 3 files changed, 34 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 9896adca7f3..32e3fbbd4ea 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1096,7 +1096,7 @@ private: value = value_reversed; } hash = apply(key, reinterpret_cast(&value), sizeof(value)); - } + } } if constexpr (first) @@ -1110,10 +1110,28 @@ private: auto value = col_from_const->template getValue(); ToType hash; - if constexpr (std::is_same_v) - hash = IntHash64Impl::apply(bit_cast(value)); + if constexpr (Impl::use_int_hash_for_pods) + { + if constexpr (std::is_same_v) + hash = IntHash64Impl::apply(bit_cast(value)); + else + hash = IntHash32Impl::apply(bit_cast(value)); + } else - hash = IntHash32Impl::apply(bit_cast(value)); + { + if constexpr (std::is_same_v) + hash = JavaHashImpl::apply(value); + else + { + if constexpr (std::endian::native == std::endian::big) + { + FromType value_reversed; + reverseMemcpy(&value_reversed, &value, sizeof(value)); + value = value_reversed; + } + hash = apply(key, reinterpret_cast(&value), sizeof(value)); + } + } size_t size = vec_to.size(); if constexpr (first) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 3606b9a41db..ccc514e7ea2 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -194,3 +194,6 @@ E28DBDE7FE22E41C 1 E28DBDE7FE22E41C 1 +Check bug with hashing of const integer values +11862823756610506724 +11862823756610506724 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 9c914f586f0..900b99f548a 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -272,3 +272,12 @@ select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; select hex(sipHash64Keyed()); SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; + +SELECT 'Check bug with hashing of const integer values'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; +INSERT INTO tab VALUES ((2, 2), 4); +-- these two statements must produce the same result +SELECT sipHash64Keyed(key, val) FROM tab; +SELECT sipHash64Keyed(key, 4::UInt64) FROM tab; +DROP TABLE tab; From 53eb360ac21c23a7de58e2e483452846619086d7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 31 May 2023 15:30:41 +0200 Subject: [PATCH 0919/2223] Fix merge_tree_min_rows_for_seek/merge_tree_min_bytes_for_seek for data skipping indexes Signed-off-by: Azat Khuzhin --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- ...ng_index_merge_tree_min_for_seek.reference | 0 ...skipping_index_merge_tree_min_for_seek.sql | 22 +++++++++++++++++++ 3 files changed, 24 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.reference create mode 100644 tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.sql diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2b16ea43179..16b27c2c820 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1729,7 +1729,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( std::max(ranges[i].begin, index_mark * index_granularity), std::min(ranges[i].end, (index_mark + 1) * index_granularity)); - if (res.empty() || res.back().end - data_range.begin > min_marks_for_seek) + if (res.empty() || data_range.begin - res.back().end > min_marks_for_seek) res.push_back(data_range); else res.back().end = data_range.end; @@ -1829,7 +1829,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex( std::max(range.begin, index_mark * index_granularity), std::min(range.end, (index_mark + 1) * index_granularity)); - if (res.empty() || res.back().end - data_range.begin > min_marks_for_seek) + if (res.empty() || data_range.begin - res.back().end > min_marks_for_seek) res.push_back(data_range); else res.back().end = data_range.end; diff --git a/tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.reference b/tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.sql b/tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.sql new file mode 100644 index 00000000000..4cebdde3dfe --- /dev/null +++ b/tests/queries/0_stateless/02781_data_skipping_index_merge_tree_min_for_seek.sql @@ -0,0 +1,22 @@ +-- Tags: no-random-merge-tree-settings, no-random-settings + +DROP TABLE IF EXISTS data; + +CREATE TABLE data +( + key Int, + v1 DateTime, + INDEX v1_index v1 TYPE minmax GRANULARITY 1 +) ENGINE=AggregatingMergeTree() +ORDER BY key +SETTINGS index_granularity=8192; + +SYSTEM STOP MERGES data; + +-- generate 50% of marks that cannot be skipped with v1_index +-- this will create a gap in marks +INSERT INTO data SELECT number, if(number/8192 % 2 == 0, now(), now() - INTERVAL 200 DAY) FROM numbers(1e6); +INSERT INTO data SELECT number+1e6, if(number/8192 % 2 == 0, now(), now() - INTERVAL 200 DAY) FROM numbers(1e6); + +SELECT * FROM data WHERE v1 >= now() - INTERVAL 180 DAY FORMAT Null SETTINGS max_threads=1, max_final_threads=1, force_data_skipping_indices='v1_index', merge_tree_min_rows_for_seek=0, max_rows_to_read=1999999; +SELECT * FROM data WHERE v1 >= now() - INTERVAL 180 DAY FORMAT Null SETTINGS max_threads=1, max_final_threads=1, force_data_skipping_indices='v1_index', merge_tree_min_rows_for_seek=1, max_rows_to_read=1999999; -- { serverError TOO_MANY_ROWS } From 92ee24acd33dc6f62c369dcbd10c90f126b72613 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 1 Jun 2023 13:10:50 +0000 Subject: [PATCH 0920/2223] Automatic style fix --- tests/integration/test_keeper_disks/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_keeper_disks/test.py b/tests/integration/test_keeper_disks/test.py index e978df18bab..11bb215be54 100644 --- a/tests/integration/test_keeper_disks/test.py +++ b/tests/integration/test_keeper_disks/test.py @@ -49,9 +49,7 @@ def stop_clickhouse(cluster, cleanup_disks): return node.exec_in_container(["rm", "-rf", "/var/lib/clickhouse/coordination/logs"]) - node.exec_in_container( - ["rm", "-rf", "/var/lib/clickhouse/coordination/snapshots"] - ) + node.exec_in_container(["rm", "-rf", "/var/lib/clickhouse/coordination/snapshots"]) s3_objects = list_s3_objects(cluster, prefix="") if len(s3_objects) == 0: From 9490cd44acf7bf8a1db59e8e0ed453b1eb85a872 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 13:16:33 +0000 Subject: [PATCH 0921/2223] Include msan fix in protobuf --- contrib/google-protobuf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/google-protobuf b/contrib/google-protobuf index 3b3d8fe1913..c47efe2d8f6 160000 --- a/contrib/google-protobuf +++ b/contrib/google-protobuf @@ -1 +1 @@ -Subproject commit 3b3d8fe191314ea903ea6b072f0e73ef18e15faa +Subproject commit c47efe2d8f6a60022b49ecd6cc23660687c8598f From e5c21e588afc6defa9b5b4bc58aaa8526a9013ce Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Jun 2023 15:19:47 +0200 Subject: [PATCH 0922/2223] Update app.py (#50407) --- tests/ci/workflow_approve_rerun_lambda/app.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 2cdbdecca6e..32cba5d466b 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -126,6 +126,7 @@ TRUSTED_CONTRIBUTORS = { "aalexfvk", "MikhailBurdukov", "tsolodov", # ClickHouse Employee + "kitaisreal", ] } From 97e120ee3c4bd83664eb1e31117103fde6a3464c Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Jun 2023 09:31:05 -0400 Subject: [PATCH 0923/2223] add explain video to docs --- docs/en/sql-reference/statements/explain.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/sql-reference/statements/explain.md b/docs/en/sql-reference/statements/explain.md index 1c93707402f..befe85ab4a4 100644 --- a/docs/en/sql-reference/statements/explain.md +++ b/docs/en/sql-reference/statements/explain.md @@ -7,6 +7,18 @@ title: "EXPLAIN Statement" Shows the execution plan of a statement. +
+ +
+ Syntax: ```sql From d57ffec72fb3a52b6d642270f3fc3907bcabbe0b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 1 Jun 2023 13:45:00 +0000 Subject: [PATCH 0924/2223] Add signal handler for SIGQUIT --- src/Client/ClientBase.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 77a93a25e9b..29b2eb5ce1e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -286,7 +286,7 @@ public: static Int32 cancelled_status() { return exit_after_signals.load(); } }; -/// This signal handler is set only for SIGINT. +/// This signal handler is set for SIGINT and SIGQUIT. void interruptSignalHandler(int signum) { if (QueryInterruptHandler::try_stop()) @@ -325,6 +325,9 @@ void ClientBase::setupSignalHandler() if (sigaction(SIGINT, &new_act, nullptr)) throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); + + if (sigaction(SIGQUIT, &new_act, nullptr)) + throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); } From 238b0927a918b8a36db60a81dc322b204bb387c5 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 1 Jun 2023 16:15:42 +0200 Subject: [PATCH 0925/2223] make test easier --- .../01763_filter_push_down_bugs.reference | 21 ++++++++----------- .../01763_filter_push_down_bugs.sql | 11 ++++++---- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index eb4e88a1f81..7df35e2948d 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -15,16 +15,13 @@ Expression ((Projection + Before ORDER BY)) PrimaryKey Keys: id - Condition: (id in 1-element set) - Parts: 0/19 - Granules: 0/1204 + Condition: (id in [101, 101]) + Parts: 1/1 + Granules: 1/1 Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) - Filter (WHERE) - ReadFromMergeTree (default.t2) - Indexes: - PrimaryKey - Keys: - delete_time - Condition: (delete_time in [1685397601, +Inf)) - Parts: 0/1 - Granules: 0/1 + ReadFromMergeTree (default.t2) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 917c350dadb..2ee249b5ce7 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -46,14 +46,17 @@ DROP TABLE IF EXISTS t; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id AS SELECT number, toDateTime(number + 1600000000) from numbers(10000000) settings min_insert_block_size_rows=100000; -CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time AS SELECT toDateTime(number + 1610000000) from numbers(100); +CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id; +CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time; + +insert into t1 values (101, '2023-05-28 00:00:00'), (102, '2023-05-28 00:00:00'); +insert into t2 values ('2023-05-31 00:00:00'); EXPLAIN indexes=1 SELECT id, delete_time FROM t1 CROSS JOIN ( SELECT delete_time - FROM t2 WHERE delete_time > '2023-05-30 00:00:00' -) AS d WHERE create_time < delete_time AND id IN (10000001); + FROM t2 +) AS d WHERE create_time < delete_time AND id = 101; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From a75598ea655e0340ede8a6d46368147378a4072e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 1 Jun 2023 16:16:39 +0200 Subject: [PATCH 0926/2223] fix test --- .../test_memory_bound_aggregation.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py index 94c788f8f91..d76c4eba409 100644 --- a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py @@ -74,7 +74,7 @@ def test_backward_compatability(start_cluster): from remote('node{1,2,3}', default, t) group by a limit 1 offset 12345 - settings optimize_aggregation_in_order = 1 + settings optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 0 """ ) == "30\n" From 54d526c75c0934b36dd170660ff7222de24a5a13 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 1 Jun 2023 14:22:57 +0000 Subject: [PATCH 0927/2223] Add cast type supprt to DateTimeTransformImpl --- .../functions/type-conversion-functions.md | 85 +++++++++++++++++++ src/Functions/DateTimeTransforms.h | 36 +++++++- src/Functions/FunctionsConversion.h | 18 +++- .../0_stateless/01601_accurate_cast.reference | 4 + .../0_stateless/01601_accurate_cast.sql | 10 +++ 5 files changed, 147 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index c7c66cc771f..a6fc6cd4dfc 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -319,6 +319,49 @@ SELECT ## toDateOrNull ## toDateOrDefault +Converts an input value to [Date](/docs/en/sql-reference/data-types/date.md) data type. +If unsuccessful, returns the lower border value supported by [Date](/docs/en/sql-reference/data-types/date.md). The default value can be specified as a second argument. +Similar to [toDate](#todate). + +**Syntax** + +``` sql +toDateOrDefault(expr [, default_value]) +``` + +**Arguments** + +- `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [Int](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md). +- `default_value` — The default value. [Date](/docs/en/sql-reference/data-types/date.md) + +If `expr` is a number and looks like a UNIX timestamp (is greater than 65535), it is interpreted as a DateTime, then truncated to Date in the current timezone. If `expr` is a number and it is smaller than 65536, it is interpreted as the number of days since 1970-01-01. + +**Returned value** + +- A calendar date. [Date](/docs/en/sql-reference/data-types/date.md) + +**Example** + +Query: + +``` sql +SELECT + toDateOrDefault('2021-01-01', '2023-01-01'::Date), + toDateOrDefault('xx2021-01-01', '2023-01-01'::Date); +``` + +Result: + +```response +┌─toDateOrDefault('2021-01-01', CAST('2023-01-01', 'Date'))─┬─toDateOrDefault('xx2021-01-01', CAST('2023-01-01', 'Date'))─┐ +│ 2021-01-01 │ 2023-01-01 │ +└───────────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────┘ +``` + +**See Also** +- [toDate](#todate) +- [toDate32OrDefault](#todate32ordefault) + ## toDateTime @@ -327,6 +370,48 @@ SELECT ## toDateTimeOrNull ## toDateTimeOrDefault +Converts an input value to [DateTime](/docs/en/sql-reference/data-types/datetime.md) data type. +If unsuccessful, returns the lower border value supported by [DateTime](/docs/en/sql-reference/data-types/datetime.md). The default value can be specified as a third argument. +Similar to [toDateTime](#todatetime). + +**Syntax** + +``` sql +toDateTimeOrDefault(expr, [, time_zone [, default_value]]) +``` + +**Arguments** + +- `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [Int](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md). +- `time_zone` — Time zone. +- `default_value` — The default value. [DateTime](/docs/en/sql-reference/data-types/datetime.md) + +If `expr` is a number, it is interpreted as the number of seconds since the beginning of the Unix Epoch (as Unix timestamp). + +**Returned value** + +- A date time. [DateTime](/docs/en/sql-reference/data-types/datetime.md) + +**Example** + +Query: + +``` sql +SELECT + toDateTimeOrDefault('2021-01-01', 'UTC', '2023-01-01'::DateTime('UTC')), + toDateTimeOrDefault('xx2021-01-01', 'UTC', '2023-01-01'::DateTime('UTC')); +``` + +Result: + +```response +┌─toDateTimeOrDefault('2021-01-01', 'UTC', CAST('2023-01-01', 'DateTime(\'UTC\')'))─┬─toDateTimeOrDefault('xx2021-01-01', 'UTC', CAST('2023-01-01', 'DateTime(\'UTC\')'))─┐ +│ 2021-01-01 00:00:00 │ 2023-01-01 00:00:00 │ +└───────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────┘ +``` + +**See Also** +- [toDateTime](#todatetime) ## toDate32 diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index f179d9fbe60..81b1ec2e356 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -21,6 +21,7 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; + extern const int CANNOT_CONVERT_TYPE; } /** Transformations. @@ -1425,8 +1426,10 @@ struct ToDateTimeComponentsImpl using FactorTransform = ZeroTransform; }; +struct DateTimeAccurateConvertStrategyAdditions {}; +struct DateTimeAccurateOrNullConvertStrategyAdditions {}; -template +template struct Transformer { template @@ -1438,6 +1441,33 @@ struct Transformer for (size_t i = 0; i < size; ++i) { + constexpr bool transformHasExtraCheck = requires(const Transform& t) + { + t.ExtraCheck(vec_from[i], time_zone); + }; + + if constexpr (transformHasExtraCheck) + { + // if constexpr (std::is_same_v + // || std::is_same_v) + { + bool checked = transform.ExtraCheck(vec_from[i], time_zone); + if (!checked) + { + if (std::is_same_v) + { + // vec_to[i] = 0; + // (*vec_null_map_to)[i] = true; + } + else + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Value in column {} cannot be safely converted into type {}", + TypeName, TypeName); + } + } + } + } + if constexpr (is_extended_result) vec_to[i] = static_cast(transform.executeExtendedResult(vec_from[i], time_zone)); else @@ -1446,14 +1476,14 @@ struct Transformer } }; - template struct DateTimeTransformImpl { + template static ColumnPtr execute( const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/, const Transform & transform = {}) { - using Op = Transformer; + using Op = Transformer; const ColumnPtr source_col = arguments[0].column; if (const auto * sources = checkAndGetColumn(source_col.get())) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index d3676349318..d3ccbb82721 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -400,7 +400,11 @@ template struct ToDateTransform8Or16Signed { static constexpr auto name = "toDate"; - + static NO_SANITIZE_UNDEFINED bool ExtraCheck(const FromType & from, const DateLUTImpl &) + { + return from >= 0; + } + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { if (from < 0) @@ -2884,8 +2888,16 @@ private: if constexpr (IsDataTypeNumber && IsDataTypeDateOrDateTime) { - result_column = ConvertImpl::execute( - arguments, result_type, input_rows_count); + if (wrapper_cast_type == CastType::accurate) + { + result_column = ConvertImpl::template execute( + arguments, result_type, input_rows_count); + } + else + { + result_column = ConvertImpl::template execute( + arguments, result_type, input_rows_count); + } return true; } diff --git a/tests/queries/0_stateless/01601_accurate_cast.reference b/tests/queries/0_stateless/01601_accurate_cast.reference index c1e7feffbe6..b662319d263 100644 --- a/tests/queries/0_stateless/01601_accurate_cast.reference +++ b/tests/queries/0_stateless/01601_accurate_cast.reference @@ -6,3 +6,7 @@ 5 1 12 +2023-05-30 14:38:20 +1970-01-01 00:00:19 +2023-05-30 +1970-01-20 \ No newline at end of file diff --git a/tests/queries/0_stateless/01601_accurate_cast.sql b/tests/queries/0_stateless/01601_accurate_cast.sql index b5fd4fb04a4..1ab98e26d1a 100644 --- a/tests/queries/0_stateless/01601_accurate_cast.sql +++ b/tests/queries/0_stateless/01601_accurate_cast.sql @@ -22,3 +22,13 @@ SELECT accurateCast(-10, 'Decimal32(9)'); -- { serverError 407 } SELECT accurateCast('123', 'FixedString(2)'); -- { serverError 131 } SELECT accurateCast('12', 'FixedString(2)'); + +SELECT accurateCast(-1, 'DateTime'); -- { serverError 70 } +SELECT accurateCast('1xxx', 'DateTime'); -- { serverError 41 } +SELECT accurateCast('2023-05-30 14:38:20', 'DateTime'); +SELECT accurateCast(19, 'DateTime'); + +SELECT accurateCast(-1, 'Date'); -- { serverError 70 } +SELECT accurateCast('1xxx', 'Date'); -- { serverError 70 } +SELECT accurateCast('2023-05-30', 'Date'); +SELECT accurateCast(19, 'Date'); From a4cb82127dfe488fb1e1ad90aebaccf469ad742e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 May 2023 13:54:33 +0200 Subject: [PATCH 0928/2223] Analyzer: WIP on distributed queries --- src/Analyzer/ColumnNode.cpp | 5 ++++- src/Analyzer/TableNode.cpp | 2 ++ src/Planner/PlannerContext.cpp | 14 +++++++------- src/Planner/PlannerJoinTree.cpp | 3 ++- src/Planner/Utils.cpp | 2 +- src/Storages/StorageDistributed.cpp | 6 ++++-- 6 files changed, 20 insertions(+), 12 deletions(-) diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index a9d47f8287d..f020040ea78 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -117,7 +117,10 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const else { const auto & table_storage_id = table_node->getStorageID(); - column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() }; + if (table_storage_id.hasDatabase()) + column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() }; + else + column_identifier_parts = { table_storage_id.getTableName() }; } } } diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index c86cbcd5a80..17d12bd6afa 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -91,6 +91,8 @@ ASTPtr TableNode::toASTImpl(const ConvertToASTOptions & /* options */) const if (!temporary_table_name.empty()) return std::make_shared(temporary_table_name); + if (!storage_id.hasDatabase()) + return std::make_shared(storage_id.getTableName()); return std::make_shared(storage_id.getDatabaseName(), storage_id.getTableName()); } diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 346cc6d2080..7ab8180eb9c 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -19,17 +19,17 @@ const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const Quer return createColumnIdentifier(column_node_typed.getColumn(), column_source_node); } -const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & column_source_node) +const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & /*column_source_node*/) { std::string column_identifier; - if (column_source_node->hasAlias()) - column_identifier += column_source_node->getAlias(); - else if (const auto * table_source_node = column_source_node->as()) - column_identifier += table_source_node->getStorageID().getFullNameNotQuoted(); + // if (column_source_node->hasAlias()) + // column_identifier += column_source_node->getAlias(); + // else if (const auto * table_source_node = column_source_node->as()) + // column_identifier += table_source_node->getStorageID().getFullNameNotQuoted(); - if (!column_identifier.empty()) - column_identifier += '.'; + // if (!column_identifier.empty()) + // column_identifier += '.'; column_identifier += column.name; column_identifier += '_' + std::to_string(column_identifiers.size()); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4f091f73187..4a1708f96d3 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -106,7 +106,8 @@ void checkAccessRights(const TableNode & table_node, const Names & column_names, storage_id.getFullTableName()); } - query_context->checkAccess(AccessType::SELECT, storage_id, column_names); + if (storage_id.hasDatabase()) + query_context->checkAccess(AccessType::SELECT, storage_id, column_names); } NameAndTypePair chooseSmallestColumnToReadFromStorage(const StoragePtr & storage, const StorageSnapshotPtr & storage_snapshot) diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 5c61b2fc2c7..cd4fb9182e9 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -107,7 +107,7 @@ Block buildCommonHeaderForUnion(const Blocks & queries_headers, SelectUnionMode ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node) { auto & query_node_typed = query_node->as(); - auto result_ast = query_node_typed.toAST(); + auto result_ast = query_node_typed.toAST({ .fully_qualified_identifiers = false }); while (true) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index bcf6f68d00d..e0bb14c62fd 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -943,7 +943,9 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, } else { - auto resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); + auto resolved_remote_storage_id = remote_storage_id; + if (remote_storage_id.hasDatabase()) + resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns()); auto table_node = std::make_shared(std::move(storage), query_context); @@ -1059,7 +1061,7 @@ void StorageDistributed::read( remote_table_function_ptr); query_ast = queryNodeToSelectQuery(query_tree_distributed); - header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_ast, local_context, SelectQueryOptions(processed_stage).analyze()); + header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, SelectQueryOptions(processed_stage).analyze()); } else { From 70c1b1de57252ace05fe69b3a791b3fe4f025273 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 May 2023 15:43:30 +0000 Subject: [PATCH 0929/2223] Remove code --- src/Planner/PlannerContext.cpp | 8 -- ...ns_optimize_read_in_window_order.reference | 10 +- .../02227_union_match_by_name.reference | 4 +- .../0_stateless/02303_query_kind.reference | 16 +-- .../02381_join_dup_columns_in_plan.reference | 44 +++--- .../0_stateless/02421_explain_subquery.sql | 2 +- .../02451_order_by_monotonic.reference | 32 ++--- .../02481_aggregation_in_order_plan.reference | 2 +- .../02514_analyzer_drop_join_on.reference | 134 +++++++++--------- ...dicate_push_down_filled_join_fix.reference | 30 ++-- ...n_merge_tree_prewhere_row_policy.reference | 8 +- 11 files changed, 141 insertions(+), 149 deletions(-) diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 7ab8180eb9c..a788a6cbc3c 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -23,14 +23,6 @@ const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const Name { std::string column_identifier; - // if (column_source_node->hasAlias()) - // column_identifier += column_source_node->getAlias(); - // else if (const auto * table_source_node = column_source_node->as()) - // column_identifier += table_source_node->getStorageID().getFullNameNotQuoted(); - - // if (!column_identifier.empty()) - // column_identifier += '.'; - column_identifier += column.name; column_identifier += '_' + std::to_string(column_identifiers.size()); diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference index f23cf03913b..8a33df9fad2 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference @@ -7,19 +7,19 @@ Partial sorting plan Prefix sort description: n ASC Result sort description: n ASC, x ASC optimize_read_in_window_order=1, allow_experimental_analyzer=1 - Prefix sort description: default.test_01655_plan_optimizations_optimize_read_in_window_order_n.n_0 ASC - Result sort description: default.test_01655_plan_optimizations_optimize_read_in_window_order_n.n_0 ASC, default.test_01655_plan_optimizations_optimize_read_in_window_order_n.x_1 ASC + Prefix sort description: n_0 ASC + Result sort description: n_0 ASC, x_1 ASC No sorting plan optimize_read_in_window_order=0 Sort description: n ASC, x ASC optimize_read_in_window_order=0, allow_experimental_analyzer=1 - Sort description: default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.n_0 ASC, default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.x_1 ASC + Sort description: n_0 ASC, x_1 ASC optimize_read_in_window_order=1 Prefix sort description: n ASC, x ASC Result sort description: n ASC, x ASC optimize_read_in_window_order=1, allow_experimental_analyzer=1 - Prefix sort description: default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.n_0 ASC, default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.x_1 ASC - Result sort description: default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.n_0 ASC, default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.x_1 ASC + Prefix sort description: n_0 ASC, x_1 ASC + Result sort description: n_0 ASC, x_1 ASC Complex ORDER BY optimize_read_in_window_order=0 3 3 1 diff --git a/tests/queries/0_stateless/02227_union_match_by_name.reference b/tests/queries/0_stateless/02227_union_match_by_name.reference index e51ea983f7f..685b3c83b05 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.reference +++ b/tests/queries/0_stateless/02227_union_match_by_name.reference @@ -30,7 +30,7 @@ Header: avgWeighted(x, y) Nullable(Float64) 255_UInt8 UInt8 1_UInt8 UInt8 Expression (Change column names to column identifiers) - Header: system.one.dummy_0 UInt8 + Header: dummy_0 UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 Expression (Conversion before UNION) @@ -46,7 +46,7 @@ Header: avgWeighted(x, y) Nullable(Float64) NULL_Nullable(Nothing) Nullable(Nothing) 1_UInt8 UInt8 Expression (Change column names to column identifiers) - Header: system.one.dummy_0 UInt8 + Header: dummy_0 UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); diff --git a/tests/queries/0_stateless/02303_query_kind.reference b/tests/queries/0_stateless/02303_query_kind.reference index 5af8c2b743f..8d119fb22b2 100644 --- a/tests/queries/0_stateless/02303_query_kind.reference +++ b/tests/queries/0_stateless/02303_query_kind.reference @@ -2,35 +2,35 @@ clickhouse-client --allow_experimental_analyzer=1 --query_kind secondary_query - Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --allow_experimental_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-client --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index 31a37862663..dd5c9d4616e 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -2,51 +2,51 @@ Expression Header: key String value String Join - Header: s1.key_0 String - s2.value_1 String + Header: key_0 String + value_1 String Expression - Header: s1.key_0 String + Header: key_0 String ReadFromStorage Header: dummy UInt8 Union - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String Expression - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String ReadFromStorage Header: dummy UInt8 Expression - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String ReadFromStorage Header: dummy UInt8 Expression Header: key String value String Join - Header: s1.key_0 String - s2.key_2 String - s2.value_1 String + Header: key_0 String + key_2 String + value_1 String Sorting - Header: s1.key_0 String + Header: key_0 String Expression - Header: s1.key_0 String + Header: key_0 String ReadFromStorage Header: dummy UInt8 Sorting - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String Union - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String Expression - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String ReadFromStorage Header: dummy UInt8 Expression - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String ReadFromStorage Header: dummy UInt8 diff --git a/tests/queries/0_stateless/02421_explain_subquery.sql b/tests/queries/0_stateless/02421_explain_subquery.sql index 32631b54d0c..4b970f81219 100644 --- a/tests/queries/0_stateless/02421_explain_subquery.sql +++ b/tests/queries/0_stateless/02421_explain_subquery.sql @@ -34,7 +34,7 @@ DROP TABLE t1; SET allow_experimental_analyzer = 1; -SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: system.numbers.number__ UInt64%'; +SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: number__ UInt64%'; SELECT count() > 0 FROM (EXPLAIN PLAN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%'; SELECT count() > 0 FROM (EXPLAIN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%'; SELECT count() > 0 FROM (EXPLAIN CURRENT TRANSACTION); diff --git a/tests/queries/0_stateless/02451_order_by_monotonic.reference b/tests/queries/0_stateless/02451_order_by_monotonic.reference index f9f0ef38be1..05f20a9bad8 100644 --- a/tests/queries/0_stateless/02451_order_by_monotonic.reference +++ b/tests/queries/0_stateless/02451_order_by_monotonic.reference @@ -4,19 +4,19 @@ 2022-09-09 12:00:00 0x 2022-09-09 12:00:00 1 2022-09-09 12:00:00 1x - Prefix sort description: toStartOfMinute(test.t_0) ASC - Result sort description: toStartOfMinute(test.t_0) ASC, test.c1_1 ASC - Prefix sort description: toStartOfMinute(test.t_0) ASC - Result sort description: toStartOfMinute(test.t_0) ASC - Prefix sort description: negate(test.a_0) ASC - Result sort description: negate(test.a_0) ASC - Prefix sort description: negate(test.a_0) ASC, negate(test.b_1) ASC - Result sort description: negate(test.a_0) ASC, negate(test.b_1) ASC - Prefix sort description: test.a_0 DESC, negate(test.b_1) ASC - Result sort description: test.a_0 DESC, negate(test.b_1) ASC - Prefix sort description: negate(test.a_0) ASC, test.b_1 DESC - Result sort description: negate(test.a_0) ASC, test.b_1 DESC - Prefix sort description: negate(test.a_0) ASC - Result sort description: negate(test.a_0) ASC, test.b_1 ASC - Prefix sort description: test.a_0 ASC - Result sort description: test.a_0 ASC, negate(test.b_1) ASC + Prefix sort description: toStartOfMinute(t_0) ASC + Result sort description: toStartOfMinute(t_0) ASC, c1_1 ASC + Prefix sort description: toStartOfMinute(t_0) ASC + Result sort description: toStartOfMinute(t_0) ASC + Prefix sort description: negate(a_0) ASC + Result sort description: negate(a_0) ASC + Prefix sort description: negate(a_0) ASC, negate(b_1) ASC + Result sort description: negate(a_0) ASC, negate(b_1) ASC + Prefix sort description: a_0 DESC, negate(b_1) ASC + Result sort description: a_0 DESC, negate(b_1) ASC + Prefix sort description: negate(a_0) ASC, b_1 DESC + Result sort description: negate(a_0) ASC, b_1 DESC + Prefix sort description: negate(a_0) ASC + Result sort description: negate(a_0) ASC, b_1 ASC + Prefix sort description: a_0 ASC + Result sort description: a_0 ASC, negate(b_1) ASC diff --git a/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference b/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference index bb4eb4ddd75..b11f3e3a1d3 100644 --- a/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference +++ b/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference @@ -6,5 +6,5 @@ Order: a ASC, c ASC ReadFromMergeTree (default.tab) Aggregating - Order: default.tab.a_0 ASC, default.tab.c_2 ASC + Order: a_0 ASC, c_2 ASC ReadFromMergeTree (default.tab) diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 8b4cafc3260..0037ab85c07 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -6,43 +6,43 @@ SELECT count() FROM a JOIN b ON b.b1 = a.a1 JOIN c ON c.c1 = b.b1 JOIN d ON d.d1 Expression ((Project names + Projection)) Header: count() UInt64 Aggregating - Header: default.a.a2_4 String + Header: a2_4 String count() UInt64 Expression ((Before GROUP BY + DROP unused columns after JOIN)) - Header: default.a.a2_4 String + Header: a2_4 String Join (JOIN FillRightFirst) - Header: default.a.a2_4 String - default.c.c1_2 UInt64 + Header: a2_4 String + c1_2 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: default.a.a2_4 String - default.c.c1_2 UInt64 + Header: a2_4 String + c1_2 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a2_4 String - default.b.b1_0 UInt64 - default.c.c1_2 UInt64 + Header: a2_4 String + b1_0 UInt64 + c1_2 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: default.a.a2_4 String - default.b.b1_0 UInt64 + Header: a2_4 String + b1_0 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a1_1 UInt64 - default.a.a2_4 String - default.b.b1_0 UInt64 + Header: a1_1 UInt64 + a2_4 String + b1_0 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.a.a1_1 UInt64 - default.a.a2_4 String + Header: a1_1 UInt64 + a2_4 String ReadFromMemoryStorage Header: a1 UInt64 a2 String Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.b.b1_0 UInt64 + Header: b1_0 UInt64 ReadFromMemoryStorage Header: b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.c.c1_2 UInt64 + Header: c1_2 UInt64 ReadFromMemoryStorage Header: c1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.d.d1_3 UInt64 + Header: d1_3 UInt64 ReadFromMemoryStorage Header: d1 UInt64 EXPLAIN PLAN header = 1 @@ -52,38 +52,38 @@ Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: a2 String d2 String Join (JOIN FillRightFirst) - Header: default.a.a2_0 String - default.a.k_2 UInt64 - default.d.d2_1 String + Header: a2_0 String + k_2 UInt64 + d2_1 String Expression (DROP unused columns after JOIN) - Header: default.a.a2_0 String - default.a.k_2 UInt64 + Header: a2_0 String + k_2 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a2_0 String - default.a.k_2 UInt64 + Header: a2_0 String + k_2 UInt64 Expression (DROP unused columns after JOIN) - Header: default.a.a2_0 String - default.a.k_2 UInt64 + Header: a2_0 String + k_2 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a2_0 String - default.a.k_2 UInt64 + Header: a2_0 String + k_2 UInt64 Expression (Change column names to column identifiers) - Header: default.a.a2_0 String - default.a.k_2 UInt64 + Header: a2_0 String + k_2 UInt64 ReadFromMemoryStorage Header: a2 String k UInt64 Expression (Change column names to column identifiers) - Header: default.b.k_3 UInt64 + Header: k_3 UInt64 ReadFromMemoryStorage Header: k UInt64 Expression (Change column names to column identifiers) - Header: default.c.k_4 UInt64 + Header: k_4 UInt64 ReadFromMemoryStorage Header: k UInt64 Expression (Change column names to column identifiers) - Header: default.d.d2_1 String - default.d.k_5 UInt64 + Header: d2_1 String + k_5 UInt64 ReadFromMemoryStorage Header: d2 String k UInt64 @@ -97,55 +97,55 @@ WHERE c.c2 != '' ORDER BY a.a2 Expression (Project names) Header: bx String Sorting (Sorting for ORDER BY) - Header: default.a.a2_6 String - b.bx_0 String + Header: a2_6 String + bx_0 String Expression ((Before ORDER BY + (Projection + ))) - Header: default.a.a2_6 String - b.bx_0 String + Header: a2_6 String + bx_0 String Join (JOIN FillRightFirst) - Header: default.a.a2_6 String - b.bx_0 String - default.c.c2_5 String - default.c.c1_3 UInt64 + Header: a2_6 String + bx_0 String + c2_5 String + c1_3 UInt64 Filter (( + (JOIN actions + DROP unused columns after JOIN))) - Header: default.a.a2_6 String - b.bx_0 String - default.c.c2_5 String - default.c.c1_3 UInt64 + Header: a2_6 String + bx_0 String + c2_5 String + c1_3 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a2_6 String - b.bx_0 String - b.b1_1 UInt64 - default.c.c2_5 String - default.c.c1_3 UInt64 + Header: a2_6 String + bx_0 String + b1_1 UInt64 + c2_5 String + c1_3 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: default.a.a2_6 String - b.bx_0 String - b.b1_1 UInt64 + Header: a2_6 String + bx_0 String + b1_1 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a1_2 UInt64 - default.a.a2_6 String - b.bx_0 String - b.b1_1 UInt64 + Header: a1_2 UInt64 + a2_6 String + bx_0 String + b1_1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.a.a1_2 UInt64 - default.a.a2_6 String + Header: a1_2 UInt64 + a2_6 String ReadFromMemoryStorage Header: a1 UInt64 a2 String Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) - Header: b.b1_1 UInt64 - b.bx_0 String + Header: b1_1 UInt64 + bx_0 String ReadFromMemoryStorage Header: b1 UInt64 b2 String Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.c.c1_3 UInt64 - default.c.c2_5 String + Header: c1_3 UInt64 + c2_5 String ReadFromMemoryStorage Header: c1 UInt64 c2 String Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) - Header: d.d1_4 UInt64 + Header: d1_4 UInt64 ReadFromStorage (SystemNumbers) Header: number UInt64 diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference index ecdb62c5cb5..986ecffcdf8 100644 --- a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference @@ -2,27 +2,27 @@ Expression ((Project names + (Projection + ))) Header: t1.id UInt64 t1.value String t2.value String -Actions: INPUT : 0 -> t1.id_0 UInt64 : 0 - INPUT : 1 -> t1.value_1 String : 1 - INPUT : 2 -> t2.value_2 String : 2 - ALIAS t1.id_0 :: 0 -> t1.id UInt64 : 3 - ALIAS t1.value_1 :: 1 -> t1.value String : 0 - ALIAS t2.value_2 :: 2 -> t2.value String : 1 +Actions: INPUT : 0 -> id_0 UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 String : 2 + ALIAS id_0 :: 0 -> t1.id UInt64 : 3 + ALIAS value_1 :: 1 -> t1.value String : 0 + ALIAS value_2 :: 2 -> t2.value String : 1 Positions: 3 0 1 FilledJoin (Filled JOIN) - Header: t1.id_0 UInt64 - t1.value_1 String - t2.value_2 String + Header: id_0 UInt64 + value_1 String + value_2 String Filter (( + (JOIN actions + Change column names to column identifiers))) - Header: t1.id_0 UInt64 - t1.value_1 String - Filter column: equals(t1.id_0, 0_UInt8) (removed) + Header: id_0 UInt64 + value_1 String + Filter column: equals(id_0, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 2 - ALIAS id :: 0 -> t1.id_0 UInt64 : 3 - ALIAS value :: 1 -> t1.value_1 String : 0 - FUNCTION equals(t1.id_0 : 3, 0_UInt8 :: 2) -> equals(t1.id_0, 0_UInt8) UInt8 : 1 + ALIAS id :: 0 -> id_0 UInt64 : 3 + ALIAS value :: 1 -> value_1 String : 0 + FUNCTION equals(id_0 : 3, 0_UInt8 :: 2) -> equals(id_0, 0_UInt8) UInt8 : 1 Positions: 1 3 0 ReadFromMergeTree (default.test_table) Header: id UInt64 diff --git a/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference b/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference index 2fe98ea1682..cc16a1fce02 100644 --- a/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference +++ b/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference @@ -29,10 +29,10 @@ Header: id UInt64 value String Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 - ALIAS id :: 0 -> default.test_table.id_0 UInt64 : 2 - ALIAS value :: 1 -> default.test_table.value_1 String : 0 - ALIAS default.test_table.id_0 :: 2 -> id UInt64 : 1 - ALIAS default.test_table.value_1 :: 0 -> value String : 2 + ALIAS id :: 0 -> id_0 UInt64 : 2 + ALIAS value :: 1 -> value_1 String : 0 + ALIAS id_0 :: 2 -> id UInt64 : 1 + ALIAS value_1 :: 0 -> value String : 2 Positions: 1 2 ReadFromMergeTree (default.test_table) Header: id UInt64 From 2c878581bf0642b7e7d8b3c63ac483f6f102639d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 May 2023 17:20:53 +0000 Subject: [PATCH 0930/2223] Update 02377_optimize_sorting_by_input_stream_properties_explain reference --- ..._input_stream_properties_explain.reference | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference index 69c325c21a9..5c9e39805b7 100644 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference @@ -8,7 +8,7 @@ Sorting (None) -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): default.optimize_sorting.a_0 ASC +Sorting (Global): a_0 ASC Sorting (None) Sorting (None) -- disable optimization -> sorting order is NOT propagated from subquery -> full sort @@ -36,8 +36,8 @@ Sorting (Stream): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): default.optimize_sorting.a_0 ASC -Sorting (Stream): default.optimize_sorting.a_0 ASC +Sorting (Global): a_0 ASC +Sorting (Stream): a_0 ASC Sorting (Stream): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) @@ -48,8 +48,8 @@ Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Global): plus(default.optimize_sorting.a_0, 1_UInt8) ASC -Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Global): plus(a_0, 1_UInt8) ASC +Sorting (Chunk): a_0 ASC Sorting (Chunk): a ASC -- ExpressionStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 @@ -61,7 +61,7 @@ Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 Sorting (Global): plus(a, 1) ASC Sorting (Sorting for ORDER BY) -Sorting (Global): plus(default.optimize_sorting.a_0, 1_UInt8) ASC +Sorting (Global): plus(a_0, 1_UInt8) ASC Sorting (None) Sorting (Chunk): a ASC -- FilterStep preserves sort mode @@ -71,7 +71,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 Sorting (Chunk): a ASC -Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a_0 ASC Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC @@ -79,7 +79,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC -Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a_0 ASC Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC @@ -87,7 +87,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC -Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a_0 ASC Sorting (Chunk): a ASC -- FilterStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 @@ -122,8 +122,8 @@ Sorting (Sorting for ORDER BY) Sorting (Global): a_0 ASC Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Global): default.optimize_sorting.a_2 ASC -Sorting (Stream): default.optimize_sorting.a_2 ASC +Sorting (Global): a_2 ASC +Sorting (Stream): a_2 ASC Sorting (Stream): a ASC -- aliases DONT break sorting order -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) @@ -152,10 +152,10 @@ Sorting (Chunk): a ASC Sorting (None) Sorting (Sorting for ORDER BY) Sorting (Global): plus(a_0, 1_UInt8) ASC -Sorting (Global): plus(default.optimize_sorting.a_3, 1_UInt8) ASC +Sorting (Global): plus(a_3, 1_UInt8) ASC Sorting (Sorting for ORDER BY) -Sorting (Global): plus(default.optimize_sorting.a_3, 1_UInt8) ASC -Sorting (Chunk): default.optimize_sorting.a_3 ASC +Sorting (Global): plus(a_3, 1_UInt8) ASC +Sorting (Chunk): a_3 ASC Sorting (Chunk): a ASC -- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a @@ -167,6 +167,6 @@ Sorting (Stream): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): default.optimize_sorting.a_0 ASC -Sorting (Stream): default.optimize_sorting.a_0 ASC +Sorting (Global): a_0 ASC +Sorting (Stream): a_0 ASC Sorting (Stream): a ASC From d79bd5694ad6345e41aa640afb4d839c46da716d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 24 May 2023 00:05:27 +0200 Subject: [PATCH 0931/2223] Fix LambdaNode::cloneImpl --- src/Analyzer/LambdaNode.cpp | 5 +++-- src/Analyzer/LambdaNode.h | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp index 0d15c4d42e6..4be4d69c190 100644 --- a/src/Analyzer/LambdaNode.cpp +++ b/src/Analyzer/LambdaNode.cpp @@ -10,9 +10,10 @@ namespace DB { -LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_) +LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_, DataTypePtr result_type_) : IQueryTreeNode(children_size) , argument_names(std::move(argument_names_)) + , result_type(std::move(result_type_)) { auto arguments_list_node = std::make_shared(); auto & nodes = arguments_list_node->getNodes(); @@ -63,7 +64,7 @@ void LambdaNode::updateTreeHashImpl(HashState & state) const QueryTreeNodePtr LambdaNode::cloneImpl() const { - return std::make_shared(argument_names, getExpression()); + return std::make_shared(argument_names, getExpression(), result_type); } ASTPtr LambdaNode::toASTImpl(const ConvertToASTOptions & options) const diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h index 355ed77cc6a..ea44a7e8187 100644 --- a/src/Analyzer/LambdaNode.h +++ b/src/Analyzer/LambdaNode.h @@ -35,7 +35,7 @@ class LambdaNode final : public IQueryTreeNode { public: /// Initialize lambda with argument names and lambda body expression - explicit LambdaNode(Names argument_names_, QueryTreeNodePtr expression_); + explicit LambdaNode(Names argument_names_, QueryTreeNodePtr expression_, DataTypePtr result_type_ = {}); /// Get argument names const Names & getArgumentNames() const From b86516131bef352b114e67c17b77706496bb0fd9 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 24 May 2023 16:22:58 +0000 Subject: [PATCH 0932/2223] Attempt to fix global JOINs and INs --- src/Storages/StorageDistributed.cpp | 29 +++++++++++++++++++++++++---- 1 file changed, 25 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index e0bb14c62fd..05f9821f6cb 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,6 +30,8 @@ #include #include #include +#include "Analyzer/IQueryTreeNode.h" +#include "Analyzer/MatcherNode.h" #include #include @@ -1003,7 +1005,17 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, planner_context->getMutableQueryContext(), global_in_or_join_node.subquery_depth); temporary_table_expression_node->setAlias(join_right_table_expression->getAlias()); - replacement_map.emplace(join_right_table_expression.get(), std::move(temporary_table_expression_node)); + + auto in_second_argument_query_node = std::make_shared(Context::createCopy(query_context)); + in_second_argument_query_node->setIsSubquery(true); + in_second_argument_query_node->getProjectionNode() = std::make_shared(); + in_second_argument_query_node->getProjection().getNodes() = { std::make_shared() }; + in_second_argument_query_node->getJoinTree() = std::move(temporary_table_expression_node); + + QueryAnalysisPass query_analysis_pass; + query_analysis_pass.run(in_second_argument_query_node, query_context); + + replacement_map.emplace(join_right_table_expression.get(), std::move(in_second_argument_query_node)); continue; } else if (auto * in_function_node = global_in_or_join_node.query_node->as()) @@ -1016,7 +1028,17 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, planner_context->getMutableQueryContext(), global_in_or_join_node.subquery_depth); - in_function_subquery_node = std::move(temporary_table_expression_node); + + auto in_second_argument_query_node = std::make_shared(Context::createCopy(query_context)); + in_second_argument_query_node->setIsSubquery(true); + in_second_argument_query_node->getProjectionNode() = std::make_shared(); + in_second_argument_query_node->getProjection().getNodes() = { std::make_shared() }; + in_second_argument_query_node->getJoinTree() = std::move(temporary_table_expression_node); + + QueryAnalysisPass query_analysis_pass; + query_analysis_pass.run(in_second_argument_query_node, query_context); + + in_function_subquery_node = std::move(in_second_argument_query_node); } else { @@ -1059,9 +1081,8 @@ void StorageDistributed::read( storage_snapshot, remote_storage_id, remote_table_function_ptr); - - query_ast = queryNodeToSelectQuery(query_tree_distributed); header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, SelectQueryOptions(processed_stage).analyze()); + query_ast = queryNodeToSelectQuery(query_tree_distributed); } else { From 85e5ed79e5b60f00df2cf7d8c41b249485c02547 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 25 May 2023 15:58:08 +0000 Subject: [PATCH 0933/2223] Fix distributed JOINs --- src/Storages/StorageDistributed.cpp | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 05f9821f6cb..98c92541f67 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1006,16 +1006,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, global_in_or_join_node.subquery_depth); temporary_table_expression_node->setAlias(join_right_table_expression->getAlias()); - auto in_second_argument_query_node = std::make_shared(Context::createCopy(query_context)); - in_second_argument_query_node->setIsSubquery(true); - in_second_argument_query_node->getProjectionNode() = std::make_shared(); - in_second_argument_query_node->getProjection().getNodes() = { std::make_shared() }; - in_second_argument_query_node->getJoinTree() = std::move(temporary_table_expression_node); - - QueryAnalysisPass query_analysis_pass; - query_analysis_pass.run(in_second_argument_query_node, query_context); - - replacement_map.emplace(join_right_table_expression.get(), std::move(in_second_argument_query_node)); + replacement_map.emplace(join_right_table_expression.get(), std::move(temporary_table_expression_node)); continue; } else if (auto * in_function_node = global_in_or_join_node.query_node->as()) From bc7b7f2cd2e1c56aacecd2855feb37bac10590c6 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 25 May 2023 16:00:13 +0000 Subject: [PATCH 0934/2223] Always add source alias in ColumnNode::toASTImpl --- src/Analyzer/ColumnNode.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index f020040ea78..76f5697afb0 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -96,7 +96,7 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const std::vector column_identifier_parts; auto column_source = getColumnSourceOrNull(); - if (column_source && options.fully_qualified_identifiers) + if (column_source) { auto node_type = column_source->getNodeType(); if (node_type == QueryTreeNodeType::TABLE || @@ -108,7 +108,8 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const { column_identifier_parts = {column_source->getAlias()}; } - else if (auto * table_node = column_source->as()) + else if (auto * table_node = column_source->as(); + table_node && options.fully_qualified_identifiers) { if (!table_node->getTemporaryTableName().empty()) { From c6dcb69b853528aacda070d00b3a873179470f82 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 30 May 2023 14:33:35 +0000 Subject: [PATCH 0935/2223] Fix GLOBAL IN --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 9 +++--- src/Planner/CollectSets.cpp | 3 +- src/Planner/Planner.cpp | 35 +++++++++++++++++++++-- src/Planner/PlannerContext.cpp | 3 +- src/Storages/StorageDistributed.cpp | 14 ++------- 5 files changed, 45 insertions(+), 19 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index b2bfa648435..c454ad9f84f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4767,13 +4767,14 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto * table_node = in_second_argument->as(); auto * table_function_node = in_second_argument->as(); - if (table_node && dynamic_cast(table_node->getStorage().get()) != nullptr) + if (table_node) { - /// If table is already prepared set, we do not replace it with subquery + /// If table is already prepared set, we do not replace it with subquery. + /// If table is not a StorageSet, we'll create plan to build set in the Planner. } - else if (table_node || table_function_node) + else if (table_function_node) { - const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + const auto & storage_snapshot = table_function_node->getStorageSnapshot(); auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); size_t columns_to_select_size = columns_to_select.size(); diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 02069aad292..eb2b02c7ccb 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -67,7 +67,8 @@ public: planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); } else if (in_second_argument_node_type == QueryTreeNodeType::QUERY || - in_second_argument_node_type == QueryTreeNodeType::UNION) + in_second_argument_node_type == QueryTreeNodeType::UNION || + in_second_argument_node_type == QueryTreeNodeType::TABLE) { planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 7292f73e21c..5abf3ec5a80 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -43,6 +43,7 @@ #include #include +#include #include #include #include @@ -909,12 +910,42 @@ void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, if (!planner_set) continue; - if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) + auto subquery_to_execute = planner_set->getSubqueryNode(); + + if (planner_set->getSet().isCreated() || !subquery_to_execute) continue; + if (auto * table_node = subquery_to_execute->as()) + { + auto storage_snapshot = table_node->getStorageSnapshot(); + auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); + + size_t columns_to_select_size = columns_to_select.size(); + + auto column_nodes_to_select = std::make_shared(); + column_nodes_to_select->getNodes().reserve(columns_to_select_size); + + NamesAndTypes projection_columns; + projection_columns.reserve(columns_to_select_size); + + for (auto & column : columns_to_select) + { + column_nodes_to_select->getNodes().emplace_back(std::make_shared(column, subquery_to_execute)); + projection_columns.emplace_back(column.name, column.type); + } + + auto subquery_for_table = std::make_shared(Context::createCopy(planner_context->getQueryContext())); + subquery_for_table->setIsSubquery(true); + subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select); + subquery_for_table->getJoinTree() = std::move(subquery_to_execute); + subquery_for_table->resolveProjectionColumns(std::move(projection_columns)); + + subquery_to_execute = std::move(subquery_for_table); + } + auto subquery_options = select_query_options.subquery(); Planner subquery_planner( - planner_set->getSubqueryNode(), + subquery_to_execute, subquery_options, planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index a788a6cbc3c..708dab04d02 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -129,7 +129,8 @@ void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) auto node_type = subquery_node->getNodeType(); if (node_type != QueryTreeNodeType::QUERY && - node_type != QueryTreeNodeType::UNION) + node_type != QueryTreeNodeType::UNION && + node_type != QueryTreeNodeType::TABLE) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid node for set table expression. Expected query or union. Actual {}", subquery_node->formatASTForErrorMessage()); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 98c92541f67..f1fb4bb0c65 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -849,7 +849,7 @@ private: /** Execute subquery node and put result in mutable context temporary table. * Returns table node that is initialized with temporary table storage. */ -QueryTreeNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, +TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, ContextMutablePtr & mutable_context, size_t subquery_depth) { @@ -1019,17 +1019,9 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, planner_context->getMutableQueryContext(), global_in_or_join_node.subquery_depth); + temporary_table_expression_node->setAlias(temporary_table_expression_node->getTemporaryTableName()); - auto in_second_argument_query_node = std::make_shared(Context::createCopy(query_context)); - in_second_argument_query_node->setIsSubquery(true); - in_second_argument_query_node->getProjectionNode() = std::make_shared(); - in_second_argument_query_node->getProjection().getNodes() = { std::make_shared() }; - in_second_argument_query_node->getJoinTree() = std::move(temporary_table_expression_node); - - QueryAnalysisPass query_analysis_pass; - query_analysis_pass.run(in_second_argument_query_node, query_context); - - in_function_subquery_node = std::move(in_second_argument_query_node); + in_function_subquery_node = std::move(temporary_table_expression_node); } else { From eb7ae91d0144a895ef53c862c79d930fe2cbdbab Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 May 2023 15:00:11 +0000 Subject: [PATCH 0936/2223] Do not add alias to a temporary table --- src/Storages/StorageDistributed.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f1fb4bb0c65..43b1333413e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1019,7 +1019,6 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, planner_context->getMutableQueryContext(), global_in_or_join_node.subquery_depth); - temporary_table_expression_node->setAlias(temporary_table_expression_node->getTemporaryTableName()); in_function_subquery_node = std::move(temporary_table_expression_node); } From 99b35eca0789cd9e3a055697486532bc1d4403d1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 May 2023 15:05:48 +0000 Subject: [PATCH 0937/2223] Update reference files --- .../01561_clickhouse_client_stage.reference | 2 +- .../0_stateless/01591_window_functions.reference | 14 +++++++------- .../02048_clickhouse_local_stage.reference | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference index 00e0f4ddb2e..8a34751b071 100644 --- a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference @@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1 "foo" 1 execute: --allow_experimental_analyzer=1 --stage fetch_columns -"system.one.dummy_0" +"dummy_0" 0 execute: --allow_experimental_analyzer=1 --stage with_mergeable_state "1_UInt8" diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index b981a46b4fd..8939ea1111d 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -917,9 +917,9 @@ from ; Expression ((Project names + Projection)) Window (Window step for window \'\') - Window (Window step for window \'PARTITION BY t.p_0\') - Window (Window step for window \'PARTITION BY t.p_0 ORDER BY t.o_1 ASC\') - Sorting (Sorting for window \'PARTITION BY t.p_0 ORDER BY t.o_1 ASC\') + Window (Window step for window \'PARTITION BY p_0\') + Window (Window step for window \'PARTITION BY p_0 ORDER BY o_1 ASC\') + Sorting (Sorting for window \'PARTITION BY p_0 ORDER BY o_1 ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) ReadFromStorage (SystemNumbers) explain select @@ -930,11 +930,11 @@ from from numbers(16)) t ; Expression ((Project names + Projection)) - Window (Window step for window \'ORDER BY t.o_0 ASC, t.number_1 ASC\') - Sorting (Sorting for window \'ORDER BY t.o_0 ASC, t.number_1 ASC\') - Window (Window step for window \'ORDER BY t.number_1 ASC\') + Window (Window step for window \'ORDER BY o_0 ASC, number_1 ASC\') + Sorting (Sorting for window \'ORDER BY o_0 ASC, number_1 ASC\') + Window (Window step for window \'ORDER BY number_1 ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) [lifted up part]) - Sorting (Sorting for window \'ORDER BY t.number_1 ASC\') + Sorting (Sorting for window \'ORDER BY number_1 ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) ReadFromStorage (SystemNumbers) -- A test case for the sort comparator found by fuzzer. diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference index 00e0f4ddb2e..8a34751b071 100644 --- a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference @@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1 "foo" 1 execute: --allow_experimental_analyzer=1 --stage fetch_columns -"system.one.dummy_0" +"dummy_0" 0 execute: --allow_experimental_analyzer=1 --stage with_mergeable_state "1_UInt8" From ffd4f7f196db8f827ea3f8cc5edeb6b0a1082d72 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Jun 2023 14:39:01 +0000 Subject: [PATCH 0938/2223] Add better support for state disks --- src/Coordination/KeeperDispatcher.cpp | 2 -- src/Coordination/KeeperStateManager.cpp | 33 ++++++++++++++++++++----- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index a5b0f0b2c01..9d9df5c7f30 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -38,8 +38,6 @@ namespace ProfileEvents extern const Event MemoryAllocatorPurgeTimeMicroseconds; } -namespace fs = std::filesystem; - namespace DB { diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 33e62684d46..450fd04b61d 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -23,6 +23,8 @@ namespace ErrorCodes namespace { +const std::string copy_lock_file = "STATE_COPY_LOCK"; + bool isLocalhost(const std::string & hostname) { try @@ -324,7 +326,13 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state) auto disk = getStateFileDisk(); if (disk->exists(server_state_file_name)) - disk->moveFile(server_state_file_name, old_path); + { + auto buf = disk->writeFile(copy_lock_file); + buf->finalize(); + disk->copyFile(server_state_file_name, *disk, old_path); + disk->removeFile(copy_lock_file); + disk->removeFile(old_path); + } auto server_state_file = disk->writeFile(server_state_file_name); auto buf = state.serialize(); @@ -339,6 +347,7 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state) server_state_file->write(reinterpret_cast(buf->data_begin()), buf->size()); server_state_file->sync(); + server_state_file->finalize(); disk->removeFileIfExists(old_path); } @@ -417,13 +426,25 @@ nuraft::ptr KeeperStateManager::read_state() if (disk->exists(old_path)) { - auto state = try_read_file(old_path); - if (state) + if (disk->exists(copy_lock_file)) { - disk->moveFile(old_path, server_state_file_name); - return state; + disk->removeFile(old_path); + disk->removeFile(copy_lock_file); } - disk->removeFile(old_path); + else + { + auto state = try_read_file(old_path); + if (state) + { + disk->moveFile(old_path, server_state_file_name); + return state; + } + disk->removeFile(old_path); + } + } + else if (disk->exists(copy_lock_file)) + { + disk->removeFile(copy_lock_file); } LOG_WARNING(logger, "No state was read"); From 883350d5c221c22bef476a62857ae7e8f692dcbf Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 1 Jun 2023 14:51:03 +0000 Subject: [PATCH 0939/2223] Fix tests --- src/Formats/CapnProtoSchema.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/CapnProtoSchema.cpp b/src/Formats/CapnProtoSchema.cpp index f9ab88d39ed..559047a6f8d 100644 --- a/src/Formats/CapnProtoSchema.cpp +++ b/src/Formats/CapnProtoSchema.cpp @@ -43,7 +43,7 @@ capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaIn /// That's not good to determine the type of error by its description, but /// this is the only way to do it here, because kj doesn't specify the type of error. auto description = std::string_view(e.getDescription().cStr()); - if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos) + if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos || description.find("no such file") != String::npos) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); if (description.find("Parse error") != String::npos) From f99a7366da356f145ee3ff124c4be80b4f5e903b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 1 Jun 2023 16:56:18 +0200 Subject: [PATCH 0940/2223] Fix tests --- src/Storages/HDFS/StorageHDFS.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 08114ed3cba..79b7b65adb4 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -233,6 +233,7 @@ ColumnsDescription StorageHDFS::getTableStructureFromData( if (ctx->getSettingsRef().hdfs_skip_empty_files && path_with_info.info && path_with_info.info->size == 0) return read_buffer_iterator(columns); + first = false; auto compression = chooseCompressionMethod(path_with_info.path, compression_method); auto impl = std::make_unique(my_uri_without_path, path_with_info.path, ctx->getGlobalContext()->getConfigRef(), ctx->getReadSettings()); const Int64 zstd_window_log_max = ctx->getSettingsRef().zstd_window_log_max; From d9113a3b757841b9b956b1d38b479af61be37b72 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 1 Jun 2023 16:57:43 +0200 Subject: [PATCH 0941/2223] Style --- src/Storages/StorageS3.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index f3cad4de31a..29f0a747372 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -424,7 +424,6 @@ public: , bucket(bucket_) , query(query_) , virtual_header(virtual_header_) - { Strings all_keys = keys_; From 480db8622df10993a2df31e24255f96f58ef0094 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 1 Jun 2023 15:21:55 +0000 Subject: [PATCH 0942/2223] Always add table name while converting ColumnNode to AST --- src/Analyzer/ColumnNode.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 76f5697afb0..dd41522ac7d 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -108,8 +108,7 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const { column_identifier_parts = {column_source->getAlias()}; } - else if (auto * table_node = column_source->as(); - table_node && options.fully_qualified_identifiers) + else if (auto * table_node = column_source->as()) { if (!table_node->getTemporaryTableName().empty()) { @@ -118,7 +117,7 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const else { const auto & table_storage_id = table_node->getStorageID(); - if (table_storage_id.hasDatabase()) + if (table_storage_id.hasDatabase() && options.fully_qualified_identifiers) column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() }; else column_identifier_parts = { table_storage_id.getTableName() }; From bdb192cf2742d6f3059f621068bbc59d78124229 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 1 Jun 2023 15:43:37 +0000 Subject: [PATCH 0943/2223] Test right join in 02725_any_join_single_row, style code changes --- src/Interpreters/HashJoin.cpp | 14 ++++---- .../02725_any_join_single_row.reference | 3 ++ .../0_stateless/02725_any_join_single_row.sql | 33 ++++++++++++++----- 3 files changed, 34 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 436ecd382cd..9306c9b99eb 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -556,7 +556,7 @@ namespace return false; } - static ALWAYS_INLINE bool insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + static ALWAYS_INLINE void insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -567,10 +567,9 @@ namespace /// The first element of the list is stored in the value of the hash table, the rest in the pool. emplace_result.getMapped().insert({stored_block, i}, pool); } - return true; } - static ALWAYS_INLINE bool insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, + static ALWAYS_INLINE void insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, const IColumn & asof_column) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -580,7 +579,6 @@ namespace if (emplace_result.isInserted()) time_series_map = new (time_series_map) typename Map::mapped_type(createAsofRowRef(asof_type, join.getAsofInequality())); (*time_series_map)->insert(asof_column, stored_block, i); - return true; } }; @@ -599,7 +597,9 @@ namespace auto key_getter = createKeyGetter(key_columns, key_sizes); - is_inserted = false; + /// For ALL and ASOF join always insert values + is_inserted = !mapped_one || is_asof_join; + for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) @@ -615,11 +615,11 @@ namespace continue; if constexpr (is_asof_join) - is_inserted |= Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); + Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); else if constexpr (mapped_one) is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); else - is_inserted |= Inserter::insertAll(join, map, key_getter, stored_block, i, pool); + Inserter::insertAll(join, map, key_getter, stored_block, i, pool); } return map.getBufferSizeInCells(); } diff --git a/tests/queries/0_stateless/02725_any_join_single_row.reference b/tests/queries/0_stateless/02725_any_join_single_row.reference index 5d748fc6dbb..1e940bdc71e 100644 --- a/tests/queries/0_stateless/02725_any_join_single_row.reference +++ b/tests/queries/0_stateless/02725_any_join_single_row.reference @@ -1,3 +1,6 @@ Join(ANY, LEFT, key) 0 1 Join(ANY, LEFT, key) 1 1 Join(ANY, LEFT, key) 1 1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02725_any_join_single_row.sql b/tests/queries/0_stateless/02725_any_join_single_row.sql index 5e5c959c278..f7ddd2f402b 100644 --- a/tests/queries/0_stateless/02725_any_join_single_row.sql +++ b/tests/queries/0_stateless/02725_any_join_single_row.sql @@ -1,26 +1,41 @@ -CREATE TABLE join_test -( - `key` UInt64, - `value` UInt64 -) -ENGINE = Join(ANY, LEFT, key); +DROP TABLE IF EXISTS join_test; +DROP TABLE IF EXISTS join_test_right; +CREATE TABLE join_test ( `key` UInt64, `value` UInt64 ) ENGINE = Join(ANY, LEFT, key); + +-- Save table size before inserting any rows CREATE TEMPORARY TABLE initial_table_size AS SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); +-- Check that table size is less than 100K SELECT engine_full, total_rows, total_bytes < 100_000 FROM initial_table_size; INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); - +-- Save table size after inserting one row CREATE TEMPORARY TABLE one_row_table_size AS SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); +-- Check that table size is less than 2x after inserting one row SELECT engine_full, total_rows, total_bytes < 2 * (SELECT total_bytes FROM initial_table_size) FROM one_row_table_size; -INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); -INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +-- Insert some more rows with the same key INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(10_000); +-- Check that rows with the same key are not duplicated SELECT engine_full, total_rows, total_bytes == (SELECT total_bytes FROM one_row_table_size) FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); + +-- For RIGHT join we save all rows from the right table +CREATE TABLE join_test_right ( `key` UInt64, `value` UInt64 ) ENGINE = Join(ANY, RIGHT, key); + +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(1); +SELECT count() == 3 FROM (SELECT 1 as key) t1 ANY RIGHT JOIN join_test_right ON t1.key = join_test_right.key; +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(7); +SELECT count() == 10 FROM (SELECT 1 as key) t1 ANY RIGHT JOIN join_test_right ON t1.key = join_test_right.key; +SELECT count() == 10 FROM (SELECT 2 as key) t1 ANY RIGHT JOIN join_test_right ON t1.key = join_test_right.key; + +DROP TABLE IF EXISTS join_test; +DROP TABLE IF EXISTS join_test_right; From 4d65be4dbc30aaa764ddd8c888dca79fdc60bf07 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Jun 2023 18:34:35 +0200 Subject: [PATCH 0944/2223] Ignore QEMU logging regarding IFA --- .../0_stateless/01103_check_cpu_instructions_at_startup.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh index 9b6e1e05f2d..01047aeb9ab 100755 --- a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh +++ b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh @@ -19,7 +19,9 @@ fi function run_with_cpu() { - qemu-x86_64-static -cpu "$@" "$command" --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: + qemu-x86_64-static -cpu "$@" "$command" --query "SELECT 1" 2>&1 | \ + grep -v -F "warning: TCG doesn't support requested feature" | \ + grep -v -F 'Unknown host IFA type' ||: } run_with_cpu qemu64 From 02e986a9e7c1c33bf8818411de538c58af8a5198 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 1 Jun 2023 19:06:11 +0200 Subject: [PATCH 0945/2223] Revert "Revert "less logs in WriteBufferFromS3" (#50390)" This reverts commit 4d4112ff536f819514973dfd0cb8274cf044bb3e. --- src/IO/WriteBufferFromS3.cpp | 8 -------- src/IO/WriteBufferFromS3TaskTracker.cpp | 11 ----------- 2 files changed, 19 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 462cf2674c3..6992c3ea4ac 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -195,18 +195,14 @@ void WriteBufferFromS3::finalizeImpl() if (request_settings.check_objects_after_upload) { - LOG_TRACE(log, "Checking object {} exists after upload", key); S3::checkObjectExists(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage, "Immediately after upload"); - LOG_TRACE(log, "Checking object {} has size as expected {}", key, total_size); size_t actual_size = S3::getObjectSize(*client_ptr, bucket, key, {}, request_settings, /* for_disk_s3= */ write_settings.for_object_storage); if (actual_size != total_size) throw Exception( ErrorCodes::S3_ERROR, "Object {} from bucket {} has unexpected size {} after upload, expected size {}, it's a bug in S3 or S3 API.", key, bucket, actual_size, total_size); - - LOG_TRACE(log, "Object {} exists after upload", key); } } @@ -292,8 +288,6 @@ void WriteBufferFromS3::reallocateFirstBuffer() WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); chassert(offset() == 0); - - LOG_TRACE(log, "Reallocated first buffer with size {}. {}", memory.size(), getLogDetails()); } void WriteBufferFromS3::detachBuffer() @@ -316,8 +310,6 @@ void WriteBufferFromS3::allocateFirstBuffer() const auto size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), max_first_buffer); memory = Memory(size); WriteBuffer::set(memory.data(), memory.size()); - - LOG_TRACE(log, "Allocated first buffer with size {}. {}", memory.size(), getLogDetails()); } void WriteBufferFromS3::allocateBuffer() diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 7ae31044012..c10af5d0672 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -36,8 +36,6 @@ ThreadPoolCallbackRunner WriteBufferFromS3::TaskTracker::syncRunner() void WriteBufferFromS3::TaskTracker::waitAll() { - LOG_TEST(log, "waitAll, in queue {}", futures.size()); - /// Exceptions are propagated for (auto & future : futures) { @@ -51,8 +49,6 @@ void WriteBufferFromS3::TaskTracker::waitAll() void WriteBufferFromS3::TaskTracker::safeWaitAll() { - LOG_TEST(log, "safeWaitAll, wait in queue {}", futures.size()); - for (auto & future : futures) { if (future.valid()) @@ -76,7 +72,6 @@ void WriteBufferFromS3::TaskTracker::safeWaitAll() void WriteBufferFromS3::TaskTracker::waitIfAny() { - LOG_TEST(log, "waitIfAny, in queue {}", futures.size()); if (futures.empty()) return; @@ -101,8 +96,6 @@ void WriteBufferFromS3::TaskTracker::waitIfAny() watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); - - LOG_TEST(log, "waitIfAny ended, in queue {}", futures.size()); } void WriteBufferFromS3::TaskTracker::add(Callback && func) @@ -147,8 +140,6 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() if (!max_tasks_inflight) return; - LOG_TEST(log, "waitTilInflightShrink, in queue {}", futures.size()); - Stopwatch watch; /// Alternative approach is to wait until at least futures.size() - max_tasks_inflight element are finished @@ -171,8 +162,6 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() watch.stop(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); - - LOG_TEST(log, "waitTilInflightShrink ended, in queue {}", futures.size()); } bool WriteBufferFromS3::TaskTracker::isAsync() const From bd047ed9e0b580fc417c56239fa93340f6089388 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 1 Jun 2023 14:16:49 -0400 Subject: [PATCH 0946/2223] Update order-by.md add a note that ORDER BY sorting is case sensitive. closes https://github.com/ClickHouse/clickhouse-docs/issues/81 --- docs/en/sql-reference/statements/select/order-by.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 712395a0357..1da6c1d8468 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -5,7 +5,7 @@ sidebar_label: ORDER BY # ORDER BY Clause -The `ORDER BY` clause contains a list of expressions, which can each be attributed with `DESC` (descending) or `ASC` (ascending) modifier which determine the sorting direction. If the direction is not specified, `ASC` is assumed, so it’s usually omitted. The sorting direction applies to a single expression, not to the entire list. Example: `ORDER BY Visits DESC, SearchPhrase`. +The `ORDER BY` clause contains a list of expressions, which can each be attributed with `DESC` (descending) or `ASC` (ascending) modifier which determine the sorting direction. If the direction is not specified, `ASC` is assumed, so it’s usually omitted. The sorting direction applies to a single expression, not to the entire list. Example: `ORDER BY Visits DESC, SearchPhrase`. Sorting is case-sensitive. If you want to sort by column numbers instead of column names, enable the setting [enable_positional_arguments](../../../operations/settings/settings.md#enable-positional-arguments). From 985cd8fc8a0fdbe09a95132f1fb549825ece636f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 26 Apr 2023 21:20:19 +0200 Subject: [PATCH 0947/2223] Improve events logging --- tests/ci/terminate_runner_lambda/app.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index c9192417575..5a78c8c8e39 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -160,7 +160,7 @@ def get_candidates_to_be_killed(event_data: dict) -> Dict[str, List[str]]: def main(access_token: str, event: dict) -> Dict[str, List[str]]: - print("Got event", json.dumps(event, sort_keys=True, indent=4)) + print("Got event", json.dumps(event, sort_keys=True).replace("\n", "")) to_kill_by_zone = how_many_instances_to_kill(event) instances_by_zone = get_candidates_to_be_killed(event) @@ -177,7 +177,8 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]: total_to_kill += num_to_kill if num_to_kill > len(candidates): raise Exception( - f"Required to kill {num_to_kill}, but have only {len(candidates)} candidates in AV {zone}" + f"Required to kill {num_to_kill}, but have only {len(candidates)}" + f" candidates in AV {zone}" ) delete_for_av = [] # type: RunnerDescriptions @@ -207,7 +208,8 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]: if len(delete_for_av) < num_to_kill: print( - f"Checked all candidates for av {zone}, get to delete {len(delete_for_av)}, but still cannot get required {num_to_kill}" + f"Checked all candidates for av {zone}, get to delete " + f"{len(delete_for_av)}, but still cannot get required {num_to_kill}" ) instances_to_kill += [runner.name for runner in delete_for_av] From 7bf9089dcd34e0c29b1f951066136fec1d990372 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 27 Apr 2023 12:00:22 +0200 Subject: [PATCH 0948/2223] Increase access_token cached time --- tests/ci/terminate_runner_lambda/app.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index 5a78c8c8e39..390375a34e4 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -64,7 +64,7 @@ cached_token = CachedToken(0, "") def get_cached_access_token() -> str: - if time.time() - 500 < cached_token.time: + if time.time() - 550 < cached_token.time: return cached_token.value private_key, app_id = get_key_and_app_from_aws() payload = { From bf9b563e0b2c3b1933405aef34d904afc7ade57f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 27 Apr 2023 16:54:37 +0200 Subject: [PATCH 0949/2223] Improve caching mechanism for token, add cached instances --- tests/ci/terminate_runner_lambda/app.py | 40 ++++++++++++++++++++++++- 1 file changed, 39 insertions(+), 1 deletion(-) diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index 390375a34e4..bf883880c8d 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -58,14 +58,19 @@ def get_access_token(jwt_token: str, installation_id: int) -> str: class CachedToken: time: int value: str + updating: bool = False cached_token = CachedToken(0, "") def get_cached_access_token() -> str: - if time.time() - 550 < cached_token.time: + if time.time() - 550 < cached_token.time or cached_token.updating: return cached_token.value + # Indicate that the value is updating now, so the cached value can be + # used. The first setting and close-to-ttl are not counted as update + if cached_token.time != 0 or time.time() - 590 < cached_token.time: + cached_token.updating = True private_key, app_id = get_key_and_app_from_aws() payload = { "iat": int(time.time()) - 60, @@ -77,9 +82,42 @@ def get_cached_access_token() -> str: installation_id = get_installation_id(encoded_jwt) cached_token.time = int(time.time()) cached_token.value = get_access_token(encoded_jwt, installation_id) + cached_token.updating = False return cached_token.value +@dataclass +class CachedInstances: + time: int + value: dict + updating: bool = False + + +cached_instances = CachedInstances(0, {}) + + +def get_cached_instances() -> dict: + """return cached instances description with updating it once per five minutes""" + if time.time() - 250 < cached_instances.time or cached_instances.updating: + return cached_instances.value + # Indicate that the value is updating now, so the cached value can be + # used. The first setting and close-to-ttl are not counted as update + if cached_instances.time != 0 or time.time() - 300 < cached_instances.time: + cached_instances.updating = True + ec2_client = boto3.client("ec2") + instances_response = ec2_client.describe_instances( + Filters=[{"Name": "instance-state-name", "Values": ["running"]}] + ) + cached_instances.time = int(time.time()) + cached_instances.value = { + instance["InstanceId"]: instance + for reservation in instances_response["Reservations"] + for instance in reservation["Instances"] + } + cached_instances.updating = False + return cached_instances.value + + RunnerDescription = namedtuple( "RunnerDescription", ["id", "name", "tags", "offline", "busy"] ) From 855afb56f913d3d7ba6dcc7f992b59cfaf5cb02e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 27 Apr 2023 16:56:17 +0200 Subject: [PATCH 0950/2223] Get instances for the region to not kill a fresh one --- tests/ci/terminate_runner_lambda/app.py | 53 ++++++++++++++++++------- 1 file changed, 39 insertions(+), 14 deletions(-) diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index bf883880c8d..e0164bc58c0 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -198,11 +198,37 @@ def get_candidates_to_be_killed(event_data: dict) -> Dict[str, List[str]]: def main(access_token: str, event: dict) -> Dict[str, List[str]]: + start = time.time() print("Got event", json.dumps(event, sort_keys=True).replace("\n", "")) to_kill_by_zone = how_many_instances_to_kill(event) instances_by_zone = get_candidates_to_be_killed(event) + # Getting ASG and instances' descriptions from the API + # We don't kill instances that alive for less than 10 minutes, since they + # could be not in the GH active runners yet + print(f"Check other hosts from the same ASG {event['AutoScalingGroupName']}") + asg_client = boto3.client("autoscaling") + as_groups_response = asg_client.describe_auto_scaling_groups( + AutoScalingGroupNames=[event["AutoScalingGroupName"]] + ) + assert len(as_groups_response["AutoScalingGroups"]) == 1 + asg = as_groups_response["AutoScalingGroups"][0] + asg_instance_ids = [instance["InstanceId"] for instance in asg["Instances"]] + instance_descriptions = get_cached_instances() + # The instances launched less than 10 minutes ago + immune_ids = [ + instance["InstanceId"] + for instance in instance_descriptions.values() + if start - instance["LaunchTime"].timestamp() < 600 + ] + # if the ASG's instance ID not in instance_descriptions, it's most probably + # is not cached yet, so we must mark it as immuned + immune_ids.extend( + iid for iid in asg_instance_ids if iid not in instance_descriptions + ) + print("Time spent on the requests to AWS: ", time.time() - start) runners = list_runners(access_token) + runner_ids = set(runner.name for runner in runners) # We used to delete potential hosts to terminate from GitHub runners pool, # but the documentation states: # --- Returning an instance first in the response data does not guarantee its termination @@ -221,13 +247,17 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]: delete_for_av = [] # type: RunnerDescriptions for candidate in candidates: - if candidate not in set(runner.name for runner in runners): + if candidate in immune_ids: + print( + f"Candidate {candidate} started less than 10 minutes ago, won't touch a child" + ) + break + if candidate not in runner_ids: print( f"Candidate {candidate} was not in runners list, simply delete it" ) instances_to_kill.append(candidate) - - for candidate in candidates: + break if len(delete_for_av) + len(instances_to_kill) == num_to_kill: break if candidate in instances_to_kill: @@ -253,16 +283,11 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]: instances_to_kill += [runner.name for runner in delete_for_av] if len(instances_to_kill) < total_to_kill: - print(f"Check other hosts from the same ASG {event['AutoScalingGroupName']}") - client = boto3.client("autoscaling") - as_groups = client.describe_auto_scaling_groups( - AutoScalingGroupNames=[event["AutoScalingGroupName"]] - ) - assert len(as_groups["AutoScalingGroups"]) == 1 - asg = as_groups["AutoScalingGroups"][0] - for instance in asg["Instances"]: + for instance in asg_instance_ids: + if instance in immune_ids: + continue for runner in runners: - if runner.name == instance["InstanceId"] and not runner.busy: + if runner.name == instance and not runner.busy: print(f"Runner {runner.name} is not busy and can be deleted") instances_to_kill.append(runner.name) @@ -270,9 +295,9 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]: print("Got enough instances to kill") break - print("Got instances to kill: ", ", ".join(instances_to_kill)) response = {"InstanceIDs": instances_to_kill} - print(response) + print("Got instances to kill: ", response) + print("Time spent on the request: ", time.time() - start) return response From db029384110be42a55e8420e6a13091cd2dca164 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 4 May 2023 12:20:57 +0200 Subject: [PATCH 0951/2223] Do not count unfinished tasks with conclusion=None --- tests/ci/workflow_jobs_lambda/app.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/workflow_jobs_lambda/app.py b/tests/ci/workflow_jobs_lambda/app.py index bc8e1212be5..c4ce68c3f8e 100644 --- a/tests/ci/workflow_jobs_lambda/app.py +++ b/tests/ci/workflow_jobs_lambda/app.py @@ -257,6 +257,7 @@ def handler(event: dict, context: Any) -> dict: else: event_data = json.loads(event["body"]) + logging.info("Got the next raw event from the github hook: %s", event_data) repo = event_data["repository"] try: wf_job = event_data["workflow_job"] @@ -265,6 +266,9 @@ def handler(event: dict, context: Any) -> dict: logging.error("The event data: %s", event) logging.error("The context data: %s", context) + # We record only finished steps + steps = len([step for step in wf_job["steps"] if step["conclusion"] is not None]) + workflow_job = WorkflowJob( wf_job["id"], wf_job["run_id"], @@ -281,7 +285,7 @@ def handler(event: dict, context: Any) -> dict: wf_job["started_at"], wf_job["completed_at"] or "1970-01-01T00:00:00", # nullable date wf_job["name"], - len(wf_job["steps"]), + steps, wf_job["check_run_url"], wf_job["labels"], wf_job["runner_id"] or 0, # nullable From 27941b4d2603a1308978bdc32c6db4d6ed0da7ef Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 22 May 2023 17:57:12 +0200 Subject: [PATCH 0952/2223] Decrease the time window for autoscale_runners_lambda --- tests/ci/autoscale_runners_lambda/app.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index cbc9f4f8901..3fbab0d13dc 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -22,10 +22,13 @@ RUNNER_TYPE_LABELS = [ "style-checker-aarch64", ] +### Update comment on the change ### # 4 HOUR - is a balance to get the most precise values # - Our longest possible running check is around 5h on the worst scenario # - The long queue won't be wiped out and replaced, so the measurmenet is fine # - If the data is spoiled by something, we are from the bills perspective +# Changed it to 3 HOUR: in average we have 1h tasks, but p90 is around 2h. +# With 4h we have too much wasted computing time in case of issues with DB QUEUE_QUERY = f"""SELECT last_status AS status, toUInt32(count()) AS length, @@ -40,7 +43,7 @@ FROM FROM default.workflow_jobs WHERE has(labels, 'self-hosted') AND hasAny({RUNNER_TYPE_LABELS}, labels) - AND started_at > now() - INTERVAL 4 HOUR + AND started_at > now() - INTERVAL 3 HOUR GROUP BY ALL HAVING last_status IN ('in_progress', 'queued') ) From 484c91c47e3aa8365eec5ae29d03fb66a8372bb0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 22 May 2023 20:39:22 +0200 Subject: [PATCH 0953/2223] Add DRY_RUN and configurable PY_VERSION to lambda deployment --- .../ci/team_keys_lambda/build_and_deploy_archive.sh | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh index 4aee85c588a..f35d6456cd3 100644 --- a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh +++ b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh @@ -5,11 +5,17 @@ WORKDIR=$(dirname "$0") WORKDIR=$(readlink -f "${WORKDIR}") cd "$WORKDIR" -PY_VERSION=3.10 +# Do not deploy the lambda to AWS +DRY_RUN=${DRY_RUN:-} +# Python runtime to install dependencies +PY_VERSION=${PY_VERSION:-3.10} PY_EXEC="python${PY_VERSION}" +# Image to build the lambda zip package DOCKER_IMAGE="python:${PY_VERSION}-slim" LAMBDA_NAME=$(basename "$WORKDIR") +# Rename the_lambda_name directory to the-lambda-name lambda in AWS LAMBDA_NAME=${LAMBDA_NAME//_/-} +# The name of directory with lambda code PACKAGE=lambda-package rm -rf "$PACKAGE" "$PACKAGE".zip mkdir "$PACKAGE" @@ -28,4 +34,6 @@ if [ -f requirements.txt ]; then fi ( cd "$PACKAGE" && zip -9 -r ../"$PACKAGE".zip . ) -aws lambda update-function-code --function-name "$LAMBDA_NAME" --zip-file fileb://"$PACKAGE".zip +if [ -z "$DRY_RUN" ]; then + aws lambda update-function-code --function-name "$LAMBDA_NAME" --zip-file fileb://"$PACKAGE".zip +fi From 7f08f218d9ee71ba52d011cabce2faea7492c5ca Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 22 May 2023 23:07:35 +0200 Subject: [PATCH 0954/2223] Create lambda_shared package for lambdas --- .github/workflows/pull_request.yml | 5 ++ tests/ci/autoscale_runners_lambda/app.py | 77 +++---------------- .../autoscale_runners_lambda_test.py | 2 +- .../ci/autoscale_runners_lambda/lambda_shared | 1 + .../autoscale_runners_lambda/requirements.txt | 2 +- tests/ci/lambda_shared_package/.gitignore | 2 + .../lambda_shared/__init__.py | 74 ++++++++++++++++++ tests/ci/lambda_shared_package/pyproject.toml | 13 ++++ tests/ci/lambda_shared_package/setup.cfg | 8 ++ .../build_and_deploy_archive.sh | 11 +-- 10 files changed, 121 insertions(+), 74 deletions(-) rename tests/ci/{ => autoscale_runners_lambda}/autoscale_runners_lambda_test.py (98%) create mode 120000 tests/ci/autoscale_runners_lambda/lambda_shared create mode 100644 tests/ci/lambda_shared_package/.gitignore create mode 100644 tests/ci/lambda_shared_package/lambda_shared/__init__.py create mode 100644 tests/ci/lambda_shared_package/pyproject.toml create mode 100644 tests/ci/lambda_shared_package/setup.cfg diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 506ed451b6d..afc08f3e637 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -46,7 +46,12 @@ jobs: - name: Python unit tests run: | cd "$GITHUB_WORKSPACE/tests/ci" + echo "Testing the main ci directory" python3 -m unittest discover -s . -p '*_test.py' + for dir in *_lambda/; do + echo "Testing $dir" + python3 -m unittest discover -s "$dir" -p '*_test.py' + done DockerHubPushAarch64: needs: CheckLabels runs-on: [self-hosted, style-checker-aarch64] diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 3fbab0d13dc..ab09afb3aa8 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -2,25 +2,19 @@ """The lambda to decrease/increase ASG desired capacity based on current queue""" -import json import logging -import time from dataclasses import dataclass from pprint import pformat from typing import Any, List, Literal, Optional, Tuple import boto3 # type: ignore -import requests # type: ignore -RUNNER_TYPE_LABELS = [ - "builder", - "func-tester", - "func-tester-aarch64", - "fuzzer-unit-tester", - "stress-tester", - "style-checker", - "style-checker-aarch64", -] +from lambda_shared import ( + CHException, + ClickHouseHelper, + RUNNER_TYPE_LABELS, + get_parameter_from_ssm, +) ### Update comment on the change ### # 4 HOUR - is a balance to get the most precise values @@ -74,61 +68,7 @@ def get_scales(runner_type: str) -> Tuple[int, int]: return scale_down, scale_up -### VENDORING -def get_parameter_from_ssm(name, decrypt=True, client=None): - if not client: - client = boto3.client("ssm", region_name="us-east-1") - return client.get_parameter(Name=name, WithDecryption=decrypt)["Parameter"]["Value"] - - -class CHException(Exception): - pass - - -class ClickHouseHelper: - def __init__( - self, - url: Optional[str] = None, - user: Optional[str] = None, - password: Optional[str] = None, - ): - self.url = url - self.auth = {} - if user: - self.auth["X-ClickHouse-User"] = user - if password: - self.auth["X-ClickHouse-Key"] = password - - def _select_and_get_json_each_row(self, db, query): - params = { - "database": db, - "query": query, - "default_format": "JSONEachRow", - } - for i in range(5): - response = None - try: - response = requests.get(self.url, params=params, headers=self.auth) - response.raise_for_status() - return response.text - except Exception as ex: - logging.warning("Cannot fetch data with exception %s", str(ex)) - if response: - logging.warning("Reponse text %s", response.text) - time.sleep(0.1 * i) - - raise CHException("Cannot fetch data from clickhouse") - - def select_json_each_row(self, db, query): - text = self._select_and_get_json_each_row(db, query) - result = [] - for line in text.split("\n"): - if line: - result.append(json.loads(line)) - return result - - -CH_CLIENT = ClickHouseHelper(get_parameter_from_ssm("clickhouse-test-stat-url"), "play") +CH_CLIENT = None # type: Optional[ClickHouseHelper] def set_capacity( @@ -222,6 +162,9 @@ def main(dry_run: bool = True) -> None: asg_client = boto3.client("autoscaling") try: global CH_CLIENT + CH_CLIENT = CH_CLIENT or ClickHouseHelper( + get_parameter_from_ssm("clickhouse-test-stat-url"), "play" + ) queues = CH_CLIENT.select_json_each_row("default", QUEUE_QUERY) except CHException as ex: logging.exception( diff --git a/tests/ci/autoscale_runners_lambda_test.py b/tests/ci/autoscale_runners_lambda/autoscale_runners_lambda_test.py similarity index 98% rename from tests/ci/autoscale_runners_lambda_test.py rename to tests/ci/autoscale_runners_lambda/autoscale_runners_lambda_test.py index 8e3828f51c0..6772e33374c 100644 --- a/tests/ci/autoscale_runners_lambda_test.py +++ b/tests/ci/autoscale_runners_lambda/autoscale_runners_lambda_test.py @@ -4,7 +4,7 @@ import unittest from dataclasses import dataclass from typing import Any, List -from autoscale_runners_lambda.app import set_capacity, Queue +from app import set_capacity, Queue @dataclass diff --git a/tests/ci/autoscale_runners_lambda/lambda_shared b/tests/ci/autoscale_runners_lambda/lambda_shared new file mode 120000 index 00000000000..ba86e090f6c --- /dev/null +++ b/tests/ci/autoscale_runners_lambda/lambda_shared @@ -0,0 +1 @@ +../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/autoscale_runners_lambda/requirements.txt b/tests/ci/autoscale_runners_lambda/requirements.txt index 3bcbe2dfd07..098e04a9798 100644 --- a/tests/ci/autoscale_runners_lambda/requirements.txt +++ b/tests/ci/autoscale_runners_lambda/requirements.txt @@ -1 +1 @@ -requests<2.30 +../lambda_shared_package diff --git a/tests/ci/lambda_shared_package/.gitignore b/tests/ci/lambda_shared_package/.gitignore new file mode 100644 index 00000000000..59d52651e06 --- /dev/null +++ b/tests/ci/lambda_shared_package/.gitignore @@ -0,0 +1,2 @@ +build +*.egg-info diff --git a/tests/ci/lambda_shared_package/lambda_shared/__init__.py b/tests/ci/lambda_shared_package/lambda_shared/__init__.py new file mode 100644 index 00000000000..c5ae4df9e17 --- /dev/null +++ b/tests/ci/lambda_shared_package/lambda_shared/__init__.py @@ -0,0 +1,74 @@ +"""The shared code and types for all our CI lambdas +It exists as __init__.py and lambda_shared/__init__.py to work both in local and venv""" + +import json +import logging +import time +from typing import List, Optional + +import boto3 # type: ignore +import requests # type: ignore + +RUNNER_TYPE_LABELS = [ + "builder", + "func-tester", + "func-tester-aarch64", + "fuzzer-unit-tester", + "stress-tester", + "style-checker", + "style-checker-aarch64", +] + + +### VENDORING +def get_parameter_from_ssm(name, decrypt=True, client=None): + if not client: + client = boto3.client("ssm", region_name="us-east-1") + return client.get_parameter(Name=name, WithDecryption=decrypt)["Parameter"]["Value"] + + +class CHException(Exception): + pass + + +class ClickHouseHelper: + def __init__( + self, + url: Optional[str] = None, + user: Optional[str] = None, + password: Optional[str] = None, + ): + self.url = url + self.auth = {} + if user: + self.auth["X-ClickHouse-User"] = user + if password: + self.auth["X-ClickHouse-Key"] = password + + def _select_and_get_json_each_row(self, db: str, query: str) -> str: + params = { + "database": db, + "query": query, + "default_format": "JSONEachRow", + } + for i in range(5): + response = None + try: + response = requests.get(self.url, params=params, headers=self.auth) + response.raise_for_status() + return response.text # type: ignore + except Exception as ex: + logging.warning("Cannot fetch data with exception %s", str(ex)) + if response: + logging.warning("Reponse text %s", response.text) + time.sleep(0.1 * i) + + raise CHException("Cannot fetch data from clickhouse") + + def select_json_each_row(self, db: str, query: str) -> List[dict]: # type: ignore + text = self._select_and_get_json_each_row(db, query) + result = [] + for line in text.split("\n"): + if line: + result.append(json.loads(line)) + return result diff --git a/tests/ci/lambda_shared_package/pyproject.toml b/tests/ci/lambda_shared_package/pyproject.toml new file mode 100644 index 00000000000..8b4b0a80948 --- /dev/null +++ b/tests/ci/lambda_shared_package/pyproject.toml @@ -0,0 +1,13 @@ +[build-system] +requires = ["setuptools"] +build-backend = "setuptools.build_meta" + +[project] +name = "lambda_shared" +version = "0.0.1" +dependencies = [ + "requests < 2.30", +] + +[tool.distutils.bdist_wheel] +universal = true diff --git a/tests/ci/lambda_shared_package/setup.cfg b/tests/ci/lambda_shared_package/setup.cfg new file mode 100644 index 00000000000..744280ae41b --- /dev/null +++ b/tests/ci/lambda_shared_package/setup.cfg @@ -0,0 +1,8 @@ +### This file exists for clear builds in docker ### +# without it the `build` directory wouldn't be # +# updated on the fly and will require manual clean # +[build] +build_base = /tmp/lambda_shared + +[egg_info] +egg_base = /tmp/ diff --git a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh index f35d6456cd3..89a2d514965 100644 --- a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh +++ b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh @@ -3,6 +3,7 @@ set -xeo pipefail WORKDIR=$(dirname "$0") WORKDIR=$(readlink -f "${WORKDIR}") +DIR_NAME=$(basename "$WORKDIR") cd "$WORKDIR" # Do not deploy the lambda to AWS @@ -12,9 +13,8 @@ PY_VERSION=${PY_VERSION:-3.10} PY_EXEC="python${PY_VERSION}" # Image to build the lambda zip package DOCKER_IMAGE="python:${PY_VERSION}-slim" -LAMBDA_NAME=$(basename "$WORKDIR") # Rename the_lambda_name directory to the-lambda-name lambda in AWS -LAMBDA_NAME=${LAMBDA_NAME//_/-} +LAMBDA_NAME=${DIR_NAME//_/-} # The name of directory with lambda code PACKAGE=lambda-package rm -rf "$PACKAGE" "$PACKAGE".zip @@ -23,8 +23,9 @@ cp app.py "$PACKAGE" if [ -f requirements.txt ]; then VENV=lambda-venv rm -rf "$VENV" lambda-package.zip - docker run --rm --user="${UID}" --volume="${WORKDIR}:/lambda" --workdir="/lambda" "${DOCKER_IMAGE}" \ - /bin/bash -c " + docker run --rm --user="${UID}" -e HOME=/tmp \ + --volume="${WORKDIR}/..:/ci" --workdir="/ci/${DIR_NAME}" "${DOCKER_IMAGE}" \ + /bin/bash -exc " '$PY_EXEC' -m venv '$VENV' && source '$VENV/bin/activate' && pip install -r requirements.txt @@ -35,5 +36,5 @@ fi ( cd "$PACKAGE" && zip -9 -r ../"$PACKAGE".zip . ) if [ -z "$DRY_RUN" ]; then - aws lambda update-function-code --function-name "$LAMBDA_NAME" --zip-file fileb://"$PACKAGE".zip + aws lambda update-function-code --function-name "$LAMBDA_NAME" --zip-file fileb://"$WORKDIR/$PACKAGE".zip fi From 0fa6a8416148a9f47f8d5c4e6a6bdb67992d2cd4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 23 May 2023 12:55:23 +0200 Subject: [PATCH 0955/2223] Move the stuff related to runners to lambda_shared --- tests/ci/ci_runners_metrics_lambda/app.py | 68 ++----------------- .../ci_runners_metrics_lambda/lambda_shared | 1 + .../requirements.txt | 2 +- .../lambda_shared/__init__.py | 67 ++++++++++++++++-- tests/ci/terminate_runner_lambda/app.py | 55 +-------------- .../ci/terminate_runner_lambda/lambda_shared | 1 + .../terminate_runner_lambda/requirements.txt | 2 +- 7 files changed, 76 insertions(+), 120 deletions(-) create mode 120000 tests/ci/ci_runners_metrics_lambda/lambda_shared create mode 120000 tests/ci/terminate_runner_lambda/lambda_shared diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py index 341e1b674ec..a12143752a1 100644 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ b/tests/ci/ci_runners_metrics_lambda/app.py @@ -10,7 +10,6 @@ import argparse import sys import json import time -from collections import namedtuple from datetime import datetime from typing import Dict, List, Tuple @@ -19,21 +18,14 @@ import requests # type: ignore import boto3 # type: ignore from botocore.exceptions import ClientError # type: ignore -UNIVERSAL_LABEL = "universal" -RUNNER_TYPE_LABELS = [ - "builder", - "func-tester", - "func-tester-aarch64", - "fuzzer-unit-tester", - "stress-tester", - "style-checker", - "style-checker-aarch64", -] - -RunnerDescription = namedtuple( - "RunnerDescription", ["id", "name", "tags", "offline", "busy"] +from lambda_shared import ( + RUNNER_TYPE_LABELS, + RunnerDescription, + RunnerDescriptions, + list_runners, ) -RunnerDescriptions = List[RunnerDescription] + +UNIVERSAL_LABEL = "universal" def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions: @@ -193,52 +185,6 @@ def get_access_token(jwt_token: str, installation_id: int) -> str: return data["token"] # type: ignore -def list_runners(access_token: str) -> RunnerDescriptions: - headers = { - "Authorization": f"token {access_token}", - "Accept": "application/vnd.github.v3+json", - } - per_page = 100 - response = requests.get( - f"https://api.github.com/orgs/ClickHouse/actions/runners?per_page={per_page}", - headers=headers, - ) - response.raise_for_status() - data = response.json() - total_runners = data["total_count"] - print("Expected total runners", total_runners) - runners = data["runners"] - - # round to 0 for 0, 1 for 1..100, but to 2 for 101..200 - total_pages = (total_runners - 1) // per_page + 1 - - print("Total pages", total_pages) - for i in range(2, total_pages + 1): - response = requests.get( - "https://api.github.com/orgs/ClickHouse/actions/runners" - f"?page={i}&per_page={per_page}", - headers=headers, - ) - response.raise_for_status() - data = response.json() - runners += data["runners"] - - print("Total runners", len(runners)) - result = [] - for runner in runners: - tags = [tag["name"] for tag in runner["labels"]] - desc = RunnerDescription( - id=runner["id"], - name=runner["name"], - tags=tags, - offline=runner["status"] == "offline", - busy=runner["busy"], - ) - result.append(desc) - - return result - - def group_runners_by_tag( listed_runners: RunnerDescriptions, ) -> Dict[str, RunnerDescriptions]: diff --git a/tests/ci/ci_runners_metrics_lambda/lambda_shared b/tests/ci/ci_runners_metrics_lambda/lambda_shared new file mode 120000 index 00000000000..ba86e090f6c --- /dev/null +++ b/tests/ci/ci_runners_metrics_lambda/lambda_shared @@ -0,0 +1 @@ +../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/ci_runners_metrics_lambda/requirements.txt b/tests/ci/ci_runners_metrics_lambda/requirements.txt index 98be09ab232..e99dee1743c 100644 --- a/tests/ci/ci_runners_metrics_lambda/requirements.txt +++ b/tests/ci/ci_runners_metrics_lambda/requirements.txt @@ -1,3 +1,3 @@ -requests<2.30 +../lambda_shared_package PyJWT cryptography<38 diff --git a/tests/ci/lambda_shared_package/lambda_shared/__init__.py b/tests/ci/lambda_shared_package/lambda_shared/__init__.py index c5ae4df9e17..fe52f98d5f6 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/__init__.py +++ b/tests/ci/lambda_shared_package/lambda_shared/__init__.py @@ -4,7 +4,8 @@ It exists as __init__.py and lambda_shared/__init__.py to work both in local and import json import logging import time -from typing import List, Optional +from collections import namedtuple +from typing import Any, List, Optional import boto3 # type: ignore import requests # type: ignore @@ -21,10 +22,14 @@ RUNNER_TYPE_LABELS = [ ### VENDORING -def get_parameter_from_ssm(name, decrypt=True, client=None): +def get_parameter_from_ssm( + name: str, decrypt: bool = True, client: Optional[Any] = None +) -> str: if not client: client = boto3.client("ssm", region_name="us-east-1") - return client.get_parameter(Name=name, WithDecryption=decrypt)["Parameter"]["Value"] + return client.get_parameter(Name=name, WithDecryption=decrypt)[ # type: ignore + "Parameter" + ]["Value"] class CHException(Exception): @@ -65,10 +70,64 @@ class ClickHouseHelper: raise CHException("Cannot fetch data from clickhouse") - def select_json_each_row(self, db: str, query: str) -> List[dict]: # type: ignore + def select_json_each_row(self, db: str, query: str) -> List[dict]: text = self._select_and_get_json_each_row(db, query) result = [] for line in text.split("\n"): if line: result.append(json.loads(line)) return result + + +### Runners + +RunnerDescription = namedtuple( + "RunnerDescription", ["id", "name", "tags", "offline", "busy"] +) +RunnerDescriptions = List[RunnerDescription] + + +def list_runners(access_token: str) -> RunnerDescriptions: + headers = { + "Authorization": f"token {access_token}", + "Accept": "application/vnd.github.v3+json", + } + per_page = 100 + response = requests.get( + f"https://api.github.com/orgs/ClickHouse/actions/runners?per_page={per_page}", + headers=headers, + ) + response.raise_for_status() + data = response.json() + total_runners = data["total_count"] + print("Expected total runners", total_runners) + runners = data["runners"] + + # round to 0 for 0, 1 for 1..100, but to 2 for 101..200 + total_pages = (total_runners - 1) // per_page + 1 + + print("Total pages", total_pages) + for i in range(2, total_pages + 1): + response = requests.get( + "https://api.github.com/orgs/ClickHouse/actions/runners" + f"?page={i}&per_page={per_page}", + headers=headers, + ) + response.raise_for_status() + data = response.json() + runners += data["runners"] + + print("Total runners", len(runners)) + result = [] + for runner in runners: + tags = [tag["name"] for tag in runner["labels"]] + desc = RunnerDescription( + id=runner["id"], + name=runner["name"], + tags=tags, + offline=runner["status"] == "offline", + busy=runner["busy"], + ) + result.append(desc) + + return result diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index e0164bc58c0..5799a498d5a 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -4,7 +4,6 @@ import argparse import json import sys import time -from collections import namedtuple from dataclasses import dataclass from typing import Any, Dict, List, Tuple @@ -12,6 +11,8 @@ import boto3 # type: ignore import requests # type: ignore import jwt +from lambda_shared import RunnerDescriptions, list_runners + def get_key_and_app_from_aws() -> Tuple[str, int]: secret_name = "clickhouse_github_secret_key" @@ -118,58 +119,6 @@ def get_cached_instances() -> dict: return cached_instances.value -RunnerDescription = namedtuple( - "RunnerDescription", ["id", "name", "tags", "offline", "busy"] -) -RunnerDescriptions = List[RunnerDescription] - - -def list_runners(access_token: str) -> RunnerDescriptions: - headers = { - "Authorization": f"token {access_token}", - "Accept": "application/vnd.github.v3+json", - } - per_page = 100 - response = requests.get( - f"https://api.github.com/orgs/ClickHouse/actions/runners?per_page={per_page}", - headers=headers, - ) - response.raise_for_status() - data = response.json() - total_runners = data["total_count"] - print("Expected total runners", total_runners) - runners = data["runners"] - - # round to 0 for 0, 1 for 1..100, but to 2 for 101..200 - total_pages = (total_runners - 1) // per_page + 1 - - print("Total pages", total_pages) - for i in range(2, total_pages + 1): - response = requests.get( - "https://api.github.com/orgs/ClickHouse/actions/runners" - f"?page={i}&per_page={per_page}", - headers=headers, - ) - response.raise_for_status() - data = response.json() - runners += data["runners"] - - print("Total runners", len(runners)) - result = [] - for runner in runners: - tags = [tag["name"] for tag in runner["labels"]] - desc = RunnerDescription( - id=runner["id"], - name=runner["name"], - tags=tags, - offline=runner["status"] == "offline", - busy=runner["busy"], - ) - result.append(desc) - - return result - - def how_many_instances_to_kill(event_data: dict) -> Dict[str, int]: data_array = event_data["CapacityToTerminate"] to_kill_by_zone = {} # type: Dict[str, int] diff --git a/tests/ci/terminate_runner_lambda/lambda_shared b/tests/ci/terminate_runner_lambda/lambda_shared new file mode 120000 index 00000000000..ba86e090f6c --- /dev/null +++ b/tests/ci/terminate_runner_lambda/lambda_shared @@ -0,0 +1 @@ +../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/terminate_runner_lambda/requirements.txt b/tests/ci/terminate_runner_lambda/requirements.txt index 98be09ab232..e99dee1743c 100644 --- a/tests/ci/terminate_runner_lambda/requirements.txt +++ b/tests/ci/terminate_runner_lambda/requirements.txt @@ -1,3 +1,3 @@ -requests<2.30 +../lambda_shared_package PyJWT cryptography<38 From acb9531ebf829a5c11bbeff6661e8d2122334ee6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 23 May 2023 18:47:19 +0200 Subject: [PATCH 0956/2223] Unify and put GH access token to the lambda_shared --- .../cancel_and_rerun_workflow_lambda/app.py | 176 +---------------- .../lambda_shared | 1 + .../requirements.txt | 4 +- tests/ci/ci_runners_metrics_lambda/app.py | 70 ++----- .../requirements.txt | 3 +- tests/ci/lambda_shared_package/__init__.py | 0 .../lambda_shared_package/lambda_shared/pr.py | 184 ++++++++++++++++++ .../lambda_shared/token.py | 90 +++++++++ tests/ci/lambda_shared_package/pyproject.toml | 10 + tests/ci/run_check.py | 8 +- tests/ci/runner_token_rotation_lambda/app.py | 61 +----- .../lambda_shared | 1 + .../requirements.txt | 4 +- tests/ci/team_keys_lambda/app.py | 4 +- tests/ci/team_keys_lambda/lambda_shared | 1 + tests/ci/team_keys_lambda/requirements.txt | 2 +- tests/ci/terminate_runner_lambda/app.py | 92 +-------- .../terminate_runner_lambda/requirements.txt | 4 +- tests/ci/workflow_approve_rerun_lambda/app.py | 122 +----------- .../lambda_shared | 1 + .../requirements.txt | 4 +- tests/ci/workflow_jobs_lambda/lambda_shared | 1 + 22 files changed, 332 insertions(+), 511 deletions(-) create mode 120000 tests/ci/cancel_and_rerun_workflow_lambda/lambda_shared create mode 100644 tests/ci/lambda_shared_package/__init__.py create mode 100644 tests/ci/lambda_shared_package/lambda_shared/pr.py create mode 100644 tests/ci/lambda_shared_package/lambda_shared/token.py create mode 120000 tests/ci/runner_token_rotation_lambda/lambda_shared create mode 120000 tests/ci/team_keys_lambda/lambda_shared create mode 120000 tests/ci/workflow_approve_rerun_lambda/lambda_shared create mode 120000 tests/ci/workflow_jobs_lambda/lambda_shared diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 54c87fbcfa5..250655ddeb2 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -9,9 +9,10 @@ import json import re import time -import jwt import requests # type: ignore -import boto3 # type: ignore + +from lambda_shared.pr import CATEGORY_TO_LABEL, check_pr_description +from lambda_shared.token import get_cached_access_token NEED_RERUN_ON_EDITED = { @@ -27,123 +28,6 @@ MAX_RETRY = 5 DEBUG_INFO = {} # type: Dict[str, Any] -# Descriptions are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there -# updated accordingly -# The following lists are append only, try to avoid editing them -# They still could be cleaned out after the decent time though. -LABELS = { - "pr-backward-incompatible": ["Backward Incompatible Change"], - "pr-bugfix": [ - "Bug Fix", - "Bug Fix (user-visible misbehavior in an official stable release)", - "Bug Fix (user-visible misbehaviour in official stable or prestable release)", - "Bug Fix (user-visible misbehavior in official stable or prestable release)", - ], - "pr-build": [ - "Build/Testing/Packaging Improvement", - "Build Improvement", - "Build/Testing Improvement", - "Build", - "Packaging Improvement", - ], - "pr-documentation": [ - "Documentation (changelog entry is not required)", - "Documentation", - ], - "pr-feature": ["New Feature"], - "pr-improvement": ["Improvement"], - "pr-not-for-changelog": [ - "Not for changelog (changelog entry is not required)", - "Not for changelog", - ], - "pr-performance": ["Performance Improvement"], -} - -CATEGORY_TO_LABEL = {c: lb for lb, categories in LABELS.items() for c in categories} - - -def check_pr_description(pr_body: str) -> Tuple[str, str]: - """The function checks the body to being properly formatted according to - .github/PULL_REQUEST_TEMPLATE.md, if the first returned string is not empty, - then there is an error.""" - lines = list(map(lambda x: x.strip(), pr_body.split("\n") if pr_body else [])) - lines = [re.sub(r"\s+", " ", line) for line in lines] - - # Check if body contains "Reverts ClickHouse/ClickHouse#36337" - if [ - True - for line in lines - if re.match(r"\AReverts {GITHUB_REPOSITORY}#[\d]+\Z", line) - ]: - return "", LABELS["pr-not-for-changelog"][0] - - category = "" - entry = "" - description_error = "" - - i = 0 - while i < len(lines): - if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]): - i += 1 - if i >= len(lines): - break - # Can have one empty line between header and the category - # itself. Filter it out. - if not lines[i]: - i += 1 - if i >= len(lines): - break - category = re.sub(r"^[-*\s]*", "", lines[i]) - i += 1 - - # Should not have more than one category. Require empty line - # after the first found category. - if i >= len(lines): - break - if lines[i]: - second_category = re.sub(r"^[-*\s]*", "", lines[i]) - description_error = ( - "More than one changelog category specified: " - f"'{category}', '{second_category}'" - ) - return description_error, category - - elif re.match( - r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i] - ): - i += 1 - # Can have one empty line between header and the entry itself. - # Filter it out. - if i < len(lines) and not lines[i]: - i += 1 - # All following lines until empty one are the changelog entry. - entry_lines = [] - while i < len(lines) and lines[i]: - entry_lines.append(lines[i]) - i += 1 - entry = " ".join(entry_lines) - # Don't accept changelog entries like '...'. - entry = re.sub(r"[#>*_.\- ]", "", entry) - # Don't accept changelog entries like 'Close #12345'. - entry = re.sub(r"^[\w\-\s]{0,10}#?\d{5,6}\.?$", "", entry) - else: - i += 1 - - if not category: - description_error = "Changelog category is empty" - # Filter out the PR categories that are not for changelog. - elif re.match( - r"(?i)doc|((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)", - category, - ): - pass # to not check the rest of the conditions - elif category not in CATEGORY_TO_LABEL: - description_error, category = f"Category '{category}' is not valid", "" - elif not entry: - description_error = f"Changelog entry required for category '{category}'" - - return description_error, category - class Worker(Thread): def __init__( @@ -166,58 +50,6 @@ class Worker(Thread): self.queue.task_done() -def get_installation_id(jwt_token): - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.get("https://api.github.com/app/installations", headers=headers) - response.raise_for_status() - data = response.json() - for installation in data: - if installation["account"]["login"] == "ClickHouse": - installation_id = installation["id"] - return installation_id - - -def get_access_token(jwt_token, installation_id): - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.post( - f"https://api.github.com/app/installations/{installation_id}/access_tokens", - headers=headers, - ) - response.raise_for_status() - data = response.json() - return data["token"] - - -def get_key_and_app_from_aws(): - secret_name = "clickhouse_github_secret_key" - session = boto3.session.Session() - client = session.client( - service_name="secretsmanager", - ) - get_secret_value_response = client.get_secret_value(SecretId=secret_name) - data = json.loads(get_secret_value_response["SecretString"]) - return data["clickhouse-app-key"], int(data["clickhouse-app-id"]) - - -def get_token_from_aws(): - private_key, app_id = get_key_and_app_from_aws() - payload = { - "iat": int(time.time()) - 60, - "exp": int(time.time()) + (10 * 60), - "iss": app_id, - } - - encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256") - installation_id = get_installation_id(encoded_jwt) - return get_access_token(encoded_jwt, installation_id) - - def _exec_get_with_retry(url: str, token: str) -> dict: headers = {"Authorization": f"token {token}"} for i in range(MAX_RETRY): @@ -407,7 +239,7 @@ def exec_workflow_url(urls_to_post, token): def main(event): - token = get_token_from_aws() + token = get_cached_access_token() DEBUG_INFO["event"] = event if event["isBase64Encoded"]: event_data = json.loads(b64decode(event["body"])) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/lambda_shared b/tests/ci/cancel_and_rerun_workflow_lambda/lambda_shared new file mode 120000 index 00000000000..ba86e090f6c --- /dev/null +++ b/tests/ci/cancel_and_rerun_workflow_lambda/lambda_shared @@ -0,0 +1 @@ +../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/requirements.txt b/tests/ci/cancel_and_rerun_workflow_lambda/requirements.txt index 98be09ab232..4cb3fba0f7b 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/requirements.txt +++ b/tests/ci/cancel_and_rerun_workflow_lambda/requirements.txt @@ -1,3 +1 @@ -requests<2.30 -PyJWT -cryptography<38 +../lambda_shared_package[token] diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py index a12143752a1..dc128dea739 100644 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ b/tests/ci/ci_runners_metrics_lambda/app.py @@ -8,12 +8,9 @@ Lambda function to: import argparse import sys -import json -import time from datetime import datetime -from typing import Dict, List, Tuple +from typing import Dict, List -import jwt import requests # type: ignore import boto3 # type: ignore from botocore.exceptions import ClientError # type: ignore @@ -24,6 +21,11 @@ from lambda_shared import ( RunnerDescriptions, list_runners, ) +from lambda_shared.token import ( + get_cached_access_token, + get_key_and_app_from_aws, + get_access_token_by_key_app, +) UNIVERSAL_LABEL = "universal" @@ -139,50 +141,8 @@ def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[dict]: return lost_instances -def get_key_and_app_from_aws() -> Tuple[str, int]: - secret_name = "clickhouse_github_secret_key" - session = boto3.session.Session() - client = session.client( - service_name="secretsmanager", - ) - get_secret_value_response = client.get_secret_value(SecretId=secret_name) - data = json.loads(get_secret_value_response["SecretString"]) - return data["clickhouse-app-key"], int(data["clickhouse-app-id"]) - - def handler(event, context): - private_key, app_id = get_key_and_app_from_aws() - main(private_key, app_id, True, True) - - -def get_installation_id(jwt_token: str) -> int: - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.get("https://api.github.com/app/installations", headers=headers) - response.raise_for_status() - data = response.json() - for installation in data: - if installation["account"]["login"] == "ClickHouse": - installation_id = installation["id"] - break - - return installation_id # type: ignore - - -def get_access_token(jwt_token: str, installation_id: int) -> str: - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.post( - f"https://api.github.com/app/installations/{installation_id}/access_tokens", - headers=headers, - ) - response.raise_for_status() - data = response.json() - return data["token"] # type: ignore + main(get_cached_access_token(), True, True) def group_runners_by_tag( @@ -273,20 +233,10 @@ def delete_runner(access_token: str, runner: RunnerDescription) -> bool: def main( - github_secret_key: str, - github_app_id: int, + access_token: str, push_to_cloudwatch: bool, delete_offline_runners: bool, ) -> None: - payload = { - "iat": int(time.time()) - 60, - "exp": int(time.time()) + (10 * 60), - "iss": github_app_id, - } - - encoded_jwt = jwt.encode(payload, github_secret_key, algorithm="RS256") - installation_id = get_installation_id(encoded_jwt) - access_token = get_access_token(encoded_jwt, installation_id) gh_runners = list_runners(access_token) grouped_runners = group_runners_by_tag(gh_runners) for group, group_runners in grouped_runners.items(): @@ -354,4 +304,6 @@ if __name__ == "__main__": print("Attempt to get key and id from AWS secret manager") private_key, args.app_id = get_key_and_app_from_aws() - main(private_key, args.app_id, args.push_to_cloudwatch, args.delete_offline) + token = get_access_token_by_key_app(private_key, args.app_id) + + main(token, args.push_to_cloudwatch, args.delete_offline) diff --git a/tests/ci/ci_runners_metrics_lambda/requirements.txt b/tests/ci/ci_runners_metrics_lambda/requirements.txt index e99dee1743c..e2b16067a93 100644 --- a/tests/ci/ci_runners_metrics_lambda/requirements.txt +++ b/tests/ci/ci_runners_metrics_lambda/requirements.txt @@ -1,3 +1,2 @@ ../lambda_shared_package -PyJWT -cryptography<38 +../lambda_shared_package[token] diff --git a/tests/ci/lambda_shared_package/__init__.py b/tests/ci/lambda_shared_package/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py new file mode 100644 index 00000000000..ef47eacc082 --- /dev/null +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -0,0 +1,184 @@ +#!/usr/bin/env python + +import re +from typing import Tuple + +# Individual trusted contirbutors who are not in any trusted organization. +# Can be changed in runtime: we will append users that we learned to be in +# a trusted org, to save GitHub API calls. +TRUSTED_CONTRIBUTORS = { + e.lower() + for e in [ + "achimbab", + "adevyatova ", # DOCSUP + "Algunenano", # Raúl Marín, Tinybird + "amosbird", + "AnaUvarova", # DOCSUP + "anauvarova", # technical writer, Yandex + "annvsh", # technical writer, Yandex + "atereh", # DOCSUP + "azat", + "bharatnc", # Newbie, but already with many contributions. + "bobrik", # Seasoned contributor, CloudFlare + "BohuTANG", + "codyrobert", # Flickerbox engineer + "cwurm", # Employee + "damozhaeva", # DOCSUP + "den-crane", + "flickerbox-tom", # Flickerbox + "gyuton", # DOCSUP + "hagen1778", # Roman Khavronenko, seasoned contributor + "hczhcz", + "hexiaoting", # Seasoned contributor + "ildus", # adjust, ex-pgpro + "javisantana", # a Spanish ClickHouse enthusiast, ex-Carto + "ka1bi4", # DOCSUP + "kirillikoff", # DOCSUP + "kreuzerkrieg", + "lehasm", # DOCSUP + "michon470", # DOCSUP + "nikvas0", + "nvartolomei", + "olgarev", # DOCSUP + "otrazhenia", # Yandex docs contractor + "pdv-ru", # DOCSUP + "podshumok", # cmake expert from QRator Labs + "s-mx", # Maxim Sabyanin, former employee, present contributor + "sevirov", # technical writer, Yandex + "spongedu", # Seasoned contributor + "taiyang-li", + "ucasFL", # Amos Bird's friend + "vdimir", # Employee + "vzakaznikov", + "YiuRULE", + "zlobober", # Developer of YT + "ilejn", # Arenadata, responsible for Kerberized Kafka + "thomoco", # ClickHouse + "BoloniniD", # Seasoned contributor, HSE + "tonickkozlov", # Cloudflare + "tylerhannan", # ClickHouse Employee + "myrrc", # Mike Kot, DoubleCloud + "thevar1able", # ClickHouse Employee + "aalexfvk", + "MikhailBurdukov", + "tsolodov", # ClickHouse Employee + "kitaisreal", + ] +} + +# Descriptions are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there +# updated accordingly +# The following lists are append only, try to avoid editing them +# They still could be cleaned out after the decent time though. +LABELS = { + "pr-backward-incompatible": ["Backward Incompatible Change"], + "pr-bugfix": [ + "Bug Fix", + "Bug Fix (user-visible misbehavior in an official stable release)", + "Bug Fix (user-visible misbehaviour in official stable or prestable release)", + "Bug Fix (user-visible misbehavior in official stable or prestable release)", + ], + "pr-build": [ + "Build/Testing/Packaging Improvement", + "Build Improvement", + "Build/Testing Improvement", + "Build", + "Packaging Improvement", + ], + "pr-documentation": [ + "Documentation (changelog entry is not required)", + "Documentation", + ], + "pr-feature": ["New Feature"], + "pr-improvement": ["Improvement"], + "pr-not-for-changelog": [ + "Not for changelog (changelog entry is not required)", + "Not for changelog", + ], + "pr-performance": ["Performance Improvement"], +} + +CATEGORY_TO_LABEL = {c: lb for lb, categories in LABELS.items() for c in categories} + + +def check_pr_description(pr_body: str) -> Tuple[str, str]: + """The function checks the body to being properly formatted according to + .github/PULL_REQUEST_TEMPLATE.md, if the first returned string is not empty, + then there is an error.""" + lines = list(map(lambda x: x.strip(), pr_body.split("\n") if pr_body else [])) + lines = [re.sub(r"\s+", " ", line) for line in lines] + + # Check if body contains "Reverts ClickHouse/ClickHouse#36337" + if [ + True + for line in lines + if re.match(r"\AReverts {GITHUB_REPOSITORY}#[\d]+\Z", line) + ]: + return "", LABELS["pr-not-for-changelog"][0] + + category = "" + entry = "" + description_error = "" + + i = 0 + while i < len(lines): + if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]): + i += 1 + if i >= len(lines): + break + # Can have one empty line between header and the category + # itself. Filter it out. + if not lines[i]: + i += 1 + if i >= len(lines): + break + category = re.sub(r"^[-*\s]*", "", lines[i]) + i += 1 + + # Should not have more than one category. Require empty line + # after the first found category. + if i >= len(lines): + break + if lines[i]: + second_category = re.sub(r"^[-*\s]*", "", lines[i]) + description_error = ( + "More than one changelog category specified: " + f"'{category}', '{second_category}'" + ) + return description_error, category + + elif re.match( + r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i] + ): + i += 1 + # Can have one empty line between header and the entry itself. + # Filter it out. + if i < len(lines) and not lines[i]: + i += 1 + # All following lines until empty one are the changelog entry. + entry_lines = [] + while i < len(lines) and lines[i]: + entry_lines.append(lines[i]) + i += 1 + entry = " ".join(entry_lines) + # Don't accept changelog entries like '...'. + entry = re.sub(r"[#>*_.\- ]", "", entry) + # Don't accept changelog entries like 'Close #12345'. + entry = re.sub(r"^[\w\-\s]{0,10}#?\d{5,6}\.?$", "", entry) + else: + i += 1 + + if not category: + description_error = "Changelog category is empty" + # Filter out the PR categories that are not for changelog. + elif re.match( + r"(?i)doc|((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)", + category, + ): + pass # to not check the rest of the conditions + elif category not in CATEGORY_TO_LABEL: + description_error, category = f"Category '{category}' is not valid", "" + elif not entry: + description_error = f"Changelog entry required for category '{category}'" + + return description_error, category diff --git a/tests/ci/lambda_shared_package/lambda_shared/token.py b/tests/ci/lambda_shared_package/lambda_shared/token.py new file mode 100644 index 00000000000..174ea4625a3 --- /dev/null +++ b/tests/ci/lambda_shared_package/lambda_shared/token.py @@ -0,0 +1,90 @@ +"""Module to get the token for GitHub""" +from dataclasses import dataclass +import json +import time +from typing import Tuple + +import boto3 # type: ignore +import jwt +import requests # type: ignore + + +def get_key_and_app_from_aws() -> Tuple[str, int]: + secret_name = "clickhouse_github_secret_key" + session = boto3.session.Session() + client = session.client( + service_name="secretsmanager", + ) + get_secret_value_response = client.get_secret_value(SecretId=secret_name) + data = json.loads(get_secret_value_response["SecretString"]) + return data["clickhouse-app-key"], int(data["clickhouse-app-id"]) + + +def get_installation_id(jwt_token: str) -> int: + headers = { + "Authorization": f"Bearer {jwt_token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.get("https://api.github.com/app/installations", headers=headers) + response.raise_for_status() + data = response.json() + for installation in data: + if installation["account"]["login"] == "ClickHouse": + installation_id = installation["id"] + + return installation_id # type: ignore + + +def get_access_token_by_jwt(jwt_token: str, installation_id: int) -> str: + headers = { + "Authorization": f"Bearer {jwt_token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.post( + f"https://api.github.com/app/installations/{installation_id}/access_tokens", + headers=headers, + ) + response.raise_for_status() + data = response.json() + return data["token"] # type: ignore + + +def get_token_from_aws() -> str: + private_key, app_id = get_key_and_app_from_aws() + return get_access_token_by_key_app(private_key, app_id) + + +def get_access_token_by_key_app(private_key: str, app_id: int) -> str: + payload = { + "iat": int(time.time()) - 60, + "exp": int(time.time()) + (10 * 60), + "iss": app_id, + } + + encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256") + installation_id = get_installation_id(encoded_jwt) + return get_access_token_by_jwt(encoded_jwt, installation_id) + + +@dataclass +class CachedToken: + time: int + value: str + updating: bool = False + + +_cached_token = CachedToken(0, "") + + +def get_cached_access_token() -> str: + if time.time() - 550 < _cached_token.time or _cached_token.updating: + return _cached_token.value + # Indicate that the value is updating now, so the cached value can be + # used. The first setting and close-to-ttl are not counted as update + if _cached_token.time != 0 or time.time() - 590 < _cached_token.time: + _cached_token.updating = True + private_key, app_id = get_key_and_app_from_aws() + _cached_token.time = int(time.time()) + _cached_token.value = get_access_token_by_key_app(private_key, app_id) + _cached_token.updating = False + return _cached_token.value diff --git a/tests/ci/lambda_shared_package/pyproject.toml b/tests/ci/lambda_shared_package/pyproject.toml index 8b4b0a80948..bbf74cc0649 100644 --- a/tests/ci/lambda_shared_package/pyproject.toml +++ b/tests/ci/lambda_shared_package/pyproject.toml @@ -9,5 +9,15 @@ dependencies = [ "requests < 2.30", ] +[project.optional-dependencies] +token = [ + "PyJWT", + "cryptography<38", +] +dev = [ + "boto3", + "lambda_shared[token]", +] + [tool.distutils.bdist_wheel] universal = true diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 9849f19a1e4..330a1309016 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -20,9 +20,11 @@ from docs_check import NAME as DOCS_NAME from env_helper import GITHUB_REPOSITORY, GITHUB_SERVER_URL from get_robot_token import get_best_robot_token from pr_info import FORCE_TESTS_LABEL, PRInfo - -from cancel_and_rerun_workflow_lambda.app import CATEGORY_TO_LABEL, check_pr_description -from workflow_approve_rerun_lambda.app import TRUSTED_CONTRIBUTORS +from lambda_shared_package.lambda_shared.pr import ( + CATEGORY_TO_LABEL, + TRUSTED_CONTRIBUTORS, + check_pr_description, +) TRUSTED_ORG_IDS = { 54801242, # clickhouse diff --git a/tests/ci/runner_token_rotation_lambda/app.py b/tests/ci/runner_token_rotation_lambda/app.py index 70ee5da01f4..6544eee9581 100644 --- a/tests/ci/runner_token_rotation_lambda/app.py +++ b/tests/ci/runner_token_rotation_lambda/app.py @@ -2,40 +2,11 @@ import argparse import sys -import json -import time import boto3 # type: ignore -import jwt import requests # type: ignore - -def get_installation_id(jwt_token): - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.get("https://api.github.com/app/installations", headers=headers) - response.raise_for_status() - data = response.json() - for installation in data: - if installation["account"]["login"] == "ClickHouse": - installation_id = installation["id"] - return installation_id - - -def get_access_token(jwt_token, installation_id): - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.post( - f"https://api.github.com/app/installations/{installation_id}/access_tokens", - headers=headers, - ) - response.raise_for_status() - data = response.json() - return data["token"] +from lambda_shared.token import get_cached_access_token, get_access_token_by_key_app def get_runner_registration_token(access_token): @@ -52,32 +23,10 @@ def get_runner_registration_token(access_token): return data["token"] -def get_key_and_app_from_aws(): - secret_name = "clickhouse_github_secret_key" - session = boto3.session.Session() - client = session.client( - service_name="secretsmanager", - ) - get_secret_value_response = client.get_secret_value(SecretId=secret_name) - data = json.loads(get_secret_value_response["SecretString"]) - return data["clickhouse-app-key"], int(data["clickhouse-app-id"]) - - -def main(github_secret_key, github_app_id, push_to_ssm, ssm_parameter_name): - payload = { - "iat": int(time.time()) - 60, - "exp": int(time.time()) + (10 * 60), - "iss": github_app_id, - } - - encoded_jwt = jwt.encode(payload, github_secret_key, algorithm="RS256") - installation_id = get_installation_id(encoded_jwt) - access_token = get_access_token(encoded_jwt, installation_id) +def main(access_token, push_to_ssm, ssm_parameter_name): runner_registration_token = get_runner_registration_token(access_token) if push_to_ssm: - import boto3 - print("Trying to put params into ssm manager") client = boto3.client("ssm") client.put_parameter( @@ -94,8 +43,7 @@ def main(github_secret_key, github_app_id, push_to_ssm, ssm_parameter_name): def handler(event, context): - private_key, app_id = get_key_and_app_from_aws() - main(private_key, app_id, True, "github_runner_registration_token") + main(get_cached_access_token(), True, "github_runner_registration_token") if __name__ == "__main__": @@ -140,4 +88,5 @@ if __name__ == "__main__": with open(args.private_key_path, "r") as key_file: private_key = key_file.read() - main(private_key, args.app_id, args.push_to_ssm, args.ssm_parameter_name) + token = get_access_token_by_key_app(private_key, args.app_id) + main(token, args.push_to_ssm, args.ssm_parameter_name) diff --git a/tests/ci/runner_token_rotation_lambda/lambda_shared b/tests/ci/runner_token_rotation_lambda/lambda_shared new file mode 120000 index 00000000000..ba86e090f6c --- /dev/null +++ b/tests/ci/runner_token_rotation_lambda/lambda_shared @@ -0,0 +1 @@ +../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/runner_token_rotation_lambda/requirements.txt b/tests/ci/runner_token_rotation_lambda/requirements.txt index 98be09ab232..4cb3fba0f7b 100644 --- a/tests/ci/runner_token_rotation_lambda/requirements.txt +++ b/tests/ci/runner_token_rotation_lambda/requirements.txt @@ -1,3 +1 @@ -requests<2.30 -PyJWT -cryptography<38 +../lambda_shared_package[token] diff --git a/tests/ci/team_keys_lambda/app.py b/tests/ci/team_keys_lambda/app.py index 870d41c441e..f562fbe101d 100644 --- a/tests/ci/team_keys_lambda/app.py +++ b/tests/ci/team_keys_lambda/app.py @@ -81,6 +81,8 @@ def get_cached_members_keys(members: set) -> Keys: def get_token_from_aws() -> str: + # We need a separate token, since the clickhouse-ci app does not have + # access to the organization members' endpoint secret_name = "clickhouse_robot_token" session = boto3.session.Session() client = session.client( @@ -130,4 +132,4 @@ if __name__ == "__main__": args = parser.parse_args() output = main(args.token, args.organization, args.team) - print(f"# Just shoing off the keys:\n{output}") + print(f"# Just showing off the keys:\n{output}") diff --git a/tests/ci/team_keys_lambda/lambda_shared b/tests/ci/team_keys_lambda/lambda_shared new file mode 120000 index 00000000000..ba86e090f6c --- /dev/null +++ b/tests/ci/team_keys_lambda/lambda_shared @@ -0,0 +1 @@ +../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/team_keys_lambda/requirements.txt b/tests/ci/team_keys_lambda/requirements.txt index 3bcbe2dfd07..098e04a9798 100644 --- a/tests/ci/team_keys_lambda/requirements.txt +++ b/tests/ci/team_keys_lambda/requirements.txt @@ -1 +1 @@ -requests<2.30 +../lambda_shared_package diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index 5799a498d5a..98b14508314 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -5,86 +5,12 @@ import json import sys import time from dataclasses import dataclass -from typing import Any, Dict, List, Tuple +from typing import Any, Dict, List import boto3 # type: ignore -import requests # type: ignore -import jwt from lambda_shared import RunnerDescriptions, list_runners - - -def get_key_and_app_from_aws() -> Tuple[str, int]: - secret_name = "clickhouse_github_secret_key" - session = boto3.session.Session() - client = session.client( - service_name="secretsmanager", - ) - get_secret_value_response = client.get_secret_value(SecretId=secret_name) - data = json.loads(get_secret_value_response["SecretString"]) - return data["clickhouse-app-key"], int(data["clickhouse-app-id"]) - - -def get_installation_id(jwt_token: str) -> int: - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.get("https://api.github.com/app/installations", headers=headers) - response.raise_for_status() - data = response.json() - for installation in data: - if installation["account"]["login"] == "ClickHouse": - installation_id = installation["id"] - break - - return installation_id # type: ignore - - -def get_access_token(jwt_token: str, installation_id: int) -> str: - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.post( - f"https://api.github.com/app/installations/{installation_id}/access_tokens", - headers=headers, - ) - response.raise_for_status() - data = response.json() - return data["token"] # type: ignore - - -@dataclass -class CachedToken: - time: int - value: str - updating: bool = False - - -cached_token = CachedToken(0, "") - - -def get_cached_access_token() -> str: - if time.time() - 550 < cached_token.time or cached_token.updating: - return cached_token.value - # Indicate that the value is updating now, so the cached value can be - # used. The first setting and close-to-ttl are not counted as update - if cached_token.time != 0 or time.time() - 590 < cached_token.time: - cached_token.updating = True - private_key, app_id = get_key_and_app_from_aws() - payload = { - "iat": int(time.time()) - 60, - "exp": int(time.time()) + (10 * 60), - "iss": app_id, - } - - encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256") - installation_id = get_installation_id(encoded_jwt) - cached_token.time = int(time.time()) - cached_token.value = get_access_token(encoded_jwt, installation_id) - cached_token.updating = False - return cached_token.value +from lambda_shared.token import get_access_token_by_key_app, get_cached_access_token @dataclass @@ -284,6 +210,8 @@ if __name__ == "__main__": with open(args.private_key_path, "r") as key_file: private_key = key_file.read() + token = get_access_token_by_key_app(private_key, args.app_id) + sample_event = { "AutoScalingGroupARN": "arn:aws:autoscaling:us-east-1::autoScalingGroup:d4738357-2d40-4038-ae7e-b00ae0227003:autoScalingGroupName/my-asg", "AutoScalingGroupName": "my-asg", @@ -328,14 +256,4 @@ if __name__ == "__main__": "Cause": "SCALE_IN", } - payload = { - "iat": int(time.time()) - 60, - "exp": int(time.time()) + (10 * 60), - "iss": args.app_id, - } - - encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256") - installation_id = get_installation_id(encoded_jwt) - access_token = get_access_token(encoded_jwt, args.app_id) - - main(access_token, sample_event) + main(token, sample_event) diff --git a/tests/ci/terminate_runner_lambda/requirements.txt b/tests/ci/terminate_runner_lambda/requirements.txt index e99dee1743c..4cb3fba0f7b 100644 --- a/tests/ci/terminate_runner_lambda/requirements.txt +++ b/tests/ci/terminate_runner_lambda/requirements.txt @@ -1,3 +1 @@ -../lambda_shared_package -PyJWT -cryptography<38 +../lambda_shared_package[token] diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 32cba5d466b..3db62430d85 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -5,9 +5,10 @@ import fnmatch import json import time -import jwt import requests # type: ignore -import boto3 # type: ignore + +from lambda_shared.pr import TRUSTED_CONTRIBUTORS +from lambda_shared.token import get_cached_access_token SUSPICIOUS_CHANGED_FILES_NUMBER = 200 @@ -67,108 +68,6 @@ NEED_RERUN_WORKFLOWS = { "ReleaseBranchCI", } -# Individual trusted contirbutors who are not in any trusted organization. -# Can be changed in runtime: we will append users that we learned to be in -# a trusted org, to save GitHub API calls. -TRUSTED_CONTRIBUTORS = { - e.lower() - for e in [ - "achimbab", - "adevyatova ", # DOCSUP - "Algunenano", # Raúl Marín, Tinybird - "amosbird", - "AnaUvarova", # DOCSUP - "anauvarova", # technical writer, Yandex - "annvsh", # technical writer, Yandex - "atereh", # DOCSUP - "azat", - "bharatnc", # Newbie, but already with many contributions. - "bobrik", # Seasoned contributor, CloudFlare - "BohuTANG", - "codyrobert", # Flickerbox engineer - "cwurm", # Employee - "damozhaeva", # DOCSUP - "den-crane", - "flickerbox-tom", # Flickerbox - "gyuton", # DOCSUP - "hagen1778", # Roman Khavronenko, seasoned contributor - "hczhcz", - "hexiaoting", # Seasoned contributor - "ildus", # adjust, ex-pgpro - "javisantana", # a Spanish ClickHouse enthusiast, ex-Carto - "ka1bi4", # DOCSUP - "kirillikoff", # DOCSUP - "kreuzerkrieg", - "lehasm", # DOCSUP - "michon470", # DOCSUP - "nikvas0", - "nvartolomei", - "olgarev", # DOCSUP - "otrazhenia", # Yandex docs contractor - "pdv-ru", # DOCSUP - "podshumok", # cmake expert from QRator Labs - "s-mx", # Maxim Sabyanin, former employee, present contributor - "sevirov", # technical writer, Yandex - "spongedu", # Seasoned contributor - "taiyang-li", - "ucasFL", # Amos Bird's friend - "vdimir", # Employee - "vzakaznikov", - "YiuRULE", - "zlobober", # Developer of YT - "ilejn", # Arenadata, responsible for Kerberized Kafka - "thomoco", # ClickHouse - "BoloniniD", # Seasoned contributor, HSE - "tonickkozlov", # Cloudflare - "tylerhannan", # ClickHouse Employee - "myrrc", # Mike Kot, DoubleCloud - "thevar1able", # ClickHouse Employee - "aalexfvk", - "MikhailBurdukov", - "tsolodov", # ClickHouse Employee - "kitaisreal", - ] -} - - -def get_installation_id(jwt_token): - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.get("https://api.github.com/app/installations", headers=headers) - response.raise_for_status() - data = response.json() - for installation in data: - if installation["account"]["login"] == "ClickHouse": - installation_id = installation["id"] - return installation_id - - -def get_access_token(jwt_token, installation_id): - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.post( - f"https://api.github.com/app/installations/{installation_id}/access_tokens", - headers=headers, - ) - response.raise_for_status() - data = response.json() - return data["token"] - - -def get_key_and_app_from_aws(): - secret_name = "clickhouse_github_secret_key" - session = boto3.session.Session() - client = session.client( - service_name="secretsmanager", - ) - get_secret_value_response = client.get_secret_value(SecretId=secret_name) - data = json.loads(get_secret_value_response["SecretString"]) - return data["clickhouse-app-key"], int(data["clickhouse-app-id"]) - def is_trusted_contributor(pr_user_login, pr_user_orgs): if pr_user_login.lower() in TRUSTED_CONTRIBUTORS: @@ -331,19 +230,6 @@ def label_manual_approve(pull_request, token): _exec_post_with_retry(url, token, data) -def get_token_from_aws(): - private_key, app_id = get_key_and_app_from_aws() - payload = { - "iat": int(time.time()) - 60, - "exp": int(time.time()) + (10 * 60), - "iss": app_id, - } - - encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256") - installation_id = get_installation_id(encoded_jwt) - return get_access_token(encoded_jwt, installation_id) - - def get_workflow_jobs(workflow_description, token): jobs_url = ( workflow_description.api_url + f"/attempts/{workflow_description.attempt}/jobs" @@ -443,7 +329,7 @@ def check_workflow_completed( def main(event): - token = get_token_from_aws() + token = get_cached_access_token() event_data = json.loads(event["body"]) print("The body received:", event["body"]) workflow_description = get_workflow_description_from_event(event_data) diff --git a/tests/ci/workflow_approve_rerun_lambda/lambda_shared b/tests/ci/workflow_approve_rerun_lambda/lambda_shared new file mode 120000 index 00000000000..ba86e090f6c --- /dev/null +++ b/tests/ci/workflow_approve_rerun_lambda/lambda_shared @@ -0,0 +1 @@ +../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/workflow_approve_rerun_lambda/requirements.txt b/tests/ci/workflow_approve_rerun_lambda/requirements.txt index 98be09ab232..4cb3fba0f7b 100644 --- a/tests/ci/workflow_approve_rerun_lambda/requirements.txt +++ b/tests/ci/workflow_approve_rerun_lambda/requirements.txt @@ -1,3 +1 @@ -requests<2.30 -PyJWT -cryptography<38 +../lambda_shared_package[token] diff --git a/tests/ci/workflow_jobs_lambda/lambda_shared b/tests/ci/workflow_jobs_lambda/lambda_shared new file mode 120000 index 00000000000..ba86e090f6c --- /dev/null +++ b/tests/ci/workflow_jobs_lambda/lambda_shared @@ -0,0 +1 @@ +../lambda_shared_package/lambda_shared \ No newline at end of file From 2dca0eac1b5024e97f8e36889d8d39f43e8e4c2b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 23 May 2023 21:51:35 +0200 Subject: [PATCH 0957/2223] Delete __init__.py in lambda directories to break subpackage --- tests/ci/cancel_and_rerun_workflow_lambda/__init__.py | 0 tests/ci/runner_token_rotation_lambda/__init__.py | 0 tests/ci/team_keys_lambda/__init__.py | 0 tests/ci/terminate_runner_lambda/__init__.py | 0 tests/ci/workflow_approve_rerun_lambda/__init__.py | 1 - 5 files changed, 1 deletion(-) delete mode 100644 tests/ci/cancel_and_rerun_workflow_lambda/__init__.py delete mode 100644 tests/ci/runner_token_rotation_lambda/__init__.py delete mode 100644 tests/ci/team_keys_lambda/__init__.py delete mode 100644 tests/ci/terminate_runner_lambda/__init__.py delete mode 100644 tests/ci/workflow_approve_rerun_lambda/__init__.py diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/__init__.py b/tests/ci/cancel_and_rerun_workflow_lambda/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/ci/runner_token_rotation_lambda/__init__.py b/tests/ci/runner_token_rotation_lambda/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/ci/team_keys_lambda/__init__.py b/tests/ci/team_keys_lambda/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/ci/terminate_runner_lambda/__init__.py b/tests/ci/terminate_runner_lambda/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/ci/workflow_approve_rerun_lambda/__init__.py b/tests/ci/workflow_approve_rerun_lambda/__init__.py deleted file mode 100644 index 4265cc3e6c1..00000000000 --- a/tests/ci/workflow_approve_rerun_lambda/__init__.py +++ /dev/null @@ -1 +0,0 @@ -#!/usr/bin/env python From e8b03d74986a4e0f51f9cd064493cd5419c78add Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 26 May 2023 17:17:49 +0200 Subject: [PATCH 0958/2223] Move insert part for ClickHouseHelper to shared --- .../lambda_shared/__init__.py | 91 +++++++++- tests/ci/workflow_jobs_lambda/app.py | 165 +++--------------- .../ci/workflow_jobs_lambda/requirements.txt | 2 +- 3 files changed, 110 insertions(+), 148 deletions(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/__init__.py b/tests/ci/lambda_shared_package/lambda_shared/__init__.py index fe52f98d5f6..534d7773ddd 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/__init__.py +++ b/tests/ci/lambda_shared_package/lambda_shared/__init__.py @@ -5,7 +5,7 @@ import json import logging import time from collections import namedtuple -from typing import Any, List, Optional +from typing import Any, Dict, Iterable, List, Optional import boto3 # type: ignore import requests # type: ignore @@ -36,10 +36,14 @@ class CHException(Exception): pass +class InsertException(CHException): + pass + + class ClickHouseHelper: def __init__( self, - url: Optional[str] = None, + url: str, user: Optional[str] = None, password: Optional[str] = None, ): @@ -50,6 +54,89 @@ class ClickHouseHelper: if password: self.auth["X-ClickHouse-Key"] = password + @staticmethod + def _insert_json_str_info_impl( + url: str, auth: Dict[str, str], db: str, table: str, json_str: str + ) -> None: + params = { + "database": db, + "query": f"INSERT INTO {table} FORMAT JSONEachRow", + "date_time_input_format": "best_effort", + "send_logs_level": "warning", + } + + for i in range(5): + try: + response = requests.post( + url, params=params, data=json_str, headers=auth + ) + except Exception as e: + error = f"Received exception while sending data to {url} on {i} attempt: {e}" + logging.warning(error) + continue + + logging.info("Response content '%s'", response.content) + + if response.ok: + break + + error = ( + "Cannot insert data into clickhouse at try " + + str(i) + + ": HTTP code " + + str(response.status_code) + + ": '" + + str(response.text) + + "'" + ) + + if response.status_code >= 500: + # A retriable error + time.sleep(1) + continue + + logging.info( + "Request headers '%s', body '%s'", + response.request.headers, + response.request.body, + ) + + raise InsertException(error) + else: + raise InsertException(error) + + def _insert_json_str_info(self, db: str, table: str, json_str: str) -> None: + self._insert_json_str_info_impl(self.url, self.auth, db, table, json_str) + + def insert_event_into( + self, db: str, table: str, event: object, safe: bool = True + ) -> None: + event_str = json.dumps(event) + try: + self._insert_json_str_info(db, table, event_str) + except InsertException as e: + logging.error( + "Exception happened during inserting data into clickhouse: %s", e + ) + if not safe: + raise + + def insert_events_into( + self, db: str, table: str, events: Iterable[object], safe: bool = True + ) -> None: + jsons = [] + for event in events: + jsons.append(json.dumps(event)) + + try: + self._insert_json_str_info(db, table, ",".join(jsons)) + except InsertException as e: + logging.error( + "Exception happened during inserting data into clickhouse: %s", e + ) + if not safe: + raise + def _select_and_get_json_each_row(self, db: str, query: str) -> str: params = { "database": db, diff --git a/tests/ci/workflow_jobs_lambda/app.py b/tests/ci/workflow_jobs_lambda/app.py index c4ce68c3f8e..c624a492604 100644 --- a/tests/ci/workflow_jobs_lambda/app.py +++ b/tests/ci/workflow_jobs_lambda/app.py @@ -10,13 +10,11 @@ fields for private repositories from base64 import b64decode from dataclasses import dataclass -from typing import Any, List +from typing import Any, List, Optional import json import logging -import time -import boto3 # type: ignore -import requests # type: ignore +from lambda_shared import ClickHouseHelper, InsertException, get_parameter_from_ssm logging.getLogger().setLevel(logging.INFO) @@ -66,137 +64,7 @@ class WorkflowJob: return self.__dict__ -### VENDORING -def get_parameter_from_ssm(name, decrypt=True, client=None): - if not client: - client = boto3.client("ssm", region_name="us-east-1") - return client.get_parameter(Name=name, WithDecryption=decrypt)["Parameter"]["Value"] - - -class InsertException(Exception): - pass - - -class ClickHouseHelper: - def __init__(self, url=None): - if url is None: - url = get_parameter_from_ssm("clickhouse-test-stat-url") - - self.url = url - self.auth = { - "X-ClickHouse-User": get_parameter_from_ssm("clickhouse-test-stat-login"), - "X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password"), - } - - @staticmethod - def _insert_json_str_info_impl(url, auth, db, table, json_str): - params = { - "database": db, - "query": f"INSERT INTO {table} FORMAT JSONEachRow", - "date_time_input_format": "best_effort", - "send_logs_level": "warning", - } - - for i in range(5): - try: - response = requests.post( - url, params=params, data=json_str, headers=auth - ) - except Exception as e: - error = f"Received exception while sending data to {url} on {i} attempt: {e}" - logging.warning(error) - continue - - logging.info("Response content '%s'", response.content) - - if response.ok: - break - - error = ( - "Cannot insert data into clickhouse at try " - + str(i) - + ": HTTP code " - + str(response.status_code) - + ": '" - + str(response.text) - + "'" - ) - - if response.status_code >= 500: - # A retriable error - time.sleep(1) - continue - - logging.info( - "Request headers '%s', body '%s'", - response.request.headers, - response.request.body, - ) - - raise InsertException(error) - else: - raise InsertException(error) - - def _insert_json_str_info(self, db, table, json_str): - self._insert_json_str_info_impl(self.url, self.auth, db, table, json_str) - - def insert_event_into(self, db, table, event, safe=True): - event_str = json.dumps(event) - try: - self._insert_json_str_info(db, table, event_str) - except InsertException as e: - logging.error( - "Exception happened during inserting data into clickhouse: %s", e - ) - if not safe: - raise - - def insert_events_into(self, db, table, events, safe=True): - jsons = [] - for event in events: - jsons.append(json.dumps(event)) - - try: - self._insert_json_str_info(db, table, ",".join(jsons)) - except InsertException as e: - logging.error( - "Exception happened during inserting data into clickhouse: %s", e - ) - if not safe: - raise - - def _select_and_get_json_each_row(self, db, query): - params = { - "database": db, - "query": query, - "default_format": "JSONEachRow", - } - for i in range(5): - response = None - try: - response = requests.get(self.url, params=params, headers=self.auth) - response.raise_for_status() - return response.text - except Exception as ex: - logging.warning("Cannot insert with exception %s", str(ex)) - if response: - logging.warning("Reponse text %s", response.text) - time.sleep(0.1 * i) - - raise Exception("Cannot fetch data from clickhouse") - - def select_json_each_row(self, db, query): - text = self._select_and_get_json_each_row(db, query) - result = [] - for line in text.split("\n"): - if line: - result.append(json.loads(line)) - return result - - -### VENDORING END - -clickhouse_client = ClickHouseHelper() +CH_CLIENT = None # type: Optional[ClickHouseHelper] def send_event_workflow_job(workflow_job: WorkflowJob) -> None: @@ -232,23 +100,30 @@ def send_event_workflow_job(workflow_job: WorkflowJob) -> None: # PARTITION BY toStartOfMonth(started_at) # ORDER BY (id, updated_at) # SETTINGS index_granularity = 8192 - global clickhouse_client - kwargs = { - "db": "default", - "table": "workflow_jobs", - "event": workflow_job.as_dict(), - "safe": False, - } + global CH_CLIENT + CH_CLIENT = CH_CLIENT or ClickHouseHelper( + get_parameter_from_ssm("clickhouse-test-stat-url"), + get_parameter_from_ssm("clickhouse-test-stat-login"), + get_parameter_from_ssm("clickhouse-test-stat-password"), + ) try: - clickhouse_client.insert_event_into(**kwargs) + CH_CLIENT.insert_event_into( + "default", "workflow_jobs", workflow_job.as_dict(), False + ) except InsertException as ex: logging.exception( "Got an exception on insert, tryuing to update the client " "credentials and repeat", exc_info=ex, ) - clickhouse_client = ClickHouseHelper() - clickhouse_client.insert_event_into(**kwargs) + CH_CLIENT = ClickHouseHelper( + get_parameter_from_ssm("clickhouse-test-stat-url"), + get_parameter_from_ssm("clickhouse-test-stat-login"), + get_parameter_from_ssm("clickhouse-test-stat-password"), + ) + CH_CLIENT.insert_event_into( + "default", "workflow_jobs", workflow_job.as_dict(), False + ) def handler(event: dict, context: Any) -> dict: diff --git a/tests/ci/workflow_jobs_lambda/requirements.txt b/tests/ci/workflow_jobs_lambda/requirements.txt index 3bcbe2dfd07..098e04a9798 100644 --- a/tests/ci/workflow_jobs_lambda/requirements.txt +++ b/tests/ci/workflow_jobs_lambda/requirements.txt @@ -1 +1 @@ -requests<2.30 +../lambda_shared_package From f62faaedc3c837b009479cb971e868526a6464fa Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Jun 2023 21:20:39 +0200 Subject: [PATCH 0959/2223] paranoid fix for removing parts from zk --- .../MergeTree/ReplicatedMergeTreeSink.cpp | 19 +- .../MergeTree/ReplicatedMergeTreeSink.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 171 +++++++----------- src/Storages/StorageReplicatedMergeTree.h | 1 + ...tem_parts_race_condition_zookeeper_long.sh | 2 + .../0_stateless/01154_move_partition_long.sh | 2 + 6 files changed, 86 insertions(+), 111 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index a38e9eba844..28dad454afe 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -532,12 +532,12 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF try { - commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false); + bool deduplicated = commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false).second; - last_block_is_duplicate = last_block_is_duplicate || part->is_duplicate; + last_block_is_duplicate = last_block_is_duplicate || deduplicated; /// Set a special error code if the block is duplicate - int error = (deduplicate && part->is_duplicate) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; + int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus(error)); storage.incrementInsertedPartsProfileEvent(part->getType()); @@ -575,7 +575,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa while (true) { partition.temp_part.finalize(); - auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num, false); + auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num, false).first; if (conflict_block_ids.empty()) break; ++retry_times; @@ -620,7 +620,7 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: } template -std::vector ReplicatedMergeTreeSinkImpl::commitPart( +std::pair, bool> ReplicatedMergeTreeSinkImpl::commitPart( const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const BlockIDsType & block_id, @@ -644,6 +644,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( /// for retries due to keeper error bool part_committed_locally_but_zookeeper = false; + bool part_was_deduplicated = false; Coordination::Error write_part_info_keeper_error = Coordination::Error::ZOK; std::vector conflict_block_ids; @@ -844,7 +845,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( /// If it exists on our replica, ignore it. if (storage.getActiveContainingPart(existing_part_name)) { - part->is_duplicate = true; + part_was_deduplicated = true; ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); if (isQuorumEnabled()) { @@ -1040,7 +1041,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( ++loop_counter; if (loop_counter == max_iterations) { - part->is_duplicate = true; /// Part is duplicate, just remove it from local FS + part_was_deduplicated = true; /// Part is duplicate, just remove it from local FS throw Exception(ErrorCodes::DUPLICATE_DATA_PART, "Too many transaction retries - it may indicate an error"); } retries_ctl.requestUnconditionalRetry(); /// we want one more iteration w/o counting it as a try and timeout @@ -1093,7 +1094,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( [&zookeeper]() { zookeeper->cleanupEphemeralNodes(); }); if (!conflict_block_ids.empty()) - return conflict_block_ids; + return {conflict_block_ids, part_was_deduplicated}; if (isQuorumEnabled()) { @@ -1129,7 +1130,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( return; }); } - return {}; + return {conflict_block_ids, part_was_deduplicated}; } template diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 3777a9f7285..3efd364fc9c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -87,7 +87,7 @@ private: size_t checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. - std::vector commitPart( + std::pair, bool> commitPart( const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const BlockIDsType & block_id, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0f79e9f8f19..44403fc708b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -1992,6 +1993,16 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) /// Forcibly remove parts from ZooKeeper removePartsFromZooKeeperWithRetries(parts_to_remove); +#ifdef ABORT_ON_LOGICAL_ERROR + Strings parts_remain = getZooKeeper()->getChildren(replica_path + "/parts"); + for (const auto & part_name : parts_remain) + { + auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + if (drop_range_info.contains(part_info)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} remains in ZooKeeper after DROP_RANGE {}", part_name, entry.new_part_name); + } +#endif + if (entry.detach) LOG_DEBUG(log, "Detached {} parts inside {}.", parts_to_remove.size(), entry.new_part_name); else @@ -6634,8 +6645,7 @@ bool StorageReplicatedMergeTree::hasLightweightDeletedMask() const void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() { - auto table_lock = lockForShare( - RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto table_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto zookeeper = getZooKeeper(); /// Now these parts are in Deleting state. If we fail to remove some of them we must roll them back to Outdated state. @@ -6644,6 +6654,12 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() if (parts.empty()) return; + NOEXCEPT_SCOPE({ clearOldPartsAndRemoveFromZKImpl(zookeeper, std::move(parts)); }); +} + +void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZKImpl(zkutil::ZooKeeperPtr zookeeper, DataPartsVector && parts) +{ + DataPartsVector parts_to_delete_only_from_filesystem; // Only duplicates DataPartsVector parts_to_delete_completely; // All parts except duplicates DataPartsVector parts_to_retry_deletion; // Parts that should be retried due to network problems @@ -6654,7 +6670,11 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() /// Broken part can be removed from zk by removePartAndEnqueueFetch(...) only. /// Removal without enqueueing a fetch leads to intersecting parts. if (part->is_duplicate || part->outdated_because_broken) + { + LOG_WARNING(log, "Will not remove part {} from ZooKeeper (is_duplicate: {}, outdated_because_broken: {})", + part->name, part->is_duplicate, part->outdated_because_broken); parts_to_delete_only_from_filesystem.emplace_back(part); + } else parts_to_delete_completely.emplace_back(part); } @@ -6680,7 +6700,7 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() if (!rollback_parts.empty()) rollbackDeletingParts(rollback_parts); } - else /// all parts was successfully removed + else /// all parts were successfully removed { finally_remove_parts = parts_to_delete; } @@ -6764,114 +6784,57 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeperWithRetries(PartsToRemo void StorageReplicatedMergeTree::removePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries) { + auto zookeeper = getZooKeeper(); + NameSet parts_to_retry_set; + removePartsFromZooKeeper(zookeeper, part_names, &parts_to_retry_set); + size_t num_tries = 0; - bool success = false; - - while (!success && (max_retries == 0 || num_tries < max_retries)) + while (!parts_to_retry_set.empty() && (max_retries == 0 || num_tries < max_retries)) { - try - { - ++num_tries; - success = true; - - auto zookeeper = getZooKeeper(); - - Strings exists_paths; - exists_paths.reserve(part_names.size()); - for (const String & part_name : part_names) - { - exists_paths.emplace_back(fs::path(replica_path) / "parts" / part_name); - } - - auto exists_results = zookeeper->exists(exists_paths); - - std::vector> remove_futures; - remove_futures.reserve(part_names.size()); - for (size_t i = 0; i < part_names.size(); ++i) - { - Coordination::ExistsResponse exists_resp = exists_results[i]; - if (exists_resp.error == Coordination::Error::ZOK) - { - Coordination::Requests ops; - getRemovePartFromZooKeeperOps(part_names[i], ops, exists_resp.stat.numChildren > 0); - remove_futures.emplace_back(zookeeper->asyncTryMultiNoThrow(ops)); - } - } - - for (auto & future : remove_futures) - { - auto response = future.get(); - - if (response.error == Coordination::Error::ZOK || response.error == Coordination::Error::ZNONODE) - continue; - - if (Coordination::isHardwareError(response.error)) - { - success = false; - continue; - } - - throw Coordination::Exception(response.error); - } - } - catch (Coordination::Exception & e) - { - success = false; - - if (Coordination::isHardwareError(e.code)) - tryLogCurrentException(log, __PRETTY_FUNCTION__); - else - throw; - } - - if (!success && num_tries < max_retries) - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + zookeeper = getZooKeeper(); + Strings parts_to_retry; + std::move(parts_to_retry_set.begin(), parts_to_retry_set.end(), std::back_inserter(parts_to_retry)); + parts_to_retry_set.clear(); + removePartsFromZooKeeper(zookeeper, parts_to_retry, &parts_to_retry_set); + ++num_tries; } - if (!success) - throw Exception(ErrorCodes::UNFINISHED, "Failed to remove parts from ZooKeeper after {} retries", num_tries); + if (!parts_to_retry_set.empty()) + throw Exception(ErrorCodes::UNFINISHED, "Failed to remove {} parts from ZooKeeper after {} retries", parts_to_retry_set.size(), num_tries); } void StorageReplicatedMergeTree::removePartsFromZooKeeper( zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names, NameSet * parts_should_be_retried) +try { Strings exists_paths; std::vector> remove_futures; exists_paths.reserve(part_names.size()); remove_futures.reserve(part_names.size()); - try + /// Exception can be thrown from loop + /// if zk session will be dropped + for (const String & part_name : part_names) { - /// Exception can be thrown from loop - /// if zk session will be dropped - for (const String & part_name : part_names) - { - exists_paths.emplace_back(fs::path(replica_path) / "parts" / part_name); - } - - auto exists_results = zookeeper->exists(exists_paths); - - for (size_t i = 0; i < part_names.size(); ++i) - { - auto exists_resp = exists_results[i]; - if (exists_resp.error == Coordination::Error::ZOK) - { - Coordination::Requests ops; - getRemovePartFromZooKeeperOps(part_names[i], ops, exists_resp.stat.numChildren > 0); - remove_futures.emplace_back(zookeeper->asyncTryMultiNoThrow(ops)); - } - else - { - LOG_DEBUG(log, "There is no part {} in ZooKeeper, it was only in filesystem", part_names[i]); - // emplace invalid future so that the total number of futures is the same as part_names.size(); - remove_futures.emplace_back(); - } - } + exists_paths.emplace_back(fs::path(replica_path) / "parts" / part_name); } - catch (const Coordination::Exception & e) + + auto exists_results = zookeeper->exists(exists_paths); + + for (size_t i = 0; i < part_names.size(); ++i) { - if (parts_should_be_retried && Coordination::isHardwareError(e.code)) - parts_should_be_retried->insert(part_names.begin(), part_names.end()); - throw; + auto exists_resp = exists_results[i]; + if (exists_resp.error == Coordination::Error::ZOK) + { + Coordination::Requests ops; + getRemovePartFromZooKeeperOps(part_names[i], ops, exists_resp.stat.numChildren > 0); + remove_futures.emplace_back(zookeeper->asyncTryMultiNoThrow(ops)); + } + else + { + LOG_DEBUG(log, "There is no part {} in ZooKeeper, it was only in filesystem", part_names[i]); + // emplace invalid future so that the total number of futures is the same as part_names.size(); + remove_futures.emplace_back(); + } } for (size_t i = 0; i < remove_futures.size(); ++i) @@ -6884,21 +6847,27 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper( auto response = future.get(); if (response.error == Coordination::Error::ZOK) continue; - else if (response.error == Coordination::Error::ZNONODE) + + if (response.error == Coordination::Error::ZNONODE) { LOG_DEBUG(log, "There is no part {} in ZooKeeper, it was only in filesystem", part_names[i]); - continue; } - else if (Coordination::isHardwareError(response.error)) + else { if (parts_should_be_retried) parts_should_be_retried->insert(part_names[i]); - continue; + + if (!Coordination::isHardwareError(response.error)) + LOG_WARNING(log, "Cannot remove part {} from ZooKeeper: {}", part_names[i], Coordination::errorMessage(response.error)); } - else - LOG_WARNING(log, "Cannot remove part {} from ZooKeeper: {}", part_names[i], Coordination::errorMessage(response.error)); } } +catch (...) +{ + if (parts_should_be_retried) + parts_should_be_retried->insert(part_names.begin(), part_names.end()); + throw; +} void StorageReplicatedMergeTree::clearLockedBlockNumbersInPartition( zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index cb93dd0b5e3..dd7ea84f76b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -344,6 +344,7 @@ private: /// Delete old parts from disk and from ZooKeeper. void clearOldPartsAndRemoveFromZK(); + void clearOldPartsAndRemoveFromZKImpl(zkutil::ZooKeeperPtr zookeeper, DataPartsVector && parts); template friend class ReplicatedMergeTreeSinkImpl; diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index 5b1c50262bf..862cc90fb1c 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -79,6 +79,8 @@ timeout $TIMEOUT bash -c thread5 2> /dev/null & wait check_replication_consistency "alter_table" "count(), sum(a), sum(b), round(sum(c))" +$CLICKHOUSE_CLIENT -q "SELECT table, lost_part_count FROM system.replicas WHERE database=currentDatabase() AND lost_part_count!=0"; + $CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table0;" 2> >(grep -F -v 'is already started to be removing by another replica right now') & $CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table1;" 2> >(grep -F -v 'is already started to be removing by another replica right now') & wait diff --git a/tests/queries/0_stateless/01154_move_partition_long.sh b/tests/queries/0_stateless/01154_move_partition_long.sh index c68b0944407..3e068fa0e2d 100755 --- a/tests/queries/0_stateless/01154_move_partition_long.sh +++ b/tests/queries/0_stateless/01154_move_partition_long.sh @@ -125,6 +125,8 @@ wait check_replication_consistency "dst_" "count(), sum(p), sum(k), sum(v)" try_sync_replicas "src_" 300 +$CLICKHOUSE_CLIENT -q "SELECT table, lost_part_count FROM system.replicas WHERE database=currentDatabase() AND lost_part_count!=0"; + for ((i=0; i<16; i++)) do $CLICKHOUSE_CLIENT -q "DROP TABLE dst_$i" 2>&1| grep -Fv "is already started to be removing" & $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS src_$i" 2>&1| grep -Fv "is already started to be removing" & From 38abcd1c44bc580217081d9fb3d72a1dcd951fa3 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 1 Jun 2023 19:25:53 +0000 Subject: [PATCH 0960/2223] Add nullable support to DateTimeTransformImpl --- src/Functions/DateTimeTransforms.h | 44 ++++++++++++------ src/Functions/FunctionsConversion.h | 46 ++++++++++++++++++- .../01556_accurate_cast_or_null.reference | 8 ++++ .../01556_accurate_cast_or_null.sql | 10 ++++ 4 files changed, 93 insertions(+), 15 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 81b1ec2e356..0008b36071b 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include #include @@ -1433,7 +1435,8 @@ template - static void vector(const FromTypeVector & vec_from, ToTypeVector & vec_to, const DateLUTImpl & time_zone, const Transform & transform) + static void vector(const FromTypeVector & vec_from, ToTypeVector & vec_to, const DateLUTImpl & time_zone, const Transform & transform, + ColumnUInt8::Container * vec_null_map_to [[maybe_unused]]) { using ValueType = typename ToTypeVector::value_type; size_t size = vec_from.size(); @@ -1441,28 +1444,30 @@ struct Transformer for (size_t i = 0; i < size; ++i) { - constexpr bool transformHasExtraCheck = requires(const Transform& t) + constexpr bool transformHasIsConvertible = requires(const Transform& t) { - t.ExtraCheck(vec_from[i], time_zone); + t.IsConvertible(vec_from[i], time_zone); }; - if constexpr (transformHasExtraCheck) + if constexpr (transformHasIsConvertible) { - // if constexpr (std::is_same_v - // || std::is_same_v) + if constexpr (std::is_same_v + || std::is_same_v) { - bool checked = transform.ExtraCheck(vec_from[i], time_zone); + bool checked = transform.IsConvertible(vec_from[i], time_zone); if (!checked) { - if (std::is_same_v) + if (std::is_same_v) { - // vec_to[i] = 0; - // (*vec_null_map_to)[i] = true; + vec_to[i] = 0; + if (vec_null_map_to) + (*vec_null_map_to)[i] = true; + continue; } else { throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Value in column {} cannot be safely converted into type {}", - TypeName, TypeName); + TypeName, TypeName); } } } @@ -1488,6 +1493,14 @@ struct DateTimeTransformImpl const ColumnPtr source_col = arguments[0].column; if (const auto * sources = checkAndGetColumn(source_col.get())) { + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr; + if constexpr (std::is_same_v) + { + col_null_map_to = ColumnUInt8::create(sources->getData().size(), false); + vec_null_map_to = &col_null_map_to->getData(); + } + auto mutable_result_col = result_type->createColumn(); auto * col_to = assert_cast(mutable_result_col.get()); @@ -1495,7 +1508,7 @@ struct DateTimeTransformImpl if (result_data_type.isDateTime() || result_data_type.isDateTime64()) { const auto & time_zone = dynamic_cast(*result_type).getTimeZone(); - Op::vector(sources->getData(), col_to->getData(), time_zone, transform); + Op::vector(sources->getData(), col_to->getData(), time_zone, transform, vec_null_map_to); } else { @@ -1504,7 +1517,12 @@ struct DateTimeTransformImpl time_zone_argument_position = 2; const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, time_zone_argument_position, 0); - Op::vector(sources->getData(), col_to->getData(), time_zone, transform); + Op::vector(sources->getData(), col_to->getData(), time_zone, transform, vec_null_map_to); + } + + if (vec_null_map_to) + { + return ColumnNullable::create(std::move(mutable_result_col), std::move(col_null_map_to)); } return mutable_result_col; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index d3ccbb82721..4b25a59ecc6 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -370,6 +370,11 @@ struct ToDateTransform32Or64 { static constexpr auto name = "toDate"; + static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) + { + return from >= 0; + } + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { // since converting to Date, no need in values outside of default LUT range. @@ -384,6 +389,11 @@ struct ToDateTransform32Or64Signed { static constexpr auto name = "toDate"; + static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) + { + return from >= 0; + } + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { // TODO: decide narrow or extended range based on FromType @@ -400,7 +410,8 @@ template struct ToDateTransform8Or16Signed { static constexpr auto name = "toDate"; - static NO_SANITIZE_UNDEFINED bool ExtraCheck(const FromType & from, const DateLUTImpl &) + + static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) { return from >= 0; } @@ -423,6 +434,11 @@ struct ToDate32Transform32Or64 { static constexpr auto name = "toDate32"; + static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) + { + return from >= 0; + } + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { return (from < DATE_LUT_MAX_EXTEND_DAY_NUM) @@ -436,6 +452,11 @@ struct ToDate32Transform32Or64Signed { static constexpr auto name = "toDate32"; + static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) + { + return from >= 0; + } + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { static const Int32 daynum_min_offset = -static_cast(DateLUT::instance().getDayNumOffsetEpoch()); @@ -452,6 +473,11 @@ struct ToDate32Transform8Or16Signed { static constexpr auto name = "toDate32"; + static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) + { + return from >= 0; + } + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { return from; @@ -507,6 +533,11 @@ struct ToDateTimeTransform64 { static constexpr auto name = "toDateTime"; + static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) + { + return from >= 0; + } + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { return static_cast(std::min(time_t(from), time_t(0xFFFFFFFF))); @@ -518,6 +549,11 @@ struct ToDateTimeTransformSigned { static constexpr auto name = "toDateTime"; + static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) + { + return from >= 0; + } + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { if (from < 0) @@ -531,6 +567,11 @@ struct ToDateTimeTransform64Signed { static constexpr auto name = "toDateTime"; + static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) + { + return from >= 0; + } + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { if (from < 0) @@ -2886,7 +2927,8 @@ private: return true; } - if constexpr (IsDataTypeNumber && IsDataTypeDateOrDateTime) + if constexpr (IsDataTypeNumber + && (std::is_same_v || std::is_same_v)) { if (wrapper_cast_type == CastType::accurate) { diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference index b329aede01a..8429d5d0e64 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference @@ -30,3 +30,11 @@ \N 127 \N +\N +\N +2023-05-30 14:38:20 +1970-01-01 00:00:19 +\N +\N +2023-05-30 +1970-01-20 \ No newline at end of file diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql index b45bbe35662..a9038a1d230 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql @@ -35,3 +35,13 @@ SELECT accurateCastOrNull(nan, 'UInt64'); SELECT accurateCastOrNull(nan, 'UInt256'); SELECT accurateCastOrNull(number + 127, 'Int8') AS x FROM numbers (2) ORDER BY x; + +SELECT accurateCastOrNull(-1, 'DateTime'); +SELECT accurateCastOrNull('1xxx', 'DateTime'); +SELECT accurateCastOrNull('2023-05-30 14:38:20', 'DateTime'); +SELECT accurateCastOrNull(19, 'DateTime'); + +SELECT accurateCastOrNull(-1, 'Date'); +SELECT accurateCastOrNull('1xxx', 'Date'); +SELECT accurateCastOrNull('2023-05-30', 'Date'); +SELECT accurateCastOrNull(19, 'Date'); From 11ead24bf9d0426e15a94d87746430f16035da20 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 1 Jun 2023 19:38:46 +0000 Subject: [PATCH 0961/2223] Fix tests after nullable fixing --- .../queries/0_stateless/01556_accurate_cast_or_null.reference | 2 +- tests/queries/0_stateless/01601_accurate_cast.reference | 2 +- tests/queries/0_stateless/01601_accurate_cast.sql | 2 +- .../0_stateless/01746_convert_type_with_default.reference | 3 ++- tests/queries/0_stateless/01746_convert_type_with_default.sql | 3 ++- 5 files changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference index 8429d5d0e64..cbdf72e9910 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference @@ -37,4 +37,4 @@ \N \N 2023-05-30 -1970-01-20 \ No newline at end of file +1970-01-20 diff --git a/tests/queries/0_stateless/01601_accurate_cast.reference b/tests/queries/0_stateless/01601_accurate_cast.reference index b662319d263..3c6dceb1f16 100644 --- a/tests/queries/0_stateless/01601_accurate_cast.reference +++ b/tests/queries/0_stateless/01601_accurate_cast.reference @@ -9,4 +9,4 @@ 2023-05-30 14:38:20 1970-01-01 00:00:19 2023-05-30 -1970-01-20 \ No newline at end of file +1970-01-20 diff --git a/tests/queries/0_stateless/01601_accurate_cast.sql b/tests/queries/0_stateless/01601_accurate_cast.sql index 1ab98e26d1a..7611b1d96b9 100644 --- a/tests/queries/0_stateless/01601_accurate_cast.sql +++ b/tests/queries/0_stateless/01601_accurate_cast.sql @@ -29,6 +29,6 @@ SELECT accurateCast('2023-05-30 14:38:20', 'DateTime'); SELECT accurateCast(19, 'DateTime'); SELECT accurateCast(-1, 'Date'); -- { serverError 70 } -SELECT accurateCast('1xxx', 'Date'); -- { serverError 70 } +SELECT accurateCast('1xxx', 'Date'); -- { serverError 38 } SELECT accurateCast('2023-05-30', 'Date'); SELECT accurateCast(19, 'Date'); diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.reference b/tests/queries/0_stateless/01746_convert_type_with_default.reference index 892a12434b9..85bf2064fdc 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.reference +++ b/tests/queries/0_stateless/01746_convert_type_with_default.reference @@ -40,7 +40,8 @@ 1970-01-20 1970-01-20 2023-05-30 -1970-01-01 +2023-05-30 +2023-05-30 14:38:20 2023-05-30 14:38:20 2023-05-30 14:38:20 2023-05-30 14:38:20 diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index 75e1510f330..1065eefa94d 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -54,12 +54,13 @@ select toDateOrDefault(cast(19 as Int256)); select toDateOrDefault(cast(19 as UInt256)); select toDateOrDefault(19507, '2000-01-01'::Date); -select toDateOrDefault(-1, '2000-01-01'::Date); +select toDateOrDefault(-1, '2023-05-30'::Date); select toDateTimeOrDefault('2023-05-30 14:38:20'); select toDateTimeOrDefault('2023-05-30 14:38:20', 'UTC'); select toDateTimeOrDefault('1xxx', 'UTC', '2023-05-30 14:38:20'::DateTime('UTC')); select toDateTimeOrDefault(1685457500, 'UTC'); +select toDateTimeOrDefault(-1, 'UTC', '2023-05-30 14:38:20'::DateTime('UTC')); select toDateTimeOrDefault(cast(19 as Int8), 'UTC'); select toDateTimeOrDefault(cast(19 as UInt8), 'UTC'); From a22e80eed56ca0e6ab35ba0ad8c53c0a629a4839 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 1 Jun 2023 19:52:48 +0000 Subject: [PATCH 0962/2223] Remove whitespaces --- src/Functions/DateTimeTransforms.h | 6 +++--- src/Functions/FunctionsConversion.h | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 0008b36071b..9f8f4df2465 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1448,7 +1448,7 @@ struct Transformer { t.IsConvertible(vec_from[i], time_zone); }; - + if constexpr (transformHasIsConvertible) { if constexpr (std::is_same_v @@ -1472,7 +1472,7 @@ struct Transformer } } } - + if constexpr (is_extended_result) vec_to[i] = static_cast(transform.executeExtendedResult(vec_from[i], time_zone)); else @@ -1500,7 +1500,7 @@ struct DateTimeTransformImpl col_null_map_to = ColumnUInt8::create(sources->getData().size(), false); vec_null_map_to = &col_null_map_to->getData(); } - + auto mutable_result_col = result_type->createColumn(); auto * col_to = assert_cast(mutable_result_col.get()); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 4b25a59ecc6..d77090afe71 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -415,7 +415,7 @@ struct ToDateTransform8Or16Signed { return from >= 0; } - + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { if (from < 0) From 1544067fb37b53b5ba0e1101db9ab068e9903217 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Jun 2023 23:28:19 +0300 Subject: [PATCH 0963/2223] Update run.sh --- docker/test/unit/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/unit/run.sh b/docker/test/unit/run.sh index abc35fa40d2..a4784466e27 100644 --- a/docker/test/unit/run.sh +++ b/docker/test/unit/run.sh @@ -3,5 +3,5 @@ set -x service zookeeper start && sleep 7 && /usr/share/zookeeper/bin/zkCli.sh -server localhost:2181 -create create /clickhouse_test ''; -gdb -q -ex 'set print inferior-events off' -ex 'set confirm off' -ex 'set print thread-events off' -ex run -ex bt -ex quit --args ./unit_tests_dbms | tee test_output/test_result.txt +timeout 40m gdb -q -ex 'set print inferior-events off' -ex 'set confirm off' -ex 'set print thread-events off' -ex run -ex bt -ex quit --args ./unit_tests_dbms | tee test_output/test_result.txt ./process_unit_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv From 92859ebb3baad37e6538118f5e2c5c8016754b7f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 1 Jun 2023 21:15:15 +0000 Subject: [PATCH 0964/2223] Fixing more tests. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Functions/in.cpp | 12 +- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Interpreters/PreparedSets.cpp | 119 ++++++++++-------- src/Interpreters/PreparedSets.h | 25 ++-- src/Interpreters/Set.cpp | 30 ++++- src/Interpreters/Set.h | 7 +- src/Planner/CollectSets.cpp | 2 +- .../CreateSetAndFilterOnTheFlyStep.cpp | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 8 +- src/Processors/QueryPlan/CreatingSetsStep.h | 6 +- .../Transforms/CreatingSetsTransform.cpp | 11 +- .../Transforms/CreatingSetsTransform.h | 6 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 10 +- src/QueryPipeline/QueryPipelineBuilder.h | 10 +- src/Storages/MergeTree/RPNBuilder.cpp | 53 ++++---- src/Storages/StorageSet.cpp | 4 +- .../01786_explain_merge_tree.reference | 4 +- 19 files changed, 195 insertions(+), 122 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 05b3a545dca..b39aff86d32 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5141,7 +5141,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, settings.transform_null_in); + auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, 0, settings.transform_null_in); set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); set->insertFromBlock(result_block.getColumnsWithTypeAndName()); diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index 6a88a413c63..0e576b92aad 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -55,13 +55,9 @@ public: /// It is needed to perform type analysis without creation of set. static constexpr auto name = FunctionInName::name; - FunctionIn(SizeLimits size_limits_, bool transform_null_in_) - : size_limits(std::move(size_limits_)), transform_null_in(transform_null_in_) {} - - static FunctionPtr create(ContextPtr context) + static FunctionPtr create(ContextPtr) { - const auto & settings = context->getSettingsRef(); - return std::make_shared(FutureSet::getSizeLimitsForSet(settings, false), settings.transform_null_in); + return std::make_shared(); } String getName() const override @@ -182,10 +178,6 @@ public: return res; } - -private: - SizeLimits size_limits; - bool transform_null_in; }; template diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 142b6f73b75..59bbc74ca3a 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1435,7 +1435,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool // String set_id = right_in_operand->getColumnName(); //bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; SubqueryForSet subquery_for_set; // = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); - subquery_for_set.key = right_in_operand->getColumnName(); + subquery_for_set.key = set_key.toString(); //right_in_operand->getColumnName(); /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, @@ -1450,7 +1450,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool subquery_for_set.createSource(*interpreter); } - return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), std::move(external_table_set)); + return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), data.getContext()->getSettingsRef(), std::move(external_table_set)); } else { diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 624b33ea66b..cbdfb826f85 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -197,7 +197,7 @@ public: subquery_for_set.createSource(*interpreter); //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; - auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), getContext()->getSettingsRef(), nullptr); // std::cerr << "... Future set " << reinterpret_cast(external_storage_holder.get()) << " " << reinterpret_cast(future_set.get()) << std::endl; external_storage_holder->future_set = std::move(future_set); } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index c71273e0baa..1e475e8403d 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -132,10 +132,10 @@ FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, return it->second; } -FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set) +FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) { auto id = subquery.key; - auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set)); + auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings.transform_null_in); auto [it, inserted] = sets.emplace(key, from_subquery); if (!inserted) @@ -210,6 +210,8 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo { std::lock_guard lock(cache_mutex); + // std::cerr << "PreparedSetsCache::findOrPromiseToBuild " << key << "\n" << StackTrace().toString() << std::endl; + auto it = cache.find(key); if (it != cache.end()) { @@ -231,10 +233,10 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) if (!context->getSettingsRef().use_index_for_in_with_subqueries) return nullptr; - if (set) + if (subquery.set) { - if (set->hasExplicitSetElements()) - return set; + if (subquery.set->hasExplicitSetElements()) + return subquery.set; return nullptr; } @@ -242,7 +244,7 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) // std::cerr << "... external_table_set " << reinterpret_cast(external_table_set.get()) << std::endl; if (external_table_set) - return set = external_table_set->buildOrderedSetInplace(context); + return subquery.set = external_table_set->buildOrderedSetInplace(context); auto plan = buildPlan(context, true); if (!plan) @@ -255,37 +257,44 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) CompletedPipelineExecutor executor(pipeline); executor.execute(); - return set; + subquery.set->checkIsCreated(); + + return subquery.set; +} + +static SizeLimits getSizeLimitsForSet(const Settings & settings) +{ + return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); } std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) { - if (set) + if (subquery.set) return nullptr; // std::cerr << StackTrace().toString() << std::endl; - auto set_cache = context->getPreparedSetsCache(); - if (set_cache) - { - auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); - if (from_cache.index() == 0) - { - subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); - } - else - { - LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); - set = std::get<1>(from_cache).get(); - return nullptr; - } - } + // auto set_cache = context->getPreparedSetsCache(); + // if (set_cache) + // { + // auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); + // if (from_cache.index() == 0) + // { + // subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); + // } + // else + // { + // LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); + // set = std::get<1>(from_cache).get(); + // return nullptr; + // } + // } const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + auto size_limits = getSizeLimitsForSet(settings); - subquery.set = set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + subquery.set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); auto plan = subquery.detachSource(); auto description = subquery.key; @@ -297,7 +306,8 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c auto creating_set = std::make_unique( plan->getCurrentDataStream(), description, - std::move(subquery), + subquery, + shared_from_this(), SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), context); creating_set->setStepDescription("Create set for subquery"); @@ -305,31 +315,25 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c return plan; } +// static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) +// { +// if (settings.use_index_for_in_with_subqueries_max_values && +// settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) +// return getSizeLimitsForUnorderedSet(settings); -static SizeLimits getSizeLimitsForUnorderedSet(const Settings & settings) -{ - return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); -} +// return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); +// } -static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) -{ - if (settings.use_index_for_in_with_subqueries_max_values && - settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) - return getSizeLimitsForUnorderedSet(settings); - - return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); -} - -SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) -{ - return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); -} +// SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) +// { +// return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); +// } FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) { bool create_ordered_set = false; - auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); - set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + auto size_limits = getSizeLimitsForSet(settings); + set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); Columns columns; @@ -345,8 +349,16 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) //block(std::move(block_)) } -FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_) - : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)) {} +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_) + : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)), transform_null_in(transform_null_in_) {} + +DataTypes FutureSetFromSubquery::getTypes() const +{ + if (subquery.set) + return subquery.set->getElementsTypes(); + + return Set::getElementTypes(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName(), transform_null_in); +} FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} @@ -356,13 +368,14 @@ SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) return set; const auto & settings = context->getSettingsRef(); - auto limits = getSizeLimitsForSet(settings, true); + size_t max_values = settings.use_index_for_in_with_subqueries_max_values; + bool too_many_values = max_values && max_values < set->getTotalRowCount(); + if (!too_many_values) + { + set->initSetElements(); + set->appendSetElements(set_key_columns); + } - if (!limits.check(set->getTotalRowCount(), set->getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED)) - return nullptr; - - set->initSetElements(); - set->appendSetElements(set_key_columns); return set; } diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index c510e690e25..fa7f7069994 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -72,7 +72,9 @@ public: virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; virtual std::unique_ptr build(const ContextPtr & context) = 0; - static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); + virtual DataTypes getTypes() const = 0; + + // static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); }; using FutureSetPtr = std::shared_ptr; @@ -90,6 +92,8 @@ public: std::unique_ptr build(const ContextPtr &) override; + DataTypes getTypes() const override { return set->getElementsTypes(); } + /// void buildForTuple(SizeLimits size_limits, bool transform_null_in); private: @@ -129,14 +133,14 @@ public: std::unique_ptr source; }; -class FutureSetFromSubquery : public FutureSet +class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this { public: - FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_); + FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_); - bool isReady() const override { return set != nullptr; } + bool isReady() const override { return subquery.set != nullptr && subquery.set->isCreated(); } bool isFilled() const override { return isReady(); } - SetPtr get() const override { return set; } + SetPtr get() const override { return subquery.set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; @@ -145,12 +149,15 @@ public: return buildPlan(context, false); } - void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } + DataTypes getTypes() const override; + + // void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } private: - SetPtr set; + //SetPtr set; SubqueryForSet subquery; FutureSetPtr external_table_set; + bool transform_null_in; std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; @@ -169,6 +176,8 @@ public: return set->hasExplicitSetElements() ? set : nullptr; } + DataTypes getTypes() const override { return set->getElementsTypes(); } + std::unique_ptr build(const ContextPtr &) override { return nullptr; } private: @@ -240,7 +249,7 @@ public: FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set); + FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); //void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index bd9cafc66eb..52e9a3a9bcb 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -103,6 +103,25 @@ void NO_INLINE Set::insertFromBlockImplCase( } +DataTypes Set::getElementTypes(const ColumnsWithTypeAndName & header, bool transform_null_in) +{ + DataTypes data_types; + data_types.reserve(header.size()); + + for (const auto & column : header) + { + data_types.push_back(column.type); + if (const auto * low_cardinality_type = typeid_cast(data_types.back().get())) + data_types.back() = low_cardinality_type->getDictionaryType(); + + if (!transform_null_in) + data_types.back() = removeNullable(data_types.back()); + } + + return data_types; +} + + void Set::setHeader(const ColumnsWithTypeAndName & header) { std::lock_guard lock(rwlock); @@ -190,7 +209,16 @@ bool Set::insertFromColumns(const Columns & columns) bool inserted = insertFromColumns(columns, holder); if (inserted && fill_set_elements) - appendSetElements(holder); + { + if (max_elements_to_fill && max_elements_to_fill < data.getTotalRowCount()) + { + /// Drop filled elementes + fill_set_elements = false; + set_elements.clear(); + } + else + appendSetElements(holder); + } return inserted; } diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index c2931d79de0..9b76ab30a1b 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -30,9 +30,9 @@ public: /// (that is useful only for checking that some value is in the set and may not store the original values), /// store all set elements in explicit form. /// This is needed for subsequent use for index. - Set(const SizeLimits & limits_, bool fill_set_elements_, bool transform_null_in_) + Set(const SizeLimits & limits_, bool fill_set_elements_, size_t max_elements_to_fill_, bool transform_null_in_) : log(&Poco::Logger::get("Set")), - limits(limits_), fill_set_elements(fill_set_elements_), transform_null_in(transform_null_in_) + limits(limits_), fill_set_elements(fill_set_elements_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_) { } @@ -90,6 +90,8 @@ public: bool areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const; void checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const; + static DataTypes getElementTypes(const ColumnsWithTypeAndName & header, bool transform_null_in); + private: size_t keys_size = 0; Sizes key_sizes; @@ -126,6 +128,7 @@ private: /// Do we need to additionally store all elements of the set in explicit form for subsequent use for index. bool fill_set_elements; + size_t max_elements_to_fill; /// If true, insert NULL values to set. bool transform_null_in; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 913cf1c6ee3..d264810645a 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -110,7 +110,7 @@ public: subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); /// TODO - sets.addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + sets.addFromSubquery(set_key, std::move(subquery_for_set), settings, nullptr); //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index fe362f64b96..ebd58c3dc95 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -105,7 +105,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , column_names(column_names_) , max_rows_in_set(max_rows_in_set_) - , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, true)) + , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, 0, true)) , filtering_set(nullptr) , crosswise_connection(crosswise_connection_) , position(position_) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 38506412531..0909ee9f1eb 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -35,12 +35,14 @@ static ITransformingStep::Traits getTraits() CreatingSetStep::CreatingSetStep( const DataStream & input_stream_, String description_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, ContextPtr context_) : ITransformingStep(input_stream_, Block{}, getTraits()) , description(std::move(description_)) - , subquery_for_set(std::move(subquery_for_set_)) + , subquery_for_set(subquery_for_set_) + , set(std::move(set_)) , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { @@ -48,7 +50,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context->getPreparedSetsCache()); + pipeline.addCreatingSetsTransform(getOutputStream().header, subquery_for_set, std::move(set), network_transfer_limits, context->getPreparedSetsCache()); } void CreatingSetStep::updateOutputStream() diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 244bb27ba78..2a6cb43c45a 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -15,7 +15,8 @@ public: CreatingSetStep( const DataStream & input_stream_, String description_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, ContextPtr context_); @@ -30,7 +31,8 @@ private: void updateOutputStream() override; String description; - SubqueryForSet subquery_for_set; + SubqueryForSet & subquery_for_set; + FutureSetPtr set; SizeLimits network_transfer_limits; ContextPtr context; }; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 24e17ff210f..f6e6f7a191e 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -25,11 +25,13 @@ CreatingSetsTransform::~CreatingSetsTransform() = default; CreatingSetsTransform::CreatingSetsTransform( Block in_header_, Block out_header_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, PreparedSetsCachePtr prepared_sets_cache_) : IAccumulatingTransform(std::move(in_header_), std::move(out_header_)) - , subquery(std::move(subquery_for_set_)) + , subquery(subquery_for_set_) + , set(std::move(set_)) , network_transfer_limits(std::move(network_transfer_limits_)) , prepared_sets_cache(std::move(prepared_sets_cache_)) { @@ -61,6 +63,7 @@ void CreatingSetsTransform::startSubquery() auto from_cache = prepared_sets_cache->findOrPromiseToBuild(subquery.key); if (from_cache.index() == 0) { + LOG_TRACE(log, "Building set, key: {}", subquery.key); promise_to_build = std::move(std::get<0>(from_cache)); } else @@ -74,8 +77,8 @@ void CreatingSetsTransform::startSubquery() continue; } - subquery.promise_to_fill_set.set_value(ready_set); - subquery.set.reset(); + //subquery.promise_to_fill_set.set_value(ready_set); + subquery.set = ready_set; //.reset(); done_with_set = true; set_from_cache = true; } diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index ef586b0655c..27c330bdbc3 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -29,7 +29,8 @@ public: CreatingSetsTransform( Block in_header_, Block out_header_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, PreparedSetsCachePtr prepared_sets_cache_); @@ -42,7 +43,8 @@ public: Chunk generate() override; private: - SubqueryForSet subquery; + SubqueryForSet & subquery; + FutureSetPtr set; std::optional> promise_to_build; QueryPipeline table_out; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 4898501b980..08d90899f4a 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -569,14 +569,20 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe return left; } -void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache) +void QueryPipelineBuilder::addCreatingSetsTransform( + const Block & res_header, + SubqueryForSet & subquery_for_set, + FutureSetPtr set, + const SizeLimits & limits, + PreparedSetsCachePtr prepared_sets_cache) { resize(1); auto transform = std::make_shared( getHeader(), res_header, - std::move(subquery_for_set), + subquery_for_set, + std::move(set), limits, std::move(prepared_sets_cache)); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 85b6f5c6772..da8443a7e33 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -33,6 +33,9 @@ class TableJoin; class QueryPipelineBuilder; using QueryPipelineBuilderPtr = std::unique_ptr; +class FutureSet; +using FutureSetPtr = std::shared_ptr; + class PreparedSetsCache; using PreparedSetsCachePtr = std::shared_ptr; @@ -141,7 +144,12 @@ public: /// This is used for CreatingSets. void addPipelineBefore(QueryPipelineBuilder pipeline); - void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache); + void addCreatingSetsTransform( + const Block & res_header, + SubqueryForSet & subquery_for_set, + FutureSetPtr set, + const SizeLimits & limits, + PreparedSetsCachePtr prepared_sets_cache); PipelineExecutorPtr execute(); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index cc7ec45be6a..c9feb533ea3 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -353,34 +353,34 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( { const auto & prepared_sets = getTreeContext().getPreparedSets(); + /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information + /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets + /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check + /// that the types it was prepared with are compatible with the types of the primary key. + auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) + { + assert(indexes_mapping.size() == data_types.size()); + + for (size_t i = 0; i < indexes_mapping.size(); ++i) + { + if (indexes_mapping[i].tuple_index >= set_types.size()) + return false; + + auto lhs = recursiveRemoveLowCardinality(data_types[i]); + auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); + + if (!lhs->equals(*rhs)) + return false; + } + + return true; + }; + if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); - /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information - /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets - /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check - /// that the types it was prepared with are compatible with the types of the primary key. - auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) - { - assert(indexes_mapping.size() == data_types.size()); - - for (size_t i = 0; i < indexes_mapping.size(); ++i) - { - if (indexes_mapping[i].tuple_index >= set_types.size()) - return false; - - auto lhs = recursiveRemoveLowCardinality(data_types[i]); - auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); - - if (!lhs->equals(*rhs)) - return false; - } - - return true; - }; - auto tree_hash = ast_node->getTreeHash(); for (const auto & [key, future_set] : prepared_sets->getSets()) { @@ -392,7 +392,12 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( { const auto * node_without_alias = getNodeWithoutAlias(dag_node); if (node_without_alias->column) - return tryGetSetFromDAGNode(node_without_alias); + { + auto future_set = tryGetSetFromDAGNode(node_without_alias); + auto set_types = future_set->getTypes(); + if (types_match(future_set->getTypes())) + return future_set; + } } return nullptr; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 00b5dbfc5e3..da94c87b69e 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -147,7 +147,7 @@ StorageSet::StorageSet( const String & comment, bool persistent_) : StorageSetOrJoinBase{disk_, relative_path_, table_id_, columns_, constraints_, comment, persistent_} - , set(std::make_shared(SizeLimits(), false, true)) + , set(std::make_shared(SizeLimits(), false, 0, true)) { Block header = getInMemoryMetadataPtr()->getSampleBlock(); set->setHeader(header.getColumnsWithTypeAndName()); @@ -176,7 +176,7 @@ void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn Block header = metadata_snapshot->getSampleBlock(); increment = 0; - set = std::make_shared(SizeLimits(), false, true); + set = std::make_shared(SizeLimits(), false, 0, true); set->setHeader(header.getColumnsWithTypeAndName()); } diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.reference b/tests/queries/0_stateless/01786_explain_merge_tree.reference index e6628813dbd..8d3954484dd 100644 --- a/tests/queries/0_stateless/01786_explain_merge_tree.reference +++ b/tests/queries/0_stateless/01786_explain_merge_tree.reference @@ -10,7 +10,7 @@ Keys: y bitAnd(z, 3) - Condition: and((bitAnd(z, 3) not in [1, 1]), and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))) + Condition: and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])) Parts: 3/4 Granules: 10/11 PrimaryKey @@ -46,7 +46,7 @@ { "Type": "Partition", "Keys": ["y", "bitAnd(z, 3)"], - "Condition": "and((bitAnd(z, 3) not in [1, 1]), and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])))", + "Condition": "and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))", "Initial Parts": 4, "Selected Parts": 3, "Initial Granules": 11, From 6b2c906dfd57a895b93605a4bfb07fda7a72945d Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Jun 2023 19:03:35 -0400 Subject: [PATCH 0965/2223] add docs for boundingRatio --- .../aggregate-functions/reference/boundrat.md | 44 ++++++ .../aggregate-functions/reference/index.md | 131 +++++++++--------- 2 files changed, 110 insertions(+), 65 deletions(-) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/boundrat.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/boundrat.md b/docs/en/sql-reference/aggregate-functions/reference/boundrat.md new file mode 100644 index 00000000000..f3907af8030 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/boundrat.md @@ -0,0 +1,44 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/boundingRatio +sidebar_position: 2 +title: boundingRatio +--- + +Aggregate function that calculates the slope between the leftmost and rightmost points across a group of values. + +Example: + +Sample data: +```sql +SELECT + number, + number * 1.5 +FROM numbers(10) +``` +```response +┌─number─┬─multiply(number, 1.5)─┐ +│ 0 │ 0 │ +│ 1 │ 1.5 │ +│ 2 │ 3 │ +│ 3 │ 4.5 │ +│ 4 │ 6 │ +│ 5 │ 7.5 │ +│ 6 │ 9 │ +│ 7 │ 10.5 │ +│ 8 │ 12 │ +│ 9 │ 13.5 │ +└────────┴───────────────────────┘ +``` + +The boundingRatio() function returns the slope of the line between the leftmost and rightmost points, in the above data these points are `(0,0)` and `(9,13.5)`. + +```sql +SELECT boundingRatio(number, number * 1.5) +FROM numbers(10) +``` +```response +┌─boundingRatio(number, multiply(number, 1.5))─┐ +│ 1.5 │ +└──────────────────────────────────────────────┘ +``` + diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index 50208352f38..17ef494e9ad 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -9,74 +9,75 @@ toc_hidden: true Standard aggregate functions: -- [count](../../../sql-reference/aggregate-functions/reference/count.md) -- [min](../../../sql-reference/aggregate-functions/reference/min.md) -- [max](../../../sql-reference/aggregate-functions/reference/max.md) -- [sum](../../../sql-reference/aggregate-functions/reference/sum.md) -- [avg](../../../sql-reference/aggregate-functions/reference/avg.md) -- [any](../../../sql-reference/aggregate-functions/reference/any.md) -- [stddevPop](../../../sql-reference/aggregate-functions/reference/stddevpop.md) -- [stddevSamp](../../../sql-reference/aggregate-functions/reference/stddevsamp.md) -- [varPop](../../../sql-reference/aggregate-functions/reference/varpop.md) -- [varSamp](../../../sql-reference/aggregate-functions/reference/varsamp.md) -- [covarPop](../../../sql-reference/aggregate-functions/reference/covarpop.md) -- [covarSamp](../../../sql-reference/aggregate-functions/reference/covarsamp.md) +- [count](/docs/en/sql-reference/aggregate-functions/reference/count.md) +- [min](/docs/en/sql-reference/aggregate-functions/reference/min.md) +- [max](/docs/en/sql-reference/aggregate-functions/reference/max.md) +- [sum](/docs/en/sql-reference/aggregate-functions/reference/sum.md) +- [avg](/docs/en/sql-reference/aggregate-functions/reference/avg.md) +- [any](/docs/en/sql-reference/aggregate-functions/reference/any.md) +- [stddevPop](/docs/en/sql-reference/aggregate-functions/reference/stddevpop.md) +- [stddevSamp](/docs/en/sql-reference/aggregate-functions/reference/stddevsamp.md) +- [varPop](/docs/en/sql-reference/aggregate-functions/reference/varpop.md) +- [varSamp](/docs/en/sql-reference/aggregate-functions/reference/varsamp.md) +- [covarPop](/docs/en/sql-reference/aggregate-functions/reference/covarpop.md) +- [covarSamp](/docs/en/sql-reference/aggregate-functions/reference/covarsamp.md) ClickHouse-specific aggregate functions: -- [anyHeavy](../../../sql-reference/aggregate-functions/reference/anyheavy.md) -- [anyLast](../../../sql-reference/aggregate-functions/reference/anylast.md) -- [first_value](../../../sql-reference/aggregate-functions/reference/first_value.md) -- [last_value](../../../sql-reference/aggregate-functions/reference/last_value.md) -- [argMin](../../../sql-reference/aggregate-functions/reference/argmin.md) -- [argMax](../../../sql-reference/aggregate-functions/reference/argmax.md) -- [avgWeighted](../../../sql-reference/aggregate-functions/reference/avgweighted.md) -- [topK](../../../sql-reference/aggregate-functions/reference/topk.md) -- [topKWeighted](../../../sql-reference/aggregate-functions/reference/topkweighted.md) -- [groupArray](../../../sql-reference/aggregate-functions/reference/grouparray.md) -- [groupArrayLast](../../../sql-reference/aggregate-functions/reference/grouparraylast.md) -- [groupUniqArray](../../../sql-reference/aggregate-functions/reference/groupuniqarray.md) -- [groupArrayInsertAt](../../../sql-reference/aggregate-functions/reference/grouparrayinsertat.md) -- [groupArrayMovingAvg](../../../sql-reference/aggregate-functions/reference/grouparraymovingavg.md) -- [groupArrayMovingSum](../../../sql-reference/aggregate-functions/reference/grouparraymovingsum.md) -- [groupBitAnd](../../../sql-reference/aggregate-functions/reference/groupbitand.md) -- [groupBitOr](../../../sql-reference/aggregate-functions/reference/groupbitor.md) -- [groupBitXor](../../../sql-reference/aggregate-functions/reference/groupbitxor.md) -- [groupBitmap](../../../sql-reference/aggregate-functions/reference/groupbitmap.md) -- [groupBitmapAnd](../../../sql-reference/aggregate-functions/reference/groupbitmapand.md) -- [groupBitmapOr](../../../sql-reference/aggregate-functions/reference/groupbitmapor.md) -- [groupBitmapXor](../../../sql-reference/aggregate-functions/reference/groupbitmapxor.md) -- [sumWithOverflow](../../../sql-reference/aggregate-functions/reference/sumwithoverflow.md) -- [sumMap](../../../sql-reference/aggregate-functions/reference/summap.md) -- [minMap](../../../sql-reference/aggregate-functions/reference/minmap.md) -- [maxMap](../../../sql-reference/aggregate-functions/reference/maxmap.md) -- [skewSamp](../../../sql-reference/aggregate-functions/reference/skewsamp.md) -- [skewPop](../../../sql-reference/aggregate-functions/reference/skewpop.md) -- [kurtSamp](../../../sql-reference/aggregate-functions/reference/kurtsamp.md) -- [kurtPop](../../../sql-reference/aggregate-functions/reference/kurtpop.md) -- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md) -- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md) -- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md) -- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md) -- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md) -- [uniqTheta](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md) -- [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md) -- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md) -- [quantileExact](../../../sql-reference/aggregate-functions/reference/quantileexact.md) -- [quantileExactLow](../../../sql-reference/aggregate-functions/reference/quantileexact.md#quantileexactlow) -- [quantileExactHigh](../../../sql-reference/aggregate-functions/reference/quantileexact.md#quantileexacthigh) -- [quantileExactWeighted](../../../sql-reference/aggregate-functions/reference/quantileexactweighted.md) -- [quantileTiming](../../../sql-reference/aggregate-functions/reference/quantiletiming.md) -- [quantileTimingWeighted](../../../sql-reference/aggregate-functions/reference/quantiletimingweighted.md) -- [quantileDeterministic](../../../sql-reference/aggregate-functions/reference/quantiledeterministic.md) -- [quantileTDigest](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) -- [quantileTDigestWeighted](../../../sql-reference/aggregate-functions/reference/quantiletdigestweighted.md) -- [quantileBFloat16](../../../sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16) -- [quantileBFloat16Weighted](../../../sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16weighted) -- [simpleLinearRegression](../../../sql-reference/aggregate-functions/reference/simplelinearregression.md) -- [stochasticLinearRegression](../../../sql-reference/aggregate-functions/reference/stochasticlinearregression.md) -- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md) -- [categoricalInformationValue](../../../sql-reference/aggregate-functions/reference/categoricalinformationvalue.md) +- [anyHeavy](/docs/en/sql-reference/aggregate-functions/reference/anyheavy.md) +- [anyLast](/docs/en/sql-reference/aggregate-functions/reference/anylast.md) +- [boundingRatio](/docs/en/sql-reference/aggregate-functions/reference/boundrat.md) +- [first_value](/docs/en/sql-reference/aggregate-functions/reference/first_value.md) +- [last_value](/docs/en/sql-reference/aggregate-functions/reference/last_value.md) +- [argMin](/docs/en/sql-reference/aggregate-functions/reference/argmin.md) +- [argMax](/docs/en/sql-reference/aggregate-functions/reference/argmax.md) +- [avgWeighted](/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md) +- [topK](/docs/en/sql-reference/aggregate-functions/reference/topk.md) +- [topKWeighted](/docs/en/sql-reference/aggregate-functions/reference/topkweighted.md) +- [groupArray](/docs/en/sql-reference/aggregate-functions/reference/grouparray.md) +- [groupArrayLast](/docs/en/sql-reference/aggregate-functions/reference/grouparraylast.md) +- [groupUniqArray](/docs/en/sql-reference/aggregate-functions/reference/groupuniqarray.md) +- [groupArrayInsertAt](/docs/en/sql-reference/aggregate-functions/reference/grouparrayinsertat.md) +- [groupArrayMovingAvg](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingavg.md) +- [groupArrayMovingSum](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingsum.md) +- [groupBitAnd](/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md) +- [groupBitOr](/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md) +- [groupBitXor](/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md) +- [groupBitmap](/docs/en/sql-reference/aggregate-functions/reference/groupbitmap.md) +- [groupBitmapAnd](/docs/en/sql-reference/aggregate-functions/reference/groupbitmapand.md) +- [groupBitmapOr](/docs/en/sql-reference/aggregate-functions/reference/groupbitmapor.md) +- [groupBitmapXor](/docs/en/sql-reference/aggregate-functions/reference/groupbitmapxor.md) +- [sumWithOverflow](/docs/en/sql-reference/aggregate-functions/reference/sumwithoverflow.md) +- [sumMap](/docs/en/sql-reference/aggregate-functions/reference/summap.md) +- [minMap](/docs/en/sql-reference/aggregate-functions/reference/minmap.md) +- [maxMap](/docs/en/sql-reference/aggregate-functions/reference/maxmap.md) +- [skewSamp](/docs/en/sql-reference/aggregate-functions/reference/skewsamp.md) +- [skewPop](/docs/en/sql-reference/aggregate-functions/reference/skewpop.md) +- [kurtSamp](/docs/en/sql-reference/aggregate-functions/reference/kurtsamp.md) +- [kurtPop](/docs/en/sql-reference/aggregate-functions/reference/kurtpop.md) +- [uniq](/docs/en/sql-reference/aggregate-functions/reference/uniq.md) +- [uniqExact](/docs/en/sql-reference/aggregate-functions/reference/uniqexact.md) +- [uniqCombined](/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md) +- [uniqCombined64](/docs/en/sql-reference/aggregate-functions/reference/uniqcombined64.md) +- [uniqHLL12](/docs/en/sql-reference/aggregate-functions/reference/uniqhll12.md) +- [uniqTheta](/docs/en/sql-reference/aggregate-functions/reference/uniqthetasketch.md) +- [quantile](/docs/en/sql-reference/aggregate-functions/reference/quantile.md) +- [quantiles](/docs/en/sql-reference/aggregate-functions/reference/quantiles.md) +- [quantileExact](/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md) +- [quantileExactLow](/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md#quantileexactlow) +- [quantileExactHigh](/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md#quantileexacthigh) +- [quantileExactWeighted](/docs/en/sql-reference/aggregate-functions/reference/quantileexactweighted.md) +- [quantileTiming](/docs/en/sql-reference/aggregate-functions/reference/quantiletiming.md) +- [quantileTimingWeighted](/docs/en/sql-reference/aggregate-functions/reference/quantiletimingweighted.md) +- [quantileDeterministic](/docs/en/sql-reference/aggregate-functions/reference/quantiledeterministic.md) +- [quantileTDigest](/docs/en/sql-reference/aggregate-functions/reference/quantiletdigest.md) +- [quantileTDigestWeighted](/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md) +- [quantileBFloat16](/docs/en/sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16) +- [quantileBFloat16Weighted](/docs/en/sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16weighted) +- [simpleLinearRegression](/docs/en/sql-reference/aggregate-functions/reference/simplelinearregression.md) +- [stochasticLinearRegression](/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md) +- [stochasticLogisticRegression](/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md) +- [categoricalInformationValue](/docs/en/sql-reference/aggregate-functions/reference/categoricalinformationvalue.md) - [contingency](./contingency.md) - [cramersV](./cramersv.md) - [cramersVBiasCorrected](./cramersvbiascorrected.md) From e7868e576c71de07d7dd4d921382d4cd549d6493 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 2 Jun 2023 00:50:14 +0000 Subject: [PATCH 0966/2223] Don't crash if config doesn't have logger section --- src/Loggers/Loggers.cpp | 2 +- src/Loggers/Loggers.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 645ae5dcc7a..0c3a7bd615d 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -51,7 +51,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log #endif auto current_logger = config.getString("logger", ""); - if (config_logger == current_logger) + if (config_logger.has_value() && *config_logger == current_logger) return; config_logger = current_logger; diff --git a/src/Loggers/Loggers.h b/src/Loggers/Loggers.h index 31a215aa9ce..ebc10954b94 100644 --- a/src/Loggers/Loggers.h +++ b/src/Loggers/Loggers.h @@ -39,7 +39,7 @@ private: Poco::AutoPtr syslog_channel; /// Previous value of logger element in config. It is used to reinitialize loggers whenever the value changed. - std::string config_logger; + std::optional config_logger; #ifndef WITHOUT_TEXT_LOG std::weak_ptr text_log; From c5165c2236bfceed4ec9246d6964fd2f77f68a07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Jun 2023 03:25:12 +0200 Subject: [PATCH 0967/2223] More thorough check for the availability of TaskStats --- src/Common/TaskStatsInfoGetter.cpp | 5 ++++- src/Common/ThreadStatus.h | 1 - 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index 25030ee9670..d21229609dd 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -9,6 +9,7 @@ #include "hasLinuxCapability.h" #include +#include #include #include @@ -202,10 +203,12 @@ bool checkPermissionsImpl() /// Check that we can successfully initialize TaskStatsInfoGetter. /// It will ask about family id through Netlink. /// On some LXC containers we have capability but we still cannot use Netlink. + /// There is an evidence that Linux fedora-riscv 6.1.22 gives something strange instead of the expected result. try { - TaskStatsInfoGetter(); + ::taskstats stats{}; + TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); } catch (const Exception & e) { diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 600dfc56d2b..061959d9f1f 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -37,7 +37,6 @@ class QueryThreadLog; class TasksStatsCounters; struct RUsageCounters; struct PerfEventsCounters; -class TaskStatsInfoGetter; class InternalTextLogsQueue; struct ViewRuntimeData; class QueryViewsLog; From b51064a5081ed9339e30650ccd6466262b258522 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 5 May 2023 03:11:51 +0000 Subject: [PATCH 0968/2223] Get rid of SeekableReadBufferFactory, add SeekableReadBuffer::readBigAt() instead --- src/Core/Settings.h | 1 - src/Formats/FormatFactory.cpp | 102 ++----- src/Formats/FormatFactory.h | 56 +--- src/Formats/FormatSettings.h | 2 - src/IO/MMapReadBufferFromFileDescriptor.cpp | 11 + src/IO/MMapReadBufferFromFileDescriptor.h | 3 + src/IO/ParallelReadBuffer.cpp | 258 ++++++++---------- src/IO/ParallelReadBuffer.h | 45 +-- src/IO/ReadBufferFromFileDescriptor.cpp | 48 +++- src/IO/ReadBufferFromFileDescriptor.h | 11 + src/IO/ReadBufferFromS3.cpp | 187 +++++++------ src/IO/ReadBufferFromS3.h | 49 +--- src/IO/ReadWriteBufferFromHTTP.cpp | 209 +++++++------- src/IO/ReadWriteBufferFromHTTP.h | 61 +---- src/IO/SeekableReadBuffer.cpp | 46 ++++ src/IO/SeekableReadBuffer.h | 52 ++-- src/IO/WithFileName.cpp | 2 +- src/IO/WithFileSize.cpp | 8 - .../Formats/Impl/ArrowBufferedStreams.cpp | 98 ++----- .../Formats/Impl/ArrowBufferedStreams.h | 19 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 24 +- .../Formats/Impl/ParquetBlockInputFormat.h | 6 +- src/Storages/StorageS3.cpp | 43 +-- src/Storages/StorageS3.h | 8 +- src/Storages/StorageURL.cpp | 78 +++--- src/Storages/StorageURL.h | 3 +- .../test_redirect_url_storage/test.py | 4 +- 27 files changed, 658 insertions(+), 776 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 566c9f32484..2747094451d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -832,7 +832,6 @@ class IColumn; M(Bool, input_format_orc_case_insensitive_column_matching, false, "Ignore case when matching ORC columns with CH columns.", 0) \ M(Bool, input_format_parquet_import_nested, false, "Allow to insert array of structs into Nested table in Parquet input format.", 0) \ M(Bool, input_format_parquet_case_insensitive_column_matching, false, "Ignore case when matching Parquet columns with CH columns.", 0) \ - /* TODO: Consider unifying this with https://github.com/ClickHouse/ClickHouse/issues/38755 */ \ M(Bool, input_format_parquet_preserve_order, false, "Avoid reordering rows when reading from Parquet files. Usually makes it much slower.", 0) \ M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \ M(Bool, input_format_orc_allow_missing_columns, false, "Allow missing columns while reading ORC input formats", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index dd6252b96f1..586e1bb7251 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -221,68 +221,16 @@ template FormatSettings getFormatSettings(ContextPtr context, const Se InputFormatPtr FormatFactory::getInput( const String & name, - ReadBuffer & buf, + ReadBuffer & _buf, const Block & sample, ContextPtr context, UInt64 max_block_size, - const std::optional & format_settings, - std::optional max_parsing_threads) const -{ - return getInputImpl( - name, - nullptr, - &buf, - sample, - context, - max_block_size, - /* is_remote_fs */ false, - CompressionMethod::None, - format_settings, - /* max_download_threads */ 1, - max_parsing_threads); -} - -InputFormatPtr FormatFactory::getInputRandomAccess( - const String & name, - SeekableReadBufferFactoryPtr buf_factory, - const Block & sample, - ContextPtr context, - UInt64 max_block_size, - bool is_remote_fs, - CompressionMethod compression, - const std::optional & format_settings, - std::optional max_download_threads, - std::optional max_parsing_threads) const -{ - return getInputImpl( - name, - std::move(buf_factory), - nullptr, - sample, - context, - max_block_size, - is_remote_fs, - compression, - format_settings, - max_download_threads, - max_parsing_threads); -} - -InputFormatPtr FormatFactory::getInputImpl( - const String & name, - // exactly one of the following two is nullptr - SeekableReadBufferFactoryPtr buf_factory, - ReadBuffer * _buf, - const Block & sample, - ContextPtr context, - UInt64 max_block_size, - bool is_remote_fs, - CompressionMethod compression, const std::optional & _format_settings, + std::optional _max_parsing_threads, std::optional _max_download_threads, - std::optional _max_parsing_threads) const + bool is_remote_fs, + CompressionMethod compression) const { - chassert((!_buf) != (!buf_factory)); const auto& creators = getCreators(name); if (!creators.input_creator && !creators.random_access_input_creator) throw Exception(ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT, "Format {} is not suitable for input", name); @@ -302,14 +250,12 @@ InputFormatPtr FormatFactory::getInputImpl( if (context->hasQueryContext() && settings.log_queries) context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); - // Prepare a read buffer. + // Add ParallelReadBuffer and decompression if needed. - std::unique_ptr owned_buf; - if (buf_factory) - owned_buf = prepareReadBuffer(buf_factory, compression, creators, format_settings, settings, max_download_threads); - auto * buf = owned_buf ? owned_buf.get() : _buf; + auto owned_buf = wrapReadBufferIfNeeded(_buf, compression, creators, format_settings, settings, is_remote_fs, max_download_threads); + auto & buf = owned_buf ? *owned_buf : _buf; - // Decide whether to use parallel ParallelParsingInputFormat. + // Decide whether to use ParallelParsingInputFormat. bool parallel_parsing = max_parsing_threads > 1 && settings.input_format_parallel_parsing && creators.file_segmentation_engine && !creators.random_access_input_creator; @@ -322,7 +268,7 @@ InputFormatPtr FormatFactory::getInputImpl( { const auto & non_trivial_prefix_and_suffix_checker = creators.non_trivial_prefix_and_suffix_checker; /// Disable parallel parsing for input formats with non-trivial readPrefix() and readSuffix(). - if (non_trivial_prefix_and_suffix_checker && non_trivial_prefix_and_suffix_checker(*buf)) + if (non_trivial_prefix_and_suffix_checker && non_trivial_prefix_and_suffix_checker(buf)) parallel_parsing = false; } @@ -340,7 +286,7 @@ InputFormatPtr FormatFactory::getInputImpl( { return input_getter(input, sample, row_input_format_params, format_settings); }; ParallelParsingInputFormat::Params params{ - *buf, sample, parser_creator, creators.file_segmentation_engine, name, max_parsing_threads, + buf, sample, parser_creator, creators.file_segmentation_engine, name, max_parsing_threads, settings.min_chunk_bytes_for_parallel_parsing, max_block_size, context->getApplicationType() == Context::ApplicationType::SERVER}; format = std::make_shared(params); @@ -349,7 +295,6 @@ InputFormatPtr FormatFactory::getInputImpl( { format = creators.random_access_input_creator( buf, - std::move(buf_factory), sample, format_settings, context->getReadSettings(), @@ -359,7 +304,7 @@ InputFormatPtr FormatFactory::getInputImpl( } else { - format = creators.input_creator(*buf, sample, row_input_format_params, format_settings); + format = creators.input_creator(buf, sample, row_input_format_params, format_settings); } if (owned_buf) @@ -375,26 +320,28 @@ InputFormatPtr FormatFactory::getInputImpl( return format; } -std::unique_ptr FormatFactory::prepareReadBuffer( - SeekableReadBufferFactoryPtr & buf_factory, +std::unique_ptr FormatFactory::wrapReadBufferIfNeeded( + ReadBuffer & buf, CompressionMethod compression, const Creators & creators, const FormatSettings & format_settings, const Settings & settings, + bool is_remote_fs, size_t max_download_threads) const { std::unique_ptr res; - bool parallel_read = max_download_threads > 1 && buf_factory && format_settings.seekable_read; + bool parallel_read = is_remote_fs && max_download_threads > 1 && format_settings.seekable_read && isBufferWithFileSize(buf); if (creators.random_access_input_creator) parallel_read &= compression != CompressionMethod::None; + size_t file_size = 0; if (parallel_read) { try { - parallel_read = buf_factory->checkIfActuallySeekable() - && buf_factory->getFileSize() >= 2 * settings.max_download_buffer_size; + file_size = getFileSizeFromReadBuffer(buf); + parallel_read = file_size >= 2 * settings.max_download_buffer_size; } catch (const Poco::Exception & e) { @@ -415,23 +362,18 @@ std::unique_ptr FormatFactory::prepareReadBuffer( max_download_threads, settings.max_download_buffer_size); - res = std::make_unique( - std::move(buf_factory), - threadPoolCallbackRunner(IOThreadPool::get(), "ParallelRead"), - max_download_threads, - settings.max_download_buffer_size); + res = wrapInParallelReadBufferIfSupported( + buf, threadPoolCallbackRunner(IOThreadPool::get(), "ParallelRead"), + max_download_threads, settings.max_download_buffer_size, file_size); } if (compression != CompressionMethod::None) { if (!res) - res = buf_factory->getReader(); // NOLINT + res = wrapReadBufferReference(buf); res = wrapReadBufferWithCompressionMethod(std::move(res), compression, static_cast(settings.zstd_window_log_max)); } - if (!creators.random_access_input_creator && !res) - res = buf_factory->getReader(); - return res; } diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 677e34845d8..1d258beca8d 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -90,15 +90,11 @@ private: const FormatSettings & settings)>; // Incompatible with FileSegmentationEngine. - // When created using SeekableReadBufferFactoryPtr, the IInputFormat doesn't support - // resetParser() and setReadBuffer(). // // In future we may also want to pass some information about WHERE conditions (SelectQueryInfo?) // and get some information about projections (min/max/count per column per row group). using RandomAccessInputCreator = std::function & format_settings = std::nullopt, - std::optional max_parsing_threads = std::nullopt) const; - - // Format parser from a random-access source (factory of seekable read buffers). - // Parallelizes both parsing and reading when possible. - // Prefer this over getInput() when reading from random-access source like file or HTTP. - InputFormatPtr getInputRandomAccess( - const String & name, - SeekableReadBufferFactoryPtr buf_factory, - const Block & sample, - ContextPtr context, - UInt64 max_block_size, - bool is_remote_fs, - CompressionMethod compression, - // if nullopt, getFormatSettings(context) is used - const std::optional & format_settings = std::nullopt, + std::optional max_parsing_threads = std::nullopt, std::optional max_download_threads = std::nullopt, - std::optional max_parsing_threads = std::nullopt) const; + // affects things like buffer sizes and parallel reading + bool is_remote_fs = false, + // allows to do: buf -> parallel read -> decompression, + // because parallel read after decompression is not possible + CompressionMethod compression = CompressionMethod::None) const; /// Checks all preconditions. Returns ordinary format if parallel formatting cannot be done. OutputFormatPtr getOutputFormatParallelIfPossible( @@ -272,28 +260,14 @@ private: const Creators & getCreators(const String & name) const; - InputFormatPtr getInputImpl( - const String & name, - // exactly one of the following two is nullptr - SeekableReadBufferFactoryPtr buf_factory, - ReadBuffer * buf, - const Block & sample, - ContextPtr context, - UInt64 max_block_size, - bool is_remote_fs, - CompressionMethod compression, - const std::optional & format_settings, - std::optional max_download_threads, - std::optional max_parsing_threads) const; - - // Creates a ReadBuffer to give to an input format. - // Returns nullptr if we should give it the whole factory. - std::unique_ptr prepareReadBuffer( - SeekableReadBufferFactoryPtr & buf_factory, + // Creates a ReadBuffer to give to an input format. Returns nullptr if we should use `buf` directly. + std::unique_ptr wrapReadBufferIfNeeded( + ReadBuffer & buf, CompressionMethod compression, const Creators & creators, const FormatSettings & format_settings, const Settings & settings, + bool is_remote_fs, size_t max_download_threads) const; }; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index c88af650671..e332bd749a1 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -214,8 +214,6 @@ struct FormatSettings std::unordered_set skip_row_groups = {}; bool output_string_as_string = false; bool output_fixed_string_as_fixed_byte_array = true; - // TODO: This should probably be shared among all formats and with - // https://github.com/ClickHouse/ClickHouse/issues/38755 bool preserve_order = false; UInt64 max_block_size = 8192; ParquetVersion output_version; diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index c0eb73f8638..9b1c132cc01 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -91,4 +91,15 @@ size_t MMapReadBufferFromFileDescriptor::getFileSize() { return getSizeFromFileDescriptor(getFD(), getFileName()); } + +size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) +{ + if (offset >= mapped.getLength()) + return 0; + + n = std::min(n, mapped.getLength() - offset); + memcpy(to, mapped.getData() + offset, n); + return n; +} + } diff --git a/src/IO/MMapReadBufferFromFileDescriptor.h b/src/IO/MMapReadBufferFromFileDescriptor.h index 1a4bcd4f3ed..2a039e04971 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.h +++ b/src/IO/MMapReadBufferFromFileDescriptor.h @@ -39,6 +39,9 @@ public: int getFD() const; size_t getFileSize() override; + + size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) override; + bool supportsReadAt() override { return true; } }; } diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index fff02db1bd6..07240ab3a4f 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -13,51 +14,44 @@ namespace ErrorCodes } -// A subrange of the input, read by one SeekableReadBuffer. +// A subrange of the input, read by one thread. struct ParallelReadBuffer::ReadWorker { - ReadWorker(std::unique_ptr reader_, size_t offset_, size_t size) - : reader(std::move(reader_)), offset(offset_), bytes_left(size), range_end(offset + bytes_left) + ReadWorker(SeekableReadBuffer & input_, size_t offset, size_t size) + : input(input_), start_offset(offset), segment(size) { - assert(bytes_left); + chassert(size); + chassert(segment.size() == size); } - auto hasSegment() const { return current_segment_index < segments.size(); } + bool hasBytesToConsume() const { return bytes_produced > bytes_consumed; } + bool hasBytesToProduce() const { return bytes_produced < segment.size(); } - auto nextSegment() - { - assert(hasSegment()); - auto next_segment = std::move(segments[current_segment_index]); - ++current_segment_index; - offset += next_segment.size(); - return next_segment; - } + SeekableReadBuffer & input; + const size_t start_offset; // start of the segment - std::unique_ptr reader; - // Reader thread produces segments, nextImpl() consumes them. - std::vector> segments; // segments that were produced - size_t current_segment_index = 0; // first segment that's not consumed - bool finished{false}; // no more segments will be produced - size_t offset; // start of segments[current_segment_idx] - size_t bytes_left; // bytes left to produce above segments end - size_t range_end; // segments end + bytes_left, i.e. how far this worker will read - - // segments[current_segment_idx..end] range_end - // |-------------|--------------------------------------|------------| - // offset bytes_left + Memory<> segment; + /// Reader thread produces data, nextImpl() consumes it. + /// segment[bytes_consumed..bytes_produced-1] is data waiting to be picked up by nextImpl() + /// segment[bytes_produced..] needs to be read from the input ReadBuffer + size_t bytes_produced = 0; + size_t bytes_consumed = 0; std::atomic_bool cancel{false}; std::mutex worker_mutex; }; ParallelReadBuffer::ParallelReadBuffer( - std::unique_ptr reader_factory_, ThreadPoolCallbackRunner schedule_, size_t max_working_readers_, size_t range_step_) + SeekableReadBuffer & input_, ThreadPoolCallbackRunner schedule_, size_t max_working_readers_, size_t range_step_, size_t file_size_) : SeekableReadBuffer(nullptr, 0) , max_working_readers(max_working_readers_) , schedule(std::move(schedule_)) - , reader_factory(std::move(reader_factory_)) + , input(input_) + , file_size(file_size_) , range_step(std::max(1ul, range_step_)) { + LOG_TRACE(&Poco::Logger::get("ParallelReadBuffer"), "Parallel reading is used"); + try { addReaders(); @@ -71,22 +65,15 @@ ParallelReadBuffer::ParallelReadBuffer( bool ParallelReadBuffer::addReaderToPool() { - size_t file_size = reader_factory->getFileSize(); if (next_range_start >= file_size) return false; size_t range_start = next_range_start; size_t size = std::min(range_step, file_size - range_start); next_range_start += size; - auto reader = reader_factory->getReader(); - if (!reader) - { - return false; - } + auto worker = read_workers.emplace_back(std::make_shared(input, range_start, size)); - auto worker = read_workers.emplace_back(std::make_shared(std::move(reader), range_start, size)); - - ++active_working_reader; + ++active_working_readers; schedule([this, my_worker = std::move(worker)]() mutable { readerThreadFunction(std::move(my_worker)); }, Priority{}); return true; @@ -116,9 +103,9 @@ off_t ParallelReadBuffer::seek(off_t offset, int whence) } const auto offset_is_in_range - = [&](const auto & worker) { return static_cast(offset) >= worker->offset && static_cast(offset) < worker->range_end; }; + = [&](const auto & worker) { return static_cast(offset) >= worker->start_offset && static_cast(offset) < worker->start_offset + worker->segment.size(); }; - while (!read_workers.empty() && (offset < current_position || !offset_is_in_range(read_workers.front()))) + while (!read_workers.empty() && !offset_is_in_range(read_workers.front())) { read_workers.front()->cancel = true; read_workers.pop_front(); @@ -126,32 +113,31 @@ off_t ParallelReadBuffer::seek(off_t offset, int whence) if (!read_workers.empty()) { - auto & front_worker = read_workers.front(); - current_position = front_worker->offset; + auto & w = read_workers.front(); + size_t diff = static_cast(offset) - w->start_offset; while (true) { - std::unique_lock lock{front_worker->worker_mutex}; - next_condvar.wait(lock, [&] { return emergency_stop || front_worker->hasSegment(); }); + std::unique_lock lock{w->worker_mutex}; if (emergency_stop) handleEmergencyStop(); - auto next_segment = front_worker->nextSegment(); - current_position += next_segment.size(); - if (offset < current_position) + if (w->bytes_produced > diff) { - current_segment = std::move(next_segment); - working_buffer = internal_buffer = Buffer(current_segment.data(), current_segment.data() + current_segment.size()); - pos = working_buffer.end() - (current_position - offset); + working_buffer = internal_buffer = Buffer( + w->segment.data() + diff, w->segment.data() + w->bytes_produced); + w->bytes_consumed = w->bytes_produced; + current_position += w->start_offset + w->bytes_consumed; addReaders(); return offset; } + + next_condvar.wait_for(lock, std::chrono::seconds(10)); } } finishAndWait(); - all_completed = false; read_workers.clear(); next_range_start = offset; @@ -166,7 +152,7 @@ off_t ParallelReadBuffer::seek(off_t offset, int whence) size_t ParallelReadBuffer::getFileSize() { - return reader_factory->getFileSize(); + return file_size; } off_t ParallelReadBuffer::getPosition() @@ -174,17 +160,6 @@ off_t ParallelReadBuffer::getPosition() return current_position - available(); } -bool ParallelReadBuffer::currentWorkerReady() const -{ - assert(!read_workers.empty()); - return read_workers.front()->finished || read_workers.front()->hasSegment(); -} - -bool ParallelReadBuffer::currentWorkerCompleted() const -{ - return read_workers.front()->finished && !read_workers.front()->hasSegment(); -} - void ParallelReadBuffer::handleEmergencyStop() { // this can only be called from the main thread when there is an exception @@ -194,106 +169,99 @@ void ParallelReadBuffer::handleEmergencyStop() bool ParallelReadBuffer::nextImpl() { - if (all_completed) - return false; - while (true) { - std::unique_lock lock{read_workers.front()->worker_mutex}; - next_condvar.wait( - lock, - [this]() - { - /// Check if no more readers left or current reader can be processed - return emergency_stop || currentWorkerReady(); - }); - - bool worker_removed = false; - /// Remove completed units - while (currentWorkerCompleted() && !emergency_stop) - { - lock.unlock(); - read_workers.pop_front(); - worker_removed = true; - - if (read_workers.empty()) - break; - - lock = std::unique_lock{read_workers.front()->worker_mutex}; - } - - if (emergency_stop) - handleEmergencyStop(); - - if (worker_removed) - addReaders(); - /// All readers processed, stop if (read_workers.empty()) { - all_completed = true; + chassert(next_range_start >= file_size); return false; } - auto & front_worker = read_workers.front(); - /// Read data from first segment of the first reader - if (front_worker->hasSegment()) + auto * w = read_workers.front().get(); + + std::unique_lock lock{w->worker_mutex}; + + if (emergency_stop) + handleEmergencyStop(); // throws + + /// Read data from front reader + if (w->bytes_produced > w->bytes_consumed) { - current_segment = front_worker->nextSegment(); - if (currentWorkerCompleted()) - { - lock.unlock(); - read_workers.pop_front(); - all_completed = !addReaderToPool() && read_workers.empty(); - } - break; + chassert(w->start_offset + w->bytes_consumed == static_cast(current_position)); + + working_buffer = internal_buffer = Buffer( + w->segment.data() + w->bytes_consumed, w->segment.data() + w->bytes_produced); + current_position += working_buffer.size(); + w->bytes_consumed = w->bytes_produced; + + return true; } + + /// Front reader is done, remove it and add another + if (!w->hasBytesToProduce()) + { + lock.unlock(); + read_workers.pop_front(); + addReaders(); + + continue; + } + + /// Nothing to do right now, wait for something to change. + /// + /// The timeout is a workaround for a race condition. + /// emergency_stop is assigned while holding a *different* mutex from the one we're holding + /// (exception_mutex vs worker_mutex). So it's possible that our emergency_stop check (above) + /// happens before a onBackgroundException() call, but our wait(lock) happens after it. + /// Then the wait may get stuck forever. + /// + /// Note that using wait(lock, [&]{ return emergency_stop || ...; }) wouldn't help because + /// it does effectively the same "check, then wait" sequence. + /// + /// One possible proper fix would be to make onBackgroundException() lock all read_workers + /// mutexes too (not necessarily simultaneously - just locking+unlocking them one by one + /// between the emergency_stop change and the notify_all() would be enough), but then we + /// need another mutex to protect read_workers itself... + next_condvar.wait_for(lock, std::chrono::seconds(10)); } - working_buffer = internal_buffer = Buffer(current_segment.data(), current_segment.data() + current_segment.size()); - current_position += working_buffer.size(); - return true; + chassert(false); + return false; } void ParallelReadBuffer::readerThreadFunction(ReadWorkerPtr read_worker) { SCOPE_EXIT({ - if (active_working_reader.fetch_sub(1) == 1) - active_working_reader.notify_all(); + if (active_working_readers.fetch_sub(1) == 1) + active_working_readers.notify_all(); }); try { - read_worker->reader->setReadUntilPosition(read_worker->range_end); - read_worker->reader->seek(read_worker->offset, SEEK_SET); - - while (!emergency_stop && !read_worker->cancel) + auto on_progress = [&](size_t bytes_read) -> bool { - if (!read_worker->reader->next()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Failed to read all the data from the reader, missing {} bytes", read_worker->bytes_left); - if (emergency_stop || read_worker->cancel) - break; + return true; - Buffer buffer = read_worker->reader->buffer(); - size_t bytes_to_copy = std::min(buffer.size(), read_worker->bytes_left); - Memory<> new_segment(bytes_to_copy); - memcpy(new_segment.data(), buffer.begin(), bytes_to_copy); - read_worker->reader->ignore(bytes_to_copy); - read_worker->bytes_left -= bytes_to_copy; - { - /// New data ready to be read - std::lock_guard lock(read_worker->worker_mutex); - read_worker->segments.emplace_back(std::move(new_segment)); - read_worker->finished = read_worker->bytes_left == 0; + std::lock_guard lock(read_worker->worker_mutex); + if (bytes_read <= read_worker->bytes_produced) + return false; + + bool need_notify = read_worker->bytes_produced == read_worker->bytes_consumed; + read_worker->bytes_produced = bytes_read; + if (need_notify) next_condvar.notify_all(); - } - if (read_worker->finished) - { - break; - } - } + return false; + }; + + size_t r = input.readBigAt(read_worker->segment.data(), read_worker->segment.size(), read_worker->start_offset); + + if (!on_progress(r) && r < read_worker->segment.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Failed to read all the data from the reader at offset {}, got {}/{} bytes", + read_worker->start_offset, r, read_worker->segment.size()); } catch (...) { @@ -315,12 +283,24 @@ void ParallelReadBuffer::finishAndWait() { emergency_stop = true; - size_t active_readers = active_working_reader.load(); + size_t active_readers = active_working_readers.load(); while (active_readers != 0) { - active_working_reader.wait(active_readers); - active_readers = active_working_reader.load(); + active_working_readers.wait(active_readers); + active_readers = active_working_readers.load(); } } +std::unique_ptr wrapInParallelReadBufferIfSupported( + ReadBuffer & buf, ThreadPoolCallbackRunner schedule, size_t max_working_readers, + size_t range_step, size_t file_size) +{ + auto * seekable = dynamic_cast(&buf); + if (!seekable || !seekable->supportsReadAt()) + return nullptr; + + return std::make_unique( + *seekable, schedule, max_working_readers, range_step, file_size); +} + } diff --git a/src/IO/ParallelReadBuffer.h b/src/IO/ParallelReadBuffer.h index 70f925f9735..e76b40f77b7 100644 --- a/src/IO/ParallelReadBuffer.h +++ b/src/IO/ParallelReadBuffer.h @@ -10,18 +10,17 @@ namespace DB { /** - * Reads from multiple ReadBuffers in parallel. - * Preserves order of readers obtained from SeekableReadBufferFactory. + * Reads from multiple positions in a ReadBuffer in parallel. + * Then reassembles the data into one stream in the original order. * - * It consumes multiple readers and yields data from them in order as it passed. - * Each working reader save segments of data to internal queue. + * Each working reader reads its segment of data into a buffer. * - * ParallelReadBuffer in nextImpl method take first available segment from first reader in deque and fed it to user. - * When first reader finish reading, they will be removed from worker deque and data from next reader consumed. + * ParallelReadBuffer in nextImpl method take first available segment from first reader in deque and reports it it to user. + * When first reader finishes reading, they will be removed from worker deque and data from next reader consumed. * * Number of working readers limited by max_working_readers. */ -class ParallelReadBuffer : public SeekableReadBuffer +class ParallelReadBuffer : public SeekableReadBuffer, public WithFileSize { private: /// Blocks until data occurred in the first reader or this reader indicate finishing @@ -29,19 +28,19 @@ private: bool nextImpl() override; public: - ParallelReadBuffer(SeekableReadBufferFactoryPtr reader_factory_, ThreadPoolCallbackRunner schedule_, size_t max_working_readers, size_t range_step_); + ParallelReadBuffer(SeekableReadBuffer & input, ThreadPoolCallbackRunner schedule_, size_t max_working_readers, size_t range_step_, size_t file_size); ~ParallelReadBuffer() override { finishAndWait(); } off_t seek(off_t off, int whence) override; - size_t getFileSize(); + size_t getFileSize() override; off_t getPosition() override; - const SeekableReadBufferFactory & getReadBufferFactory() const { return *reader_factory; } - SeekableReadBufferFactory & getReadBufferFactory() { return *reader_factory; } + const SeekableReadBuffer & getReadBuffer() const { return input; } + SeekableReadBuffer & getReadBuffer() { return input; } private: - /// Reader in progress with a list of read segments + /// Reader in progress with a buffer for the segment struct ReadWorker; using ReadWorkerPtr = std::shared_ptr; @@ -55,28 +54,28 @@ private: void addReaders(); bool addReaderToPool(); - /// Process read_worker, read data and save into internal segments queue + /// Process read_worker, read data and save into the buffer void readerThreadFunction(ReadWorkerPtr read_worker); void onBackgroundException(); void finishAndWait(); - Memory<> current_segment; - size_t max_working_readers; - std::atomic_size_t active_working_reader{0}; + std::atomic_size_t active_working_readers{0}; ThreadPoolCallbackRunner schedule; - std::unique_ptr reader_factory; + SeekableReadBuffer & input; + size_t file_size; size_t range_step; size_t next_range_start{0}; /** * FIFO queue of readers. - * Each worker contains reader itself and downloaded segments. - * When reader read all available data it will be removed from - * deque and data from next reader will be consumed to user. + * Each worker contains a buffer for the downloaded segment. + * After all data for the segment is read and delivered to the user, the reader will be removed + * from deque and data from next reader will be delivered. + * After removing from deque, call addReaders(). */ std::deque read_workers; @@ -92,4 +91,10 @@ private: bool all_completed{false}; }; +/// If `buf` is a SeekableReadBuffer with supportsReadAt() == true, creates a ParallelReadBuffer +/// from it. Otherwise returns nullptr; +std::unique_ptr wrapInParallelReadBufferIfSupported( + ReadBuffer & buf, ThreadPoolCallbackRunner schedule, size_t max_working_readers, + size_t range_step, size_t file_size); + } diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index bf44d9d10da..67bc01279c3 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -50,30 +50,30 @@ std::string ReadBufferFromFileDescriptor::getFileName() const } -bool ReadBufferFromFileDescriptor::nextImpl() +size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset) { - /// If internal_buffer size is empty, then read() cannot be distinguished from EOF - assert(!internal_buffer.empty()); + chassert(min_bytes <= max_bytes); - /// This is a workaround of a read pass EOF bug in linux kernel with pread() - if (file_size.has_value() && file_offset_of_buffer_end >= *file_size) - return false; + /// This is a workaround of a read past EOF bug in linux kernel with pread() + if (file_size.has_value() && offset >= *file_size) + return 0; size_t bytes_read = 0; - while (!bytes_read) + while (bytes_read < min_bytes) { ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorRead); Stopwatch watch(profile_callback ? clock_type : CLOCK_MONOTONIC); ssize_t res = 0; + size_t to_read = max_bytes - bytes_read; { CurrentMetrics::Increment metric_increment{CurrentMetrics::Read}; if (use_pread) - res = ::pread(fd, internal_buffer.begin(), internal_buffer.size(), file_offset_of_buffer_end); + res = ::pread(fd, to + bytes_read, to_read, offset + bytes_read); else - res = ::read(fd, internal_buffer.begin(), internal_buffer.size()); + res = ::read(fd, to + bytes_read, to_read); } if (!res) break; @@ -102,18 +102,31 @@ bool ReadBufferFromFileDescriptor::nextImpl() if (profile_callback) { ProfileInfo info; - info.bytes_requested = internal_buffer.size(); + info.bytes_requested = to_read; info.bytes_read = res; info.nanoseconds = watch.elapsed(); profile_callback(info); } } + if (bytes_read) + ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadBytes, bytes_read); + + return bytes_read; +} + + +bool ReadBufferFromFileDescriptor::nextImpl() +{ + /// If internal_buffer size is empty, then read() cannot be distinguished from EOF + assert(!internal_buffer.empty()); + + size_t bytes_read = readImpl(internal_buffer.begin(), 1, internal_buffer.size(), file_offset_of_buffer_end); + file_offset_of_buffer_end += bytes_read; if (bytes_read) { - ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadBytes, bytes_read); working_buffer = internal_buffer; working_buffer.resize(bytes_read); } @@ -259,4 +272,17 @@ size_t ReadBufferFromFileDescriptor::getFileSize() return getSizeFromFileDescriptor(fd, getFileName()); } +bool ReadBufferFromFileDescriptor::checkIfActuallySeekable() +{ + struct stat stat; + auto res = ::fstat(fd, &stat); + return res == 0 && S_ISREG(stat.st_mode); +} + +size_t ReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) +{ + chassert(use_pread); + return readImpl(to, n, n, offset); +} + } diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index 10f140275bb..64340770cf2 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -30,6 +30,12 @@ protected: /// Name or some description of file. std::string getFileName() const override; + /// Does the read()/pread(), with all the metric increments, error handling, throttling, etc. + /// Doesn't seek (`offset` must match fd's position if !use_pread). + /// Stops after min_bytes or eof. Returns 0 if eof. + /// Thread safe. + size_t readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset); + public: explicit ReadBufferFromFileDescriptor( int fd_, @@ -65,6 +71,11 @@ public: size_t getFileSize() override; + bool checkIfActuallySeekable() override; + + size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) override; + bool supportsReadAt() override { return use_pread; } + private: /// Assuming file descriptor supports 'select', check that we have data to read or wait until timeout. bool poll(size_t timeout_microseconds) const; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 93e2c46b080..d1cb1ec9ab0 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -109,9 +109,12 @@ bool ReadBufferFromS3::nextImpl() } size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 0; attempt < request_settings.max_single_read_retries && !next_result; ++attempt) + for (size_t attempt = 0; !next_result; ++attempt) { - Stopwatch watch; + bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; + + ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); + try { if (!impl) @@ -133,44 +136,11 @@ bool ReadBufferFromS3::nextImpl() /// Try to read a next portion of data. next_result = impl->next(); - watch.stop(); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds()); break; } catch (Exception & e) { - watch.stop(); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds()); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3RequestsErrors, 1); - - if (auto * s3_exception = dynamic_cast(&e)) - { - /// It doesn't make sense to retry Access Denied or No Such Key - if (!s3_exception->isRetryableError()) - { - s3_exception->addMessage("while reading key: {}, from bucket: {}", key, bucket); - throw; - } - } - - /// It doesn't make sense to retry allocator errors - if (e.code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY) - { - tryLogCurrentException(log); - throw; - } - - LOG_DEBUG( - log, - "Caught exception while reading S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}, Attempt: {}, Message: {}", - bucket, - key, - version_id.empty() ? "Latest" : version_id, - getPosition(), - attempt, - e.message()); - - if (attempt + 1 == request_settings.max_single_read_retries) + if (!processException(e, getPosition(), attempt) || last_attempt) throw; /// Pause before next attempt. @@ -197,6 +167,74 @@ bool ReadBufferFromS3::nextImpl() } +size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) +{ + if (n == 0) + return 0; + + size_t sleep_time_with_backoff_milliseconds = 100; + for (size_t attempt = 0;; ++attempt) + { + bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; + + ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); + + try + { + auto result = sendRequest(range_begin, range_begin + n - 1); + std::istream & istr = result.GetBody(); + + size_t bytes = copyFromIStreamWithProgressCallback(istr, to, n, progress_callback); + + ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, bytes); + + if (read_settings.remote_throttler) + read_settings.remote_throttler->add(bytes, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); + + return bytes; + } + catch (Poco::Exception & e) + { + if (!processException(e, range_begin, attempt) || last_attempt) + throw; + + sleepForMilliseconds(sleep_time_with_backoff_milliseconds); + sleep_time_with_backoff_milliseconds *= 2; + } + } +} + +bool ReadBufferFromS3::processException(Poco::Exception & e, size_t read_offset, size_t attempt) const +{ + ProfileEvents::increment(ProfileEvents::ReadBufferFromS3RequestsErrors, 1); + + LOG_DEBUG( + log, + "Caught exception while reading S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}, " + "Attempt: {}, Message: {}", + bucket, key, version_id.empty() ? "Latest" : version_id, read_offset, attempt, e.message()); + + if (auto * s3_exception = dynamic_cast(&e)) + { + /// It doesn't make sense to retry Access Denied or No Such Key + if (!s3_exception->isRetryableError()) + { + s3_exception->addMessage("while reading key: {}, from bucket: {}", key, bucket); + return false; + } + } + + /// It doesn't make sense to retry allocator errors + if (e.code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY) + { + tryLogCurrentException(log); + return false; + } + + return true; +} + + off_t ReadBufferFromS3::seek(off_t offset_, int whence) { if (offset_ == getPosition() && whence == SEEK_SET) @@ -315,44 +353,40 @@ bool ReadBufferFromS3::atEndOfRequestedRangeGuess() std::unique_ptr ReadBufferFromS3::initialize() { - S3::GetObjectRequest req; - req.SetBucket(bucket); - req.SetKey(key); - if (!version_id.empty()) - { - req.SetVersionId(version_id); - } - /** * If remote_filesystem_read_method = 'threadpool', then for MergeTree family tables * exact byte ranges to read are always passed here. */ - if (read_until_position) - { - if (offset >= read_until_position) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); + if (read_until_position && offset >= read_until_position) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); - req.SetRange(fmt::format("bytes={}-{}", offset, read_until_position - 1)); - LOG_TEST( - log, - "Read S3 object. Bucket: {}, Key: {}, Version: {}, Range: {}-{}", - bucket, - key, - version_id.empty() ? "Latest" : version_id, - offset, - read_until_position - 1); - } - else + read_result = sendRequest(offset, read_until_position ? std::make_optional(read_until_position - 1) : std::nullopt); + + size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; + return std::make_unique(read_result.GetBody(), buffer_size); +} + +Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t range_begin, std::optional range_end_incl) const +{ + S3::GetObjectRequest req; + req.SetBucket(bucket); + req.SetKey(key); + if (!version_id.empty()) + req.SetVersionId(version_id); + + if (range_end_incl) { - if (offset) - req.SetRange(fmt::format("bytes={}-", offset)); + req.SetRange(fmt::format("bytes={}-{}", range_begin, *range_end_incl)); LOG_TEST( - log, - "Read S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}", - bucket, - key, - version_id.empty() ? "Latest" : version_id, - offset); + log, "Read S3 object. Bucket: {}, Key: {}, Version: {}, Range: {}-{}", + bucket, key, version_id.empty() ? "Latest" : version_id, range_begin, *range_end_incl); + } + else if (range_begin) + { + req.SetRange(fmt::format("bytes={}-", range_begin)); + LOG_TEST( + log, "Read S3 object. Bucket: {}, Key: {}, Version: {}, Offset: {}", + bucket, key, version_id.empty() ? "Latest" : version_id, range_begin); } ProfileEvents::increment(ProfileEvents::S3GetObject); @@ -371,9 +405,7 @@ std::unique_ptr ReadBufferFromS3::initialize() { ResourceCost bytes_read = outcome.GetResult().GetContentLength(); read_settings.resource_link.adjust(estimated_cost, bytes_read); - size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; - read_result = outcome.GetResultWithOwnership(); - return std::make_unique(read_result.GetBody(), buffer_size); + return outcome.GetResultWithOwnership(); } else { @@ -383,21 +415,6 @@ std::unique_ptr ReadBufferFromS3::initialize() } } -std::unique_ptr ReadBufferS3Factory::getReader() -{ - return std::make_unique( - client_ptr, - bucket, - key, - version_id, - request_settings, - read_settings.adjustBufferSize(object_size)); -} - -size_t ReadBufferS3Factory::getFileSize() -{ - return object_size; -} } #endif diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 52dd74bdd14..0f665861a1e 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -77,12 +77,22 @@ public: String getFileName() const override { return bucket + "/" + key; } + size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) override; + + bool supportsReadAt() override { return true; } + private: std::unique_ptr initialize(); - // If true, if we destroy impl now, no work was wasted. Just for metrics. + /// If true, if we destroy impl now, no work was wasted. Just for metrics. bool atEndOfRequestedRangeGuess(); + /// Call inside catch() block if GetObject fails. Bumps metrics, logs the error. + /// Returns true if the error looks retriable. + bool processException(Poco::Exception & e, size_t read_offset, size_t attempt) const; + + Aws::S3::Model::GetObjectResult sendRequest(size_t range_begin, std::optional range_end_incl) const; + ReadSettings read_settings; bool use_external_buffer; @@ -92,43 +102,6 @@ private: bool restricted_seek; }; -/// Creates separate ReadBufferFromS3 for sequence of ranges of particular object -class ReadBufferS3Factory : public SeekableReadBufferFactory, public WithFileName -{ -public: - explicit ReadBufferS3Factory( - std::shared_ptr client_ptr_, - const String & bucket_, - const String & key_, - const String & version_id_, - size_t object_size_, - const S3Settings::RequestSettings & request_settings_, - const ReadSettings & read_settings_) - : client_ptr(client_ptr_) - , bucket(bucket_) - , key(key_) - , version_id(version_id_) - , read_settings(read_settings_) - , object_size(object_size_) - , request_settings(request_settings_) - {} - - std::unique_ptr getReader() override; - - size_t getFileSize() override; - - String getFileName() const override { return bucket + "/" + key; } - -private: - std::shared_ptr client_ptr; - const String bucket; - const String key; - const String version_id; - ReadSettings read_settings; - size_t object_size; - const S3Settings::RequestSettings request_settings; -}; - } #endif diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 8c3ab704d2b..cf1159bfb4b 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -40,6 +40,12 @@ void UpdatableSession::updateSession(const Poco::URI & uri) throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, "Too many redirects while trying to access {}", initial_uri.toString()); } +template +typename UpdatableSession::SessionPtr UpdatableSession::createDetachedSession(const Poco::URI & uri) +{ + return session_factory->buildNewSession(uri); +} + template std::shared_ptr> UpdatableSession::clone(const Poco::URI & uri) { @@ -89,21 +95,11 @@ bool ReadWriteBufferFromHTTPBase::withPartialContent(const } template -size_t ReadWriteBufferFromHTTPBase::getRangeBegin() const { return read_range.begin.value_or(0); } +size_t ReadWriteBufferFromHTTPBase::getOffset() const { return read_range.begin.value_or(0) + offset_from_begin_pos; } template -size_t ReadWriteBufferFromHTTPBase::getOffset() const { return getRangeBegin() + offset_from_begin_pos; } - -template -std::istream * ReadWriteBufferFromHTTPBase::callImpl( - UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, - const std::string & method_, bool for_object_info) +void ReadWriteBufferFromHTTPBase::prepareRequest(Poco::Net::HTTPRequest & request, Poco::URI uri_, std::optional range) const { - // With empty path poco will send "POST HTTP/1.1" its bug. - if (uri_.getPath().empty()) - uri_.setPath("/"); - - Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(uri_.getHost()); // use original, not resolved host name in header if (out_stream_callback) @@ -111,16 +107,9 @@ std::istream * ReadWriteBufferFromHTTPBase::callImpl( else if (method == Poco::Net::HTTPRequest::HTTP_POST) request.setContentLength(0); /// No callback - no body - for (auto & [header, value] : http_header_entries) + for (const auto & [header, value] : http_header_entries) request.set(header, value); - std::optional range; - if (!for_object_info) - { - if (withPartialContent(read_range)) - range = HTTPRange{getOffset(), read_range.end}; - } - if (range) { String range_header_value; @@ -134,6 +123,25 @@ std::istream * ReadWriteBufferFromHTTPBase::callImpl( if (!credentials.getUsername().empty()) credentials.authenticate(request); +} + +template +std::istream * ReadWriteBufferFromHTTPBase::callImpl( + UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_, bool for_object_info) +{ + // With empty path poco will send "POST HTTP/1.1" its bug. + if (uri_.getPath().empty()) + uri_.setPath("/"); + + std::optional range; + if (!for_object_info) + { + if (withPartialContent(read_range)) + range = HTTPRange{getOffset(), read_range.end}; + } + + Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + prepareRequest(request, uri_, range); LOG_TRACE(log, "Sending request to {}", uri_.toString()); @@ -176,6 +184,14 @@ size_t ReadWriteBufferFromHTTPBase::getFileSize() throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", uri.toString()); } +template +bool ReadWriteBufferFromHTTPBase::supportsReadAt() +{ + if (!file_info) + file_info = getFileInfo(); + return method == Poco::Net::HTTPRequest::HTTP_GET && file_info->seekable; +} + template bool ReadWriteBufferFromHTTPBase::checkIfActuallySeekable() { @@ -405,7 +421,7 @@ void ReadWriteBufferFromHTTPBase::initialize() { /// We could have range.begin == 0 and range.end != 0 in case of DiskWeb and failing to read with partial content /// will affect only performance, so a warning is enough. - LOG_WARNING(log, "Unable to read with range header: [{}, {}]", getRangeBegin(), *read_range.end); + LOG_WARNING(log, "Unable to read with range header: [{}, {}]", read_range.begin.value_or(0), *read_range.end); } } @@ -538,8 +554,8 @@ bool ReadWriteBufferFromHTTPBase::nextImpl() throw; /** Retry request unconditionally if nothing has been read yet. - * Otherwise if it is GET method retry with range header. - */ + * Otherwise if it is GET method retry with range header. + */ bool can_retry_request = !offset_from_begin_pos || method == Poco::Net::HTTPRequest::HTTP_GET; if (!can_retry_request) throw; @@ -574,6 +590,83 @@ bool ReadWriteBufferFromHTTPBase::nextImpl() return true; } +template +size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) +{ + /// Caller must have checked supportsReadAt(). + /// This ensures we've sent at least one HTTP request and populated saved_uri_redirect. + chassert(file_info && file_info->seekable); + + if (n == 0) + return 0; + + Poco::URI uri_ = saved_uri_redirect.value_or(uri); + if (uri_.getPath().empty()) + uri_.setPath("/"); + + size_t milliseconds_to_wait = settings.http_retry_initial_backoff_ms; + + for (size_t attempt = 0;; ++attempt) + { + bool last_attempt = attempt + 1 >= settings.http_max_tries; + + Poco::Net::HTTPRequest request(method, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + prepareRequest(request, uri_, HTTPRange { .begin = offset, .end = offset + n - 1}); + + LOG_TRACE(log, "Sending request to {} for range [{}, {})", uri_.toString(), offset, offset + n); + + auto sess = session->createDetachedSession(uri_); + + Poco::Net::HTTPResponse response; + std::istream * result_istr; + + try + { + sess->sendRequest(request); + result_istr = receiveResponse(*sess, request, response, /*allow_redirects*/ false); + + if (response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT && + (offset != 0 || offset + n < *file_info->file_size)) + throw Exception( + ErrorCodes::HTTP_RANGE_NOT_SATISFIABLE, + "Expected 206 Partial Content, got {} when reading {} range [{}, {})", + toString(response.getStatus()), uri_.toString(), offset, offset + n); + + bool cancelled; + size_t r = copyFromIStreamWithProgressCallback(*result_istr, to, n, progress_callback, &cancelled); + + return r; + } + catch (const Poco::Exception & e) + { + sess->attachSessionData(e.message()); + + LOG_ERROR( + log, + "HTTP request (positioned) to `{}` with range [{}, {}) failed at try {}/{}: {}", + uri_.toString(), offset, offset + n, attempt + 1, settings.http_max_tries, + e.what()); + + /// Decide whether to retry. + + if (last_attempt) + throw; + + /// Too many open files - non-retryable. + if (e.code() == POCO_EMFILE) + throw; + + if (const auto * h = dynamic_cast(&e); + h && !isRetriableError(static_cast(h->getHTTPStatus()))) + throw; + + sleepForMilliseconds(milliseconds_to_wait); + milliseconds_to_wait = std::min(milliseconds_to_wait * 2, settings.http_retry_max_backoff_ms); + continue; + } + } +} + template off_t ReadWriteBufferFromHTTPBase::getPosition() { return getOffset() - available(); } @@ -793,75 +886,6 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP( skip_not_found_url_, file_info_) {} -RangedReadWriteBufferFromHTTPFactory::RangedReadWriteBufferFromHTTPFactory( - Poco::URI uri_, - std::string method_, - OutStreamCallback out_stream_callback_, - ConnectionTimeouts timeouts_, - const Poco::Net::HTTPBasicCredentials & credentials_, - UInt64 max_redirects_, - size_t buffer_size_, - ReadSettings settings_, - HTTPHeaderEntries http_header_entries_, - const RemoteHostFilter * remote_host_filter_, - bool delay_initialization_, - bool use_external_buffer_, - bool skip_not_found_url_) - : uri(uri_) - , method(std::move(method_)) - , out_stream_callback(out_stream_callback_) - , timeouts(std::move(timeouts_)) - , credentials(credentials_) - , max_redirects(max_redirects_) - , buffer_size(buffer_size_) - , settings(std::move(settings_)) - , http_header_entries(std::move(http_header_entries_)) - , remote_host_filter(remote_host_filter_) - , delay_initialization(delay_initialization_) - , use_external_buffer(use_external_buffer_) - , skip_not_found_url(skip_not_found_url_) {} - -std::unique_ptr RangedReadWriteBufferFromHTTPFactory::getReader() -{ - return std::make_unique( - uri, - method, - out_stream_callback, - timeouts, - credentials, - max_redirects, - buffer_size, - settings, - http_header_entries, - remote_host_filter, - delay_initialization, - use_external_buffer, - skip_not_found_url, - file_info); -} - -size_t RangedReadWriteBufferFromHTTPFactory::getFileSize() -{ - auto s = getFileInfo().file_size; - if (!s) - throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", uri.toString()); - return *s; -} - -bool RangedReadWriteBufferFromHTTPFactory::checkIfActuallySeekable() -{ - return getFileInfo().seekable; -} - -HTTPFileInfo RangedReadWriteBufferFromHTTPFactory::getFileInfo() -{ - if (!file_info) - file_info = static_cast(getReader().get())->getFileInfo(); - return *file_info; -} - -String RangedReadWriteBufferFromHTTPFactory::getFileName() const { return uri.toString(); } - PooledSessionFactory::PooledSessionFactory( const ConnectionTimeouts & timeouts_, size_t per_endpoint_pool_size_) @@ -891,6 +915,7 @@ PooledReadWriteBufferFromHTTP::PooledReadWriteBufferFromHTTP( out_stream_callback_, buffer_size_) {} + template class UpdatableSession; template class UpdatableSession; template class detail::ReadWriteBufferFromHTTPBase>>; diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index de1946ced48..2d2ae5fe724 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -42,6 +42,9 @@ public: void updateSession(const Poco::URI & uri); + /// Thread safe. + SessionPtr createDetachedSession(const Poco::URI & uri); + std::shared_ptr> clone(const Poco::URI & uri); private: @@ -110,14 +113,16 @@ namespace detail bool withPartialContent(const HTTPRange & range) const; - size_t getRangeBegin() const; - size_t getOffset() const; + void prepareRequest(Poco::Net::HTTPRequest & request, Poco::URI uri_, std::optional range) const; + std::istream * callImpl(UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_, bool for_object_info = false); size_t getFileSize() override; + bool supportsReadAt() override; + bool checkIfActuallySeekable() override; String getFileName() const override; @@ -171,6 +176,8 @@ namespace detail bool nextImpl() override; + size_t readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) override; + off_t getPosition() override; off_t seek(off_t offset_, int whence) override; @@ -237,53 +244,6 @@ public: std::optional file_info_ = std::nullopt); }; -class RangedReadWriteBufferFromHTTPFactory : public SeekableReadBufferFactory, public WithFileName -{ - using OutStreamCallback = ReadWriteBufferFromHTTP::OutStreamCallback; - -public: - RangedReadWriteBufferFromHTTPFactory( - Poco::URI uri_, - std::string method_, - OutStreamCallback out_stream_callback_, - ConnectionTimeouts timeouts_, - const Poco::Net::HTTPBasicCredentials & credentials_, - UInt64 max_redirects_ = 0, - size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, - ReadSettings settings_ = {}, - HTTPHeaderEntries http_header_entries_ = {}, - const RemoteHostFilter * remote_host_filter_ = nullptr, - bool delay_initialization_ = true, - bool use_external_buffer_ = false, - bool skip_not_found_url_ = false); - - std::unique_ptr getReader() override; - - size_t getFileSize() override; - - bool checkIfActuallySeekable() override; - - HTTPFileInfo getFileInfo(); - - String getFileName() const override; - -private: - Poco::URI uri; - std::string method; - OutStreamCallback out_stream_callback; - ConnectionTimeouts timeouts; - const Poco::Net::HTTPBasicCredentials & credentials; - UInt64 max_redirects; - size_t buffer_size; - ReadSettings settings; - HTTPHeaderEntries http_header_entries; - const RemoteHostFilter * remote_host_filter; - std::optional file_info; - bool delay_initialization; - bool use_external_buffer; - bool skip_not_found_url; -}; - class PooledSessionFactory { public: @@ -292,7 +252,9 @@ public: using SessionType = PooledHTTPSessionPtr; + /// Thread safe. SessionType buildNewSession(const Poco::URI & uri); + private: ConnectionTimeouts timeouts; size_t per_endpoint_pool_size; @@ -315,6 +277,7 @@ public: size_t max_connections_per_endpoint = DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT); }; + extern template class UpdatableSession; extern template class UpdatableSession; extern template class detail::ReadWriteBufferFromHTTPBase>>; diff --git a/src/IO/SeekableReadBuffer.cpp b/src/IO/SeekableReadBuffer.cpp index 99c43d6671b..b83e382db01 100644 --- a/src/IO/SeekableReadBuffer.cpp +++ b/src/IO/SeekableReadBuffer.cpp @@ -3,6 +3,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_READ_FROM_ISTREAM; +} namespace { @@ -60,4 +64,46 @@ std::unique_ptr wrapSeekableReadBufferPointer(SeekableReadBu return std::make_unique>(*ptr, SeekableReadBufferPtr{ptr}); } +size_t copyFromIStreamWithProgressCallback(std::istream & istr, char * to, size_t n, const std::function & progress_callback, bool * out_cancelled) +{ + const size_t chunk = DBMS_DEFAULT_BUFFER_SIZE; + if (out_cancelled) + *out_cancelled = false; + + size_t copied = 0; + while (copied < n) + { + size_t to_copy = std::min(chunk, n - copied); + istr.read(to + copied, to_copy); + size_t gcount = istr.gcount(); + + copied += gcount; + + bool cancelled = false; + if (gcount && progress_callback) + cancelled = progress_callback(copied); + + if (gcount != to_copy) + { + if (!istr.eof()) + throw Exception( + ErrorCodes::CANNOT_READ_FROM_ISTREAM, + "{} at offset {}", + istr.fail() ? "Cannot read from istream" : "Unexpected state of istream", + copied); + + break; + } + + if (cancelled) + { + if (out_cancelled != nullptr) + *out_cancelled = true; + break; + } + } + + return copied; +} + } diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index 736ab5bbc71..8ced9d752de 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -59,39 +59,41 @@ public: /// * Sometimes when we create such buffer we don't know in advance whether we'll need it to be /// seekable or not. So we don't want to pay the price for this check in advance. virtual bool checkIfActuallySeekable() { return true; } + + /// Unbuffered positional read. + /// Doesn't affect the buffer state (position, working_buffer, etc). + /// + /// `progress_callback` may be called periodically during the read, reporting that to[0..m-1] + /// has been filled. If it returns true, reading is stopped, and readBigAt() returns bytes read + /// so far. Called only from inside readBigAt(), from the same thread, with increasing m. + /// + /// Stops either after n bytes, or at end of file, or on exception. Returns number of bytes read. + /// If offset is past the end of file, may return 0 or throw exception. + /// + /// Caller needs to be careful: + /// * supportsReadAt() must be checked (called and return true) before calling readBigAt(). + /// Otherwise readBigAt() may crash. + /// * Thread safety: multiple readBigAt() calls may be performed in parallel. + /// But readBigAt() may not be called in parallel with any other methods + /// (e.g. next() or supportsReadAt()). + /// * Performance: there's no buffering. Each readBigAt() call typically translates into actual + /// IO operation (e.g. HTTP request). Don't use it for small adjacent reads. + virtual size_t readBigAt(char * /*to*/, size_t /*n*/, size_t /*offset*/, const std::function & /*progress_callback*/ = nullptr) + { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method readBigAt() not implemented"); } + + /// Checks if readBigAt() is allowed. May be slow, may throw (e.g. it may do an HTTP request or an fstat). + virtual bool supportsReadAt() { return false; } }; -/// Useful for reading in parallel. -/// The created read buffers may outlive the factory. -/// -/// There are 2 ways to use this: -/// (1) Never call seek() or getFileSize(), read the file sequentially. -/// For HTTP, this usually translates to just one HTTP request. -/// (2) Call checkIfActuallySeekable(), then: -/// a. If it returned false, go to (1). seek() and getFileSize() are not available (throw if called). -/// b. If it returned true, seek() and getFileSize() are available, knock yourself out. -/// For HTTP, checkIfActuallySeekable() sends a HEAD request and returns false if the web server -/// doesn't support ranges (or doesn't support HEAD requests). -class SeekableReadBufferFactory : public WithFileSize -{ -public: - ~SeekableReadBufferFactory() override = default; - - // We usually call setReadUntilPosition() and seek() on the returned buffer before reading. - // So it's recommended that the returned implementation be lazy, i.e. don't start reading - // before the first call to nextImpl(). - virtual std::unique_ptr getReader() = 0; - - virtual bool checkIfActuallySeekable() { return true; } -}; using SeekableReadBufferPtr = std::shared_ptr; -using SeekableReadBufferFactoryPtr = std::unique_ptr; - /// Wraps a reference to a SeekableReadBuffer into an unique pointer to SeekableReadBuffer. /// This function is like wrapReadBufferReference() but for SeekableReadBuffer. std::unique_ptr wrapSeekableReadBufferReference(SeekableReadBuffer & ref); std::unique_ptr wrapSeekableReadBufferPointer(SeekableReadBufferPtr ptr); +/// Helper for implementing readBigAt(). +size_t copyFromIStreamWithProgressCallback(std::istream & istr, char * to, size_t n, const std::function & progress_callback, bool * out_cancelled = nullptr); + } diff --git a/src/IO/WithFileName.cpp b/src/IO/WithFileName.cpp index 9d9f264c861..2383182f7e7 100644 --- a/src/IO/WithFileName.cpp +++ b/src/IO/WithFileName.cpp @@ -19,7 +19,7 @@ String getFileNameFromReadBuffer(const ReadBuffer & in) if (const auto * compressed = dynamic_cast(&in)) return getFileName(compressed->getWrappedReadBuffer()); else if (const auto * parallel = dynamic_cast(&in)) - return getFileName(parallel->getReadBufferFactory()); + return getFileName(parallel->getReadBuffer()); else if (const auto * peekable = dynamic_cast(&in)) return getFileNameFromReadBuffer(peekable->getSubBuffer()); else diff --git a/src/IO/WithFileSize.cpp b/src/IO/WithFileSize.cpp index f71690fcdee..28542db7a73 100644 --- a/src/IO/WithFileSize.cpp +++ b/src/IO/WithFileSize.cpp @@ -33,10 +33,6 @@ size_t getFileSizeFromReadBuffer(ReadBuffer & in) { return getFileSize(compressed->getWrappedReadBuffer()); } - else if (auto * parallel = dynamic_cast(&in)) - { - return getFileSize(parallel->getReadBufferFactory()); - } return getFileSize(in); } @@ -51,10 +47,6 @@ bool isBufferWithFileSize(const ReadBuffer & in) { return isBufferWithFileSize(compressed->getWrappedReadBuffer()); } - else if (const auto * parallel = dynamic_cast(&in)) - { - return dynamic_cast(¶llel->getReadBufferFactory()) != nullptr; - } return dynamic_cast(&in) != nullptr; } diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index 54f3b76ff60..37505f94e98 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -146,45 +146,19 @@ arrow::Status ArrowInputStreamFromReadBuffer::Close() return arrow::Status(); } -RandomAccessFileFromManyReadBuffers::RandomAccessFileFromManyReadBuffers(SeekableReadBufferFactory & factory) : buf_factory(factory) {} +RandomAccessFileFromRandomAccessReadBuffer::RandomAccessFileFromRandomAccessReadBuffer(SeekableReadBuffer & in_, size_t file_size_) : in(in_), file_size(file_size_) {} -arrow::Result RandomAccessFileFromManyReadBuffers::GetSize() +arrow::Result RandomAccessFileFromRandomAccessReadBuffer::GetSize() { - return buf_factory.getFileSize(); + return file_size; } -arrow::Result RandomAccessFileFromManyReadBuffers::ReadAt(int64_t position, int64_t nbytes, void* out) +arrow::Result RandomAccessFileFromRandomAccessReadBuffer::ReadAt(int64_t position, int64_t nbytes, void* out) { - std::unique_lock lock(mutex); - if (free_bufs.empty()) - free_bufs.push_back(buf_factory.getReader()); - auto buf = std::move(free_bufs.back()); - free_bufs.pop_back(); - lock.unlock(); - - // To work well with this, ReadBuffer implementations need to respect setReadUntilPosition() and - // not read above it. We often do very small reads here. - // Also nice if they: - // * Make readBig() read directly into the provided memory, instead of copying from internal - // buffer. - // * Allocate the internal buffer (if any) lazily in first nextImpl() call. If all reads are - // tiny readBig() calls (as is typical here), it won't allocate an unnecessary 1 MB buffer. - - buf->seek(position, SEEK_SET); - buf->setReadUntilPosition(position + nbytes); - size_t bytes_read = buf->readBig(reinterpret_cast(out), nbytes); - - // Seeking to a position above a previous setReadUntilPosition() confuses some of the - // ReadBuffer implementations. So we reset it before next seek. - buf->setReadUntilEnd(); - - lock.lock(); - free_bufs.push_back(std::move(buf)); - - return static_cast(bytes_read); + return in.readBigAt(reinterpret_cast(out), nbytes, position); } -arrow::Result> RandomAccessFileFromManyReadBuffers::ReadAt(int64_t position, int64_t nbytes) +arrow::Result> RandomAccessFileFromRandomAccessReadBuffer::ReadAt(int64_t position, int64_t nbytes) { ARROW_ASSIGN_OR_RAISE(auto buffer, arrow::AllocateResizableBuffer(nbytes)) ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, ReadAt(position, nbytes, buffer->mutable_data())) @@ -195,22 +169,23 @@ arrow::Result> RandomAccessFileFromManyReadBuffer return buffer; } -arrow::Future> RandomAccessFileFromManyReadBuffers::ReadAsync(const arrow::io::IOContext&, int64_t position, int64_t nbytes) +arrow::Future> RandomAccessFileFromRandomAccessReadBuffer::ReadAsync(const arrow::io::IOContext&, int64_t position, int64_t nbytes) { return arrow::Future>::MakeFinished(ReadAt(position, nbytes)); } -arrow::Status RandomAccessFileFromManyReadBuffers::Close() +arrow::Status RandomAccessFileFromRandomAccessReadBuffer::Close() { chassert(is_open); is_open = false; return arrow::Status::OK(); } -arrow::Status RandomAccessFileFromManyReadBuffers::Seek(int64_t) { return arrow::Status::NotImplemented(""); } -arrow::Result RandomAccessFileFromManyReadBuffers::Tell() const { return arrow::Status::NotImplemented(""); } -arrow::Result RandomAccessFileFromManyReadBuffers::Read(int64_t, void*) { return arrow::Status::NotImplemented(""); } -arrow::Result> RandomAccessFileFromManyReadBuffers::Read(int64_t) { return arrow::Status::NotImplemented(""); } +arrow::Status RandomAccessFileFromRandomAccessReadBuffer::Seek(int64_t) { return arrow::Status::NotImplemented(""); } +arrow::Result RandomAccessFileFromRandomAccessReadBuffer::Tell() const { return arrow::Status::NotImplemented(""); } +arrow::Result RandomAccessFileFromRandomAccessReadBuffer::Read(int64_t, void*) { return arrow::Status::NotImplemented(""); } +arrow::Result> RandomAccessFileFromRandomAccessReadBuffer::Read(int64_t) { return arrow::Status::NotImplemented(""); } + std::shared_ptr asArrowFile( ReadBuffer & in, @@ -220,19 +195,16 @@ std::shared_ptr asArrowFile( const std::string & magic_bytes, bool avoid_buffering) { - if (auto * fd_in = dynamic_cast(&in)) + bool has_file_size = isBufferWithFileSize(in); + auto * seekable_in = dynamic_cast(&in); + + if (has_file_size && seekable_in && settings.seekable_read) { - struct stat stat; - auto res = ::fstat(fd_in->getFD(), &stat); - // if fd is a regular file i.e. not stdin - if (res == 0 && S_ISREG(stat.st_mode)) - return std::make_shared(*fd_in, stat.st_size, avoid_buffering); - } - else if (auto * seekable_in = dynamic_cast(&in); - seekable_in && settings.seekable_read && isBufferWithFileSize(in) && - seekable_in->checkIfActuallySeekable()) - { - return std::make_shared(in, std::nullopt, avoid_buffering); + if (avoid_buffering && seekable_in->supportsReadAt()) + return std::make_shared(*seekable_in, getFileSizeFromReadBuffer(in)); + + if (seekable_in->checkIfActuallySeekable()) + return std::make_shared(*seekable_in, std::nullopt, avoid_buffering); } // fallback to loading the entire file in memory @@ -245,26 +217,16 @@ std::shared_ptr asArrowFileLoadIntoMemory( const std::string & format_name, const std::string & magic_bytes) { - std::string file_data; - { - PeekableReadBuffer buf(in); - std::string magic_bytes_from_data; - magic_bytes_from_data.resize(magic_bytes.size()); - bool read_magic_bytes = false; - try - { - PeekableReadBufferCheckpoint checkpoint(buf, true); - buf.readStrict(magic_bytes_from_data.data(), magic_bytes_from_data.size()); - read_magic_bytes = true; - } - catch (const Exception &) {} + std::string file_data(magic_bytes.size(), '\0'); - if (!read_magic_bytes || magic_bytes_from_data != magic_bytes) - throw Exception(ErrorCodes::INCORRECT_DATA, "Not a {} file", format_name); + /// Avoid loading the whole file if it doesn't seem to even be in the correct format. + size_t bytes_read = in.read(file_data.data(), magic_bytes.size()); + if (bytes_read < magic_bytes.size() || file_data != magic_bytes) + throw Exception(ErrorCodes::INCORRECT_DATA, "Not a {} file", format_name); - WriteBufferFromString file_buffer(file_data); - copyData(buf, file_buffer, is_cancelled); - } + WriteBufferFromString file_buffer(file_data, AppendModeTag{}); + copyData(in, file_buffer, is_cancelled); + file_buffer.finalize(); return std::make_shared(arrow::Buffer::FromString(std::move(file_data))); } diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.h b/src/Processors/Formats/Impl/ArrowBufferedStreams.h index 9307172cb11..f455bcdfb1a 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.h +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.h @@ -18,7 +18,6 @@ class ReadBuffer; class WriteBuffer; class SeekableReadBuffer; -class SeekableReadBufferFactory; struct FormatSettings; class ArrowBufferedOutputStream : public arrow::io::OutputStream @@ -78,19 +77,17 @@ private: ARROW_DISALLOW_COPY_AND_ASSIGN(RandomAccessFileFromSeekableReadBuffer); }; -// Thread-safe. -// Maintains a pool of SeekableReadBuffer-s. For each ReadAt(), takes a buffer, seeks it, and reads. -class RandomAccessFileFromManyReadBuffers : public arrow::io::RandomAccessFile +class RandomAccessFileFromRandomAccessReadBuffer : public arrow::io::RandomAccessFile { public: - explicit RandomAccessFileFromManyReadBuffers(SeekableReadBufferFactory & factory); + explicit RandomAccessFileFromRandomAccessReadBuffer(SeekableReadBuffer & in_, size_t file_size_); // These are thread safe. arrow::Result GetSize() override; arrow::Result ReadAt(int64_t position, int64_t nbytes, void* out) override; arrow::Result> ReadAt(int64_t position, int64_t nbytes) override; - arrow::Future> ReadAsync(const arrow::io::IOContext&, int64_t position, - int64_t nbytes) override; + arrow::Future> ReadAsync( + const arrow::io::IOContext&, int64_t position, int64_t nbytes) override; // These are not thread safe, and arrow shouldn't call them. Return NotImplemented error. arrow::Status Seek(int64_t) override; @@ -102,13 +99,11 @@ public: bool closed() const override { return !is_open; } private: - SeekableReadBufferFactory & buf_factory; + SeekableReadBuffer & in; + size_t file_size; bool is_open = true; - std::mutex mutex; - std::vector> free_bufs; - - ARROW_DISALLOW_COPY_AND_ASSIGN(RandomAccessFileFromManyReadBuffers); + ARROW_DISALLOW_COPY_AND_ASSIGN(RandomAccessFileFromRandomAccessReadBuffer); }; class ArrowInputStreamFromReadBuffer : public arrow::io::InputStream diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 7f90c1197ce..2f3c68aa481 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -43,14 +43,12 @@ namespace ErrorCodes } while (false) ParquetBlockInputFormat::ParquetBlockInputFormat( - ReadBuffer * buf, - SeekableReadBufferFactoryPtr buf_factory_, + ReadBuffer & buf, const Block & header_, const FormatSettings & format_settings_, size_t max_decoding_threads_, size_t min_bytes_for_seek_) - : IInputFormat(header_, buf) - , buf_factory(std::move(buf_factory_)) + : IInputFormat(header_, &buf) , format_settings(format_settings_) , skip_row_groups(format_settings.parquet.skip_row_groups) , max_decoding_threads(max_decoding_threads_) @@ -71,17 +69,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() // Create arrow file adapter. // TODO: Make the adapter do prefetching on IO threads, based on the full set of ranges that // we'll need to read (which we know in advance). Use max_download_threads for that. - if (buf_factory) - { - if (format_settings.seekable_read && buf_factory->checkIfActuallySeekable()) - arrow_file = std::make_shared(*buf_factory); - else - arrow_file = asArrowFileLoadIntoMemory(*buf_factory->getReader(), is_stopped, "Parquet", PARQUET_MAGIC_BYTES); - } - else - { - arrow_file = asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); - } + arrow_file = asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); if (is_stopped) return; @@ -388,7 +376,7 @@ ParquetSchemaReader::ParquetSchemaReader(ReadBuffer & in_, const FormatSettings NamesAndTypesList ParquetSchemaReader::readSchema() { std::atomic is_stopped{0}; - auto file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES); + auto file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); auto metadata = parquet::ReadMetaData(file); @@ -406,8 +394,7 @@ void registerInputFormatParquet(FormatFactory & factory) { factory.registerRandomAccessInputFormat( "Parquet", - [](ReadBuffer * buf, - SeekableReadBufferFactoryPtr buf_factory, + [](ReadBuffer & buf, const Block & sample, const FormatSettings & settings, const ReadSettings& read_settings, @@ -418,7 +405,6 @@ void registerInputFormatParquet(FormatFactory & factory) size_t min_bytes_for_seek = is_remote_fs ? read_settings.remote_read_min_bytes_for_seek : 8 * 1024; return std::make_shared( buf, - std::move(buf_factory), sample, settings, max_parsing_threads, diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index f17eee59414..ad7074547fc 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -15,7 +15,6 @@ namespace DB { class ArrowColumnToCHColumn; -class SeekableReadBufferFactory; // Parquet files contain a metadata block with the following information: // * list of columns, @@ -48,9 +47,7 @@ class ParquetBlockInputFormat : public IInputFormat { public: ParquetBlockInputFormat( - // exactly one of these two is nullptr - ReadBuffer * buf, - std::unique_ptr buf_factory, + ReadBuffer & buf, const Block & header, const FormatSettings & format_settings, size_t max_decoding_threads, @@ -234,7 +231,6 @@ private: }; }; - std::unique_ptr buf_factory; const FormatSettings format_settings; const std::unordered_set & skip_row_groups; size_t max_decoding_threads; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 8380fa26a39..2d8aaec0f07 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -582,31 +582,11 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); auto compression_method = chooseCompressionMethod(current_key, compression_hint); - InputFormatPtr input_format; - std::unique_ptr owned_read_buf; - - auto read_buf_or_factory = createS3ReadBuffer(current_key, object_size); - if (read_buf_or_factory.buf_factory) - { - input_format = FormatFactory::instance().getInputRandomAccess( - format, - std::move(read_buf_or_factory.buf_factory), - sample_block, - getContext(), - max_block_size, - /* is_remote_fs */ true, - compression_method, - format_settings); - } - else - { - owned_read_buf = wrapReadBufferWithCompressionMethod( - std::move(read_buf_or_factory.buf), - compression_method, - static_cast(getContext()->getSettingsRef().zstd_window_log_max)); - input_format = FormatFactory::instance().getInput( - format, *owned_read_buf, sample_block, getContext(), max_block_size, format_settings); - } + auto read_buf = createS3ReadBuffer(current_key, object_size); + auto input_format = FormatFactory::instance().getInput( + format, *read_buf, sample_block, getContext(), max_block_size, + format_settings, std::nullopt, std::nullopt, + /* is_remote_fs */ true, compression_method); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -621,7 +601,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(bucket) / current_key, std::move(owned_read_buf), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{fs::path(bucket) / current_key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() @@ -629,7 +609,7 @@ std::future StorageS3Source::createReaderAsync() return create_reader_scheduler([this] { return createReader(); }, Priority{}); } -StorageS3Source::ReadBufferOrFactory StorageS3Source::createS3ReadBuffer(const String & key, size_t object_size) +std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & key, size_t object_size) { auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); read_settings.enable_filesystem_cache = false; @@ -642,12 +622,13 @@ StorageS3Source::ReadBufferOrFactory StorageS3Source::createS3ReadBuffer(const S if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); - return {.buf = createAsyncS3ReadBuffer(key, read_settings, object_size)}; + return createAsyncS3ReadBuffer(key, read_settings, object_size); } - auto factory = std::make_unique( - client, bucket, key, version_id, object_size, request_settings, read_settings); - return {.buf_factory = std::move(factory)}; + return std::make_unique( + client, bucket, key, version_id, request_settings, read_settings, + /*use_external_buffer*/ false, /*offset_*/ 0, /*read_until_position_*/ 0, + /*restricted_seek_*/ false, object_size); } std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 66652a45e3a..a4c120b99a6 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -204,12 +204,6 @@ private: std::unique_ptr reader; }; - struct ReadBufferOrFactory - { - std::unique_ptr buf; - SeekableReadBufferFactoryPtr buf_factory; - }; - ReaderHolder reader; std::vector requested_virtual_columns; @@ -230,7 +224,7 @@ private: ReaderHolder createReader(); std::future createReaderAsync(); - ReadBufferOrFactory createS3ReadBuffer(const String & key, size_t object_size); + std::unique_ptr createS3ReadBuffer(const String & key, size_t object_size); std::unique_ptr createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); }; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 4c5ed08e26e..efc44a069dd 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -248,7 +248,7 @@ StorageURLSource::StorageURLSource( throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list"); auto first_option = uri_options.begin(); - auto [actual_uri, buf_factory] = getFirstAvailableURIAndReadBuffer( + auto [actual_uri, buf] = getFirstAvailableURIAndReadBuffer( first_option, uri_options.end(), context, @@ -262,10 +262,11 @@ StorageURLSource::StorageURLSource( uri_options.size() == 1); curr_uri = actual_uri; + read_buf = std::move(buf); try { - total_size += buf_factory->getFileSize(); + total_size += getFileSizeFromReadBuffer(*read_buf); } catch (...) { @@ -273,16 +274,17 @@ StorageURLSource::StorageURLSource( } // TODO: Pass max_parsing_threads and max_download_threads adjusted for num_streams. - auto input_format = FormatFactory::instance().getInputRandomAccess( + auto input_format = FormatFactory::instance().getInput( format, - std::move(buf_factory), + *read_buf, sample_block, context, max_block_size, - /* is_remote_fs */ true, - compression_method, format_settings, - download_threads); + download_threads, + /*max_download_threads*/ std::nullopt, + /* is_remote_fs */ true, + compression_method); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -348,7 +350,7 @@ Chunk StorageURLSource::generate() return {}; } -std::tuple StorageURLSource::getFirstAvailableURIAndReadBuffer( +std::tuple> StorageURLSource::getFirstAvailableURIAndReadBuffer( std::vector::const_iterator & option, const std::vector::const_iterator & end, ContextPtr context, @@ -376,40 +378,38 @@ std::tuple StorageURLSource::getFirstAv setCredentials(credentials, request_uri); const auto settings = context->getSettings(); - auto res = std::make_unique( - request_uri, - http_method, - callback, - timeouts, - credentials, - settings.max_http_get_redirects, - settings.max_read_buffer_size, - read_settings, - headers, - &context->getRemoteHostFilter(), - delay_initialization, - /* use_external_buffer */ false, - /* skip_url_not_found_error */ skip_url_not_found_error); - if (options > 1) + try { - // Send a HEAD request to check availability. - try - { - res->getFileInfo(); - } - catch (...) - { - if (first_exception_message.empty()) - first_exception_message = getCurrentExceptionMessage(false); + auto res = std::make_unique( + request_uri, + http_method, + callback, + timeouts, + credentials, + settings.max_http_get_redirects, + settings.max_read_buffer_size, + read_settings, + headers, + &context->getRemoteHostFilter(), + delay_initialization, + /* use_external_buffer */ false, + /* skip_url_not_found_error */ skip_url_not_found_error); - tryLogCurrentException(__PRETTY_FUNCTION__); - - continue; - } + return std::make_tuple(request_uri, std::move(res)); } + catch (...) + { + if (options == 1) + throw; - return std::make_tuple(request_uri, std::move(res)); + if (first_exception_message.empty()) + first_exception_message = getCurrentExceptionMessage(false); + + tryLogCurrentException(__PRETTY_FUNCTION__); + + continue; + } } throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message); @@ -598,7 +598,7 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( if (it == urls_to_check.cend()) return nullptr; - auto [_, buf_factory] = StorageURLSource::getFirstAvailableURIAndReadBuffer( + auto [_, buf] = StorageURLSource::getFirstAvailableURIAndReadBuffer( it, urls_to_check.cend(), context, @@ -612,7 +612,7 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( false); ++it; return wrapReadBufferWithCompressionMethod( - buf_factory->getReader(), + std::move(buf), compression_method, static_cast(context->getSettingsRef().zstd_window_log_max)); }; diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index d53b72105e4..316b142aec0 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -183,7 +183,7 @@ public: static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); - static std::tuple getFirstAvailableURIAndReadBuffer( + static std::tuple> getFirstAvailableURIAndReadBuffer( std::vector::const_iterator & option, const std::vector::const_iterator & end, ContextPtr context, @@ -205,6 +205,7 @@ private: std::shared_ptr uri_iterator; Poco::URI curr_uri; + std::unique_ptr read_buf; std::unique_ptr pipeline; std::unique_ptr reader; diff --git a/tests/integration/test_redirect_url_storage/test.py b/tests/integration/test_redirect_url_storage/test.py index 06ff78707d7..b2178655444 100644 --- a/tests/integration/test_redirect_url_storage/test.py +++ b/tests/integration/test_redirect_url_storage/test.py @@ -151,7 +151,7 @@ def test_url_reconnect(started_cluster): result = node1.query( "select sum(cityHash64(id)) from url('http://hdfs1:50075/webhdfs/v1/storage_big?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'id Int32') settings http_max_tries = 10, http_retry_max_backoff_ms=1000" ) - assert (int(result), 6581218782194912115) + assert int(result) == 6581218782194912115 thread = threading.Thread(target=select) thread.start() @@ -161,5 +161,5 @@ def test_url_reconnect(started_cluster): thread.join() - assert (int(result), 6581218782194912115) + assert int(result) == 6581218782194912115 assert node1.contains_in_log("Timeout: connect timed out") From 7398b22fa5fc10f015be31035a65f3e6f5bd379f Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 17 May 2023 10:42:52 +0800 Subject: [PATCH 0969/2223] Add redis storage --- src/CMakeLists.txt | 1 + src/Common/ErrorCodes.cpp | 1 + src/Storages/NamedCollectionsHelpers.h | 4 + src/Storages/StorageRedis.cpp | 231 ++++++++++++++++++ src/Storages/StorageRedis.h | 91 +++++++ src/TableFunctions/TableFunctionRedis.cpp | 88 +++++++ src/TableFunctions/TableFunctionRedis.h | 29 +++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + 9 files changed, 447 insertions(+) create mode 100644 src/Storages/StorageRedis.cpp create mode 100644 src/Storages/StorageRedis.h create mode 100644 src/TableFunctions/TableFunctionRedis.cpp create mode 100644 src/TableFunctions/TableFunctionRedis.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 622e18d4ff7..6608d86b5ed 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -412,6 +412,7 @@ dbms_target_link_libraries ( boost::system clickhouse_common_io Poco::MongoDB + Poco::Redis ) if (TARGET ch::mysqlxx) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 83a7314ac7a..505cf0aac8f 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -580,6 +580,7 @@ M(695, ASYNC_LOAD_FAILED) \ M(696, ASYNC_LOAD_CANCELED) \ M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \ + M(698, INVALID_REDIS_STORAGE_TYPE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index 1473a3fbe48..d0d6a526f9b 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -36,6 +36,10 @@ struct MongoDBEqualKeysSet static constexpr std::array, 4> equal_keys{ std::pair{"username", "user"}, std::pair{"database", "db"}, std::pair{"hostname", "host"}, std::pair{"table", "collection"}}; }; +struct RedisEqualKeysSet +{ + static constexpr std::array, 4> equal_keys{std::pair{"hostname", "host"}}; +}; template struct NamedCollectionValidateKey { diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp new file mode 100644 index 00000000000..1daeed255ea --- /dev/null +++ b/src/Storages/StorageRedis.cpp @@ -0,0 +1,231 @@ +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int INVALID_REDIS_STORAGE_TYPE; + extern const int NOT_IMPLEMENTED; +} + +StorageRedis::StorageRedis( + const StorageID & table_id_, + const Configuration & configuration_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment_) : ta +{ + +} + + +Pipe StorageRedis::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /*query_info*/, + ContextPtr /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t /*num_streams*/) +{ + connectIfNotConnected(); + + storage_snapshot->check(column_names); + + Block sample_block; + for (const String & column_name : column_names) + { + auto column_data = storage_snapshot->metadata->getColumns().getPhysical(column_name); + sample_block.insert({column_data.type, column_data.name}); + } + + return Pipe(std::make_shared( + connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size)); +} + + +SinkToStoragePtr StorageRedis::write( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + ContextPtr /*context*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is unsupported for StorageRedis"); +} + +StorageRedis::Configuration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr context) +{ + Configuration configuration; + + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) + { + validateNamedCollection( + *named_collection, + ValidateKeysMultiset{"host", "port", "hostname", "password", "db_id", "storage_type"}, + {}); + + configuration.host = named_collection->getAny({"host", "hostname"}); + configuration.port = static_cast(named_collection->get("port")); + configuration.password = named_collection->get("password"); + configuration.db_id = named_collection->getAny({"db_id"}); + configuration.storage_type = toStorageType(named_collection->getOrDefault("storage_type", "")); + } + else + { + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); + + /// 6379 is the default Redis port. + auto parsed_host_port = parseAddress(checkAndGetLiteralArgument(engine_args[0], "host:port"), 6379); + + configuration.host = parsed_host_port.first; + configuration.port = parsed_host_port.second; + configuration.db_id = checkAndGetLiteralArgument(engine_args[1], "db_id"); + configuration.password = checkAndGetLiteralArgument(engine_args[2], "password"); + configuration.storage_type = toStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); + } + + context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); + + return configuration; +} + +void StorageRedis::connectIfNotConnected() +{ + +} + + +class StorageRedisSink : public SinkToStorage +{ +public: + explicit StorageRedisSink( + const std::string & collection_name_, + const std::string & db_name_, + const StorageMetadataPtr & metadata_snapshot_, + std::shared_ptr connection_) + : SinkToStorage(metadata_snapshot_->getSampleBlock()) + , collection_name(collection_name_) + , db_name(db_name_) + , metadata_snapshot{metadata_snapshot_} + , connection(connection_) + { + } + + String getName() const override { return "StorageRedisSink"; } + + void consume(Chunk chunk) override + { + Poco::MongoDB::Database db(db_name); + Poco::MongoDB::Document::Ptr index = new Poco::MongoDB::Document(); + + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + + size_t num_rows = block.rows(); + size_t num_cols = block.columns(); + + const auto columns = block.getColumns(); + const auto data_types = block.getDataTypes(); + const auto data_names = block.getNames(); + + std::vector row(num_cols); + for (const auto i : collections::range(0, num_rows)) + { + for (const auto j : collections::range(0, num_cols)) + { + WriteBufferFromOwnString ostr; + data_types[j]->getDefaultSerialization()->serializeText(*columns[j], i, ostr, FormatSettings{}); + row[j] = ostr.str(); + index->add(data_names[j], row[j]); + } + } + Poco::SharedPtr insert_request = db.createInsertRequest(collection_name); + insert_request->documents().push_back(index); + connection->sendRequest(*insert_request); + } + +private: + String collection_name; + String db_name; + StorageMetadataPtr metadata_snapshot; + std::shared_ptr connection; +}; + + +using StorageType = StorageRedis::StorageType; + +String StorageRedis::toString(StorageType storage_type) +{ + static const std::unordered_map type_to_str_map + = {{StorageType::SIMPLE, "simple"}, + {StorageType::LIST, "list"}, + {StorageType::SET, "set"}, + {StorageType::HASH, "hash"}, + {StorageType::ZSET, "zset"}}; + + auto iter = type_to_str_map.find(storage_type); + return iter->second; +} + +StorageType StorageRedis::toStorageType(const String & storage_type) +{ + static const std::unordered_map str_to_type_map + = {{"simple", StorageType::SIMPLE}, + {"list", StorageType::LIST}, + {"set", StorageType::SET}, + {"hash", StorageType::HASH}, + {"zset", StorageType::ZSET}}; + + auto iter = str_to_type_map.find(storage_type); + if (iter == str_to_type_map.end()) + { + throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "invalid redis storage type: {}", storage_type); + } + return iter->second; +} + +void registerStorageRedis(StorageFactory & factory) +{ + factory.registerStorage( + "MongoDB", + [](const StorageFactory::Arguments & args) + { + auto configuration = StorageRedis::getConfiguration(args.engine_args, args.getLocalContext()); + + return std::make_shared( + args.table_id, + configuration.host, + configuration.port, + configuration.database, + configuration.table, + configuration.username, + configuration.password, + configuration.options, + args.columns, + args.constraints, + args.comment); + }, + { + .source_access_type = AccessType::MONGO, + }); +} + +} diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h new file mode 100644 index 00000000000..8ba4ec831bb --- /dev/null +++ b/src/Storages/StorageRedis.h @@ -0,0 +1,91 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ +/* Implements storage in the Redis. + * Use ENGINE = Redis(host:port, db_id, password, storage_type); + * Read only. + */ +class StorageRedis : public IStorage +{ +public: + enum class StorageType + { + SIMPLE, + LIST, + SET, + HASH, + ZSET + }; + + static String toString(StorageType storage_type); + static StorageType toStorageType(const String & storage_type); + + struct Configuration + { + String host; + uint32_t port; + String db_id; + String password; + StorageType storage_type; + }; + + using RedisArray = Poco::Redis::Array; + using RedisCommand = Poco::Redis::Command; + + using ClientPtr = std::unique_ptr; + using Pool = BorrowedObjectPool; + using PoolPtr = std::shared_ptr; + + struct Connection + { + Connection(PoolPtr pool_, ClientPtr client_); + ~Connection(); + + PoolPtr pool; + ClientPtr client; + }; + + using ConnectionPtr = std::unique_ptr; + + static Configuration getConfiguration(ASTs engine_args, ContextPtr context); + + StorageRedis( + const StorageID & table_id_, + const Configuration & configuration_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment_); + + std::string getName() const override { return "Redis"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & /*metadata_snapshot*/, + ContextPtr context) override; + +private: + Configuration configuration; + StorageID table_id; + ColumnsDescription columns; + ConstraintsDescription constraints; + String comment; + + std::shared_ptr connection; + void connectIfNotConnected(); +}; + +} diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp new file mode 100644 index 00000000000..9432f766aa8 --- /dev/null +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -0,0 +1,88 @@ +#include + +#include + +#include +#include + +#include +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +StoragePtr TableFunctionRedis::executeImpl( + const ASTPtr & /*ast_function*/, ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/) const +{ + auto columns = getActualTableStructure(context); + auto storage = std::make_shared( + StorageID(configuration->db_id, table_name), configuration, columns, ConstraintsDescription(), String{});// TODO + storage->startup(); + return storage; +} + +ColumnsDescription TableFunctionRedis::getActualTableStructure(ContextPtr context) const +{ + /// generate table structure by storage type. + String structure; + switch (configuration->storage_type) + { + case StorageRedis::StorageType::SIMPLE: + structure = "key String, value String"; + break; + case StorageRedis::StorageType::HASH: + structure = "key String, field, String, value String"; + break; + case StorageRedis::StorageType::LIST: + structure = "key String, value Array(String)"; + break; + case StorageRedis::StorageType::SET: + structure = "key String, value Array(String)"; + break; + case StorageRedis::StorageType::ZSET: + structure = "key String, value Array(String)"; + break; + } + return parseColumnsListFromString(structure, context); +} + +void TableFunctionRedis::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + const auto & func_args = ast_function->as(); + if (!func_args.arguments) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'redis' must have arguments."); + + ASTs & args = func_args.arguments->children; + + if (args.size() != 4) + { + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Table function 'Redis' requires from 4 parameters: " + "redis('host:port', db_id, 'password', 'storage_type')"); + } + configuration = StorageRedis::getConfiguration(args, context); +} + + +void registerTableFunctionRedis(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/TableFunctions/TableFunctionRedis.h b/src/TableFunctions/TableFunctionRedis.h new file mode 100644 index 00000000000..d333cd5a42f --- /dev/null +++ b/src/TableFunctions/TableFunctionRedis.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class TableFunctionRedis : public ITableFunction +{ +public: + static constexpr auto name = "redis"; + String getName() const override { return name; } + +private: + StoragePtr executeImpl( + const ASTPtr & ast_function, ContextPtr context, + const String & table_name, ColumnsDescription cached_columns) const override; + + const char * getStorageTypeName() const override { return "Redis"; } + + ColumnsDescription getActualTableStructure(ContextPtr context) const override; + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + std::optional configuration; +}; + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 4f3411df4c5..bfb83818f22 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -21,6 +21,7 @@ void registerTableFunctions() registerTableFunctionInput(factory); registerTableFunctionGenerate(factory); registerTableFunctionMongoDB(factory); + registerTableFunctionRedis(factory); registerTableFunctionMeiliSearch(factory); diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index c51522a5e99..cf0dee7f792 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -18,6 +18,7 @@ void registerTableFunctionValues(TableFunctionFactory & factory); void registerTableFunctionInput(TableFunctionFactory & factory); void registerTableFunctionGenerate(TableFunctionFactory & factory); void registerTableFunctionMongoDB(TableFunctionFactory & factory); +void registerTableFunctionRedis(TableFunctionFactory & factory); void registerTableFunctionMeiliSearch(TableFunctionFactory & factory); From e91867373cca6d91455b51af05575c48e6d1af9e Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Sat, 20 May 2023 11:48:57 +0800 Subject: [PATCH 0970/2223] Add table function Redis --- src/Access/Common/AccessType.h | 1 + src/Dictionaries/RedisDictionarySource.cpp | 80 +---------- src/Dictionaries/RedisDictionarySource.h | 44 +----- src/Dictionaries/RedisSource.cpp | 7 +- src/Dictionaries/RedisSource.h | 19 +-- src/Storages/RedisCommon.cpp | 98 +++++++++++++ src/Storages/RedisCommon.h | 61 ++++++++ src/Storages/StorageRedis.cpp | 156 +++++---------------- src/Storages/StorageRedis.h | 55 ++------ src/Storages/registerStorages.cpp | 2 + src/TableFunctions/TableFunctionRedis.cpp | 28 ++-- src/TableFunctions/TableFunctionRedis.h | 2 +- 12 files changed, 231 insertions(+), 322 deletions(-) create mode 100644 src/Storages/RedisCommon.cpp create mode 100644 src/Storages/RedisCommon.h diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 6394c0279a7..78c341cdcb5 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -201,6 +201,7 @@ enum class AccessType M(URL, "", GLOBAL, SOURCES) \ M(REMOTE, "", GLOBAL, SOURCES) \ M(MONGO, "", GLOBAL, SOURCES) \ + M(Redis, "", GLOBAL, SOURCES) \ M(MEILISEARCH, "", GLOBAL, SOURCES) \ M(MYSQL, "", GLOBAL, SOURCES) \ M(POSTGRES, "", GLOBAL, SOURCES) \ diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 6e4c5d1d5d9..db27801a38e 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -3,10 +3,6 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" -#include -#include -#include -#include #include #include #include @@ -52,7 +48,7 @@ namespace DB auto port = config.getUInt(redis_config_prefix + ".port"); global_context->getRemoteHostFilter().checkHostAndPort(host, toString(port)); - RedisDictionarySource::Configuration configuration = + RedisConfiguration configuration = { .host = host, .port = static_cast(port), @@ -68,26 +64,13 @@ namespace DB factory.registerSource("redis", create_table_source); } - RedisDictionarySource::Connection::Connection(PoolPtr pool_, ClientPtr client_) - : pool(std::move(pool_)), client(std::move(client_)) - { - } - - RedisDictionarySource::Connection::~Connection() - { - pool->returnObject(std::move(client)); - } - - static constexpr size_t REDIS_MAX_BLOCK_SIZE = DEFAULT_BLOCK_SIZE; - static constexpr size_t REDIS_LOCK_ACQUIRE_TIMEOUT_MS = 5000; - RedisDictionarySource::RedisDictionarySource( const DictionaryStructure & dict_struct_, - const Configuration & configuration_, + const RedisConfiguration & configuration_, const Block & sample_block_) : dict_struct{dict_struct_} , configuration(configuration_) - , pool(std::make_shared(configuration.pool_size)) + , pool(std::make_shared(configuration.pool_size)) , sample_block{sample_block_} { if (dict_struct.attributes.size() != 1) @@ -139,7 +122,7 @@ namespace DB QueryPipeline RedisDictionarySource::loadAll() { - auto connection = getConnection(); + auto connection = getRedisConnection(pool, configuration); RedisCommand command_for_keys("KEYS"); command_for_keys << "*"; @@ -195,7 +178,7 @@ namespace DB QueryPipeline RedisDictionarySource::loadIds(const std::vector & ids) { - auto connection = getConnection(); + auto connection = getRedisConnection(pool, configuration); if (configuration.storage_type == RedisStorageType::HASH_MAP) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use loadIds with 'hash_map' storage type"); @@ -215,7 +198,7 @@ namespace DB QueryPipeline RedisDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { - auto connection = getConnection(); + auto connection = getRedisConnection(pool, configuration); if (key_columns.size() != dict_struct.key->size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The size of key_columns does not equal to the size of dictionary key"); @@ -248,55 +231,4 @@ namespace DB return "Redis: " + configuration.host + ':' + DB::toString(configuration.port); } - RedisDictionarySource::ConnectionPtr RedisDictionarySource::getConnection() const - { - ClientPtr client; - bool ok = pool->tryBorrowObject(client, - [] { return std::make_unique(); }, - REDIS_LOCK_ACQUIRE_TIMEOUT_MS); - - if (!ok) - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, - "Could not get connection from pool, timeout exceeded {} seconds", - REDIS_LOCK_ACQUIRE_TIMEOUT_MS); - - if (!client->isConnected()) - { - try - { - client->connect(configuration.host, configuration.port); - - if (!configuration.password.empty()) - { - RedisCommand command("AUTH"); - command << configuration.password; - String reply = client->execute(command); - if (reply != "OK") - throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, - "Authentication failed with reason {}", reply); - } - - if (configuration.db_index != 0) - { - RedisCommand command("SELECT"); - command << std::to_string(configuration.db_index); - String reply = client->execute(command); - if (reply != "OK") - throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, - "Selecting database with index {} failed with reason {}", - configuration.db_index, reply); - } - } - catch (...) - { - if (client->isConnected()) - client->disconnect(); - - pool->returnObject(std::move(client)); - throw; - } - } - - return std::make_unique(pool, std::move(client)); - } } diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index 8fb6f93193b..c7786284dc4 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -5,6 +5,7 @@ #include "DictionaryStructure.h" #include "IDictionarySource.h" +#include namespace Poco { @@ -23,47 +24,12 @@ namespace DB extern const int NOT_IMPLEMENTED; } - enum class RedisStorageType - { - SIMPLE, - HASH_MAP, - UNKNOWN - }; - class RedisDictionarySource final : public IDictionarySource { public: - using RedisArray = Poco::Redis::Array; - using RedisCommand = Poco::Redis::Command; - - using ClientPtr = std::unique_ptr; - using Pool = BorrowedObjectPool; - using PoolPtr = std::shared_ptr; - - struct Configuration - { - const std::string host; - const UInt16 port; - const UInt32 db_index; - const std::string password; - const RedisStorageType storage_type; - const size_t pool_size; - }; - - struct Connection - { - Connection(PoolPtr pool_, ClientPtr client_); - ~Connection(); - - PoolPtr pool; - ClientPtr client; - }; - - using ConnectionPtr = std::unique_ptr; - RedisDictionarySource( const DictionaryStructure & dict_struct_, - const Configuration & configuration_, + const RedisConfiguration & configuration_, const Block & sample_block_); RedisDictionarySource(const RedisDictionarySource & other); @@ -92,12 +58,10 @@ namespace DB std::string toString() const override; private: - ConnectionPtr getConnection() const; - const DictionaryStructure dict_struct; - const Configuration configuration; + const RedisConfiguration configuration; - PoolPtr pool; + RedisPoolPtr pool; Block sample_block; }; } diff --git a/src/Dictionaries/RedisSource.cpp b/src/Dictionaries/RedisSource.cpp index 4622f65a1a9..9abaf7f0ac5 100644 --- a/src/Dictionaries/RedisSource.cpp +++ b/src/Dictionaries/RedisSource.cpp @@ -3,11 +3,6 @@ #include #include -#include -#include -#include -#include - #include #include #include @@ -30,7 +25,7 @@ namespace DB RedisSource::RedisSource( - ConnectionPtr connection_, + RedisConnectionPtr connection_, const RedisArray & keys_, const RedisStorageType & storage_type_, const DB::Block & sample_block, diff --git a/src/Dictionaries/RedisSource.h b/src/Dictionaries/RedisSource.h index 0f8cc317003..4537e496061 100644 --- a/src/Dictionaries/RedisSource.h +++ b/src/Dictionaries/RedisSource.h @@ -6,29 +6,18 @@ #include #include #include +#include #include "RedisDictionarySource.h" -namespace Poco -{ - namespace Redis - { - class Client; - } -} - namespace DB { class RedisSource final : public ISource { public: - using RedisArray = Poco::Redis::Array; - using RedisBulkString = Poco::Redis::BulkString; - using ConnectionPtr = RedisDictionarySource::ConnectionPtr; - RedisSource( - ConnectionPtr connection_, - const Poco::Redis::Array & keys_, + RedisConnectionPtr connection_, + const RedisArray & keys_, const RedisStorageType & storage_type_, const Block & sample_block, size_t max_block_size); @@ -40,7 +29,7 @@ namespace DB private: Chunk generate() override; - ConnectionPtr connection; + RedisConnectionPtr connection; Poco::Redis::Array keys; RedisStorageType storage_type; const size_t max_block_size; diff --git a/src/Storages/RedisCommon.cpp b/src/Storages/RedisCommon.cpp new file mode 100644 index 00000000000..397189e7485 --- /dev/null +++ b/src/Storages/RedisCommon.cpp @@ -0,0 +1,98 @@ +#include "RedisCommon.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_REDIS_STORAGE_TYPE; + extern const int INTERNAL_REDIS_ERROR; + extern const int TIMEOUT_EXCEEDED; +} + +RedisConnection::RedisConnection(RedisPoolPtr pool_, RedisClientPtr client_) + : pool(std::move(pool_)), client(std::move(client_)) +{ +} + +RedisConnection::~RedisConnection() +{ + pool->returnObject(std::move(client)); +} + +String toString(RedisStorageType storage_type) +{ + static const std::unordered_map type_to_str_map + = {{RedisStorageType::SIMPLE, "simple"}, {RedisStorageType::HASH_MAP, "hash_map"}}; + + auto iter = type_to_str_map.find(storage_type); + return iter->second; +} + +RedisStorageType toRedisStorageType(const String & storage_type) +{ + static const std::unordered_map str_to_type_map + = {{"simple", RedisStorageType::SIMPLE}, {"hash", RedisStorageType::HASH_MAP}}; + + auto iter = str_to_type_map.find(storage_type); + if (iter == str_to_type_map.end()) + { + throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "invalid redis storage type: {}", storage_type); + } + return iter->second; +} + +RedisConnectionPtr getRedisConnection(RedisPoolPtr pool, const RedisConfiguration & configuration) +{ + RedisClientPtr client; + bool ok = pool->tryBorrowObject(client, + [] { return std::make_unique(); }, + REDIS_LOCK_ACQUIRE_TIMEOUT_MS); + + if (!ok) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, + "Could not get connection from pool, timeout exceeded {} seconds", + REDIS_LOCK_ACQUIRE_TIMEOUT_MS); + + if (!client->isConnected()) + { + try + { + client->connect(configuration.host, configuration.port); + + if (!configuration.password.empty()) + { + RedisCommand command("AUTH"); + command << configuration.password; + String reply = client->execute(command); + if (reply != "OK") + throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, + "Authentication failed with reason {}", reply); + } + + if (configuration.db_index != 0) + { + RedisCommand command("SELECT"); + command << std::to_string(configuration.db_index); + String reply = client->execute(command); + if (reply != "OK") + throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, + "Selecting database with index {} failed with reason {}", + configuration.db_index, reply); + } + } + catch (...) + { + if (client->isConnected()) + client->disconnect(); + + pool->returnObject(std::move(client)); + throw; + } + } + + return std::make_unique(pool, std::move(client)); +} + +} diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h new file mode 100644 index 00000000000..6069d3d9a0c --- /dev/null +++ b/src/Storages/RedisCommon.h @@ -0,0 +1,61 @@ +#pragma once + +#include +#include +#include + +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +static constexpr size_t REDIS_MAX_BLOCK_SIZE = DEFAULT_BLOCK_SIZE; +static constexpr size_t REDIS_LOCK_ACQUIRE_TIMEOUT_MS = 5000; + +enum class RedisStorageType +{ + SIMPLE, + HASH_MAP, + UNKNOWN +}; + +String toString(RedisStorageType storage_type); +RedisStorageType toRedisStorageType(const String & storage_type); + +struct RedisConfiguration +{ + String host; + uint32_t port; + uint32_t db_index; + String password; + RedisStorageType storage_type; + uint32_t pool_size; +}; + +using RedisArray = Poco::Redis::Array; +using RedisCommand = Poco::Redis::Command; +using RedisBulkString = Poco::Redis::BulkString; + +using RedisClientPtr = std::unique_ptr; +using RedisPool = BorrowedObjectPool; +using RedisPoolPtr = std::shared_ptr; + +struct RedisConnection +{ + RedisConnection(RedisPoolPtr pool_, RedisClientPtr client_); + ~RedisConnection(); + + RedisPoolPtr pool; + RedisClientPtr client; +}; + +using RedisConnectionPtr = std::unique_ptr; + +RedisConnectionPtr getRedisConnection(RedisPoolPtr pool, const RedisConfiguration & configuration) ; + +} diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 1daeed255ea..055617b6a96 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -1,6 +1,6 @@ -#include -#include #include +#include +#include #include #include @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -23,21 +22,25 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int INVALID_REDIS_STORAGE_TYPE; extern const int NOT_IMPLEMENTED; } StorageRedis::StorageRedis( const StorageID & table_id_, - const Configuration & configuration_, + const RedisConfiguration & configuration_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - const String & comment_) : ta + const String & comment_) + : IStorage(table_id_) + , table_id(table_id_) + , configuration(configuration_) + , columns(columns_) + , constraints(constraints_) + , comment(comment_) { - + pool = std::make_shared(configuration.pool_size); } - Pipe StorageRedis::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -47,7 +50,7 @@ Pipe StorageRedis::read( size_t max_block_size, size_t /*num_streams*/) { - connectIfNotConnected(); + auto connection = getRedisConnection(pool, configuration); storage_snapshot->check(column_names); @@ -58,8 +61,14 @@ Pipe StorageRedis::read( sample_block.insert({column_data.type, column_data.name}); } - return Pipe(std::make_shared( - connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size)); + RedisArray keys; + RedisCommand command_for_keys("KEYS"); + /// generate keys by table name prefix + command_for_keys << table_id.getTableName() + ":" + toString(configuration.storage_type) + ":*"; + + /// Get only keys for specified storage type. + auto all_keys = connection->client->execute(command_for_keys); + return Pipe(std::make_shared(std::move(connection), all_keys, configuration.storage_type, sample_block, max_block_size)); } @@ -71,22 +80,23 @@ SinkToStoragePtr StorageRedis::write( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is unsupported for StorageRedis"); } -StorageRedis::Configuration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr context) +RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr context) { - Configuration configuration; + RedisConfiguration configuration; if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) { validateNamedCollection( *named_collection, - ValidateKeysMultiset{"host", "port", "hostname", "password", "db_id", "storage_type"}, + ValidateKeysMultiset{"host", "port", "hostname", "password", "db_index", "storage_type"}, {}); configuration.host = named_collection->getAny({"host", "hostname"}); - configuration.port = static_cast(named_collection->get("port")); + configuration.port = static_cast(named_collection->get("port")); configuration.password = named_collection->get("password"); - configuration.db_id = named_collection->getAny({"db_id"}); - configuration.storage_type = toStorageType(named_collection->getOrDefault("storage_type", "")); + configuration.db_index = static_cast(named_collection->get({"db_index"})); + configuration.storage_type = toRedisStorageType(named_collection->getOrDefault("storage_type", "")); + configuration.pool_size = 16; /// TODO } else { @@ -98,133 +108,33 @@ StorageRedis::Configuration StorageRedis::getConfiguration(ASTs engine_args, Con configuration.host = parsed_host_port.first; configuration.port = parsed_host_port.second; - configuration.db_id = checkAndGetLiteralArgument(engine_args[1], "db_id"); + configuration.db_index = static_cast(checkAndGetLiteralArgument(engine_args[1], "db_index")); configuration.password = checkAndGetLiteralArgument(engine_args[2], "password"); - configuration.storage_type = toStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); + configuration.storage_type = toRedisStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); + configuration.pool_size = 16; /// TODO } context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); - return configuration; } -void StorageRedis::connectIfNotConnected() -{ - -} - - -class StorageRedisSink : public SinkToStorage -{ -public: - explicit StorageRedisSink( - const std::string & collection_name_, - const std::string & db_name_, - const StorageMetadataPtr & metadata_snapshot_, - std::shared_ptr connection_) - : SinkToStorage(metadata_snapshot_->getSampleBlock()) - , collection_name(collection_name_) - , db_name(db_name_) - , metadata_snapshot{metadata_snapshot_} - , connection(connection_) - { - } - - String getName() const override { return "StorageRedisSink"; } - - void consume(Chunk chunk) override - { - Poco::MongoDB::Database db(db_name); - Poco::MongoDB::Document::Ptr index = new Poco::MongoDB::Document(); - - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); - - size_t num_rows = block.rows(); - size_t num_cols = block.columns(); - - const auto columns = block.getColumns(); - const auto data_types = block.getDataTypes(); - const auto data_names = block.getNames(); - - std::vector row(num_cols); - for (const auto i : collections::range(0, num_rows)) - { - for (const auto j : collections::range(0, num_cols)) - { - WriteBufferFromOwnString ostr; - data_types[j]->getDefaultSerialization()->serializeText(*columns[j], i, ostr, FormatSettings{}); - row[j] = ostr.str(); - index->add(data_names[j], row[j]); - } - } - Poco::SharedPtr insert_request = db.createInsertRequest(collection_name); - insert_request->documents().push_back(index); - connection->sendRequest(*insert_request); - } - -private: - String collection_name; - String db_name; - StorageMetadataPtr metadata_snapshot; - std::shared_ptr connection; -}; - - -using StorageType = StorageRedis::StorageType; - -String StorageRedis::toString(StorageType storage_type) -{ - static const std::unordered_map type_to_str_map - = {{StorageType::SIMPLE, "simple"}, - {StorageType::LIST, "list"}, - {StorageType::SET, "set"}, - {StorageType::HASH, "hash"}, - {StorageType::ZSET, "zset"}}; - - auto iter = type_to_str_map.find(storage_type); - return iter->second; -} - -StorageType StorageRedis::toStorageType(const String & storage_type) -{ - static const std::unordered_map str_to_type_map - = {{"simple", StorageType::SIMPLE}, - {"list", StorageType::LIST}, - {"set", StorageType::SET}, - {"hash", StorageType::HASH}, - {"zset", StorageType::ZSET}}; - - auto iter = str_to_type_map.find(storage_type); - if (iter == str_to_type_map.end()) - { - throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "invalid redis storage type: {}", storage_type); - } - return iter->second; -} - void registerStorageRedis(StorageFactory & factory) { factory.registerStorage( - "MongoDB", + "Redis", [](const StorageFactory::Arguments & args) { auto configuration = StorageRedis::getConfiguration(args.engine_args, args.getLocalContext()); return std::make_shared( args.table_id, - configuration.host, - configuration.port, - configuration.database, - configuration.table, - configuration.username, - configuration.password, - configuration.options, + configuration, args.columns, args.constraints, args.comment); }, { - .source_access_type = AccessType::MONGO, + .source_access_type = AccessType::Redis, }); } diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h index 8ba4ec831bb..60db75dd384 100644 --- a/src/Storages/StorageRedis.h +++ b/src/Storages/StorageRedis.h @@ -7,56 +7,15 @@ namespace DB { /* Implements storage in the Redis. - * Use ENGINE = Redis(host:port, db_id, password, storage_type); + * Use ENGINE = Redis(host:port, db_index, password, storage_type); * Read only. */ class StorageRedis : public IStorage { public: - enum class StorageType - { - SIMPLE, - LIST, - SET, - HASH, - ZSET - }; - - static String toString(StorageType storage_type); - static StorageType toStorageType(const String & storage_type); - - struct Configuration - { - String host; - uint32_t port; - String db_id; - String password; - StorageType storage_type; - }; - - using RedisArray = Poco::Redis::Array; - using RedisCommand = Poco::Redis::Command; - - using ClientPtr = std::unique_ptr; - using Pool = BorrowedObjectPool; - using PoolPtr = std::shared_ptr; - - struct Connection - { - Connection(PoolPtr pool_, ClientPtr client_); - ~Connection(); - - PoolPtr pool; - ClientPtr client; - }; - - using ConnectionPtr = std::unique_ptr; - - static Configuration getConfiguration(ASTs engine_args, ContextPtr context); - StorageRedis( const StorageID & table_id_, - const Configuration & configuration_, + const RedisConfiguration & configuration_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const String & comment_); @@ -77,15 +36,17 @@ public: const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + static RedisConfiguration getConfiguration(ASTs engine_args, ContextPtr context); + private: - Configuration configuration; StorageID table_id; + RedisConfiguration configuration; + ColumnsDescription columns; ConstraintsDescription constraints; - String comment; - std::shared_ptr connection; - void connectIfNotConnected(); + String comment; + RedisPoolPtr pool; }; } diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 8be176a5375..84994298b8e 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -59,6 +59,7 @@ void registerStorageMySQL(StorageFactory & factory); #endif void registerStorageMongoDB(StorageFactory & factory); +void registerStorageRedis(StorageFactory & factory); #if USE_RDKAFKA @@ -156,6 +157,7 @@ void registerStorages() #endif registerStorageMongoDB(factory); + registerStorageRedis(factory); #if USE_RDKAFKA registerStorageKafka(factory); diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp index 9432f766aa8..db612806652 100644 --- a/src/TableFunctions/TableFunctionRedis.cpp +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -1,17 +1,15 @@ #include #include +#include #include -#include #include #include -#include #include #include -#include #include #include @@ -23,6 +21,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int INVALID_REDIS_STORAGE_TYPE; } @@ -31,7 +30,11 @@ StoragePtr TableFunctionRedis::executeImpl( { auto columns = getActualTableStructure(context); auto storage = std::make_shared( - StorageID(configuration->db_id, table_name), configuration, columns, ConstraintsDescription(), String{});// TODO + StorageID(toString(configuration->db_index), table_name), // TODO + *configuration, + columns, + ConstraintsDescription(), + String{}); storage->startup(); return storage; } @@ -42,21 +45,14 @@ ColumnsDescription TableFunctionRedis::getActualTableStructure(ContextPtr contex String structure; switch (configuration->storage_type) { - case StorageRedis::StorageType::SIMPLE: + case RedisStorageType::SIMPLE: structure = "key String, value String"; break; - case StorageRedis::StorageType::HASH: + case RedisStorageType::HASH_MAP: structure = "key String, field, String, value String"; break; - case StorageRedis::StorageType::LIST: - structure = "key String, value Array(String)"; - break; - case StorageRedis::StorageType::SET: - structure = "key String, value Array(String)"; - break; - case StorageRedis::StorageType::ZSET: - structure = "key String, value Array(String)"; - break; + case RedisStorageType::UNKNOWN: + throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "invalid redis storage type."); } return parseColumnsListFromString(structure, context); } @@ -74,7 +70,7 @@ void TableFunctionRedis::parseArguments(const ASTPtr & ast_function, ContextPtr throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function 'Redis' requires from 4 parameters: " - "redis('host:port', db_id, 'password', 'storage_type')"); + "redis('host:port', db_index, 'password', 'storage_type')"); } configuration = StorageRedis::getConfiguration(args, context); } diff --git a/src/TableFunctions/TableFunctionRedis.h b/src/TableFunctions/TableFunctionRedis.h index d333cd5a42f..5c6f483fda7 100644 --- a/src/TableFunctions/TableFunctionRedis.h +++ b/src/TableFunctions/TableFunctionRedis.h @@ -23,7 +23,7 @@ private: ColumnsDescription getActualTableStructure(ContextPtr context) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - std::optional configuration; + std::optional configuration; }; } From 9a495cbf997d4c0c4ff00b9c75f958b5fa7292d6 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 23 May 2023 15:31:50 +0800 Subject: [PATCH 0971/2223] Push down filter into Redis --- src/Dictionaries/RedisDictionarySource.cpp | 43 +----- src/Dictionaries/RedisSource.cpp | 52 +++++++- src/Dictionaries/RedisSource.h | 10 ++ src/Storages/RedisCommon.cpp | 68 +++++++--- src/Storages/RedisCommon.h | 28 +++- src/Storages/StorageRedis.cpp | 144 ++++++++++++++++++--- src/Storages/StorageRedis.h | 11 +- src/TableFunctions/TableFunctionRedis.cpp | 10 +- 8 files changed, 271 insertions(+), 95 deletions(-) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index db27801a38e..f96c9231827 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -105,21 +105,6 @@ namespace DB RedisDictionarySource::~RedisDictionarySource() = default; - static String storageTypeToKeyType(RedisStorageType type) - { - switch (type) - { - case RedisStorageType::SIMPLE: - return "string"; - case RedisStorageType::HASH_MAP: - return "hash"; - default: - return "none"; - } - - UNREACHABLE(); - } - QueryPipeline RedisDictionarySource::loadAll() { auto connection = getRedisConnection(pool, configuration); @@ -142,33 +127,7 @@ namespace DB if (configuration.storage_type == RedisStorageType::HASH_MAP) { - RedisArray hkeys; - for (const auto & key : keys) - { - RedisCommand command_for_secondary_keys("HKEYS"); - command_for_secondary_keys.addRedisType(key); - - auto secondary_keys = connection->client->execute(command_for_secondary_keys); - - RedisArray primary_with_secondary; - primary_with_secondary.addRedisType(key); - for (const auto & secondary_key : secondary_keys) - { - primary_with_secondary.addRedisType(secondary_key); - /// Do not store more than max_block_size values for one request. - if (primary_with_secondary.size() == REDIS_MAX_BLOCK_SIZE + 1) - { - hkeys.add(primary_with_secondary); - primary_with_secondary.clear(); - primary_with_secondary.addRedisType(key); - } - } - - if (primary_with_secondary.size() > 1) - hkeys.add(primary_with_secondary); - } - - keys = hkeys; + keys = *getRedisHashMapKeys(connection, keys); } return QueryPipeline(std::make_shared( diff --git a/src/Dictionaries/RedisSource.cpp b/src/Dictionaries/RedisSource.cpp index 9abaf7f0ac5..20e0838886c 100644 --- a/src/Dictionaries/RedisSource.cpp +++ b/src/Dictionaries/RedisSource.cpp @@ -30,11 +30,29 @@ namespace DB const RedisStorageType & storage_type_, const DB::Block & sample_block, size_t max_block_size_) + : ISource(sample_block), max_block_size(max_block_size_)// TODO + { + RedisColumnTypes columns_types_; + if (storage_type_ == RedisStorageType::HASH_MAP) + columns_types_ = REDIS_HASH_MAP_COLUMN_TYPES; + else + columns_types_ = REDIS_SIMPLE_COLUMN_TYPES; + RedisSource(std::move(connection_), keys_, storage_type_, sample_block, columns_types_, max_block_size_); + } + + RedisSource::RedisSource( + RedisConnectionPtr connection_, + const RedisArray & keys_, + const RedisStorageType & storage_type_, + const DB::Block & sample_block, + const RedisColumnTypes & columns_types_, + size_t max_block_size_) : ISource(sample_block) , connection(std::move(connection_)) , keys(keys_) , storage_type(storage_type_) , max_block_size{max_block_size_} + , columns_types(columns_types_) { description.init(sample_block); } @@ -173,15 +191,27 @@ namespace DB const auto & primary_key = keys_array.get(0); for (size_t i = 0; i < values.size(); ++i) { - const auto & secondary_key = keys_array.get(i + 1); const auto & value = values.get(i); + const auto & secondary_key = keys_array.get(i + 1); /// null string means 'no value for requested key' if (!value.isNull()) { - insert_value_by_idx(0, primary_key); - insert_value_by_idx(1, secondary_key); - insert_value_by_idx(2, value); + for (size_t idx=0; idxreturnObject(std::move(client)); } -String toString(RedisStorageType storage_type) +String storageTypeToKeyType(RedisStorageType storage_type) { - static const std::unordered_map type_to_str_map - = {{RedisStorageType::SIMPLE, "simple"}, {RedisStorageType::HASH_MAP, "hash_map"}}; - - auto iter = type_to_str_map.find(storage_type); - return iter->second; + switch (storage_type) + { + case RedisStorageType::SIMPLE: + return "string"; + case RedisStorageType::HASH_MAP: + return "hash"; + default: + return "none"; + } } -RedisStorageType toRedisStorageType(const String & storage_type) +RedisStorageType keyTypeToStorageType(const String & key_type) { - static const std::unordered_map str_to_type_map - = {{"simple", RedisStorageType::SIMPLE}, {"hash", RedisStorageType::HASH_MAP}}; - - auto iter = str_to_type_map.find(storage_type); - if (iter == str_to_type_map.end()) - { - throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "invalid redis storage type: {}", storage_type); - } - return iter->second; + if (key_type == "string") + return RedisStorageType::SIMPLE; + else if (key_type == "hash") + return RedisStorageType::HASH_MAP; + else + return RedisStorageType::UNKNOWN; } RedisConnectionPtr getRedisConnection(RedisPoolPtr pool, const RedisConfiguration & configuration) @@ -95,4 +99,36 @@ RedisConnectionPtr getRedisConnection(RedisPoolPtr pool, const RedisConfiguratio return std::make_unique(pool, std::move(client)); } + +RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisArray & keys) +{ + RedisArrayPtr hkeys = std::make_shared(); + for (const auto & key : keys) + { + RedisCommand command_for_secondary_keys("HKEYS"); + command_for_secondary_keys.addRedisType(key); + + auto secondary_keys = connection->client->execute(command_for_secondary_keys); + + RedisArray primary_with_secondary; + primary_with_secondary.addRedisType(key); + for (const auto & secondary_key : secondary_keys) + { + primary_with_secondary.addRedisType(secondary_key); + /// Do not store more than max_block_size values for one request. + if (primary_with_secondary.size() == REDIS_MAX_BLOCK_SIZE + 1) + { + hkeys->add(primary_with_secondary); + primary_with_secondary.clear(); + primary_with_secondary.addRedisType(key); + } + } + + if (primary_with_secondary.size() > 1) + hkeys->add(primary_with_secondary); + } + + return hkeys; +} + } diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index 6069d3d9a0c..590ea1476c4 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -24,8 +24,23 @@ enum class RedisStorageType UNKNOWN }; -String toString(RedisStorageType storage_type); -RedisStorageType toRedisStorageType(const String & storage_type); +enum class RedisColumnType +{ + /// Redis key + KEY, + /// Redis map field + FIELD, + /// Redis value + VALUE +}; + +using RedisColumnTypes = std::vector; + +extern RedisColumnTypes REDIS_HASH_MAP_COLUMN_TYPES; +extern RedisColumnTypes REDIS_SIMPLE_COLUMN_TYPES; + +String storageTypeToKeyType(RedisStorageType storage_type); +RedisStorageType keyTypeToStorageType(const String & key_type); struct RedisConfiguration { @@ -34,10 +49,13 @@ struct RedisConfiguration uint32_t db_index; String password; RedisStorageType storage_type; + /// column name of redis key + String key;// TODO remove uint32_t pool_size; }; using RedisArray = Poco::Redis::Array; +using RedisArrayPtr = std::shared_ptr; using RedisCommand = Poco::Redis::Command; using RedisBulkString = Poco::Redis::BulkString; @@ -56,6 +74,10 @@ struct RedisConnection using RedisConnectionPtr = std::unique_ptr; -RedisConnectionPtr getRedisConnection(RedisPoolPtr pool, const RedisConfiguration & configuration) ; +RedisConnectionPtr getRedisConnection(RedisPoolPtr pool, const RedisConfiguration & configuration); + +///get all redis hash key array +/// eg: keys -> [key1, key2] and get [[key1, field1, field2], [key2, field1, field2]] +RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisArray & keys); } diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 055617b6a96..45ebe0696d6 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -11,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -22,9 +24,33 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int INVALID_REDIS_STORAGE_TYPE; extern const int NOT_IMPLEMENTED; } +namespace +{ + RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column) + { + String redis_col_key = all_columns.at(0); + if (column == redis_col_key) + return RedisColumnType::KEY; + + if (storage_type == RedisStorageType::HASH_MAP) + { + String redis_col_field = all_columns.at(1); + if (column == redis_col_field) + return RedisColumnType::FIELD; + else + return RedisColumnType::VALUE; + } + else + { + return RedisColumnType::VALUE; + } + } +} + StorageRedis::StorageRedis( const StorageID & table_id_, const RedisConfiguration & configuration_, @@ -34,41 +60,120 @@ StorageRedis::StorageRedis( : IStorage(table_id_) , table_id(table_id_) , configuration(configuration_) - , columns(columns_) - , constraints(constraints_) - , comment(comment_) + , log(&Poco::Logger::get("StorageRedis")) { pool = std::make_shared(configuration.pool_size); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment_); + setInMemoryMetadata(storage_metadata); } Pipe StorageRedis::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & /*query_info*/, - ContextPtr /*context*/, + SelectQueryInfo & query_info, + ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - size_t /*num_streams*/) + size_t num_streams) { + LOG_INFO(log, "num_streams {}", num_streams);// TODO delete auto connection = getRedisConnection(pool, configuration); storage_snapshot->check(column_names); Block sample_block; + RedisColumnTypes redis_types; + auto all_columns = storage_snapshot->metadata->getColumns().getNamesOfPhysical(); + for (const String & column_name : column_names) { auto column_data = storage_snapshot->metadata->getColumns().getPhysical(column_name); sample_block.insert({column_data.type, column_data.name}); + redis_types.push_back(getRedisColumnType(configuration.storage_type, all_columns, column_name)); + LOG_INFO(log, "Request column: {}, Redis type: {}", column_data.name, *redis_types.crbegin()); // TODO delete } - RedisArray keys; - RedisCommand command_for_keys("KEYS"); - /// generate keys by table name prefix - command_for_keys << table_id.getTableName() + ":" + toString(configuration.storage_type) + ":*"; + FieldVectorPtr fields; + bool all_scan = false; - /// Get only keys for specified storage type. - auto all_keys = connection->client->execute(command_for_keys); - return Pipe(std::make_shared(std::move(connection), all_keys, configuration.storage_type, sample_block, max_block_size)); + String primary_key = all_columns.at(0); + auto primary_key_data_type = sample_block.getByName(primary_key).type; + + std::tie(fields, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info, context); + + /// TODO hash_map hgetall + if (all_scan) + { + RedisCommand command_for_keys("KEYS"); + /// generate keys by table name prefix + command_for_keys << table_id.getTableName() + ":" + toString(configuration.storage_type) + ":*"; + + auto all_keys = connection->client->execute(command_for_keys); + + if (all_keys.size() == 0) + return {}; + + Pipes pipes; + + size_t num_keys = all_keys.size(); + size_t num_threads = std::min(num_streams, all_keys.size()); + + assert(num_keys <= std::numeric_limits::max()); + + for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) + { + size_t begin = num_keys * thread_idx / num_threads; + size_t end = num_keys * (thread_idx + 1) / num_threads; + + RedisArray keys; + for (size_t pos=begin; pos(pos)); + + if (configuration.storage_type == RedisStorageType::HASH_MAP) + { + keys = *getRedisHashMapKeys(connection, keys); + } + + /// TODO reduce keys copy + pipes.emplace_back(std::make_shared( + std::move(connection), keys, configuration.storage_type, sample_block, redis_types, max_block_size)); + } + return Pipe::unitePipes(std::move(pipes)); + } + else + { + if (fields->empty()) + return {}; + + Pipes pipes; + + size_t num_keys = fields->size(); + size_t num_threads = std::min(num_streams, fields->size()); + + assert(num_keys <= std::numeric_limits::max()); + + for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) + { + size_t begin = num_keys * thread_idx / num_threads; + size_t end = num_keys * (thread_idx + 1) / num_threads; + + RedisArray keys; + for (size_t pos=begin; posat(pos).get()); + + if (configuration.storage_type == RedisStorageType::HASH_MAP) + { + keys = *getRedisHashMapKeys(connection, keys); + } + + pipes.emplace_back(std::make_shared( + std::move(connection), keys, configuration.storage_type, sample_block, redis_types, max_block_size)); + } + return Pipe::unitePipes(std::move(pipes)); + } } @@ -88,15 +193,15 @@ RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr c { validateNamedCollection( *named_collection, - ValidateKeysMultiset{"host", "port", "hostname", "password", "db_index", "storage_type"}, + ValidateKeysMultiset{"host", "port", "hostname", "password", "db_index", "storage_type", "pool_size"}, {}); configuration.host = named_collection->getAny({"host", "hostname"}); configuration.port = static_cast(named_collection->get("port")); configuration.password = named_collection->get("password"); configuration.db_index = static_cast(named_collection->get({"db_index"})); - configuration.storage_type = toRedisStorageType(named_collection->getOrDefault("storage_type", "")); - configuration.pool_size = 16; /// TODO + configuration.storage_type = keyTypeToStorageType(named_collection->getOrDefault("storage_type", "")); + configuration.pool_size = static_cast(named_collection->get("pool_size")); } else { @@ -110,10 +215,13 @@ RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr c configuration.port = parsed_host_port.second; configuration.db_index = static_cast(checkAndGetLiteralArgument(engine_args[1], "db_index")); configuration.password = checkAndGetLiteralArgument(engine_args[2], "password"); - configuration.storage_type = toRedisStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); - configuration.pool_size = 16; /// TODO + configuration.storage_type = keyTypeToStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); + configuration.pool_size = static_cast(checkAndGetLiteralArgument(engine_args[4], "pool_size")); } + if (configuration.storage_type == RedisStorageType::UNKNOWN) + throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "Invalid Redis storage type"); + context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); return configuration; } diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h index 60db75dd384..1bffc6a64bf 100644 --- a/src/Storages/StorageRedis.h +++ b/src/Storages/StorageRedis.h @@ -7,8 +7,12 @@ namespace DB { /* Implements storage in the Redis. - * Use ENGINE = Redis(host:port, db_index, password, storage_type); + * Use ENGINE = Redis(host:port, db_index, password, storage_type, conn_pool_size); * Read only. + * + * Note If storage_type is + * simple: there should be 2 columns and the first one is key in Redis, the second one is value. + * hash_map: there should be 3 columns and the first one is key in Redis and the second is the field of Redis Map. */ class StorageRedis : public IStorage { @@ -42,10 +46,7 @@ private: StorageID table_id; RedisConfiguration configuration; - ColumnsDescription columns; - ConstraintsDescription constraints; - - String comment; + Poco::Logger * log; RedisPoolPtr pool; }; diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp index db612806652..e410ad799a6 100644 --- a/src/TableFunctions/TableFunctionRedis.cpp +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -49,10 +49,10 @@ ColumnsDescription TableFunctionRedis::getActualTableStructure(ContextPtr contex structure = "key String, value String"; break; case RedisStorageType::HASH_MAP: - structure = "key String, field, String, value String"; + structure = "key String, field String, value String"; break; case RedisStorageType::UNKNOWN: - throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "invalid redis storage type."); + throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "Invalid Redis storage type."); } return parseColumnsListFromString(structure, context); } @@ -65,12 +65,12 @@ void TableFunctionRedis::parseArguments(const ASTPtr & ast_function, ContextPtr ASTs & args = func_args.arguments->children; - if (args.size() != 4) + if (args.size() != 5) { throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Table function 'Redis' requires from 4 parameters: " - "redis('host:port', db_index, 'password', 'storage_type')"); + "Table function 'Redis' requires from 5 parameters: " + "redis('host:port', db_index, 'password', 'storage_type', 'pool_size')"); } configuration = StorageRedis::getConfiguration(args, context); } From 23d6c835d831b2fa9650c693370219ee1d1f9727 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 23 May 2023 16:42:46 +0800 Subject: [PATCH 0972/2223] fix poco redis array NPE --- src/Storages/RedisCommon.cpp | 22 ++++++++++++++++++++++ src/Storages/RedisCommon.h | 7 ++++++- src/Storages/StorageRedis.cpp | 35 ++--------------------------------- src/Storages/StorageRedis.h | 4 ++-- 4 files changed, 32 insertions(+), 36 deletions(-) diff --git a/src/Storages/RedisCommon.cpp b/src/Storages/RedisCommon.cpp index b910759fe52..0a13e40b1ec 100644 --- a/src/Storages/RedisCommon.cpp +++ b/src/Storages/RedisCommon.cpp @@ -109,6 +109,8 @@ RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisAr command_for_secondary_keys.addRedisType(key); auto secondary_keys = connection->client->execute(command_for_secondary_keys); + if (secondary_keys.isNull()) + continue; RedisArray primary_with_secondary; primary_with_secondary.addRedisType(key); @@ -131,4 +133,24 @@ RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisAr return hkeys; } +RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column) +{ + String redis_col_key = all_columns.at(0); + if (column == redis_col_key) + return RedisColumnType::KEY; + + if (storage_type == RedisStorageType::HASH_MAP) + { + String redis_col_field = all_columns.at(1); + if (column == redis_col_field) + return RedisColumnType::FIELD; + else + return RedisColumnType::VALUE; + } + else + { + return RedisColumnType::VALUE; + } +} + } diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index 590ea1476c4..384a02d76e4 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -6,6 +6,7 @@ #include #include +#include namespace DB { @@ -28,7 +29,7 @@ enum class RedisColumnType { /// Redis key KEY, - /// Redis map field + /// Redis hash field FIELD, /// Redis value VALUE @@ -80,4 +81,8 @@ RedisConnectionPtr getRedisConnection(RedisPoolPtr pool, const RedisConfiguratio /// eg: keys -> [key1, key2] and get [[key1, field1, field2], [key2, field1, field2]] RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisArray & keys); +/// Get RedisColumnType of a column, If storage_type is +/// SIMPLE: all_columns must have 2 iterm and the first one is Redis key the second one is value +/// HASH_MAP: all_columns must have 2 iterm and the first one is Redis key the second is field, the third is value. +RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column); } diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 45ebe0696d6..0e1b3a24e4f 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -5,15 +5,11 @@ #include #include -#include -#include -#include #include #include #include #include #include -#include #include #include #include @@ -28,29 +24,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -namespace -{ - RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column) - { - String redis_col_key = all_columns.at(0); - if (column == redis_col_key) - return RedisColumnType::KEY; - - if (storage_type == RedisStorageType::HASH_MAP) - { - String redis_col_field = all_columns.at(1); - if (column == redis_col_field) - return RedisColumnType::FIELD; - else - return RedisColumnType::VALUE; - } - else - { - return RedisColumnType::VALUE; - } - } -} - StorageRedis::StorageRedis( const StorageID & table_id_, const RedisConfiguration & configuration_, @@ -79,9 +52,7 @@ Pipe StorageRedis::read( size_t max_block_size, size_t num_streams) { - LOG_INFO(log, "num_streams {}", num_streams);// TODO delete auto connection = getRedisConnection(pool, configuration); - storage_snapshot->check(column_names); Block sample_block; @@ -93,7 +64,6 @@ Pipe StorageRedis::read( auto column_data = storage_snapshot->metadata->getColumns().getPhysical(column_name); sample_block.insert({column_data.type, column_data.name}); redis_types.push_back(getRedisColumnType(configuration.storage_type, all_columns, column_name)); - LOG_INFO(log, "Request column: {}, Redis type: {}", column_data.name, *redis_types.crbegin()); // TODO delete } FieldVectorPtr fields; @@ -104,16 +74,15 @@ Pipe StorageRedis::read( std::tie(fields, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info, context); - /// TODO hash_map hgetall if (all_scan) { RedisCommand command_for_keys("KEYS"); /// generate keys by table name prefix - command_for_keys << table_id.getTableName() + ":" + toString(configuration.storage_type) + ":*"; + command_for_keys << table_id.getTableName() + ":" + storageTypeToKeyType(configuration.storage_type) + ":*"; auto all_keys = connection->client->execute(command_for_keys); - if (all_keys.size() == 0) + if (all_keys.isNull() || all_keys.size() == 0) return {}; Pipes pipes; diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h index 1bffc6a64bf..2c6c6193982 100644 --- a/src/Storages/StorageRedis.h +++ b/src/Storages/StorageRedis.h @@ -11,8 +11,8 @@ namespace DB * Read only. * * Note If storage_type is - * simple: there should be 2 columns and the first one is key in Redis, the second one is value. - * hash_map: there should be 3 columns and the first one is key in Redis and the second is the field of Redis Map. + * SIMPLE: there should be 2 columns and the first one is key in Redis, the second one is value. + * HASH_MAP: there should be 3 columns and the first one is key in Redis and the second is the field of Redis Map. */ class StorageRedis : public IStorage { From ce203b5ce6cb2329a7a26bcb4999e040ecbdbf7d Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 23 May 2023 20:54:26 +0800 Subject: [PATCH 0973/2223] Check redis table structure --- src/Common/ErrorCodes.cpp | 1 + src/Dictionaries/RedisSource.cpp | 5 +---- src/Dictionaries/RedisSource.h | 1 - src/Storages/RedisCommon.cpp | 13 +++++++++++++ src/Storages/RedisCommon.h | 7 +++++-- src/Storages/StorageRedis.cpp | 20 ++++++++++++++++---- src/TableFunctions/TableFunctionRedis.cpp | 6 ++++-- 7 files changed, 40 insertions(+), 13 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 505cf0aac8f..4c08d762df2 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -581,6 +581,7 @@ M(696, ASYNC_LOAD_CANCELED) \ M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \ M(698, INVALID_REDIS_STORAGE_TYPE) \ + M(699, INVALID_REDIS_TABLE_STRUCTURE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Dictionaries/RedisSource.cpp b/src/Dictionaries/RedisSource.cpp index 20e0838886c..27125077c10 100644 --- a/src/Dictionaries/RedisSource.cpp +++ b/src/Dictionaries/RedisSource.cpp @@ -1,15 +1,12 @@ #include "RedisSource.h" -#include #include - #include #include #include #include #include - -#include "DictionaryStructure.h" +#include namespace DB diff --git a/src/Dictionaries/RedisSource.h b/src/Dictionaries/RedisSource.h index fe5a973d57c..e8e78db67bc 100644 --- a/src/Dictionaries/RedisSource.h +++ b/src/Dictionaries/RedisSource.h @@ -7,7 +7,6 @@ #include #include #include -#include "RedisDictionarySource.h" namespace DB diff --git a/src/Storages/RedisCommon.cpp b/src/Storages/RedisCommon.cpp index 0a13e40b1ec..916ac3b69bc 100644 --- a/src/Storages/RedisCommon.cpp +++ b/src/Storages/RedisCommon.cpp @@ -7,6 +7,7 @@ namespace DB namespace ErrorCodes { extern const int INVALID_REDIS_STORAGE_TYPE; + extern const int INVALID_REDIS_TABLE_STRUCTURE; extern const int INTERNAL_REDIS_ERROR; extern const int TIMEOUT_EXCEEDED; } @@ -153,4 +154,16 @@ RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & } } +void checkRedisTableStructure(const ColumnsDescription & columns, const RedisConfiguration & configuration) +{ + /// TODO check data type + if (configuration.storage_type == RedisStorageType::HASH_MAP && columns.size() != 3) + throw Exception(ErrorCodes::INVALID_REDIS_TABLE_STRUCTURE, + "Redis hash table must have 3 columns, but found {}", columns.size()); + + if (configuration.storage_type == RedisStorageType::SIMPLE && columns.size() != 2) + throw Exception(ErrorCodes::INVALID_REDIS_TABLE_STRUCTURE, + "Redis string table must have 2 columns, but found {}", columns.size()); +} + } diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index 384a02d76e4..e663faa5fab 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -50,8 +51,6 @@ struct RedisConfiguration uint32_t db_index; String password; RedisStorageType storage_type; - /// column name of redis key - String key;// TODO remove uint32_t pool_size; }; @@ -85,4 +84,8 @@ RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisAr /// SIMPLE: all_columns must have 2 iterm and the first one is Redis key the second one is value /// HASH_MAP: all_columns must have 2 iterm and the first one is Redis key the second is field, the third is value. RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column); + +/// checking Redis table/table-function when creating +void checkRedisTableStructure(const ColumnsDescription & columns, const RedisConfiguration & configuration); + } diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 0e1b3a24e4f..7721665e9dd 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -78,7 +78,8 @@ Pipe StorageRedis::read( { RedisCommand command_for_keys("KEYS"); /// generate keys by table name prefix - command_for_keys << table_id.getTableName() + ":" + storageTypeToKeyType(configuration.storage_type) + ":*"; +// command_for_keys << table_id.getTableName() + ":" + storageTypeToKeyType(configuration.storage_type) + ":*"; + command_for_keys << "*"; auto all_keys = connection->client->execute(command_for_keys); @@ -90,6 +91,7 @@ Pipe StorageRedis::read( size_t num_keys = all_keys.size(); size_t num_threads = std::min(num_streams, all_keys.size()); + num_threads = std::min(num_threads, configuration.pool_size); assert(num_keys <= std::numeric_limits::max()); for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) @@ -106,9 +108,12 @@ Pipe StorageRedis::read( keys = *getRedisHashMapKeys(connection, keys); } + delete connection.release(); + /// TODO reduce keys copy pipes.emplace_back(std::make_shared( - std::move(connection), keys, configuration.storage_type, sample_block, redis_types, max_block_size)); + getRedisConnection(pool, configuration), keys, + configuration.storage_type, sample_block, redis_types, max_block_size)); } return Pipe::unitePipes(std::move(pipes)); } @@ -122,6 +127,7 @@ Pipe StorageRedis::read( size_t num_keys = fields->size(); size_t num_threads = std::min(num_streams, fields->size()); + num_threads = std::min(num_threads, configuration.pool_size); assert(num_keys <= std::numeric_limits::max()); for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) @@ -138,8 +144,11 @@ Pipe StorageRedis::read( keys = *getRedisHashMapKeys(connection, keys); } + delete connection.release(); + pipes.emplace_back(std::make_shared( - std::move(connection), keys, configuration.storage_type, sample_block, redis_types, max_block_size)); + getRedisConnection(pool, configuration), keys, + configuration.storage_type, sample_block, redis_types, max_block_size)); } return Pipe::unitePipes(std::move(pipes)); } @@ -151,9 +160,10 @@ SinkToStoragePtr StorageRedis::write( const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is unsupported for StorageRedis"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is unsupported for StorageRedis"); } +/// TODO make "password", "db_index", "storage_type", "pool_size" optional RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr context) { RedisConfiguration configuration; @@ -203,6 +213,8 @@ void registerStorageRedis(StorageFactory & factory) { auto configuration = StorageRedis::getConfiguration(args.engine_args, args.getLocalContext()); + checkRedisTableStructure(args.columns, configuration); + return std::make_shared( args.table_id, configuration, diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp index e410ad799a6..9e4a39b1b85 100644 --- a/src/TableFunctions/TableFunctionRedis.cpp +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -1,7 +1,6 @@ #include #include -#include #include @@ -29,8 +28,10 @@ StoragePtr TableFunctionRedis::executeImpl( const ASTPtr & /*ast_function*/, ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/) const { auto columns = getActualTableStructure(context); + checkRedisTableStructure(columns, *configuration); + auto storage = std::make_shared( - StorageID(toString(configuration->db_index), table_name), // TODO + StorageID(toString(configuration->db_index), table_name), *configuration, columns, ConstraintsDescription(), @@ -39,6 +40,7 @@ StoragePtr TableFunctionRedis::executeImpl( return storage; } +/// TODO support user customized table structure ColumnsDescription TableFunctionRedis::getActualTableStructure(ContextPtr context) const { /// generate table structure by storage type. From 40cc8d210792cd7a3e7f80f9ed9bf95fe938e9d8 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 May 2023 10:34:37 +0800 Subject: [PATCH 0974/2223] fix code style --- src/Dictionaries/RedisDictionarySource.cpp | 2 -- src/Storages/RedisCommon.cpp | 1 - src/Storages/RedisCommon.h | 5 ----- src/Storages/StorageRedis.cpp | 1 - 4 files changed, 9 deletions(-) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index f96c9231827..1056383bc84 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -17,9 +17,7 @@ namespace DB { extern const int UNSUPPORTED_METHOD; extern const int INVALID_CONFIG_PARAMETER; - extern const int INTERNAL_REDIS_ERROR; extern const int LOGICAL_ERROR; - extern const int TIMEOUT_EXCEEDED; } static RedisStorageType parseStorageType(const String & storage_type_str) diff --git a/src/Storages/RedisCommon.cpp b/src/Storages/RedisCommon.cpp index 916ac3b69bc..63a8d911bf0 100644 --- a/src/Storages/RedisCommon.cpp +++ b/src/Storages/RedisCommon.cpp @@ -6,7 +6,6 @@ namespace DB namespace ErrorCodes { - extern const int INVALID_REDIS_STORAGE_TYPE; extern const int INVALID_REDIS_TABLE_STRUCTURE; extern const int INTERNAL_REDIS_ERROR; extern const int TIMEOUT_EXCEEDED; diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index e663faa5fab..2668311125f 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -11,11 +11,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - static constexpr size_t REDIS_MAX_BLOCK_SIZE = DEFAULT_BLOCK_SIZE; static constexpr size_t REDIS_LOCK_ACQUIRE_TIMEOUT_MS = 5000; diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 7721665e9dd..3d7721bdc0e 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INVALID_REDIS_STORAGE_TYPE; extern const int NOT_IMPLEMENTED; } From 70cfd7a222342bfd1d9a3ddb59eaa2391d014d28 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 May 2023 10:45:21 +0800 Subject: [PATCH 0975/2223] fix typos --- src/Storages/RedisCommon.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index 2668311125f..348c2494632 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -76,8 +76,8 @@ RedisConnectionPtr getRedisConnection(RedisPoolPtr pool, const RedisConfiguratio RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisArray & keys); /// Get RedisColumnType of a column, If storage_type is -/// SIMPLE: all_columns must have 2 iterm and the first one is Redis key the second one is value -/// HASH_MAP: all_columns must have 2 iterm and the first one is Redis key the second is field, the third is value. +/// SIMPLE: all_columns must have 2 items and the first one is Redis key the second one is value +/// HASH_MAP: all_columns must have 2 items and the first one is Redis key the second is field, the third is value. RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column); /// checking Redis table/table-function when creating From d594bb1c7a3d6f0fe2ef4e27bb01e132ed82a8e7 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 May 2023 10:53:42 +0800 Subject: [PATCH 0976/2223] fix fast tests --- src/Access/Common/AccessType.h | 2 +- tests/queries/0_stateless/01271_show_privileges.reference | 2 +- .../0_stateless/02117_show_create_table_system.reference | 2 +- .../02414_all_new_table_functions_must_be_documented.reference | 1 + 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 78c341cdcb5..c9cce610f2c 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -201,7 +201,7 @@ enum class AccessType M(URL, "", GLOBAL, SOURCES) \ M(REMOTE, "", GLOBAL, SOURCES) \ M(MONGO, "", GLOBAL, SOURCES) \ - M(Redis, "", GLOBAL, SOURCES) \ + M(REDIS, "", GLOBAL, SOURCES) \ M(MEILISEARCH, "", GLOBAL, SOURCES) \ M(MYSQL, "", GLOBAL, SOURCES) \ M(POSTGRES, "", GLOBAL, SOURCES) \ diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index ec245d8b9e0..5ada21e31f4 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -148,7 +148,7 @@ INTROSPECTION ['INTROSPECTION FUNCTIONS'] \N ALL FILE [] GLOBAL SOURCES URL [] GLOBAL SOURCES REMOTE [] GLOBAL SOURCES -MONGO [] GLOBAL SOURCES +REDIS [] GLOBAL SOURCES MEILISEARCH [] GLOBAL SOURCES MYSQL [] GLOBAL SOURCES POSTGRES [] GLOBAL SOURCES diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 09cc62dac00..724118f7bc1 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -297,7 +297,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'Redis' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), diff --git a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference index 2277e19cf25..4f16e57d606 100644 --- a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference @@ -9,6 +9,7 @@ jdbc meilisearch merge mongodb +redis null numbers numbers_mt From b35867d907d39b790c9f8b3feb709cb9e76a6434 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 May 2023 18:06:42 +0800 Subject: [PATCH 0977/2223] unify storage type --- src/Dictionaries/RedisDictionarySource.cpp | 10 - src/Dictionaries/RedisDictionarySource.h | 10 - src/Storages/RedisCommon.cpp | 19 +- src/Storages/RedisCommon.h | 5 +- src/Storages/StorageRedis.cpp | 8 +- .../test_storage_redis/__init__.py | 0 .../configs/named_collections.xml | 12 + .../configs_secure/config.d/ssl_conf.xml | 8 + tests/integration/test_storage_redis/test.py | 426 ++++++++++++++++++ 9 files changed, 464 insertions(+), 34 deletions(-) create mode 100644 tests/integration/test_storage_redis/__init__.py create mode 100644 tests/integration/test_storage_redis/configs/named_collections.xml create mode 100644 tests/integration/test_storage_redis/configs_secure/config.d/ssl_conf.xml create mode 100644 tests/integration/test_storage_redis/test.py diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 1056383bc84..d28b7528d23 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -20,16 +20,6 @@ namespace DB extern const int LOGICAL_ERROR; } - static RedisStorageType parseStorageType(const String & storage_type_str) - { - if (storage_type_str == "hash_map") - return RedisStorageType::HASH_MAP; - else if (!storage_type_str.empty() && storage_type_str != "simple") - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Unknown storage type {} for Redis dictionary", storage_type_str); - - return RedisStorageType::SIMPLE; - } - void registerDictionarySourceRedis(DictionarySourceFactory & factory) { auto create_table_source = [=](const DictionaryStructure & dict_struct, diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index c7786284dc4..a55f220321d 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -7,16 +7,6 @@ #include "IDictionarySource.h" #include -namespace Poco -{ - namespace Redis - { - class Client; - class Array; - class Command; - } -} - namespace DB { namespace ErrorCodes diff --git a/src/Storages/RedisCommon.cpp b/src/Storages/RedisCommon.cpp index 63a8d911bf0..8cc94c45dae 100644 --- a/src/Storages/RedisCommon.cpp +++ b/src/Storages/RedisCommon.cpp @@ -9,6 +9,7 @@ namespace ErrorCodes extern const int INVALID_REDIS_TABLE_STRUCTURE; extern const int INTERNAL_REDIS_ERROR; extern const int TIMEOUT_EXCEEDED; + extern const int INVALID_REDIS_STORAGE_TYPE; } RedisColumnTypes REDIS_HASH_MAP_COLUMN_TYPES = {RedisColumnType::KEY, RedisColumnType::FIELD, RedisColumnType::VALUE}; @@ -24,27 +25,27 @@ RedisConnection::~RedisConnection() pool->returnObject(std::move(client)); } -String storageTypeToKeyType(RedisStorageType storage_type) +String serializeStorageType(RedisStorageType storage_type) { switch (storage_type) { case RedisStorageType::SIMPLE: - return "string"; + return "simple"; case RedisStorageType::HASH_MAP: - return "hash"; + return "hash_map"; default: return "none"; } } -RedisStorageType keyTypeToStorageType(const String & key_type) +RedisStorageType parseStorageType(const String & storage_type_str) { - if (key_type == "string") - return RedisStorageType::SIMPLE; - else if (key_type == "hash") + if (storage_type_str == "hash_map") return RedisStorageType::HASH_MAP; - else - return RedisStorageType::UNKNOWN; + else if (!storage_type_str.empty() && storage_type_str != "simple") + throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "Unknown storage type {} for Redis dictionary", storage_type_str); + + return RedisStorageType::SIMPLE; } RedisConnectionPtr getRedisConnection(RedisPoolPtr pool, const RedisConfiguration & configuration) diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index 348c2494632..02d0b435b9d 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -36,8 +36,11 @@ using RedisColumnTypes = std::vector; extern RedisColumnTypes REDIS_HASH_MAP_COLUMN_TYPES; extern RedisColumnTypes REDIS_SIMPLE_COLUMN_TYPES; +/// storage type to Redis key type String storageTypeToKeyType(RedisStorageType storage_type); -RedisStorageType keyTypeToStorageType(const String & key_type); + +RedisStorageType parseStorageType(const String & storage_type_str); +String serializeStorageType(RedisStorageType storage_type); struct RedisConfiguration { diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 3d7721bdc0e..819ab01d733 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -77,7 +77,7 @@ Pipe StorageRedis::read( { RedisCommand command_for_keys("KEYS"); /// generate keys by table name prefix -// command_for_keys << table_id.getTableName() + ":" + storageTypeToKeyType(configuration.storage_type) + ":*"; +// command_for_keys << table_id.getTableName() + ":" + serializeStorageType(configuration.storage_type) + ":*"; command_for_keys << "*"; auto all_keys = connection->client->execute(command_for_keys); @@ -178,7 +178,7 @@ RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr c configuration.port = static_cast(named_collection->get("port")); configuration.password = named_collection->get("password"); configuration.db_index = static_cast(named_collection->get({"db_index"})); - configuration.storage_type = keyTypeToStorageType(named_collection->getOrDefault("storage_type", "")); + configuration.storage_type = parseStorageType(named_collection->getOrDefault("storage_type", "")); configuration.pool_size = static_cast(named_collection->get("pool_size")); } else @@ -193,7 +193,7 @@ RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr c configuration.port = parsed_host_port.second; configuration.db_index = static_cast(checkAndGetLiteralArgument(engine_args[1], "db_index")); configuration.password = checkAndGetLiteralArgument(engine_args[2], "password"); - configuration.storage_type = keyTypeToStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); + configuration.storage_type = parseStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); configuration.pool_size = static_cast(checkAndGetLiteralArgument(engine_args[4], "pool_size")); } @@ -222,7 +222,7 @@ void registerStorageRedis(StorageFactory & factory) args.comment); }, { - .source_access_type = AccessType::Redis, + .source_access_type = AccessType::REDIS, }); } diff --git a/tests/integration/test_storage_redis/__init__.py b/tests/integration/test_storage_redis/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_redis/configs/named_collections.xml b/tests/integration/test_storage_redis/configs/named_collections.xml new file mode 100644 index 00000000000..5f7db390982 --- /dev/null +++ b/tests/integration/test_storage_redis/configs/named_collections.xml @@ -0,0 +1,12 @@ + + + + root + clickhouse + mongo1 + 27017 + test + simple_table + + + diff --git a/tests/integration/test_storage_redis/configs_secure/config.d/ssl_conf.xml b/tests/integration/test_storage_redis/configs_secure/config.d/ssl_conf.xml new file mode 100644 index 00000000000..3efe98e7045 --- /dev/null +++ b/tests/integration/test_storage_redis/configs_secure/config.d/ssl_conf.xml @@ -0,0 +1,8 @@ + + + + + none + + + diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py new file mode 100644 index 00000000000..6ba5520704d --- /dev/null +++ b/tests/integration/test_storage_redis/test.py @@ -0,0 +1,426 @@ +import pymongo + +import pytest +from helpers.client import QueryRuntimeException + +from helpers.cluster import ClickHouseCluster +import datetime + + +@pytest.fixture(scope="module") +def started_cluster(request): + try: + cluster = ClickHouseCluster(__file__) + node = cluster.add_instance( + "node", + main_configs=[ + "configs_secure/config.d/ssl_conf.xml", + "configs/named_collections.xml", + ], + with_mongo=True, + with_mongo_secure=request.param, + ) + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def get_mongo_connection(started_cluster, secure=False, with_credentials=True): + connection_str = "" + if with_credentials: + connection_str = "mongodb://root:clickhouse@localhost:{}".format( + started_cluster.mongo_port + ) + else: + connection_str = "mongodb://localhost:{}".format( + started_cluster.mongo_no_cred_port + ) + if secure: + connection_str += "/?tls=true&tlsAllowInvalidCertificates=true" + return pymongo.MongoClient(connection_str) + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_simple_select(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + node.query( + "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" + ) + + assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" + assert ( + node.query("SELECT sum(key) FROM simple_mongo_table") + == str(sum(range(0, 100))) + "\n" + ) + + assert ( + node.query("SELECT data from simple_mongo_table where key = 42") + == hex(42 * 42) + "\n" + ) + node.query("DROP TABLE simple_mongo_table") + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_arrays(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + arrays_mongo_table = db["arrays_table"] + data = [] + for i in range(0, 100): + data.append( + { + "key": i, + "arr_int64": [-(i + 1), -(i + 2), -(i + 3)], + "arr_int32": [-(i + 1), -(i + 2), -(i + 3)], + "arr_int16": [-(i + 1), -(i + 2), -(i + 3)], + "arr_int8": [-(i + 1), -(i + 2), -(i + 3)], + "arr_uint64": [i + 1, i + 2, i + 3], + "arr_uint32": [i + 1, i + 2, i + 3], + "arr_uint16": [i + 1, i + 2, i + 3], + "arr_uint8": [i + 1, i + 2, i + 3], + "arr_float32": [i + 1.125, i + 2.5, i + 3.750], + "arr_float64": [i + 1.125, i + 2.5, i + 3.750], + "arr_date": [ + datetime.datetime(2002, 10, 27), + datetime.datetime(2024, 1, 8), + ], + "arr_datetime": [ + datetime.datetime(2023, 3, 31, 6, 3, 12), + datetime.datetime(1999, 2, 28, 12, 46, 34), + ], + "arr_string": [str(i + 1), str(i + 2), str(i + 3)], + "arr_uuid": [ + "f0e77736-91d1-48ce-8f01-15123ca1c7ed", + "93376a07-c044-4281-a76e-ad27cf6973c5", + ], + "arr_arr_bool": [ + [True, False, True], + [True], + [], + None, + [False], + [None], + ], + "arr_empty": [], + "arr_null": None, + "arr_nullable": None, + } + ) + + arrays_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + node.query( + "CREATE TABLE arrays_mongo_table(" + "key UInt64," + "arr_int64 Array(Int64)," + "arr_int32 Array(Int32)," + "arr_int16 Array(Int16)," + "arr_int8 Array(Int8)," + "arr_uint64 Array(UInt64)," + "arr_uint32 Array(UInt32)," + "arr_uint16 Array(UInt16)," + "arr_uint8 Array(UInt8)," + "arr_float32 Array(Float32)," + "arr_float64 Array(Float64)," + "arr_date Array(Date)," + "arr_datetime Array(DateTime)," + "arr_string Array(String)," + "arr_uuid Array(UUID)," + "arr_arr_bool Array(Array(Bool))," + "arr_empty Array(UInt64)," + "arr_null Array(UInt64)," + "arr_arr_null Array(Array(UInt64))," + "arr_nullable Array(Nullable(UInt64))" + ") ENGINE = MongoDB('mongo1:27017', 'test', 'arrays_table', 'root', 'clickhouse')" + ) + + assert node.query("SELECT COUNT() FROM arrays_mongo_table") == "100\n" + + for column_name in ["arr_int64", "arr_int32", "arr_int16", "arr_int8"]: + assert ( + node.query(f"SELECT {column_name} FROM arrays_mongo_table WHERE key = 42") + == "[-43,-44,-45]\n" + ) + + for column_name in ["arr_uint64", "arr_uint32", "arr_uint16", "arr_uint8"]: + assert ( + node.query(f"SELECT {column_name} FROM arrays_mongo_table WHERE key = 42") + == "[43,44,45]\n" + ) + + for column_name in ["arr_float32", "arr_float64"]: + assert ( + node.query(f"SELECT {column_name} FROM arrays_mongo_table WHERE key = 42") + == "[43.125,44.5,45.75]\n" + ) + + assert ( + node.query(f"SELECT arr_date FROM arrays_mongo_table WHERE key = 42") + == "['2002-10-27','2024-01-08']\n" + ) + + assert ( + node.query(f"SELECT arr_datetime FROM arrays_mongo_table WHERE key = 42") + == "['2023-03-31 06:03:12','1999-02-28 12:46:34']\n" + ) + + assert ( + node.query(f"SELECT arr_string FROM arrays_mongo_table WHERE key = 42") + == "['43','44','45']\n" + ) + + assert ( + node.query(f"SELECT arr_uuid FROM arrays_mongo_table WHERE key = 42") + == "['f0e77736-91d1-48ce-8f01-15123ca1c7ed','93376a07-c044-4281-a76e-ad27cf6973c5']\n" + ) + + assert ( + node.query(f"SELECT arr_arr_bool FROM arrays_mongo_table WHERE key = 42") + == "[[true,false,true],[true],[],[],[false],[false]]\n" + ) + + assert ( + node.query(f"SELECT arr_empty FROM arrays_mongo_table WHERE key = 42") == "[]\n" + ) + + assert ( + node.query(f"SELECT arr_null FROM arrays_mongo_table WHERE key = 42") == "[]\n" + ) + + assert ( + node.query(f"SELECT arr_arr_null FROM arrays_mongo_table WHERE key = 42") + == "[]\n" + ) + + assert ( + node.query(f"SELECT arr_nullable FROM arrays_mongo_table WHERE key = 42") + == "[]\n" + ) + + node.query("DROP TABLE arrays_mongo_table") + arrays_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_complex_data_type(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + incomplete_mongo_table = db["complex_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i), "dict": {"a": i, "b": str(i)}}) + incomplete_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + node.query( + "CREATE TABLE incomplete_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse')" + ) + + assert node.query("SELECT COUNT() FROM incomplete_mongo_table") == "100\n" + assert ( + node.query("SELECT sum(key) FROM incomplete_mongo_table") + == str(sum(range(0, 100))) + "\n" + ) + + assert ( + node.query("SELECT data from incomplete_mongo_table where key = 42") + == hex(42 * 42) + "\n" + ) + node.query("DROP TABLE incomplete_mongo_table") + incomplete_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_incorrect_data_type(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + strange_mongo_table = db["strange_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i), "aaaa": "Hello"}) + strange_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + node.query( + "CREATE TABLE strange_mongo_table(key String, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse')" + ) + + with pytest.raises(QueryRuntimeException): + node.query("SELECT COUNT() FROM strange_mongo_table") + + with pytest.raises(QueryRuntimeException): + node.query("SELECT uniq(key) FROM strange_mongo_table") + + node.query( + "CREATE TABLE strange_mongo_table2(key UInt64, data String, bbbb String) ENGINE = MongoDB('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse')" + ) + + node.query("DROP TABLE strange_mongo_table") + node.query("DROP TABLE strange_mongo_table2") + strange_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [True], indirect=["started_cluster"]) +def test_secure_connection(started_cluster): + mongo_connection = get_mongo_connection(started_cluster, secure=True) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + node.query( + "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'ssl=true')" + ) + + assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" + assert ( + node.query("SELECT sum(key) FROM simple_mongo_table") + == str(sum(range(0, 100))) + "\n" + ) + + assert ( + node.query("SELECT data from simple_mongo_table where key = 42") + == hex(42 * 42) + "\n" + ) + node.query("DROP TABLE simple_mongo_table") + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_predefined_connection_configuration(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + node.query("drop table if exists simple_mongo_table") + node.query( + "create table simple_mongo_table(key UInt64, data String) engine = MongoDB(mongo1)" + ) + assert node.query("SELECT count() FROM simple_mongo_table") == "100\n" + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_no_credentials(started_cluster): + mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) + db = mongo_connection["test"] + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + node.query( + "create table simple_mongo_table_2(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', '', '')" + ) + assert node.query("SELECT count() FROM simple_mongo_table_2") == "100\n" + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_auth_source(started_cluster): + mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) + admin_db = mongo_connection["admin"] + admin_db.add_user( + "root", + "clickhouse", + roles=[{"role": "userAdminAnyDatabase", "db": "admin"}, "readWriteAnyDatabase"], + ) + simple_mongo_table = admin_db["simple_table"] + data = [] + for i in range(0, 50): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + db = mongo_connection["test"] + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + node.query( + "create table simple_mongo_table_fail(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse')" + ) + node.query_and_get_error("SELECT count() FROM simple_mongo_table_fail") + node.query( + "create table simple_mongo_table_ok(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', 'authSource=admin')" + ) + assert node.query("SELECT count() FROM simple_mongo_table_ok") == "100\n" + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_missing_columns(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 10): + data.append({"key": i, "data": hex(i * i)}) + for i in range(0, 10): + data.append({"key": i}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + node.query("drop table if exists simple_mongo_table") + node.query( + "create table simple_mongo_table(key UInt64, data Nullable(String)) engine = MongoDB(mongo1)" + ) + result = node.query("SELECT count() FROM simple_mongo_table WHERE isNull(data)") + assert result == "10\n" + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_simple_insert_select(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + + node = started_cluster.instances["node"] + node.query("DROP TABLE IF EXISTS simple_mongo_table") + node.query( + "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" + ) + node.query("INSERT INTO simple_mongo_table SELECT 1, 'kek'") + + assert ( + node.query("SELECT data from simple_mongo_table where key = 1").strip() == "kek" + ) + node.query("INSERT INTO simple_mongo_table(key) SELECT 12") + assert int(node.query("SELECT count() from simple_mongo_table")) == 2 + assert ( + node.query("SELECT data from simple_mongo_table where key = 12").strip() == "" + ) + + node.query("DROP TABLE simple_mongo_table") + simple_mongo_table.drop() From 412d9ba259cb91243d28765326d65384a60e18cc Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 25 May 2023 12:33:07 +0800 Subject: [PATCH 0978/2223] add tests for redis storage --- src/Storages/RedisCommon.cpp | 15 + src/Storages/RedisCommon.h | 2 +- .../configs/named_collections.xml | 12 - tests/integration/test_storage_redis/test.py | 443 ++---------------- .../test_table_function_redis/__init__.py | 0 .../configs_secure/config.d/ssl_conf.xml | 0 .../test_table_function_redis/test.py | 276 +++++++++++ 7 files changed, 329 insertions(+), 419 deletions(-) delete mode 100644 tests/integration/test_storage_redis/configs/named_collections.xml create mode 100644 tests/integration/test_table_function_redis/__init__.py rename tests/integration/{test_storage_redis => test_table_function_redis}/configs_secure/config.d/ssl_conf.xml (100%) create mode 100644 tests/integration/test_table_function_redis/test.py diff --git a/src/Storages/RedisCommon.cpp b/src/Storages/RedisCommon.cpp index 8cc94c45dae..fc789057019 100644 --- a/src/Storages/RedisCommon.cpp +++ b/src/Storages/RedisCommon.cpp @@ -25,6 +25,21 @@ RedisConnection::~RedisConnection() pool->returnObject(std::move(client)); } +String storageTypeToKeyType(RedisStorageType type) +{ + switch (type) + { + case RedisStorageType::SIMPLE: + return "string"; + case RedisStorageType::HASH_MAP: + return "hash"; + default: + return "none"; + } + + UNREACHABLE(); +} + String serializeStorageType(RedisStorageType storage_type) { switch (storage_type) diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index 02d0b435b9d..d68f2567248 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -37,7 +37,7 @@ extern RedisColumnTypes REDIS_HASH_MAP_COLUMN_TYPES; extern RedisColumnTypes REDIS_SIMPLE_COLUMN_TYPES; /// storage type to Redis key type -String storageTypeToKeyType(RedisStorageType storage_type); +String storageTypeToKeyType(RedisStorageType type); RedisStorageType parseStorageType(const String & storage_type_str); String serializeStorageType(RedisStorageType storage_type); diff --git a/tests/integration/test_storage_redis/configs/named_collections.xml b/tests/integration/test_storage_redis/configs/named_collections.xml deleted file mode 100644 index 5f7db390982..00000000000 --- a/tests/integration/test_storage_redis/configs/named_collections.xml +++ /dev/null @@ -1,12 +0,0 @@ - - - - root - clickhouse - mongo1 - 27017 - test - simple_table - - - diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py index 6ba5520704d..4220563c229 100644 --- a/tests/integration/test_storage_redis/test.py +++ b/tests/integration/test_storage_redis/test.py @@ -1,426 +1,57 @@ -import pymongo - +import redis import pytest -from helpers.client import QueryRuntimeException - from helpers.cluster import ClickHouseCluster -import datetime + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance("node", with_redis=True) @pytest.fixture(scope="module") -def started_cluster(request): +def started_cluster(): try: - cluster = ClickHouseCluster(__file__) - node = cluster.add_instance( - "node", - main_configs=[ - "configs_secure/config.d/ssl_conf.xml", - "configs/named_collections.xml", - ], - with_mongo=True, - with_mongo_secure=request.param, - ) cluster.start() yield cluster finally: cluster.shutdown() -def get_mongo_connection(started_cluster, secure=False, with_credentials=True): - connection_str = "" - if with_credentials: - connection_str = "mongodb://root:clickhouse@localhost:{}".format( - started_cluster.mongo_port - ) - else: - connection_str = "mongodb://localhost:{}".format( - started_cluster.mongo_no_cred_port - ) - if secure: - connection_str += "/?tls=true&tlsAllowInvalidCertificates=true" - return pymongo.MongoClient(connection_str) +def get_redis_connection(db_id=0): + client = redis.Redis( + host=node.name, port=started_cluster.redis_port, password="clickhouse", db=db_id + ) + return client -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_simple_select(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) +def get_address(): + return node.name + started_cluster.redis_port - node = started_cluster.instances["node"] + +@pytest.mark.parametrize("started_cluster") +def test_storage_simple_select(started_cluster): + client = get_redis_connection() + address = get_address() + + data = {} + for i in range(100): + data['key{}'.format(i)] = 'value{}'.format(i) + + client.mset(data) + + # create table node.query( - "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" + f""" + CREATE TABLE test_storage_simple_select( + k String, + v String + ) Engine=Redis('{address}', 0, '','simple', 10) + """ ) - assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" - assert ( - node.query("SELECT sum(key) FROM simple_mongo_table") - == str(sum(range(0, 100))) + "\n" - ) + select_query = "SELECT k, v from test_storage_simple_select where k='0' FORMAT Values" + assert (node.query(select_query) == "('0','0')") - assert ( - node.query("SELECT data from simple_mongo_table where key = 42") - == hex(42 * 42) + "\n" - ) - node.query("DROP TABLE simple_mongo_table") - simple_mongo_table.drop() + select_query = "SELECT * from test_storage_simple_select FORMAT Values" + assert (len(node.query(select_query)) == 100) + assert (node.query(select_query)[0] == "('0','0')") - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_arrays(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - arrays_mongo_table = db["arrays_table"] - data = [] - for i in range(0, 100): - data.append( - { - "key": i, - "arr_int64": [-(i + 1), -(i + 2), -(i + 3)], - "arr_int32": [-(i + 1), -(i + 2), -(i + 3)], - "arr_int16": [-(i + 1), -(i + 2), -(i + 3)], - "arr_int8": [-(i + 1), -(i + 2), -(i + 3)], - "arr_uint64": [i + 1, i + 2, i + 3], - "arr_uint32": [i + 1, i + 2, i + 3], - "arr_uint16": [i + 1, i + 2, i + 3], - "arr_uint8": [i + 1, i + 2, i + 3], - "arr_float32": [i + 1.125, i + 2.5, i + 3.750], - "arr_float64": [i + 1.125, i + 2.5, i + 3.750], - "arr_date": [ - datetime.datetime(2002, 10, 27), - datetime.datetime(2024, 1, 8), - ], - "arr_datetime": [ - datetime.datetime(2023, 3, 31, 6, 3, 12), - datetime.datetime(1999, 2, 28, 12, 46, 34), - ], - "arr_string": [str(i + 1), str(i + 2), str(i + 3)], - "arr_uuid": [ - "f0e77736-91d1-48ce-8f01-15123ca1c7ed", - "93376a07-c044-4281-a76e-ad27cf6973c5", - ], - "arr_arr_bool": [ - [True, False, True], - [True], - [], - None, - [False], - [None], - ], - "arr_empty": [], - "arr_null": None, - "arr_nullable": None, - } - ) - - arrays_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE arrays_mongo_table(" - "key UInt64," - "arr_int64 Array(Int64)," - "arr_int32 Array(Int32)," - "arr_int16 Array(Int16)," - "arr_int8 Array(Int8)," - "arr_uint64 Array(UInt64)," - "arr_uint32 Array(UInt32)," - "arr_uint16 Array(UInt16)," - "arr_uint8 Array(UInt8)," - "arr_float32 Array(Float32)," - "arr_float64 Array(Float64)," - "arr_date Array(Date)," - "arr_datetime Array(DateTime)," - "arr_string Array(String)," - "arr_uuid Array(UUID)," - "arr_arr_bool Array(Array(Bool))," - "arr_empty Array(UInt64)," - "arr_null Array(UInt64)," - "arr_arr_null Array(Array(UInt64))," - "arr_nullable Array(Nullable(UInt64))" - ") ENGINE = MongoDB('mongo1:27017', 'test', 'arrays_table', 'root', 'clickhouse')" - ) - - assert node.query("SELECT COUNT() FROM arrays_mongo_table") == "100\n" - - for column_name in ["arr_int64", "arr_int32", "arr_int16", "arr_int8"]: - assert ( - node.query(f"SELECT {column_name} FROM arrays_mongo_table WHERE key = 42") - == "[-43,-44,-45]\n" - ) - - for column_name in ["arr_uint64", "arr_uint32", "arr_uint16", "arr_uint8"]: - assert ( - node.query(f"SELECT {column_name} FROM arrays_mongo_table WHERE key = 42") - == "[43,44,45]\n" - ) - - for column_name in ["arr_float32", "arr_float64"]: - assert ( - node.query(f"SELECT {column_name} FROM arrays_mongo_table WHERE key = 42") - == "[43.125,44.5,45.75]\n" - ) - - assert ( - node.query(f"SELECT arr_date FROM arrays_mongo_table WHERE key = 42") - == "['2002-10-27','2024-01-08']\n" - ) - - assert ( - node.query(f"SELECT arr_datetime FROM arrays_mongo_table WHERE key = 42") - == "['2023-03-31 06:03:12','1999-02-28 12:46:34']\n" - ) - - assert ( - node.query(f"SELECT arr_string FROM arrays_mongo_table WHERE key = 42") - == "['43','44','45']\n" - ) - - assert ( - node.query(f"SELECT arr_uuid FROM arrays_mongo_table WHERE key = 42") - == "['f0e77736-91d1-48ce-8f01-15123ca1c7ed','93376a07-c044-4281-a76e-ad27cf6973c5']\n" - ) - - assert ( - node.query(f"SELECT arr_arr_bool FROM arrays_mongo_table WHERE key = 42") - == "[[true,false,true],[true],[],[],[false],[false]]\n" - ) - - assert ( - node.query(f"SELECT arr_empty FROM arrays_mongo_table WHERE key = 42") == "[]\n" - ) - - assert ( - node.query(f"SELECT arr_null FROM arrays_mongo_table WHERE key = 42") == "[]\n" - ) - - assert ( - node.query(f"SELECT arr_arr_null FROM arrays_mongo_table WHERE key = 42") - == "[]\n" - ) - - assert ( - node.query(f"SELECT arr_nullable FROM arrays_mongo_table WHERE key = 42") - == "[]\n" - ) - - node.query("DROP TABLE arrays_mongo_table") - arrays_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_complex_data_type(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - incomplete_mongo_table = db["complex_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i), "dict": {"a": i, "b": str(i)}}) - incomplete_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE incomplete_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse')" - ) - - assert node.query("SELECT COUNT() FROM incomplete_mongo_table") == "100\n" - assert ( - node.query("SELECT sum(key) FROM incomplete_mongo_table") - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query("SELECT data from incomplete_mongo_table where key = 42") - == hex(42 * 42) + "\n" - ) - node.query("DROP TABLE incomplete_mongo_table") - incomplete_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_incorrect_data_type(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - strange_mongo_table = db["strange_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i), "aaaa": "Hello"}) - strange_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE strange_mongo_table(key String, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse')" - ) - - with pytest.raises(QueryRuntimeException): - node.query("SELECT COUNT() FROM strange_mongo_table") - - with pytest.raises(QueryRuntimeException): - node.query("SELECT uniq(key) FROM strange_mongo_table") - - node.query( - "CREATE TABLE strange_mongo_table2(key UInt64, data String, bbbb String) ENGINE = MongoDB('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse')" - ) - - node.query("DROP TABLE strange_mongo_table") - node.query("DROP TABLE strange_mongo_table2") - strange_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [True], indirect=["started_cluster"]) -def test_secure_connection(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, secure=True) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'ssl=true')" - ) - - assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" - assert ( - node.query("SELECT sum(key) FROM simple_mongo_table") - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query("SELECT data from simple_mongo_table where key = 42") - == hex(42 * 42) + "\n" - ) - node.query("DROP TABLE simple_mongo_table") - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_predefined_connection_configuration(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query("drop table if exists simple_mongo_table") - node.query( - "create table simple_mongo_table(key UInt64, data String) engine = MongoDB(mongo1)" - ) - assert node.query("SELECT count() FROM simple_mongo_table") == "100\n" - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_no_credentials(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) - db = mongo_connection["test"] - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "create table simple_mongo_table_2(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', '', '')" - ) - assert node.query("SELECT count() FROM simple_mongo_table_2") == "100\n" - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_auth_source(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) - admin_db = mongo_connection["admin"] - admin_db.add_user( - "root", - "clickhouse", - roles=[{"role": "userAdminAnyDatabase", "db": "admin"}, "readWriteAnyDatabase"], - ) - simple_mongo_table = admin_db["simple_table"] - data = [] - for i in range(0, 50): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - db = mongo_connection["test"] - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "create table simple_mongo_table_fail(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse')" - ) - node.query_and_get_error("SELECT count() FROM simple_mongo_table_fail") - node.query( - "create table simple_mongo_table_ok(key UInt64, data String) engine = MongoDB('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', 'authSource=admin')" - ) - assert node.query("SELECT count() FROM simple_mongo_table_ok") == "100\n" - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_missing_columns(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 10): - data.append({"key": i, "data": hex(i * i)}) - for i in range(0, 10): - data.append({"key": i}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query("drop table if exists simple_mongo_table") - node.query( - "create table simple_mongo_table(key UInt64, data Nullable(String)) engine = MongoDB(mongo1)" - ) - result = node.query("SELECT count() FROM simple_mongo_table WHERE isNull(data)") - assert result == "10\n" - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_simple_insert_select(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - - node = started_cluster.instances["node"] - node.query("DROP TABLE IF EXISTS simple_mongo_table") - node.query( - "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" - ) - node.query("INSERT INTO simple_mongo_table SELECT 1, 'kek'") - - assert ( - node.query("SELECT data from simple_mongo_table where key = 1").strip() == "kek" - ) - node.query("INSERT INTO simple_mongo_table(key) SELECT 12") - assert int(node.query("SELECT count() from simple_mongo_table")) == 2 - assert ( - node.query("SELECT data from simple_mongo_table where key = 12").strip() == "" - ) - - node.query("DROP TABLE simple_mongo_table") - simple_mongo_table.drop() diff --git a/tests/integration/test_table_function_redis/__init__.py b/tests/integration/test_table_function_redis/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_redis/configs_secure/config.d/ssl_conf.xml b/tests/integration/test_table_function_redis/configs_secure/config.d/ssl_conf.xml similarity index 100% rename from tests/integration/test_storage_redis/configs_secure/config.d/ssl_conf.xml rename to tests/integration/test_table_function_redis/configs_secure/config.d/ssl_conf.xml diff --git a/tests/integration/test_table_function_redis/test.py b/tests/integration/test_table_function_redis/test.py new file mode 100644 index 00000000000..e0ad71b0079 --- /dev/null +++ b/tests/integration/test_table_function_redis/test.py @@ -0,0 +1,276 @@ +import pymongo + +import pytest +from helpers.client import QueryRuntimeException + +from helpers.cluster import ClickHouseCluster + + +@pytest.fixture(scope="module") +def started_cluster(request): + try: + cluster = ClickHouseCluster(__file__) + node = cluster.add_instance( + "node", + with_mongo=True, + main_configs=[ + "configs_secure/config.d/ssl_conf.xml", + ], + with_mongo_secure=request.param, + ) + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def get_mongo_connection(started_cluster, secure=False, with_credentials=True): + connection_str = "" + if with_credentials: + connection_str = "mongodb://root:clickhouse@localhost:{}".format( + started_cluster.mongo_port + ) + else: + connection_str = "mongodb://localhost:{}".format( + started_cluster.mongo_no_cred_port + ) + if secure: + connection_str += "/?tls=true&tlsAllowInvalidCertificates=true" + return pymongo.MongoClient(connection_str) + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_simple_select(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + + node = started_cluster.instances["node"] + for i in range(0, 100): + node.query( + "INSERT INTO FUNCTION mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String') (key, data) VALUES ({}, '{}')".format( + i, hex(i * i) + ) + ) + assert ( + node.query( + "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" + ) + == "100\n" + ) + assert ( + node.query( + "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" + ) + == str(sum(range(0, 100))) + "\n" + ) + assert ( + node.query( + "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'key UInt64, data String')" + ) + == str(sum(range(0, 100))) + "\n" + ) + + assert ( + node.query( + "SELECT data from mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String') where key = 42" + ) + == hex(42 * 42) + "\n" + ) + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_complex_data_type(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + incomplete_mongo_table = db["complex_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i), "dict": {"a": i, "b": str(i)}}) + incomplete_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + + assert ( + node.query( + "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)')" + ) + == "100\n" + ) + assert ( + node.query( + "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)')" + ) + == str(sum(range(0, 100))) + "\n" + ) + + assert ( + node.query( + "SELECT data from mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)') where key = 42" + ) + == hex(42 * 42) + "\n" + ) + incomplete_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_incorrect_data_type(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + strange_mongo_table = db["strange_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i), "aaaa": "Hello"}) + strange_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + + with pytest.raises(QueryRuntimeException): + node.query( + "SELECT aaaa FROM mongodb('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse', structure='key UInt64, data String')" + ) + + strange_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [True], indirect=["started_cluster"]) +def test_secure_connection(started_cluster): + mongo_connection = get_mongo_connection(started_cluster, secure=True) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + + assert ( + node.query( + "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true')" + ) + == "100\n" + ) + assert ( + node.query( + "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true')" + ) + == str(sum(range(0, 100))) + "\n" + ) + assert ( + node.query( + "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'key UInt64, data String', 'ssl=true')" + ) + == str(sum(range(0, 100))) + "\n" + ) + + assert ( + node.query( + "SELECT data from mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true') where key = 42" + ) + == hex(42 * 42) + "\n" + ) + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_predefined_connection_configuration(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + assert ( + node.query( + "SELECT count() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" + ) + == "100\n" + ) + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_no_credentials(started_cluster): + mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) + db = mongo_connection["test"] + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + assert ( + node.query( + "SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', '', '', structure='key UInt64, data String')" + ) + == "100\n" + ) + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_auth_source(started_cluster): + mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) + admin_db = mongo_connection["admin"] + admin_db.add_user( + "root", + "clickhouse", + roles=[{"role": "userAdminAnyDatabase", "db": "admin"}, "readWriteAnyDatabase"], + ) + simple_mongo_table = admin_db["simple_table"] + data = [] + for i in range(0, 50): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + db = mongo_connection["test"] + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 100): + data.append({"key": i, "data": hex(i * i)}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + + node.query_and_get_error( + "SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" + ) + + assert ( + node.query( + "SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='authSource=admin')" + ) + == "100\n" + ) + simple_mongo_table.drop() + + +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) +def test_missing_columns(started_cluster): + mongo_connection = get_mongo_connection(started_cluster) + db = mongo_connection["test"] + db.add_user("root", "clickhouse") + simple_mongo_table = db["simple_table"] + data = [] + for i in range(0, 10): + data.append({"key": i, "data": hex(i * i)}) + for i in range(0, 10): + data.append({"key": i}) + simple_mongo_table.insert_many(data) + + node = started_cluster.instances["node"] + result = node.query( + "SELECT count() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data Nullable(String)') WHERE isNull(data)" + ) + assert result == "10\n" + simple_mongo_table.drop() From 3281aec3357777845f3899d159c2282f07715073 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 25 May 2023 17:12:56 +0800 Subject: [PATCH 0979/2223] make some redis engine args optional --- src/Storages/StorageRedis.cpp | 30 +++++++++++++++++++++--------- src/Storages/StorageRedis.h | 2 +- 2 files changed, 22 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 819ab01d733..0cc0e566d5c 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -166,6 +166,10 @@ SinkToStoragePtr StorageRedis::write( RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr context) { RedisConfiguration configuration; + configuration.db_index = 0; + configuration.password = ""; + configuration.storage_type = RedisStorageType::SIMPLE; + configuration.pool_size = 10; if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) { @@ -175,11 +179,15 @@ RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr c {}); configuration.host = named_collection->getAny({"host", "hostname"}); - configuration.port = static_cast(named_collection->get("port")); - configuration.password = named_collection->get("password"); - configuration.db_index = static_cast(named_collection->get({"db_index"})); - configuration.storage_type = parseStorageType(named_collection->getOrDefault("storage_type", "")); - configuration.pool_size = static_cast(named_collection->get("pool_size")); + configuration.port = static_cast(named_collection->getOrDefault("port", 6379)); + if (engine_args.size() > 1) + configuration.password = named_collection->get("password"); + if (engine_args.size() > 2) + configuration.db_index = static_cast(named_collection->get("db_index")); + if (engine_args.size() > 3) + configuration.storage_type = parseStorageType(named_collection->get("storage_type")); + if (engine_args.size() > 4) + configuration.pool_size = static_cast(named_collection->get("pool_size")); } else { @@ -191,10 +199,14 @@ RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr c configuration.host = parsed_host_port.first; configuration.port = parsed_host_port.second; - configuration.db_index = static_cast(checkAndGetLiteralArgument(engine_args[1], "db_index")); - configuration.password = checkAndGetLiteralArgument(engine_args[2], "password"); - configuration.storage_type = parseStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); - configuration.pool_size = static_cast(checkAndGetLiteralArgument(engine_args[4], "pool_size")); + if (engine_args.size() > 1) + configuration.db_index = static_cast(checkAndGetLiteralArgument(engine_args[1], "db_index")); + if (engine_args.size() > 2) + configuration.password = checkAndGetLiteralArgument(engine_args[2], "password"); + if (engine_args.size() > 3) + configuration.storage_type = parseStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); + if (engine_args.size() > 4) + configuration.pool_size = static_cast(checkAndGetLiteralArgument(engine_args[4], "pool_size")); } if (configuration.storage_type == RedisStorageType::UNKNOWN) diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h index 2c6c6193982..1ae90b2d1ba 100644 --- a/src/Storages/StorageRedis.h +++ b/src/Storages/StorageRedis.h @@ -7,7 +7,7 @@ namespace DB { /* Implements storage in the Redis. - * Use ENGINE = Redis(host:port, db_index, password, storage_type, conn_pool_size); + * Use ENGINE = Redis(host:port, db_index, password, storage_type, pool_size); * Read only. * * Note If storage_type is From 3c2b44747299b7b87d7e1e1c4219409ade1d7c34 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 25 May 2023 17:29:22 +0800 Subject: [PATCH 0980/2223] move get configuration to RedisCommon --- src/Storages/RedisCommon.cpp | 57 ++++++++++++++++++++++- src/Storages/RedisCommon.h | 3 ++ src/Storages/StorageRedis.cpp | 54 --------------------- src/Storages/StorageRedis.h | 2 - src/TableFunctions/TableFunctionRedis.cpp | 2 +- 5 files changed, 59 insertions(+), 59 deletions(-) diff --git a/src/Storages/RedisCommon.cpp b/src/Storages/RedisCommon.cpp index fc789057019..0d33b9c7aa3 100644 --- a/src/Storages/RedisCommon.cpp +++ b/src/Storages/RedisCommon.cpp @@ -151,13 +151,13 @@ RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisAr RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column) { - String redis_col_key = all_columns.at(0); + const String & redis_col_key = all_columns.at(0); if (column == redis_col_key) return RedisColumnType::KEY; if (storage_type == RedisStorageType::HASH_MAP) { - String redis_col_field = all_columns.at(1); + const String & redis_col_field = all_columns.at(1); if (column == redis_col_field) return RedisColumnType::FIELD; else @@ -169,6 +169,59 @@ RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & } } +RedisConfiguration getRedisConfiguration(const ASTs & engine_args, ContextPtr context) +{ + RedisConfiguration configuration; + configuration.db_index = 0; + configuration.password = ""; + configuration.storage_type = RedisStorageType::SIMPLE; + configuration.pool_size = 10; + + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) + { + validateNamedCollection( + *named_collection, + ValidateKeysMultiset{"host", "port", "hostname", "password", "db_index", "storage_type", "pool_size"}, + {}); + + configuration.host = named_collection->getAny({"host", "hostname"}); + configuration.port = static_cast(named_collection->getOrDefault("port", 6379)); + if (engine_args.size() > 1) + configuration.password = named_collection->get("password"); + if (engine_args.size() > 2) + configuration.db_index = static_cast(named_collection->get("db_index")); + if (engine_args.size() > 3) + configuration.storage_type = parseStorageType(named_collection->get("storage_type")); + if (engine_args.size() > 4) + configuration.pool_size = static_cast(named_collection->get("pool_size")); + } + else + { + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); + + /// 6379 is the default Redis port. + auto parsed_host_port = parseAddress(checkAndGetLiteralArgument(engine_args[0], "host:port"), 6379); + + configuration.host = parsed_host_port.first; + configuration.port = parsed_host_port.second; + if (engine_args.size() > 1) + configuration.db_index = static_cast(checkAndGetLiteralArgument(engine_args[1], "db_index")); + if (engine_args.size() > 2) + configuration.password = checkAndGetLiteralArgument(engine_args[2], "password"); + if (engine_args.size() > 3) + configuration.storage_type = parseStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); + if (engine_args.size() > 4) + configuration.pool_size = static_cast(checkAndGetLiteralArgument(engine_args[4], "pool_size")); + } + + if (configuration.storage_type == RedisStorageType::UNKNOWN) + throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "Invalid Redis storage type"); + + context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); + return configuration; +} + void checkRedisTableStructure(const ColumnsDescription & columns, const RedisConfiguration & configuration) { /// TODO check data type diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index d68f2567248..30e771d2471 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -83,6 +83,9 @@ RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisAr /// HASH_MAP: all_columns must have 2 items and the first one is Redis key the second is field, the third is value. RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column); +/// parse redis table engine/function configuration from engine_args +RedisConfiguration getRedisConfiguration(const ASTs & engine_args, ContextPtr context); + /// checking Redis table/table-function when creating void checkRedisTableStructure(const ColumnsDescription & columns, const RedisConfiguration & configuration); diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 0cc0e566d5c..5010aada8c4 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -162,60 +162,6 @@ SinkToStoragePtr StorageRedis::write( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is unsupported for StorageRedis"); } -/// TODO make "password", "db_index", "storage_type", "pool_size" optional -RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr context) -{ - RedisConfiguration configuration; - configuration.db_index = 0; - configuration.password = ""; - configuration.storage_type = RedisStorageType::SIMPLE; - configuration.pool_size = 10; - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) - { - validateNamedCollection( - *named_collection, - ValidateKeysMultiset{"host", "port", "hostname", "password", "db_index", "storage_type", "pool_size"}, - {}); - - configuration.host = named_collection->getAny({"host", "hostname"}); - configuration.port = static_cast(named_collection->getOrDefault("port", 6379)); - if (engine_args.size() > 1) - configuration.password = named_collection->get("password"); - if (engine_args.size() > 2) - configuration.db_index = static_cast(named_collection->get("db_index")); - if (engine_args.size() > 3) - configuration.storage_type = parseStorageType(named_collection->get("storage_type")); - if (engine_args.size() > 4) - configuration.pool_size = static_cast(named_collection->get("pool_size")); - } - else - { - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); - - /// 6379 is the default Redis port. - auto parsed_host_port = parseAddress(checkAndGetLiteralArgument(engine_args[0], "host:port"), 6379); - - configuration.host = parsed_host_port.first; - configuration.port = parsed_host_port.second; - if (engine_args.size() > 1) - configuration.db_index = static_cast(checkAndGetLiteralArgument(engine_args[1], "db_index")); - if (engine_args.size() > 2) - configuration.password = checkAndGetLiteralArgument(engine_args[2], "password"); - if (engine_args.size() > 3) - configuration.storage_type = parseStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); - if (engine_args.size() > 4) - configuration.pool_size = static_cast(checkAndGetLiteralArgument(engine_args[4], "pool_size")); - } - - if (configuration.storage_type == RedisStorageType::UNKNOWN) - throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "Invalid Redis storage type"); - - context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); - return configuration; -} - void registerStorageRedis(StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h index 1ae90b2d1ba..619a83f3851 100644 --- a/src/Storages/StorageRedis.h +++ b/src/Storages/StorageRedis.h @@ -40,8 +40,6 @@ public: const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; - static RedisConfiguration getConfiguration(ASTs engine_args, ContextPtr context); - private: StorageID table_id; RedisConfiguration configuration; diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp index 9e4a39b1b85..cd08837aae2 100644 --- a/src/TableFunctions/TableFunctionRedis.cpp +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -74,7 +74,7 @@ void TableFunctionRedis::parseArguments(const ASTPtr & ast_function, ContextPtr "Table function 'Redis' requires from 5 parameters: " "redis('host:port', db_index, 'password', 'storage_type', 'pool_size')"); } - configuration = StorageRedis::getConfiguration(args, context); + configuration = getRedisConfiguration(args, context); } From ff961834d674a8ce099da43811eb2eedf3b0d011 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 25 May 2023 17:29:49 +0800 Subject: [PATCH 0981/2223] add tests to redis engine --- tests/integration/test_storage_redis/test.py | 151 +++++++++++++++++-- 1 file changed, 138 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py index 4220563c229..af01c2e9ff1 100644 --- a/tests/integration/test_storage_redis/test.py +++ b/tests/integration/test_storage_redis/test.py @@ -1,6 +1,11 @@ +import time + import redis import pytest + +from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) @@ -18,40 +23,160 @@ def started_cluster(): def get_redis_connection(db_id=0): client = redis.Redis( - host=node.name, port=started_cluster.redis_port, password="clickhouse", db=db_id + host='localhost', port=cluster.redis_port, password="clickhouse", db=db_id ) return client -def get_address(): - return node.name + started_cluster.redis_port +def get_address_for_ch(): + return cluster.redis_host + ':6379' + + +def drop_table(table): + node.query(f"DROP TABLE IF EXISTS {table} SYNC"); -@pytest.mark.parametrize("started_cluster") def test_storage_simple_select(started_cluster): client = get_redis_connection() - address = get_address() + address = get_address_for_ch() + + # clean all + client.flushall() + drop_table('test_storage_simple_select') data = {} for i in range(100): - data['key{}'.format(i)] = 'value{}'.format(i) + data[str(i)] = str(i) client.mset(data) + client.close() # create table node.query( f""" CREATE TABLE test_storage_simple_select( k String, - v String - ) Engine=Redis('{address}', 0, '','simple', 10) + v UInt32 + ) Engine=Redis('{address}', 0, 'clickhouse') """ ) - select_query = "SELECT k, v from test_storage_simple_select where k='0' FORMAT Values" - assert (node.query(select_query) == "('0','0')") + response = TSV.toMat(node.query("SELECT k, v from test_storage_simple_select where k='0' FORMAT TSV")) + assert (len(response) == 1) + assert (response[0] == ['0', '0']) - select_query = "SELECT * from test_storage_simple_select FORMAT Values" - assert (len(node.query(select_query)) == 100) - assert (node.query(select_query)[0] == "('0','0')") + response = TSV.toMat(node.query("SELECT * from test_storage_simple_select order by k FORMAT TSV")) + assert (len(response) == 100) + assert (response[0] == ['0', '0']) + + +def test_storage_hash_map_select(started_cluster): + client = get_redis_connection() + address = get_address_for_ch() + + # clean all + client.flushall() + drop_table('test_storage_hash_map_select') + + key = 'k' + data = {} + for i in range(100): + data[str(i)] = str(i) + + client.hset(key, mapping=data) + client.close() + + # create table + node.query( + f""" + CREATE TABLE test_storage_hash_map_select( + k String, + f String, + v UInt32 + ) Engine=Redis('{address}', 0, 'clickhouse','hash_map') + """ + ) + + response = TSV.toMat(node.query("SELECT k, f, v from test_storage_hash_map_select where f='0' FORMAT TSV")) + assert (len(response) == 1) + assert (response[0] == ['k', '0', '0']) + + response = TSV.toMat(node.query("SELECT * from test_storage_hash_map_select FORMAT TSV")) + assert (len(response) == 100) + assert (response[0] == ['k', '0', '0']) + + +def test_create_table(started_cluster): + address = get_address_for_ch() + + # simple creation + drop_table('test_create_table') + node.query( + f""" + CREATE TABLE test_create_table( + k String, + v UInt32 + ) Engine=Redis('{address}') + """ + ) + + # simple creation with full engine args + drop_table('test_create_table') + node.query( + f""" + CREATE TABLE test_create_table( + k String, + v UInt32 + ) Engine=Redis('{address}', 0, 'clickhouse','simple', 10) + """ + ) + + drop_table('test_create_table') + node.query( + f""" + CREATE TABLE test_create_table( + k String, + f String, + v UInt32 + ) Engine=Redis('{address}', 0, 'clickhouse','hash_map', 10) + """ + ) + + # illegal columns + drop_table('test_create_table') + with pytest.raises(QueryRuntimeException): + node.query( + f""" + CREATE TABLE test_create_table( + k String, + f String, + v UInt32 + ) Engine=Redis('{address}', 0, 'clickhouse','simple', 10) + """ + ) + + drop_table('test_create_table') + with pytest.raises(QueryRuntimeException): + node.query( + f""" + CREATE TABLE test_create_table( + k String, + f String, + v UInt32, + n UInt32 + ) Engine=Redis('{address}', 0, 'clickhouse','hash_map', 10) + """ + ) + + # illegal storage type + drop_table('test_create_table') + with pytest.raises(QueryRuntimeException): + node.query( + f""" + CREATE TABLE test_create_table( + k String, + v UInt32 + ) Engine=Redis('{address}', 0, 'clickhouse','not_exist', 10) + """ + ) From 8c822a7edfb919844a94a46485dfeebbf9caad57 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 26 May 2023 10:34:37 +0800 Subject: [PATCH 0982/2223] add tests to redis engine --- src/Storages/RedisCommon.cpp | 6 +- src/Storages/RedisCommon.h | 2 +- src/Storages/StorageRedis.cpp | 2 +- src/TableFunctions/TableFunctionRedis.cpp | 40 +- src/TableFunctions/TableFunctionRedis.h | 1 + tests/integration/test_storage_redis/test.py | 9 +- .../configs_secure/config.d/ssl_conf.xml | 8 - .../test_table_function_redis/test.py | 385 ++++++------------ 8 files changed, 166 insertions(+), 287 deletions(-) delete mode 100644 tests/integration/test_table_function_redis/configs_secure/config.d/ssl_conf.xml diff --git a/src/Storages/RedisCommon.cpp b/src/Storages/RedisCommon.cpp index 0d33b9c7aa3..86312f49f41 100644 --- a/src/Storages/RedisCommon.cpp +++ b/src/Storages/RedisCommon.cpp @@ -1,5 +1,9 @@ #include "RedisCommon.h" #include +#include +#include +#include +#include namespace DB { @@ -169,7 +173,7 @@ RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & } } -RedisConfiguration getRedisConfiguration(const ASTs & engine_args, ContextPtr context) +RedisConfiguration getRedisConfiguration(ASTs & engine_args, ContextPtr context) { RedisConfiguration configuration; configuration.db_index = 0; diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index 30e771d2471..c378006f7a5 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -84,7 +84,7 @@ RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisAr RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column); /// parse redis table engine/function configuration from engine_args -RedisConfiguration getRedisConfiguration(const ASTs & engine_args, ContextPtr context); +RedisConfiguration getRedisConfiguration(ASTs & engine_args, ContextPtr context); /// checking Redis table/table-function when creating void checkRedisTableStructure(const ColumnsDescription & columns, const RedisConfiguration & configuration); diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 5010aada8c4..cd1cd06b4c4 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -168,7 +168,7 @@ void registerStorageRedis(StorageFactory & factory) "Redis", [](const StorageFactory::Arguments & args) { - auto configuration = StorageRedis::getConfiguration(args.engine_args, args.getLocalContext()); + auto configuration = getRedisConfiguration(args.engine_args, args.getLocalContext()); checkRedisTableStructure(args.columns, configuration); diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp index cd08837aae2..f90a30af8a1 100644 --- a/src/TableFunctions/TableFunctionRedis.cpp +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -43,19 +44,6 @@ StoragePtr TableFunctionRedis::executeImpl( /// TODO support user customized table structure ColumnsDescription TableFunctionRedis::getActualTableStructure(ContextPtr context) const { - /// generate table structure by storage type. - String structure; - switch (configuration->storage_type) - { - case RedisStorageType::SIMPLE: - structure = "key String, value String"; - break; - case RedisStorageType::HASH_MAP: - structure = "key String, field String, value String"; - break; - case RedisStorageType::UNKNOWN: - throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "Invalid Redis storage type."); - } return parseColumnsListFromString(structure, context); } @@ -66,15 +54,25 @@ void TableFunctionRedis::parseArguments(const ASTPtr & ast_function, ContextPtr throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'redis' must have arguments."); ASTs & args = func_args.arguments->children; - - if (args.size() != 5) - { - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Table function 'Redis' requires from 5 parameters: " - "redis('host:port', db_index, 'password', 'storage_type', 'pool_size')"); - } configuration = getRedisConfiguration(args, context); + + if (args.size() > 5) + structure = checkAndGetLiteralArgument(args[5], "structure"); + + if (structure.empty()) + { + switch (configuration->storage_type) + { + case RedisStorageType::SIMPLE: + structure = "key String, value String"; + break; + case RedisStorageType::HASH_MAP: + structure = "key String, field String, value String"; + break; + case RedisStorageType::UNKNOWN: + throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "Invalid Redis storage type."); + } + } } diff --git a/src/TableFunctions/TableFunctionRedis.h b/src/TableFunctions/TableFunctionRedis.h index 5c6f483fda7..1328d54a2a6 100644 --- a/src/TableFunctions/TableFunctionRedis.h +++ b/src/TableFunctions/TableFunctionRedis.h @@ -24,6 +24,7 @@ private: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; std::optional configuration; + String structure; }; } diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py index af01c2e9ff1..d4fbdaddd7f 100644 --- a/tests/integration/test_storage_redis/test.py +++ b/tests/integration/test_storage_redis/test.py @@ -1,5 +1,6 @@ import time +## sudo -H pip install redis import redis import pytest @@ -61,11 +62,11 @@ def test_storage_simple_select(started_cluster): """ ) - response = TSV.toMat(node.query("SELECT k, v from test_storage_simple_select where k='0' FORMAT TSV")) + response = TSV.toMat(node.query("SELECT k, v FROM test_storage_simple_select WHERE k='0' FORMAT TSV")) assert (len(response) == 1) assert (response[0] == ['0', '0']) - response = TSV.toMat(node.query("SELECT * from test_storage_simple_select order by k FORMAT TSV")) + response = TSV.toMat(node.query("SELECT * FROM test_storage_simple_select ORDER BY k FORMAT TSV")) assert (len(response) == 100) assert (response[0] == ['0', '0']) @@ -97,11 +98,11 @@ def test_storage_hash_map_select(started_cluster): """ ) - response = TSV.toMat(node.query("SELECT k, f, v from test_storage_hash_map_select where f='0' FORMAT TSV")) + response = TSV.toMat(node.query("SELECT k, f, v FROM test_storage_hash_map_select WHERE f='0' FORMAT TSV")) assert (len(response) == 1) assert (response[0] == ['k', '0', '0']) - response = TSV.toMat(node.query("SELECT * from test_storage_hash_map_select FORMAT TSV")) + response = TSV.toMat(node.query("SELECT * FROM test_storage_hash_map_select ORDER BY f FORMAT TSV")) assert (len(response) == 100) assert (response[0] == ['k', '0', '0']) diff --git a/tests/integration/test_table_function_redis/configs_secure/config.d/ssl_conf.xml b/tests/integration/test_table_function_redis/configs_secure/config.d/ssl_conf.xml deleted file mode 100644 index 3efe98e7045..00000000000 --- a/tests/integration/test_table_function_redis/configs_secure/config.d/ssl_conf.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - - none - - - diff --git a/tests/integration/test_table_function_redis/test.py b/tests/integration/test_table_function_redis/test.py index e0ad71b0079..e53022095c9 100644 --- a/tests/integration/test_table_function_redis/test.py +++ b/tests/integration/test_table_function_redis/test.py @@ -1,276 +1,159 @@ -import pymongo +import time +import redis import pytest -from helpers.client import QueryRuntimeException +from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance("node", with_redis=True) @pytest.fixture(scope="module") -def started_cluster(request): +def started_cluster(): try: - cluster = ClickHouseCluster(__file__) - node = cluster.add_instance( - "node", - with_mongo=True, - main_configs=[ - "configs_secure/config.d/ssl_conf.xml", - ], - with_mongo_secure=request.param, - ) cluster.start() yield cluster finally: cluster.shutdown() -def get_mongo_connection(started_cluster, secure=False, with_credentials=True): - connection_str = "" - if with_credentials: - connection_str = "mongodb://root:clickhouse@localhost:{}".format( - started_cluster.mongo_port - ) - else: - connection_str = "mongodb://localhost:{}".format( - started_cluster.mongo_no_cred_port - ) - if secure: - connection_str += "/?tls=true&tlsAllowInvalidCertificates=true" - return pymongo.MongoClient(connection_str) - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_simple_select(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - - node = started_cluster.instances["node"] - for i in range(0, 100): - node.query( - "INSERT INTO FUNCTION mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String') (key, data) VALUES ({}, '{}')".format( - i, hex(i * i) - ) - ) - assert ( - node.query( - "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" - ) - == "100\n" +def get_redis_connection(db_id=0): + client = redis.Redis( + host='localhost', port=cluster.redis_port, password="clickhouse", db=db_id ) - assert ( - node.query( - "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" - ) - == str(sum(range(0, 100))) + "\n" - ) - assert ( - node.query( - "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'key UInt64, data String')" - ) - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query( - "SELECT data from mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String') where key = 42" - ) - == hex(42 * 42) + "\n" - ) - simple_mongo_table.drop() + return client -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_complex_data_type(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - incomplete_mongo_table = db["complex_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i), "dict": {"a": i, "b": str(i)}}) - incomplete_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - - assert ( - node.query( - "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)')" - ) - == "100\n" - ) - assert ( - node.query( - "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)')" - ) - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query( - "SELECT data from mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)') where key = 42" - ) - == hex(42 * 42) + "\n" - ) - incomplete_mongo_table.drop() +def get_address_for_ch(): + return cluster.redis_host + ':6379' -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_incorrect_data_type(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - strange_mongo_table = db["strange_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i), "aaaa": "Hello"}) - strange_mongo_table.insert_many(data) +def test_storage_simple(started_cluster): + client = get_redis_connection() + address = get_address_for_ch() - node = started_cluster.instances["node"] + # clean all + client.flushall() + data = {} + for i in range(100): + data[str(i)] = str(i) + + client.mset(data) + client.close() + + response = TSV.toMat(node.query( + f""" + SELECT + key, value + FROM + redis('{address}', 0, 'clickhouse') + WHERE + key='0' + FORMAT TSV + """)) + + assert (len(response) == 1) + assert (response[0] == ['0', '0']) + + response = TSV.toMat(node.query( + f""" + SELECT + * + FROM + redis('{address}', 0, 'clickhouse') + ORDER BY + key + FORMAT TSV + """)) + + assert (len(response) == 100) + assert (response[0] == ['0', '0']) + + +def test_storage_hash_map(started_cluster): + client = get_redis_connection() + address = get_address_for_ch() + + # clean all + client.flushall() + + key = 'k' + data = {} + for i in range(100): + data[str(i)] = str(i) + + client.hset(key, mapping=data) + client.close() + + response = TSV.toMat(node.query( + f""" + SELECT + key, field, value + FROM + redis('{address}', 0, 'clickhouse','hash_map') + WHERE + field='0' + FORMAT TSV + """)) + + assert (len(response) == 1) + assert (response[0] == ['k', '0', '0']) + + response = TSV.toMat(node.query( + f""" + SELECT + * + FROM + redis('{address}', 0, 'clickhouse','hash_map') + ORDER BY + field + FORMAT TSV + """)) + + assert (len(response) == 100) + assert (response[0] == ['k', '0', '0']) + + +def test_customized_table_structure(started_cluster): + address = get_address_for_ch() + + node.query( + f""" + SELECT + * + FROM + redis('{address}', 0, 'clickhouse', "simple", 10, "k String, v UInt8") + """) + + node.query( + f""" + SELECT + * + FROM + redis('{address}', 0, 'clickhouse', "hash_map", 10, "k String, f UInt8, v String") + """) + + # illegal columns with pytest.raises(QueryRuntimeException): node.query( - "SELECT aaaa FROM mongodb('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse', structure='key UInt64, data String')" - ) + f""" + SELECT + * + FROM + redis('{address}', 0, 'clickhouse', "hash_map", 10, "k String, v String") + """) - strange_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [True], indirect=["started_cluster"]) -def test_secure_connection(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, secure=True) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - - assert ( + # illegal data type + with pytest.raises(QueryRuntimeException): node.query( - "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true')" - ) - == "100\n" - ) - assert ( - node.query( - "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true')" - ) - == str(sum(range(0, 100))) + "\n" - ) - assert ( - node.query( - "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'key UInt64, data String', 'ssl=true')" - ) - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query( - "SELECT data from mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true') where key = 42" - ) - == hex(42 * 42) + "\n" - ) - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_predefined_connection_configuration(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - assert ( - node.query( - "SELECT count() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" - ) - == "100\n" - ) - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_no_credentials(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) - db = mongo_connection["test"] - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - assert ( - node.query( - "SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', '', '', structure='key UInt64, data String')" - ) - == "100\n" - ) - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_auth_source(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) - admin_db = mongo_connection["admin"] - admin_db.add_user( - "root", - "clickhouse", - roles=[{"role": "userAdminAnyDatabase", "db": "admin"}, "readWriteAnyDatabase"], - ) - simple_mongo_table = admin_db["simple_table"] - data = [] - for i in range(0, 50): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - db = mongo_connection["test"] - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - - node.query_and_get_error( - "SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" - ) - - assert ( - node.query( - "SELECT count() FROM mongodb('mongo2:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='authSource=admin')" - ) - == "100\n" - ) - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_missing_columns(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 10): - data.append({"key": i, "data": hex(i * i)}) - for i in range(0, 10): - data.append({"key": i}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - result = node.query( - "SELECT count() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data Nullable(String)') WHERE isNull(data)" - ) - assert result == "10\n" - simple_mongo_table.drop() + f""" + SELECT + * + FROM + redis('{address}', 0, 'clickhouse', "simple", 10, "k Ss, v String") + """) From 357df40c8f6af947223fc54360340e86016e4eae Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 29 May 2023 15:22:29 +0800 Subject: [PATCH 0983/2223] fix tests --- src/Dictionaries/RedisSource.cpp | 8 ++- src/Storages/StorageRedis.cpp | 14 ++++- tests/integration/test_storage_redis/test.py | 1 + .../test_table_function_redis/test.py | 60 +++++++++++++++++++ 4 files changed, 78 insertions(+), 5 deletions(-) diff --git a/src/Dictionaries/RedisSource.cpp b/src/Dictionaries/RedisSource.cpp index 27125077c10..261242c627f 100644 --- a/src/Dictionaries/RedisSource.cpp +++ b/src/Dictionaries/RedisSource.cpp @@ -27,14 +27,18 @@ namespace DB const RedisStorageType & storage_type_, const DB::Block & sample_block, size_t max_block_size_) - : ISource(sample_block), max_block_size(max_block_size_)// TODO + : ISource(sample_block) + , connection(std::move(connection_)) + , keys(keys_) + , storage_type(storage_type_) + , max_block_size{max_block_size_} { RedisColumnTypes columns_types_; if (storage_type_ == RedisStorageType::HASH_MAP) columns_types_ = REDIS_HASH_MAP_COLUMN_TYPES; else columns_types_ = REDIS_SIMPLE_COLUMN_TYPES; - RedisSource(std::move(connection_), keys_, storage_type_, sample_block, columns_types_, max_block_size_); + description.init(sample_block); } RedisSource::RedisSource( diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index cd1cd06b4c4..e670012d060 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -75,9 +75,8 @@ Pipe StorageRedis::read( if (all_scan) { + /// TODO use scan to avoid performance issue RedisCommand command_for_keys("KEYS"); - /// generate keys by table name prefix -// command_for_keys << table_id.getTableName() + ":" + serializeStorageType(configuration.storage_type) + ":*"; command_for_keys << "*"; auto all_keys = connection->client->execute(command_for_keys); @@ -136,7 +135,16 @@ Pipe StorageRedis::read( RedisArray keys; for (size_t pos=begin; posat(pos).get()); + { + if (WhichDataType(*primary_key_data_type).isStringOrFixedString()) + { + keys.add(fields->at(pos).get()); + } + else + { + keys.add(toString(fields->at(pos))); /// TODO redis source deserialize + } + } if (configuration.storage_type == RedisStorageType::HASH_MAP) { diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py index d4fbdaddd7f..19e7b4e5340 100644 --- a/tests/integration/test_storage_redis/test.py +++ b/tests/integration/test_storage_redis/test.py @@ -181,3 +181,4 @@ def test_create_table(started_cluster): """ ) + diff --git a/tests/integration/test_table_function_redis/test.py b/tests/integration/test_table_function_redis/test.py index e53022095c9..7c342690027 100644 --- a/tests/integration/test_table_function_redis/test.py +++ b/tests/integration/test_table_function_redis/test.py @@ -157,3 +157,63 @@ def test_customized_table_structure(started_cluster): FROM redis('{address}', 0, 'clickhouse', "simple", 10, "k Ss, v String") """) + + +def test_data_type(started_cluster): + client = get_redis_connection() + address = get_address_for_ch() + + # string + client.flushall() + client.set('0', '0') + + response = TSV.toMat(node.query( + f""" + SELECT + * + FROM + redis('{address}', 0, 'clickhouse', 'simple', 10, "k String, v UInt8") + WHERE + k='0' + FORMAT TSV + """)) + + assert (len(response) == 1) + assert (response[0] == ['0', '0']) + + # number + client.flushall() + client.set('0', '0') + + response = TSV.toMat(node.query( + f""" + SELECT + * + FROM + redis('{address}', 0, 'clickhouse', 'simple', 10, "k UInt8, v UInt8") + WHERE + k=0 + FORMAT TSV + """)) + + assert (len(response) == 1) + assert (response[0] == ['0', '0']) + + # datetime + client.flushall() + client.set('2023-06-01 00:00:00', '0') + + response = TSV.toMat(node.query( + f""" + SELECT + * + FROM + redis('{address}', 0, 'clickhouse', 'simple', 10, "k DateTime, v UInt8") + WHERE + k='2023-06-01 00:00:00' + FORMAT TSV + """)) + + # TODO open + # assert (len(response) == 1) + # assert (response[0] == ['2023-06-01 00:00:00', '0']) From f4f939162dcd3b6814a9e4a288f5e0a0538ae283 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 30 May 2023 20:31:23 +0800 Subject: [PATCH 0984/2223] new redis engine schema design --- src/Dictionaries/RedisDictionarySource.cpp | 4 +- src/Dictionaries/RedisSource.cpp | 58 +-- src/Dictionaries/RedisSource.h | 10 - src/Storages/KVStorageUtils.cpp | 2 +- src/Storages/RedisCommon.cpp | 91 +--- src/Storages/RedisCommon.h | 33 +- src/Storages/StorageFactory.h | 3 +- src/Storages/StorageRedis.cpp | 425 +++++++++++++----- src/Storages/StorageRedis.h | 43 +- src/TableFunctions/TableFunctionRedis.cpp | 60 ++- src/TableFunctions/TableFunctionRedis.h | 6 +- tests/integration/test_storage_redis/test.py | 101 +++-- .../test_table_function_redis/test.py | 141 +++--- 13 files changed, 535 insertions(+), 442 deletions(-) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index d28b7528d23..c52c3425d1b 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -40,10 +40,10 @@ namespace DB { .host = host, .port = static_cast(port), - .db_index = config.getUInt(redis_config_prefix + ".db_index", 0), + .db_index = config.getUInt(redis_config_prefix + ".db_index", DEFAULT_REDIS_DB_INDEX), .password = config.getString(redis_config_prefix + ".password", ""), .storage_type = parseStorageType(config.getString(redis_config_prefix + ".storage_type", "")), - .pool_size = config.getUInt(redis_config_prefix + ".pool_size", 16), + .pool_size = config.getUInt(redis_config_prefix + ".pool_size", DEFAULT_REDIS_POOL_SIZE), }; return std::make_unique(dict_struct, configuration, sample_block); diff --git a/src/Dictionaries/RedisSource.cpp b/src/Dictionaries/RedisSource.cpp index 261242c627f..5d8a475cad4 100644 --- a/src/Dictionaries/RedisSource.cpp +++ b/src/Dictionaries/RedisSource.cpp @@ -21,7 +21,7 @@ namespace DB } - RedisSource::RedisSource( + RedisSource::RedisSource( RedisConnectionPtr connection_, const RedisArray & keys_, const RedisStorageType & storage_type_, @@ -32,28 +32,6 @@ namespace DB , keys(keys_) , storage_type(storage_type_) , max_block_size{max_block_size_} - { - RedisColumnTypes columns_types_; - if (storage_type_ == RedisStorageType::HASH_MAP) - columns_types_ = REDIS_HASH_MAP_COLUMN_TYPES; - else - columns_types_ = REDIS_SIMPLE_COLUMN_TYPES; - description.init(sample_block); - } - - RedisSource::RedisSource( - RedisConnectionPtr connection_, - const RedisArray & keys_, - const RedisStorageType & storage_type_, - const DB::Block & sample_block, - const RedisColumnTypes & columns_types_, - size_t max_block_size_) - : ISource(sample_block) - , connection(std::move(connection_)) - , keys(keys_) - , storage_type(storage_type_) - , max_block_size{max_block_size_} - , columns_types(columns_types_) { description.init(sample_block); } @@ -192,27 +170,15 @@ namespace DB const auto & primary_key = keys_array.get(0); for (size_t i = 0; i < values.size(); ++i) { - const auto & value = values.get(i); const auto & secondary_key = keys_array.get(i + 1); + const auto & value = values.get(i); /// null string means 'no value for requested key' if (!value.isNull()) { - for (size_t idx=0; idx serializeKeysToRawString(const ColumnWithTypeAndName & return result; } -/// In current implementation rocks db can have key with only one column. +/// In current implementation rocks db/redis can have key with only one column. size_t getPrimaryKeyPos(const Block & header, const Names & primary_key) { if (primary_key.size() != 1) diff --git a/src/Storages/RedisCommon.cpp b/src/Storages/RedisCommon.cpp index 86312f49f41..ba7c02fdac5 100644 --- a/src/Storages/RedisCommon.cpp +++ b/src/Storages/RedisCommon.cpp @@ -1,9 +1,7 @@ #include "RedisCommon.h" #include #include -#include #include -#include namespace DB { @@ -13,12 +11,10 @@ namespace ErrorCodes extern const int INVALID_REDIS_TABLE_STRUCTURE; extern const int INTERNAL_REDIS_ERROR; extern const int TIMEOUT_EXCEEDED; + extern const int BAD_ARGUMENTS; extern const int INVALID_REDIS_STORAGE_TYPE; } -RedisColumnTypes REDIS_HASH_MAP_COLUMN_TYPES = {RedisColumnType::KEY, RedisColumnType::FIELD, RedisColumnType::VALUE}; -RedisColumnTypes REDIS_SIMPLE_COLUMN_TYPES = {RedisColumnType::KEY, RedisColumnType::VALUE}; - RedisConnection::RedisConnection(RedisPoolPtr pool_, RedisClientPtr client_) : pool(std::move(pool_)), client(std::move(client_)) { @@ -153,89 +149,4 @@ RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisAr return hkeys; } -RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column) -{ - const String & redis_col_key = all_columns.at(0); - if (column == redis_col_key) - return RedisColumnType::KEY; - - if (storage_type == RedisStorageType::HASH_MAP) - { - const String & redis_col_field = all_columns.at(1); - if (column == redis_col_field) - return RedisColumnType::FIELD; - else - return RedisColumnType::VALUE; - } - else - { - return RedisColumnType::VALUE; - } -} - -RedisConfiguration getRedisConfiguration(ASTs & engine_args, ContextPtr context) -{ - RedisConfiguration configuration; - configuration.db_index = 0; - configuration.password = ""; - configuration.storage_type = RedisStorageType::SIMPLE; - configuration.pool_size = 10; - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) - { - validateNamedCollection( - *named_collection, - ValidateKeysMultiset{"host", "port", "hostname", "password", "db_index", "storage_type", "pool_size"}, - {}); - - configuration.host = named_collection->getAny({"host", "hostname"}); - configuration.port = static_cast(named_collection->getOrDefault("port", 6379)); - if (engine_args.size() > 1) - configuration.password = named_collection->get("password"); - if (engine_args.size() > 2) - configuration.db_index = static_cast(named_collection->get("db_index")); - if (engine_args.size() > 3) - configuration.storage_type = parseStorageType(named_collection->get("storage_type")); - if (engine_args.size() > 4) - configuration.pool_size = static_cast(named_collection->get("pool_size")); - } - else - { - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); - - /// 6379 is the default Redis port. - auto parsed_host_port = parseAddress(checkAndGetLiteralArgument(engine_args[0], "host:port"), 6379); - - configuration.host = parsed_host_port.first; - configuration.port = parsed_host_port.second; - if (engine_args.size() > 1) - configuration.db_index = static_cast(checkAndGetLiteralArgument(engine_args[1], "db_index")); - if (engine_args.size() > 2) - configuration.password = checkAndGetLiteralArgument(engine_args[2], "password"); - if (engine_args.size() > 3) - configuration.storage_type = parseStorageType(checkAndGetLiteralArgument(engine_args[3], "storage_type")); - if (engine_args.size() > 4) - configuration.pool_size = static_cast(checkAndGetLiteralArgument(engine_args[4], "pool_size")); - } - - if (configuration.storage_type == RedisStorageType::UNKNOWN) - throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "Invalid Redis storage type"); - - context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); - return configuration; -} - -void checkRedisTableStructure(const ColumnsDescription & columns, const RedisConfiguration & configuration) -{ - /// TODO check data type - if (configuration.storage_type == RedisStorageType::HASH_MAP && columns.size() != 3) - throw Exception(ErrorCodes::INVALID_REDIS_TABLE_STRUCTURE, - "Redis hash table must have 3 columns, but found {}", columns.size()); - - if (configuration.storage_type == RedisStorageType::SIMPLE && columns.size() != 2) - throw Exception(ErrorCodes::INVALID_REDIS_TABLE_STRUCTURE, - "Redis string table must have 2 columns, but found {}", columns.size()); -} - } diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index c378006f7a5..cb551a9a11a 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB { @@ -21,20 +22,6 @@ enum class RedisStorageType UNKNOWN }; -enum class RedisColumnType -{ - /// Redis key - KEY, - /// Redis hash field - FIELD, - /// Redis value - VALUE -}; - -using RedisColumnTypes = std::vector; - -extern RedisColumnTypes REDIS_HASH_MAP_COLUMN_TYPES; -extern RedisColumnTypes REDIS_SIMPLE_COLUMN_TYPES; /// storage type to Redis key type String storageTypeToKeyType(RedisStorageType type); @@ -52,6 +39,10 @@ struct RedisConfiguration uint32_t pool_size; }; +static uint32_t DEFAULT_REDIS_DB_INDEX = 0; +static uint32_t DEFAULT_REDIS_POOL_SIZE = 16; +static String DEFAULT_REDIS_PASSWORD; + using RedisArray = Poco::Redis::Array; using RedisArrayPtr = std::shared_ptr; using RedisCommand = Poco::Redis::Command; @@ -61,6 +52,9 @@ using RedisClientPtr = std::unique_ptr; using RedisPool = BorrowedObjectPool; using RedisPoolPtr = std::shared_ptr; +/// Redis scan interator +using RedisIterator = int64_t; + struct RedisConnection { RedisConnection(RedisPoolPtr pool_, RedisClientPtr client_); @@ -78,15 +72,4 @@ RedisConnectionPtr getRedisConnection(RedisPoolPtr pool, const RedisConfiguratio /// eg: keys -> [key1, key2] and get [[key1, field1, field2], [key2, field1, field2]] RedisArrayPtr getRedisHashMapKeys(const RedisConnectionPtr & connection, RedisArray & keys); -/// Get RedisColumnType of a column, If storage_type is -/// SIMPLE: all_columns must have 2 items and the first one is Redis key the second one is value -/// HASH_MAP: all_columns must have 2 items and the first one is Redis key the second is field, the third is value. -RedisColumnType getRedisColumnType(RedisStorageType storage_type, const Names & all_columns, const String & column); - -/// parse redis table engine/function configuration from engine_args -RedisConfiguration getRedisConfiguration(ASTs & engine_args, ContextPtr context); - -/// checking Redis table/table-function when creating -void checkRedisTableStructure(const ColumnsDescription & columns, const RedisConfiguration & configuration); - } diff --git a/src/Storages/StorageFactory.h b/src/Storages/StorageFactory.h index 77309541374..f1c1c237393 100644 --- a/src/Storages/StorageFactory.h +++ b/src/Storages/StorageFactory.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -14,8 +15,6 @@ namespace DB { class Context; -class ASTCreateQuery; -class ASTStorage; struct StorageID; diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index e670012d060..ceed448b4a7 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -1,11 +1,9 @@ #include #include #include -#include #include #include -#include #include #include #include @@ -13,32 +11,142 @@ #include #include #include +#include +#include +#include +#include +#include namespace DB { namespace ErrorCodes { - extern const int INVALID_REDIS_STORAGE_TYPE; extern const int NOT_IMPLEMENTED; } +class RedisDataSource : public ISource +{ +public: + RedisDataSource( + StorageRedis & storage_, + const Block & header, + FieldVectorPtr keys_, + FieldVector::const_iterator begin_, + FieldVector::const_iterator end_, + const size_t max_block_size_) + : ISource(header) + , storage(storage_) + , primary_key_pos(getPrimaryKeyPos(header, storage.getPrimaryKey())) + , keys(keys_) + , begin(begin_) + , end(end_) + , it(begin) + , max_block_size(max_block_size_) + { + } + + RedisDataSource( + StorageRedis & storage_, + const Block & header, + const size_t max_block_size_, + const String & pattern_ = "*") + : ISource(header) + , storage(storage_) + , primary_key_pos(getPrimaryKeyPos(header, storage.getPrimaryKey())) + , iterator(-1) + , pattern(pattern_) + , max_block_size(max_block_size_) + { + } + + String getName() const override { return storage.getName(); } + + Chunk generate() override + { + if (keys) + return generateWithKeys(); + return generateFullScan(); + } + + Chunk generateWithKeys() + { + const auto & sample_block = getPort().getHeader(); + if (it >= end) + { + it = {}; + return {}; + } + + const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey().at(0)).type; + auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size); + return storage.getBySerializedKeys(raw_keys, nullptr); + } + + Chunk generateFullScan() + { + /// redis scan ending + if (iterator == 0) + return {}; + + RedisArray scan_keys; + RedisIterator next_iterator; + + std::tie(next_iterator, scan_keys) = storage.scan(iterator == -1 ? 0 : iterator, pattern, max_block_size); + iterator = next_iterator; + + /// redis scan can return nothing + if (scan_keys.isNull() || scan_keys.size() == 0) + return generateFullScan(); + + const auto & sample_block = getPort().getHeader(); + MutableColumns columns = sample_block.cloneEmptyColumns(); + + RedisArray values = storage.multiGet(scan_keys); + for (size_t i = 0; i(i).isNull(); i++) + { + fillColumns(scan_keys.get(i).value(), + values.get(i).value(), + primary_key_pos, sample_block, columns + ); + } + + Block block = sample_block.cloneWithColumns(std::move(columns)); + return Chunk(block.getColumns(), block.rows()); + } + +private: + StorageRedis & storage; + + size_t primary_key_pos; + + /// For key scan + FieldVectorPtr keys = nullptr; + FieldVector::const_iterator begin; + FieldVector::const_iterator end; + FieldVector::const_iterator it; + + /// For full scan + RedisIterator iterator; + String pattern; + + const size_t max_block_size; +}; + StorageRedis::StorageRedis( const StorageID & table_id_, const RedisConfiguration & configuration_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment_) + ContextPtr context_, + const StorageInMemoryMetadata & storage_metadata, + const String & primary_key_) : IStorage(table_id_) + , WithContext(context_->getGlobalContext()) , table_id(table_id_) , configuration(configuration_) , log(&Poco::Logger::get("StorageRedis")) + , primary_key(primary_key_) { pool = std::make_shared(configuration.pool_size); - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment_); setInMemoryMetadata(storage_metadata); } @@ -46,84 +154,37 @@ Pipe StorageRedis::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, size_t num_streams) { - auto connection = getRedisConnection(pool, configuration); storage_snapshot->check(column_names); - Block sample_block; - RedisColumnTypes redis_types; - auto all_columns = storage_snapshot->metadata->getColumns().getNamesOfPhysical(); - - for (const String & column_name : column_names) - { - auto column_data = storage_snapshot->metadata->getColumns().getPhysical(column_name); - sample_block.insert({column_data.type, column_data.name}); - redis_types.push_back(getRedisColumnType(configuration.storage_type, all_columns, column_name)); - } - - FieldVectorPtr fields; + FieldVectorPtr keys; bool all_scan = false; - String primary_key = all_columns.at(0); - auto primary_key_data_type = sample_block.getByName(primary_key).type; + Block header = storage_snapshot->metadata->getSampleBlock(); + auto primary_key_data_type = header.getByName(primary_key).type; - std::tie(fields, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info, context); + std::tie(keys, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info, context_); if (all_scan) { - /// TODO use scan to avoid performance issue - RedisCommand command_for_keys("KEYS"); - command_for_keys << "*"; - - auto all_keys = connection->client->execute(command_for_keys); - - if (all_keys.isNull() || all_keys.size() == 0) - return {}; - - Pipes pipes; - - size_t num_keys = all_keys.size(); - size_t num_threads = std::min(num_streams, all_keys.size()); - - num_threads = std::min(num_threads, configuration.pool_size); - assert(num_keys <= std::numeric_limits::max()); - - for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) - { - size_t begin = num_keys * thread_idx / num_threads; - size_t end = num_keys * (thread_idx + 1) / num_threads; - - RedisArray keys; - for (size_t pos=begin; pos(pos)); - - if (configuration.storage_type == RedisStorageType::HASH_MAP) - { - keys = *getRedisHashMapKeys(connection, keys); - } - - delete connection.release(); - - /// TODO reduce keys copy - pipes.emplace_back(std::make_shared( - getRedisConnection(pool, configuration), keys, - configuration.storage_type, sample_block, redis_types, max_block_size)); - } - return Pipe::unitePipes(std::move(pipes)); + return Pipe(std::make_shared(*this, header, max_block_size)); } else { - if (fields->empty()) + if (keys->empty()) return {}; Pipes pipes; - size_t num_keys = fields->size(); - size_t num_threads = std::min(num_streams, fields->size()); + ::sort(keys->begin(), keys->end()); + keys->erase(std::unique(keys->begin(), keys->end()), keys->end()); + + size_t num_keys = keys->size(); + size_t num_threads = std::min(num_streams, keys->size()); num_threads = std::min(num_threads, configuration.pool_size); assert(num_keys <= std::numeric_limits::max()); @@ -133,34 +194,191 @@ Pipe StorageRedis::read( size_t begin = num_keys * thread_idx / num_threads; size_t end = num_keys * (thread_idx + 1) / num_threads; - RedisArray keys; - for (size_t pos=begin; posat(pos).get()); - } - else - { - keys.add(toString(fields->at(pos))); /// TODO redis source deserialize - } - } - - if (configuration.storage_type == RedisStorageType::HASH_MAP) - { - keys = *getRedisHashMapKeys(connection, keys); - } - - delete connection.release(); - - pipes.emplace_back(std::make_shared( - getRedisConnection(pool, configuration), keys, - configuration.storage_type, sample_block, redis_types, max_block_size)); + pipes.emplace_back(std::make_shared( + *this, header, keys, keys->begin() + begin, keys->begin() + end, max_block_size)); } return Pipe::unitePipes(std::move(pipes)); } } +namespace +{ + // host:port, db_index, password, pool_size + RedisConfiguration getRedisConfiguration(ASTs & engine_args, ContextPtr context) + { + RedisConfiguration configuration; + + if (engine_args.size() < 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad arguments count when creating Redis table engine"); + + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) + { + validateNamedCollection( + *named_collection, + ValidateKeysMultiset{"host", "port", "hostname", "password", "db_index", "pool_size"}, + {}); + + configuration.host = named_collection->getAny({"host", "hostname"}); + configuration.port = static_cast(named_collection->getOrDefault("port", 6379)); + configuration.password = named_collection->getOrDefault("password", DEFAULT_REDIS_PASSWORD); + configuration.db_index = static_cast(named_collection->getOrDefault("db_index", DEFAULT_REDIS_DB_INDEX)); + configuration.pool_size = static_cast(named_collection->getOrDefault("pool_size", DEFAULT_REDIS_POOL_SIZE)); + } + else + { + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); + + /// 6379 is the default Redis port. + auto parsed_host_port = parseAddress(checkAndGetLiteralArgument(engine_args[0], "host:port"), 6379); + configuration.host = parsed_host_port.first; + configuration.port = parsed_host_port.second; + + if (engine_args.size() > 1) + configuration.db_index = static_cast(checkAndGetLiteralArgument(engine_args[1], "db_index")); + else + configuration.db_index = DEFAULT_REDIS_DB_INDEX; + if (engine_args.size() > 2) + configuration.password = checkAndGetLiteralArgument(engine_args[2], "password"); + else + configuration.password = DEFAULT_REDIS_PASSWORD; + if (engine_args.size() > 3) + configuration.pool_size = static_cast(checkAndGetLiteralArgument(engine_args[3], "pool_size")); + else + configuration.pool_size = DEFAULT_REDIS_POOL_SIZE; + } + + context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); + return configuration; + } + + StoragePtr createStorageRedis(const StorageFactory::Arguments & args) + { + auto configuration = getRedisConfiguration(args.engine_args, args.getLocalContext()); + + StorageInMemoryMetadata metadata; + metadata.setColumns(args.columns); + metadata.setConstraints(args.constraints); + metadata.setComment(args.comment); + + if (!args.storage_def->primary_key) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageRedis must require one column in primary key"); + + auto primary_key_desc = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); + auto primary_key_names = primary_key_desc.expression->getRequiredColumns(); + + if (primary_key_names.size() != 1) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageRedis must require one column in primary key"); + } + + return std::make_shared( + args.table_id, + configuration, + args.getContext(), + metadata, + primary_key_names[0]); + } +} + +Chunk StorageRedis::getBySerializedKeys( + const std::vector & keys, + PaddedPODArray * null_map) const +{ + RedisArray redis_keys; + for (const auto & key : keys) + redis_keys.add(key); + return getBySerializedKeys(redis_keys, null_map); +} + +Chunk StorageRedis::getBySerializedKeys( + const RedisArray & keys, + PaddedPODArray * null_map) const +{ + Block sample_block = getInMemoryMetadataPtr()->getSampleBlock(); + + size_t primary_key_pos = getPrimaryKeyPos(sample_block, getPrimaryKey()); + MutableColumns columns = sample_block.cloneEmptyColumns(); + + RedisArray values = multiGet(keys); + if (values.isNull() || values.size() == 0) + return {}; + + if (null_map) + { + null_map->clear(); + null_map->resize_fill(keys.size(), 1); + } + + for (size_t i = 0; i < values.size(); ++i) + { + if (!values.get(i).isNull()) + { + fillColumns(keys.get(i).value(), + values.get(i).value(), + primary_key_pos, sample_block, columns + ); + } + else /// key not found + { + if (null_map) + { + (*null_map)[i] = 0; + for (size_t col_idx = 0; col_idx < sample_block.columns(); ++col_idx) + { + columns[col_idx]->insert(sample_block.getByPosition(col_idx).type->getDefault()); + } + } + } + } + + size_t num_rows = columns.at(0)->size(); + return Chunk(std::move(columns), num_rows); +} + +std::pair StorageRedis::scan(RedisIterator iterator, const String & pattern, const uint64_t max_count) +{ + auto connection = getRedisConnection(pool, configuration); + RedisCommand scan("SCAN"); + scan << toString(iterator) << "MATCH" << pattern << "COUNT" << toString(max_count); + + const auto & result = connection->client->execute(scan); + RedisIterator next = parse(result.get(0).value()); + + return {next, result.get(1)}; +} + +RedisArray StorageRedis::multiGet(const RedisArray & keys) const +{ + auto connection = getRedisConnection(pool, configuration); + + RedisCommand cmd_mget("MGET"); + for (size_t i = 0; i < keys.size(); ++i) + cmd_mget.add(keys.get(i)); + + return connection->client->execute(cmd_mget); +} + +Chunk StorageRedis::getByKeys( + const ColumnsWithTypeAndName & keys, + PaddedPODArray & null_map, + const Names &) const +{ + if (keys.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "StorageRedis supports only one key, got: {}", keys.size()); + + auto raw_keys = serializeKeysToRawString(keys[0]); + + if (raw_keys.size() != keys[0].column->size()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); + + return getBySerializedKeys(raw_keys, &null_map); +} + +Block StorageRedis::getSampleBlock(const Names &) const +{ + return getInMemoryMetadataPtr()->getSampleBlock(); +} SinkToStoragePtr StorageRedis::write( const ASTPtr & /*query*/, @@ -172,24 +390,13 @@ SinkToStoragePtr StorageRedis::write( void registerStorageRedis(StorageFactory & factory) { - factory.registerStorage( - "Redis", - [](const StorageFactory::Arguments & args) - { - auto configuration = getRedisConfiguration(args.engine_args, args.getLocalContext()); + StorageFactory::StorageFeatures features{ + .supports_sort_order = true, + .supports_parallel_insert = true, + .source_access_type = AccessType::REDIS, + }; - checkRedisTableStructure(args.columns, configuration); - - return std::make_shared( - args.table_id, - configuration, - args.columns, - args.constraints, - args.comment); - }, - { - .source_access_type = AccessType::REDIS, - }); + factory.registerStorage("Redis", createStorageRedis, features); } } diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h index 619a83f3851..4a0418e6091 100644 --- a/src/Storages/StorageRedis.h +++ b/src/Storages/StorageRedis.h @@ -2,27 +2,24 @@ #include #include -#include +#include +#include +#include namespace DB { /* Implements storage in the Redis. - * Use ENGINE = Redis(host:port, db_index, password, storage_type, pool_size); - * Read only. - * - * Note If storage_type is - * SIMPLE: there should be 2 columns and the first one is key in Redis, the second one is value. - * HASH_MAP: there should be 3 columns and the first one is key in Redis and the second is the field of Redis Map. + * Use ENGINE = Redis(host:port[, db_index[, password[, pool_size]]]) PRIMARY KEY(key); */ -class StorageRedis : public IStorage +class StorageRedis : public IStorage, public IKeyValueEntity, WithContext { public: StorageRedis( const StorageID & table_id_, const RedisConfiguration & configuration_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment_); + ContextPtr context_, + const StorageInMemoryMetadata & storage_metadata, + const String & primary_key_); std::string getName() const override { return "Redis"; } @@ -30,7 +27,7 @@ public: const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr context_, QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams) override; @@ -40,12 +37,34 @@ public: const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + Names getPrimaryKey() const override { return {primary_key}; } + + /// Return chunk with data for given serialized keys. + /// If out_null_map is passed, fill it with 1/0 depending on key was/wasn't found. Result chunk may contain default values. + /// If out_null_map is not passed. Not found rows excluded from result chunk. + Chunk getBySerializedKeys( + const std::vector & keys, + PaddedPODArray * out_null_map) const; + + Chunk getBySerializedKeys( + const RedisArray & keys, + PaddedPODArray * out_null_map) const; + + std::pair scan(RedisIterator iterator, const String & pattern, const uint64_t max_count); + + RedisArray multiGet(const RedisArray & keys) const; + + Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, const Names &) const override; + + Block getSampleBlock(const Names &) const override; private: StorageID table_id; RedisConfiguration configuration; Poco::Logger * log; RedisPoolPtr pool; + + const String primary_key; }; } diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp index f90a30af8a1..3db174fbcd8 100644 --- a/src/TableFunctions/TableFunctionRedis.cpp +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -1,6 +1,7 @@ #include #include +#include #include @@ -12,6 +13,8 @@ #include #include #include +#include + namespace DB @@ -21,7 +24,6 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int INVALID_REDIS_STORAGE_TYPE; } @@ -29,19 +31,16 @@ StoragePtr TableFunctionRedis::executeImpl( const ASTPtr & /*ast_function*/, ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/) const { auto columns = getActualTableStructure(context); - checkRedisTableStructure(columns, *configuration); + + StorageInMemoryMetadata metadata; + metadata.setColumns(columns); auto storage = std::make_shared( - StorageID(toString(configuration->db_index), table_name), - *configuration, - columns, - ConstraintsDescription(), - String{}); + StorageID(toString(configuration.db_index), table_name), configuration, context, metadata, primary_key); storage->startup(); return storage; } -/// TODO support user customized table structure ColumnsDescription TableFunctionRedis::getActualTableStructure(ContextPtr context) const { return parseColumnsListFromString(structure, context); @@ -54,25 +53,38 @@ void TableFunctionRedis::parseArguments(const ASTPtr & ast_function, ContextPtr throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'redis' must have arguments."); ASTs & args = func_args.arguments->children; - configuration = getRedisConfiguration(args, context); + if (args.size() < 3) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad arguments count when creating Redis table function"); + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + auto parsed_host_port = parseAddress(checkAndGetLiteralArgument(args[0], "host:port"), 6379); + configuration.host = parsed_host_port.first; + configuration.port = parsed_host_port.second; + + primary_key = checkAndGetLiteralArgument(args[1], "key"); + structure = checkAndGetLiteralArgument(args[2], "structure"); + + if (args.size() > 3) + configuration.db_index = static_cast(checkAndGetLiteralArgument(args[3], "db_index")); + else + configuration.db_index = DEFAULT_REDIS_DB_INDEX; + if (args.size() > 4) + configuration.password = checkAndGetLiteralArgument(args[4], "password"); + else + configuration.password = DEFAULT_REDIS_PASSWORD; if (args.size() > 5) - structure = checkAndGetLiteralArgument(args[5], "structure"); + configuration.pool_size = static_cast(checkAndGetLiteralArgument(args[5], "pool_size")); + else + configuration.pool_size = DEFAULT_REDIS_POOL_SIZE; - if (structure.empty()) - { - switch (configuration->storage_type) - { - case RedisStorageType::SIMPLE: - structure = "key String, value String"; - break; - case RedisStorageType::HASH_MAP: - structure = "key String, field String, value String"; - break; - case RedisStorageType::UNKNOWN: - throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "Invalid Redis storage type."); - } - } + context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); + + auto columns = parseColumnsListFromString(structure, context); + if (!columns.has(primary_key)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad arguments redis table function structure should contains key."); } diff --git a/src/TableFunctions/TableFunctionRedis.h b/src/TableFunctions/TableFunctionRedis.h index 1328d54a2a6..b985a89e3d7 100644 --- a/src/TableFunctions/TableFunctionRedis.h +++ b/src/TableFunctions/TableFunctionRedis.h @@ -7,6 +7,9 @@ namespace DB { +/* Implements Redis table function. + * Use redis(host:port, key, structure[, db_index[, password[, pool_size]]]); + */ class TableFunctionRedis : public ITableFunction { public: @@ -23,8 +26,9 @@ private: ColumnsDescription getActualTableStructure(ContextPtr context) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - std::optional configuration; + RedisConfiguration configuration; String structure; + String primary_key; }; } diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py index 19e7b4e5340..1f65a9df2f3 100644 --- a/tests/integration/test_storage_redis/test.py +++ b/tests/integration/test_storage_redis/test.py @@ -3,6 +3,8 @@ import time ## sudo -H pip install redis import redis import pytest +import struct +import sys from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster @@ -37,17 +39,50 @@ def drop_table(table): node.query(f"DROP TABLE IF EXISTS {table} SYNC"); -def test_storage_simple_select(started_cluster): +# see SerializationString.serializeBinary +def serialize_binary_for_string(x): + var_uint_max = (1 << 63) - 1 + buf = bytearray() + # write length + length = len(x) + # length = (length << 1) ^ (length >> 63) + if length > var_uint_max: + raise ValueError("Value too large for varint encoding") + for i in range(9): + byte = length & 0x7F + if length > 0x7F: + byte |= 0x80 + buf += (bytes([byte])) + length >>= 7 + if not length: + break + # write data + buf += x.encode('utf-8') + return bytes(buf) + + +# see SerializationNumber.serializeBinary +def serialize_binary_for_uint32(x): + buf = bytearray() + packed_num = struct.pack('I', x) + buf += packed_num + if sys.byteorder != 'little': + buf.reverse() + return bytes(buf) + + +def test_simple_select(started_cluster): client = get_redis_connection() address = get_address_for_ch() # clean all client.flushall() - drop_table('test_storage_simple_select') + drop_table('test_simple_select') data = {} for i in range(100): - data[str(i)] = str(i) + packed = serialize_binary_for_string(str(i)) + data[packed] = packed client.mset(data) client.close() @@ -55,56 +90,55 @@ def test_storage_simple_select(started_cluster): # create table node.query( f""" - CREATE TABLE test_storage_simple_select( + CREATE TABLE test_simple_select( k String, - v UInt32 - ) Engine=Redis('{address}', 0, 'clickhouse') + v String + ) Engine=Redis('{address}', 0, 'clickhouse') PRIMARY KEY (k) """ ) - response = TSV.toMat(node.query("SELECT k, v FROM test_storage_simple_select WHERE k='0' FORMAT TSV")) + response = TSV.toMat(node.query("SELECT k, v FROM test_simple_select WHERE k='0' FORMAT TSV")) assert (len(response) == 1) assert (response[0] == ['0', '0']) - response = TSV.toMat(node.query("SELECT * FROM test_storage_simple_select ORDER BY k FORMAT TSV")) + response = TSV.toMat(node.query("SELECT * FROM test_simple_select ORDER BY k FORMAT TSV")) assert (len(response) == 100) assert (response[0] == ['0', '0']) -def test_storage_hash_map_select(started_cluster): +def test_select_int(started_cluster): client = get_redis_connection() address = get_address_for_ch() # clean all client.flushall() - drop_table('test_storage_hash_map_select') + drop_table('test_select_int') - key = 'k' data = {} for i in range(100): - data[str(i)] = str(i) + packed = serialize_binary_for_uint32(i) + data[packed] = packed - client.hset(key, mapping=data) + client.mset(data) client.close() # create table node.query( f""" - CREATE TABLE test_storage_hash_map_select( - k String, - f String, + CREATE TABLE test_select_int( + k UInt32, v UInt32 - ) Engine=Redis('{address}', 0, 'clickhouse','hash_map') + ) Engine=Redis('{address}', 0, 'clickhouse') PRIMARY KEY (k) """ ) - response = TSV.toMat(node.query("SELECT k, f, v FROM test_storage_hash_map_select WHERE f='0' FORMAT TSV")) + response = TSV.toMat(node.query("SELECT k, v FROM test_select_int WHERE k=0 FORMAT TSV")) assert (len(response) == 1) - assert (response[0] == ['k', '0', '0']) + assert (response[0] == ['0', '0']) - response = TSV.toMat(node.query("SELECT * FROM test_storage_hash_map_select ORDER BY f FORMAT TSV")) + response = TSV.toMat(node.query("SELECT * FROM test_select_int ORDER BY k FORMAT TSV")) assert (len(response) == 100) - assert (response[0] == ['k', '0', '0']) + assert (response[0] == ['0', '0']) def test_create_table(started_cluster): @@ -117,7 +151,7 @@ def test_create_table(started_cluster): CREATE TABLE test_create_table( k String, v UInt32 - ) Engine=Redis('{address}') + ) Engine=Redis('{address}') PRIMARY KEY (k) """ ) @@ -128,7 +162,7 @@ def test_create_table(started_cluster): CREATE TABLE test_create_table( k String, v UInt32 - ) Engine=Redis('{address}', 0, 'clickhouse','simple', 10) + ) Engine=Redis('{address}', 0, 'clickhouse', 10) PRIMARY KEY (k) """ ) @@ -139,11 +173,10 @@ def test_create_table(started_cluster): k String, f String, v UInt32 - ) Engine=Redis('{address}', 0, 'clickhouse','hash_map', 10) + ) Engine=Redis('{address}', 0, 'clickhouse', 10) PRIMARY KEY (k) """ ) - # illegal columns drop_table('test_create_table') with pytest.raises(QueryRuntimeException): node.query( @@ -152,7 +185,7 @@ def test_create_table(started_cluster): k String, f String, v UInt32 - ) Engine=Redis('{address}', 0, 'clickhouse','simple', 10) + ) Engine=Redis('{address}', 0, 'clickhouse', 10) PRIMARY KEY () """ ) @@ -163,22 +196,8 @@ def test_create_table(started_cluster): CREATE TABLE test_create_table( k String, f String, - v UInt32, - n UInt32 - ) Engine=Redis('{address}', 0, 'clickhouse','hash_map', 10) - """ - ) - - # illegal storage type - drop_table('test_create_table') - with pytest.raises(QueryRuntimeException): - node.query( - f""" - CREATE TABLE test_create_table( - k String, v UInt32 - ) Engine=Redis('{address}', 0, 'clickhouse','not_exist', 10) + ) Engine=Redis('{address}', 0, 'clickhouse', 10) """ ) - diff --git a/tests/integration/test_table_function_redis/test.py b/tests/integration/test_table_function_redis/test.py index 7c342690027..111276ec6dc 100644 --- a/tests/integration/test_table_function_redis/test.py +++ b/tests/integration/test_table_function_redis/test.py @@ -1,7 +1,9 @@ -import time +import datetime import redis import pytest +import sys +import struct from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster @@ -32,7 +34,39 @@ def get_address_for_ch(): return cluster.redis_host + ':6379' -def test_storage_simple(started_cluster): +# see SerializationString.serializeBinary +def serialize_binary_for_string(x): + var_uint_max = (1 << 63) - 1 + buf = bytearray() + # write length + length = len(x) + # length = (length << 1) ^ (length >> 63) + if length > var_uint_max: + raise ValueError("Value too large for varint encoding") + for i in range(9): + byte = length & 0x7F + if length > 0x7F: + byte |= 0x80 + buf += (bytes([byte])) + length >>= 7 + if not length: + break + # write data + buf += x.encode('utf-8') + return bytes(buf) + + +# see SerializationNumber.serializeBinary +def serialize_binary_for_uint32(x): + buf = bytearray() + packed_num = struct.pack('I', x) + buf += packed_num + if sys.byteorder != 'little': + buf.reverse() + return bytes(buf) + + +def test_simple_select(started_cluster): client = get_redis_connection() address = get_address_for_ch() @@ -41,7 +75,8 @@ def test_storage_simple(started_cluster): data = {} for i in range(100): - data[str(i)] = str(i) + packed = serialize_binary_for_string(str(i)) + data[packed] = packed client.mset(data) client.close() @@ -51,7 +86,7 @@ def test_storage_simple(started_cluster): SELECT key, value FROM - redis('{address}', 0, 'clickhouse') + redis('{address}', 'key', 'key String, value String', 0, 'clickhouse', 10) WHERE key='0' FORMAT TSV @@ -65,7 +100,7 @@ def test_storage_simple(started_cluster): SELECT * FROM - redis('{address}', 0, 'clickhouse') + redis('{address}', 'key', 'key String, value String', 0, 'clickhouse', 10) ORDER BY key FORMAT TSV @@ -75,79 +110,22 @@ def test_storage_simple(started_cluster): assert (response[0] == ['0', '0']) -def test_storage_hash_map(started_cluster): +def test_create_table(started_cluster): client = get_redis_connection() address = get_address_for_ch() # clean all client.flushall() - - key = 'k' - data = {} - for i in range(100): - data[str(i)] = str(i) - - client.hset(key, mapping=data) client.close() - response = TSV.toMat(node.query( - f""" - SELECT - key, field, value - FROM - redis('{address}', 0, 'clickhouse','hash_map') - WHERE - field='0' - FORMAT TSV - """)) - - assert (len(response) == 1) - assert (response[0] == ['k', '0', '0']) - - response = TSV.toMat(node.query( - f""" - SELECT - * - FROM - redis('{address}', 0, 'clickhouse','hash_map') - ORDER BY - field - FORMAT TSV - """)) - - assert (len(response) == 100) - assert (response[0] == ['k', '0', '0']) - - -def test_customized_table_structure(started_cluster): - address = get_address_for_ch() - node.query( f""" SELECT * FROM - redis('{address}', 0, 'clickhouse', "simple", 10, "k String, v UInt8") + redis('{address}', 'k', 'k String, v UInt32', 0, 'clickhouse', 10) """) - node.query( - f""" - SELECT - * - FROM - redis('{address}', 0, 'clickhouse', "hash_map", 10, "k String, f UInt8, v String") - """) - - # illegal columns - with pytest.raises(QueryRuntimeException): - node.query( - f""" - SELECT - * - FROM - redis('{address}', 0, 'clickhouse', "hash_map", 10, "k String, v String") - """) - # illegal data type with pytest.raises(QueryRuntimeException): node.query( @@ -155,7 +133,17 @@ def test_customized_table_structure(started_cluster): SELECT * FROM - redis('{address}', 0, 'clickhouse', "simple", 10, "k Ss, v String") + redis('{address}', 'k', 'k not_exist_type, v String', 0, 'clickhouse', 10) + """) + + # illegal key + with pytest.raises(QueryRuntimeException): + node.query( + f""" + SELECT + * + FROM + redis('{address}', 'not_exist_key', 'k not_exist_type, v String', 0, 'clickhouse', 10) """) @@ -165,14 +153,15 @@ def test_data_type(started_cluster): # string client.flushall() - client.set('0', '0') + value = serialize_binary_for_string('0') + client.set(value, value) response = TSV.toMat(node.query( f""" SELECT * FROM - redis('{address}', 0, 'clickhouse', 'simple', 10, "k String, v UInt8") + redis('{address}', 'k', 'k String, v String', 0, 'clickhouse', 10) WHERE k='0' FORMAT TSV @@ -183,14 +172,15 @@ def test_data_type(started_cluster): # number client.flushall() - client.set('0', '0') + value = serialize_binary_for_uint32(0) + client.set(value, value) response = TSV.toMat(node.query( f""" SELECT * FROM - redis('{address}', 0, 'clickhouse', 'simple', 10, "k UInt8, v UInt8") + redis('{address}', 'k', 'k UInt32, v UInt32', 0, 'clickhouse', 10) WHERE k=0 FORMAT TSV @@ -201,19 +191,22 @@ def test_data_type(started_cluster): # datetime client.flushall() - client.set('2023-06-01 00:00:00', '0') + # clickhouse store datatime as uint32 in internal + dt = datetime.datetime(2023, 6, 1, 0, 0, 0) + seconds_since_epoch = dt.timestamp() + value = serialize_binary_for_uint32(int(seconds_since_epoch)) + client.set(value, value) response = TSV.toMat(node.query( f""" SELECT * FROM - redis('{address}', 0, 'clickhouse', 'simple', 10, "k DateTime, v UInt8") + redis('{address}', 'k', 'k DateTime, v DateTime', 0, 'clickhouse', 10) WHERE k='2023-06-01 00:00:00' FORMAT TSV """)) - # TODO open - # assert (len(response) == 1) - # assert (response[0] == ['2023-06-01 00:00:00', '0']) + assert (len(response) == 1) + assert (response[0] == ['2023-06-01 00:00:00', '2023-06-01 00:00:00']) From 1df1dfc3e54b9e9b2b0f05a516ffc83ff3147c76 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 31 May 2023 14:31:06 +0800 Subject: [PATCH 0985/2223] add update/delete/insert to redis storage --- src/Storages/RedisCommon.h | 4 +- src/Storages/StorageRedis.cpp | 198 ++++++++++++++++++- src/Storages/StorageRedis.h | 14 +- tests/integration/test_storage_redis/test.py | 135 ++++++++++++- 4 files changed, 344 insertions(+), 7 deletions(-) diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index cb551a9a11a..49c21c3277f 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -43,10 +43,12 @@ static uint32_t DEFAULT_REDIS_DB_INDEX = 0; static uint32_t DEFAULT_REDIS_POOL_SIZE = 16; static String DEFAULT_REDIS_PASSWORD; +using RedisCommand = Poco::Redis::Command; using RedisArray = Poco::Redis::Array; using RedisArrayPtr = std::shared_ptr; -using RedisCommand = Poco::Redis::Command; using RedisBulkString = Poco::Redis::BulkString; +using RedisSimpleString = String; +using RedisInteger = Int64; using RedisClientPtr = std::unique_ptr; using RedisPool = BorrowedObjectPool; diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index ceed448b4a7..f9a25470e2d 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -13,9 +13,12 @@ #include #include #include +#include #include #include #include +#include +#include namespace DB { @@ -23,6 +26,7 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int INTERNAL_REDIS_ERROR; } class RedisDataSource : public ISource @@ -133,6 +137,64 @@ private: const size_t max_block_size; }; + +class RedisSink : public SinkToStorage +{ +public: + RedisSink( + StorageRedis & storage_, + const StorageMetadataPtr & metadata_snapshot_); + + void consume(Chunk chunk) override; + String getName() const override { return "RedisSink"; } + +private: + StorageRedis & storage; + StorageMetadataPtr metadata_snapshot; + size_t primary_key_pos = 0; +}; + +RedisSink::RedisSink( + StorageRedis & storage_, + const StorageMetadataPtr & metadata_snapshot_) + : SinkToStorage(metadata_snapshot_->getSampleBlock()) + , storage(storage_) + , metadata_snapshot(metadata_snapshot_) +{ + for (const auto & column : getHeader()) + { + if (column.name == storage.getPrimaryKey()[0]) + break; + ++primary_key_pos; + } +} + +void RedisSink::consume(Chunk chunk) +{ + auto rows = chunk.getNumRows(); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + + WriteBufferFromOwnString wb_key; + WriteBufferFromOwnString wb_value; + + RedisArray data; + for (size_t i = 0; i < rows; ++i) + { + wb_key.restart(); + wb_value.restart(); + + size_t idx = 0; + for (const auto & elem : block) + { + elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value, {}); + ++idx; + } + data.add(wb_key.str()); + data.add(wb_value.str()); + } + storage.multiSet(data); +} + StorageRedis::StorageRedis( const StorageID & table_id_, const RedisConfiguration & configuration_, @@ -336,7 +398,7 @@ Chunk StorageRedis::getBySerializedKeys( return Chunk(std::move(columns), num_rows); } -std::pair StorageRedis::scan(RedisIterator iterator, const String & pattern, const uint64_t max_count) +std::pair StorageRedis::scan(RedisIterator iterator, const String & pattern, uint64_t max_count) { auto connection = getRedisConnection(pool, configuration); RedisCommand scan("SCAN"); @@ -359,6 +421,36 @@ RedisArray StorageRedis::multiGet(const RedisArray & keys) const return connection->client->execute(cmd_mget); } +void StorageRedis::multiSet(const RedisArray & data) const +{ + auto connection = getRedisConnection(pool, configuration); + + RedisCommand cmd_mget("MSET"); + for (size_t i = 0; i < data.size(); ++i) + cmd_mget.add(data.get(i)); + + auto ret = connection->client->execute(cmd_mget); + if (ret != "OK") + throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, "Fail to write to redis table {}, for {}", + table_id.getFullNameNotQuoted(), ret); +} + +RedisInteger StorageRedis::multiDelete(const RedisArray & keys) const +{ + auto connection = getRedisConnection(pool, configuration); + + RedisCommand cmd("DEL"); + for (size_t i = 0; i < keys.size(); ++i) + cmd.add(keys.get(i)); + + auto ret = connection->client->execute(cmd); + if (ret != static_cast(keys.size())) + LOG_DEBUG(log, "Try to delete {} rows but actually deleted {} rows from redis table {}.", + keys.size(), ret, table_id.getFullNameNotQuoted()); + + return ret; +} + Chunk StorageRedis::getByKeys( const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, @@ -382,10 +474,110 @@ Block StorageRedis::getSampleBlock(const Names &) const SinkToStoragePtr StorageRedis::write( const ASTPtr & /*query*/, - const StorageMetadataPtr & /*metadata_snapshot*/, + const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is unsupported for StorageRedis"); + return std::make_shared(*this, metadata_snapshot); +} + +/// TODO use scan to reduce latency +void StorageRedis::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) +{ + auto connection = getRedisConnection(pool, configuration); + + RedisCommand cmd_flush_db("FLUSHDB"); + cmd_flush_db << toString(configuration.db_index); + auto ret = connection->client->execute(cmd_flush_db); + + if (ret.isNull() || ret.value() != "OK") + throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, "Fail to truncate redis table {}, for {}", table_id.getFullNameNotQuoted(), ret.value()); +} + +void StorageRedis::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const +{ + if (commands.empty()) + return; + + if (commands.size() > 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mutations cannot be combined for StorageRedis"); + + const auto command_type = commands.front().type; + if (command_type != MutationCommand::Type::UPDATE && command_type != MutationCommand::Type::DELETE) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only DELETE and UPDATE mutation supported for StorageRedis"); +} + +void StorageRedis::mutate(const MutationCommands & commands, ContextPtr context_) +{ + if (commands.empty()) + return; + + assert(commands.size() == 1); + + auto metadata_snapshot = getInMemoryMetadataPtr(); + auto storage = getStorageID(); + auto storage_ptr = DatabaseCatalog::instance().getTable(storage, context_); + + if (commands.front().type == MutationCommand::Type::DELETE) + { + auto interpreter = std::make_unique( + storage_ptr, + metadata_snapshot, + commands, + context_, + /*can_execute_*/ true, + /*return_all_columns_*/ true, + /*return_mutated_rows*/ true); + auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); + PullingPipelineExecutor executor(pipeline); + + auto sink = std::make_shared(*this, metadata_snapshot); + + auto header = interpreter->getUpdatedHeader(); + auto primary_key_pos = header.getPositionByName(primary_key); + + Block block; + while (executor.pull(block)) + { + auto & column_type_name = block.getByPosition(primary_key_pos); + + auto column = column_type_name.column; + auto size = column->size(); + + RedisArray keys; + WriteBufferFromOwnString wb_key; + for (size_t i = 0; i < size; ++i) + { + wb_key.restart(); + column_type_name.type->getDefaultSerialization()->serializeBinary(*column, i, wb_key, {}); + keys.add(wb_key.str()); + } + multiDelete(keys); + } + return; + } + + assert(commands.front().type == MutationCommand::Type::UPDATE); + if (commands.front().column_to_update_expression.contains(primary_key)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key cannot be updated (cannot update column {})", primary_key); + + auto interpreter = std::make_unique( + storage_ptr, + metadata_snapshot, + commands, + context_, + /*can_execute_*/ true, + /*return_all_columns*/ true, + /*return_mutated_rows*/ true); + auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); + PullingPipelineExecutor executor(pipeline); + + auto sink = std::make_shared(*this, metadata_snapshot); + + Block block; + while (executor.pull(block)) + { + sink->consume(Chunk{block.getColumns(), block.rows()}); + } } void registerStorageRedis(StorageFactory & factory) diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h index 4a0418e6091..a4ab9a6aa4e 100644 --- a/src/Storages/StorageRedis.h +++ b/src/Storages/StorageRedis.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -37,6 +38,14 @@ public: const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + void truncate(const ASTPtr &, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr, + TableExclusiveLockHolder &) override; + + void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; + void mutate(const MutationCommands &, ContextPtr) override; + Names getPrimaryKey() const override { return {primary_key}; } /// Return chunk with data for given serialized keys. @@ -50,13 +59,16 @@ public: const RedisArray & keys, PaddedPODArray * out_null_map) const; - std::pair scan(RedisIterator iterator, const String & pattern, const uint64_t max_count); + std::pair scan(RedisIterator iterator, const String & pattern, uint64_t max_count); RedisArray multiGet(const RedisArray & keys) const; + void multiSet(const RedisArray & data) const; + RedisInteger multiDelete(const RedisArray & keys) const; Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, const Names &) const override; Block getSampleBlock(const Names &) const override; + private: StorageID table_id; RedisConfiguration configuration; diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py index 1f65a9df2f3..e77de99c649 100644 --- a/tests/integration/test_storage_redis/test.py +++ b/tests/integration/test_storage_redis/test.py @@ -1,5 +1,3 @@ -import time - ## sudo -H pip install redis import redis import pytest @@ -201,3 +199,136 @@ def test_create_table(started_cluster): """ ) + +def test_simple_insert(started_cluster): + client = get_redis_connection() + address = get_address_for_ch() + + # clean all + client.flushall() + drop_table('test_simple_insert') + + node.query( + f""" + CREATE TABLE test_simple_insert( + k UInt32, + m DateTime, + n String + ) Engine=Redis('{address}', 0, 'clickhouse') PRIMARY KEY (k) + """ + ) + + node.query( + """ + INSERT INTO test_simple_insert Values + (1, '2023-06-01 00:00:00', 'lili'), (2, '2023-06-02 00:00:00', 'lucy') + """ + ) + + response = node.query("SELECT COUNT(*) FROM test_simple_insert FORMAT Values") + assert (response == '(2)') + + response = TSV.toMat(node.query("SELECT k, m, n FROM test_simple_insert WHERE k=1 FORMAT TSV")) + assert (len(response) == 1) + assert (response[0] == ['1', '2023-06-01 00:00:00', 'lili']) + + response = TSV.toMat(node.query("SELECT k, m, n FROM test_simple_insert WHERE m='2023-06-01 00:00:00' FORMAT TSV")) + assert (len(response) == 1) + assert (response[0] == ['1', '2023-06-01 00:00:00', 'lili']) + + response = TSV.toMat(node.query("SELECT k, m, n FROM test_simple_insert WHERE n='lili' FORMAT TSV")) + assert (len(response) == 1) + assert (response[0] == ['1', '2023-06-01 00:00:00', 'lili']) + + +def test_update(started_cluster): + client = get_redis_connection() + address = get_address_for_ch() + # clean all + client.flushall() + drop_table('test_update') + + node.query( + f""" + CREATE TABLE test_update( + k UInt32, + m DateTime, + n String + ) Engine=Redis('{address}', 0, 'clickhouse') PRIMARY KEY (k) + """ + ) + + node.query( + """ + INSERT INTO test_update Values + (1, '2023-06-01 00:00:00', 'lili'), (2, '2023-06-02 00:00:00', 'lucy') + """ + ) + + response = node.query( + """ + ALTER TABLE test_update UPDATE m='2023-06-03 00:00:00' WHERE k=1 + """ + ) + + print("update response: ", response) + + response = TSV.toMat(node.query("SELECT k, m, n FROM test_update WHERE k=1 FORMAT TSV")) + assert (len(response) == 1) + assert (response[0] == ['1', '2023-06-03 00:00:00', 'lili']) + + # can not update key + with pytest.raises(QueryRuntimeException): + node.query( + """ + ALTER TABLE test_update UPDATE k=2 WHERE k=1 + """ + ) + + +def test_delete(started_cluster): + client = get_redis_connection() + address = get_address_for_ch() + + # clean all + client.flushall() + drop_table('test_delete') + + node.query( + f""" + CREATE TABLE test_delete( + k UInt32, + m DateTime, + n String + ) Engine=Redis('{address}', 0, 'clickhouse') PRIMARY KEY (k) + """ + ) + + node.query( + """ + INSERT INTO test_delete Values + (1, '2023-06-01 00:00:00', 'lili'), (2, '2023-06-02 00:00:00', 'lucy') + """ + ) + + response = node.query( + """ + ALTER TABLE test_delete DELETE WHERE k=1 + """ + ) + + print("delete response: ", response) + + response = TSV.toMat(node.query("SELECT k, m, n FROM test_delete FORMAT TSV")) + assert (len(response) == 1) + assert (response[0] == ['2', '2023-06-02 00:00:00', 'lucy']) + + response = node.query( + """ + ALTER TABLE test_delete DELETE WHERE m='2023-06-02 00:00:00' + """ + ) + + response = TSV.toMat(node.query("SELECT k, m, n FROM test_delete FORMAT TSV")) + assert (len(response) == 0) + From 010670457359862da5622baa24d50f0d5bf42557 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 31 May 2023 16:31:06 +0800 Subject: [PATCH 0986/2223] add truncate to redis storage --- src/Storages/StorageRedis.cpp | 11 +++--- tests/integration/test_storage_redis/test.py | 37 ++++++++++++++++++++ 2 files changed, 43 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index f9a25470e2d..dd33f6e6839 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -87,6 +87,7 @@ public: return storage.getBySerializedKeys(raw_keys, nullptr); } + /// TODO scan may get duplicated keys Chunk generateFullScan() { /// redis scan ending @@ -480,17 +481,16 @@ SinkToStoragePtr StorageRedis::write( return std::make_shared(*this, metadata_snapshot); } -/// TODO use scan to reduce latency void StorageRedis::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { auto connection = getRedisConnection(pool, configuration); RedisCommand cmd_flush_db("FLUSHDB"); - cmd_flush_db << toString(configuration.db_index); - auto ret = connection->client->execute(cmd_flush_db); + cmd_flush_db.add("ASYNC"); + auto ret = connection->client->execute(cmd_flush_db); - if (ret.isNull() || ret.value() != "OK") - throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, "Fail to truncate redis table {}, for {}", table_id.getFullNameNotQuoted(), ret.value()); + if (ret != "OK") + throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, "Fail to truncate redis table {}, for {}", table_id.getFullNameNotQuoted(), ret); } void StorageRedis::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const @@ -580,6 +580,7 @@ void StorageRedis::mutate(const MutationCommands & commands, ContextPtr context_ } } +/// TODO support ttl void registerStorageRedis(StorageFactory & factory) { StorageFactory::StorageFeatures features{ diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py index e77de99c649..ad1b0ada068 100644 --- a/tests/integration/test_storage_redis/test.py +++ b/tests/integration/test_storage_redis/test.py @@ -332,3 +332,40 @@ def test_delete(started_cluster): response = TSV.toMat(node.query("SELECT k, m, n FROM test_delete FORMAT TSV")) assert (len(response) == 0) + +def test_truncate(started_cluster): + client = get_redis_connection() + address = get_address_for_ch() + # clean all + client.flushall() + drop_table('test_truncate') + + node.query( + f""" + CREATE TABLE test_truncate( + k UInt32, + m DateTime, + n String + ) Engine=Redis('{address}', 0, 'clickhouse') PRIMARY KEY (k) + """ + ) + + node.query( + """ + INSERT INTO test_truncate Values + (1, '2023-06-01 00:00:00', 'lili'), (2, '2023-06-02 00:00:00', 'lucy') + """ + ) + + response = node.query( + """ + TRUNCATE TABLE test_truncate + """ + ) + + print("truncate table response: ", response) + + response = TSV.toMat(node.query("SELECT COUNT(*) FROM test_truncate FORMAT TSV")) + assert (len(response) == 1) + assert (response[0] == ['0']) + From bcf22c1ec79fad247be8c48f4bada508ef0d8063 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 31 May 2023 18:15:38 +0800 Subject: [PATCH 0987/2223] fix code style --- src/Storages/RedisCommon.cpp | 2 - src/Storages/RedisCommon.h | 2 +- src/Storages/StorageRedis.cpp | 3 +- src/TableFunctions/TableFunctionRedis.cpp | 3 - tests/integration/test_storage_redis/test.py | 119 ++++++++++-------- .../test_table_function_redis/test.py | 34 ++--- 6 files changed, 89 insertions(+), 74 deletions(-) diff --git a/src/Storages/RedisCommon.cpp b/src/Storages/RedisCommon.cpp index ba7c02fdac5..a0534a9e23b 100644 --- a/src/Storages/RedisCommon.cpp +++ b/src/Storages/RedisCommon.cpp @@ -8,10 +8,8 @@ namespace DB namespace ErrorCodes { - extern const int INVALID_REDIS_TABLE_STRUCTURE; extern const int INTERNAL_REDIS_ERROR; extern const int TIMEOUT_EXCEEDED; - extern const int BAD_ARGUMENTS; extern const int INVALID_REDIS_STORAGE_TYPE; } diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index 49c21c3277f..cf39be20ba9 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -54,7 +54,7 @@ using RedisClientPtr = std::unique_ptr; using RedisPool = BorrowedObjectPool; using RedisPoolPtr = std::shared_ptr; -/// Redis scan interator +/// Redis scan iterator using RedisIterator = int64_t; struct RedisConnection diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index dd33f6e6839..b17528c7eae 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -25,7 +25,8 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; extern const int INTERNAL_REDIS_ERROR; } diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp index 3db174fbcd8..bf147c08776 100644 --- a/src/TableFunctions/TableFunctionRedis.cpp +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -16,17 +16,14 @@ #include - namespace DB { namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } - StoragePtr TableFunctionRedis::executeImpl( const ASTPtr & /*ast_function*/, ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/) const { diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py index ad1b0ada068..66d34ebc711 100644 --- a/tests/integration/test_storage_redis/test.py +++ b/tests/integration/test_storage_redis/test.py @@ -24,17 +24,17 @@ def started_cluster(): def get_redis_connection(db_id=0): client = redis.Redis( - host='localhost', port=cluster.redis_port, password="clickhouse", db=db_id + host="localhost", port=cluster.redis_port, password="clickhouse", db=db_id ) return client def get_address_for_ch(): - return cluster.redis_host + ':6379' + return cluster.redis_host + ":6379" def drop_table(table): - node.query(f"DROP TABLE IF EXISTS {table} SYNC"); + node.query(f"DROP TABLE IF EXISTS {table} SYNC") # see SerializationString.serializeBinary @@ -50,21 +50,21 @@ def serialize_binary_for_string(x): byte = length & 0x7F if length > 0x7F: byte |= 0x80 - buf += (bytes([byte])) + buf += bytes([byte]) length >>= 7 if not length: break # write data - buf += x.encode('utf-8') + buf += x.encode("utf-8") return bytes(buf) # see SerializationNumber.serializeBinary def serialize_binary_for_uint32(x): buf = bytearray() - packed_num = struct.pack('I', x) + packed_num = struct.pack("I", x) buf += packed_num - if sys.byteorder != 'little': + if sys.byteorder != "little": buf.reverse() return bytes(buf) @@ -75,7 +75,7 @@ def test_simple_select(started_cluster): # clean all client.flushall() - drop_table('test_simple_select') + drop_table("test_simple_select") data = {} for i in range(100): @@ -95,13 +95,17 @@ def test_simple_select(started_cluster): """ ) - response = TSV.toMat(node.query("SELECT k, v FROM test_simple_select WHERE k='0' FORMAT TSV")) - assert (len(response) == 1) - assert (response[0] == ['0', '0']) + response = TSV.toMat( + node.query("SELECT k, v FROM test_simple_select WHERE k='0' FORMAT TSV") + ) + assert len(response) == 1 + assert response[0] == ["0", "0"] - response = TSV.toMat(node.query("SELECT * FROM test_simple_select ORDER BY k FORMAT TSV")) - assert (len(response) == 100) - assert (response[0] == ['0', '0']) + response = TSV.toMat( + node.query("SELECT * FROM test_simple_select ORDER BY k FORMAT TSV") + ) + assert len(response) == 100 + assert response[0] == ["0", "0"] def test_select_int(started_cluster): @@ -110,7 +114,7 @@ def test_select_int(started_cluster): # clean all client.flushall() - drop_table('test_select_int') + drop_table("test_select_int") data = {} for i in range(100): @@ -130,20 +134,25 @@ def test_select_int(started_cluster): """ ) - response = TSV.toMat(node.query("SELECT k, v FROM test_select_int WHERE k=0 FORMAT TSV")) - assert (len(response) == 1) - assert (response[0] == ['0', '0']) + response = TSV.toMat( + node.query("SELECT k, v FROM test_select_int WHERE k=0 FORMAT TSV") + ) + assert len(response) == 1 + assert response[0] == ["0", "0"] + + response = TSV.toMat( + node.query("SELECT * FROM test_select_int ORDER BY k FORMAT TSV") + ) + assert len(response) == 100 + assert response[0] == ["0", "0"] - response = TSV.toMat(node.query("SELECT * FROM test_select_int ORDER BY k FORMAT TSV")) - assert (len(response) == 100) - assert (response[0] == ['0', '0']) def test_create_table(started_cluster): address = get_address_for_ch() # simple creation - drop_table('test_create_table') + drop_table("test_create_table") node.query( f""" CREATE TABLE test_create_table( @@ -154,7 +163,7 @@ def test_create_table(started_cluster): ) # simple creation with full engine args - drop_table('test_create_table') + drop_table("test_create_table") node.query( f""" CREATE TABLE test_create_table( @@ -164,7 +173,7 @@ def test_create_table(started_cluster): """ ) - drop_table('test_create_table') + drop_table("test_create_table") node.query( f""" CREATE TABLE test_create_table( @@ -175,7 +184,7 @@ def test_create_table(started_cluster): """ ) - drop_table('test_create_table') + drop_table("test_create_table") with pytest.raises(QueryRuntimeException): node.query( f""" @@ -187,7 +196,7 @@ def test_create_table(started_cluster): """ ) - drop_table('test_create_table') + drop_table("test_create_table") with pytest.raises(QueryRuntimeException): node.query( f""" @@ -226,19 +235,27 @@ def test_simple_insert(started_cluster): ) response = node.query("SELECT COUNT(*) FROM test_simple_insert FORMAT Values") - assert (response == '(2)') + assert response == "(2)" - response = TSV.toMat(node.query("SELECT k, m, n FROM test_simple_insert WHERE k=1 FORMAT TSV")) - assert (len(response) == 1) - assert (response[0] == ['1', '2023-06-01 00:00:00', 'lili']) + response = TSV.toMat( + node.query("SELECT k, m, n FROM test_simple_insert WHERE k=1 FORMAT TSV") + ) + assert len(response) == 1 + assert response[0] == ["1", "2023-06-01 00:00:00", "lili"] - response = TSV.toMat(node.query("SELECT k, m, n FROM test_simple_insert WHERE m='2023-06-01 00:00:00' FORMAT TSV")) - assert (len(response) == 1) - assert (response[0] == ['1', '2023-06-01 00:00:00', 'lili']) + response = TSV.toMat( + node.query( + "SELECT k, m, n FROM test_simple_insert WHERE m='2023-06-01 00:00:00' FORMAT TSV" + ) + ) + assert len(response) == 1 + assert response[0] == ["1", "2023-06-01 00:00:00", "lili"] - response = TSV.toMat(node.query("SELECT k, m, n FROM test_simple_insert WHERE n='lili' FORMAT TSV")) - assert (len(response) == 1) - assert (response[0] == ['1', '2023-06-01 00:00:00', 'lili']) + response = TSV.toMat( + node.query("SELECT k, m, n FROM test_simple_insert WHERE n='lili' FORMAT TSV") + ) + assert len(response) == 1 + assert response[0] == ["1", "2023-06-01 00:00:00", "lili"] def test_update(started_cluster): @@ -246,7 +263,7 @@ def test_update(started_cluster): address = get_address_for_ch() # clean all client.flushall() - drop_table('test_update') + drop_table("test_update") node.query( f""" @@ -271,11 +288,13 @@ def test_update(started_cluster): """ ) - print("update response: ", response) + print("update response: ", response) - response = TSV.toMat(node.query("SELECT k, m, n FROM test_update WHERE k=1 FORMAT TSV")) - assert (len(response) == 1) - assert (response[0] == ['1', '2023-06-03 00:00:00', 'lili']) + response = TSV.toMat( + node.query("SELECT k, m, n FROM test_update WHERE k=1 FORMAT TSV") + ) + assert len(response) == 1 + assert response[0] == ["1", "2023-06-03 00:00:00", "lili"] # can not update key with pytest.raises(QueryRuntimeException): @@ -292,7 +311,7 @@ def test_delete(started_cluster): # clean all client.flushall() - drop_table('test_delete') + drop_table("test_delete") node.query( f""" @@ -317,11 +336,11 @@ def test_delete(started_cluster): """ ) - print("delete response: ", response) + print("delete response: ", response) response = TSV.toMat(node.query("SELECT k, m, n FROM test_delete FORMAT TSV")) - assert (len(response) == 1) - assert (response[0] == ['2', '2023-06-02 00:00:00', 'lucy']) + assert len(response) == 1 + assert response[0] == ["2", "2023-06-02 00:00:00", "lucy"] response = node.query( """ @@ -330,7 +349,7 @@ def test_delete(started_cluster): ) response = TSV.toMat(node.query("SELECT k, m, n FROM test_delete FORMAT TSV")) - assert (len(response) == 0) + assert len(response) == 0 def test_truncate(started_cluster): @@ -338,7 +357,7 @@ def test_truncate(started_cluster): address = get_address_for_ch() # clean all client.flushall() - drop_table('test_truncate') + drop_table("test_truncate") node.query( f""" @@ -363,9 +382,9 @@ def test_truncate(started_cluster): """ ) - print("truncate table response: ", response) + print("truncate table response: ", response) response = TSV.toMat(node.query("SELECT COUNT(*) FROM test_truncate FORMAT TSV")) - assert (len(response) == 1) - assert (response[0] == ['0']) + assert len(response) == 1 + assert esponse[0] == ["0"] diff --git a/tests/integration/test_table_function_redis/test.py b/tests/integration/test_table_function_redis/test.py index 111276ec6dc..772e6d28141 100644 --- a/tests/integration/test_table_function_redis/test.py +++ b/tests/integration/test_table_function_redis/test.py @@ -25,13 +25,13 @@ def started_cluster(): def get_redis_connection(db_id=0): client = redis.Redis( - host='localhost', port=cluster.redis_port, password="clickhouse", db=db_id + host="localhost", port=cluster.redis_port, password="clickhouse", db=db_id ) return client def get_address_for_ch(): - return cluster.redis_host + ':6379' + return cluster.redis_host + ":6379" # see SerializationString.serializeBinary @@ -47,21 +47,21 @@ def serialize_binary_for_string(x): byte = length & 0x7F if length > 0x7F: byte |= 0x80 - buf += (bytes([byte])) + buf += bytes([byte]) length >>= 7 if not length: break # write data - buf += x.encode('utf-8') + buf += x.encode("utf-8") return bytes(buf) # see SerializationNumber.serializeBinary def serialize_binary_for_uint32(x): buf = bytearray() - packed_num = struct.pack('I', x) + packed_num = struct.pack("I", x) buf += packed_num - if sys.byteorder != 'little': + if sys.byteorder != "little": buf.reverse() return bytes(buf) @@ -92,8 +92,8 @@ def test_simple_select(started_cluster): FORMAT TSV """)) - assert (len(response) == 1) - assert (response[0] == ['0', '0']) + assert len(response) == 1 + assert response[0] == ["0", "0"] response = TSV.toMat(node.query( f""" @@ -106,8 +106,8 @@ def test_simple_select(started_cluster): FORMAT TSV """)) - assert (len(response) == 100) - assert (response[0] == ['0', '0']) + assert len(response) == 100 + assert response[0] == ["0", "0"] def test_create_table(started_cluster): @@ -153,7 +153,7 @@ def test_data_type(started_cluster): # string client.flushall() - value = serialize_binary_for_string('0') + value = serialize_binary_for_string("0") client.set(value, value) response = TSV.toMat(node.query( @@ -167,8 +167,8 @@ def test_data_type(started_cluster): FORMAT TSV """)) - assert (len(response) == 1) - assert (response[0] == ['0', '0']) + assert len(response) == 1 + assert response[0] == ["0", "0"] # number client.flushall() @@ -186,8 +186,8 @@ def test_data_type(started_cluster): FORMAT TSV """)) - assert (len(response) == 1) - assert (response[0] == ['0', '0']) + assert len(response) == 1 + assert response[0] == ["0", "0"] # datetime client.flushall() @@ -208,5 +208,5 @@ def test_data_type(started_cluster): FORMAT TSV """)) - assert (len(response) == 1) - assert (response[0] == ['2023-06-01 00:00:00', '2023-06-01 00:00:00']) + assert len(response) == 1 + assert response[0] == ["2023-06-01 00:00:00", "2023-06-01 00:00:00"] From 7cc37ab4b877dca8dcb76e328e39c8a62995cd0b Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 31 May 2023 19:35:17 +0800 Subject: [PATCH 0988/2223] add redis table engine/function docs --- docs/en/engines/table-engines/index.md | 1 + .../table-engines/integrations/index.md | 1 + .../table-engines/integrations/redis.md | 111 ++++++++++++++++++ .../en/sql-reference/table-functions/redis.md | 67 +++++++++++ 4 files changed, 180 insertions(+) create mode 100644 docs/en/engines/table-engines/integrations/redis.md create mode 100644 docs/en/sql-reference/table-functions/redis.md diff --git a/docs/en/engines/table-engines/index.md b/docs/en/engines/table-engines/index.md index d7c582164de..bd704d0e87e 100644 --- a/docs/en/engines/table-engines/index.md +++ b/docs/en/engines/table-engines/index.md @@ -53,6 +53,7 @@ Engines in the family: - [JDBC](../../engines/table-engines/integrations/jdbc.md) - [MySQL](../../engines/table-engines/integrations/mysql.md) - [MongoDB](../../engines/table-engines/integrations/mongodb.md) +- [Redis](../../engines/table-engines/integrations/redis.md) - [HDFS](../../engines/table-engines/integrations/hdfs.md) - [S3](../../engines/table-engines/integrations/s3.md) - [Kafka](../../engines/table-engines/integrations/kafka.md) diff --git a/docs/en/engines/table-engines/integrations/index.md b/docs/en/engines/table-engines/integrations/index.md index b321a644d32..93691a8adad 100644 --- a/docs/en/engines/table-engines/integrations/index.md +++ b/docs/en/engines/table-engines/integrations/index.md @@ -14,6 +14,7 @@ List of supported integrations: - [JDBC](../../../engines/table-engines/integrations/jdbc.md) - [MySQL](../../../engines/table-engines/integrations/mysql.md) - [MongoDB](../../../engines/table-engines/integrations/mongodb.md) +- [Redis](../../../engines/table-engines/integrations/redis.md) - [HDFS](../../../engines/table-engines/integrations/hdfs.md) - [S3](../../../engines/table-engines/integrations/s3.md) - [Kafka](../../../engines/table-engines/integrations/kafka.md) diff --git a/docs/en/engines/table-engines/integrations/redis.md b/docs/en/engines/table-engines/integrations/redis.md new file mode 100644 index 00000000000..8e5a974c459 --- /dev/null +++ b/docs/en/engines/table-engines/integrations/redis.md @@ -0,0 +1,111 @@ +--- +slug: /en/sql-reference/table-functions/redis +sidebar_position: 43 +sidebar_label: Redis +--- + +# Redis + +This engine allows integrating ClickHouse with [Redis](https://redis.io/). + +## Creating a Table {#creating-a-table} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name +( + name1 [type1], + name2 [type2], + ... +) ENGINE = Redis(host:port[, db_index[, password[, pool_size]]]) PRIMARY KEY(primary_key_name); +``` + +**Engine Parameters** + +- `host:port` — Redis server address, you can ignore port and default Redis port 6379 will be used. + +- `db_index` — Redis db index range from 0 to 15, default is 0. + +- `password` — User password, default is blank string. + +- `pool_size` — Redis max connection pool size, default is 16. + +- `primary_key_name` - any column name in the column list. + +- `primary` must be specified, it supports only one column in the primary key. The primary key will be serialized in binary as a Redis key. + +- columns other than the primary key will be serialized in binary as Redis value in corresponding order. + +- queries with key equals or in filtering will be optimized to multi keys lookup from Redis. If queries without filtering key full table scan will happen which is a heavy operation. + +## Usage Example {#usage-example} + +Create a table in ClickHouse which allows to read data from Redis: + +``` sql +CREATE TABLE redis_table +( + `k` String, + `m` String, + `n` UInt32 +) +ENGINE = Redis('redis1:6379') PRIMARY KEY(k); +``` + +Insert: + +```sql +INSERT INTO redis_table Values('1', 1, '1', 1.0), ('2', 2, '2', 2.0); +``` + +Query: + +``` sql +SELECT COUNT(*) FROM redis_table; +``` + +``` text +┌─count()─┐ +│ 2 │ +└─────────┘ +``` + +``` sql +SELECT * FROM redis_table WHERE key='1'; +``` + +```text +┌─key─┬─v1─┬─v2─┬─v3─┐ +│ 1 │ 1 │ 1 │ 1 │ +└─────┴────┴────┴────┘ +``` + +``` sql +SELECT * FROM redis_table WHERE v1=2; +``` + +```text +┌─key─┬─v1─┬─v2─┬─v3─┐ +│ 2 │ 2 │ 2 │ 2 │ +└─────┴────┴────┴────┘ +``` + +Update: + +Note that the primary key cannot be updated. + +```sql +ALTER TABLE redis_table UPDATE v1=2 WHERE key='1'; +``` + +Delete: + +```sql +ALTER TABLE redis_table DELETE WHERE key='1'; +``` + +Truncate: + +Redis engine will flush db asynchronously. +```sql +TRUNCATE TABLE redis_table; +``` diff --git a/docs/en/sql-reference/table-functions/redis.md b/docs/en/sql-reference/table-functions/redis.md new file mode 100644 index 00000000000..5b32f118fb8 --- /dev/null +++ b/docs/en/sql-reference/table-functions/redis.md @@ -0,0 +1,67 @@ +--- +slug: /en/sql-reference/table-functions/redis +sidebar_position: 10 +sidebar_label: Redis +--- + +# Redis + +This table function allows integrating ClickHouse with [Redis](https://redis.io/). + +**Syntax** + +```sql +redis(host:port, key, structure[, db_index[, password[, pool_size]]]) +``` + +**Arguments** + +- `host:port` — Redis server address, you can ignore port and default Redis port 6379 will be used. + +- `key` — any column name in the column list. + +- `structure` — The schema for the ClickHouse table returned from this function. + +- `db_index` — Redis db index range from 0 to 15, default is 0. + +- `password` — User password, default is blank string. + +- `pool_size` — Redis max connection pool size, default is 16. + +- `primary` must be specified, it supports only one column in the primary key. The primary key will be serialized in binary as a Redis key. + +- columns other than the primary key will be serialized in binary as Redis value in corresponding order. + +- queries with key equals or in filtering will be optimized to multi keys lookup from Redis. If queries without filtering key full table scan will happen which is a heavy operation. + + +**Returned Value** + +A table object with key as Redis key, other columns packaged together as Redis value. + +## Usage Example {#usage-example} + +Create a table in ClickHouse which allows to read data from Redis: + +``` sql +CREATE TABLE redis_table +( + `k` String, + `m` String, + `n` UInt32 +) +ENGINE = Redis('redis1:6379') PRIMARY KEY(k); +``` + +```sql +SELECT * FROM redis( + 'redis1:6379', + 'key', + 'key String, v1 String, v2 UInt32' +) +``` + +**See Also** + +- [The `Redis` table engine](/docs/en/engines/table-engines/integrations/redis.md) +- [Using redis as a dictionary source](/docs/en/sql-reference/dictionaries/index.md#redis) From 4302ba44d46107a7e72b5f31fa948b52b0c9c40b Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 31 May 2023 20:32:34 +0800 Subject: [PATCH 0989/2223] fix code style --- tests/integration/test_storage_redis/test.py | 6 +- .../test_table_function_redis/test.py | 121 ++++++++++-------- 2 files changed, 71 insertions(+), 56 deletions(-) diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py index 66d34ebc711..2fd97b9bebd 100644 --- a/tests/integration/test_storage_redis/test.py +++ b/tests/integration/test_storage_redis/test.py @@ -147,7 +147,6 @@ def test_select_int(started_cluster): assert response[0] == ["0", "0"] - def test_create_table(started_cluster): address = get_address_for_ch() @@ -215,7 +214,7 @@ def test_simple_insert(started_cluster): # clean all client.flushall() - drop_table('test_simple_insert') + drop_table("test_simple_insert") node.query( f""" @@ -386,5 +385,4 @@ def test_truncate(started_cluster): response = TSV.toMat(node.query("SELECT COUNT(*) FROM test_truncate FORMAT TSV")) assert len(response) == 1 - assert esponse[0] == ["0"] - + assert response[0] == ["0"] diff --git a/tests/integration/test_table_function_redis/test.py b/tests/integration/test_table_function_redis/test.py index 772e6d28141..8e9dd66d9e5 100644 --- a/tests/integration/test_table_function_redis/test.py +++ b/tests/integration/test_table_function_redis/test.py @@ -81,30 +81,36 @@ def test_simple_select(started_cluster): client.mset(data) client.close() - response = TSV.toMat(node.query( - f""" - SELECT - key, value - FROM - redis('{address}', 'key', 'key String, value String', 0, 'clickhouse', 10) - WHERE - key='0' - FORMAT TSV - """)) + response = TSV.toMat( + node.query( + f""" + SELECT + key, value + FROM + redis('{address}', 'key', 'key String, value String', 0, 'clickhouse', 10) + WHERE + key='0' + FORMAT TSV + """ + ) + ) assert len(response) == 1 assert response[0] == ["0", "0"] - response = TSV.toMat(node.query( - f""" - SELECT - * - FROM - redis('{address}', 'key', 'key String, value String', 0, 'clickhouse', 10) - ORDER BY - key - FORMAT TSV - """)) + response = TSV.toMat( + node.query( + f""" + SELECT + * + FROM + redis('{address}', 'key', 'key String, value String', 0, 'clickhouse', 10) + ORDER BY + key + FORMAT TSV + """ + ) + ) assert len(response) == 100 assert response[0] == ["0", "0"] @@ -124,7 +130,8 @@ def test_create_table(started_cluster): * FROM redis('{address}', 'k', 'k String, v UInt32', 0, 'clickhouse', 10) - """) + """ + ) # illegal data type with pytest.raises(QueryRuntimeException): @@ -134,7 +141,8 @@ def test_create_table(started_cluster): * FROM redis('{address}', 'k', 'k not_exist_type, v String', 0, 'clickhouse', 10) - """) + """ + ) # illegal key with pytest.raises(QueryRuntimeException): @@ -156,16 +164,19 @@ def test_data_type(started_cluster): value = serialize_binary_for_string("0") client.set(value, value) - response = TSV.toMat(node.query( - f""" - SELECT - * - FROM - redis('{address}', 'k', 'k String, v String', 0, 'clickhouse', 10) - WHERE - k='0' - FORMAT TSV - """)) + response = TSV.toMat( + node.query( + f""" + SELECT + * + FROM + redis('{address}', 'k', 'k String, v String', 0, 'clickhouse', 10) + WHERE + k='0' + FORMAT TSV + """ + ) + ) assert len(response) == 1 assert response[0] == ["0", "0"] @@ -175,16 +186,19 @@ def test_data_type(started_cluster): value = serialize_binary_for_uint32(0) client.set(value, value) - response = TSV.toMat(node.query( - f""" - SELECT - * - FROM - redis('{address}', 'k', 'k UInt32, v UInt32', 0, 'clickhouse', 10) - WHERE - k=0 - FORMAT TSV - """)) + response = TSV.toMat( + node.query( + f""" + SELECT + * + FROM + redis('{address}', 'k', 'k UInt32, v UInt32', 0, 'clickhouse', 10) + WHERE + k=0 + FORMAT TSV + """ + ) + ) assert len(response) == 1 assert response[0] == ["0", "0"] @@ -197,16 +211,19 @@ def test_data_type(started_cluster): value = serialize_binary_for_uint32(int(seconds_since_epoch)) client.set(value, value) - response = TSV.toMat(node.query( - f""" - SELECT - * - FROM - redis('{address}', 'k', 'k DateTime, v DateTime', 0, 'clickhouse', 10) - WHERE - k='2023-06-01 00:00:00' - FORMAT TSV - """)) + response = TSV.toMat( + node.query( + f""" + SELECT + * + FROM + redis('{address}', 'k', 'k DateTime, v DateTime', 0, 'clickhouse', 10) + WHERE + k='2023-06-01 00:00:00' + FORMAT TSV + """ + ) + ) assert len(response) == 1 assert response[0] == ["2023-06-01 00:00:00", "2023-06-01 00:00:00"] From 119c2200a95563e6814042618c4f3365bf7386fb Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 31 May 2023 22:23:09 +0800 Subject: [PATCH 0990/2223] fix merged error --- src/Storages/StorageRedis.cpp | 16 ++++++++++------ .../test_table_function_redis/test.py | 3 ++- 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index b17528c7eae..973a77a5f98 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -520,14 +520,16 @@ void StorageRedis::mutate(const MutationCommands & commands, ContextPtr context_ if (commands.front().type == MutationCommand::Type::DELETE) { + MutationsInterpreter::Settings settings(true); + settings.return_all_columns = true; + settings.return_mutated_rows = true; + auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, context_, - /*can_execute_*/ true, - /*return_all_columns_*/ true, - /*return_mutated_rows*/ true); + settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); @@ -561,14 +563,16 @@ void StorageRedis::mutate(const MutationCommands & commands, ContextPtr context_ if (commands.front().column_to_update_expression.contains(primary_key)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key cannot be updated (cannot update column {})", primary_key); + MutationsInterpreter::Settings settings(true); + settings.return_all_columns = true; + settings.return_mutated_rows = true; + auto interpreter = std::make_unique( storage_ptr, metadata_snapshot, commands, context_, - /*can_execute_*/ true, - /*return_all_columns*/ true, - /*return_mutated_rows*/ true); + settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); diff --git a/tests/integration/test_table_function_redis/test.py b/tests/integration/test_table_function_redis/test.py index 8e9dd66d9e5..f4bcebe2f90 100644 --- a/tests/integration/test_table_function_redis/test.py +++ b/tests/integration/test_table_function_redis/test.py @@ -152,7 +152,8 @@ def test_create_table(started_cluster): * FROM redis('{address}', 'not_exist_key', 'k not_exist_type, v String', 0, 'clickhouse', 10) - """) + """ + ) def test_data_type(started_cluster): From e6d1b3c35141d7f20e17fd1b823df4c86b7f6c29 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 31 May 2023 23:55:41 +0800 Subject: [PATCH 0991/2223] little fix --- src/Dictionaries/RedisDictionarySource.cpp | 2 +- src/Dictionaries/RedisSource.cpp | 2 +- src/Storages/StorageRedis.cpp | 113 ++++++++------------- 3 files changed, 45 insertions(+), 72 deletions(-) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index c52c3425d1b..1736cdff306 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -41,7 +41,7 @@ namespace DB .host = host, .port = static_cast(port), .db_index = config.getUInt(redis_config_prefix + ".db_index", DEFAULT_REDIS_DB_INDEX), - .password = config.getString(redis_config_prefix + ".password", ""), + .password = config.getString(redis_config_prefix + ".password", DEFAULT_REDIS_PASSWORD), .storage_type = parseStorageType(config.getString(redis_config_prefix + ".storage_type", "")), .pool_size = config.getUInt(redis_config_prefix + ".pool_size", DEFAULT_REDIS_POOL_SIZE), }; diff --git a/src/Dictionaries/RedisSource.cpp b/src/Dictionaries/RedisSource.cpp index 5d8a475cad4..719c0278707 100644 --- a/src/Dictionaries/RedisSource.cpp +++ b/src/Dictionaries/RedisSource.cpp @@ -21,7 +21,7 @@ namespace DB } - RedisSource::RedisSource( + RedisSource::RedisSource( RedisConnectionPtr connection_, const RedisArray & keys_, const RedisStorageType & storage_type_, diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 973a77a5f98..64e5c1ad5f7 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -1,24 +1,25 @@ -#include -#include -#include -#include - #include +#include +#include +#include +#include #include +#include #include #include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include namespace DB { @@ -51,11 +52,7 @@ public: { } - RedisDataSource( - StorageRedis & storage_, - const Block & header, - const size_t max_block_size_, - const String & pattern_ = "*") + RedisDataSource(StorageRedis & storage_, const Block & header, const size_t max_block_size_, const String & pattern_ = "*") : ISource(header) , storage(storage_) , primary_key_pos(getPrimaryKeyPos(header, storage.getPrimaryKey())) @@ -85,6 +82,7 @@ public: const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey().at(0)).type; auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size); + return storage.getBySerializedKeys(raw_keys, nullptr); } @@ -109,11 +107,11 @@ public: MutableColumns columns = sample_block.cloneEmptyColumns(); RedisArray values = storage.multiGet(scan_keys); - for (size_t i = 0; i(i).isNull(); i++) + for (size_t i = 0; i < scan_keys.size() && !values.get(i).isNull(); i++) { fillColumns(scan_keys.get(i).value(), - values.get(i).value(), - primary_key_pos, sample_block, columns + values.get(i).value(), + primary_key_pos, sample_block, columns ); } @@ -143,9 +141,7 @@ private: class RedisSink : public SinkToStorage { public: - RedisSink( - StorageRedis & storage_, - const StorageMetadataPtr & metadata_snapshot_); + RedisSink(StorageRedis & storage_, const StorageMetadataPtr & metadata_snapshot_); void consume(Chunk chunk) override; String getName() const override { return "RedisSink"; } @@ -156,9 +152,7 @@ private: size_t primary_key_pos = 0; }; -RedisSink::RedisSink( - StorageRedis & storage_, - const StorageMetadataPtr & metadata_snapshot_) +RedisSink::RedisSink(StorageRedis & storage_, const StorageMetadataPtr & metadata_snapshot_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) , storage(storage_) , metadata_snapshot(metadata_snapshot_) @@ -194,6 +188,7 @@ void RedisSink::consume(Chunk chunk) data.add(wb_key.str()); data.add(wb_value.str()); } + storage.multiSet(data); } @@ -258,8 +253,8 @@ Pipe StorageRedis::read( size_t begin = num_keys * thread_idx / num_threads; size_t end = num_keys * (thread_idx + 1) / num_threads; - pipes.emplace_back(std::make_shared( - *this, header, keys, keys->begin() + begin, keys->begin() + end, max_block_size)); + pipes.emplace_back( + std::make_shared(*this, header, keys, keys->begin() + begin, keys->begin() + end, max_block_size)); } return Pipe::unitePipes(std::move(pipes)); } @@ -272,7 +267,7 @@ namespace { RedisConfiguration configuration; - if (engine_args.size() < 1) + if (engine_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad arguments count when creating Redis table engine"); if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) @@ -336,18 +331,11 @@ namespace throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageRedis must require one column in primary key"); } - return std::make_shared( - args.table_id, - configuration, - args.getContext(), - metadata, - primary_key_names[0]); + return std::make_shared(args.table_id, configuration, args.getContext(), metadata, primary_key_names[0]); } } -Chunk StorageRedis::getBySerializedKeys( - const std::vector & keys, - PaddedPODArray * null_map) const +Chunk StorageRedis::getBySerializedKeys(const std::vector & keys, PaddedPODArray * null_map) const { RedisArray redis_keys; for (const auto & key : keys) @@ -355,9 +343,7 @@ Chunk StorageRedis::getBySerializedKeys( return getBySerializedKeys(redis_keys, null_map); } -Chunk StorageRedis::getBySerializedKeys( - const RedisArray & keys, - PaddedPODArray * null_map) const +Chunk StorageRedis::getBySerializedKeys(const RedisArray & keys, PaddedPODArray * null_map) const { Block sample_block = getInMemoryMetadataPtr()->getSampleBlock(); @@ -379,8 +365,8 @@ Chunk StorageRedis::getBySerializedKeys( if (!values.get(i).isNull()) { fillColumns(keys.get(i).value(), - values.get(i).value(), - primary_key_pos, sample_block, columns + values.get(i).value(), + primary_key_pos, sample_block, columns ); } else /// key not found @@ -433,8 +419,7 @@ void StorageRedis::multiSet(const RedisArray & data) const auto ret = connection->client->execute(cmd_mget); if (ret != "OK") - throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, "Fail to write to redis table {}, for {}", - table_id.getFullNameNotQuoted(), ret); + throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, "Fail to write to redis table {}, for {}", table_id.getFullNameNotQuoted(), ret); } RedisInteger StorageRedis::multiDelete(const RedisArray & keys) const @@ -447,16 +432,17 @@ RedisInteger StorageRedis::multiDelete(const RedisArray & keys) const auto ret = connection->client->execute(cmd); if (ret != static_cast(keys.size())) - LOG_DEBUG(log, "Try to delete {} rows but actually deleted {} rows from redis table {}.", - keys.size(), ret, table_id.getFullNameNotQuoted()); + LOG_DEBUG( + log, + "Try to delete {} rows but actually deleted {} rows from redis table {}.", + keys.size(), + ret, + table_id.getFullNameNotQuoted()); return ret; } -Chunk StorageRedis::getByKeys( - const ColumnsWithTypeAndName & keys, - PaddedPODArray & null_map, - const Names &) const +Chunk StorageRedis::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray & null_map, const Names &) const { if (keys.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "StorageRedis supports only one key, got: {}", keys.size()); @@ -474,10 +460,7 @@ Block StorageRedis::getSampleBlock(const Names &) const return getInMemoryMetadataPtr()->getSampleBlock(); } -SinkToStoragePtr StorageRedis::write( - const ASTPtr & /*query*/, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr /*context*/) +SinkToStoragePtr StorageRedis::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) { return std::make_shared(*this, metadata_snapshot); } @@ -524,12 +507,7 @@ void StorageRedis::mutate(const MutationCommands & commands, ContextPtr context_ settings.return_all_columns = true; settings.return_mutated_rows = true; - auto interpreter = std::make_unique( - storage_ptr, - metadata_snapshot, - commands, - context_, - settings); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context_, settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); @@ -567,12 +545,7 @@ void StorageRedis::mutate(const MutationCommands & commands, ContextPtr context_ settings.return_all_columns = true; settings.return_mutated_rows = true; - auto interpreter = std::make_unique( - storage_ptr, - metadata_snapshot, - commands, - context_, - settings); + auto interpreter = std::make_unique(storage_ptr, metadata_snapshot, commands, context_, settings); auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute()); PullingPipelineExecutor executor(pipeline); From dc6102392785f6368c13ea59a5c0f5273425567c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 1 Jun 2023 14:36:03 +0800 Subject: [PATCH 0992/2223] add sync mode to redis storage truncate --- docs/en/engines/table-engines/integrations/redis.md | 5 +++-- src/Storages/StorageRedis.cpp | 10 ++++++++-- .../02117_show_create_table_system.reference | 6 +++--- ...ll_new_table_functions_must_be_documented.reference | 2 +- 4 files changed, 15 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/redis.md b/docs/en/engines/table-engines/integrations/redis.md index 8e5a974c459..6cfc60c836c 100644 --- a/docs/en/engines/table-engines/integrations/redis.md +++ b/docs/en/engines/table-engines/integrations/redis.md @@ -105,7 +105,8 @@ ALTER TABLE redis_table DELETE WHERE key='1'; Truncate: -Redis engine will flush db asynchronously. +Flush Redis db asynchronously. Also `Truncate` support SYNC mode. + ```sql -TRUNCATE TABLE redis_table; +TRUNCATE TABLE redis_table SYNC; ``` diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 64e5c1ad5f7..97f1dbce6da 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -465,12 +466,17 @@ SinkToStoragePtr StorageRedis::write(const ASTPtr & /*query*/, const StorageMeta return std::make_shared(*this, metadata_snapshot); } -void StorageRedis::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) +void StorageRedis::truncate(const ASTPtr & query, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { auto connection = getRedisConnection(pool, configuration); + auto * truncate_query = query->as(); + assert(truncate_query != nullptr); + RedisCommand cmd_flush_db("FLUSHDB"); - cmd_flush_db.add("ASYNC"); + if (!truncate_query->sync) + cmd_flush_db.add("ASYNC"); + auto ret = connection->client->execute(cmd_flush_db); if (ret != "OK") diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 724118f7bc1..10149bfc7bf 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -297,7 +297,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'Redis' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'REDIS' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -581,10 +581,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'REDIS' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'REDIS' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' diff --git a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference index 4f16e57d606..bc83e626207 100644 --- a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.reference @@ -9,11 +9,11 @@ jdbc meilisearch merge mongodb -redis null numbers numbers_mt odbc +redis remote remoteSecure url From ee363920f804107c6b8d96b746364cb252c6ae8e Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 1 Jun 2023 15:34:21 +0800 Subject: [PATCH 0993/2223] fix fast test --- tests/queries/0_stateless/01271_show_privileges.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 5ada21e31f4..2c98b8cc190 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -148,6 +148,7 @@ INTROSPECTION ['INTROSPECTION FUNCTIONS'] \N ALL FILE [] GLOBAL SOURCES URL [] GLOBAL SOURCES REMOTE [] GLOBAL SOURCES +MONGO [] GLOBAL SOURCES REDIS [] GLOBAL SOURCES MEILISEARCH [] GLOBAL SOURCES MYSQL [] GLOBAL SOURCES From ef6fde8264557a737fdc47a0486805f07b489abd Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 1 Jun 2023 19:28:13 +0800 Subject: [PATCH 0994/2223] fix build error for dwrwin --- src/Storages/RedisCommon.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/RedisCommon.h b/src/Storages/RedisCommon.h index cf39be20ba9..4cc358e6536 100644 --- a/src/Storages/RedisCommon.h +++ b/src/Storages/RedisCommon.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -48,7 +49,7 @@ using RedisArray = Poco::Redis::Array; using RedisArrayPtr = std::shared_ptr; using RedisBulkString = Poco::Redis::BulkString; using RedisSimpleString = String; -using RedisInteger = Int64; +using RedisInteger = Poco::Int64; using RedisClientPtr = std::unique_ptr; using RedisPool = BorrowedObjectPool; From b01db870d8335ea0bdc9f4462b42c61c389774f7 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 2 Jun 2023 10:04:16 +0800 Subject: [PATCH 0995/2223] normalize redis table function db name --- src/TableFunctions/TableFunctionRedis.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp index bf147c08776..ec659ae61e0 100644 --- a/src/TableFunctions/TableFunctionRedis.cpp +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -32,8 +32,9 @@ StoragePtr TableFunctionRedis::executeImpl( StorageInMemoryMetadata metadata; metadata.setColumns(columns); + String db_name = "redis" + getDatabaseName() + "_db_" + toString(configuration.db_index); auto storage = std::make_shared( - StorageID(toString(configuration.db_index), table_name), configuration, context, metadata, primary_key); + StorageID(db_name, table_name), configuration, context, metadata, primary_key); storage->startup(); return storage; } From 8a10baec7f73d1e40fbd018daed0f1786d95442a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 2 Jun 2023 06:25:00 +0000 Subject: [PATCH 0996/2223] Add dateTime range check --- src/Functions/FunctionsConversion.h | 38 +++++++++---------- .../01556_accurate_cast_or_null.reference | 1 + .../01556_accurate_cast_or_null.sql | 5 ++- .../0_stateless/01601_accurate_cast.sql | 4 +- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index d77090afe71..d5b5f6ae28a 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -365,6 +365,12 @@ template struct ConvertImpl +static bool CheckDateRange(const FromType & value) +{ + return value >= 0 && value <= DATE_LUT_MAX_DAY_NUM; +} + template struct ToDateTransform32Or64 { @@ -372,7 +378,7 @@ struct ToDateTransform32Or64 static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) { - return from >= 0; + return CheckDateRange(from); } static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) @@ -391,7 +397,7 @@ struct ToDateTransform32Or64Signed static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) { - return from >= 0; + return CheckDateRange(from); } static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) @@ -413,7 +419,7 @@ struct ToDateTransform8Or16Signed static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) { - return from >= 0; + return CheckDateRange(from); } static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) @@ -434,11 +440,6 @@ struct ToDate32Transform32Or64 { static constexpr auto name = "toDate32"; - static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) - { - return from >= 0; - } - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { return (from < DATE_LUT_MAX_EXTEND_DAY_NUM) @@ -452,11 +453,6 @@ struct ToDate32Transform32Or64Signed { static constexpr auto name = "toDate32"; - static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) - { - return from >= 0; - } - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { static const Int32 daynum_min_offset = -static_cast(DateLUT::instance().getDayNumOffsetEpoch()); @@ -473,11 +469,6 @@ struct ToDate32Transform8Or16Signed { static constexpr auto name = "toDate32"; - static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) - { - return from >= 0; - } - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { return from; @@ -527,6 +518,11 @@ template struct ConvertImpl struct ConvertImpl : DateTimeTransformImpl> {}; +template +static bool CheckDateTimeRange(const FromType & value) +{ + return value >= 0 && value <= 0xFFFFFFFF; +} template struct ToDateTimeTransform64 @@ -535,7 +531,7 @@ struct ToDateTimeTransform64 static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) { - return from >= 0; + return CheckDateTimeRange(from); } static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) @@ -551,7 +547,7 @@ struct ToDateTimeTransformSigned static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) { - return from >= 0; + return CheckDateTimeRange(from); } static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) @@ -569,7 +565,7 @@ struct ToDateTimeTransform64Signed static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) { - return from >= 0; + return CheckDateTimeRange(from); } static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference index cbdf72e9910..21faa830636 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference @@ -32,6 +32,7 @@ \N \N \N +\N 2023-05-30 14:38:20 1970-01-01 00:00:19 \N diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql index a9038a1d230..3f57358576e 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql @@ -37,9 +37,10 @@ SELECT accurateCastOrNull(nan, 'UInt256'); SELECT accurateCastOrNull(number + 127, 'Int8') AS x FROM numbers (2) ORDER BY x; SELECT accurateCastOrNull(-1, 'DateTime'); +SELECT accurateCastOrNull(5000000000, 'DateTime'); SELECT accurateCastOrNull('1xxx', 'DateTime'); -SELECT accurateCastOrNull('2023-05-30 14:38:20', 'DateTime'); -SELECT accurateCastOrNull(19, 'DateTime'); +select toString(accurateCastOrNull('2023-05-30 14:38:20', 'DateTime'), timezone()); +SELECT toString(accurateCastOrNull(19, 'DateTime'), 'UTC'); SELECT accurateCastOrNull(-1, 'Date'); SELECT accurateCastOrNull('1xxx', 'Date'); diff --git a/tests/queries/0_stateless/01601_accurate_cast.sql b/tests/queries/0_stateless/01601_accurate_cast.sql index 7611b1d96b9..f7f4d588ccc 100644 --- a/tests/queries/0_stateless/01601_accurate_cast.sql +++ b/tests/queries/0_stateless/01601_accurate_cast.sql @@ -24,11 +24,13 @@ SELECT accurateCast('123', 'FixedString(2)'); -- { serverError 131 } SELECT accurateCast('12', 'FixedString(2)'); SELECT accurateCast(-1, 'DateTime'); -- { serverError 70 } +SELECT accurateCast(5000000000, 'DateTime'); -- { serverError 70 } SELECT accurateCast('1xxx', 'DateTime'); -- { serverError 41 } SELECT accurateCast('2023-05-30 14:38:20', 'DateTime'); -SELECT accurateCast(19, 'DateTime'); +SELECT toString(accurateCast(19, 'DateTime'), 'UTC'); SELECT accurateCast(-1, 'Date'); -- { serverError 70 } +SELECT accurateCast(999999, 'Date'); -- { serverError 70 } SELECT accurateCast('1xxx', 'Date'); -- { serverError 38 } SELECT accurateCast('2023-05-30', 'Date'); SELECT accurateCast(19, 'Date'); From c7088a8180a245311885899e702e668719159123 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 2 Jun 2023 07:26:55 +0000 Subject: [PATCH 0997/2223] Trying to fix build --- src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index d5b5f6ae28a..6aa5843ff65 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -521,7 +521,7 @@ template struct ConvertImpl static bool CheckDateTimeRange(const FromType & value) { - return value >= 0 && value <= 0xFFFFFFFF; + return value >= 0 && value <= 0xFFFFFFFFL; } template From 16be379fa55c6dc8172004460799be93b7a52b88 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Jun 2023 11:18:46 +0200 Subject: [PATCH 0998/2223] Update src/Common/AsynchronousMetrics.cpp --- src/Common/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index a4cb18249b6..7064559800a 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -950,7 +950,7 @@ void AsynchronousMetrics::update(TimePoint update_time) auto space = line.find_first_of(" "); - if (line.rfind("max", 0) == std::string::npos) + if (line.rfind("max", space) == std::string::npos) { auto field1 = line.substr(0, space); quota = std::stoull(field1); From 57a2bfd0ff59f3ad4c5f6966c08ae7ddb6f0dda2 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Jun 2023 11:19:11 +0200 Subject: [PATCH 0999/2223] Update src/Common/AsynchronousMetrics.cpp --- src/Common/AsynchronousMetrics.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 7064559800a..cf8d451385b 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -956,8 +956,11 @@ void AsynchronousMetrics::update(TimePoint update_time) quota = std::stoull(field1); } - auto field2 = line.substr(space + 1); - period = std::stoull(field2); + if (space != std::string::npos) + { + auto field2 = line.substr(space + 1); + period = std::stoull(field2); + } new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; From 2f08b6738f307c7e04886a879632e1183b40b725 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 1 Jun 2023 18:34:00 +0200 Subject: [PATCH 1000/2223] Support parallel replicas with the analyzer --- src/Storages/StorageReplicatedMergeTree.cpp | 15 ++++-- ...02771_parallel_replicas_analyzer.reference | 12 +++++ .../02771_parallel_replicas_analyzer.sql | 52 +++++++++++++++++++ 3 files changed, 75 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference create mode 100644 tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 56896f88423..893e976d432 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -77,16 +77,17 @@ #include #include -#include #include +#include #include #include #include #include +#include #include +#include #include #include -#include #include @@ -4707,8 +4708,14 @@ void StorageReplicatedMergeTree::read( auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); - Block header = - InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + Block header; + + if (local_context->getSettingsRef().allow_experimental_analyzer) + header = InterpreterSelectQueryAnalyzer::getSampleBlock( + modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()); + else + header + = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory( diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference new file mode 100644 index 00000000000..4e93c530f7b --- /dev/null +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference @@ -0,0 +1,12 @@ +-8888150036649430454 +-2788931093724180887 +-75175454385331084 +368066018677693974 +821735343441964030 +2804162938822577320 +4357435422797280898 +5935810273536892891 +7885388429666205427 +8124171311239967992 +1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n allow_experimental_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,\n use_hedged_requests = 0; +0 2 SELECT `default`.`join_inner_table__fuzz_146_replicated`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql new file mode 100644 index 00000000000..35089c0cedb --- /dev/null +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql @@ -0,0 +1,52 @@ +-- Tags: zookeeper + +CREATE TABLE join_inner_table__fuzz_146_replicated +( + `id` UUID, + `key` String, + `number` Int64, + `value1` String, + `value2` String, + `time` Nullable(Int64) +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/join_inner_table__fuzz_146_replicated', '{replica}') +ORDER BY (id, number, key) +SETTINGS index_granularity = 8192; + +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; + +-- Simple query with analyzer and pure parallel replicas +SELECT number +FROM join_inner_table__fuzz_146_replicated + SETTINGS + allow_experimental_analyzer = 1, + max_parallel_replicas = 2, + cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', + allow_experimental_parallel_reading_from_replicas = 1, + use_hedged_requests = 0; + +SYSTEM FLUSH LOGS; +-- There should be 2 different queries +-- The initial query +-- The query sent to each replica (which should appear 2 times as we are setting max_parallel_replicas to 2) +SELECT + is_initial_query, + count() as c, query, +FROM system.query_log +WHERE + event_date >= yesterday() + AND type = 'QueryFinish' + AND initial_query_id = + ( + SELECT query_id + FROM system.query_log + WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND type = 'QueryFinish' + AND query LIKE '-- Simple query with analyzer and pure parallel replicas%' + ) +GROUP BY is_initial_query, query +ORDER BY is_initial_query DESC, c, query; From f6da66cbb87d0c7c54c9b885cb5efeb0c330f801 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 18:27:34 +0000 Subject: [PATCH 1001/2223] Fix some typos --- docs/en/sql-reference/functions/files.md | 8 +- .../functions/functions-for-nulls.md | 117 ++-- .../functions/logical-functions.md | 24 +- .../functions/other-functions.md | 549 +++++++----------- .../functions/random-functions.md | 181 ++++-- 5 files changed, 442 insertions(+), 437 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 5cd2d8e0a74..73d72aa50e5 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -6,7 +6,7 @@ sidebar_label: Files ## file -Reads file as string and loads the data into the specified column. The actual file content is not interpreted. +Reads a file as string and loads the data into the specified column. The file content is not interpreted. Also see table function [file](../table-functions/file.md). @@ -18,15 +18,13 @@ file(path[, default]) **Arguments** -- `path` — The path of the file relative to [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports the following wildcards: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` are numbers and `'abc', 'def'` are strings. -- `default` — The value that will be returned in the case the file does not exist or cannot be accessed. Supported data types: [String](../../sql-reference/data-types/string.md) and [NULL](../../sql-reference/syntax.md#null-literal). +- `path` — The path of the file relative to [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports wildcards `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` are numbers and `'abc', 'def'` are strings. +- `default` — The value returned if the file does not exist or cannot be accessed. Supported data types: [String](../../sql-reference/data-types/string.md) and [NULL](../../sql-reference/syntax.md#null-literal). **Example** Inserting data from files a.txt and b.txt into a table as strings: -Query: - ``` sql INSERT INTO table SELECT file('a.txt'), file('b.txt'); ``` diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index 0b7df54b776..f4ca27e9b16 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -8,7 +8,7 @@ sidebar_label: Nullable ## isNull -Checks whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). +Returns whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). ``` sql isNull(x) @@ -18,7 +18,7 @@ Alias: `ISNULL`. **Arguments** -- `x` — A value with a non-compound data type. +- `x` — A value of non-compound data type. **Returned value** @@ -27,7 +27,7 @@ Alias: `ISNULL`. **Example** -Input table +Table: ``` text ┌─x─┬────y─┐ @@ -36,12 +36,14 @@ Input table └───┴──────┘ ``` -Query +Query: ``` sql SELECT x FROM t_null WHERE isNull(y); ``` +Result: + ``` text ┌─x─┐ │ 1 │ @@ -50,7 +52,7 @@ SELECT x FROM t_null WHERE isNull(y); ## isNotNull -Checks whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). +Returns whether the argument is not [NULL](../../sql-reference/syntax.md#null-literal). ``` sql isNotNull(x) @@ -58,16 +60,16 @@ isNotNull(x) **Arguments:** -- `x` — A value with a non-compound data type. +- `x` — A value of non-compound data type. **Returned value** -- `0` if `x` is `NULL`. - `1` if `x` is not `NULL`. +- `0` if `x` is `NULL`. **Example** -Input table +Table: ``` text ┌─x─┬────y─┐ @@ -76,12 +78,14 @@ Input table └───┴──────┘ ``` -Query +Query: ``` sql SELECT x FROM t_null WHERE isNotNull(y); ``` +Result: + ``` text ┌─x─┐ │ 2 │ @@ -90,7 +94,7 @@ SELECT x FROM t_null WHERE isNotNull(y); ## coalesce -Checks from left to right whether `NULL` arguments were passed and returns the first non-`NULL` argument. +Returns the leftmost non-`NULL` argument. ``` sql coalesce(x,...) @@ -98,11 +102,11 @@ coalesce(x,...) **Arguments:** -- Any number of parameters of a non-compound type. All parameters must be compatible by data type. +- Any number of parameters of non-compound type. All parameters must be of mutually compatible data types. **Returned values** -- The first non-`NULL` argument. +- The first non-`NULL` argument - `NULL`, if all arguments are `NULL`. **Example** @@ -110,10 +114,10 @@ coalesce(x,...) Consider a list of contacts that may specify multiple ways to contact a customer. ``` text -┌─name─────┬─mail─┬─phone─────┬──icq─┐ -│ client 1 │ ᴺᵁᴸᴸ │ 123-45-67 │ 123 │ -│ client 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -└──────────┴──────┴───────────┴──────┘ +┌─name─────┬─mail─┬─phone─────┬──telegram─┐ +│ client 1 │ ᴺᵁᴸᴸ │ 123-45-67 │ 123 │ +│ client 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +└──────────┴──────┴───────────┴───────────┘ ``` The `mail` and `phone` fields are of type String, but the `icq` field is `UInt32`, so it needs to be converted to `String`. @@ -121,22 +125,22 @@ The `mail` and `phone` fields are of type String, but the `icq` field is `UInt32 Get the first available contact method for the customer from the contact list: ``` sql -SELECT name, coalesce(mail, phone, CAST(icq,'Nullable(String)')) FROM aBook; +SELECT name, coalesce(mail, phone, CAST(telegram,'Nullable(String)')) FROM aBook; ``` ``` text -┌─name─────┬─coalesce(mail, phone, CAST(icq, 'Nullable(String)'))─┐ -│ client 1 │ 123-45-67 │ -│ client 2 │ ᴺᵁᴸᴸ │ -└──────────┴──────────────────────────────────────────────────────┘ +┌─name─────┬─coalesce(mail, phone, CAST(telegram, 'Nullable(String)'))─┐ +│ client 1 │ 123-45-67 │ +│ client 2 │ ᴺᵁᴸᴸ │ +└──────────┴───────────────────────────────────────────────────────────┘ ``` ## ifNull -Returns an alternative value if the main argument is `NULL`. +Returns an alternative value if the argument is `NULL`. ``` sql -ifNull(x,alt) +ifNull(x, alt) ``` **Arguments:** @@ -146,25 +150,33 @@ ifNull(x,alt) **Returned values** -- The value `x`, if `x` is not `NULL`. -- The value `alt`, if `x` is `NULL`. +- `x` if `x` is not `NULL`. +- `alt` if `x` is `NULL`. **Example** +Query: + ``` sql SELECT ifNull('a', 'b'); ``` +Result: + ``` text ┌─ifNull('a', 'b')─┐ │ a │ └──────────────────┘ ``` +Query: + ``` sql SELECT ifNull(NULL, 'b'); ``` +Result: + ``` text ┌─ifNull(NULL, 'b')─┐ │ b │ @@ -173,7 +185,7 @@ SELECT ifNull(NULL, 'b'); ## nullIf -Returns `NULL` if the arguments are equal. +Returns `NULL` if both arguments are equal. ``` sql nullIf(x, y) @@ -181,29 +193,37 @@ nullIf(x, y) **Arguments:** -`x`, `y` — Values for comparison. They must be compatible types, or ClickHouse will generate an exception. +`x`, `y` — Values to compare. Must be of compatible types. **Returned values** -- `NULL`, if the arguments are equal. -- The `x` value, if the arguments are not equal. +- `NULL` if the arguments are equal. +- `x` if the arguments are not equal. **Example** +Query: + ``` sql SELECT nullIf(1, 1); ``` +Result: + ``` text ┌─nullIf(1, 1)─┐ │ ᴺᵁᴸᴸ │ └──────────────┘ ``` +Query: + ``` sql SELECT nullIf(1, 2); ``` +Result: + ``` text ┌─nullIf(1, 2)─┐ │ 1 │ @@ -212,7 +232,7 @@ SELECT nullIf(1, 2); ## assumeNotNull -Results in an equivalent non-`Nullable` value for a [Nullable](../../sql-reference/data-types/nullable.md) type. In case the original value is `NULL` the result is undetermined. See also `ifNull` and `coalesce` functions. +Returns the corresponding non-`Nullable` value for a value of [Nullable](../../sql-reference/data-types/nullable.md) type. If the original value is `NULL`, an arbitrary result can be returned. See also functions `ifNull` and `coalesce`. ``` sql assumeNotNull(x) @@ -224,36 +244,29 @@ assumeNotNull(x) **Returned values** -- The original value from the non-`Nullable` type, if it is not `NULL`. -- Implementation specific result if the original value was `NULL`. +- The input value as non-`Nullable` type, if it is not `NULL`. +- An arbirary value, if the input value is `NULL`. **Example** -Consider the `t_null` table. - -``` sql -SHOW CREATE TABLE t_null; -``` +Table: ``` text -┌─statement─────────────────────────────────────────────────────────────────┐ -│ CREATE TABLE default.t_null ( x Int8, y Nullable(Int8)) ENGINE = TinyLog │ -└───────────────────────────────────────────────────────────────────────────┘ -``` -``` text ┌─x─┬────y─┐ │ 1 │ ᴺᵁᴸᴸ │ │ 2 │ 3 │ └───┴──────┘ ``` -Apply the `assumeNotNull` function to the `y` column. +Query: ``` sql -SELECT assumeNotNull(y) FROM t_null; +SELECT assumeNotNull(y) FROM table; ``` +Result: + ``` text ┌─assumeNotNull(y)─┐ │ 0 │ @@ -261,10 +274,14 @@ SELECT assumeNotNull(y) FROM t_null; └──────────────────┘ ``` +Query: + ``` sql SELECT toTypeName(assumeNotNull(y)) FROM t_null; ``` +Result: + ``` text ┌─toTypeName(assumeNotNull(y))─┐ │ Int8 │ @@ -282,28 +299,36 @@ toNullable(x) **Arguments:** -- `x` — The value of any non-compound type. +- `x` — A value of non-compound type. **Returned value** -- The input value with a `Nullable` type. +- The input value but of `Nullable` type. **Example** +Query: + ``` sql SELECT toTypeName(10); ``` +Result: + ``` text ┌─toTypeName(10)─┐ │ UInt8 │ └────────────────┘ ``` +Query: + ``` sql SELECT toTypeName(toNullable(10)); ``` +Result: + ``` text ┌─toTypeName(toNullable(10))─┐ │ Nullable(UInt8) │ diff --git a/docs/en/sql-reference/functions/logical-functions.md b/docs/en/sql-reference/functions/logical-functions.md index f5a1a6aac12..2e1a8f28227 100644 --- a/docs/en/sql-reference/functions/logical-functions.md +++ b/docs/en/sql-reference/functions/logical-functions.md @@ -12,7 +12,9 @@ Zero as an argument is considered `false`, non-zero values are considered `true` ## and -Calculates the logical conjunction between two or more values. +Calculates the logical conjunction of two or more values. + +Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `(val_1 AND val_2 AND ... AND val_{i-1})` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT and(number = 2, intDiv(1, number)) FROM numbers(5)`. **Syntax** @@ -20,9 +22,7 @@ Calculates the logical conjunction between two or more values. and(val1, val2...) ``` -Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `(val_1 AND val_2 AND ... AND val_{i-1})` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT and(number = 2, intDiv(1, number)) FROM numbers(5)`. - -Alias: The [AND Operator](../../sql-reference/operators/index.md#logical-and-operator). +Alias: The [AND operator](../../sql-reference/operators/index.md#logical-and-operator). **Arguments** @@ -30,7 +30,7 @@ Alias: The [AND Operator](../../sql-reference/operators/index.md#logical-and-ope **Returned value** -- `0`, if there at least one argument evaluates to `false`, +- `0`, if at least one argument evaluates to `false`, - `NULL`, if no argumetn evaluates to `false` and at least one argument is `NULL`, - `1`, otherwise. @@ -66,7 +66,9 @@ Result: ## or -Calculates the logical disjunction between two or more values. +Calculates the logical disjunction of two or more values. + +Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `((NOT val_1) AND (NOT val_2) AND ... AND (NOT val_{i-1}))` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT or(number = 0, intDiv(1, number) != 0) FROM numbers(5)`. **Syntax** @@ -74,9 +76,7 @@ Calculates the logical disjunction between two or more values. or(val1, val2...) ``` -Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `((NOT val_1) AND (NOT val_2) AND ... AND (NOT val_{i-1}))` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT or(number = 0, intDiv(1, number) != 0) FROM numbers(5)`. - -Alias: The [OR Operator](../../sql-reference/operators/index.md#logical-or-operator). +Alias: The [OR operator](../../sql-reference/operators/index.md#logical-or-operator). **Arguments** @@ -120,7 +120,7 @@ Result: ## not -Calculates logical negation of a value. +Calculates the logical negation of a value. **Syntax** @@ -128,7 +128,7 @@ Calculates logical negation of a value. not(val); ``` -Alias: The [Negation Operator](../../sql-reference/operators/index.md#logical-negation-operator). +Alias: The [Negation operator](../../sql-reference/operators/index.md#logical-negation-operator). **Arguments** @@ -158,7 +158,7 @@ Result: ## xor -Calculates the logical exclusive disjunction between two or more values. For more than two values the function first xor-s the first two values, then xor-s the result with the third value etc. +Calculates the logical exclusive disjunction of two or more values. For more than two input values, the function first xor-s the first two values, then xor-s the result with the third value etc. **Syntax** diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index efe1a77c285..037eb9e63c5 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -8,12 +8,12 @@ sidebar_label: Other ## hostName() -Returns a string with the name of the host that this function was performed on. For distributed processing, this is the name of the remote server host, if the function is performed on a remote server. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +Returns the name of the host on which this function was executed. If the function executes on a remote server (distributed processing), the remote server name is returned. +If the function executes in the context of a distributed table, it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## getMacro -Gets a named value from the [macros](../../operations/server-configuration-parameters/settings.md#macros) section of the server configuration. +Returns a named value from the [macros](../../operations/server-configuration-parameters/settings.md#macros) section of the server configuration. **Syntax** @@ -23,7 +23,7 @@ getMacro(name); **Arguments** -- `name` — Name to retrieve from the `macros` section. [String](../../sql-reference/data-types/string.md#string). +- `name` — Macro name to retrieve from the `` section. [String](../../sql-reference/data-types/string.md#string). **Returned value** @@ -33,7 +33,7 @@ Type: [String](../../sql-reference/data-types/string.md). **Example** -The example `macros` section in the server configuration file: +Example `` section in the server configuration file: ``` xml @@ -55,7 +55,7 @@ Result: └──────────────────┘ ``` -An alternative way to get the same value: +The same value can be retrieved as follows: ``` sql SELECT * FROM system.macros @@ -70,7 +70,7 @@ WHERE macro = 'test'; ## FQDN -Returns the fully qualified domain name. +Returns the fully qualified domain name of the ClickHouse server. **Syntax** @@ -88,8 +88,6 @@ Type: `String`. **Example** -Query: - ``` sql SELECT FQDN(); ``` @@ -104,52 +102,61 @@ Result: ## basename -Extracts the trailing part of a string after the last slash or backslash. This function if often used to extract the filename from a path. +Extracts the tail of a string following its last slash or backslash. This function if often used to extract the filename from a path. ``` sql -basename( expr ) +basename(expr) ``` **Arguments** -- `expr` — Expression resulting in a [String](../../sql-reference/data-types/string.md) type value. All the backslashes must be escaped in the resulting value. +- `expr` — A value of type [String](../../sql-reference/data-types/string.md). Backslashes must be escaped. **Returned Value** A string that contains: -- The trailing part of a string after the last slash or backslash. - - If the input string contains a path ending with slash or backslash, for example, `/` or `c:\`, the function returns an empty string. - +- The tail of the input string after its last slash or backslash. If the input string ends with a slash or backslash (e.g. `/` or `c:\`), the function returns an empty string. - The original string if there are no slashes or backslashes. **Example** +Query: + ``` sql SELECT 'some/long/path/to/file' AS a, basename(a) ``` +Result: + ``` text ┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ │ some\long\path\to\file │ file │ └────────────────────────┴────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT 'some\\long\\path\\to\\file' AS a, basename(a) ``` +Result: + ``` text ┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ │ some\long\path\to\file │ file │ └────────────────────────┴────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT 'some-file-name' AS a, basename(a) ``` +Result: + ``` text ┌─a──────────────┬─basename('some-file-name')─┐ │ some-file-name │ some-file-name │ @@ -159,7 +166,7 @@ SELECT 'some-file-name' AS a, basename(a) ## visibleWidth(x) Calculates the approximate width when outputting values to the console in text format (tab-separated). -This function is used by the system for implementing Pretty formats. +This function is used by the system to implement Pretty formats. `NULL` is represented as a string corresponding to `NULL` in `Pretty` formats. @@ -175,18 +182,18 @@ SELECT visibleWidth(NULL) ## toTypeName(x) -Returns a string containing the type name of the passed argument. +Returns the type name of the passed argument. -If `NULL` is passed to the function as input, then it returns the `Nullable(Nothing)` type, which corresponds to an internal `NULL` representation in ClickHouse. +If `NULL` is passed, then the function returns type `Nullable(Nothing)`, which corresponds to ClickHouse's internal `NULL` representation. ## blockSize() -Gets the size of the block. -In ClickHouse, queries are always run on blocks (sets of column parts). This function allows getting the size of the block that you called it for. +In ClickHouse, queries are processed in blocks (chunks). +This function returns the size (row count) of the block the function is called on. ## byteSize -Returns estimation of uncompressed byte size of its arguments in memory. +Returns an estimation of uncompressed byte size of its arguments in memory. **Syntax** @@ -206,7 +213,7 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** -For [String](../../sql-reference/data-types/string.md) arguments the funtion returns the string length + 9 (terminating zero + length). +For [String](../../sql-reference/data-types/string.md) arguments, the funtion returns the string length + 9 (terminating zero + length). Query: @@ -265,7 +272,7 @@ byteSize(Float32): 4 byteSize(Float64): 8 ``` -If the function takes multiple arguments, it returns their combined byte size. +If the function has multiple arguments, the function accumulates their byte sizes. Query: @@ -283,30 +290,30 @@ Result: ## materialize(x) -Turns a constant into a full column containing just one value. -In ClickHouse, full columns and constants are represented differently in memory. Functions work differently for constant arguments and normal arguments (different code is executed), although the result is almost always the same. This function is for debugging this behavior. +Turns a constant into a full column containing a single value. +Full columns and constants are represented differently in memory. Functions usually execute different code for normal and constant arguments, although the result should typically be the same. This function can be used to debug this behavior. ## ignore(…) -Accepts any arguments, including `NULL`. Always returns 0. -However, the argument is still evaluated. This can be used for benchmarks. +Accepts any arguments, including `NULL` and does nothing. Always returns 0. +The argument is internally still evaluated. Useful e.g. for benchmarks. ## sleep(seconds) -Sleeps ‘seconds’ seconds on each data block. You can specify an integer or a floating-point number. +Sleeps ‘seconds’ seconds for each data block. The sleep time can be specified as integer or as floating-point number. ## sleepEachRow(seconds) -Sleeps ‘seconds’ seconds on each row. You can specify an integer or a floating-point number. +Sleeps ‘seconds’ seconds for each row. The sleep time can be specified as integer or as floating-point number. ## currentDatabase() Returns the name of the current database. -You can use this function in table engine parameters in a CREATE TABLE query where you need to specify the database. +Useful in table engine parameters of `CREATE TABLE` queries where you need to specify the database. ## currentUser() -Returns the login of current user. Login of user, that initiated query, will be returned in case distributed query. +Returns the name of the current user. In case of a distributed query, the name of the user who initiated the query is returned. ``` sql SELECT currentUser(); @@ -316,15 +323,13 @@ Alias: `user()`, `USER()`. **Returned values** -- Login of current user. -- Login of user that initiated query in case of distributed query. +- The name of the current user. +- In distributed queries, the login of the user who initiated the query. Type: `String`. **Example** -Query: - ``` sql SELECT currentUser(); ``` @@ -339,11 +344,11 @@ Result: ## isConstant -Checks whether the argument is a constant expression. +Returns whether the argument is a constant expression. -A constant expression means an expression whose resulting value is known at the query analysis (i.e. before execution). For example, expressions over [literals](../../sql-reference/syntax.md#literals) are constant expressions. +A constant expression is an expression whose result is known during query analysis, i.e. before execution. For example, expressions over [literals](../../sql-reference/syntax.md#literals) are constant expressions. -The function is intended for development, debugging and demonstration. +This function is mostly intended for development, debugging and demonstration. **Syntax** @@ -357,8 +362,8 @@ isConstant(x) **Returned values** -- `1` — `x` is constant. -- `0` — `x` is non-constant. +- `1` if `x` is constant. +- `0` if `x` is non-constant. Type: [UInt8](../../sql-reference/data-types/int-uint.md). @@ -408,23 +413,25 @@ Result: ## isFinite(x) -Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is not infinite and not a NaN, otherwise 0. +Returns 1 if the Float32 or Float64 argument not infinite and not a NaN, otherwise this function returns 0. ## isInfinite(x) -Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is infinite, otherwise 0. Note that 0 is returned for a NaN. +Returns 1 if the Float32 or Float64 argument is infinite, otherwise this function returns 0. Note that 0 is returned for a NaN. ## ifNotFinite -Checks whether floating point value is finite. +Checks whether a floating point value is finite. **Syntax** - ifNotFinite(x,y) +``` sql +ifNotFinite(x,y) +``` **Arguments** -- `x` — Value to be checked for infinity. Type: [Float\*](../../sql-reference/data-types/float.md). +- `x` — Value to check for infinity. Type: [Float\*](../../sql-reference/data-types/float.md). - `y` — Fallback value. Type: [Float\*](../../sql-reference/data-types/float.md). **Returned value** @@ -444,23 +451,23 @@ Result: │ inf │ 42 │ └─────────┴───────────────────────────────┘ -You can get similar result by using [ternary operator](../../sql-reference/functions/conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. +You can get similar result by using the [ternary operator](../../sql-reference/functions/conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. ## isNaN(x) -Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is a NaN, otherwise 0. +Returns 1 if the Float32 and Float64 argument is NaN, otherwise this function 0. ## hasColumnInTable(\[‘hostname’\[, ‘username’\[, ‘password’\]\],\] ‘database’, ‘table’, ‘column’) -Accepts constant strings: database name, table name, and column name. Returns a UInt8 constant expression equal to 1 if there is a column, otherwise 0. If the hostname parameter is set, the test will run on a remote server. -The function throws an exception if the table does not exist. +Given the database name, the table name, and the column name as constant strings, returns 1 if the given column exists, otherwise 0. If parameter `hostname` is given, the check is performed on a remote server. +If the table does not exist, an exception is thrown. For elements in a nested data structure, the function checks for the existence of a column. For the nested data structure itself, the function returns 0. ## bar -Allows building a unicode-art diagram. +Builds a bar chart. -`bar(x, min, max, width)` draws a band with a width proportional to `(x - min)` and equal to `width` characters when `x = max`. +`bar(x, min, max, width)` draws a band with width proportional to `(x - min)` and equal to `width` characters when `x = max`. **Arguments** @@ -520,23 +527,23 @@ There are two variations of this function: `x` – What to transform. -`array_from` – Constant array of values for converting. +`array_from` – Constant array of values to convert. `array_to` – Constant array of values to convert the values in ‘from’ to. `default` – Which value to use if ‘x’ is not equal to any of the values in ‘from’. -`array_from` and `array_to` – Arrays of the same size. +`array_from` and `array_to` must have equally many elements. -Types: +Signature: + +For `x` equal to one of the elements in `array_from`, the function returns the corresponding element in `array_to`, i.e. the one at the same array index. Otherwise, it returns `default`. If multiple matching elements exist `array_from`, an arbitrary corresponding element from `array_to` is returned. `transform(T, Array(T), Array(U), U) -> U` `T` and `U` can be numeric, string, or Date or DateTime types. -Where the same letter is indicated (T or U), for numeric types these might not be matching types, but types that have a common type. -For example, the first argument can have the Int64 type, while the second has the Array(UInt16) type. - -If the ‘x’ value is equal to one of the elements in the ‘array_from’ array, it returns the existing element (that is numbered the same) from the ‘array_to’ array. Otherwise, it returns ‘default’. If there are multiple matching elements in ‘array_from’, it returns one of the matches. +The same letter (T or U) means that types must be mutually compatible and not necessarily equal. +For example, the first argument could have type `Int64`, while the second argument could have type `Array(UInt16)`. Example: @@ -560,12 +567,7 @@ ORDER BY c DESC ### transform(x, array_from, array_to) -Differs from the first variation in that the ‘default’ argument is omitted. -If the ‘x’ value is equal to one of the elements in the ‘array_from’ array, it returns the matching element (that is numbered the same) from the ‘array_to’ array. Otherwise, it returns ‘x’. - -Types: - -`transform(T, Array(T), Array(T)) -> T` +Similar to the other variation but has no ‘default’ argument. In case no match can be found, `x` is returned. Example: @@ -595,7 +597,7 @@ LIMIT 10 ## formatReadableDecimalSize(x) -Accepts the size (number of bytes). Returns a rounded size with a suffix (KB, MB, etc.) as a string. +Given a size (number of bytes), this function returns a readable, rounded size with suffix (KB, MB, etc.) as string. Example: @@ -616,7 +618,7 @@ SELECT ## formatReadableSize(x) -Accepts the size (number of bytes). Returns a rounded size with a suffix (KiB, MiB, etc.) as a string. +Given a size (number of bytes), this function returns a readable, rounded size with suffix (KiB, MiB, etc.) as string. Example: @@ -637,9 +639,7 @@ SELECT ## formatReadableQuantity(x) -Accepts the number. Returns a rounded number with a suffix (thousand, million, billion, etc.) as a string. - -It is useful for reading big numbers by human. +Given a number, this function returns a rounded number with suffix (thousand, million, billion, etc.) as string. Example: @@ -660,7 +660,7 @@ SELECT ## formatReadableTimeDelta -Accepts the time delta in seconds. Returns a time delta with (year, month, day, hour, minute, second) as a string. +Given a time interval (delta) in seconds, this function returns a time delta with year/month/day/hour/minute/second as string. **Syntax** @@ -670,8 +670,8 @@ formatReadableTimeDelta(column[, maximum_unit]) **Arguments** -- `column` — A column with numeric time delta. -- `maximum_unit` — Optional. Maximum unit to show. Acceptable values seconds, minutes, hours, days, months, years. +- `column` — A column with a numeric time delta. +- `maximum_unit` — Optional. Maximum unit to show. Acceptable values `seconds`, `minutes`, `hours`, `days`, `months`, `years`. Example: @@ -746,33 +746,32 @@ SELECT parseTimeDelta('1yr2mo') ## least(a, b) -Returns the smallest value from a and b. +Returns the smaller value of a and b. ## greatest(a, b) -Returns the largest value of a and b. +Returns the larger value of a and b. ## uptime() Returns the server’s uptime in seconds. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## version() -Returns the version of the server as a string. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +Returns the server version as a string. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## buildId() Returns the build ID generated by a compiler for the running ClickHouse server binary. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. - -## blockNumber +## blockNumber() Returns the sequence number of the data block where the row is located. -## rowNumberInBlock +## rowNumberInBlock() Returns the ordinal number of the row in the data block. Different data blocks are always recalculated. @@ -782,7 +781,7 @@ Returns the ordinal number of the row in the data block. This function only cons ## neighbor -The window function that provides access to a row at a specified offset which comes before or after the current row of a given column. +The window function that provides access to a row at a specified offset before or after the current row of a given column. **Syntax** @@ -792,23 +791,23 @@ neighbor(column, offset[, default_value]) The result of the function depends on the affected data blocks and the order of data in the block. -:::tip -It can reach the neighbor rows only inside the currently processed data block. +:::note +Only returns neighbor inside the currently processed data block. ::: -The rows order used during the calculation of `neighbor` can differ from the order of rows returned to the user. -To prevent that you can make a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. +The order of rows during calculation of `neighbor()` can differ from the order of rows returned to the user. +To prevent that you can create a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. **Arguments** - `column` — A column name or scalar expression. -- `offset` — The number of rows forwards or backwards from the current row of `column`. [Int64](../../sql-reference/data-types/int-uint.md). -- `default_value` — Optional. The value to be returned if offset goes beyond the scope of the block. Type of data blocks affected. +- `offset` — The number of rows to look before or ahead of the current row in `column`. [Int64](../../sql-reference/data-types/int-uint.md). +- `default_value` — Optional. The returned value if offset is beyond the block boundaries. Type of data blocks affected. **Returned values** -- Value for `column` in `offset` distance from current row if `offset` value is not outside block bounds. -- Default value for `column` if `offset` value is outside block bounds. If `default_value` is given, then it will be used. +- Value of `column` with `offset` distance from current row, if `offset` is not outside the block boundaries. +- The default value of `column` or `default_value` (if given), if `offset` is outside the block boundaries. Type: type of data blocks affected or default value type. @@ -899,17 +898,17 @@ Result: ## runningDifference(x) -Calculates the difference between successive row values ​​in the data block. -Returns 0 for the first row and the difference from the previous row for each subsequent row. +Calculates the difference between two consecutive row values in the data block. +Returns 0 for the first row, and for subsequent rows the difference to the previous row. -:::tip -It can reach the previous row only inside the currently processed data block. +:::note +Only returns differences inside the currently processed data block. ::: The result of the function depends on the affected data blocks and the order of data in the block. -The rows order used during the calculation of `runningDifference` can differ from the order of rows returned to the user. -To prevent that you can make a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. +The order of rows during calculation of `runningDifference()` can differ from the order of rows returned to the user. +To prevent that you can create a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. Example: @@ -940,7 +939,7 @@ FROM └─────────┴─────────────────────┴───────┘ ``` -Please note - block size affects the result. With each new block, the `runningDifference` state is reset. +Please note that the block size affects the result. The internal state of `runningDifference` state is reset for each new block. ``` sql SELECT @@ -977,7 +976,7 @@ WHERE diff != 1 ## runningDifferenceStartingWithFirstValue -Same as for [runningDifference](./other-functions.md#other_functions-runningdifference), the difference is the value of the first row, returned the value of the first row, and each subsequent row returns the difference from the previous row. +Same as [runningDifference](./other-functions.md#other_functions-runningdifference), but returns the value of the first row as the value on the first row. ## runningConcurrency @@ -1039,7 +1038,7 @@ Result: ## MACNumToString(num) -Accepts a UInt64 number. Interprets it as a MAC address in big endian. Returns a string containing the corresponding MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form). +Interprets a UInt64 number as a MAC address in big endian format. Returns the corresponding MAC address in format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form) as string. ## MACStringToNum(s) @@ -1047,11 +1046,12 @@ The inverse function of MACNumToString. If the MAC address has an invalid format ## MACStringToOUI(s) -Accepts a MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form). Returns the first three octets as a UInt64 number. If the MAC address has an invalid format, it returns 0. +Given a MAC address in format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form), returns the first three octets as a UInt64 number. If the MAC address has an invalid format, it returns 0. ## getSizeOfEnumType Returns the number of fields in [Enum](../../sql-reference/data-types/enum.md). +An exception is thrown if the type is not `Enum`. ``` sql getSizeOfEnumType(value) @@ -1064,7 +1064,6 @@ getSizeOfEnumType(value) **Returned values** - The number of fields with `Enum` input values. -- An exception is thrown if the type is not `Enum`. **Example** @@ -1080,7 +1079,7 @@ SELECT getSizeOfEnumType( CAST('a' AS Enum8('a' = 1, 'b' = 2) ) ) AS x ## blockSerializedSize -Returns size on disk (without taking into account compression). +Returns the size on disk without considering compression. ``` sql blockSerializedSize(value[, value[, ...]]) @@ -1092,7 +1091,7 @@ blockSerializedSize(value[, value[, ...]]) **Returned values** -- The number of bytes that will be written to disk for block of values (without compression). +- The number of bytes that will be written to disk for block of values without compression. **Example** @@ -1112,7 +1111,7 @@ Result: ## toColumnTypeName -Returns the name of the class that represents the data type of the column in RAM. +Returns the internal name of the data type that represents the value. ``` sql toColumnTypeName(value) @@ -1124,31 +1123,39 @@ toColumnTypeName(value) **Returned values** -- A string with the name of the class that is used for representing the `value` data type in RAM. +- The internal data type name used to represent `value`. -**Example of the difference between`toTypeName ' and ' toColumnTypeName`** +**Example** + +Difference between `toTypeName ' and ' toColumnTypeName`: ``` sql SELECT toTypeName(CAST('2018-01-01 01:02:03' AS DateTime)) ``` +Result: + ``` text ┌─toTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ DateTime │ └─────────────────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT toColumnTypeName(CAST('2018-01-01 01:02:03' AS DateTime)) ``` +Result: + ``` text ┌─toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ Const(UInt32) │ └───────────────────────────────────────────────────────────┘ ``` -The example shows that the `DateTime` data type is stored in memory as `Const(UInt32)`. +The example shows that the `DateTime` data type is internally stored as `Const(UInt32)`. ## dumpColumnStructure @@ -1164,7 +1171,7 @@ dumpColumnStructure(value) **Returned values** -- A string describing the structure that is used for representing the `value` data type in RAM. +- A description of the column structure used for representing `value`. **Example** @@ -1180,7 +1187,7 @@ SELECT dumpColumnStructure(CAST('2018-01-01 01:02:03', 'DateTime')) ## defaultValueOfArgumentType -Outputs the default value for the data type. +Returns the default value for the given data type. Does not include default values for custom columns set by the user. @@ -1200,20 +1207,28 @@ defaultValueOfArgumentType(expression) **Example** +Query: + ``` sql SELECT defaultValueOfArgumentType( CAST(1 AS Int8) ) ``` +Result: + ``` text ┌─defaultValueOfArgumentType(CAST(1, 'Int8'))─┐ │ 0 │ └─────────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT defaultValueOfArgumentType( CAST(1 AS Nullable(Int8) ) ) ``` +Result: + ``` text ┌─defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)'))─┐ │ ᴺᵁᴸᴸ │ @@ -1222,7 +1237,7 @@ SELECT defaultValueOfArgumentType( CAST(1 AS Nullable(Int8) ) ) ## defaultValueOfTypeName -Outputs the default value for given type name. +Returns the default value for the given type name. Does not include default values for custom columns set by the user. @@ -1242,20 +1257,28 @@ defaultValueOfTypeName(type) **Example** +Query: + ``` sql SELECT defaultValueOfTypeName('Int8') ``` +Result: + ``` text ┌─defaultValueOfTypeName('Int8')─┐ │ 0 │ └────────────────────────────────┘ ``` +Query: + ``` sql SELECT defaultValueOfTypeName('Nullable(Int8)') ``` +Result: + ``` text ┌─defaultValueOfTypeName('Nullable(Int8)')─┐ │ ᴺᵁᴸᴸ │ @@ -1263,9 +1286,10 @@ SELECT defaultValueOfTypeName('Nullable(Int8)') ``` ## indexHint -The function is intended for debugging and introspection purposes. The function ignores it's argument and always returns 1. Arguments are not even evaluated. -But for the purpose of index analysis, the argument of this function is analyzed as if it was present directly without being wrapped inside `indexHint` function. This allows to select data in index ranges by the corresponding condition but without further filtering by this condition. The index in ClickHouse is sparse and using `indexHint` will yield more data than specifying the same condition directly. +This function is intended for debugging and introspection. It ignores its argument and always returns 1. The arguments are not evaluated. + +But during index analysis, the argument of this function is assumed to be not wrapped in `indexHint`. This allows to select data in index ranges by the corresponding condition but without further filtering by this condition. The index in ClickHouse is sparse and using `indexHint` will yield more data than specifying the same condition directly. **Syntax** @@ -1275,13 +1299,13 @@ SELECT * FROM table WHERE indexHint() **Returned value** -1. Type: [Uint8](https://clickhouse.com/docs/en/data_types/int_uint/#diapazony-uint). +Type: [Uint8](https://clickhouse.com/docs/en/data_types/int_uint/#diapazony-uint). **Example** Here is the example of test data from the table [ontime](../../getting-started/example-datasets/ontime.md). -Input table: +Table: ```sql SELECT count() FROM ontime @@ -1295,9 +1319,7 @@ SELECT count() FROM ontime The table has indexes on the fields `(FlightDate, (Year, FlightDate))`. -Create a query, where the index is not used. - -Query: +Create a query which does not use the index: ```sql SELECT FlightDate AS k, count() FROM ontime GROUP BY k ORDER BY k @@ -1318,15 +1340,13 @@ Result: └────────────┴─────────┘ ``` -To apply the index, select a specific date. - -Query: +To apply the index, select a specific date: ```sql SELECT FlightDate AS k, count() FROM ontime WHERE k = '2017-09-15' GROUP BY k ORDER BY k ``` -By using the index, ClickHouse processed a significantly smaller number of rows (`Processed 32.74 thousand rows`). +ClickHouse now uses the index to process a significantly smaller number of rows (`Processed 32.74 thousand rows`). Result: @@ -1336,7 +1356,7 @@ Result: └────────────┴─────────┘ ``` -Now wrap the expression `k = '2017-09-15'` into `indexHint` function. +Now wrap the expression `k = '2017-09-15'` in function `indexHint`: Query: @@ -1350,9 +1370,9 @@ GROUP BY k ORDER BY k ASC ``` -ClickHouse used the index in the same way as the previous time (`Processed 32.74 thousand rows`). +ClickHouse used the index the same way as previously (`Processed 32.74 thousand rows`). The expression `k = '2017-09-15'` was not used when generating the result. -In examle the `indexHint` function allows to see adjacent dates. +In example, the `indexHint` function allows to see adjacent dates. Result: @@ -1369,7 +1389,7 @@ Result: Creates an array with a single value. -Used for internal implementation of [arrayJoin](../../sql-reference/functions/array-join.md#functions_arrayjoin). +Used for the internal implementation of [arrayJoin](../../sql-reference/functions/array-join.md#functions_arrayjoin). ``` sql SELECT replicate(x, arr); @@ -1377,12 +1397,12 @@ SELECT replicate(x, arr); **Arguments:** -- `arr` — Original array. ClickHouse creates a new array of the same length as the original and fills it with the value `x`. -- `x` — The value that the resulting array will be filled with. +- `arr` — An array. +- `x` — The value to fill the result array with. **Returned value** -An array filled with the value `x`. +An array of the lame length as `arr` filled with value `x`. Type: `Array`. @@ -1404,7 +1424,7 @@ Result: ## filesystemAvailable -Returns amount of remaining space on the filesystem where the files of the databases located. It is always smaller than total free space ([filesystemFree](#filesystemfree)) because some space is reserved for OS. +Returns the amount of free space in the filesystem hosting the database persistence. The returned value is always smaller than total free space ([filesystemFree](#filesystemfree)) because some space is reserved for the operating system. **Syntax** @@ -1423,20 +1443,20 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT formatReadableSize(filesystemAvailable()) AS "Available space", toTypeName(filesystemAvailable()) AS "Type"; +SELECT formatReadableSize(filesystemAvailable()) AS "Available space"; ``` Result: ``` text -┌─Available space─┬─Type───┐ -│ 30.75 GiB │ UInt64 │ -└─────────────────┴────────┘ +┌─Available space─┐ +│ 30.75 GiB │ +└─────────────────┘ ``` ## filesystemFree -Returns total amount of the free space on the filesystem where the files of the databases located. See also `filesystemAvailable` +Returns the total amount of the free space on the filesystem hosting the database persistence. See also `filesystemAvailable` **Syntax** @@ -1446,7 +1466,7 @@ filesystemFree() **Returned value** -- Amount of free space in bytes. +- The amount of free space in bytes. Type: [UInt64](../../sql-reference/data-types/int-uint.md). @@ -1455,20 +1475,20 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT formatReadableSize(filesystemFree()) AS "Free space", toTypeName(filesystemFree()) AS "Type"; +SELECT formatReadableSize(filesystemFree()) AS "Free space"; ``` Result: ``` text -┌─Free space─┬─Type───┐ -│ 32.39 GiB │ UInt64 │ -└────────────┴────────┘ +┌─Free space─┐ +│ 32.39 GiB │ +└────────────┘ ``` ## filesystemCapacity -Returns the capacity of the filesystem in bytes. For evaluation, the [path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) to the data directory must be configured. +Returns the capacity of the filesystem in bytes. Needs the [path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) to the data directory to be configured. **Syntax** @@ -1478,7 +1498,7 @@ filesystemCapacity() **Returned value** -- Capacity information of the filesystem in bytes. +- Capacity of the filesystem in bytes. Type: [UInt64](../../sql-reference/data-types/int-uint.md). @@ -1487,20 +1507,20 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT formatReadableSize(filesystemCapacity()) AS "Capacity", toTypeName(filesystemCapacity()) AS "Type" +SELECT formatReadableSize(filesystemCapacity()) AS "Capacity"; ``` Result: ``` text -┌─Capacity──┬─Type───┐ -│ 39.32 GiB │ UInt64 │ -└───────────┴────────┘ +┌─Capacity──┐ +│ 39.32 GiB │ +└───────────┘ ``` ## initializeAggregation -Calculates result of aggregate function based on single value. It is intended to use this function to initialize aggregate functions with combinator [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state). You can create states of aggregate functions and insert them to columns of type [AggregateFunction](../../sql-reference/data-types/aggregatefunction.md#data-type-aggregatefunction) or use initialized aggregates as default values. +Calculates the result of an aggregate function based on a single value. This function can be used to initialize aggregate functions with combinator [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state). You can create states of aggregate functions and insert them to columns of type [AggregateFunction](../../sql-reference/data-types/aggregatefunction.md#data-type-aggregatefunction) or use initialized aggregates as default values. **Syntax** @@ -1539,6 +1559,7 @@ Query: ```sql SELECT finalizeAggregation(state), toTypeName(state) FROM (SELECT initializeAggregation('sumState', number % 3) AS state FROM numbers(5)); ``` + Result: ```text @@ -1568,11 +1589,12 @@ INSERT INTO metrics VALUES (0, initializeAggregation('sumState', toUInt64(42))) ``` **See Also** + - [arrayReduce](../../sql-reference/functions/array-functions.md#arrayreduce) ## finalizeAggregation -Takes state of aggregate function. Returns result of aggregation (or finalized state when using[-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) combinator). +Given a state of aggregate function, this function returns the result of aggregation (or finalized state when using a [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) combinator). **Syntax** @@ -1667,15 +1689,16 @@ Result: ``` **See Also** + - [arrayReduce](../../sql-reference/functions/array-functions.md#arrayreduce) - [initializeAggregation](#initializeaggregation) ## runningAccumulate -Accumulates states of an aggregate function for each row of a data block. +Accumulates the states of an aggregate function for each row of a data block. -:::tip -The state is reset for each new data block. +:::note +The state is reset for each new block of data. ::: **Syntax** @@ -1726,10 +1749,10 @@ The subquery generates `sumState` for every number from `0` to `9`. `sumState` r The whole query does the following: -1. For the first row, `runningAccumulate` takes `sumState(0)` and returns `0`. -2. For the second row, the function merges `sumState(0)` and `sumState(1)` resulting in `sumState(0 + 1)`, and returns `1` as a result. -3. For the third row, the function merges `sumState(0 + 1)` and `sumState(2)` resulting in `sumState(0 + 1 + 2)`, and returns `3` as a result. -4. The actions are repeated until the block ends. +1. For the first row, `runningAccumulate` takes `sumState(0)` and returns `0`. +2. For the second row, the function merges `sumState(0)` and `sumState(1)` resulting in `sumState(0 + 1)`, and returns `1` as a result. +3. For the third row, the function merges `sumState(0 + 1)` and `sumState(2)` resulting in `sumState(0 + 1 + 2)`, and returns `3` as a result. +4. The actions are repeated until the block ends. The following example shows the `groupping` parameter usage: @@ -1780,7 +1803,7 @@ As you can see, `runningAccumulate` merges states for each group of rows separat The function lets you extract data from the table the same way as from a [dictionary](../../sql-reference/dictionaries/index.md). -Gets data from [Join](../../engines/table-engines/special/join.md#creating-a-table) tables using the specified join key. +Gets the data from [Join](../../engines/table-engines/special/join.md#creating-a-table) tables using the specified join key. Only supports tables created with the `ENGINE = Join(ANY, LEFT, )` statement. @@ -1792,13 +1815,13 @@ joinGet(join_storage_table_name, `value_column`, join_keys) **Arguments** -- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicates where search is performed. The identifier is searched in the default database (see parameter `default_database` in the config file). To override the default database, use the `USE db_name` or specify the database and the table through the separator `db_name.db_table`, see the example. +- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicating where the search is performed. The identifier is searched in the default database (see setting `default_database` in the config file). To override the default database, use `USE db_name` or specify the database and the table through the separator `db_name.db_table` as in the example. - `value_column` — name of the column of the table that contains required data. - `join_keys` — list of keys. **Returned value** -Returns list of values corresponded to list of keys. +Returns a list of values corresponded to list of keys. If certain does not exist in source table then `0` or `null` will be returned based on [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) setting. @@ -1825,7 +1848,7 @@ INSERT INTO db_test.id_val VALUES (1,11)(2,12)(4,13) Query: ``` sql -SELECT joinGet(db_test.id_val,'val',toUInt32(number)) from numbers(4) SETTINGS join_use_nulls = 1 +SELECT joinGet(db_test.id_val, 'val', toUInt32(number)) from numbers(4) SETTINGS join_use_nulls = 1 ``` Result: @@ -1845,7 +1868,7 @@ Result: This function is not available in ClickHouse Cloud. ::: -Evaluate external catboost model. [CatBoost](https://catboost.ai) is an open-source gradient boosting library developed by Yandex for machine learing. +Evaluate an external catboost model. [CatBoost](https://catboost.ai) is an open-source gradient boosting library developed by Yandex for machine learing. Accepts a path to a catboost model and model arguments (features). Returns Float64. ``` sql @@ -1886,16 +1909,24 @@ See [Training and applying models](https://catboost.ai/docs/features/training.ht ## throwIf(x\[, message\[, error_code\]\]) -Throw an exception if the argument is non zero. -`message` - is an optional parameter: a constant string providing a custom error message -`error_code` - is an optional parameter: a constant integer providing a custom error code +Throw an exception if argument `x` is true. + +**Arguments** + +- `x` - the condition to check. +- `message` - a constant string providing a custom error message. Optional. +- `error_code` - A constant integer providing a custom error code. Optional. To use the `error_code` argument, configuration parameter `allow_custom_error_code_in_throwif` must be enabled. +**Example** + ``` sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); ``` +Result: + ``` text ↙ Progress: 0.00 rows, 0.00 B (0.00 rows/s., 0.00 B/s.) Received exception from server (version 19.14.1): Code: 395. DB::Exception: Received from localhost:9000. DB::Exception: Too many. @@ -1903,7 +1934,7 @@ Code: 395. DB::Exception: Received from localhost:9000. DB::Exception: Too many. ## identity -Returns the same value that was used as its argument. Used for debugging and testing, allows to cancel using index, and get the query performance of a full scan. When query is analyzed for possible use of index, the analyzer does not look inside `identity` functions. Also constant folding is not applied too. +Returns its argument. Intended for debugging and testing. Allows to cancel using index, and get the query performance of a full scan. When the query is analyzed for possible use of an index, the analyzer ignores everything in `identity` functions. Also disables constant folding. **Syntax** @@ -1916,7 +1947,7 @@ identity(x) Query: ``` sql -SELECT identity(42) +SELECT identity(42); ``` Result: @@ -1927,164 +1958,6 @@ Result: └──────────────┘ ``` -## randomPrintableASCII - -Generates a string with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. - -**Syntax** - -``` sql -randomPrintableASCII(length) -``` - -**Arguments** - -- `length` — Resulting string length. Positive integer. - - If you pass `length < 0`, behavior of the function is undefined. - -**Returned value** - -- String with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. - -Type: [String](../../sql-reference/data-types/string.md) - -**Example** - -``` sql -SELECT number, randomPrintableASCII(30) as str, length(str) FROM system.numbers LIMIT 3 -``` - -``` text -┌─number─┬─str────────────────────────────┬─length(randomPrintableASCII(30))─┐ -│ 0 │ SuiCOSTvC0csfABSw=UcSzp2.`rv8x │ 30 │ -│ 1 │ 1Ag NlJ &RCN:*>HVPG;PE-nO"SUFD │ 30 │ -│ 2 │ /"+<"wUTh:=LjJ Vm!c&hI*m#XTfzz │ 30 │ -└────────┴────────────────────────────────┴──────────────────────────────────┘ -``` - -## randomString - -Generates a binary string of the specified length filled with random bytes (including zero bytes). - -**Syntax** - -``` sql -randomString(length) -``` - -**Arguments** - -- `length` — String length. Positive integer. - -**Returned value** - -- String filled with random bytes. - -Type: [String](../../sql-reference/data-types/string.md). - -**Example** - -Query: - -``` sql -SELECT randomString(30) AS str, length(str) AS len FROM numbers(2) FORMAT Vertical; -``` - -Result: - -``` text -Row 1: -────── -str: 3 G : pT ?w тi k aV f6 -len: 30 - -Row 2: -────── -str: 9 ,] ^ ) ]?? 8 -len: 30 -``` - -**See Also** - -- [generateRandom](../../sql-reference/table-functions/generate.md#generaterandom) -- [randomPrintableASCII](../../sql-reference/functions/other-functions.md#randomascii) - - -## randomFixedString - -Generates a binary string of the specified length filled with random bytes (including zero bytes). - -**Syntax** - -``` sql -randomFixedString(length); -``` - -**Arguments** - -- `length` — String length in bytes. [UInt64](../../sql-reference/data-types/int-uint.md). - -**Returned value(s)** - -- String filled with random bytes. - -Type: [FixedString](../../sql-reference/data-types/fixedstring.md). - -**Example** - -Query: - -```sql -SELECT randomFixedString(13) as rnd, toTypeName(rnd) -``` - -Result: - -```text -┌─rnd──────┬─toTypeName(randomFixedString(13))─┐ -│ j▒h㋖HɨZ'▒ │ FixedString(13) │ -└──────────┴───────────────────────────────────┘ - -``` - -## randomStringUTF8 - -Generates a random string of a specified length. Result string contains valid UTF-8 code points. The value of code points may be outside of the range of assigned Unicode. - -**Syntax** - -``` sql -randomStringUTF8(length); -``` - -**Arguments** - -- `length` — Required length of the resulting string in code points. [UInt64](../../sql-reference/data-types/int-uint.md). - -**Returned value(s)** - -- UTF-8 random string. - -Type: [String](../../sql-reference/data-types/string.md). - -**Example** - -Query: - -```sql -SELECT randomStringUTF8(13) -``` - -Result: - -```text -┌─randomStringUTF8(13)─┐ -│ 𘤗𙉝д兠庇󡅴󱱎󦐪􂕌𔊹𓰛 │ -└──────────────────────┘ - -``` - ## getSetting Returns the current value of a [custom setting](../../operations/settings/index.md#custom_settings). @@ -2101,7 +1974,7 @@ getSetting('custom_setting'); **Returned value** -- The setting current value. +- The setting's current value. **Example** @@ -2110,7 +1983,7 @@ SET custom_a = 123; SELECT getSetting('custom_a'); ``` -**Result** +Result: ``` 123 @@ -2122,7 +1995,7 @@ SELECT getSetting('custom_a'); ## isDecimalOverflow -Checks whether the [Decimal](../../sql-reference/data-types/decimal.md) value is out of its (or specified) precision. +Checks whether the [Decimal](../../sql-reference/data-types/decimal.md) value is outside its precision or outside the specified precision. **Syntax** @@ -2133,11 +2006,11 @@ isDecimalOverflow(d, [p]) **Arguments** - `d` — value. [Decimal](../../sql-reference/data-types/decimal.md). -- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. Using of this paratemer could be helpful for data extraction to another DBMS or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). +- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. This paratemer can be helpful to migrate data from/to another database or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). **Returned values** -- `1` — Decimal value has more digits then it's precision allow, +- `1` — Decimal value has more digits then allowed by its precision, - `0` — Decimal value satisfies the specified precision. **Example** @@ -2159,7 +2032,7 @@ Result: ## countDigits -Returns number of decimal digits you need to represent the value. +Returns number of decimal digits need to represent a value. **Syntax** @@ -2199,9 +2072,7 @@ Result: ## errorCodeToName -**Returned value** - -- Variable name for the error code. +Returns the textual name of an error code. Type: [LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md). @@ -2220,7 +2091,7 @@ UNSUPPORTED_METHOD ## tcpPort Returns [native interface](../../interfaces/tcp.md) TCP port number listened by this server. -If it is executed in the context of a distributed table, then it generates a normal column, otherwise it produces a constant value. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. **Syntax** @@ -2310,7 +2181,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## currentRoles -Returns the names of the roles which are current for the current user. The current roles can be changed by the [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) statement. If the `SET ROLE` statement was not used, the function `currentRoles` returns the same as `defaultRoles`. +Returns the roles assigned to the current user. The roles can be changed by the [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) statement. If no `SET ROLE` statement was not, the function `currentRoles` returns the same as `defaultRoles`. **Syntax** @@ -2320,7 +2191,7 @@ currentRoles() **Returned value** -- List of the current roles for the current user. +- A list of the current roles for the current user. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). @@ -2342,7 +2213,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## defaultRoles -Returns the names of the roles which are enabled by default for the current user when he logins. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant.md#grant-select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. +Returns the roles which are enabled by default for the current user when he logs in. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant.md#grant-select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. **Syntax** @@ -2358,7 +2229,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## getServerPort -Returns the number of the server port. When the port is not used by the server, throws an exception. +Returns the server port number. When the port is not used by the server, throws an exception. **Syntax** @@ -2407,7 +2278,7 @@ Result: Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. -In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see example). +In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see the example). **Syntax** @@ -2477,7 +2348,7 @@ Result: ## shardNum -Returns the index of a shard which processes a part of data for a distributed query. Indices are started from `1`. +Returns the index of a shard which processes a part of data in a distributed query. Indices are started from `1`. If a query is not distributed then constant value `0` is returned. **Syntax** diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index e90d537fb74..63d5174b494 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -31,7 +31,7 @@ Uses a linear congruential generator. ## randCanonical -Returns a Float64 value, evenly distributed in [0, 1). +Returns a random Float64 value, evenly distributed in interval [0, 1). ## randConstant @@ -54,11 +54,9 @@ Result: └────────────┴────────────┴──────────────┴────────────────┴─────────────────┴──────────────────────┘ ``` -# Functions for Generating Random Numbers based on a Distribution - ## randUniform -Returns a Float64 drawn uniformly from the interval between `min` and `max` ([continuous uniform distribution](https://en.wikipedia.org/wiki/Continuous_uniform_distribution)). +Returns a random Float64 drawn uniformly from interval [`min`, `max`) ([continuous uniform distribution](https://en.wikipedia.org/wiki/Continuous_uniform_distribution)). **Syntax** @@ -68,8 +66,8 @@ randUniform(min, max) **Arguments** -- `min` - `Float64` - min value of the range, -- `max` - `Float64` - max value of the range. +- `min` - `Float64` - left boundary of the range, +- `max` - `Float64` - reight boundary of the range. **Returned value** @@ -97,7 +95,7 @@ Result: ## randNormal -Returns a Float64 drawn from a [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution). +Returns a random Float64 drawn from a [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution). **Syntax** @@ -108,7 +106,7 @@ randNormal(mean, variance) **Arguments** - `mean` - `Float64` - mean value of distribution, -- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance). +- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance) of the distribution. **Returned value** @@ -136,7 +134,7 @@ Result: ## randLogNormal -Returns a Float64 drawn from a [log-normal distribution](https://en.wikipedia.org/wiki/Log-normal_distribution). +Returns a random Float64 drawn from a [log-normal distribution](https://en.wikipedia.org/wiki/Log-normal_distribution). **Syntax** @@ -147,7 +145,7 @@ randLogNormal(mean, variance) **Arguments** - `mean` - `Float64` - mean value of distribution, -- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance). +- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance) of the distribution. **Returned value** @@ -175,7 +173,7 @@ Result: ## randBinomial -Returns a UInt64 drawn from a [binomial distribution](https://en.wikipedia.org/wiki/Binomial_distribution). +Returns a random UInt64 drawn from a [binomial distribution](https://en.wikipedia.org/wiki/Binomial_distribution). **Syntax** @@ -186,7 +184,7 @@ randBinomial(experiments, probability) **Arguments** - `experiments` - `UInt64` - number of experiments, -- `probability` - `Float64` - probability of success in each experiment (values in `0...1` range only). +- `probability` - `Float64` - probability of success in each experiment, a value between 0 and 1. **Returned value** @@ -214,7 +212,7 @@ Result: ## randNegativeBinomial -Returns a UInt64 drawn from a [negative binomial distribution](https://en.wikipedia.org/wiki/Negative_binomial_distribution). +Returns a random UInt64 drawn from a [negative binomial distribution](https://en.wikipedia.org/wiki/Negative_binomial_distribution). **Syntax** @@ -225,7 +223,7 @@ randNegativeBinomial(experiments, probability) **Arguments** - `experiments` - `UInt64` - number of experiments, -- `probability` - `Float64` - probability of failure in each experiment (values in `0...1` range only). +- `probability` - `Float64` - probability of failure in each experiment, a value between 0 and 1. **Returned value** @@ -253,7 +251,7 @@ Result: ## randPoisson -Returns a UInt64 drawn from a [Poisson distribution](https://en.wikipedia.org/wiki/Poisson_distribution). +Returns a random UInt64 drawn from a [Poisson distribution](https://en.wikipedia.org/wiki/Poisson_distribution). **Syntax** @@ -291,7 +289,7 @@ Result: ## randBernoulli -Returns a UInt64 drawn from a [Bernoulli distribution](https://en.wikipedia.org/wiki/Bernoulli_distribution). +Returns a random UInt64 drawn from a [Bernoulli distribution](https://en.wikipedia.org/wiki/Bernoulli_distribution). **Syntax** @@ -301,7 +299,7 @@ randBernoulli(probability) **Arguments** -- `probability` - `Float64` - probability of success (values in `0...1` range only). +- `probability` - `Float64` - probability of success, a value between 0 and 1. **Returned value** @@ -329,7 +327,7 @@ Result: ## randExponential -Returns a Float64 drawn from a [exponential distribution](https://en.wikipedia.org/wiki/Exponential_distribution). +Returns a random Float64 drawn from a [exponential distribution](https://en.wikipedia.org/wiki/Exponential_distribution). **Syntax** @@ -367,7 +365,7 @@ Result: ## randChiSquared -Returns a Float64 drawn from a [Chi-square distribution](https://en.wikipedia.org/wiki/Chi-squared_distribution) - a distribution of a sum of the squares of k independent standard normal random variables. +Returns a random Float64 drawn from a [Chi-square distribution](https://en.wikipedia.org/wiki/Chi-squared_distribution) - a distribution of a sum of the squares of k independent standard normal random variables. **Syntax** @@ -405,7 +403,7 @@ Result: ## randStudentT -Returns a Float64 drawn from a [Student's t-distribution](https://en.wikipedia.org/wiki/Student%27s_t-distribution). +Returns a random Float64 drawn from a [Student's t-distribution](https://en.wikipedia.org/wiki/Student%27s_t-distribution). **Syntax** @@ -443,7 +441,7 @@ Result: ## randFisherF -Returns a Float64 drawn from a [F-distribution](https://en.wikipedia.org/wiki/F-distribution). +Returns a random Float64 drawn from a [F-distribution](https://en.wikipedia.org/wiki/F-distribution). **Syntax** @@ -480,47 +478,160 @@ Result: └─────────────────────┘ ``` -# Functions for Generating Random Strings - ## randomString -Returns a random String of specified `length`. Not all characters may be printable. +Generates a string of the specified length filled with random bytes (including zero bytes). Not all characters may be printable. **Syntax** -```sql +``` sql randomString(length) ``` +**Arguments** + +- `length` — String length in bytes. Positive integer. + +**Returned value** + +- String filled with random bytes. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT randomString(30) AS str, length(str) AS len FROM numbers(2) FORMAT Vertical; +``` + +Result: + +``` text +Row 1: +────── +str: 3 G : pT ?w тi k aV f6 +len: 30 + +Row 2: +────── +str: 9 ,] ^ ) ]?? 8 +len: 30 +``` + ## randomFixedString -Like `randomString` but returns a FixedString. - -## randomPrintableASCII - -Returns a random String of specified `length`. All characters are printable. +Generates a binary string of the specified length filled with random bytes (including zero bytes). Not all characters may be printable. **Syntax** +``` sql +randomFixedString(length); +``` + +**Arguments** + +- `length` — String length in bytes. [UInt64](../../sql-reference/data-types/int-uint.md). + +**Returned value(s)** + +- String filled with random bytes. + +Type: [FixedString](../../sql-reference/data-types/fixedstring.md). + +**Example** + +Query: + ```sql +SELECT randomFixedString(13) as rnd, toTypeName(rnd) +``` + +Result: + +```text +┌─rnd──────┬─toTypeName(randomFixedString(13))─┐ +│ j▒h㋖HɨZ'▒ │ FixedString(13) │ +└──────────┴───────────────────────────────────┘ +``` + +## randomPrintableASCII + +Generates a string with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) characters. All characters are printable. +If you pass `length < 0`, the behavior of the function is undefined. + +**Syntax** + +``` sql randomPrintableASCII(length) ``` +**Arguments** + +- `length` — String length in bytes. Positive integer. + +**Returned value** + +- String with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. + +Type: [String](../../sql-reference/data-types/string.md) + +**Example** + +``` sql +SELECT number, randomPrintableASCII(30) as str, length(str) FROM system.numbers LIMIT 3 +``` + +``` text +┌─number─┬─str────────────────────────────┬─length(randomPrintableASCII(30))─┐ +│ 0 │ SuiCOSTvC0csfABSw=UcSzp2.`rv8x │ 30 │ +│ 1 │ 1Ag NlJ &RCN:*>HVPG;PE-nO"SUFD │ 30 │ +│ 2 │ /"+<"wUTh:=LjJ Vm!c&hI*m#XTfzz │ 30 │ +└────────┴────────────────────────────────┴──────────────────────────────────┘ +``` + ## randomStringUTF8 -Returns a random String containing `length` many UTF8 codepoints. Not all characters may be printable +Generates a random string of a specified length. Result string contains valid UTF-8 code points. The value of code points may be outside of the range of assigned Unicode. **Syntax** +``` sql +randomStringUTF8(length); +``` + +**Arguments** + +- `length` — Length of the string in code points. [UInt64](../../sql-reference/data-types/int-uint.md). + +**Returned value(s)** + +- UTF-8 random string. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + ```sql -randomStringUTF8(length) +SELECT randomStringUTF8(13) +``` + +Result: + +```text +┌─randomStringUTF8(13)─┐ +│ 𘤗𙉝д兠庇󡅴󱱎󦐪􂕌𔊹𓰛 │ +└──────────────────────┘ ``` ## fuzzBits **Syntax** -Inverts the bits of String or FixedString `s`, each with probability `prob`. +Flips the bits of String or FixedString `s`, each with probability `prob`. **Syntax** @@ -529,8 +640,8 @@ fuzzBits(s, prob) ``` **Arguments** -- `s` - `String` or `FixedString` -- `prob` - constant `Float32/64` +- `s` - `String` or `FixedString`, +- `prob` - constant `Float32/64` between 0.0 and 1.0. **Returned value** From 0181ea63993bc6dd9c924c092bc7294a1bf71e2f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 12:55:46 +0300 Subject: [PATCH 1002/2223] Revert "make filter push down through cross join" --- .../Optimizations/filterPushDown.cpp | 6 +++--- .../01763_filter_push_down_bugs.reference | 19 ------------------- .../01763_filter_push_down_bugs.sql | 19 ------------------- 3 files changed, 3 insertions(+), 41 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index db29038999b..37bc894339f 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -272,7 +272,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { /// If totals step has HAVING expression, skip it for now. /// TODO: - /// We can merge HAVING expression with current filter. + /// We can merge HAVING expression with current filer. /// Also, we can push down part of HAVING which depend only on aggregation keys. if (totals_having->getActions()) return 0; @@ -323,9 +323,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin(); - /// Only inner, cross and left(/right) join are supported. Other types may generate default values for left table keys. + /// Only inner and left(/right) join are supported. Other types may generate default values for left table keys. /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (table_join.kind() != JoinKind::Inner && table_join.kind() != JoinKind::Cross && table_join.kind() != kind) + if (table_join.kind() != JoinKind::Inner && table_join.kind() != kind) return 0; bool is_left = kind == JoinKind::Left; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 7df35e2948d..5aa2e645509 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -6,22 +6,3 @@ String1_0 String2_0 String3_0 String4_0 1 String1_0 String2_0 String3_0 String4_0 1 1 [0,1,2] 1 -Expression ((Projection + Before ORDER BY)) - Filter (WHERE) - Join (JOIN FillRightFirst) - Filter (( + Before JOIN)) - ReadFromMergeTree (default.t1) - Indexes: - PrimaryKey - Keys: - id - Condition: (id in [101, 101]) - Parts: 1/1 - Granules: 1/1 - Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) - ReadFromMergeTree (default.t2) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 2ee249b5ce7..1058bf75144 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -38,25 +38,6 @@ DROP TABLE IF EXISTS Test; select x, y from (select [0, 1, 2] as y, 1 as a, 2 as b) array join y as x where a = 1 and b = 2 and (x = 1 or x != 1) and x = 1; -DROP TABLE IF EXISTS t; create table t(a UInt8) engine=MergeTree order by a; insert into t select * from numbers(2); select a from t t1 join t t2 on t1.a = t2.a where t1.a; -DROP TABLE IF EXISTS t; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id; -CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time; - -insert into t1 values (101, '2023-05-28 00:00:00'), (102, '2023-05-28 00:00:00'); -insert into t2 values ('2023-05-31 00:00:00'); - -EXPLAIN indexes=1 SELECT id, delete_time FROM t1 - CROSS JOIN ( - SELECT delete_time - FROM t2 -) AS d WHERE create_time < delete_time AND id = 101; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; From 0ddd53088d48134cdbd55bcda2e6bc3bfad423de Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 31 May 2023 23:17:41 +0200 Subject: [PATCH 1003/2223] Add a new runner type for ci metrics and autoscaling --- tests/ci/autoscale_runners_lambda/app.py | 4 ++++ tests/ci/lambda_shared_package/lambda_shared/__init__.py | 1 + 2 files changed, 5 insertions(+) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index ab09afb3aa8..825708cabbc 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -65,6 +65,10 @@ def get_scales(runner_type: str) -> Tuple[int, int]: # 10. I am trying 7 now. # UPDATE THE COMMENT ON CHANGES scale_up = 7 + elif runner_type == "limited-tester": + # The limited runners should inflate and deflate faster + scale_down = 1 + scale_up = 2 return scale_down, scale_up diff --git a/tests/ci/lambda_shared_package/lambda_shared/__init__.py b/tests/ci/lambda_shared_package/lambda_shared/__init__.py index 534d7773ddd..c56994cc86a 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/__init__.py +++ b/tests/ci/lambda_shared_package/lambda_shared/__init__.py @@ -15,6 +15,7 @@ RUNNER_TYPE_LABELS = [ "func-tester", "func-tester-aarch64", "fuzzer-unit-tester", + "limited-tester", "stress-tester", "style-checker", "style-checker-aarch64", From b7c5fdab77c41361af7b5130256a2afdf2bc1488 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 31 May 2023 23:56:39 +0200 Subject: [PATCH 1004/2223] Move lambda package building to public.ecr.aws/lambda/python for compatibility --- tests/ci/lambda_shared_package/pyproject.toml | 5 +++-- tests/ci/team_keys_lambda/build_and_deploy_archive.sh | 6 +++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/ci/lambda_shared_package/pyproject.toml b/tests/ci/lambda_shared_package/pyproject.toml index bbf74cc0649..dff36b89fbb 100644 --- a/tests/ci/lambda_shared_package/pyproject.toml +++ b/tests/ci/lambda_shared_package/pyproject.toml @@ -6,13 +6,14 @@ build-backend = "setuptools.build_meta" name = "lambda_shared" version = "0.0.1" dependencies = [ - "requests < 2.30", + "requests", + "urllib3 < 2" ] [project.optional-dependencies] token = [ "PyJWT", - "cryptography<38", + "cryptography", ] dev = [ "boto3", diff --git a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh index 89a2d514965..02d5638cf18 100644 --- a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh +++ b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh @@ -12,7 +12,7 @@ DRY_RUN=${DRY_RUN:-} PY_VERSION=${PY_VERSION:-3.10} PY_EXEC="python${PY_VERSION}" # Image to build the lambda zip package -DOCKER_IMAGE="python:${PY_VERSION}-slim" +DOCKER_IMAGE="public.ecr.aws/lambda/python:${PY_VERSION}" # Rename the_lambda_name directory to the-lambda-name lambda in AWS LAMBDA_NAME=${DIR_NAME//_/-} # The name of directory with lambda code @@ -23,9 +23,9 @@ cp app.py "$PACKAGE" if [ -f requirements.txt ]; then VENV=lambda-venv rm -rf "$VENV" lambda-package.zip - docker run --rm --user="${UID}" -e HOME=/tmp \ + docker run --rm --user="${UID}" -e HOME=/tmp --entrypoint=/bin/bash \ --volume="${WORKDIR}/..:/ci" --workdir="/ci/${DIR_NAME}" "${DOCKER_IMAGE}" \ - /bin/bash -exc " + -exc " '$PY_EXEC' -m venv '$VENV' && source '$VENV/bin/activate' && pip install -r requirements.txt From e657e2ba10c425caa550de6e9b3814dca8fc3f32 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Jun 2023 17:12:19 +0200 Subject: [PATCH 1005/2223] Additional logging in autoscale_runners_lambda --- tests/ci/autoscale_runners_lambda/app.py | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 825708cabbc..bebfb594b59 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -117,7 +117,17 @@ def set_capacity( # Finally, should the capacity be even changed stop = stop or asg["DesiredCapacity"] == desired_capacity if stop: + logging.info( + "Do not increase ASG %s capacity, current capacity=%s, " + "maximum capacity=%s, running jobs=%s, queue size=%s", + asg["AutoScalingGroupName"], + desired_capacity, + asg["MaxSize"], + running, + queued, + ) return + logging.info( "The ASG %s capacity will be increased to %s, current capacity=%s, " "maximum capacity=%s, running jobs=%s, queue size=%s", @@ -142,6 +152,15 @@ def set_capacity( desired_capacity = min(desired_capacity, asg["MaxSize"]) stop = stop or asg["DesiredCapacity"] == desired_capacity if stop: + logging.info( + "Do not decrease ASG %s capacity, current capacity=%s, " + "minimum capacity=%s, running jobs=%s, queue size=%s", + asg["AutoScalingGroupName"], + desired_capacity, + asg["MinSize"], + running, + queued, + ) return logging.info( From b775b5cfd6eb924bc119ea80b08fde9a4bb42a1b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Jun 2023 17:44:57 +0200 Subject: [PATCH 1006/2223] Move all CI runners metrics into one namespace --- tests/ci/ci_runners_metrics_lambda/app.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py index dc128dea739..d8b9489b1a7 100644 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ b/tests/ci/ci_runners_metrics_lambda/app.py @@ -171,18 +171,21 @@ def group_runners_by_tag( def push_metrics_to_cloudwatch( - listed_runners: RunnerDescriptions, namespace: str + listed_runners: RunnerDescriptions, group_name: str ) -> None: client = boto3.client("cloudwatch") + namespace = "RunnersMetrics" metrics_data = [] busy_runners = sum( 1 for runner in listed_runners if runner.busy and not runner.offline ) + dimensions = [{"Name": "group", "Value": group_name}] metrics_data.append( { "MetricName": "BusyRunners", "Value": busy_runners, "Unit": "Count", + "Dimensions": dimensions, } ) total_active_runners = sum(1 for runner in listed_runners if not runner.offline) @@ -191,6 +194,7 @@ def push_metrics_to_cloudwatch( "MetricName": "ActiveRunners", "Value": total_active_runners, "Unit": "Count", + "Dimensions": dimensions, } ) total_runners = len(listed_runners) @@ -199,6 +203,7 @@ def push_metrics_to_cloudwatch( "MetricName": "TotalRunners", "Value": total_runners, "Unit": "Count", + "Dimensions": dimensions, } ) if total_active_runners == 0: @@ -211,6 +216,7 @@ def push_metrics_to_cloudwatch( "MetricName": "BusyRunnersRatio", "Value": busy_ratio, "Unit": "Percent", + "Dimensions": dimensions, } ) @@ -242,7 +248,7 @@ def main( for group, group_runners in grouped_runners.items(): if push_to_cloudwatch: print(f"Pushing metrics for group '{group}'") - push_metrics_to_cloudwatch(group_runners, "RunnersMetrics/" + group) + push_metrics_to_cloudwatch(group_runners, group) else: print(group, f"({len(group_runners)})") for runner in group_runners: From 3c13eaa1592b5cf46d6aca8b3764aefc546eeff7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Jun 2023 22:03:42 +0200 Subject: [PATCH 1007/2223] Simplify get_dead_runners_in_ec2 a little bit, update runners --- tests/ci/ci_runners_metrics_lambda/app.py | 62 +++++++++++------------ 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py index d8b9489b1a7..9c38659269b 100644 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ b/tests/ci/ci_runners_metrics_lambda/app.py @@ -99,44 +99,44 @@ def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions: def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[dict]: client = boto3.client("ec2") reservations = client.describe_instances( - Filters=[{"Name": "tag-key", "Values": ["github:runner-type"]}] + Filters=[{"Name": "tag-key", "Values": ["github:runner-type"]}], )["Reservations"] - lost_instances = [] - offline_runners = [ - runner.name for runner in runners if runner.offline and not runner.busy + # flatten the reservation into instances + instances = [ + instance + for reservation in reservations + for instance in reservation["Instances"] ] - # Here we refresh the runners to get the most recent state + lost_instances = [] + offline_runner_names = { + runner.name for runner in runners if runner.offline and not runner.busy + } + runner_names = {runner.name for runner in runners} now = datetime.now().timestamp() - for reservation in reservations: - for instance in reservation["Instances"]: - # Do not consider instances started 20 minutes ago as problematic - if now - instance["LaunchTime"].timestamp() < 1200: - continue + for instance in instances: + # Do not consider instances started 20 minutes ago as problematic + if now - instance["LaunchTime"].timestamp() < 1200: + continue - runner_type = [ - tag["Value"] - for tag in instance["Tags"] - if tag["Key"] == "github:runner-type" - ][0] - # If there's no necessary labels in runner type it's fine - if not ( - UNIVERSAL_LABEL in runner_type or runner_type in RUNNER_TYPE_LABELS - ): - continue + runner_type = [ + tag["Value"] + for tag in instance["Tags"] + if tag["Key"] == "github:runner-type" + ][0] + # If there's no necessary labels in runner type it's fine + if not (UNIVERSAL_LABEL in runner_type or runner_type in RUNNER_TYPE_LABELS): + continue - if instance["InstanceId"] in offline_runners: - lost_instances.append(instance) - continue + if instance["InstanceId"] in offline_runner_names: + lost_instances.append(instance) + continue - if instance["State"]["Name"] == "running" and ( - not [ - runner - for runner in runners - if runner.name == instance["InstanceId"] - ] - ): - lost_instances.append(instance) + if ( + instance["State"]["Name"] == "running" + and not instance["InstanceId"] in runner_names + ): + lost_instances.append(instance) return lost_instances From 9bd0a53e7c3ce6b1ad86bc09b134319a8563ff47 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Jun 2023 22:28:02 +0200 Subject: [PATCH 1008/2223] Get only online instances in get_lost_ec2_instances --- tests/ci/ci_runners_metrics_lambda/app.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py index 9c38659269b..71a644fe072 100644 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ b/tests/ci/ci_runners_metrics_lambda/app.py @@ -99,7 +99,10 @@ def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions: def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[dict]: client = boto3.client("ec2") reservations = client.describe_instances( - Filters=[{"Name": "tag-key", "Values": ["github:runner-type"]}], + Filters=[ + {"Name": "tag-key", "Values": ["github:runner-type"]}, + {"Name": "instance-state-name", "Values": ["pending", "running"]}, + ], )["Reservations"] # flatten the reservation into instances instances = [ From 8cadd89ebedc2fee73c3081992d35bbf8ad3280c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 2 Jun 2023 12:34:22 +0200 Subject: [PATCH 1009/2223] Update src/Common/TaskStatsInfoGetter.cpp --- src/Common/TaskStatsInfoGetter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index d21229609dd..867a50c8cce 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -208,7 +208,7 @@ bool checkPermissionsImpl() try { ::taskstats stats{}; - TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); + TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); } catch (const Exception & e) { From 79c14c89ee39bea6017e41e375f9b6a59e3e7899 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Jun 2023 12:53:25 +0200 Subject: [PATCH 1010/2223] Add some assertions --- src/Interpreters/Cache/FileCache.cpp | 6 +++--- src/Interpreters/Cache/FileSegment.cpp | 18 +++++++++++++----- src/Interpreters/Cache/Metadata.cpp | 21 ++++++++++++--------- src/Interpreters/Cache/Metadata.h | 4 ++-- 4 files changed, 30 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 1908a4ce895..163e1b71ab9 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -48,12 +48,12 @@ const String & FileCache::getBasePath() const String FileCache::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const { - return metadata.getPathInLocalCache(key, offset, segment_kind); + return metadata.getPathForFileSegment(key, offset, segment_kind); } String FileCache::getPathInLocalCache(const Key & key) const { - return metadata.getPathInLocalCache(key); + return metadata.getPathForKey(key); } void FileCache::assertInitialized() const @@ -1019,7 +1019,7 @@ std::vector FileCache::tryGetCachePaths(const Key & key) for (const auto & [offset, file_segment_metadata] : *locked_key->getKeyMetadata()) { if (file_segment_metadata->file_segment->state() == FileSegment::State::DOWNLOADED) - cache_paths.push_back(metadata.getPathInLocalCache(key, offset, file_segment_metadata->file_segment->getKind())); + cache_paths.push_back(metadata.getPathForFileSegment(key, offset, file_segment_metadata->file_segment->getKind())); } return cache_paths; } diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 60228573666..fb0ba0eba14 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -313,6 +313,13 @@ void FileSegment::write(const char * from, size_t size, size_t offset) if (!size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed"); + auto file_segment_path = getPathInLocalCache(); + if (offset == range().left && fs::exists(file_segment_path)) + { + fs::remove(file_segment_path); + chassert(false); + } + { auto lock = segment_guard.lock(); @@ -365,7 +372,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) downloaded_size += size; - chassert(std::filesystem::file_size(getPathInLocalCache()) == downloaded_size); + chassert(std::filesystem::file_size(file_segment_path) == downloaded_size); } catch (ErrnoException & e) { @@ -375,7 +382,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) int code = e.getErrno(); if (code == /* No space left on device */28 || code == /* Quota exceeded */122) { - const auto file_size = fs::file_size(getPathInLocalCache()); + const auto file_size = fs::file_size(file_segment_path); chassert(downloaded_size <= file_size); chassert(reserved_size >= file_size); if (downloaded_size != file_size) @@ -520,8 +527,8 @@ void FileSegment::setDownloadedUnlocked(const FileSegmentGuard::Lock &) remote_file_reader.reset(); } - chassert(getDownloadedSize(false) > 0); - chassert(fs::file_size(getPathInLocalCache()) > 0); + chassert(downloaded_size > 0); + chassert(fs::file_size(getPathInLocalCache()) == downloaded_size); } void FileSegment::setDownloadFailedUnlocked(const FileSegmentGuard::Lock & lock) @@ -845,7 +852,8 @@ void FileSegment::detach(const FileSegmentGuard::Lock & lock, const LockedKey &) if (download_state == State::DETACHED) return; - resetDownloaderUnlocked(lock); + if (!downloader_id.empty()) + resetDownloaderUnlocked(lock); setDetachedState(lock); } diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 843ffd45b63..9dff77e2af8 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -145,15 +145,12 @@ String CacheMetadata::getFileNameForFileSegment(size_t offset, FileSegmentKind s return std::to_string(offset) + file_suffix; } -String CacheMetadata::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const +String CacheMetadata::getPathForFileSegment(const Key & key, size_t offset, FileSegmentKind segment_kind) const { - String file_suffix; - - const auto key_str = key.toString(); - return fs::path(path) / key_str.substr(0, 3) / key_str / getFileNameForFileSegment(offset, segment_kind); + return fs::path(getPathForKey(key)) / getFileNameForFileSegment(offset, segment_kind); } -String CacheMetadata::getPathInLocalCache(const Key & key) const +String CacheMetadata::getPathForKey(const Key & key) const { const auto key_str = key.toString(); return fs::path(path) / key_str.substr(0, 3) / key_str; @@ -178,7 +175,7 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata( it = emplace( key, std::make_shared( - key, getPathInLocalCache(key), *cleanup_queue, is_initial_load)).first; + key, getPathForKey(key), *cleanup_queue, is_initial_load)).first; } key_metadata = it->second; @@ -260,7 +257,7 @@ void CacheMetadata::doCleanup() erase(it); LOG_DEBUG(log, "Key {} is removed from metadata", cleanup_key); - const fs::path key_directory = getPathInLocalCache(cleanup_key); + const fs::path key_directory = getPathForKey(cleanup_key); const fs::path key_prefix_directory = key_directory.parent_path(); try @@ -370,8 +367,14 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm file_segment->queue_iterator->annul(); const auto path = key_metadata->getFileSegmentPath(*file_segment); - if (fs::exists(path)) + bool exists = fs::exists(path); + if (exists) + { + LOG_TEST(log, "Removed file segment at path: {}", path); fs::remove(path); + } + else if (file_segment->downloaded_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist"); file_segment->detach(segment_lock, *this); return key_metadata->erase(it); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 2e015b07ed0..a7e101c3d9d 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -85,12 +85,12 @@ public: const String & getBaseDirectory() const { return path; } - String getPathInLocalCache( + String getPathForFileSegment( const Key & key, size_t offset, FileSegmentKind segment_kind) const; - String getPathInLocalCache(const Key & key) const; + String getPathForKey(const Key & key) const; static String getFileNameForFileSegment(size_t offset, FileSegmentKind segment_kind); void iterate(IterateCacheMetadataFunc && func); From 572f15b2cd70fa7b4293c7ec9682e361c9989d77 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 11:14:52 +0000 Subject: [PATCH 1011/2223] Fix typo --- docs/en/sql-reference/functions/functions-for-nulls.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index f4ca27e9b16..6f82fedaab7 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -245,7 +245,7 @@ assumeNotNull(x) **Returned values** - The input value as non-`Nullable` type, if it is not `NULL`. -- An arbirary value, if the input value is `NULL`. +- An arbitrary value, if the input value is `NULL`. **Example** From fa5f890a7ad43fde1cd75d6c07170a6df3ec119d Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 2 Jun 2023 12:03:50 +0000 Subject: [PATCH 1012/2223] Added ru function descriptions (docs) --- .../functions/type-conversion-functions.md | 2 +- .../functions/type-conversion-functions.md | 86 +++++++++++++++++++ src/Functions/DateTimeTransforms.h | 7 +- 3 files changed, 92 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index a6fc6cd4dfc..eb210863c32 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -383,7 +383,7 @@ toDateTimeOrDefault(expr, [, time_zone [, default_value]]) **Arguments** - `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [Int](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md). -- `time_zone` — Time zone. +- `time_zone` — Time zone. [String](/docs/en/sql-reference/data-types/string.md). - `default_value` — The default value. [DateTime](/docs/en/sql-reference/data-types/datetime.md) If `expr` is a number, it is interpreted as the number of seconds since the beginning of the Unix Epoch (as Unix timestamp). diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 298b7bbc93e..67d1732d34e 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -173,6 +173,49 @@ Cиноним: `DATE`. ## toDateOrDefault {#todateordefault} +Конвертирует аргумент в значение [Date](/docs/ru/sql-reference/data-types/date.md) data type. +Если получен недопустимый аргумент, то возвращает значение по умолчанию (нижняя граница [Date](/docs/ru/sql-reference/data-types/date.md). Значение по умолчанию может быть указано вторым аргументом. +Похожа на [toDate](#todate). + +**Синтаксис** + +``` sql +toDateOrDefault(expr [, default_value]) +``` + +**Аргументы** + +- `expr` — Значение для преобразования. [String](/docs/ru/sql-reference/data-types/string.md), [Int](/docs/ru/sql-reference/data-types/int-uint.md), [Date](/docs/ru/sql-reference/data-types/date.md) или [DateTime](/docs/ru/sql-reference/data-types/datetime.md). +- `default_value` — Значение по умолчанию. [Date](/docs/ru/sql-reference/data-types/date.md) + +Если `expr` является числом выглядит как UNIX timestamp (больше чем 65535), оно интерпретируется как DateTime, затем обрезается до Date учитывавая текущую часовой пояс. Если `expr` является числом и меньше чем 65536, оно интерпретируется как количество дней с 1970-01-01. + +**Возвращаемое значение** + +- Календарная дата. [Date](/docs/ru/sql-reference/data-types/date.md). + +**Пример** + +Запрос: + +``` sql +SELECT + toDateOrDefault('2021-01-01', '2023-01-01'::Date), + toDateOrDefault('xx2021-01-01', '2023-01-01'::Date); +``` + +Результат: + +```response +┌─toDateOrDefault('2021-01-01', CAST('2023-01-01', 'Date'))─┬─toDateOrDefault('xx2021-01-01', CAST('2023-01-01', 'Date'))─┐ +│ 2021-01-01 │ 2023-01-01 │ +└───────────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────┘ +``` + +**Смотрите также** +- [toDate](#todate) +- [toDate32OrDefault](#todate32ordefault) + ## toDateTime {#todatetime} ## toDateTimeOrZero {#todatetimeorzero} @@ -181,6 +224,49 @@ Cиноним: `DATE`. ## toDateTimeOrDefault {#todatetimeordefault} +Конвертирует аргумент в значение [DateTime](/docs/ru/sql-reference/data-types/datetime.md). +Если получен недопустимый аргумент, то возвращает значение по умолчанию (нижняя граница [DateTime](/docs/ru/sql-reference/data-types/datetime.md)). Значение по умолчанию может быть указано третьим аргументом. +Похожа на [toDateTime](#todatetime). + +**Синтаксис** + +``` sql +toDateTimeOrDefault(expr, [, time_zone [, default_value]]) +``` + +**Аргументы** + +- `expr` — Значение для преобразования. [String](/docs/ru/sql-reference/data-types/string.md), [Int](/docs/ru/sql-reference/data-types/int-uint.md), [Date](/docs/ru/sql-reference/data-types/date.md) или [DateTime](/docs/ru/sql-reference/data-types/datetime.md). +- `time_zone` — Часовой пояс. [String](/docs/ru/sql-reference/data-types/string.md). +- `default_value` — Значение по умолчанию. [DateTime](/docs/ru/sql-reference/data-types/datetime.md) + +Если `expr` является числом, оно интерпретируется как количество секунд от начала unix эпохи. + +**Возвращаемое значение** + +- Время. [DateTime](/docs/ru/sql-reference/data-types/datetime.md) + +**Пример** + +Запрос: + +``` sql +SELECT + toDateTimeOrDefault('2021-01-01', 'UTC', '2023-01-01'::DateTime('UTC')), + toDateTimeOrDefault('xx2021-01-01', 'UTC', '2023-01-01'::DateTime('UTC')); +``` + +Результат: + +```response +┌─toDateTimeOrDefault('2021-01-01', 'UTC', CAST('2023-01-01', 'DateTime(\'UTC\')'))─┬─toDateTimeOrDefault('xx2021-01-01', 'UTC', CAST('2023-01-01', 'DateTime(\'UTC\')'))─┐ +│ 2021-01-01 00:00:00 │ 2023-01-01 00:00:00 │ +└───────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────┘ +``` + +**Смотрите также** +- [toDateTime](#todatetime) + ## toDate32 {#todate32} Конвертирует аргумент в значение типа [Date32](../../sql-reference/data-types/date32.md). Если значение выходит за границы диапазона, возвращается пограничное значение `Date32`. Если аргумент имеет тип [Date](../../sql-reference/data-types/date.md), учитываются границы типа `Date`. diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 9f8f4df2465..d154dd9ffa2 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1520,9 +1520,12 @@ struct DateTimeTransformImpl Op::vector(sources->getData(), col_to->getData(), time_zone, transform, vec_null_map_to); } - if (vec_null_map_to) + if constexpr (std::is_same_v) { - return ColumnNullable::create(std::move(mutable_result_col), std::move(col_null_map_to)); + if (vec_null_map_to) + { + return ColumnNullable::create(std::move(mutable_result_col), std::move(col_null_map_to)); + } } return mutable_result_col; From 73db383727550e040a939d367e6e59bb037780bd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 2 Jun 2023 14:10:26 +0200 Subject: [PATCH 1013/2223] Limit the number of in-flight tasks for loading outdated parts (#50450) * Done * Update programs/local/LocalServer.cpp Co-authored-by: Alexander Tokmakov * Bump --------- Co-authored-by: Alexander Tokmakov --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 2 +- src/Core/ServerSettings.h | 1 - src/Interpreters/threadPoolCallbackRunner.h | 4 ++-- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 944a60d4e4c..96c1ca261b5 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -138,7 +138,7 @@ void LocalServer::initialize(Poco::Util::Application & self) OutdatedPartsLoadingThreadPool::initialize( config().getUInt("max_outdated_parts_loading_thread_pool_size", 16), 0, // We don't need any threads one all the parts will be loaded - config().getUInt("outdated_part_loading_thread_pool_queue_size", 10000)); + config().getUInt("max_outdated_parts_loading_thread_pool_size", 16)); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 03ada89e86f..9eb3e6c9ebc 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -696,7 +696,7 @@ try OutdatedPartsLoadingThreadPool::initialize( server_settings.max_outdated_parts_loading_thread_pool_size, 0, // We don't need any threads one all the parts will be loaded - server_settings.outdated_part_loading_thread_pool_queue_size); + server_settings.max_outdated_parts_loading_thread_pool_size); /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 511b8d68f6d..cb43d62ecd1 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -22,7 +22,6 @@ namespace DB M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \ M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The maximum number of threads that would be used for loading outdated data parts on startup", 0) \ - M(UInt64, outdated_part_loading_thread_pool_queue_size, 10000, "Queue size for parts loading thread pool.", 0) \ M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \ diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index 55c6a848b77..f7324bfafe6 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -13,7 +13,7 @@ namespace DB template > using ThreadPoolCallbackRunner = std::function(Callback &&, Priority)>; -/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'. +/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrowOnError()'. template > ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name) { @@ -44,7 +44,7 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & auto future = task->get_future(); - my_pool->scheduleOrThrow([my_task = std::move(task)]{ (*my_task)(); }, priority); + my_pool->scheduleOrThrowOnError([my_task = std::move(task)]{ (*my_task)(); }, priority); return future; }; From c1958c8bed68529c04c37a4b81d139088da3f2f9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Jun 2023 14:24:32 +0200 Subject: [PATCH 1014/2223] Remove 02764_parallel_replicas_plain_merge_tree from list of broken tests --- tests/broken_tests.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index 02935712325..96219323700 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -135,6 +135,5 @@ 02703_row_policy_for_database 02721_url_cluster 02534_s3_cluster_insert_select_schema_inference -02764_parallel_replicas_plain_merge_tree 02765_parallel_replicas_final_modifier From 65cc92a78d89f088c6c160dd4cb1748f48ed726d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 11:30:05 +0000 Subject: [PATCH 1015/2223] CI: Fix aspell on nested docs --- .../database-engines/materialized-mysql.md | 2 +- .../materialized-postgresql.md | 4 +- docs/en/engines/database-engines/sqlite.md | 2 +- .../table-engines/integrations/hdfs.md | 4 +- .../table-engines/integrations/hive.md | 2 +- .../table-engines/integrations/nats.md | 4 +- .../table-engines/integrations/postgresql.md | 2 +- .../engines/table-engines/integrations/s3.md | 6 +- .../mergetree-family/annindexes.md | 4 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/mergetree.md | 4 +- .../table-engines/special/executable.md | 4 +- .../table-engines/special/keepermap.md | 2 +- .../example-datasets/amazon-reviews.md | 4 +- .../example-datasets/cell-towers.md | 4 +- .../example-datasets/github.md | 4 +- .../example-datasets/opensky.md | 2 +- .../example-datasets/reddit-comments.md | 4 +- .../example-datasets/youtube-dislikes.md | 6 +- docs/en/operations/settings/settings.md | 2 +- docs/en/operations/system-tables/columns.md | 2 +- .../system-tables/dropped_tables.md | 2 +- .../system-tables/information_schema.md | 2 +- docs/en/operations/system-tables/licenses.md | 2 +- docs/en/operations/system-tables/parts.md | 2 +- docs/en/operations/system-tables/tables.md | 2 +- docs/en/operations/system-tables/trace_log.md | 2 +- .../operations/system-tables/zookeeper_log.md | 4 +- .../utilities/clickhouse-obfuscator.md | 2 +- .../aggregate-functions/reference/cramersv.md | 4 +- .../reference/cramersvbiascorrected.md | 2 +- .../reference/groupbitand.md | 2 +- .../reference/groupbitor.md | 2 +- .../reference/groupbitxor.md | 2 +- .../reference/kolmogorovsmirnovtest.md | 4 +- .../reference/quantiletdigestweighted.md | 2 +- .../reference/stochasticlinearregression.md | 2 +- docs/en/sql-reference/data-types/datetime.md | 8 +- docs/en/sql-reference/data-types/index.md | 2 +- docs/en/sql-reference/dictionaries/index.md | 2 +- .../sql-reference/functions/bit-functions.md | 4 +- .../functions/encryption-functions.md | 8 +- docs/en/sql-reference/functions/geo/h3.md | 2 +- docs/en/sql-reference/functions/geo/s2.md | 2 +- .../sql-reference/functions/hash-functions.md | 2 +- .../functions/logical-functions.md | 2 +- .../sql-reference/functions/math-functions.md | 2 +- .../functions/other-functions.md | 4 +- .../functions/string-functions.md | 2 +- .../functions/string-replace-functions.md | 2 +- docs/en/sql-reference/functions/udf.md | 2 +- docs/en/sql-reference/operators/in.md | 2 +- .../sql-reference/statements/alter/quota.md | 2 +- .../sql-reference/statements/create/quota.md | 2 +- docs/en/sql-reference/statements/explain.md | 4 +- .../sql-reference/statements/select/from.md | 2 +- .../statements/select/order-by.md | 2 +- .../sql-reference/window-functions/index.md | 4 +- .../aspell-ignore/en/aspell-dict.txt | 2429 +++++++++++++++-- utils/check-style/check-doc-aspell | 3 + 60 files changed, 2300 insertions(+), 302 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 1f1c996d4bf..f7cc52e622e 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -119,7 +119,7 @@ When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree]( The data of TIME type in MySQL is converted to microseconds in ClickHouse. -Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws exception "Unhandled data type" and stops replication. +Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws an exception and stops replication. ## Specifics and Recommendations {#specifics-and-recommendations} diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 08e9f998626..33d75dc9582 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -55,7 +55,7 @@ ATTACH TABLE postgres_database.new_table; ``` :::warning -Before version 22.1, adding a table to replication left an unremoved temporary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in ClickHouse version before 22.1, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. This issue is fixed in 22.1. +Before version 22.1, adding a table to replication left a non-removed temporary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in ClickHouse version before 22.1, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. This issue is fixed in 22.1. ::: ## Dynamically removing tables from replication {#dynamically-removing-table-from-replication} @@ -257,7 +257,7 @@ Please note that this should be used only if it is actually needed. If there is 1. [CREATE PUBLICATION](https://postgrespro.ru/docs/postgresql/14/sql-createpublication) -- create query privilege. -2. [CREATE_REPLICATION_SLOT](https://postgrespro.ru/docs/postgrespro/10/protocol-replication#PROTOCOL-REPLICATION-CREATE-SLOT) -- replication privelege. +2. [CREATE_REPLICATION_SLOT](https://postgrespro.ru/docs/postgrespro/10/protocol-replication#PROTOCOL-REPLICATION-CREATE-SLOT) -- replication privilege. 3. [pg_drop_replication_slot](https://postgrespro.ru/docs/postgrespro/9.5/functions-admin#functions-replication) -- replication privilege or superuser. diff --git a/docs/en/engines/database-engines/sqlite.md b/docs/en/engines/database-engines/sqlite.md index fc2a6525a68..0fa3c0fff58 100644 --- a/docs/en/engines/database-engines/sqlite.md +++ b/docs/en/engines/database-engines/sqlite.md @@ -30,7 +30,7 @@ Allows to connect to [SQLite](https://www.sqlite.org/index.html) database and pe ## Specifics and Recommendations {#specifics-and-recommendations} -SQLite stores the entire database (definitions, tables, indices, and the data itself) as a single cross-platform file on a host machine. During writing SQLite locks the entire database file, therefore write operations are performed sequentially. Read operations can be multitasked. +SQLite stores the entire database (definitions, tables, indices, and the data itself) as a single cross-platform file on a host machine. During writing SQLite locks the entire database file, therefore write operations are performed sequentially. Read operations can be multi-tasked. SQLite does not require service management (such as startup scripts) or access control based on `GRANT` and passwords. Access control is handled by means of file-system permissions given to the database file itself. ## Usage Example {#usage-example} diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index b9db0fae68f..08cd88826e5 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -156,7 +156,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us | rpc\_client\_connect\_timeout | 600 * 1000 | | rpc\_client\_read\_timeout | 3600 * 1000 | | rpc\_client\_write\_timeout | 3600 * 1000 | -| rpc\_client\_socekt\_linger\_timeout | -1 | +| rpc\_client\_socket\_linger\_timeout | -1 | | rpc\_client\_connect\_retry | 10 | | rpc\_client\_timeout | 3600 * 1000 | | dfs\_default\_replica | 3 | @@ -176,7 +176,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us | output\_write\_timeout | 3600 * 1000 | | output\_close\_timeout | 3600 * 1000 | | output\_packetpool\_size | 1024 | -| output\_heeartbeat\_interval | 10 * 1000 | +| output\_heartbeat\_interval | 10 * 1000 | | dfs\_client\_failover\_max\_attempts | 15 | | dfs\_client\_read\_shortcircuit\_streams\_cache\_size | 256 | | dfs\_client\_socketcache\_expiryMsec | 3000 | diff --git a/docs/en/engines/table-engines/integrations/hive.md b/docs/en/engines/table-engines/integrations/hive.md index adcb73605bb..5d10e417ae3 100644 --- a/docs/en/engines/table-engines/integrations/hive.md +++ b/docs/en/engines/table-engines/integrations/hive.md @@ -6,7 +6,7 @@ sidebar_label: Hive # Hive -The Hive engine allows you to perform `SELECT` quries on HDFS Hive table. Currently it supports input formats as below: +The Hive engine allows you to perform `SELECT` queries on HDFS Hive table. Currently it supports input formats as below: - Text: only supports simple scalar column types except `binary` diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md index 7f09c516d6f..a82d74e0d95 100644 --- a/docs/en/engines/table-engines/integrations/nats.md +++ b/docs/en/engines/table-engines/integrations/nats.md @@ -10,7 +10,7 @@ This engine allows integrating ClickHouse with [NATS](https://nats.io/). `NATS` lets you: -- Publish or subcribe to message subjects. +- Publish or subscribe to message subjects. - Process new messages as they become available. ## Creating a Table {#table_engine-redisstreams-creating-a-table} @@ -46,7 +46,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Required parameters: - `nats_url` – host:port (for example, `localhost:5672`).. -- `nats_subjects` – List of subject for NATS table to subscribe/publsh to. Supports wildcard subjects like `foo.*.bar` or `baz.>` +- `nats_subjects` – List of subject for NATS table to subscribe/publish to. Supports wildcard subjects like `foo.*.bar` or `baz.>` - `nats_format` – Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section. Optional parameters: diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index f27d4d48f75..51b3048706f 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -57,7 +57,7 @@ or via config (since version 21.11): ``` -Some parameters can be overriden by key value arguments: +Some parameters can be overridden by key value arguments: ``` sql SELECT * FROM postgresql(postgres1, schema='schema1', table='table1'); ``` diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 595bc0c344f..f2eaacd92a5 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -23,7 +23,7 @@ CREATE TABLE s3_engine_table (name String, value UInt32) - `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. +- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will auto-detect compression by file extension. ### PARTITION BY @@ -140,8 +140,8 @@ The following settings can be set before query execution or placed into configur - `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). - `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. - `s3_upload_part_size_multiply_factor` - Multiply `s3_min_upload_part_size` by this factor each time `s3_multiply_parts_count_threshold` parts were uploaded from a single write to S3. Default values is `2`. -- `s3_upload_part_size_multiply_parts_count_threshold` - Each time this number of parts was uploaded to S3 `s3_min_upload_part_size multiplied` by `s3_upload_part_size_multiply_factor`. DEfault value us `500`. -- `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurenly for one object. Its number should be limited. The value `0` means unlimited. Default value is `20`. Each inflight part has a buffer with size `s3_min_upload_part_size` for the first `s3_upload_part_size_multiply_factor` parts and more when file is big enought, see `upload_part_size_multiply_factor`. With default settings one uploaded file consumes not more than `320Mb` for a file which is less than `8G`. The consumption is greater for a larger file. +- `s3_upload_part_size_multiply_parts_count_threshold` - Each time this number of parts was uploaded to S3 `s3_min_upload_part_size multiplied` by `s3_upload_part_size_multiply_factor`. Default value us `500`. +- `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurrently for one object. Its number should be limited. The value `0` means unlimited. Default value is `20`. Each in-flight part has a buffer with size `s3_min_upload_part_size` for the first `s3_upload_part_size_multiply_factor` parts and more when file is big enough, see `upload_part_size_multiply_factor`. With default settings one uploaded file consumes not more than `320Mb` for a file which is less than `8G`. The consumption is greater for a larger file. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index f841f157376..03617a1a709 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -78,7 +78,7 @@ ENGINE = MergeTree ORDER BY id; ``` -With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyperparameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes. +With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyper parameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes. As the indexes are built only during insertions into table, `INSERT` and `OPTIMIZE` queries are slower than for ordinary table. At this stage indexes remember all the information about the given data. ANNIndexes should be used if you have immutable or rarely changed data and many read requests. @@ -135,7 +135,7 @@ ORDER BY id; Annoy supports `L2Distance` and `cosineDistance`. -In the `SELECT` in the settings (`ann_index_select_query_params`) you can specify the size of the internal buffer (more details in the description above or in the [original repository](https://github.com/spotify/annoy)). During the query it will inspect up to `search_k` nodes which defaults to `n_trees * n` if not provided. `search_k` gives you a run-time tradeoff between better accuracy and speed. +In the `SELECT` in the settings (`ann_index_select_query_params`) you can specify the size of the internal buffer (more details in the description above or in the [original repository](https://github.com/spotify/annoy)). During the query it will inspect up to `search_k` nodes which defaults to `n_trees * n` if not provided. `search_k` gives you a run-time trade-off between better accuracy and speed. __Example__: ``` sql diff --git a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md index edb320a2507..7e564b23676 100644 --- a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -165,7 +165,7 @@ Performance of such a query heavily depends on the table layout. Because of that The key factors for a good performance: -- number of partitions involved in the query should be sufficiently large (more than `max_threads / 2`), otherwise query will underutilize the machine +- number of partitions involved in the query should be sufficiently large (more than `max_threads / 2`), otherwise query will under-utilize the machine - partitions shouldn't be too small, so batch processing won't degenerate into row-by-row processing - partitions should be comparable in size, so all threads will do roughly the same amount of work diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 79ced0b6ce5..1ab0f4057ff 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -856,7 +856,7 @@ Tags: - `perform_ttl_move_on_insert` — Disables TTL move on data part INSERT. By default if we insert a data part that already expired by the TTL move rule it immediately goes to a volume/disk declared in move rule. This can significantly slowdown insert in case if destination volume/disk is slow (e.g. S3). - `load_balancing` - Policy for disk balancing, `round_robin` or `least_used`. -Cofiguration examples: +Configuration examples: ``` xml @@ -1224,7 +1224,7 @@ Limit parameters (mainly for internal usage): * `max_single_read_retries` - Limits the number of attempts to read a chunk of data from Blob Storage. * `max_single_download_retries` - Limits the number of attempts to download a readable buffer from Blob Storage. * `thread_pool_size` - Limits the number of threads with which `IDiskRemote` is instantiated. -* `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurenly for one object. +* `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurrently for one object. Other parameters: * `metadata_path` - Path on local FS to store metadata files for Blob Storage. Default value is `/var/lib/clickhouse/disks//`. diff --git a/docs/en/engines/table-engines/special/executable.md b/docs/en/engines/table-engines/special/executable.md index 25049d7b46e..d976beeab6c 100644 --- a/docs/en/engines/table-engines/special/executable.md +++ b/docs/en/engines/table-engines/special/executable.md @@ -65,7 +65,7 @@ if __name__ == "__main__": main() ``` -The following `my_executable_table` is built from the output of `my_script.py`, which will generate 10 random strings everytime you run a `SELECT` from `my_executable_table`: +The following `my_executable_table` is built from the output of `my_script.py`, which will generate 10 random strings every time you run a `SELECT` from `my_executable_table`: ```sql CREATE TABLE my_executable_table ( @@ -223,4 +223,4 @@ SETTINGS pool_size = 4; ``` -ClickHouse will maintain 4 processes on-demand when your client queries the `sentiment_pooled` table. \ No newline at end of file +ClickHouse will maintain 4 processes on-demand when your client queries the `sentiment_pooled` table. diff --git a/docs/en/engines/table-engines/special/keepermap.md b/docs/en/engines/table-engines/special/keepermap.md index a1c7009b712..6ce56adbae9 100644 --- a/docs/en/engines/table-engines/special/keepermap.md +++ b/docs/en/engines/table-engines/special/keepermap.md @@ -72,7 +72,7 @@ Additionally, number of keys will have a soft limit of 4 for the number of keys. If multiple tables are created on the same ZooKeeper path, the values are persisted until there exists at least 1 table using it. As a result, it is possible to use `ON CLUSTER` clause when creating the table and sharing the data from multiple ClickHouse instances. -Of course, it's possible to manually run `CREATE TABLE` with same path on nonrelated ClickHouse instances to have same data sharing effect. +Of course, it's possible to manually run `CREATE TABLE` with same path on unrelated ClickHouse instances to have same data sharing effect. ## Supported operations {#table_engine-KeeperMap-supported-operations} diff --git a/docs/en/getting-started/example-datasets/amazon-reviews.md b/docs/en/getting-started/example-datasets/amazon-reviews.md index f35806aa66f..75e4549cb78 100644 --- a/docs/en/getting-started/example-datasets/amazon-reviews.md +++ b/docs/en/getting-started/example-datasets/amazon-reviews.md @@ -87,7 +87,7 @@ ORDER BY (marketplace, review_date, product_category); 3. We are now ready to insert the data into ClickHouse. Before we do, check out the [list of files in the dataset](https://s3.amazonaws.com/amazon-reviews-pds/tsv/index.txt) and decide which ones you want to include. -4. We will insert all of the US reviews - which is about 151M rows. The following `INSERT` command uses the `s3Cluster` table function, which allows the processing of mulitple S3 files in parallel using all the nodes of your cluster. We also use a wildcard to insert any file that starts with the name `https://s3.amazonaws.com/amazon-reviews-pds/tsv/amazon_reviews_us_`: +4. We will insert all of the US reviews - which is about 151M rows. The following `INSERT` command uses the `s3Cluster` table function, which allows the processing of multiple S3 files in parallel using all the nodes of your cluster. We also use a wildcard to insert any file that starts with the name `https://s3.amazonaws.com/amazon-reviews-pds/tsv/amazon_reviews_us_`: ```sql INSERT INTO amazon_reviews @@ -473,4 +473,4 @@ It runs quite a bit faster - which means the cache is helping us out here: └────────────┴───────────────────────────────────────────────────────────────────────┴────────────────────┴───────┘ 50 rows in set. Elapsed: 33.954 sec. Processed 150.96 million rows, 68.95 GB (4.45 million rows/s., 2.03 GB/s.) -``` \ No newline at end of file +``` diff --git a/docs/en/getting-started/example-datasets/cell-towers.md b/docs/en/getting-started/example-datasets/cell-towers.md index 048eecb285b..a84eb5d561f 100644 --- a/docs/en/getting-started/example-datasets/cell-towers.md +++ b/docs/en/getting-started/example-datasets/cell-towers.md @@ -317,7 +317,7 @@ To build a Superset dashboard using the OpenCelliD dataset you should: Make sure that you set **SSL** on when connecting to ClickHouse Cloud or other ClickHouse systems that enforce the use of SSL. ::: - ![Add ClickHouse as a Superset datasource](@site/docs/en/getting-started/example-datasets/images/superset-connect-a-database.png) + ![Add ClickHouse as a Superset data source](@site/docs/en/getting-started/example-datasets/images/superset-connect-a-database.png) ### Add the table **cell_towers** as a Superset **dataset** @@ -364,5 +364,5 @@ The data is also available for interactive queries in the [Playground](https://p This [example](https://play.clickhouse.com/play?user=play#U0VMRUNUIG1jYywgY291bnQoKSBGUk9NIGNlbGxfdG93ZXJzIEdST1VQIEJZIG1jYyBPUkRFUiBCWSBjb3VudCgpIERFU0M=) will populate the username and even the query for you. -Although you cannot create tables in the Playground, you can run all of the queries and even use Superset (adjust the hostname and port number). +Although you cannot create tables in the Playground, you can run all of the queries and even use Superset (adjust the host name and port number). ::: diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index 02965ed5e33..9ed8782e512 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -806,7 +806,7 @@ FROM 31 rows in set. Elapsed: 0.043 sec. Processed 7.54 million rows, 40.53 MB (176.71 million rows/s., 950.40 MB/s.) ``` -Maybe a little more near the end of the month, but overall we keep a good even distribution. Again this is unrealiable due to the filtering of the docs filter during data insertion. +Maybe a little more near the end of the month, but overall we keep a good even distribution. Again this is unreliable due to the filtering of the docs filter during data insertion. ## Authors with the most diverse impact @@ -940,7 +940,7 @@ LIMIT 10 10 rows in set. Elapsed: 0.106 sec. Processed 798.15 thousand rows, 13.97 MB (7.51 million rows/s., 131.41 MB/s.) ``` -This makes sense because Alexey has been responsible for maintaining the Change log. But what if we use the basename of the file to identify his popular files - this allows for renames and should focus on code contributions. +This makes sense because Alexey has been responsible for maintaining the Change log. But what if we use the base name of the file to identify his popular files - this allows for renames and should focus on code contributions. [play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBiYXNlLAogICAgY291bnQoKSBBUyBjCkZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCldIRVJFIChhdXRob3IgPSAnQWxleGV5IE1pbG92aWRvdicpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkKR1JPVVAgQlkgYmFzZW5hbWUocGF0aCkgQVMgYmFzZQpPUkRFUiBCWSBjIERFU0MKTElNSVQgMTA=) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index 7093a2df04f..df28809495c 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -9,7 +9,7 @@ The data in this dataset is derived and cleaned from the full OpenSky dataset to Source: https://zenodo.org/record/5092942#.YRBCyTpRXYd -Martin Strohmeier, Xavier Olive, Jannis Lübbe, Matthias Schäfer, and Vincent Lenders +Martin Strohmeier, Xavier Olive, Jannis Luebbe, Matthias Schaefer, and Vincent Lenders "Crowdsourced air traffic data from the OpenSky Network 2019–2020" Earth System Science Data 13(2), 2021 https://doi.org/10.5194/essd-13-357-2021 diff --git a/docs/en/getting-started/example-datasets/reddit-comments.md b/docs/en/getting-started/example-datasets/reddit-comments.md index e1e372746c9..49c7bd25f9f 100644 --- a/docs/en/getting-started/example-datasets/reddit-comments.md +++ b/docs/en/getting-started/example-datasets/reddit-comments.md @@ -469,7 +469,7 @@ The response is: 10 rows in set. Elapsed: 27.824 sec. Processed 6.74 billion rows, 53.26 GB (242.22 million rows/s., 1.91 GB/s.) ``` -11. Let's see which subreddits had the biggest increase in commnents from 2018 to 2019: +11. Let's see which subreddits had the biggest increase in comments from 2018 to 2019: ```sql SELECT @@ -633,4 +633,4 @@ ORDER BY quarter ASC; └────────────┴────────────┴───────────┴──────────┘ 58 rows in set. Elapsed: 2663.751 sec. Processed 6.74 billion rows, 1.21 TB (2.53 million rows/s., 454.37 MB/s.) -``` \ No newline at end of file +``` diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index 5f4ef696b8b..e24c6e5a6dc 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -22,7 +22,7 @@ The steps below will easily work on a local install of ClickHouse too. The only ## Step-by-step instructions -1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the reult: +1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the result: ```sql DESCRIBE s3Cluster( @@ -322,7 +322,7 @@ ORDER BY month ASC; A spike of uploaders [around covid is noticeable](https://www.theverge.com/2020/3/27/21197642/youtube-with-me-style-videos-views-coronavirus-cook-workout-study-home-beauty). -### More subtitiles over time and when +### More subtitles over time and when With advances in speech recognition, it’s easier than ever to create subtitles for video with youtube adding auto-captioning in late 2009 - was the jump then? @@ -484,4 +484,4 @@ ARRAY JOIN │ 20th │ 16 │ │ 10th │ 6 │ └────────────┴─────────┘ -``` \ No newline at end of file +``` diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4f3b4e43358..ad113c58bce 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3492,7 +3492,7 @@ Default value: `0`. ## database_replicated_initial_query_timeout_sec {#database_replicated_initial_query_timeout_sec} -Sets how long initial DDL query should wait for Replicated database to precess previous DDL queue entries in seconds. +Sets how long initial DDL query should wait for Replicated database to process previous DDL queue entries in seconds. Possible values: diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index ccdc2d8c742..2915b053458 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -28,7 +28,7 @@ The `system.columns` table contains the following columns (the column type is sh - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name. - `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bit width for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned. - `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned. diff --git a/docs/en/operations/system-tables/dropped_tables.md b/docs/en/operations/system-tables/dropped_tables.md index 144c03109ac..e2a09094c87 100644 --- a/docs/en/operations/system-tables/dropped_tables.md +++ b/docs/en/operations/system-tables/dropped_tables.md @@ -12,7 +12,7 @@ Columns: - `table` ([String](../../sql-reference/data-types/string.md)) — Table name. - `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Table uuid. - `engine` ([String](../../sql-reference/data-types/string.md)) — Table engine name. -- `metadata_dropped_path` ([String](../../sql-reference/data-types/string.md)) — Path of table's metadata file in metadate_dropped directory. +- `metadata_dropped_path` ([String](../../sql-reference/data-types/string.md)) — Path of table's metadata file in metadata_dropped directory. - `table_dropped_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The time when the next attempt to remove table's data is scheduled on. Usually it's the table when the table was dropped plus `database_atomic_delay_before_drop_table_sec` **Example** diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index 07e9a9e2f58..35fd3a753b5 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -43,7 +43,7 @@ Columns: - `data_type` ([String](../../sql-reference/data-types/string.md)) — Column type. - `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. - `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bit width for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned. - `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned. diff --git a/docs/en/operations/system-tables/licenses.md b/docs/en/operations/system-tables/licenses.md index 9296e78c797..0f09d559d8b 100644 --- a/docs/en/operations/system-tables/licenses.md +++ b/docs/en/operations/system-tables/licenses.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/licenses --- # licenses -Сontains licenses of third-party libraries that are located in the [contrib](https://github.com/ClickHouse/ClickHouse/tree/master/contrib) directory of ClickHouse sources. +Contains licenses of third-party libraries that are located in the [contrib](https://github.com/ClickHouse/ClickHouse/tree/master/contrib) directory of ClickHouse sources. Columns: diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index e61c6ed2ba4..9159d1e9284 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -100,7 +100,7 @@ Columns: - `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of expressions. Each expression defines a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). :::note -The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simpliest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. +The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simplest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. ::: - `move_ttl_info.min` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the minimum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). diff --git a/docs/en/operations/system-tables/tables.md b/docs/en/operations/system-tables/tables.md index 82e9fa206ea..e4461e14236 100644 --- a/docs/en/operations/system-tables/tables.md +++ b/docs/en/operations/system-tables/tables.md @@ -50,7 +50,7 @@ Columns: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - [Distributed](../../engines/table-engines/special/distributed.md#distributed) -- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `NULL` (including underying `Buffer` table). +- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `NULL` (including underlying `Buffer` table). - `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of bytes, if it is possible to quickly determine exact number of bytes for the table on storage, otherwise `NULL` (does not includes any underlying storage). diff --git a/docs/en/operations/system-tables/trace_log.md b/docs/en/operations/system-tables/trace_log.md index a5aae422be7..89d54adc30d 100644 --- a/docs/en/operations/system-tables/trace_log.md +++ b/docs/en/operations/system-tables/trace_log.md @@ -43,7 +43,7 @@ Columns: - `event` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) - For trace type `ProfileEvent` is the name of updated profile event, for other trace types is an empty string. -- `increment` ([UInt64](../../sql-reference/data-types/int-uint.md)) - For trace type `ProfileEvent` is the amount of incremnt of profile event, for other trace types is 0. +- `increment` ([UInt64](../../sql-reference/data-types/int-uint.md)) - For trace type `ProfileEvent` is the amount of increment of profile event, for other trace types is 0. **Example** diff --git a/docs/en/operations/system-tables/zookeeper_log.md b/docs/en/operations/system-tables/zookeeper_log.md index 970ed192a48..b7cc4e22cd6 100644 --- a/docs/en/operations/system-tables/zookeeper_log.md +++ b/docs/en/operations/system-tables/zookeeper_log.md @@ -33,7 +33,7 @@ Columns with request response parameters: - `zxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — ZooKeeper transaction ID. The serial number issued by the ZooKeeper server in response to a successfully executed request (`0` if the request was not executed/returned an error/the client does not know whether the request was executed). - `error` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — Error code. Can have many values, here are just some of them: - - `ZOK` — The request was executed seccessfully. + - `ZOK` — The request was executed successfully. - `ZCONNECTIONLOSS` — The connection was lost. - `ZOPERATIONTIMEOUT` — The request execution timeout has expired. - `ZSESSIONEXPIRED` — The session has expired. @@ -43,7 +43,7 @@ Columns with request response parameters: - `path_created` ([String](../../sql-reference/data-types/string.md)) — The path to the created ZooKeeper node (for responses to the `CREATE` request), may differ from the `path` if the node is created as a `sequential`. - `stat_czxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The `zxid` of the change that caused this ZooKeeper node to be created. - `stat_mzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The `zxid` of the change that last modified this ZooKeeper node. -- `stat_pzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The transaction ID of the change that last modified childern of this ZooKeeper node. +- `stat_pzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The transaction ID of the change that last modified children of this ZooKeeper node. - `stat_version` ([Int32](../../sql-reference/data-types/int-uint.md)) — The number of changes to the data of this ZooKeeper node. - `stat_cversion` ([Int32](../../sql-reference/data-types/int-uint.md)) — The number of changes to the children of this ZooKeeper node. - `stat_dataLength` ([Int32](../../sql-reference/data-types/int-uint.md)) — The length of the data field of this ZooKeeper node. diff --git a/docs/en/operations/utilities/clickhouse-obfuscator.md b/docs/en/operations/utilities/clickhouse-obfuscator.md index 077489ba76e..ad51e9c7776 100644 --- a/docs/en/operations/utilities/clickhouse-obfuscator.md +++ b/docs/en/operations/utilities/clickhouse-obfuscator.md @@ -24,7 +24,7 @@ It is designed to retain the following properties of data: Most of the properties above are viable for performance testing: -reading data, filtering, aggregatio, and sorting will work at almost the same speed +reading data, filtering, aggregation, and sorting will work at almost the same speed as on original data due to saved cardinalities, magnitudes, compression ratios, etc. It works in a deterministic fashion: you define a seed value and the transformation is determined by input data and by seed. diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersv.md b/docs/en/sql-reference/aggregate-functions/reference/cramersv.md index f412724ea08..e9e2c367610 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersv.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersv.md @@ -5,7 +5,7 @@ sidebar_position: 351 # cramersV -[Cramér's V](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V) (sometimes referred to as Cramér's phi) is a measure of association between two columns in a table. The result of the `cramersV` function ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. It may be viewed as the association between two variables as a percentage of their maximum possible variation. +[Cramer's V](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V) (sometimes referred to as Cramer's phi) is a measure of association between two columns in a table. The result of the `cramersV` function ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. It may be viewed as the association between two variables as a percentage of their maximum possible variation. **Syntax** @@ -69,4 +69,4 @@ Result: ┌─────cramersV(a, b)─┐ │ 0.8944271909999159 │ └────────────────────┘ -``` \ No newline at end of file +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md index 8e577efbc4d..f5ad3a8a937 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md @@ -6,7 +6,7 @@ sidebar_position: 352 # cramersVBiasCorrected -Cramér's V is a measure of association between two columns in a table. The result of the [`cramersV` function](./cramersv.md) ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. The function can be heavily biased, so this version of Cramér's V uses the [bias correction](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V#Bias_correction). +Cramer's V is a measure of association between two columns in a table. The result of the [`cramersV` function](./cramersv.md) ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. The function can be heavily biased, so this version of Cramer's V uses the [bias correction](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V#Bias_correction). diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md b/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md index 5f57407a419..3d833555a43 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md @@ -5,7 +5,7 @@ sidebar_position: 125 # groupBitAnd -Applies bitwise `AND` for series of numbers. +Applies bit-wise `AND` for series of numbers. ``` sql groupBitAnd(expr) diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md b/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md index 59be69540b0..138ee998405 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md @@ -5,7 +5,7 @@ sidebar_position: 126 # groupBitOr -Applies bitwise `OR` for series of numbers. +Applies bit-wise `OR` for series of numbers. ``` sql groupBitOr(expr) diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md b/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md index b00876a2fdf..168335a010c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md @@ -5,7 +5,7 @@ sidebar_position: 127 # groupBitXor -Applies bitwise `XOR` for series of numbers. +Applies bit-wise `XOR` for series of numbers. ``` sql groupBitXor(expr) diff --git a/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md b/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md index 3da9645181e..d159eec7ce6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md +++ b/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md @@ -30,11 +30,11 @@ Samples must belong to continuous, one-dimensional probability distributions. The null hypothesis is that samples come from the same distribution, e.g. F(x) = G(x) for all x. And the alternative is that the distributions are not identical. - `'greater'` - The null hypothesis is that values in the first sample are *stohastically smaller* than those in the second one, + The null hypothesis is that values in the first sample are *stochastically smaller* than those in the second one, e.g. the CDF of first distribution lies above and hence to the left of that for the second one. Which in fact means that F(x) >= G(x) for all x. And the alternative in this case is that F(x) < G(x) for at least one x. - `'less'`. - The null hypothesis is that values in the first sample are *stohastically greater* than those in the second one, + The null hypothesis is that values in the first sample are *stochastically greater* than those in the second one, e.g. the CDF of first distribution lies below and hence to the right of that for the second one. Which in fact means that F(x) <= G(x) for all x. And the alternative in this case is that F(x) > G(x) for at least one x. - `computation_method` — the method used to compute p-value. (Optional, default: `'auto'`.) [String](../../../sql-reference/data-types/string.md). diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md index 7b9addbbdde..b3e21e0e69e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md @@ -14,7 +14,7 @@ The result depends on the order of running the query, and is nondeterministic. When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. :::note -Using `quantileTDigestWeighted` [is not recommended for tiny data sets](https://github.com/tdunning/t-digest/issues/167#issuecomment-828650275) and can lead to significat error. In this case, consider possibility of using [`quantileTDigest`](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) instead. +Using `quantileTDigestWeighted` [is not recommended for tiny data sets](https://github.com/tdunning/t-digest/issues/167#issuecomment-828650275) and can lead to significant error. In this case, consider possibility of using [`quantileTDigest`](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) instead. ::: **Syntax** diff --git a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md index 9481172c25b..f7615d90790 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md @@ -18,7 +18,7 @@ stochasticLinearRegression(1.0, 1.0, 10, 'SGD') 1. `learning rate` is the coefficient on step length, when gradient descent step is performed. Too big learning rate may cause infinite weights of the model. Default is `0.00001`. 2. `l2 regularization coefficient` which may help to prevent overfitting. Default is `0.1`. 3. `mini-batch size` sets the number of elements, which gradients will be computed and summed to perform one step of gradient descent. Pure stochastic descent uses one element, however having small batches(about 10 elements) make gradient steps more stable. Default is `15`. -4. `method for updating weights`, they are: `Adam` (by default), `SGD`, `Momentum`, `Nesterov`. `Momentum` and `Nesterov` require little bit more computations and memory, however they happen to be useful in terms of speed of convergance and stability of stochastic gradient methods. +4. `method for updating weights`, they are: `Adam` (by default), `SGD`, `Momentum`, `Nesterov`. `Momentum` and `Nesterov` require little bit more computations and memory, however they happen to be useful in terms of speed of convergence and stability of stochastic gradient methods. ### Usage diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index 059c6acdb9e..0da273e01ad 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -22,7 +22,7 @@ Resolution: 1 second. The point in time is saved as a [Unix timestamp](https://en.wikipedia.org/wiki/Unix_time), regardless of the time zone or daylight saving time. The time zone affects how the values of the `DateTime` type values are displayed in text format and how the values specified as strings are parsed (‘2020-01-01 05:00:01’). -Timezone agnostic unix timestamp is stored in tables, and the timezone is used to transform it to text format or back during data import/export or to make calendar calculations on the values (example: `toDate`, `toHour` functions et cetera). The time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. +Timezone agnostic Unix timestamp is stored in tables, and the timezone is used to transform it to text format or back during data import/export or to make calendar calculations on the values (example: `toDate`, `toHour` functions etc.). The time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. A list of supported time zones can be found in the [IANA Time Zone Database](https://www.iana.org/time-zones) and also can be queried by `SELECT * FROM system.time_zones`. [The list](https://en.wikipedia.org/wiki/List_of_tz_database_time_zones) is also available at Wikipedia. @@ -30,7 +30,7 @@ You can explicitly set a time zone for `DateTime`-type columns when creating a t The [clickhouse-client](../../interfaces/cli.md) applies the server time zone by default if a time zone isn’t explicitly set when initializing the data type. To use the client time zone, run `clickhouse-client` with the `--use_client_time_zone` parameter. -ClickHouse outputs values depending on the value of the [date_time_output_format](../../operations/settings/settings.md#settings-date_time_output_format) setting. `YYYY-MM-DD hh:mm:ss` text format by default. Additionaly you can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function. +ClickHouse outputs values depending on the value of the [date_time_output_format](../../operations/settings/settings.md#settings-date_time_output_format) setting. `YYYY-MM-DD hh:mm:ss` text format by default. Additionally, you can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function. When inserting data into ClickHouse, you can use different formats of date and time strings, depending on the value of the [date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format) setting. @@ -120,9 +120,9 @@ FROM dt As timezone conversion only changes the metadata, the operation has no computation cost. -## Limitations on timezones support +## Limitations on time zones support -Some timezones may not be supported completely. There are a few cases: +Some time zones may not be supported completely. There are a few cases: If the offset from UTC is not a multiple of 15 minutes, the calculation of hours and minutes can be incorrect. For example, the time zone in Monrovia, Liberia has offset UTC -0:44:30 before 7 Jan 1972. If you are doing calculations on the historical time in Monrovia timezone, the time processing functions may give incorrect results. The results after 7 Jan 1972 will be correct nevertheless. diff --git a/docs/en/sql-reference/data-types/index.md b/docs/en/sql-reference/data-types/index.md index 88663968e50..508307a0543 100644 --- a/docs/en/sql-reference/data-types/index.md +++ b/docs/en/sql-reference/data-types/index.md @@ -27,7 +27,7 @@ ClickHouse data types include: - **Aggregation function types**: use [`SimpleAggregateFunction`](./simpleaggregatefunction.md) and [`AggregateFunction`](./aggregatefunction.md) for storing the intermediate status of aggregate function results - **Nested data structures**: A [`Nested` data structure](./nested-data-structures/index.md) is like a table inside a cell - **Tuples**: A [`Tuple` of elements](./tuple.md), each having an individual type. -- **Nullable**: [`Nullable`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column gettings its default value for the data type) +- **Nullable**: [`Nullable`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column settings its default value for the data type) - **IP addresses**: use [`IPv4`](./domains/ipv4.md) and [`IPv6`](./domains/ipv6.md) to efficiently store IP addresses - **Geo types**: for [geographical data](./geo.md), including `Point`, `Ring`, `Polygon` and `MultiPolygon` - **Special data types**: including [`Expression`](./special-data-types/expression.md), [`Set`](./special-data-types/set.md), [`Nothing`](./special-data-types/nothing.md) and [`Interval`](./special-data-types/interval.md) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index f7b4be64851..3a968992c13 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -984,7 +984,7 @@ SOURCE(ODBC(... invalidate_query 'SELECT update_time FROM dictionary_source wher ... ``` -For `Cache`, `ComplexKeyCache`, `SSDCache`, and `SSDComplexKeyCache` dictionaries both synchronious and asynchronous updates are supported. +For `Cache`, `ComplexKeyCache`, `SSDCache`, and `SSDComplexKeyCache` dictionaries both synchronous and asynchronous updates are supported. It is also possible for `Flat`, `Hashed`, `ComplexKeyHashed` dictionaries to only request data that was changed after the previous update. If `update_field` is specified as part of the dictionary source configuration, value of the previous update time in seconds will be added to the data request. Depends on source type (Executable, HTTP, MySQL, PostgreSQL, ClickHouse, or ODBC) different logic will be applied to `update_field` before request data from an external source. diff --git a/docs/en/sql-reference/functions/bit-functions.md b/docs/en/sql-reference/functions/bit-functions.md index 5b342fe4f24..3c07fe8bcbe 100644 --- a/docs/en/sql-reference/functions/bit-functions.md +++ b/docs/en/sql-reference/functions/bit-functions.md @@ -226,7 +226,7 @@ Result: Returns result of [logical conjuction](https://en.wikipedia.org/wiki/Logical_conjunction) (AND operator) of all bits at given positions. The countdown starts from 0 from the right to the left. -The conjuction for bitwise operations: +The conjuction for bit-wise operations: 0 AND 0 = 0 @@ -291,7 +291,7 @@ Result: Returns result of [logical disjunction](https://en.wikipedia.org/wiki/Logical_disjunction) (OR operator) of all bits at given positions. The countdown starts from 0 from the right to the left. -The disjunction for bitwise operations: +The disjunction for bit-wise operations: 0 OR 0 = 0 diff --git a/docs/en/sql-reference/functions/encryption-functions.md b/docs/en/sql-reference/functions/encryption-functions.md index 1224b7bc92b..b11bee83582 100644 --- a/docs/en/sql-reference/functions/encryption-functions.md +++ b/docs/en/sql-reference/functions/encryption-functions.md @@ -31,9 +31,9 @@ encrypt('mode', 'plaintext', 'key' [, iv, aad]) **Arguments** - `mode` — Encryption mode. [String](../../sql-reference/data-types/string.md#string). -- `plaintext` — Text thats need to be encrypted. [String](../../sql-reference/data-types/string.md#string). +- `plaintext` — Text that need to be encrypted. [String](../../sql-reference/data-types/string.md#string). - `key` — Encryption key. [String](../../sql-reference/data-types/string.md#string). -- `iv` — Initialization vector. Required for `-gcm` modes, optinal for others. [String](../../sql-reference/data-types/string.md#string). +- `iv` — Initialization vector. Required for `-gcm` modes, optional for others. [String](../../sql-reference/data-types/string.md#string). - `aad` — Additional authenticated data. It isn't encrypted, but it affects decryption. Works only in `-gcm` modes, for others would throw an exception. [String](../../sql-reference/data-types/string.md#string). **Returned value** @@ -233,7 +233,7 @@ decrypt('mode', 'ciphertext', 'key' [, iv, aad]) - `mode` — Decryption mode. [String](../../sql-reference/data-types/string.md#string). - `ciphertext` — Encrypted text that needs to be decrypted. [String](../../sql-reference/data-types/string.md#string). - `key` — Decryption key. [String](../../sql-reference/data-types/string.md#string). -- `iv` — Initialization vector. Required for `-gcm` modes, optinal for others. [String](../../sql-reference/data-types/string.md#string). +- `iv` — Initialization vector. Required for `-gcm` modes, Optional for others. [String](../../sql-reference/data-types/string.md#string). - `aad` — Additional authenticated data. Won't decrypt if this value is incorrect. Works only in `-gcm` modes, for others would throw an exception. [String](../../sql-reference/data-types/string.md#string). **Returned value** @@ -364,7 +364,7 @@ aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv]) - `mode` — Decryption mode. [String](../../sql-reference/data-types/string.md#string). - `ciphertext` — Encrypted text that needs to be decrypted. [String](../../sql-reference/data-types/string.md#string). - `key` — Decryption key. [String](../../sql-reference/data-types/string.md#string). -- `iv` — Initialization vector. Optinal. [String](../../sql-reference/data-types/string.md#string). +- `iv` — Initialization vector. Optional. [String](../../sql-reference/data-types/string.md#string). **Returned value** diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index 1f695a13598..29486c58e6a 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -12,7 +12,7 @@ A latitude and longitude pair can be transformed to a 64-bit H3 index, identifyi The H3 index is used primarily for bucketing locations and other geospatial manipulations. -The full description of the H3 system is available at [the Uber Engeneering site](https://eng.uber.com/h3/). +The full description of the H3 system is available at [the Uber Engineering site](https://eng.uber.com/h3/). ## h3IsValid diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index 63fe5ca8530..f4702eff44b 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -249,7 +249,7 @@ s2RectAdd(s2pointLow, s2pointHigh, s2Point) **Returned values** - `s2PointLow` — Low S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2PointHigh` — Hight S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/float.md). +- `s2PointHigh` — Height S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/float.md). **Example** diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 89afcca3799..2cf3408534f 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -1161,7 +1161,7 @@ wordShingleSimHashUTF8(string[, shinglesize]) **Arguments** - `string` — String. [String](/docs/en/sql-reference/data-types/string.md). -- `shinglesize` — The size of a word shingle. Optinal. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](/docs/en/sql-reference/data-types/int-uint.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](/docs/en/sql-reference/data-types/int-uint.md). **Returned value** diff --git a/docs/en/sql-reference/functions/logical-functions.md b/docs/en/sql-reference/functions/logical-functions.md index f5a1a6aac12..17a9fbb19fe 100644 --- a/docs/en/sql-reference/functions/logical-functions.md +++ b/docs/en/sql-reference/functions/logical-functions.md @@ -31,7 +31,7 @@ Alias: The [AND Operator](../../sql-reference/operators/index.md#logical-and-ope **Returned value** - `0`, if there at least one argument evaluates to `false`, -- `NULL`, if no argumetn evaluates to `false` and at least one argument is `NULL`, +- `NULL`, if no argument evaluates to `false` and at least one argument is `NULL`, - `1`, otherwise. Type: [UInt8](../../sql-reference/data-types/int-uint.md) or [Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md)). diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 9851378d4fd..22492f2830b 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -52,7 +52,7 @@ Alias: `ln(x)` ## exp2 -Returns 2 to the power of the given argumetn +Returns 2 to the power of the given argument **Syntax** diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index efe1a77c285..41ccfe121a4 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -206,7 +206,7 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** -For [String](../../sql-reference/data-types/string.md) arguments the funtion returns the string length + 9 (terminating zero + length). +For [String](../../sql-reference/data-types/string.md) arguments the function returns the string length + 9 (terminating zero + length). Query: @@ -1352,7 +1352,7 @@ ORDER BY k ASC ClickHouse used the index in the same way as the previous time (`Processed 32.74 thousand rows`). The expression `k = '2017-09-15'` was not used when generating the result. -In examle the `indexHint` function allows to see adjacent dates. +In example the `indexHint` function allows to see adjacent dates. Result: diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 8662d08431c..5175bbf0615 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -393,7 +393,7 @@ Reverses a sequence of Unicode code points in a string. Assumes that the string ## format -Format the `pattern` string with the strings listed in the arguments, similar to formatting in Python. The pattern string can contain replacement fields surrounded by curly braces `{}`. Anything not contained in braces is considered literal text and copied verbatim into the output. Literal brace character can be escaped by two braces: `{{ '{{' }}` and `{{ '}}' }}`. Field names can be numbers (starting from zero) or empty (then they are implicitely given monotonically increasing numbers). +Format the `pattern` string with the strings listed in the arguments, similar to formatting in Python. The pattern string can contain replacement fields surrounded by curly braces `{}`. Anything not contained in braces is considered literal text and copied verbatim into the output. Literal brace character can be escaped by two braces: `{{ '{{' }}` and `{{ '}}' }}`. Field names can be numbers (starting from zero) or empty (then they are implicitly given monotonically increasing numbers). **Syntax** diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 56c527d734e..74d5d747193 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -6,7 +6,7 @@ sidebar_label: Replacing in Strings # Functions for Replacing in Strings -[General strings functions](string-functions.md) and [functions for searchin in strings](string-search-functions.md) are described separately. +[General strings functions](string-functions.md) and [functions for searching in strings](string-search-functions.md) are described separately. ## replaceOne diff --git a/docs/en/sql-reference/functions/udf.md b/docs/en/sql-reference/functions/udf.md index a58c1364780..9c6b1b0c66b 100644 --- a/docs/en/sql-reference/functions/udf.md +++ b/docs/en/sql-reference/functions/udf.md @@ -19,7 +19,7 @@ A function configuration contains the following settings: - `argument` - argument description with the `type`, and optional `name` of an argument. Each argument is described in a separate setting. Specifying name is necessary if argument names are part of serialization for user defined function format like [Native](../../interfaces/formats.md#native) or [JSONEachRow](../../interfaces/formats.md#jsoneachrow). Default argument name value is `c` + argument_number. - `format` - a [format](../../interfaces/formats.md) in which arguments are passed to the command. - `return_type` - the type of a returned value. -- `return_name` - name of retuned value. Specifying return name is necessary if return name is part of serialization for user defined function format like [Native](../../interfaces/formats.md#native) or [JSONEachRow](../../interfaces/formats.md#jsoneachrow). Optional. Default value is `result`. +- `return_name` - name of returned value. Specifying return name is necessary if return name is part of serialization for user defined function format like [Native](../../interfaces/formats.md#native) or [JSONEachRow](../../interfaces/formats.md#jsoneachrow). Optional. Default value is `result`. - `type` - an executable type. If `type` is set to `executable` then single command is started. If it is set to `executable_pool` then a pool of commands is created. - `max_command_execution_time` - maximum execution time in seconds for processing block of data. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`. - `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time `SIGTERM` is sent to the process executing the command. Optional. Default value is `10`. diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 8a8c86624d2..bfad16f8365 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -222,7 +222,7 @@ It also makes sense to specify a local table in the `GLOBAL IN` clause, in case ### Distributed Subqueries and max_rows_in_set -You can use [`max_rows_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) and [`max_bytes_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) to control how much data is tranferred during distributed queries. +You can use [`max_rows_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) and [`max_bytes_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) to control how much data is transferred during distributed queries. This is specially important if the `global in` query returns a large amount of data. Consider the following sql - ```sql diff --git a/docs/en/sql-reference/statements/alter/quota.md b/docs/en/sql-reference/statements/alter/quota.md index 74a184c1479..d41e2ff0f61 100644 --- a/docs/en/sql-reference/statements/alter/quota.md +++ b/docs/en/sql-reference/statements/alter/quota.md @@ -32,7 +32,7 @@ Limit the maximum number of queries for the current user with 123 queries in 15 ALTER QUOTA IF EXISTS qA FOR INTERVAL 15 month MAX queries = 123 TO CURRENT_USER; ``` -For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quaters: +For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quarters: ``` sql ALTER QUOTA IF EXISTS qB FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default; diff --git a/docs/en/sql-reference/statements/create/quota.md b/docs/en/sql-reference/statements/create/quota.md index 7c31f93fff7..c69285171ab 100644 --- a/docs/en/sql-reference/statements/create/quota.md +++ b/docs/en/sql-reference/statements/create/quota.md @@ -32,7 +32,7 @@ Limit the maximum number of queries for the current user with 123 queries in 15 CREATE QUOTA qA FOR INTERVAL 15 month MAX queries = 123 TO CURRENT_USER; ``` -For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quaters: +For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quarters: ``` sql CREATE QUOTA qB FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default; diff --git a/docs/en/sql-reference/statements/explain.md b/docs/en/sql-reference/statements/explain.md index 1c93707402f..2d7204c2796 100644 --- a/docs/en/sql-reference/statements/explain.md +++ b/docs/en/sql-reference/statements/explain.md @@ -115,7 +115,7 @@ CROSS JOIN system.numbers AS c Settings: -- `run_passes` — Run all query tree passes before dumping the query tree. Defaul: `1`. +- `run_passes` — Run all query tree passes before dumping the query tree. Default: `1`. - `dump_passes` — Dump information about used passes before dumping the query tree. Default: `0`. - `passes` — Specifies how many passes to run. If set to `-1`, runs all the passes. Default: `-1`. @@ -463,5 +463,5 @@ Result: ``` :::note -The validation is not complete, so a successfull query does not guarantee that the override would not cause issues. +The validation is not complete, so a successful query does not guarantee that the override would not cause issues. ::: diff --git a/docs/en/sql-reference/statements/select/from.md b/docs/en/sql-reference/statements/select/from.md index 4ca8e8287c0..a4f449ad321 100644 --- a/docs/en/sql-reference/statements/select/from.md +++ b/docs/en/sql-reference/statements/select/from.md @@ -34,7 +34,7 @@ Queries that use `FINAL` are executed slightly slower than similar queries that - Data is merged during query execution. - Queries with `FINAL` read primary key columns in addition to the columns specified in the query. -**In most cases, avoid using `FINAL`.** The common approach is to use different queries that assume the background processes of the `MergeTree` engine have’t happened yet and deal with it by applying aggregation (for example, to discard duplicates). +**In most cases, avoid using `FINAL`.** The common approach is to use different queries that assume the background processes of the `MergeTree` engine haven’t happened yet and deal with it by applying aggregation (for example, to discard duplicates). `FINAL` can be applied automatically using [FINAL](../../../operations/settings/settings.md#final) setting to all tables in a query using a session or a user profile. diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 712395a0357..3dfbd133364 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -289,7 +289,7 @@ When `FROM const_expr` not defined sequence of filling use minimal `expr` field When `TO const_expr` not defined sequence of filling use maximum `expr` field value from `ORDER BY`. When `STEP const_numeric_expr` defined then `const_numeric_expr` interprets `as is` for numeric types, as `days` for Date type, as `seconds` for DateTime type. It also supports [INTERVAL](https://clickhouse.com/docs/en/sql-reference/data-types/special-data-types/interval/) data type representing time and date intervals. When `STEP const_numeric_expr` omitted then sequence of filling use `1.0` for numeric type, `1 day` for Date type and `1 second` for DateTime type. -`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous fields values by applying `expr`. If `expr` is not present will repeate previous value. Omitted list will result in including all allowed columns. +`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous fields values by applying `expr`. If `expr` is not present will repeat previous value. Omitted list will result in including all allowed columns. Example of a query without `WITH FILL`: diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 7ee2102c14d..a8f494a5afc 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -21,7 +21,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `lag/lead(value, offset)` | Not supported. Workarounds: | | | 1) replace with `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead` | | | 2) use `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | -| ntile(buckets) | Supported. Specify window like, (partition by x order by y rows between unbounded preceding and unounded following). | +| ntile(buckets) | Supported. Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | ## ClickHouse-specific Window Functions @@ -39,7 +39,7 @@ The computed value is the following for each row: The roadmap for the initial support of window functions is [in this issue](https://github.com/ClickHouse/ClickHouse/issues/18097). -All GitHub issues related to window funtions have the [comp-window-functions](https://github.com/ClickHouse/ClickHouse/labels/comp-window-functions) tag. +All GitHub issues related to window functions have the [comp-window-functions](https://github.com/ClickHouse/ClickHouse/labels/comp-window-functions) tag. ### Tests diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ded7a4643a9..583a49631a3 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,376 +1,1159 @@ personal_ws-1.1 en 543 -AArch -ACLs -AMQP -ARMv -ASLR -ASan -Actian -AddressSanitizer -AppleClang -ArrowStream -AvroConfluent -BSON -BSONEachRow -Bool -BuilderBinAarch -BuilderBinAmd -CCTOOLS -CLion -CMake -CMakeLists -CPUs -CSVWithNames -CSVWithNamesAndTypes -CamelCase -CapnProto -CentOS -ClickHouse -ClickHouse's -ClickableSquare -CodeBlock -CodeLLDB -Compat -Config -ConnectionDetails -Contrib -Ctrl -CustomSeparated -CustomSeparatedWithNames -CustomSeparatedWithNamesAndTypes -DBMSs -DateTime -DateTimes -DockerHub -Doxygen -Encodings -Enum -Eoan -FixedString -FreeBSD -Fuzzer -Fuzzers -GTest -Gb -Gcc -GoogleTest -HDDs -Heredoc -Homebrew -Homebrew's -HorizontalDivide -Hostname -INSERTed -IPv -IntN -Integrations -JSONAsObject -JSONAsString -JSONColumns -JSONColumnsWithMetadata -JSONCompact -JSONCompactColumns -JSONCompactEachRow -JSONCompactEachRowWithNames -JSONCompactEachRowWithNamesAndTypes -JSONCompactStrings -JSONCompactStringsEachRow -JSONCompactStringsEachRowWithNames -JSONCompactStringsEachRowWithNamesAndTypes -JSONEachRow -JSONEachRowWithProgress -JSONObjectEachRow -JSONStrings -JSONStringsEachRow -JSONStringsEachRowWithProgress -JSONs -Jaeger -Jemalloc -Jepsen -KDevelop -LGPL -LLDB -LLVM's -LOCALTIME -LOCALTIMESTAMP -LibFuzzer -LineAsString -LinksDeployment -LowCardinality -MEMTABLE -MSan -MVCC -MacOS -Memcheck -MemorySanitizer -MergeTree -MessagePack -MiB -MsgPack -Multiline -Multithreading -MySQLDump -NEKUDOTAYIM -NULLIF -NVME -NYPD -NuRaft -OLAP -OLTP -ObjectId -Observability -Ok -OpenSSL -OpenSUSE -OpenStack -OpenTelemetry -PAAMAYIM -ParquetMetadata -Parsers -Postgres -Precompiled -PrettyCompact -PrettyCompactMonoBlock -PrettyCompactNoEscapes -PrettyCompactNoEscapesMonoBlock -PrettyJSONEachRow -PrettyMonoBlock -PrettyNoEscapes -PrettyNoEscapesMonoBlock -PrettySpace -PrettySpaceMonoBlock -PrettySpaceNoEscapes -PrettySpaceNoEscapesMonoBlock -Protobuf -ProtobufSingle -QEMU -QTCreator -QueryCacheHits -QueryCacheMisses -RBAC -RawBLOB -RedHat -ReplicatedMergeTree -RowBinary -RowBinaryWithNames -RowBinaryWithNamesAndTypes -Runtime -SATA -SELECTs -SERIALIZABLE -SIMD -SLES -SMALLINT -SQLInsert -SQLSTATE -SSSE -Schemas -SelfManaged -Stateful -Submodules -Subqueries -TSVRaw -TSan -TabItem -TabSeparated -TabSeparatedRaw -TabSeparatedRawWithNames -TabSeparatedRawWithNamesAndTypes -TabSeparatedWithNames -TabSeparatedWithNamesAndTypes -TargetSpecific -TemplateIgnoreSpaces -Testflows -Tgz -Toolset -Tradeoff -Transactional -TwoColumnList -UBSan -UInt -UIntN -UPDATEs -Uint -Updatable -Util -Valgrind -Vectorized -VideoContainer -ViewAllLink -VirtualBox -Werror -WithNamesAndTypes -Woboq -WriteBuffer -WriteBuffers -XCode -YAML -YYYY -Yasm -Zipkin -ZooKeeper -ZooKeeper's aarch +AArch +accurateCast +accurateCastOrDefault +accurateCastOrNull +ACLs +acos +acosh +Actian +ActionsMenu +activecube +activerecord +ActiveRecord +addDays +addHours +Additionaly +addMinutes +addMonths +addQuarters +addr +AddressSanitizer +addressToLine +addressToLineWithInlines +addressToSymbol +addSeconds +addWeeks +addYears +adhoc +adviced +aggregatefunction +Aggregatefunction +AggregateFunction +aggregatingmergetree +AggregatingMergeTree +aggregatio +AggregatorThreads +AggregatorThreadsActive +aggretate +aiochclient +Akka +alculates +AlertManager +Alexey allocator +alphaTokens +ALTERs +amplab +AMPLab +AMQP analytics +ANNIndex +ANNIndexes +anonymize anonymized ansi +AnyEvent +anyheavy +anyHeavy +anyIf +anylast +anyLast +AORM +APIs +appendTrailingCharIfAbsent +AppleClang +approximatly +argmax +argMax +argmin +argMin +arguments's +argumetn +arithmetics +ARMv +arrayAll +arrayAUC +arrayAvg +arrayCompact +arrayConcat +arrayCount +arrayCumSum +arrayCumSumNonNegative +arrayDifference +arrayDistinct +arrayElement +arrayEnumerate +arrayEnumerateDense +arrayEnumerateUniq +arrayExists +arrayFill +arrayFilter +arrayFirst +arrayFirstIndex +arrayFlatten +arrayIntersect +arrayJoin +ArrayJoin +arrayLast +arrayLastIndex +arrayMap +arrayMax +arrayMin +arrayPartialReverseSort +arrayPartialSort +arrayPopBack +arrayPopFront +arrayProduct +arrayPushBack +arrayPushFront +arrayReduce +arrayReduceInRanges +arrayResize +arrayReverse +arrayReverseFill +arrayReverseSort +arrayReverseSplit +arraySlice +arraySort +arraySplit +arrayStringConcat +arraySum +arrayUniq +arrayWithConstant +arrayZip +ArrowStream +ASan +ascii +asin +asinh +ASLR +ASOF +assumeNotNull +asterics async +asynch +AsynchronousHeavyMetricsCalculationTimeSpent +AsynchronousHeavyMetricsUpdateInterval +AsynchronousInsert +AsynchronousInsertThreads +AsynchronousInsertThreadsActive +AsynchronousMetricsCalculationTimeSpent +AsynchronousMetricsUpdateInterval +AsynchronousReadWait +AsyncInsertCacheSize +atan +atanh atomicity +auth +authenticator +Authenticator +authenticators +Authenticators +autocompletion +Autocompletion +autodetect +autodetected autogenerated autogeneration +AutoML autostart +avgweighted +avgWeighted avro +AvroConfluent avx aws backend +BackgroundBufferFlushSchedulePool +BackgroundBufferFlushSchedulePoolSize +BackgroundBufferFlushSchedulePoolTask +BackgroundCommonPoolSize +BackgroundCommonPoolTask +BackgroundDistributedSchedulePool +BackgroundDistributedSchedulePoolSize +BackgroundDistributedSchedulePoolTask +BackgroundFetchesPoolSize +BackgroundFetchesPoolTask +BackgroundMergesAndMutationsPoolSize +BackgroundMergesAndMutationsPoolTask +BackgroundMessageBrokerSchedulePoolSize +BackgroundMessageBrokerSchedulePoolTask +BackgroundMovePoolSize +BackgroundMovePoolTask +BackgroundProcessingPool +BackgroundSchedulePool +BackgroundSchedulePoolSize +BackgroundSchedulePoolTask backoff backticks +BackupsIO +BackupsIOThreads +BackupsIOThreadsActive +BackupsThreads +BackupsThreadsActive +balancer +basename +bcrypt benchmarking +BestEffort +BestEffortOrNull +BestEffortOrZero +BestEffortUS +BestEffortUSOrNull +BestEffortUSOrZero +bfloat +BIGINT +BIGSERIAL +binlog +bitAnd +bitCount +bitHammingDistance +bitmapAnd +bitmapAndCardinality +bitmapAndnot +bitmapAndnotCardinality +bitmapBuild +bitmapCardinality +bitmapContains +bitmapHasAll +bitmapHasAny +bitmapMax +bitmapMin +bitmapOr +bitmapOrCardinality +bitmapSubsetInRange +bitmapSubsetLimit +bitmapToArray +bitmapTransform +bitmapXor +bitmapXorCardinality +bitmask +bitmaskToArray +bitmaskToList +bitNot +bitOr +bitov +bitPositionsToArray +bitRotateLeft +bitRotateRight +bitShiftLeft +bitShiftRight +bitSlice +bitTest +bitTestAll +bitTestAny +bitXor blake +Blazingly +BlockActiveTime +BlockDiscardBytes +BlockDiscardMerges +BlockDiscardOps +BlockDiscardTime +BlockInFlightOps +blockinfo +blockNumber +BlockQueueTime +BlockReadBytes +blockreader +BlockReadMerges +BlockReadOps +BlockReadTime +blockSerializedSize +blocksize blockSize +BlockWriteBytes +BlockWriteMerges +BlockWriteOps +BlockWriteTime +bool +Bool boolean bools boringssl +BORO +bozerkins +broadcasted +BrokenDistributedFilesToInsert brotli bson +BSON bsoneachrow +BSONEachRow +buffersize buildable +BuilderBinAarch +BuilderBinAmd +buildId +BuildID +builtins +bytebase +Bytebase +byteSize +bytesToCutForIPv +CacheDetachedFileSegments +CacheDictionaries +CacheDictionary +CacheDictionaryThreads +CacheDictionaryThreadsActive +CacheDictionaryUpdateQueueBatches +CacheDictionaryUpdateQueueKeys +CacheFileSegments +cacheSessions +cachesize +caConfig camelCase +CamelCase +CapContains capn +Cap'n capnproto +CapnProto +CapUnion +cardinalities cardinality +cartesian cassandra +casted +catboost +CatBoost +catboostEvaluate +categoricalinformationvalue +categoricalInformationValue +cathetus cbindgen +cbrt ccache +CCTOOLS cctz +CDATA +CDFs +CDMA +ceil +CellAreaM +CellAreaRads +CellsIntersect +CentOS +centroid +certificateFile +CertificateHandler +CESU +cetera cfg +chadmin +Chadmin changelog changelogs +ChannelID charset charsets +chconn checkouting checksummed checksumming checksums +childern +chproxy +chunksize +cickhouse +Cidr +CIDR +CIDRToRange +cipherList +ciphertext +Ciphertext cityhash +cityHash +CityHash +CLang cli +ClickableSquare +clickcat +ClickCat clickhouse +ClickHouse +ClickHouseClient +clickhousedb +ClickHouseMigrator +ClickHouseNIO +ClickHouse's +ClickHouseVapor +clickhousex +clickmate clickstream +clickvisual +ClickVisual +CLion +CLOB +clockhour +cLoki +CloudDetails +clusterAllReplicas cmake +CMake +CMakeLists +CMPLNT codebase +CodeBlock codec +codecs +Codecs +CODECS +CodeLLDB +codepoint +codepoints +collapsingmergetree +CollapsingMergeTree +combinator +combinators +Combinators +commnents comparising +Compat +compatitalbe +CompiledExpressionCacheBytes +CompiledExpressionCacheCount +ComplexKeyCache +ComplexKeyDirect +ComplexKeyHashed +compressability +concat +concatAssumeInjective +concatWithSeparator +concatWithSeparatorAssumeInjective +concurenly +cond +conf config +Config configs +congruential +conjuction +conjuctive +ConnectionDetails +const +Const +ContextLockWait contrib +Contrib +convergance +convertCharset coroutines +cosineDistance +countDigits +countEqual +countMatches +countSubstrings +covariates +covarpop +covarPop +covarsamp +covarSamp +covid +Covid +COVID cpp cppkafka cpu +CPUFrequencyMHz +CPUs +Cramer's +cramersv +cramersV +cramersvbiascorrected +cramersVBiasCorrected +criteo +Criteo crlf croaring cronjob +Crotty +Crowdsourced +cryptocurrencies +cryptocurrency +cryptographic csv +CSVs csvwithnames +CSVWithNames csvwithnamesandtypes +CSVWithNamesAndTypes +CTEs +Ctrl +currentDatabase +CurrentMetrics +currentProfiles +currentRoles +currentUser +customizable +customizations customseparated +CustomSeparated customseparatedwithnames +CustomSeparatedWithNames customseparatedwithnamesandtypes +CustomSeparatedWithNamesAndTypes +cutFragment +cutIPv +cutQueryString +cutQueryStringAndFragment +cutToFirstSignificantSubdomain +cutToFirstSignificantSubdomainCustom +cutToFirstSignificantSubdomainCustomWithWWW +cutToFirstSignificantSubdomainWithWWW +cutURLParameter +cutWWW cyrus +DatabaseCatalog +DatabaseCatalogThreads +DatabaseCatalogThreadsActive +DatabaseOnDisk +DatabaseOnDiskThreads +DatabaseOnDiskThreadsActive +DatabaseOrdinaryThreads +DatabaseOrdinaryThreadsActive datacenter +datacenters datafiles +datagrip +DataGrip +datalens +DataLens +datanode dataset datasets +datasource +DataTime +datatypes +DataTypes +dateName datetime +dateTime +DateTime datetimes +DateTimes +dateTimeToSnowflake +dayofyear +dbal +DBAs +DbCL +dbeaver +DBeaver +dbgen dbms +DBMSs ddl +DDLWorker +DDLWORKER +DDLWorkerThreads +DDLWorkerThreadsActive deallocation +deallocations debian +decodeURLComponent +decodeURLFormComponent +decodeXMLComponent decompressor +decrypt +DECRYPT +decrypted +Decrypted +decrypts +deduplicate +Deduplicate +deduplicated +deduplicating +deduplication +Deduplication +defaultProfiles +defaultRoles +defaultValueOfArgumentType +defaultValueOfTypeName +DelayedInserts +DeliveryTag +deltalake +deltaLake +DeltaLake +deltasum +deltaSum +deltasumtimestamp +deltaSumTimestamp +demangle +denormalize +Denormalize +denormalized denormalizing denormals +DESC deserialization deserialized +deserializing +DestroyAggregatesThreads +DestroyAggregatesThreadsActive destructor destructors +detectCharset +detectLanguage +detectLanguageMixed +detectLanguageUnknown +determinator +deterministically +DictCacheRequests +dictGet +dictGetChildren +dictGetDescendant +dictGetHierarchy +dictGetOrDefault +dictGetOrNull +dictGetUUID +dictHas +dictIsIn +disableProtocols +disjunction +disjunctions +DiskAvailable +DiskObjectStorage +DiskObjectStorageAsyncThreads +DiskObjectStorageAsyncThreadsActive +DiskSpaceReservedForMerge +DiskTotal +DiskUnreserved +DiskUsed +displaySecretsInShowAndSelect +DistributedFilesToInsert +DistributedSend +distro +divideDecimal dmesg +DockerHub +DOGEFI +domainWithoutWWW dont +dotProduct +DoubleDelta +Doxygen +dplyr dragonbox +dropoff +dumpColumnStructure durations +ECMA +ecto +Ecto +EdgeAngle +EdgeLengthKm +EdgeLengthM +EmbeddedRocksDB +embeddings +Embeddings +emptyArray +emptyArrayDate +emptyArrayDateTime +emptyArrayFloat +emptyArrayInt +emptyArrayString +emptyArrayToSingle +emptyArrayUInt +enabledProfiles +enabledRoles +encodeURLComponent +encodeURLFormComponent +encodeXMLComponent encodings +Encodings +encryptions endian +endsWith +Engeneering enum +Enum +enum's +enums +Enums +Eoan +EphemeralNode +erfc +errorCodeToName +Ethereum +evalMLMethod +everytime +ExactEdgeLengthKm +ExactEdgeLengthM +ExactEdgeLengthRads +ExecutablePool exFAT +expiryMsec +exponentialmovingaverage +exponentialMovingAverage +expr +exprN +extendedVerification +extention +ExternalDistributed +extractAll +extractAllGroups +extractAllGroupsHorizontal +extractAllGroupsVertical +extractKeyValuePairs +extractKeyValuePairsWithEscaping +extractTextFromHTML +extractURLParameter +extractURLParameterNames +extractURLParameters +ExtType +failover +Failover +farmFingerprint +farmHash +FarmHash fastops fcoverage +FFFD filesystem +filesystemAvailable +FilesystemCacheBytes +FilesystemCacheElements +FilesystemCacheFiles +FilesystemCacheReadBuffers +FilesystemCacheSize +filesystemCapacity +filesystemFree +FilesystemLogsPathAvailableBytes +FilesystemLogsPathAvailableINodes +FilesystemLogsPathTotalBytes +FilesystemLogsPathTotalINodes +FilesystemLogsPathUsedBytes +FilesystemLogsPathUsedINodes +FilesystemMainPathAvailableBytes +FilesystemMainPathAvailableINodes +FilesystemMainPathTotalBytes +FilesystemMainPathTotalINodes +FilesystemMainPathUsedBytes +FilesystemMainPathUsedINodes filesystems +finalizeAggregation +fips +FIPS +firstSignificantSubdomain +firstSignificantSubdomainCustom +fixedstring +FixedString +flamegraph flatbuffers +flink +Flink +fluentd fmtlib +ForEach +formatDateTime +formatDateTimeInJoda +formatDateTimeInJodaSyntax +formated +formatReadableDecimalSize +formatReadableQuantity +formatReadableSize +formatReadableTimeDelta +formatRow +formatRowNoNewline formatschema formatter +FOSDEM +FQDN +FreeBSD +freezed +fromModifiedJulianDay +fromModifiedJulianDayOrNull +fromUnixTimestamp +fromUnixTimestampInJodaSyntax fsync +func +funtion +fuzzBits fuzzer +Fuzzer fuzzers -gRPC +Fuzzers +Gb +Gbit +Gcc +gccMurmurHash gcem +generateRandom +GenerateRandom +generateULID +generateUUIDv +geobase +geobases +Geobases +geocode +GeoCoord +geoDistance +geohash +Geohash +geohashDecode +geohashEncode +geohashesInBox +Geoid +geoip +geospatial +geoToH +geoToS +GetBaseCell +getblockinfo +GetDestinationIndexFromUnidirectionalEdge +getevents +GetFaces +GetIndexesFromUnidirectionalEdge +getMacro +GetNeighbors +GetOriginIndexFromUnidirectionalEdge +getOSKernelVersion +GetPentagonIndexes +GetRes +GetResolution +getServerPort +getSetting +getSizeOfEnumType +GetUnidirectionalEdge +GetUnidirectionalEdgeBoundary +GetUnidirectionalEdgesFromHexagon github +GitLab glibc +globalIn +globalNotIn +GlobalThread +GlobalThreadActive +glushkovds +GoLand +golang googletest +GoogleTest +grafana +Grafana +graphitemergetree +GraphiteMergeTree +graphouse +graphql +GraphQL +greatCircleAngle +greatCircleDistance +greaterOrEquals +greenspace +Greenwald +grouparray +groupArray +grouparrayinsertat +groupArrayInsertAt +grouparraylast +groupArrayLast +grouparraymovingavg +groupArrayMovingAvg +grouparraymovingsum +groupArrayMovingSum +grouparraysample +groupArraySample +groupbitand +groupBitAnd +groupbitmap +groupBitmap +groupbitmapand +groupBitmapAnd +groupbitmapor +groupBitmapOr +groupbitmapxor +groupBitmapXor +groupbitor +groupBitOr +groupbitxor +groupBitXor +groupuniqarray +groupUniqArray grpc +gRPC grpcio gtest +GTest +gtid +GTID +gzip +gzipped +hadoop +halfday +halfMD hardlinks +hasAll +hasAny +hasColumnInTable +HashedDictionary +HashedDictionaryThreads +HashedDictionaryThreadsActive +hashtables +hasSubstr +hasToken +hasTokenCaseInsensitive +hasTokenCaseInsensitiveOrNull +hasTokenOrNull +haversine +Haversine +have't +hdbc +HDDs hdfs +hdfsCluster heredoc +Heredoc heredocs +HexAreaKm +HexAreaM +HexRing +HHMM +Hight +hiveHash +HMAC +holistics +Holistics homebrew +Homebrew +Homebrew's +hopEnd +hopStart +horgh +HorizontalDivide +hostname +hostName +Hostname +hostnames +houseops +HouseOps +hsts +HSTS +html http +HTTPConnection https +HTTPThreads +hudi +Hudi +HyperLogLog hyperscan +hypot +Hypot +hyvor +IANA +icosahedron icudata +idempotency +identifiant +Identifiant +ifNotFinite +ifNull +iframe +ilike +IMDS +implicitely +incrementing +Incrementing +incremnt +IndexesAreNeighbors +indexHint +indexOf +infi +INFILE +InfluxDB +initializeAggregation +initialQueryID +injective +innogames +inodes +INSERTed +INSERTs +Instana instantiation +intDiv +intDivOrZero integrational integrations +Integrations +IntelliJ interserver +InterserverConnection +InterserverThreads +intervalLengthSum +intExp +intHash +IntN +introspections +invalidCertificateHandler invariants +invertedindexes +IOPrefetchThreads +IOPrefetchThreadsActive +IOThreads +IOThreadsActive +IOUringInFlightEvents +IOUringPendingEvents +IOWriterThreads +IOWriterThreadsActive +IPTrie +IPv +isConstant +isDecimalOverflow +isFinite +isInfinite +isIPAddressInRange +isIPv +isNaN +isNotNull +isNull +IsPentagon +IsResClassIII +IsValid +isValidJSON +isValidUTF +iteratively +Jaeger +Jannis +javaHash +JavaHash +javaHashUTF +jbod +JBOD jdbc jemalloc +Jemalloc +Jepsen +JetBrains +Jitter +Joda +JOINed +joinGet +JOINs json +JSONArrayLength +JSONAsObject jsonasstring +JSONAsString jsoncolumns +JSONColumns jsoncolumnsmonoblock +JSONColumnsWithMetadata jsoncompact +JSONCompact jsoncompactcolumns +JSONCompactColumns jsoncompacteachrow +JSONCompactEachRow jsoncompacteachrowwithnames +JSONCompactEachRowWithNames jsoncompacteachrowwithnamesandtypes +JSONCompactEachRowWithNamesAndTypes jsoncompactstrings +JSONCompactStrings jsoncompactstringseachrow +JSONCompactStringsEachRow jsoncompactstringseachrowwithnames +JSONCompactStringsEachRowWithNames jsoncompactstringseachrowwithnamesandtypes +JSONCompactStringsEachRowWithNamesAndTypes jsoneachrow +JSONEachRow jsoneachrowwithprogress +JSONEachRowWithProgress +JSONExtract +JSONExtractArrayRaw +JSONExtractBool +JSONExtractFloat +JSONExtractInt +JSONExtractKeys +JSONExtractKeysAndValues +JSONExtractKeysAndValuesRaw +JSONExtractRaw +JSONExtractString +JSONExtractUInt +JSONHas +JSONLength jsonobjecteachrow +JSONObjectEachRow +JSONs jsonstrings +JSONStrings jsonstringseachrow +JSONStringsEachRow jsonstringseachrowwithprogress +JSONStringsEachRowWithProgress +JSONType +jumpConsistentHash +JumpConsistentHash +Jupyter kafka +KafkaAssignedPartitions +KafkaBackgroundReads kafkacat +KafkaConsumers +KafkaConsumersInUse +KafkaConsumersWithAssignment +KafkaLibrdkafkaThreads +kafkaMurmurHash +KafkaProducers +KafkaWrites +Kahan +KDevelop +KeeperAliveConnections +keepermap +KeeperMap +KeeperOutstandingRequets +kerberized +kerberos +Kerberos +kernal +keyspace +keytab +Khanna +kittenhouse +KittenHouse +Klickhouse +Kolmogorov +kolmogorovsmirnovtest +kolmogorovSmirnovTest +kolya konsole +kRing +Kubernetes +kurtosis +kurtpop +kurtPop +kurtsamp +kurtSamp laion +lang +laravel latencies +ldap +LDAP +learing +leftPad +leftPadUTF +lemmatization +lemmatize +lemmatized +lengthUTF +lessOrEquals lexicographically -libFuzzer +lgamma +LGPL libc +libcatboost libcpuid libcxx libcxxabi libdivide libfarmhash libfuzzer +libFuzzer +LibFuzzer libgsasl libhdfs libmetrohash @@ -381,193 +1164,1405 @@ libs libunwind libuv libvirt +LightHouse linearizability linearizable +linearized lineasstring +LineAsString linefeeds lineorder +Linf +LinfDistance +LinfNorm +LinfNormalize +LinksDeployment +Linq linux +LLDB llvm +LLVM's +LoadAverage +loadDefaultCAFile localhost +localread +LocalThread +LocalThreadActive +LOCALTIME +LOCALTIMESTAMP +logagent +loghouse +LogQL +Logstash +logTrace +london +LONGLONG +LookML +lowcardinality +LowCardinality +lowercased +lowerUTF +LpDistance +LpNorm +LpNormalize +Luebbe +Lyft +lzma +MacBook +MACNumToString macOS +MacOS +MACStringToNum +MACStringToOUI +mailrugo +mailto +makeDate +makeDateTime +mannwhitneyutest +mannWhitneyUTest +mapAdd +mapAll +mapApply +mapConcat +mapContains +mapContainsKeyLike +mapExists +mapExtractKeyLike +mapFilter +mapFromArrays +mapKeys +mappedfile +mapPopulateSeries +mapReverseSort +mapSort +mapSubtract +mapUpdate +mapValues mariadb +MarkCacheBytes +MarkCacheFiles +MarksLoaderThreads +MarksLoaderThreadsActive +matcher +MaterializedMySQL +MaterializedPostgreSQL +materializedview +MaterializedView +MaxDDLEntryID +maxintersections +maxIntersections +maxintersectionsposition +maxIntersectionsPosition +maxmap +maxMap +maxmind +MaxMind +MaxPartCountForPartition +MaxPushedDDLEntryID +Mbps mdadm +meanztest +meanZTest +mebibytes +MEDIUMINT +Memcheck +MemoryCode +MemoryDataAndStack +MemoryResident +MemorySanitizer +MemoryShared +MemoryTracking +MemoryVirtual +MEMTABLE +mergeable +MergeJoin +MergeState +mergetree +MergeTree +MergeTreeAllRangesAnnouncementsSent +MergeTreeBackgroundExecutor +MergeTreeBackgroundExecutorThreads +MergeTreeBackgroundExecutorThreadsActive +MergeTreeDataSelectExecutor +MergeTreeDataSelectExecutorThreads +MergeTreeDataSelectExecutorThreadsActive +MergeTreePartsCleanerThreads +MergeTreePartsCleanerThreadsActive +MergeTreePartsLoaderThreads +MergeTreePartsLoaderThreadsActive +MergeTreeReadTaskRequestsSent +MergeTreeSettings +messageID +MessagePack +metacharacters +Metastore +metasymbols +metrica +metroHash +MetroHash +mfedotov +MiB +Milli +Milovidov +mindsdb +MindsDB +MinHash +minimalistic +mininum +MinIO miniselect +minmap +minMap +minmax +MinMax +mins +misconfiguration +mispredictions +mmap +MMapCacheCells +mmapped +MMappedAllocBytes +MMappedAllocs +MMappedFileBytes +MMappedFiles +moduloOrZero +mongodb +Mongodb +monthName +moscow +MSan msgpack +MsgPack msgpk +MSSQL +multibyte +multiFuzzyMatchAllIndices +multiFuzzyMatchAny +multiFuzzyMatchAnyIndex +multiIf multiline +Multiline +multiMatchAllIndices +multiMatchAny +multiMatchAnyIndex +multiplyDecimal +multipolygon +MultiPolygon +Multiqueries +multiSearchAllPositions +multiSearchAllPositionsUTF +multiSearchAny +multiSearchFirstIndex +multiSearchFirstPosition +multisets multithread +Multithreading +multiword +Multiword +munmap murmurhash +murmurHash +MurmurHash +musqldump mutex +MVCC +mydb +myfilter mysql +MySQLConnection mysqldump +MySQLDump mysqljs +MySQLThreads +mytable +Nagios +namedatabases +namenetworks +namenode +Namenode +namepassword +nameprofile +namequota +NamesAndTypesList +namespaces +Nano +NaNs natively +nats +NATS +NCHAR +negtive +NEKUDOTAYIM +Nesterov +nestjs +netloc +NetworkReceive +NetworkReceiveBytes +NetworkReceiveDrop +NetworkReceiveErrors +NetworkReceivePackets +NetworkSend +NetworkSendBytes +NetworkSendDrop +NetworkSendErrors +NetworkSendPackets +NEWDATE +NEWDECIMAL +NFKC +NFKD +ngram +ngrambf +ngramDistance +ngramMinHash +ngramMinHashArg +ngramMinHashArgCaseInsensitive +ngramMinHashArgCaseInsensitiveUTF +ngramMinHashArgUTF +ngramMinHashCaseInsensitive +ngramMinHashCaseInsensitiveUTF +ngramMinHashUTF +ngrams +ngramSearch +ngramSimHash +ngramSimHashCaseInsensitive +ngramSimHashCaseInsensitiveUTF +ngramSimHashUTF +NodeJs +nonNegativeDerivative noop +normalizedQueryHash +normalizeQuery +normalizeUTF +notEmpty +notEquals +notILike +notIn +notLike +notretry +nowInBlock +ntile nullability nullable +nullables +nullIf +NULLIF num +NumberOfDatabases +NumberOfDetachedByUserParts +NumberOfDetachedParts +NumberOfTables +numerics +NumHexagons +NumToString +NumToStringClassC +NuRaft +NVMe +NVME +nypd +NYPD obfuscator +ObjectId +observability +Observability +Octonica odbc +OFNS ok -openSUSE +Ok +OLAP +OLTP +omclickhouse +onstraints +ontime +OnTime +OpenCelliD +OpenFileForRead +OpenFileForWrite openldap +opensky +OpenSky +openssl +openSSL +OpenSSL +OpenStack +openSUSE +OpenSUSE opentelemetry +OpenTelemetry +optinal +Optinal +OrDefault +OrNull +OrZero +OSContextSwitches +OSGuestNiceTime +OSGuestNiceTimeCPU +OSGuestNiceTimeNormalized +OSGuestTime +OSGuestTimeCPU +OSGuestTimeNormalized +OSIdleTime +OSIdleTimeCPU +OSIdleTimeNormalized +OSInterrupts +OSIOWaitTime +OSIOWaitTimeCPU +OSIOWaitTimeNormalized +OSIrqTime +OSIrqTimeCPU +OSIrqTimeNormalized +OSMemoryAvailable +OSMemoryBuffers +OSMemoryCached +OSMemoryFreePlusCached +OSMemoryFreeWithoutCached +OSMemoryTotal +OSNiceTime +OSNiceTimeCPU +OSNiceTimeNormalized +OSOpenFiles +OSProcessesBlocked +OSProcessesCreated +OSProcessesRunning +OSSoftIrqTime +OSSoftIrqTimeCPU +OSSoftIrqTimeNormalized +OSStealTime +OSStealTimeCPU +OSStealTimeNormalized +OSSystemTime +OSSystemTimeCPU +OSSystemTimeNormalized +OSThreadsRunnable +OSThreadsTotal +OSUptime +OSUserTime +OSUserTimeCPU +OSUserTimeNormalized +OTLP +outfile +OUTFILE overcommit +overcommitted +OvercommitTracker +overfitting +PAAMAYIM +packetpool +packetsize +PagerDuty +pageviews +pandahouse +ParallelFormattingOutputFormatThreads +ParallelFormattingOutputFormatThreadsActive parallelization parallelize parallelized +ParallelParsingInputFormat +ParallelParsingInputFormatThreads +ParallelParsingInputFormatThreadsActive +Parametrized +params +paratemer +ParquetMetadata +parsable +parseable +parseDateTime +parseDateTimeBestEffort +parseDateTimeBestEffortOrNull +parseDateTimeBestEffortOrZero +parseDateTimeBestEffortUS +parseDateTimeBestEffortUSOrNull +parseDateTimeBestEffortUSOrZero +parseDateTimeInJodaSyntax +parseDateTimeInJodaSyntaxOrNull +parseDateTimeInJodaSyntaxOrZero +parseDateTimeOrNull +parseDateTimeOrZero parsers +Parsers +parseTimeDelta +Partitioner +PartMutation +PartsActive +PartsCommitted +PartsCompact +PartsDeleteOnDestroy +PartsDeleting +PartsInMemory +PartsOutdated +PartsPreActive +PartsPreCommitted +PartsTemporary +PartsWide +pathFull pclmulqdq +pcre +PCRE +PendingAsyncInsert +Percona performant +perl +persistency +phpclickhouse +PhpStorm +pipelining +plaintext +plantuml +PlantUML poco +PointDistKm +PointDistM +PointDistRads +pointInEllipses +pointInPolygon +polygonAreaCartesian +polygonAreaSpherical +polygonConvexHullCartesian +polygonPerimeterCartesian +polygonPerimeterSpherical +polygonsDistanceCartesian +polygonsDistanceSpherical +polygonsEqualsCartesian +polygonsIntersectionCartesian +polygonsIntersectionSpherical +polygonsSymDifferenceCartesian +polygonsSymDifferenceSpherical +polygonsUnionCartesian +polygonsUnionSpherical +polygonsWithinCartesian +polygonsWithinSpherical popcnt +porthttps +positionCaseInsensitive +positionCaseInsensitiveUTF +positionUTF +positiveModulo postfix postfixes +Postgres postgresql +PostgreSQLConnection +PostgreSQLThreads +PostgresSQL pre +pread +preallocate prebuild prebuilt +Precompiled preemptable +preferServerCiphers +prefertch +prefetch +prefetchsize preloaded +prepend +prepended +prepends +preprocess +Preprocess preprocessed +preprocessing preprocessor presentational prestable prettycompact +PrettyCompact prettycompactmonoblock +PrettyCompactMonoBlock prettycompactnoescapes +PrettyCompactNoEscapes prettycompactnoescapesmonoblock +PrettyCompactNoEscapesMonoBlock prettyjsoneachrow +PrettyJSONEachRow prettymonoblock +PrettyMonoBlock prettynoescapes +PrettyNoEscapes prettynoescapesmonoblock +PrettyNoEscapesMonoBlock prettyspace +PrettySpace prettyspacemonoblock +PrettySpaceMonoBlock prettyspacenoescapes +PrettySpaceNoEscapes prettyspacenoescapesmonoblock +PrettySpaceNoEscapesMonoBlock +prewhere +Prewhere +PREWHERE +privateKeyFile +privateKeyPassphraseHandler +PrivateKeyPassphraseHandler prlimit +PROCESSLIST +procfs +ProfileEvents +profiler +Profiler +profuct +Proleptic prometheus +PromHouse +Promql +PromQL +Promtail proto protobuf +Protobuf protobufsingle +ProtobufSingle +proxied +ProxySQL +pseudorandom +pseudorandomize psql ptrs +publsh +pushdown +pwrite py +PyCharm +QEMU +qouta +qryn +QTCreator +quantile +Quantile +quantilebfloat +quantileBFloat +quantiledeterministic +quantileDeterministic +quantileexact +quantileExact +quantileExactExclusive +quantileExactHigh +quantileExactInclusive +quantileExactLow +quantileexactweighted +quantileExactWeighted +quantileGK +quantileInterpolatedWeighted +quantiles +quantilesExactExclusive +quantilesExactInclusive +quantilesGK +quantilesTimingWeighted +quantiletdigest +quantileTDigest +quantiletdigestweighted +quantileTDigestWeighted +quantiletiming +quantileTiming +quantiletimingweighted +quantileTimingWeighted +quartile +quaters +QueryCacheHits +QueryCacheMisses +queryID +QueryPreempted +queryString +queryStringAndFragment +QueryThread +QuoteMeta +rabbitmq +RabbitMQ +raduis +randBernoulli +randBinomial +randCanonical +randChiSquared +randConstant +randExponential +randFisherF +randLogNormal +randNegativeBinomial +randNormal +randomFixedString +randomPrintableASCII +randomString +randomStringUTF +randPoisson +randStudentT +randUniform +RangeHashed +rankCorr rapidjson rawblob +RawBLOB +RBAC +RClickHouse readahead readline readme readonly +ReadonlyReplica +ReadTaskRequestsSent +readWKTMultiPolygon +readWKTPolygon +reate +rebalance rebalanced +recency +RecipeNLG +recompress +recompressed +recompressing +Recompressing +recompression +Recompression +reconnection +RectAdd +RectContains +RectIntersection +RectUnion +recurse +redash +Redash +reddit +Reddit +RedHat +redisstreams +ReDoS +Refactorings +refcounter +ReferenceKeyed +regexpExtract +regexpQuoteMeta +RegexpTree +regionHierarchy +regionIn +regionToArea +regionToCity +regionToContinent +regionToCountry +regionToDistrict +regionToName +regionToPopulation +regionToTopContinent +reinitialization +reinitializing +reinterpretAs +reinterpretAsDate +reinterpretAsDateTime +reinterpretAsFixedString +reinterpretAsFloat +reinterpretAsInt +reinterpretAsString +reinterpretAsUInt +reinterpretAsUUID +RemoteRead +remoteSecure +replaceAll +replaceOne +replaceRegexpAll +replaceRegexpOne +replacingmergetree +ReplacingMergeTree +ReplicasMaxAbsoluteDelay +ReplicasMaxInsertsInQueue +ReplicasMaxMergesInQueue +ReplicasMaxQueueSize +ReplicasMaxRelativeDelay +ReplicasSumInsertsInQueue +ReplicasSumMergesInQueue +ReplicasSumQueueSize +replicatable +ReplicatedAggregatingMergeTree +ReplicatedChecks +ReplicatedCollapsingMergeTree +ReplicatedFetch +ReplicatedGraphiteMergeTree +replicatedmergetree +ReplicatedMergeTree +ReplicatedReplacingMergeTree +ReplicatedSend +ReplicatedSummingMergeTree +ReplicatedVersionedCollapsingMergeTree replxx repo representable requestor +requireTLSv +Resample +resharding +reshards +RestartReplicaThreads +RestartReplicaThreadsActive +RestoreThreads +RestoreThreadsActive resultset +retentions rethrow +retransmit retriable +retuned +Returnes +reult +reverseDNSQuery +reverseUTF +RHEL +rightPad +rightPadUTF risc riscv ro +roadmap +RoaringBitmap rocksdb -rowNumberInBlock +RocksDB +rollup +Rollup +ROLLUP +roundAge +roundBankers +roundDown +roundDuration +roundToExp +routineley rowbinary +RowBinary rowbinarywithnames +RowBinaryWithNames rowbinarywithnamesandtypes +RowBinaryWithNamesAndTypes +rowNumberInAllBlocks +rowNumberInBlock rsync +rsyslog runnable runningAccumulate +runningConcurrency +runningDifference +runningDifferenceStartingWithFirstValue runtime +Runtime russian rw +RWLock +RWLockActiveReaders +RWLockActiveWriters +RWLockWaitingReaders +RWLockWaitingWriters +SaaS +Sanjeev +Sankey sasl +SATA +satisfiable +scala +Scalable +Scatterplot +Schaefer schemas +Schemas +Schwartzian +searchin +SeasClick +seccessfully +seekable +seektable +SeekTable +SELECTs +SelfManaged +Sematext +SendExternalTables +SendScalars +separatelly +sequenceCount +sequenceMatch +sequenceNextNode +SERIALIZABLE +serverUUID +sessionCacheSize +sessionIdContext +sessionTimeout +seva +shardCount +sharded +sharding +shardNum +ShareAlike +shortcircuit +shoutout +SIGTERM +SIMD simdjson +Simhash +SimHash +simpleaggregatefunction +SimpleAggregateFunction +simplelinearregression +simpleLinearRegression +SimpleState +simpliest +simpod +singlepart +sinh +siphash +sipHash +SipHash +skewness +skewpop +skewPop +skewsamp +skewSamp skippingerrors +sleepEachRow +SLES +SLRU +SMALLINT +Smirnov's +Smirnov'test +snowflakeToDateTime +socketcache +soundex +Soundex +SpanKind +sparkbar sparsehash +Spearman's +speedscope +splitByChar +splitByNonAlpha +splitByRegexp +splitByString +splitByWhitespace +SPNEGO +SQEs sql +sqlalchemy +SQLAlchemy +SQLConsoleDetail sqlinsert +SQLInsert +sqlite +SQLSTATE +sqrt src +SSDCache +SSDComplexKeyCache +SSDs +SSLManager +SSRF +SSSE +stacktrace stacktraces +startsWith +StartTime +StartTLS +StartupSystemTables +StartupSystemTablesThreads +StartupSystemTablesThreadsActive statbox stateful +Stateful +stddev +stddevpop +stddevPop +stddevsamp +stddevSamp stderr stdin stdout +stochastically +stochasticlinearregression +stochasticLinearRegression +stochasticlogisticregression +stochasticLogisticRegression +StorageBufferBytes +StorageBufferRows +StorageDistributed +StorageDistributedThreads +StorageDistributedThreadsActive +StorageHive +StorageHiveThreads +StorageHiveThreadsActive +StorageODBC +storages +StorageS +storig +stringToH +StringToNum +StringToNumOrDefault +StringToNumOrNull +stripelog +Stripelog +StripeLog +Strohmeier strtod strtoll strtoull +struct structs +studentttest +studentTTest +subarray +subarrays +subBitmap +subcolumn +subcolumns +Subcolumns +subcribe subdirectories +subdirectory +subexpression +Subexpression subexpressions +subfolder +subinterval +subintervals +subkey +submatch submodule submodules +Submodules +subnet +subnetwork subpattern subpatterns subqueries +Subqueries subquery +subranges +subreddits subseconds +substracted substring +substrings +Substrings +substringUTF +subtitiles +subtractDays +subtractHours +subtractMinutes +subtractMonths +subtractQuarters +subtractSeconds +subtractWeeks +subtractYears subtree subtype sudo +sumcount +sumCount +sumkahan +sumKahan +summap +sumMap +sumMapFiltered +summingmergetree +SummingMergeTree +sumwithoverflow +sumWithOverflow +superaggregates +Superset +SuperSet +SupersetDocker +supremum symlink symlinks +synchronious syntaxes +syscall +syscalls +syslog +syslogd systemd +SystemReplicasThreads +SystemReplicasThreadsActive +TabItem +tabix +Tabix +TablesLoaderThreads +TablesLoaderThreadsActive +TablesToDropQueueSize +tablum +TABLUM tabseparated +TabSeparated tabseparatedraw +TabSeparatedRaw tabseparatedrawwithnames +TabSeparatedRawWithNames tabseparatedrawwithnamesandtypes +TabSeparatedRawWithNamesAndTypes tabseparatedwithnames +TabSeparatedWithNames tabseparatedwithnamesandtypes +TabSeparatedWithNamesAndTypes +TargetSpecific tcp +TCPConnection +tcpnodelay +tcpPort +TCPThreads +Telegraf templateignorespaces +TemplateIgnoreSpaces +TemporaryFilesForAggregation +TemporaryFilesForJoin +TemporaryFilesForSort +TemporaryFilesUnknown +Testflows +tgamma tgz +Tgz th +thats +Theil's +theilsu +theilsU +themself +threadpool +ThreadPoolFSReaderThreads +ThreadPoolFSReaderThreadsActive +ThreadPoolRemoteFSReaderThreads +ThreadPoolRemoteFSReaderThreadsActive +ThreadsActive +ThreadsInOvercommitTracker +throwIf +timeSlot +timeSlots +Timeunit +timeZone +timeZoneOf +timeZoneOffset +timezones +TINYINT +tinylog +TinyLog +Tkachenko +TKSV +TLSv tmp +ToCenterChild +ToChildren +toColumnTypeName +toDate +toDateOrDefault +toDateOrNull +toDateOrZero +toDateTime +toDateTimeOrDefault +toDateTimeOrNull +toDateTimeOrZero +toDayOfMonth +toDayOfWeek +toDayOfYear +toDecimal +toDecimalString +toFixedString +toFloat +ToGeo +ToGeoBoundary +toHour +toInt +toInterval +toIPv +ToIPv +toISOWeek +toISOYear +toJSONString +tokenbf tokenization +tokenized +tokenizer +toLastDayOfMonth +toLowCardinality +toMinute toml +toModifiedJulianDay +toModifiedJulianDayOrNull +toMonday +toMonth +toNullable toolchain toolset +Toolset +ToParent +topk +topK +TopK +topkweighted +topKWeighted +topLevelDomain +toQuarter +toRelativeDayNum +toRelativeHourNum +toRelativeMinuteNum +toRelativeMonthNum +toRelativeQuarterNum +toRelativeSecondNum +toRelativeWeekNum +toRelativeYearNum +toSecond +ToSnowflake +toStartOfDay +toStartOfFifteenMinutes +toStartOfFiveMinutes +toStartOfHour +toStartOfInterval +toStartOfISOYear +toStartOfMinute +toStartOfMonth +toStartOfQuarter +toStartOfSecond +toStartOfTenMinutes +toStartOfWeek +toStartOfYear +toString +ToString +toStringCutToZero +TotalBytesOfMergeTreeTables +TotalPartsOfMergeTreeTables +TotalRowsOfMergeTreeTables +TotalTemporaryFiles +toTime +toTimeZone +toType +toTypeName +toUInt +toUnixTimestamp +toUUID +toUUIDOrDefault +toUUIDOrNull +toUUIDOrZero +toValidUTF +toWeek +toYear +toYearWeek +toYYYYMM +toYYYYMMDD +toYYYYMMDDhhmmss +TPCH +Tradeoff transactional +Transactional transactionally +translateUTF +translocality +trie +trimBoth +trimLeft +trimRight +trunc +tryBase +tryDecrypt +TSan +TSDB tskv tsv +TSVRaw +TSVs +TThe tui +tumbleEnd +tumbleStart +tupleDivide +tupleDivideByNumber +tupleElement +tupleHammingDistance +tupleMinus +tupleMultiply +tupleMultiplyByNumber +tupleNegate +tuplePlus +tupleToNameValuePairs turbostat +TwoColumnList txt +typename +Uber +UBSan ubuntu +UDFs uint +Uint +UInt +UIntN +ulid +ULID +ULIDStringToDateTime +UMTS unary +unbin +uncomment +UncompressedCacheBytes +UncompressedCacheCells +underying +undrop +UNDROP +unencoded unencrypted +unescaped +unescaping +unhex +unicode +unidimensional +UnidirectionalEdgeIsValid +uniq +uniqcombined +uniqCombined +uniqexact +uniqExact +uniqhll +uniqHLL +uniqtheta +uniqTheta +uniqThetaIntersect +uniqThetaNot +uniqthetasketch +uniqThetaSketch +UniqThetaSketch +uniqThetaUnion +uniqUpTo +unix unixodbc +unixODBC unoptimized +unparsed +unrealiable +unreplicated +unresolvable +unrounded +untracked +untrusted +untuple +Updatable +UPDATEs +uploaders +Uppercased +upperUTF +uptime +Uptime +uptrace +Uptrace +uring +URIs url +urlCluster +URLHash +URLHierarchy +URLPathHierarchy +urls +URL's +UserID userspace userver +Util utils uuid +UUid +UUIDNumToString +UUIDs +UUIDStringToNum +Vadim +Valgrind +VARCHAR variadic varint +varpop +varPop +varsamp +varSamp vectorized +Vectorized vectorscan +verificationDepth +verificationMode +versionedcollapsingmergetree +VersionedCollapsingMergeTree +VersionInteger +vhost +VideoContainer +ViewAllLink +VIEWs +VirtualBox +virtualized +visibleWidth +visitParam +visitParamExtractBool +visitParamExtractFloat +visitParamExtractInt +visitParamExtractRaw +visitParamExtractString +visitParamExtractUInt +visitParamHas +WALs wchc wchs webpage webserver +weekyear +Welch's +welchttest +welchTTest +Werror +Wether wget +which's whitespace whitespaces +wikistat +WikiStat +windowFunnel +WindowView +WithNames +WithNamesAndTypes +Woboq +WordNet +wordshingleMinHash +wordShingleMinHash +wordShingleMinHashArg +wordShingleMinHashArgCaseInsensitive +wordShingleMinHashArgCaseInsensitiveUTF +wordShingleMinHashArgUTF +wordShingleMinHashCaseInsensitive +wordShingleMinHashCaseInsensitiveUTF +wordShingleMinHashUTF +wordShingleSimHash +wordShingleSimHashCaseInsensitive +wordShingleSimHashCaseInsensitiveUTF +wordShingleSimHashUTF +WriteBuffer +WriteBuffers wrt xcode +XCode +Xeon +xeus +XHTML +xkcd +xlarge xml +XORs +xxHash xz -zLib -zLinux +YAML +YAMLRegExpTree +yandex +Yandex +Yasm +youtube +YYYY +zabbix +Zabbix +Zipkin zkcopy zlib +zLib +zLinux +znode znodes +ZooKeeper +ZooKeeperRequest +ZooKeeper's +ZooKeepers +ZooKeeperSession +zookeeperSessionUptime +ZooKeeperWatch zstd diff --git a/utils/check-style/check-doc-aspell b/utils/check-style/check-doc-aspell index d39769aa930..952dbd5b507 100755 --- a/utils/check-style/check-doc-aspell +++ b/utils/check-style/check-doc-aspell @@ -1,5 +1,8 @@ #!/usr/bin/env bash +# force-enable double star globbing +shopt -s globstar + # Perform spell checking on the docs if [[ ${1:-} == "--help" ]] || [[ ${1:-} == "-h" ]]; then From c3a888a47bfd68c3737dee55fe1a0a2c92fa9341 Mon Sep 17 00:00:00 2001 From: Roman Vlasenko Date: Fri, 2 Jun 2023 15:41:43 +0300 Subject: [PATCH 1016/2223] Remove needless minus sign --- docs/ru/faq/integration/json-import.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/faq/integration/json-import.md b/docs/ru/faq/integration/json-import.md index bc65b5a614a..a3c89aed429 100644 --- a/docs/ru/faq/integration/json-import.md +++ b/docs/ru/faq/integration/json-import.md @@ -19,7 +19,7 @@ $ echo '{"foo":"bar"}' | curl 'http://localhost:8123/?query=INSERT%20INTO%20test При помощи [интефейса CLI](../../interfaces/cli.md): ``` bash -$ echo '{"foo":"bar"}' | clickhouse-client ---query="INSERT INTO test FORMAT JSONEachRow" +$ echo '{"foo":"bar"}' | clickhouse-client --query="INSERT INTO test FORMAT JSONEachRow" ``` Чтобы не вставлять данные вручную, используйте одну из [готовых библиотек](../../interfaces/index.md). @@ -31,4 +31,4 @@ $ echo '{"foo":"bar"}' | clickhouse-client ---query="INSERT INTO test FORMAT JS :::note "Примечание" В HTTP-интерфейсе настройки передаются через параметры `GET` запроса, в `CLI` interface — как дополнительные аргументы командной строки, начинающиеся с `--`. - ::: \ No newline at end of file + ::: From 079008058adbefea63edc8afffa0dd20d694e5e5 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 08:49:26 -0400 Subject: [PATCH 1017/2223] move from server to user settings --- .../settings.md | 51 ++----------------- docs/en/operations/settings/settings.md | 42 +++++++++++++++ 2 files changed, 46 insertions(+), 47 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 82be1c10dcc..d07fb80d1da 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -275,9 +275,9 @@ Type: UInt64 Default: 1000 -## max_concurrent_insert_queries +## max_concurrent_queries -Limit on total number of concurrent insert queries. Zero means Unlimited. +Limit on total number of concurrently executed queries. Zero means Unlimited. Note that limits on insert and select queries, and on the maximum number of queries for users must also be considered. See also max_concurrent_insert_queries, max_concurrent_select_queries, max_concurrent_queries_for_all_users. Zero means unlimited. :::note These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. @@ -287,9 +287,9 @@ Type: UInt64 Default: 0 -## max_concurrent_queries +## max_concurrent_insert_queries -Limit on total number of concurrently executed queries. Zero means Unlimited. Note that limits on insert and select queries, and on the maximum number of queries for users must also be considered. See also max_concurrent_insert_queries, max_concurrent_select_queries, max_concurrent_queries_for_all_users. Zero means unlimited. +Limit on total number of concurrent insert queries. Zero means Unlimited. :::note These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. @@ -1277,49 +1277,6 @@ For more information, see the section [Creating replicated tables](../../engines ``` - -## max_concurrent_queries_for_user {#max-concurrent-queries-for-user} - -The maximum number of simultaneously processed queries related to MergeTree table per user. - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `0`. - -**Example** - -``` xml -5 -``` - -## max_concurrent_queries_for_all_users {#max-concurrent-queries-for-all-users} - -Throw exception if the value of this setting is less or equal than the current number of simultaneously processed queries. - -Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. - -Modifying the setting for one query or user does not affect other queries. - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `0`. - -**Example** - -``` xml -99 -``` - -**See Also** - -- [max_concurrent_queries](#max-concurrent-queries) - ## max_open_files {#max-open-files} The maximum number of open files. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b868299aeff..374afb6bed7 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -646,6 +646,48 @@ Used for the same purpose as `max_block_size`, but it sets the recommended block However, the block size cannot be more than `max_block_size` rows. By default: 1,000,000. It only works when reading from MergeTree engines. +## max_concurrent_queries_for_user {#max-concurrent-queries-for-user} + +The maximum number of simultaneously processed queries related to MergeTree table per user. + +Possible values: + +- Positive integer. +- 0 — No limit. + +Default value: `0`. + +**Example** + +``` xml +5 +``` + +## max_concurrent_queries_for_all_users {#max-concurrent-queries-for-all-users} + +Throw exception if the value of this setting is less or equal than the current number of simultaneously processed queries. + +Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. + +Modifying the setting for one query or user does not affect other queries. + +Possible values: + +- Positive integer. +- 0 — No limit. + +Default value: `0`. + +**Example** + +``` xml +99 +``` + +**See Also** + +- [max_concurrent_queries](/docs/en/operations/server-configuration-parameters/settings.md/#max_concurrent_queries) + ## merge_tree_min_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} If the number of rows to be read from a file of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `merge_tree_min_rows_for_concurrent_read` then ClickHouse tries to perform a concurrent reading from this file on several threads. From 5cb4363e58bf1553bd80a930779cda2a79ef34b0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Jun 2023 14:44:28 +0200 Subject: [PATCH 1018/2223] Remove assertion --- src/Interpreters/Cache/FileSegment.cpp | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index fb0ba0eba14..f3a21749086 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -313,12 +313,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) if (!size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed"); - auto file_segment_path = getPathInLocalCache(); - if (offset == range().left && fs::exists(file_segment_path)) - { - fs::remove(file_segment_path); - chassert(false); - } + const auto file_segment_path = getPathInLocalCache(); { auto lock = segment_guard.lock(); @@ -358,7 +353,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) "Cache writer was finalized (downloaded size: {}, state: {})", current_downloaded_size, stateToString(download_state)); - cache_writer = std::make_unique(getPathInLocalCache()); + cache_writer = std::make_unique(file_segment_path); } } @@ -385,6 +380,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) const auto file_size = fs::file_size(file_segment_path); chassert(downloaded_size <= file_size); chassert(reserved_size >= file_size); + chassert(file_size <= range().right + 1); if (downloaded_size != file_size) downloaded_size = file_size; } From 21e9877098c24fc90319fae2682a815da8fef389 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Jun 2023 15:08:04 +0200 Subject: [PATCH 1019/2223] Add assertion --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index a60f5dffa96..561a66a826f 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -932,18 +932,23 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); -#ifdef ABORT_ON_LOGICAL_ERROR const size_t new_file_offset = file_offset_of_buffer_end + size; - chassert(new_file_offset - 1 <= file_segment.range().right); const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true); + if (new_file_offset > file_segment.range().right + 1) + { + auto file_segment_path = file_segment.getPathInLocalCache(); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Read unexpected size. File size: {}, file path: {}, file segment info: {}", + fs::file_size(file_segment_path), file_segment_path, file_segment.getInfoForLog()); + } if (new_file_offset > file_segment_write_offset) { - LOG_TRACE( - log, "Read {} bytes, file offset: {}, segment: {}, segment write offset: {}", + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Read unexpected size. Read {} bytes, file offset: {}, segment: {}, segment write offset: {}", size, file_offset_of_buffer_end, file_segment.range().toString(), file_segment_write_offset); - chassert(false); } -#endif } else { From e148c60d5a308b2ea86128021beba0e8321fef9b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 16:18:32 +0300 Subject: [PATCH 1020/2223] Fixes for MergeTree with readonly disks (#50244) * fixes for MergeTree with readonly disks * Automatic style fix * Update test.py * Automatic style fix * Update test.py * Update test.py * Automatic style fix * Update test.py --------- Co-authored-by: robot-clickhouse Co-authored-by: alesapin --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++ src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageMergeTree.cpp | 5 +- .../test_disk_over_web_server/test.py | 50 ++++++++++++++++--- ...02435_rollback_cancelled_queries.reference | 1 - .../02435_rollback_cancelled_queries.sh | 11 ++-- 6 files changed, 61 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 338a221e45e..32665429051 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4854,6 +4854,9 @@ void MergeTreeData::checkAlterPartitionIsPossible( void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition, ContextPtr local_context) { + if (!supportsReplication() && isStaticStorage()) + return; + DataPartsVector parts_to_remove; const auto * partition_ast = partition->as(); if (partition_ast && partition_ast->all) @@ -4874,6 +4877,9 @@ void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition, Context void MergeTreeData::checkPartCanBeDropped(const String & part_name) { + if (!supportsReplication() && isStaticStorage()) + return; + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); if (!part) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in committed state", part_name); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index fce7d989a2f..1c41de6fa19 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -865,7 +865,7 @@ public: DiskPtr tryGetDiskForDetachedPart(const String & part_name) const; DiskPtr getDiskForDetachedPart(const String & part_name) const; - bool storesDataOnDisk() const override { return true; } + bool storesDataOnDisk() const override { return !isStaticStorage(); } Strings getDataPaths() const override; /// Reserves space at least 1MB. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a721dd30cd7..3da4724471d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -114,7 +114,7 @@ StorageMergeTree::StorageMergeTree( loadDataParts(has_force_restore_data_flag); - if (!attach && !getDataPartsForInternalUsage().empty()) + if (!attach && !getDataPartsForInternalUsage().empty() && !isStaticStorage()) throw Exception(ErrorCodes::INCORRECT_DATA, "Data directory for table already containing data parts - probably " "it was unclean DROP table or manual intervention. " @@ -283,6 +283,9 @@ StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & met void StorageMergeTree::checkTableCanBeDropped() const { + if (!supportsReplication() && isStaticStorage()) + return; + auto table_id = getStorageID(); getContext()->checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes()); } diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index fd71389f71a..719de5e8bef 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -10,16 +10,22 @@ def cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance( - "node1", main_configs=["configs/storage_conf.xml"], with_nginx=True + "node1", + main_configs=["configs/storage_conf.xml"], + with_nginx=True, ) cluster.add_instance( "node2", main_configs=["configs/storage_conf_web.xml"], with_nginx=True, stay_alive=True, + with_zookeeper=True, ) cluster.add_instance( - "node3", main_configs=["configs/storage_conf_web.xml"], with_nginx=True + "node3", + main_configs=["configs/storage_conf_web.xml"], + with_nginx=True, + with_zookeeper=True, ) cluster.add_instance( @@ -95,7 +101,7 @@ def test_usage(cluster, node_name): for i in range(3): node2.query( """ - ATTACH TABLE test{} UUID '{}' + CREATE TABLE test{} UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( @@ -140,7 +146,7 @@ def test_incorrect_usage(cluster): global uuids node2.query( """ - ATTACH TABLE test0 UUID '{}' + CREATE TABLE test0 UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( @@ -173,7 +179,7 @@ def test_cache(cluster, node_name): for i in range(3): node2.query( """ - ATTACH TABLE test{} UUID '{}' + CREATE TABLE test{} UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'cached_web'; """.format( @@ -238,7 +244,7 @@ def test_unavailable_server(cluster): global uuids node2.query( """ - ATTACH TABLE test0 UUID '{}' + CREATE TABLE test0 UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( @@ -276,3 +282,35 @@ def test_unavailable_server(cluster): ) node2.start_clickhouse() node2.query("DROP TABLE test0 SYNC") + + +def test_replicated_database(cluster): + node1 = cluster.instances["node3"] + node1.query( + "CREATE DATABASE rdb ENGINE=Replicated('/test/rdb', 's1', 'r1')", + settings={"allow_experimental_database_replicated": 1}, + ) + + global uuids + node1.query( + """ + CREATE TABLE rdb.table0 UUID '{}' + (id Int32) ENGINE = MergeTree() ORDER BY id + SETTINGS storage_policy = 'web'; + """.format( + uuids[0] + ) + ) + + node2 = cluster.instances["node2"] + node2.query( + "CREATE DATABASE rdb ENGINE=Replicated('/test/rdb', 's1', 'r2')", + settings={"allow_experimental_database_replicated": 1}, + ) + node2.query("SYSTEM SYNC DATABASE REPLICA rdb") + + assert node1.query("SELECT count() FROM rdb.table0") == "5000000\n" + assert node2.query("SELECT count() FROM rdb.table0") == "5000000\n" + + node1.query("DROP DATABASE rdb SYNC") + node2.query("DROP DATABASE rdb SYNC") diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference b/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference index 2d32c17ec7c..38ff81b2371 100644 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference @@ -1,3 +1,2 @@ 1000000 0 -1 diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index 8f8e8cc7ee0..776d1f850b0 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -110,11 +110,12 @@ insert_data 1 $CLICKHOUSE_CLIENT --implicit_transaction=1 -q 'select throwIf(count() % 1000000 != 0 or count() = 0) from dedup_test' \ || $CLICKHOUSE_CLIENT -q "select name, rows, active, visible, creation_tid, creation_csn from system.parts where database=currentDatabase();" -# Ensure that thread_cancel actually did something -$CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( - message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes', - 'Query was cancelled or a client has unexpectedly dropped the connection') or - message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" +# Ensure that thread_cancel actually did something (useful when editing this test) +# We cannot check it in the CI, because sometimes it fails due to randomization +# $CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( +# message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes', +# 'Query was cancelled or a client has unexpectedly dropped the connection') or +# message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" wait_for_queries_to_finish 30 $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table dedup_test" From 8e076c33d51d6ecdfa209158d75305befd6ab308 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 2 Jun 2023 13:19:13 +0000 Subject: [PATCH 1021/2223] Try fix flaky test test_async_query_sending --- tests/integration/test_hedged_requests/configs/logger.xml | 5 +++++ tests/integration/test_hedged_requests/test.py | 5 ++++- 2 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_hedged_requests/configs/logger.xml diff --git a/tests/integration/test_hedged_requests/configs/logger.xml b/tests/integration/test_hedged_requests/configs/logger.xml new file mode 100644 index 00000000000..48fb4e91428 --- /dev/null +++ b/tests/integration/test_hedged_requests/configs/logger.xml @@ -0,0 +1,5 @@ + + + 20 + + \ No newline at end of file diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 2ca37fbb7ee..be6cea80f87 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -23,7 +23,7 @@ def started_cluster(): NODES["node"] = cluster.add_instance( "node", stay_alive=True, - main_configs=["configs/remote_servers.xml"], + main_configs=["configs/remote_servers.xml", "configs/logger.xml"], user_configs=["configs/users.xml"], ) @@ -377,6 +377,9 @@ def test_async_connect(started_cluster): check_changing_replica_events(2) check_if_query_sending_was_not_suspended() + # Restart server to reset connection pool state + NODES["node"].restart_clickhouse() + NODES["node"].query( "SELECT hostName(), id FROM distributed_connect ORDER BY id LIMIT 1 SETTINGS prefer_localhost_replica = 0, connect_timeout_with_failover_ms=5000, async_query_sending_for_remote=1, max_threads=1" ) From d9a4f8115fb7f362e93dc59e38a8d649d016e0e7 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 2 Jun 2023 13:19:41 +0000 Subject: [PATCH 1022/2223] Add new line delimiter --- tests/integration/test_hedged_requests/configs/logger.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_hedged_requests/configs/logger.xml b/tests/integration/test_hedged_requests/configs/logger.xml index 48fb4e91428..b341b14d43c 100644 --- a/tests/integration/test_hedged_requests/configs/logger.xml +++ b/tests/integration/test_hedged_requests/configs/logger.xml @@ -2,4 +2,4 @@ 20 - \ No newline at end of file + From 54872f9e7ea314da7f766c8929c212fd3d07ec21 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 13:27:56 +0000 Subject: [PATCH 1023/2223] Typos: Follow-up to #50476 --- .../mergetree-family/mergetree.md | 2 +- .../settings.md | 2 +- .../operations/settings/settings-formats.md | 2 +- docs/en/operations/settings/settings.md | 8 ++-- docs/en/operations/system-tables/quotas.md | 4 +- .../parametric-functions.md | 2 +- .../reference/exponentialmovingaverage.md | 4 +- docs/en/sql-reference/dictionaries/index.md | 6 +-- .../functions/array-functions.md | 4 +- .../functions/distance-functions.md | 2 +- .../functions/encryption-functions.md | 2 +- .../sql-reference/functions/hash-functions.md | 2 +- .../sql-reference/functions/math-functions.md | 2 +- .../functions/other-functions.md | 4 +- .../functions/type-conversion-functions.md | 2 +- docs/en/sql-reference/operators/index.md | 2 +- .../table-functions/urlCluster.md | 4 +- .../aspell-ignore/en/aspell-dict.txt | 45 ------------------- 18 files changed, 27 insertions(+), 72 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 1ab0f4057ff..07f706af91d 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -779,7 +779,7 @@ Disks, volumes and storage policies should be declared inside the ` Date: Fri, 2 Jun 2023 13:31:14 +0000 Subject: [PATCH 1024/2223] Apply uncommitted state after snapshot deser --- src/Coordination/KeeperStateMachine.cpp | 5 ++ src/Coordination/KeeperStorage.cpp | 51 ++++++++---- src/Coordination/KeeperStorage.h | 5 ++ src/Coordination/tests/gtest_coordination.cpp | 77 +++++++++++++++++++ 4 files changed, 124 insertions(+), 14 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 6635c74149a..a4568cbbdd3 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -363,6 +363,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) else if (s.get_last_log_idx() < latest_snapshot_meta->get_last_log_idx()) { LOG_INFO(log, "A snapshot with a larger last log index ({}) was created, skipping applying this snapshot", latest_snapshot_meta->get_last_log_idx()); + return true; } latest_snapshot_ptr = latest_snapshot_buf; @@ -372,6 +373,10 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) std::lock_guard lock(storage_and_responses_lock); auto snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx())); + + /// maybe some logs were preprocessed with log idx larger than the snapshot idx + /// we have to apply them to the new storage + storage->applyUncommittedState(*snapshot_deserialization_result.storage, s.get_last_log_idx()); storage = std::move(snapshot_deserialization_result.storage); latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; cluster_config = snapshot_deserialization_result.cluster_config; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 7a1a5e42632..66d6b0f5843 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -375,23 +375,26 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) delta.operation); } +void KeeperStorage::UncommittedState::addDelta(Delta new_delta) +{ + const auto & added_delta = deltas.emplace_back(std::move(new_delta)); + + if (!added_delta.path.empty()) + { + deltas_for_path[added_delta.path].push_back(&added_delta); + applyDelta(added_delta); + } + else if (const auto * auth_delta = std::get_if(&added_delta.operation)) + { + auto & uncommitted_auth = session_and_auth[auth_delta->session_id]; + uncommitted_auth.emplace_back(&auth_delta->auth_id); + } +} + void KeeperStorage::UncommittedState::addDeltas(std::vector new_deltas) { for (auto & delta : new_deltas) - { - const auto & added_delta = deltas.emplace_back(std::move(delta)); - - if (!added_delta.path.empty()) - { - deltas_for_path[added_delta.path].push_back(&added_delta); - applyDelta(added_delta); - } - else if (const auto * auth_delta = std::get_if(&added_delta.operation)) - { - auto & uncommitted_auth = session_and_auth[auth_delta->session_id]; - uncommitted_auth.emplace_back(&auth_delta->auth_id); - } - } + addDelta(std::move(delta)); } void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) @@ -602,6 +605,26 @@ namespace } +void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_zxid) +{ + for (const auto & transaction : uncommitted_transactions) + { + if (transaction.zxid <= last_zxid) + continue; + other.uncommitted_transactions.push_back(transaction); + } + + auto it = uncommitted_state.deltas.begin(); + + for (; it != uncommitted_state.deltas.end(); ++it) + { + if (it->zxid <= last_zxid) + continue; + + other.uncommitted_state.addDelta(*it); + } +} + Coordination::Error KeeperStorage::commit(int64_t commit_zxid) { // Deltas are added with increasing ZXIDs diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 7eb10be3847..844cbf85c1e 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -222,6 +222,7 @@ public: { explicit UncommittedState(KeeperStorage & storage_) : storage(storage_) { } + void addDelta(Delta new_delta); void addDeltas(std::vector new_deltas); void commit(int64_t commit_zxid); void rollback(int64_t rollback_zxid); @@ -310,6 +311,10 @@ public: UncommittedState uncommitted_state{*this}; + // Apply uncommitted state to another storage using only transactions + // with zxid > last_zxid + void applyUncommittedState(KeeperStorage & other, int64_t last_zxid); + Coordination::Error commit(int64_t zxid); // Create node in the storage diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 62217fb2dd3..453fd0f2e60 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2524,6 +2524,83 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest) } } +TEST_P(CoordinationTest, TestReapplyingDeltas) +{ + using namespace DB; + using namespace Coordination; + + static constexpr int64_t initial_zxid = 100; + + const auto create_request = std::make_shared(); + create_request->path = "/test/data"; + create_request->is_sequential = true; + + const auto process_create = [](KeeperStorage & storage, const auto & request, int64_t zxid) + { + storage.preprocessRequest(request, 1, 0, zxid); + auto responses = storage.processRequest(request, 1, zxid); + EXPECT_GE(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Error::ZOK); + }; + + const auto commit_initial_data = [&](auto & storage) + { + int64_t zxid = 1; + + const auto root_create = std::make_shared(); + root_create->path = "/test"; + process_create(storage, root_create, zxid); + ++zxid; + + for (; zxid <= initial_zxid; ++zxid) + process_create(storage, create_request, zxid); + }; + + KeeperStorage storage1{500, "", keeper_context}; + commit_initial_data(storage1); + + for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) + storage1.preprocessRequest(create_request, 1, 0, zxid); + + /// create identical new storage + KeeperStorage storage2{500, "", keeper_context}; + commit_initial_data(storage2); + + storage1.applyUncommittedState(storage2, initial_zxid); + + const auto commit_unprocessed = [&](KeeperStorage & storage) + { + for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) + { + auto responses = storage.processRequest(create_request, 1, zxid); + EXPECT_GE(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Error::ZOK); + } + }; + + commit_unprocessed(storage1); + commit_unprocessed(storage2); + + const auto get_children = [&](KeeperStorage & storage) + { + const auto list_request = std::make_shared(); + list_request->path = "/test"; + auto responses = storage.processRequest(list_request, 1, std::nullopt, /*check_acl=*/true, /*is_local=*/true); + EXPECT_EQ(responses.size(), 1); + const auto * list_response = dynamic_cast(responses[0].response.get()); + EXPECT_TRUE(list_response); + return list_response->names; + }; + + auto children1 = get_children(storage1); + std::unordered_set children1_set(children1.begin(), children1.end()); + + auto children2 = get_children(storage2); + std::unordered_set children2_set(children2.begin(), children2.end()); + + ASSERT_TRUE(children1_set == children2_set); +} + INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, CoordinationTest, ::testing::ValuesIn(std::initializer_list{ From 36129c3d20bef74c4502126b2c37001e2c80b8d2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Jun 2023 13:36:14 +0000 Subject: [PATCH 1025/2223] Try to fix last tests. --- src/Storages/MergeTree/RPNBuilder.cpp | 6 ++++-- .../0_stateless/00981_in_subquery_with_tuple.reference | 3 --- tests/queries/0_stateless/01651_bugs_from_15889.reference | 1 + tests/queries/0_stateless/01651_bugs_from_15889.sql | 5 ++--- 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index c9feb533ea3..b82c350fa1a 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -366,8 +366,10 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (indexes_mapping[i].tuple_index >= set_types.size()) return false; - auto lhs = recursiveRemoveLowCardinality(data_types[i]); - auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); + auto lhs = removeNullable(recursiveRemoveLowCardinality(data_types[i])); + auto rhs = removeNullable(recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index])); + + // std::cerr << "============ " << lhs->getName() << ' ' << rhs->getName() << std::endl; if (!lhs->equals(*rhs)) return false; diff --git a/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference b/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference index 673d035ede6..833a8c93d24 100644 --- a/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference +++ b/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference @@ -1,7 +1,4 @@ -OK1 OK2 OK3 -OK4 -OK5 2019-08-11 world 2019-08-12 hello diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.reference b/tests/queries/0_stateless/01651_bugs_from_15889.reference index 8b137891791..77ac542d4fb 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.reference +++ b/tests/queries/0_stateless/01651_bugs_from_15889.reference @@ -1 +1,2 @@ +0 diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index 4717a8dcc0d..6a8c6d35911 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -8,10 +8,9 @@ INSERT INTO xp SELECT '2020-01-01', number, '' FROM numbers(100000); CREATE TABLE xp_d AS xp ENGINE = Distributed(test_shard_localhost, currentDatabase(), xp); --- FIXME: this query spontaneously returns either 8 or 20 error code. Looks like it's potentially flaky. --- SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 8 } +SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 8 } -SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); -- { serverError 53 } +SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); DROP TABLE IF EXISTS xp; DROP TABLE IF EXISTS xp_d; From 2a4f1c82756b8937a2f87453f1bd76bc09a45114 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 14:00:57 +0000 Subject: [PATCH 1026/2223] Fix typos --- docs/en/sql-reference/functions/other-functions.md | 6 +----- docs/en/sql-reference/functions/random-functions.md | 2 +- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 6d49527fd0d..254297451f7 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2006,11 +2006,7 @@ isDecimalOverflow(d, [p]) **Arguments** - `d` — value. [Decimal](../../sql-reference/data-types/decimal.md). -<<<<<<< HEAD -- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. This paratemer can be helpful to migrate data from/to another database or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). -======= -- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. Using of this parameter could be helpful for data extraction to another DBMS or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). ->>>>>>> rschu1ze/master +- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. This parameter can be helpful to migrate data from/to another database or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). **Returned values** diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index 63d5174b494..e593d9458f0 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -67,7 +67,7 @@ randUniform(min, max) **Arguments** - `min` - `Float64` - left boundary of the range, -- `max` - `Float64` - reight boundary of the range. +- `max` - `Float64` - right boundary of the range. **Returned value** From 5e17adc9c0cab1ac911ccf6c7ad3cb1f8d8c7447 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 2 Jun 2023 14:08:14 +0000 Subject: [PATCH 1027/2223] Add `system.user_processes` table --- .../System/StorageSystemUserProcesses.cpp | 60 +++++++++++++++++++ .../System/StorageSystemUserProcesses.h | 29 +++++++++ src/Storages/System/attachSystemTables.cpp | 2 + 3 files changed, 91 insertions(+) create mode 100644 src/Storages/System/StorageSystemUserProcesses.cpp create mode 100644 src/Storages/System/StorageSystemUserProcesses.h diff --git a/src/Storages/System/StorageSystemUserProcesses.cpp b/src/Storages/System/StorageSystemUserProcesses.cpp new file mode 100644 index 00000000000..5973f9e2af3 --- /dev/null +++ b/src/Storages/System/StorageSystemUserProcesses.cpp @@ -0,0 +1,60 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList StorageSystemUserProcesses::getNamesAndTypes() +{ + return { + {"user", std::make_shared()}, + {"memory_usage", std::make_shared()}, + {"peak_memory_usage", std::make_shared()}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + }; +} + +NamesAndAliases StorageSystemUserProcesses::getNamesAndAliases() +{ + return { + {"ProfileEvents.Names", {std::make_shared(std::make_shared())}, "mapKeys(ProfileEvents)"}, + {"ProfileEvents.Values", {std::make_shared(std::make_shared())}, "mapValues(ProfileEvents)"}}; +} + +void StorageSystemUserProcesses::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +{ + const auto user_info = context->getProcessList().getUserInfo(true); + + for (const auto & [user, info] : user_info) + { + size_t i = 0; + + res_columns[i++]->insert(user); + res_columns[i++]->insert(info.memory_usage); + res_columns[i++]->insert(info.peak_memory_usage); + { + IColumn * column = res_columns[i++].get(); + + if (info.profile_counters) + ProfileEvents::dumpToMapColumn(*info.profile_counters, column, true); + else + { + column->insertDefault(); + } + } + } +} +} diff --git a/src/Storages/System/StorageSystemUserProcesses.h b/src/Storages/System/StorageSystemUserProcesses.h new file mode 100644 index 00000000000..9bdc009d849 --- /dev/null +++ b/src/Storages/System/StorageSystemUserProcesses.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/** Implements `processes` system table, which allows you to get information about the queries that are currently executing. + */ +class StorageSystemUserProcesses final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemUserProcesses"; } + + static NamesAndTypesList getNamesAndTypes(); + + static NamesAndAliases getNamesAndAliases(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 424c74662ec..7d21d9e39d2 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include #include @@ -185,6 +186,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "remote_data_paths"); attach(context, system_database, "certificates"); attach(context, system_database, "named_collections"); + attach(context, system_database, "user_processes"); if (has_zookeeper) { From 17cca6ed756eaaa58eae7ef6aa89e43dcda8ce24 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 10:08:48 -0400 Subject: [PATCH 1028/2223] add direct join docs --- .../integrations/embedded-rocksdb.md | 85 +++++++++++++++++++ 1 file changed, 85 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index a3604b3c332..dab741a9f63 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -120,3 +120,88 @@ Values can be updated using the `ALTER TABLE` query. The primary key cannot be u ```sql ALTER TABLE test UPDATE v1 = v1 * 10 + 2 WHERE key LIKE 'some%' AND v3 > 3.1; ``` + +### Joins + +A special `direct` join with EmbeddedRocksDB tables is supported. +This direct join avoids forming a hash table in memory and accesses +the data directly from the EmbeddedRocksDB. + +To enable direct joins: +```sql +SET join_algorithm = 'direct' +``` + +:::tip +When the `join_algorithm` is set to `direct`, direct joins will be used +when possible. However, direct joins are not used for RIGHT or FULL JOINs. +ClickHouse will choose another join algorithm when direct joins are not possible. +::: + +#### Example + +##### Create and populate an EmbeddedRocksDB table: +```sql +CREATE TABLE rdb +( + `key` UInt32, + `value` Array(UInt32), + `value2` String +) +ENGINE = EmbeddedRocksDB +PRIMARY KEY key +``` + +```sql +INSERT INTO rdb + SELECT + toUInt32(sipHash64(number) % 10) as key, + [key, key+1] as value, + ('val2' || toString(key)) as value2 + FROM numbers_mt(10); +``` + +##### Create and populate a table to join with table `rdb`: + +```sql +CREATE TABLE t2 +( + `k` UInt16 +) +ENGINE = TinyLog +``` + +```sql +INSERT INTO t2 SELECT number AS k +FROM numbers_mt(10) +``` + +##### Set the join algorithm to `direct`: + +```sql +SET join_algorithm = 'direct' +``` + +##### An INNER JOIN: +```sql +SELECT * +FROM +( + SELECT k AS key + FROM t2 +) AS t2 +INNER JOIN rdb ON rdb.key = t2.key +ORDER BY key ASC +``` +```response +┌─key─┬─rdb.key─┬─value──┬─value2─┐ +│ 0 │ 0 │ [0,1] │ val20 │ +│ 2 │ 2 │ [2,3] │ val22 │ +│ 3 │ 3 │ [3,4] │ val23 │ +│ 6 │ 6 │ [6,7] │ val26 │ +│ 7 │ 7 │ [7,8] │ val27 │ +│ 8 │ 8 │ [8,9] │ val28 │ +│ 9 │ 9 │ [9,10] │ val29 │ +└─────┴─────────┴────────┴────────┘ +``` + From 47966c4e8129c3777679e42966cfec9c7a2aa383 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Jun 2023 14:10:52 +0000 Subject: [PATCH 1029/2223] Finalize bufferg --- src/Coordination/KeeperSnapshotManager.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 8dd6d27c379..44e990c7b95 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -618,12 +618,14 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu auto disk = getLatestSnapshotDisk(); { - disk->writeFile(tmp_snapshot_file_name); + auto buf = disk->writeFile(tmp_snapshot_file_name); + buf->finalize(); } auto plain_buf = disk->writeFile(snapshot_file_name); copyData(reader, *plain_buf); plain_buf->sync(); + plain_buf->finalize(); disk->removeFile(tmp_snapshot_file_name); From d0da370d1e18c69e533f5d85725b7db4aa87d884 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 2 Jun 2023 16:19:24 +0200 Subject: [PATCH 1030/2223] Add a test for compound column identifier --- .../02771_resolve_compound_identifier.reference | 10 ++++++++++ .../0_stateless/02771_resolve_compound_identifier.sql | 11 +++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02771_resolve_compound_identifier.reference create mode 100644 tests/queries/0_stateless/02771_resolve_compound_identifier.sql diff --git a/tests/queries/0_stateless/02771_resolve_compound_identifier.reference b/tests/queries/0_stateless/02771_resolve_compound_identifier.reference new file mode 100644 index 00000000000..8b1acc12b63 --- /dev/null +++ b/tests/queries/0_stateless/02771_resolve_compound_identifier.reference @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/02771_resolve_compound_identifier.sql b/tests/queries/0_stateless/02771_resolve_compound_identifier.sql new file mode 100644 index 00000000000..db4d443379e --- /dev/null +++ b/tests/queries/0_stateless/02771_resolve_compound_identifier.sql @@ -0,0 +1,11 @@ +DROP DATABASE IF EXISTS test_02771; + +CREATE DATABASE test_02771; + +CREATE TABLE test_02771.t (x UInt8) ENGINE = MergeTree() ORDER BY x; + +INSERT INTO test_02771.t SELECT number FROM numbers(10); + +SELECT t.x FROM test_02771.t ORDER BY t.x; + +DROP DATABASE IF EXISTS test_02771; From ae497d398a5bcb48fef1d6b90e24fb0818e2bd05 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Jun 2023 14:23:04 +0000 Subject: [PATCH 1031/2223] Remove unneeded test --- .../__init__.py | 0 .../configs/enable_keeper1.xml | 41 -- .../configs/enable_keeper2.xml | 41 -- .../configs/enable_keeper3.xml | 41 -- .../configs/use_keeper.xml | 16 - .../test.py | 473 ------------------ 6 files changed, 612 deletions(-) delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/__init__.py delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/configs/use_keeper.xml delete mode 100644 tests/integration/test_keeper_multinode_blocade_leader/test.py diff --git a/tests/integration/test_keeper_multinode_blocade_leader/__init__.py b/tests/integration/test_keeper_multinode_blocade_leader/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml deleted file mode 100644 index 17455ed12f5..00000000000 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - 9181 - 1 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - - 5000 - 10000 - 75 - trace - - - - - 1 - node1 - 9234 - true - 3 - - - 2 - node2 - 9234 - true - true - 2 - - - 3 - node3 - 9234 - true - true - 1 - - - - diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml deleted file mode 100644 index 03a23984cc2..00000000000 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - 9181 - 2 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - - 5000 - 10000 - 75 - trace - - - - - 1 - node1 - 9234 - true - 3 - - - 2 - node2 - 9234 - true - true - 2 - - - 3 - node3 - 9234 - true - true - 1 - - - - diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml deleted file mode 100644 index a3196ac3061..00000000000 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - 9181 - 3 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - - 5000 - 10000 - 75 - trace - - - - - 1 - node1 - 9234 - true - 3 - - - 2 - node2 - 9234 - true - true - 2 - - - 3 - node3 - 9234 - true - true - 1 - - - - diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/use_keeper.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/use_keeper.xml deleted file mode 100644 index 384e984f210..00000000000 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/use_keeper.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - node1 - 9181 - - - node2 - 9181 - - - node3 - 9181 - - - diff --git a/tests/integration/test_keeper_multinode_blocade_leader/test.py b/tests/integration/test_keeper_multinode_blocade_leader/test.py deleted file mode 100644 index 3af0751b0fd..00000000000 --- a/tests/integration/test_keeper_multinode_blocade_leader/test.py +++ /dev/null @@ -1,473 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster -import helpers.keeper_utils as keeper_utils -import random -import string -import os -import time -from multiprocessing.dummy import Pool -from helpers.network import PartitionManager -from helpers.test_tools import assert_eq_with_retry - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", - main_configs=["configs/enable_keeper1.xml", "configs/use_keeper.xml"], - stay_alive=True, -) -node2 = cluster.add_instance( - "node2", - main_configs=["configs/enable_keeper2.xml", "configs/use_keeper.xml"], - stay_alive=True, -) -node3 = cluster.add_instance( - "node3", - main_configs=["configs/enable_keeper3.xml", "configs/use_keeper.xml"], - stay_alive=True, -) - -from kazoo.client import KazooClient, KazooState - -""" -In this test, we blockade RAFT leader and check that the whole system is -able to recover. It's not a good test because we use ClickHouse's replicated -tables to check connectivity, but they may require special operations (or a long -wait) after session expiration. We don't use kazoo, because this client pretends -to be very smart: SUSPEND sessions, try to recover them, and so on. The test -will be even less predictable than with ClickHouse tables. - -TODO find (or write) not so smart python client. -TODO remove this when jepsen tests will be written. -""" - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - - finally: - cluster.shutdown() - - -def smaller_exception(ex): - return "\n".join(str(ex).split("\n")[0:2]) - - -def get_fake_zk(nodename, timeout=30.0): - _fake_zk_instance = KazooClient( - hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout - ) - _fake_zk_instance.start() - return _fake_zk_instance - - -def wait_nodes(): - keeper_utils.wait_nodes(cluster, [node1, node2, node3]) - - -# in extremely rare case it can take more than 5 minutes in debug build with sanitizer -@pytest.mark.timeout(600) -def test_blocade_leader(started_cluster): - for i in range(100): - wait_nodes() - try: - for i, node in enumerate([node1, node2, node3]): - node.query( - "CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary", - settings={"allow_deprecated_database_ordinary": 1}, - ) - node.query( - "CREATE TABLE IF NOT EXISTS ordinary.t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format( - i + 1 - ) - ) - break - except Exception as ex: - print("Got exception from node", smaller_exception(ex)) - time.sleep(0.1) - - node2.query( - "INSERT INTO ordinary.t1 SELECT number FROM numbers(10) SETTINGS insert_keeper_max_retries = 0" - ) - - node1.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) - node3.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) - - assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t1", "10") - assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t1", "10") - assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t1", "10") - - with PartitionManager() as pm: - pm.partition_instances(node2, node1) - pm.partition_instances(node3, node1) - - for i in range(100): - try: - restart_replica_for_sure( - node2, "ordinary.t1", "/clickhouse/t1/replicas/2" - ) - node2.query( - "INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - try: - node2.query("ATTACH TABLE ordinary.t1") - except Exception as attach_ex: - print("Got exception node2", smaller_exception(attach_ex)) - print("Got exception node2", smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t1", "/clickhouse/t1/replicas/{}".format(num + 1) - ) - assert False, "Cannot insert anything node2" - - for i in range(100): - try: - restart_replica_for_sure( - node3, "ordinary.t1", "/clickhouse/t1/replicas/3" - ) - node3.query( - "INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - try: - node3.query("ATTACH TABLE ordinary.t1") - except Exception as attach_ex: - print("Got exception node3", smaller_exception(attach_ex)) - print("Got exception node3", smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t1", "/clickhouse/t1/replicas/{}".format(num + 1) - ) - assert False, "Cannot insert anything node3" - - for n, node in enumerate([node1, node2, node3]): - for i in range(100): - try: - restart_replica_for_sure( - node, "ordinary.t1", "/clickhouse/t1/replicas/{}".format(n + 1) - ) - break - except Exception as ex: - try: - node.query("ATTACH TABLE ordinary.t1") - except Exception as attach_ex: - print( - "Got exception node{}".format(n + 1), - smaller_exception(attach_ex), - ) - - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) - else: - assert False, "Cannot reconnect for node{}".format(n + 1) - - for i in range(100): - try: - node1.query( - "INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - print("Got exception node1", smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t1", "/clickhouse/t1/replicas/{}".format(num + 1) - ) - assert False, "Cannot insert anything node1" - - for n, node in enumerate([node1, node2, node3]): - for i in range(100): - try: - restart_replica_for_sure( - node, "ordinary.t1", "/clickhouse/t1/replicas/{}".format(n + 1) - ) - node.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) - break - except Exception as ex: - try: - node.query("ATTACH TABLE ordinary.t1") - except Exception as attach_ex: - print( - "Got exception node{}".format(n + 1), - smaller_exception(attach_ex), - ) - - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t1", "/clickhouse/t1/replicas/{}".format(num + 1) - ) - assert False, "Cannot sync replica node{}".format(n + 1) - - if node1.query("SELECT COUNT() FROM ordinary.t1") != "310\n": - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t1", "/clickhouse/t1/replicas/{}".format(num + 1) - ) - - assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t1", "310") - assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t1", "310") - assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t1", "310") - - -def dump_zk(node, zk_path, replica_path): - print(node.query("SELECT * FROM system.replication_queue FORMAT Vertical")) - print("Replicas") - print(node.query("SELECT * FROM system.replicas FORMAT Vertical")) - print("Replica 2 info") - print( - node.query( - "SELECT * FROM system.zookeeper WHERE path = '{}' FORMAT Vertical".format( - zk_path - ) - ) - ) - print("Queue") - print( - node.query( - "SELECT * FROM system.zookeeper WHERE path = '{}/queue' FORMAT Vertical".format( - replica_path - ) - ) - ) - print("Log") - print( - node.query( - "SELECT * FROM system.zookeeper WHERE path = '{}/log' FORMAT Vertical".format( - zk_path - ) - ) - ) - print("Parts") - print( - node.query( - "SELECT name FROM system.zookeeper WHERE path = '{}/parts' FORMAT Vertical".format( - replica_path - ) - ) - ) - - -def restart_replica_for_sure(node, table_name, zk_replica_path): - fake_zk = None - try: - node.query("DETACH TABLE {}".format(table_name)) - fake_zk = get_fake_zk(node.name) - if fake_zk.exists(zk_replica_path + "/is_active") is not None: - fake_zk.delete(zk_replica_path + "/is_active") - - node.query("ATTACH TABLE {}".format(table_name)) - except Exception as ex: - print("Exception", ex) - raise ex - finally: - if fake_zk: - fake_zk.stop() - fake_zk.close() - - -# in extremely rare case it can take more than 5 minutes in debug build with sanitizer -@pytest.mark.timeout(600) -def test_blocade_leader_twice(started_cluster): - for i in range(100): - wait_nodes() - try: - for i, node in enumerate([node1, node2, node3]): - node.query( - "CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary", - settings={"allow_deprecated_database_ordinary": 1}, - ) - node.query( - "CREATE TABLE IF NOT EXISTS ordinary.t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format( - i + 1 - ) - ) - break - except Exception as ex: - print("Got exception from node", smaller_exception(ex)) - time.sleep(0.1) - - node2.query( - "INSERT INTO ordinary.t2 SELECT number FROM numbers(10) SETTINGS insert_keeper_max_retries = 0" - ) - - node1.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - - assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t2", "10") - assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t2", "10") - assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t2", "10") - - with PartitionManager() as pm: - pm.partition_instances(node2, node1) - pm.partition_instances(node3, node1) - - for i in range(100): - try: - restart_replica_for_sure( - node2, "ordinary.t2", "/clickhouse/t2/replicas/2" - ) - node2.query( - "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - try: - node2.query("ATTACH TABLE ordinary.t2") - except Exception as attach_ex: - print("Got exception node2", smaller_exception(attach_ex)) - print("Got exception node2", smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - assert False, "Cannot reconnect for node2" - - for i in range(100): - try: - restart_replica_for_sure( - node3, "ordinary.t2", "/clickhouse/t2/replicas/3" - ) - node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - node3.query( - "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - try: - node3.query("ATTACH TABLE ordinary.t2") - except Exception as attach_ex: - print("Got exception node3", smaller_exception(attach_ex)) - print("Got exception node3", smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - assert False, "Cannot reconnect for node3" - - node2.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - - assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t2", "210") - assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t2", "210") - - # Total network partition - pm.partition_instances(node3, node2) - - for i in range(10): - try: - node3.query( - "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - assert False, "Node3 became leader?" - except Exception as ex: - time.sleep(0.5) - - for i in range(10): - try: - node2.query( - "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - assert False, "Node2 became leader?" - except Exception as ex: - time.sleep(0.5) - - for n, node in enumerate([node1, node2, node3]): - for i in range(100): - try: - restart_replica_for_sure( - node, "ordinary.t2", "/clickhouse/t2/replicas/{}".format(n + 1) - ) - break - except Exception as ex: - try: - node.query("ATTACH TABLE ordinary.t2") - except Exception as attach_ex: - print( - "Got exception node{}".format(n + 1), - smaller_exception(attach_ex), - ) - - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - assert False, "Cannot reconnect for node{}".format(n + 1) - - for n, node in enumerate([node1, node2, node3]): - for i in range(100): - try: - node.query( - "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" - ) - break - except Exception as ex: - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - assert False, "Cannot reconnect for node{}".format(n + 1) - - for i in range(100): - all_done = True - for n, node in enumerate([node1, node2, node3]): - try: - restart_replica_for_sure( - node, "ordinary.t2", "/clickhouse/t2/replicas/{}".format(n + 1) - ) - node.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - break - except Exception as ex: - all_done = False - try: - node.query("ATTACH TABLE ordinary.t2") - except Exception as attach_ex: - print( - "Got exception node{}".format(n + 1), - smaller_exception(attach_ex), - ) - - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) - - if all_done: - break - else: - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - assert False, "Cannot reconnect in i {} retries".format(i) - - assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t2", "510") - if node2.query("SELECT COUNT() FROM ordinary.t2") != "510\n": - for num, node in enumerate([node1, node2, node3]): - dump_zk( - node, "/clickhouse/t2", "/clickhouse/t2/replicas/{}".format(num + 1) - ) - - assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t2", "510") - assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t2", "510") From dcd5579851243a38ba9062636b72c2b60ebfadd1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 2 Jun 2023 16:51:04 +0200 Subject: [PATCH 1032/2223] Mention ON CLUSTER options for SYSTEM queries --- docs/en/sql-reference/statements/system.md | 54 ++++++++++++++-------- docs/ru/sql-reference/statements/system.md | 48 +++++++++++-------- docs/zh/sql-reference/statements/system.md | 40 +++++++++------- 3 files changed, 87 insertions(+), 55 deletions(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index c5596b7ba5f..65a35f03fbe 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -119,19 +119,35 @@ The compiled expression cache is enabled/disabled with the query/user/profile-le Resets the [query cache](../../operations/query-cache.md). +```sql +SYSTEM DROP QUERY CACHE [ON CLUSTER cluster_name] +``` + ## FLUSH LOGS Flushes buffered log messages to system tables, e.g. system.query_log. Mainly useful for debugging since most system tables have a default flush interval of 7.5 seconds. This will also create system tables even if message queue is empty. +```sql +SYSTEM FLUSH LOGS [ON CLUSTER cluster_name] +``` + ## RELOAD CONFIG Reloads ClickHouse configuration. Used when configuration is stored in ZooKeeper. Note that `SYSTEM RELOAD CONFIG` does not reload `USER` configuration stored in ZooKeeper, it only reloads `USER` configuration that is stored in `users.xml`. To reload all `USER` config use `SYSTEM RELOAD USERS` +```sql +SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name] +``` + ## RELOAD USERS Reloads all access storages, including: users.xml, local disk access storage, replicated (in ZooKeeper) access storage. +```sql +SYSTEM RELOAD USERS [ON CLUSTER cluster_name] +``` + ## SHUTDOWN Normally shuts down ClickHouse (like `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`) @@ -149,7 +165,7 @@ ClickHouse can manage [distributed](../../engines/table-engines/special/distribu Disables background data distribution when inserting data into distributed tables. ``` sql -SYSTEM STOP DISTRIBUTED SENDS [db.] +SYSTEM STOP DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ### FLUSH DISTRIBUTED @@ -157,7 +173,7 @@ SYSTEM STOP DISTRIBUTED SENDS [db.] Forces ClickHouse to send data to cluster nodes synchronously. If any nodes are unavailable, ClickHouse throws an exception and stops query execution. You can retry the query until it succeeds, which will happen when all nodes are back online. ``` sql -SYSTEM FLUSH DISTRIBUTED [db.] +SYSTEM FLUSH DISTRIBUTED [db.] [ON CLUSTER cluster_name] ``` ### START DISTRIBUTED SENDS @@ -165,7 +181,7 @@ SYSTEM FLUSH DISTRIBUTED [db.] Enables background data distribution when inserting data into distributed tables. ``` sql -SYSTEM START DISTRIBUTED SENDS [db.] +SYSTEM START DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ## Managing MergeTree Tables @@ -177,7 +193,7 @@ ClickHouse can manage background processes in [MergeTree](../../engines/table-en Provides possibility to stop background merges for tables in the MergeTree family: ``` sql -SYSTEM STOP MERGES [ON VOLUME | [db.]merge_tree_family_table_name] +SYSTEM STOP MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` :::note @@ -189,7 +205,7 @@ SYSTEM STOP MERGES [ON VOLUME | [db.]merge_tree_family_table_name] Provides possibility to start background merges for tables in the MergeTree family: ``` sql -SYSTEM START MERGES [ON VOLUME | [db.]merge_tree_family_table_name] +SYSTEM START MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` ### STOP TTL MERGES @@ -198,7 +214,7 @@ Provides possibility to stop background delete old data according to [TTL expres Returns `Ok.` even if table does not exist or table has not MergeTree engine. Returns error when database does not exist: ``` sql -SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM STOP TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START TTL MERGES @@ -207,7 +223,7 @@ Provides possibility to start background delete old data according to [TTL expre Returns `Ok.` even if table does not exist. Returns error when database does not exist: ``` sql -SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM START TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### STOP MOVES @@ -216,7 +232,7 @@ Provides possibility to stop background move data according to [TTL table expres Returns `Ok.` even if table does not exist. Returns error when database does not exist: ``` sql -SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] +SYSTEM STOP MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START MOVES @@ -225,7 +241,7 @@ Provides possibility to start background move data according to [TTL table expre Returns `Ok.` even if table does not exist. Returns error when database does not exist: ``` sql -SYSTEM START MOVES [[db.]merge_tree_family_table_name] +SYSTEM START MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### SYSTEM UNFREEZE {#query_language-system-unfreeze} @@ -241,7 +257,7 @@ SYSTEM UNFREEZE WITH NAME Wait until all asynchronously loading data parts of a table (outdated data parts) will became loaded. ``` sql -SYSTEM WAIT LOADING PARTS [db.]merge_tree_family_table_name +SYSTEM WAIT LOADING PARTS [ON CLUSTER cluster_name] [db.]merge_tree_family_table_name ``` ## Managing ReplicatedMergeTree Tables @@ -254,7 +270,7 @@ Provides possibility to stop background fetches for inserted parts for tables in Always returns `Ok.` regardless of the table engine and even if table or database does not exist. ``` sql -SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START FETCHES @@ -263,7 +279,7 @@ Provides possibility to start background fetches for inserted parts for tables i Always returns `Ok.` regardless of the table engine and even if table or database does not exist. ``` sql -SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATED SENDS @@ -271,7 +287,7 @@ SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] Provides possibility to stop background sends to other replicas in cluster for new inserted parts for tables in the `ReplicatedMergeTree` family: ``` sql -SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATED SENDS @@ -279,7 +295,7 @@ SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] Provides possibility to start background sends to other replicas in cluster for new inserted parts for tables in the `ReplicatedMergeTree` family: ``` sql -SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATION QUEUES @@ -287,7 +303,7 @@ SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] Provides possibility to stop background fetch tasks from replication queues which stored in Zookeeper for tables in the `ReplicatedMergeTree` family. Possible background tasks types - merges, fetches, mutation, DDL statements with ON CLUSTER clause: ``` sql -SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATION QUEUES @@ -295,7 +311,7 @@ SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] Provides possibility to start background fetch tasks from replication queues which stored in Zookeeper for tables in the `ReplicatedMergeTree` family. Possible background tasks types - merges, fetches, mutation, DDL statements with ON CLUSTER clause: ``` sql -SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### SYNC REPLICA @@ -318,7 +334,7 @@ Provides possibility to reinitialize Zookeeper session's state for `ReplicatedMe Initialization of replication queue based on ZooKeeper data happens in the same way as for `ATTACH TABLE` statement. For a short time, the table will be unavailable for any operations. ``` sql -SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name +SYSTEM RESTART REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name ``` ### RESTORE REPLICA @@ -384,7 +400,7 @@ Provides possibility to reinitialize Zookeeper sessions state for all `Replicate Allows to drop filesystem cache. ```sql -SYSTEM DROP FILESYSTEM CACHE +SYSTEM DROP FILESYSTEM CACHE [ON CLUSTER cluster_name] ``` ### SYNC FILE CACHE @@ -396,5 +412,5 @@ It's too heavy and has potential for misuse. Will do sync syscall. ```sql -SYSTEM SYNC FILE CACHE +SYSTEM SYNC FILE CACHE [ON CLUSTER cluster_name] ``` diff --git a/docs/ru/sql-reference/statements/system.md b/docs/ru/sql-reference/statements/system.md index 22a74648eab..ec30a031643 100644 --- a/docs/ru/sql-reference/statements/system.md +++ b/docs/ru/sql-reference/statements/system.md @@ -39,7 +39,7 @@ SELECT name, status FROM system.dictionaries; **Синтаксис** ```sql -SYSTEM RELOAD MODELS +SYSTEM RELOAD MODELS [ON CLUSTER cluster_name] ``` ## RELOAD MODEL {#query_language-system-reload-model} @@ -49,7 +49,7 @@ SYSTEM RELOAD MODELS **Синтаксис** ```sql -SYSTEM RELOAD MODEL +SYSTEM RELOAD MODEL [ON CLUSTER cluster_name] ``` ## RELOAD FUNCTIONS {#query_language-system-reload-functions} @@ -59,8 +59,8 @@ SYSTEM RELOAD MODEL **Синтаксис** ```sql -RELOAD FUNCTIONS -RELOAD FUNCTION function_name +RELOAD FUNCTIONS [ON CLUSTER cluster_name] +RELOAD FUNCTION function_name [ON CLUSTER cluster_name] ``` ## DROP DNS CACHE {#query_language-system-drop-dns-cache} @@ -106,10 +106,18 @@ Cкомпилированные выражения используются ко Записывает буферы логов в системные таблицы (например system.query_log). Позволяет не ждать 7.5 секунд при отладке. Если буфер логов пустой, то этот запрос просто создаст системные таблицы. +```sql +SYSTEM FLUSH LOGS [ON CLUSTER cluster_name] +``` + ## RELOAD CONFIG {#query_language-system-reload-config} Перечитывает конфигурацию настроек ClickHouse. Используется при хранении конфигурации в zookeeper. +```sql +SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name] +``` + ## SHUTDOWN {#query_language-system-shutdown} Штатно завершает работу ClickHouse (аналог `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`) @@ -127,7 +135,7 @@ ClickHouse может оперировать [распределёнными](.. Отключает фоновую отправку при вставке данных в распределённые таблицы. ``` sql -SYSTEM STOP DISTRIBUTED SENDS [db.] +SYSTEM STOP DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ### FLUSH DISTRIBUTED {#query_language-system-flush-distributed} @@ -135,7 +143,7 @@ SYSTEM STOP DISTRIBUTED SENDS [db.] В синхронном режиме отправляет все данные на узлы кластера. Если какие-либо узлы недоступны, ClickHouse генерирует исключение и останавливает выполнение запроса. Такой запрос можно повторять до успешного завершения, что будет означать возвращение связанности с остальными узлами кластера. ``` sql -SYSTEM FLUSH DISTRIBUTED [db.] +SYSTEM FLUSH DISTRIBUTED [db.] [ON CLUSTER cluster_name] ``` ### START DISTRIBUTED SENDS {#query_language-system-start-distributed-sends} @@ -143,7 +151,7 @@ SYSTEM FLUSH DISTRIBUTED [db.] Включает фоновую отправку при вставке данных в распределенные таблицы. ``` sql -SYSTEM START DISTRIBUTED SENDS [db.] +SYSTEM START DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ## Managing MergeTree Tables {#query-language-system-mergetree} @@ -155,7 +163,7 @@ ClickHouse может управлять фоновыми процессами Позволяет остановить фоновые мержи для таблиц семейства MergeTree: ``` sql -SYSTEM STOP MERGES [ON VOLUME | [db.]merge_tree_family_table_name] +SYSTEM STOP MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` :::note @@ -166,7 +174,7 @@ SYSTEM STOP MERGES [ON VOLUME | [db.]merge_tree_family_table_name] Включает фоновые мержи для таблиц семейства MergeTree: ``` sql -SYSTEM START MERGES [ON VOLUME | [db.]merge_tree_family_table_name] +SYSTEM START MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` ### STOP TTL MERGES {#query_language-stop-ttl-merges} @@ -175,7 +183,7 @@ SYSTEM START MERGES [ON VOLUME | [db.]merge_tree_family_table_name Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных: ``` sql -SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM STOP TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START TTL MERGES {#query_language-start-ttl-merges} @@ -184,7 +192,7 @@ SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name] Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных: ``` sql -SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM START TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### STOP MOVES {#query_language-stop-moves} @@ -193,7 +201,7 @@ SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name] Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных: ``` sql -SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] +SYSTEM STOP MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START MOVES {#query_language-start-moves} @@ -202,7 +210,7 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] Возвращает `Ok.` даже если указана несуществующая таблица или таблица имеет тип отличный от MergeTree. Возвращает ошибку если указана не существующая база данных: ``` sql -SYSTEM START MOVES [[db.]merge_tree_family_table_name] +SYSTEM START MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### SYSTEM UNFREEZE {#query_language-system-unfreeze} @@ -223,7 +231,7 @@ ClickHouse может управлять фоновыми процессами Всегда возвращает `Ok.` вне зависимости от типа таблицы и даже если таблица или база данных не существет. ``` sql -SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START FETCHES {#query_language-system-start-fetches} @@ -232,7 +240,7 @@ SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name] Всегда возвращает `Ok.` вне зависимости от типа таблицы и даже если таблица или база данных не существет. ``` sql -SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATED SENDS {#query_language-system-start-replicated-sends} @@ -240,7 +248,7 @@ SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] Позволяет остановить фоновые процессы отсылки новых вставленных кусков данных другим репликам в кластере для таблиц семейства `ReplicatedMergeTree`: ``` sql -SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATED SENDS {#query_language-system-start-replicated-sends} @@ -248,7 +256,7 @@ SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] Позволяет запустить фоновые процессы отсылки новых вставленных кусков данных другим репликам в кластере для таблиц семейства `ReplicatedMergeTree`: ``` sql -SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATION QUEUES {#query_language-system-stop-replication-queues} @@ -256,7 +264,7 @@ SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] Останавливает фоновые процессы разбора заданий из очереди репликации которая хранится в Zookeeper для таблиц семейства `ReplicatedMergeTree`. Возможные типы заданий - merges, fetches, mutation, DDL запросы с ON CLUSTER: ``` sql -SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATION QUEUES {#query_language-system-start-replication-queues} @@ -264,7 +272,7 @@ SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] Запускает фоновые процессы разбора заданий из очереди репликации которая хранится в Zookeeper для таблиц семейства `ReplicatedMergeTree`. Возможные типы заданий - merges, fetches, mutation, DDL запросы с ON CLUSTER: ``` sql -SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### SYNC REPLICA {#query_language-system-sync-replica} @@ -287,7 +295,7 @@ SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHT Инициализация очереди репликации на основе данных ZooKeeper происходит так же, как при `ATTACH TABLE`. Некоторое время таблица будет недоступна для любых операций. ``` sql -SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name +SYSTEM RESTART REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name ``` ### RESTORE REPLICA {#query_language-system-restore-replica} diff --git a/docs/zh/sql-reference/statements/system.md b/docs/zh/sql-reference/statements/system.md index 8fd2dd74d26..b41b62d72c3 100644 --- a/docs/zh/sql-reference/statements/system.md +++ b/docs/zh/sql-reference/statements/system.md @@ -71,10 +71,18 @@ SYSTEM DROP REPLICA 'replica_name' FROM ZKPATH '/path/to/table/in/zk'; 将日志信息缓冲数据刷入系统表(例如system.query_log)。调试时允许等待不超过7.5秒。当信息队列为空时,会创建系统表。 +```sql +SYSTEM FLUSH LOGS [ON CLUSTER cluster_name] +``` + ## RELOAD CONFIG {#query_language-system-reload-config} 重新加载ClickHouse的配置。用于当配置信息存放在ZooKeeper时。 +```sql +SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name] +``` + ## SHUTDOWN {#query_language-system-shutdown} 关闭ClickHouse服务(类似于 `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`) @@ -93,7 +101,7 @@ ClickHouse可以管理 [distribute](../../engines/table-engines/special/distribu 当向分布式表插入数据时,禁用后台的分布式数据分发。 ``` sql -SYSTEM STOP DISTRIBUTED SENDS [db.] +SYSTEM STOP DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ### FLUSH DISTRIBUTED {#query_language-system-flush-distributed} @@ -101,7 +109,7 @@ SYSTEM STOP DISTRIBUTED SENDS [db.] 强制让ClickHouse同步向集群节点同步发送数据。如果有节点失效,ClickHouse抛出异常并停止插入操作。当所有节点都恢复上线时,你可以重试之前的操作直到成功执行。 ``` sql -SYSTEM FLUSH DISTRIBUTED [db.] +SYSTEM FLUSH DISTRIBUTED [db.] [ON CLUSTER cluster_name] ``` ### START DISTRIBUTED SENDS {#query_language-system-start-distributed-sends} @@ -109,7 +117,7 @@ SYSTEM FLUSH DISTRIBUTED [db.] 当向分布式表插入数据时,允许后台的分布式数据分发。 ``` sql -SYSTEM START DISTRIBUTED SENDS [db.] +SYSTEM START DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_name] ``` ## Managing MergeTree Tables {#query-language-system-mergetree} @@ -121,7 +129,7 @@ ClickHouse可以管理 [MergeTree](../../engines/table-engines/mergetree-family/ 为MergeTree系列引擎表停止后台合并操作。 ``` sql -SYSTEM STOP MERGES [[db.]merge_tree_family_table_name] +SYSTEM STOP MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` :::note @@ -133,7 +141,7 @@ SYSTEM STOP MERGES [[db.]merge_tree_family_table_name] 为MergeTree系列引擎表启动后台合并操作。 ``` sql -SYSTEM START MERGES [[db.]merge_tree_family_table_name] +SYSTEM START MERGES [ON CLUSTER cluster_name] [ON VOLUME | [db.]merge_tree_family_table_name] ``` ### STOP TTL MERGES {#query_language-stop-ttl-merges} @@ -142,7 +150,7 @@ SYSTEM START MERGES [[db.]merge_tree_family_table_name] 不管表存在与否,都返回 `OK.`。当数据库不存在时返回错误。 ``` sql -SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM STOP TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START TTL MERGES {#query_language-start-ttl-merges} @@ -151,7 +159,7 @@ SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name] ``` sql -SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name] +SYSTEM START TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### STOP MOVES {#query_language-stop-moves} @@ -160,7 +168,7 @@ SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name] ``` sql -SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] +SYSTEM STOP MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### START MOVES {#query_language-start-moves} @@ -169,7 +177,7 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] ``` sql -SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] +SYSTEM START MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name] ``` ### SYSTEM UNFREEZE {#query_language-system-unfreeze} @@ -190,7 +198,7 @@ SYSTEM UNFREEZE WITH NAME 不管表引擎类型如何或表/数据库是否存,都返回 `OK.`。 ``` sql -SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START FETCHES {#query_language-system-start-fetches} @@ -199,7 +207,7 @@ SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name] 不管表引擎类型如何或表/数据库是否存,都返回 `OK.`。 ``` sql -SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATED SENDS {#query_language-system-start-replicated-sends} @@ -207,7 +215,7 @@ SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name] 停止通过后台分发 `ReplicatedMergeTree`系列引擎表中新插入的数据块到集群的其它副本节点。 ``` sql -SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATED SENDS {#query_language-system-start-replicated-sends} @@ -215,7 +223,7 @@ SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] 启动通过后台分发 `ReplicatedMergeTree`系列引擎表中新插入的数据块到集群的其它副本节点。 ``` sql -SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### STOP REPLICATION QUEUES {#query_language-system-stop-replication-queues} @@ -224,7 +232,7 @@ SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name] 停止从Zookeeper中获取 `ReplicatedMergeTree`系列表的复制队列的后台任务。可能的后台任务类型包含:merges, fetches, mutation,带有 `ON CLUSTER`的ddl语句 ``` sql -SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM STOP REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### START REPLICATION QUEUES {#query_language-system-start-replication-queues} @@ -232,7 +240,7 @@ SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] 启动从Zookeeper中获取 `ReplicatedMergeTree`系列表的复制队列的后台任务。可能的后台任务类型包含:merges, fetches, mutation,带有 `ON CLUSTER`的ddl语句 ``` sql -SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name] +SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` ### SYNC REPLICA {#query_language-system-sync-replica} @@ -250,7 +258,7 @@ SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHT ``` sql -SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name +SYSTEM RESTART REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name ``` ### RESTART REPLICAS {#query_language-system-restart-replicas} From 8e4a2a42fbab37b1d4511d45d6b01580172689db Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Jun 2023 16:51:09 +0200 Subject: [PATCH 1033/2223] Some code --- src/Access/Common/AccessType.h | 1 + .../AzureBlobStorage/AzureObjectStorage.h | 12 +- src/Storages/StorageAzure.cpp | 518 ++++++++++++++++++ src/Storages/StorageAzure.h | 121 ++++ src/Storages/registerStorages.cpp | 12 + 5 files changed, 659 insertions(+), 5 deletions(-) create mode 100644 src/Storages/StorageAzure.cpp create mode 100644 src/Storages/StorageAzure.h diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 6394c0279a7..84c99939f2d 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -210,6 +210,7 @@ enum class AccessType M(HDFS, "", GLOBAL, SOURCES) \ M(S3, "", GLOBAL, SOURCES) \ M(HIVE, "", GLOBAL, SOURCES) \ + M(AZURE, "", GLOBAL, SOURCES) \ M(SOURCES, "", GROUP, ALL) \ \ M(CLUSTER, "", GLOBAL, ALL) /* ON CLUSTER queries */ \ diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 0c2aecd5c62..1be1eb69799 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -37,11 +37,13 @@ struct AzureObjectStorageSettings { } - size_t max_single_part_upload_size; /// NOTE: on 32-bit machines it will be at most 4GB, but size_t is also used in BufferBase for offset - uint64_t min_bytes_for_seek; - size_t max_single_read_retries; - size_t max_single_download_retries; - int list_object_keys_size; + AzureObjectStorageSettings() = default; + + size_t max_single_part_upload_size = 100 * 1024 * 1024; /// NOTE: on 32-bit machines it will be at most 4GB, but size_t is also used in BufferBase for offset + uint64_t min_bytes_for_seek = 1024 * 1024; + size_t max_single_read_retries = 3; + size_t max_single_download_retries = 3; + int list_object_keys_size = 1000; }; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp new file mode 100644 index 00000000000..30fd3fcbe95 --- /dev/null +++ b/src/Storages/StorageAzure.cpp @@ -0,0 +1,518 @@ +#include + + +#if USE_AZURE_BLOB_STORAGE +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +using namespace Azure::Storage::Blobs; + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; + extern const int DATABASE_ACCESS_DENIED; +} + +bool isConnectionString(const std::string & candidate) +{ + return candidate.starts_with("DefaultEndpointsProtocol"); +} + +StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file) +{ + StorageAzure::Configuration configuration; + + /// Supported signatures: + /// + /// Azure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) + + if (engine_args.size() < 3 || engine_args.size() > 7) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage Azure requires 3 to 7 arguments: " + "Azure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression])"); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); + + std::unordered_map engine_args_to_idx; + + configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); + configuration.is_connection_string = isConnectionString(configuration.connection_url); + + configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); + configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); + + auto is_format_arg = [] (const std::string & s) -> bool + { + return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); + }; + + if (engine_args.size() == 4) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + configuration.format = fourth_arg; + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format or account name specified without account key"); + } + } + else if (engine_args.size() == 5) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + configuration.format = fourth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + } + } + else if (engine_args.size() == 6) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; + } + } + else if (engine_args.size() == 7) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); + } + } + + configuration.blobs_paths = {configuration.blob_path}; + + if (configuration.format == "auto" && get_format_from_file) + configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); + + return configuration; +} + + +void registerStorageAzure(StorageFactory & factory) +{ + factory.registerStorage("Azure", [](const StorageFactory::Arguments & args) + { + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + + auto configuration = StorageAzure::getConfiguration(engine_args, args.getLocalContext()); + auto client = StorageAzure::createClient(configuration); + // Use format settings from global server context + settings from + // the SETTINGS clause of the create query. Settings from current + // session and user are ignored. + std::optional format_settings; + if (args.storage_def->settings) + { + FormatFactorySettings user_format_settings; + + // Apply changed settings from global context, but ignore the + // unknown ones, because we only have the format settings here. + const auto & changes = args.getContext()->getSettingsRef().changes(); + for (const auto & change : changes) + { + if (user_format_settings.has(change.name)) + user_format_settings.set(change.name, change.value); + } + + // Apply changes from SETTINGS clause, with validation. + user_format_settings.applyChanges(args.storage_def->settings->changes); + format_settings = getFormatSettings(args.getContext(), user_format_settings); + } + else + { + format_settings = getFormatSettings(args.getContext()); + } + + ASTPtr partition_by; + if (args.storage_def->partition_by) + partition_by = args.storage_def->partition_by->clone(); + + return std::make_shared( + std::move(configuration), + std::make_unique("AzureStorage", std::move(client), std::make_unique()), + args.getContext(), + args.table_id, + args.columns, + args.constraints, + args.comment, + format_settings, + partition_by); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::AZURE, + }); +} + +AzureClientPtr StorageAzure::createClient(StorageAzure::Configuration configuration) +{ + AzureClientPtr result; + + if (configuration.is_connection_string) + { + result = std::make_unique(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container)); + } + else + { + if (configuration.account_name.has_value() && configuration.account_key.has_value()) + { + auto storage_shared_key_credential = std::make_shared(*configuration.account_name, *configuration.account_key); + result = std::make_unique(configuration.connection_url, storage_shared_key_credential); + } + + auto managed_identity_credential = std::make_shared(); + + result = std::make_unique(configuration.connection_url, managed_identity_credential); + } + + return result; +} + +StorageAzure::StorageAzure( + const Configuration & configuration_, + std::unique_ptr && object_storage_, + ContextPtr context_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + std::optional format_settings_, + ASTPtr partition_by_) + : IStorage(table_id_) + , name("AzureBlobStorage") + , configuration(configuration_) + , object_storage(std::move(object_storage_)) + , distributed_processing(false) + , format_settings(format_settings_) + , partition_by(partition_by_) +{ + FormatFactory::instance().checkFormatName(configuration.format); + context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(configuration.getConnectionURL())); + + StorageInMemoryMetadata storage_metadata; + if (columns_.empty()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Schema inference is not supported yet"); + //auto columns = getTableStructureFromDataImpl(configuration, format_settings, context_); + //storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + + auto default_virtuals = NamesAndTypesList{ + {"_path", std::make_shared(std::make_shared())}, + {"_file", std::make_shared(std::make_shared())}}; + + auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList(); + virtual_columns = getVirtualsForStorage(columns, default_virtuals); + for (const auto & column : virtual_columns) + virtual_block.insert({column.type->createColumn(), column.type, column.name}); +} + +void StorageAzure::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) +{ + + if (configuration.withGlobs()) + { + throw Exception( + ErrorCodes::DATABASE_ACCESS_DENIED, + "S3 key '{}' contains globs, so the table is in readonly mode", + configuration.blob_path); + } + + StoredObjects objects; + for (const auto & key : configuration.blobs_paths) + objects.emplace_back(key); + + object_storage->removeObjects(objects); +} + +namespace +{ + +class StorageAzureSink : public SinkToStorage +{ +public: + StorageAzureSink( + const String & format, + const Block & sample_block_, + ContextPtr context, + std::optional format_settings_, + const CompressionMethod compression_method, + AzureObjectStorage * object_storage, + const String & blob_path) + : SinkToStorage(sample_block_) + , sample_block(sample_block_) + , format_settings(format_settings_) + { + StoredObject object(blob_path); + write_buf = wrapWriteBufferWithCompressionMethod(object_storage->writeObject(object, WriteMode::Rewrite), compression_method, 3); + writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings); + } + + String getName() const override { return "StorageS3Sink"; } + + void consume(Chunk chunk) override + { + std::lock_guard lock(cancel_mutex); + if (cancelled) + return; + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); + } + + void onCancel() override + { + std::lock_guard lock(cancel_mutex); + finalize(); + cancelled = true; + } + + void onException() override + { + std::lock_guard lock(cancel_mutex); + finalize(); + } + + void onFinish() override + { + std::lock_guard lock(cancel_mutex); + finalize(); + } + +private: + void finalize() + { + if (!writer) + return; + + try + { + writer->finalize(); + writer->flush(); + write_buf->finalize(); + } + catch (...) + { + /// Stop ParallelFormattingOutputFormat correctly. + writer.reset(); + write_buf->finalize(); + throw; + } + } + + Block sample_block; + std::optional format_settings; + std::unique_ptr write_buf; + OutputFormatPtr writer; + bool cancelled = false; + std::mutex cancel_mutex; +}; + +class PartitionedStorageAzureSink : public PartitionedSink +{ +public: + PartitionedStorageAzureSink( + const ASTPtr & partition_by, + const String & format_, + const Block & sample_block_, + ContextPtr context_, + std::optional format_settings_, + const CompressionMethod compression_method_, + AzureObjectStorage * object_storage_, + const String & blob_) + : PartitionedSink(partition_by, context_, sample_block_) + , format(format_) + , sample_block(sample_block_) + , context(context_) + , compression_method(compression_method_) + , object_storage(object_storage_) + , blob(blob_) + , format_settings(format_settings_) + { + } + + SinkPtr createSinkForPartition(const String & partition_id) override + { + auto partition_key = replaceWildcards(blob, partition_id); + validateKey(partition_key); + + return std::make_shared( + format, + sample_block, + context, + format_settings, + compression_method, + object_storage, + partition_key + ); + } + +private: + const String format; + const Block sample_block; + const ContextPtr context; + const CompressionMethod compression_method; + AzureObjectStorage * object_storage; + const String blob; + const std::optional format_settings; + + ExpressionActionsPtr partition_by_expr; + + static void validateKey(const String & str) + { + validatePartitionKey(str, true); + } +}; + +} + +SinkToStoragePtr StorageAzure::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +{ + auto sample_block = metadata_snapshot->getSampleBlock(); + auto chosen_compression_method = chooseCompressionMethod(configuration.blobs_paths.back(), configuration.compression_method); + auto insert_query = std::dynamic_pointer_cast(query); + + auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; + bool is_partitioned_implementation = partition_by_ast && configuration.withWildcard(); + + if (is_partitioned_implementation) + { + return std::make_shared( + partition_by_ast, + configuration.format, + sample_block, + local_context, + format_settings, + chosen_compression_method, + object_storage.get(), + configuration.blobs_paths.back()); + } + else + { + if (configuration.withGlobs()) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, + "Azure key '{}' contains globs, so the table is in readonly mode", configuration.blob_path); + + bool truncate_in_insert = local_context->getSettingsRef().s3_truncate_on_insert; + + if (!truncate_in_insert && object_storage->exists(StoredObject(configuration.blob_path))) + { + if (local_context->getSettingsRef().s3_create_new_file_on_insert) + { + size_t index = configuration.blobs_paths.size(); + const auto & first_key = configuration.blobs_paths[0]; + auto pos = first_key.find_first_of('.'); + String new_key; + do + { + new_key = first_key.substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : first_key.substr(pos)); + ++index; + } + while (object_storage->exists(StoredObject(new_key))); + + configuration.blobs_paths.push_back(new_key); + } + else + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Object in bucket {} with key {} already exists. " + "If you want to overwrite it, enable setting s3_truncate_on_insert, if you " + "want to create a new file on each insert, enable setting s3_create_new_file_on_insert", + configuration.container, configuration.blobs_paths.back()); + } + } + + return std::make_shared( + configuration.format, + sample_block, + local_context, + format_settings, + chosen_compression_method, + object_storage.get(), + configuration.blobs_paths.back()); + } +} + +NamesAndTypesList StorageAzure::getVirtuals() const +{ + return virtual_columns; +} + +bool StorageAzure::supportsPartitionBy() const +{ + return true; +} + +} + +#endif diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h new file mode 100644 index 00000000000..b99df2e89a5 --- /dev/null +++ b/src/Storages/StorageAzure.h @@ -0,0 +1,121 @@ +#pragma once + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include +#include +#include + +namespace DB +{ + +struct AzureSimpleAccountConfiguration +{ + std::string storage_account_url; +}; + +using AzureConnectionString = std::string; + +using AzureCredentials = std::variant; + +class StorageAzure : public IStorage +{ +public: + using AzureClient = Azure::Storage::Blobs::BlobContainerClient; + using AzureClientPtr = std::unique_ptr; + + struct Configuration : public StatelessTableEngineConfiguration + { + Configuration() = default; + + String getPath() const { return blob_path; } + + bool update(ContextPtr context); + + void connect(ContextPtr context); + + bool withGlobs() const { return blob_path.find_first_of("*?{") != std::string::npos; } + + bool withWildcard() const + { + static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + return blobs_paths.back().find(PARTITION_ID_WILDCARD) != String::npos; + } + + std::string getConnectionURL() const + { + if (!is_connection_string) + return connection_url; + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Connection string not implemented yet"); + } + + std::string connection_url; + bool is_connection_string; + + std::optional account_name; + std::optional account_key; + + std::string container; + std::string blob_path; + std::vector blobs_paths; + }; + + StorageAzure( + const Configuration & configuration_, + std::unique_ptr && object_storage_, + ContextPtr context_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + std::optional format_settings_, + ASTPtr partition_by_); + + static StorageAzure::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file = true); + static AzureClientPtr createClient(StorageAzure::Configuration configuration); + + String getName() const override + { + return name; + } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr context) override; + + void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override; + + NamesAndTypesList getVirtuals() const override; + + bool supportsPartitionBy() const override; + + static SchemaCache & getSchemaCache(const ContextPtr & ctx); + +private: + std::string name; + Configuration configuration; + std::unique_ptr object_storage; + NamesAndTypesList virtual_columns; + Block virtual_block; + + const bool distributed_processing; + std::optional format_settings; + ASTPtr partition_by; + +}; + +} + +#endif diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 8be176a5375..03bd4dbb310 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -94,8 +94,16 @@ void registerStorageFileLog(StorageFactory & factory); void registerStorageSQLite(StorageFactory & factory); #endif + + void registerStorageKeeperMap(StorageFactory & factory); +#if USE_AZURE_BLOB_STORAGE + +void registerStorageAzure(StorageFactory & factory); + +#endif + void registerStorages() { auto & factory = StorageFactory::instance(); @@ -191,6 +199,10 @@ void registerStorages() #endif registerStorageKeeperMap(factory); + + #if USE_AZURE_BLOB_STORAGE + registerStorageAzure(factory); + #endif } } From 897325967841cbd24de32f3a136ceb26385b75b9 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 2 Jun 2023 15:00:24 +0000 Subject: [PATCH 1034/2223] Test attach gdb in stateless tests --- docker/test/stateless/run.sh | 39 ++++++++++++++++++++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 5d0a7b50741..dfee7d84cde 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -85,6 +85,45 @@ fi sleep 5 +# Set follow-fork-mode to parent, because we attach to clickhouse-server, not to watchdog +# and clickhouse-server can do fork-exec, for example, to run some bridge. +# Do not set nostop noprint for all signals, because some it may cause gdb to hang, +# explicitly ignore non-fatal signals that are used by server. +# Number of SIGRTMIN can be determined only in runtime. +RTMIN=$(kill -l SIGRTMIN) +echo " +set follow-fork-mode parent +handle SIGHUP nostop noprint pass +handle SIGINT nostop noprint pass +handle SIGQUIT nostop noprint pass +handle SIGPIPE nostop noprint pass +handle SIGTERM nostop noprint pass +handle SIGUSR1 nostop noprint pass +handle SIGUSR2 nostop noprint pass +handle SIG$RTMIN nostop noprint pass +info signals +continue +backtrace full +thread apply all backtrace full +info registers +disassemble /s +up +disassemble /s +up +disassemble /s +p \"done\" +detach +quit +" > script.gdb + +# FIXME Hung check may work incorrectly because of attached gdb +# 1. False positives are possible +# 2. We cannot attach another gdb to get stacktraces if some queries hung +gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log & +sleep 5 +# gdb will send SIGSTOP, spend some time loading debug info and then send SIGCONT, wait for it (up to send_timeout, 300s) +time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||: + function run_tests() { set -x From 975e58c56d5cf7038052e0be9699c094c7203161 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Jun 2023 15:15:44 +0000 Subject: [PATCH 1035/2223] Fix style. --- src/Interpreters/PreparedSets.cpp | 1 - src/Planner/CollectSets.cpp | 2 ++ src/Processors/QueryPlan/CreatingSetsStep.cpp | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 1e475e8403d..6df7c748e60 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int SET_SIZE_LIMIT_EXCEEDED; } PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index d264810645a..5f44994c14b 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -81,6 +81,8 @@ public: element_type = low_cardinality_type->getDictionaryType(); auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + if (sets.getFuture(set_key)) + return; sets.addFromTuple(set_key, std::move(set), settings); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 0909ee9f1eb..264c4b9ef47 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -46,6 +46,7 @@ CreatingSetStep::CreatingSetStep( , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { + std::cerr << StackTrace().toString() << std::endl; } void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) From c19866f72d3b356f2c4fed334c2beeecd441ef0e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Jun 2023 15:16:36 +0000 Subject: [PATCH 1036/2223] Fix style. --- src/Processors/QueryPlan/CreatingSetsStep.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 264c4b9ef47..0909ee9f1eb 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -46,7 +46,6 @@ CreatingSetStep::CreatingSetStep( , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { - std::cerr << StackTrace().toString() << std::endl; } void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) From 963641b20ffd09c01107fe8b210d38218ca85161 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 18:24:56 +0300 Subject: [PATCH 1037/2223] disable 00534_functions_bad_arguments with msan (#50481) --- tests/queries/0_stateless/00534_functions_bad_arguments10.sh | 2 +- tests/queries/0_stateless/00534_functions_bad_arguments5.sh | 2 +- tests/queries/0_stateless/00534_functions_bad_arguments6.sh | 2 +- tests/queries/0_stateless/00534_functions_bad_arguments9.sh | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments10.sh b/tests/queries/0_stateless/00534_functions_bad_arguments10.sh index 8525b63a989..b9733f92812 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments10.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments10.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug +# Tags: no-tsan, no-debug, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments5.sh b/tests/queries/0_stateless/00534_functions_bad_arguments5.sh index a8b0ce77677..812ba9f97fa 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments5.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments5.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug, no-fasttest +# Tags: no-tsan, no-debug, no-fasttest, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments6.sh b/tests/queries/0_stateless/00534_functions_bad_arguments6.sh index b0080c3b418..6626a6dfe55 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments6.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments6.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug +# Tags: no-tsan, no-debug, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments9.sh b/tests/queries/0_stateless/00534_functions_bad_arguments9.sh index 2975643020b..c7659db8621 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments9.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments9.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug +# Tags: no-tsan, no-debug, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 From a0901b1d1cb938759e6bcca37d0b03df0c1929e9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 2 Jun 2023 15:33:38 +0000 Subject: [PATCH 1038/2223] Add tests --- .../02771_system_user_processes.reference | 6 ++++++ .../0_stateless/02771_system_user_processes.sh | 15 +++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02771_system_user_processes.reference create mode 100755 tests/queries/0_stateless/02771_system_user_processes.sh diff --git a/tests/queries/0_stateless/02771_system_user_processes.reference b/tests/queries/0_stateless/02771_system_user_processes.reference new file mode 100644 index 00000000000..ab0ff41ddc5 --- /dev/null +++ b/tests/queries/0_stateless/02771_system_user_processes.reference @@ -0,0 +1,6 @@ +0 +0 +default +test_user_02771 +default true true +test_user_02771 2 2 diff --git a/tests/queries/0_stateless/02771_system_user_processes.sh b/tests/queries/0_stateless/02771_system_user_processes.sh new file mode 100755 index 00000000000..e8bf88a9fb2 --- /dev/null +++ b/tests/queries/0_stateless/02771_system_user_processes.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS test_user_02771" +$CLICKHOUSE_CLIENT -q "CREATE USER test_user_02771" +$CLICKHOUSE_CLIENT -u test_user_02771 -q "SELECT * FROM system.numbers LIMIT 1" +$CLICKHOUSE_CLIENT -u test_user_02771 -q "SELECT * FROM system.numbers LIMIT 1" +$CLICKHOUSE_CLIENT -q "SELECT user FROM system.user_processes" +$CLICKHOUSE_CLIENT -q "SELECT user, toBool(ProfileEvents['SelectQuery'] > 0), toBool(ProfileEvents['Query'] > 0) FROM system.user_processes WHERE user='default'" +$CLICKHOUSE_CLIENT -q "SELECT user, ProfileEvents['SelectQuery'], ProfileEvents['Query'] FROM system.user_processes WHERE user='test_user_02771'" +$CLICKHOUSE_CLIENT -q "DROP USER test_user_02771" + From 4209ccfc088f832c49d7c530b517e73a2661504a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 17:42:04 +0200 Subject: [PATCH 1039/2223] fix --- .../MergeTree/ReplicatedMergeTreeSink.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 53 +++++++++++-------- tests/config/config.d/merge_tree.xml | 2 + tests/integration/test_ttl_replicated/test.py | 6 +-- 4 files changed, 37 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 3efd364fc9c..8d9e2e14129 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -87,6 +87,7 @@ private: size_t checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. + /// Returns a list of conflicting async blocks and true if the whole parts was deduplicated std::pair, bool> commitPart( const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 44403fc708b..0a61369e163 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1936,6 +1936,30 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::executeFetchShared( } } +static void paranoidCheckForCoveredPartsInZooKeeper(const ZooKeeperPtr & zookeeper, const String & replica_path, + MergeTreeDataFormatVersion format_version, const String & covering_part_name) +{ +#ifdef ABORT_ON_LOGICAL_ERROR + constexpr bool paranoid_check_for_covered_parts_default = true; +#else + constexpr bool paranoid_check_for_covered_parts_default = false; +#endif + + bool paranoid_check_for_covered_parts = Context::getGlobalContextInstance()->getConfigRef().getBool( + "replicated_merge_tree_paranoid_check_on_drop_range", paranoid_check_for_covered_parts_default); + if (paranoid_check_for_covered_parts) + { + auto drop_range_info = MergeTreePartInfo::fromPartName(covering_part_name, format_version); + Strings parts_remain = zookeeper->getChildren(replica_path + "/parts"); + for (const auto & part_name : parts_remain) + { + auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + if (drop_range_info.contains(part_info)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Part {} remains in ZooKeeper after DROP_RANGE {}", part_name, covering_part_name); + } + } +} void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) { @@ -1992,16 +2016,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) /// Forcibly remove parts from ZooKeeper removePartsFromZooKeeperWithRetries(parts_to_remove); - -#ifdef ABORT_ON_LOGICAL_ERROR - Strings parts_remain = getZooKeeper()->getChildren(replica_path + "/parts"); - for (const auto & part_name : parts_remain) - { - auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); - if (drop_range_info.contains(part_info)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} remains in ZooKeeper after DROP_RANGE {}", part_name, entry.new_part_name); - } -#endif + paranoidCheckForCoveredPartsInZooKeeper(getZooKeeper(), replica_path, format_version, entry.new_part_name); if (entry.detach) LOG_DEBUG(log, "Detached {} parts inside {}.", parts_to_remove.size(), entry.new_part_name); @@ -2137,6 +2152,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) { LOG_INFO(log, "All parts from REPLACE PARTITION command have been already attached"); removePartsFromZooKeeperWithRetries(parts_to_remove); + if (replace) + paranoidCheckForCoveredPartsInZooKeeper(getZooKeeper(), replica_path, format_version, entry_replace.drop_range_part_name); return true; } @@ -2433,6 +2450,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } removePartsFromZooKeeperWithRetries(parts_to_remove); + if (replace) + paranoidCheckForCoveredPartsInZooKeeper(getZooKeeper(), replica_path, format_version, entry_replace.drop_range_part_name); res_parts.clear(); parts_to_remove.clear(); cleanup_thread.wakeup(); @@ -7133,7 +7152,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); } - PartsToRemoveFromZooKeeper parts_to_remove; Coordination::Responses op_results; try @@ -7184,7 +7202,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( auto data_parts_lock = lockParts(); transaction.commit(&data_parts_lock); if (replace) - parts_to_remove = removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(NO_TRANSACTION_RAW, drop_range, data_parts_lock); + removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, data_parts_lock); } PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed(), profile_events_scope.getSnapshot())); @@ -7204,11 +7222,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( for (auto & lock : ephemeral_locks) lock.assumeUnlocked(); - /// Forcibly remove replaced parts from ZooKeeper - removePartsFromZooKeeperWithRetries(parts_to_remove); - - /// Speedup removing of replaced parts from filesystem - parts_to_remove.clear(); cleanup_thread.wakeup(); lock2.reset(); @@ -7377,7 +7390,6 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); - PartsToRemoveFromZooKeeper parts_to_remove; Coordination::Responses op_results; try @@ -7414,7 +7426,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta else zkutil::KeeperMultiException::check(code, ops, op_results); - parts_to_remove = removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(NO_TRANSACTION_RAW, drop_range, src_data_parts_lock); + removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, src_data_parts_lock); transaction.commit(&src_data_parts_lock); } @@ -7436,9 +7448,6 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta for (auto & lock : ephemeral_locks) lock.assumeUnlocked(); - removePartsFromZooKeeperWithRetries(parts_to_remove); - - parts_to_remove.clear(); cleanup_thread.wakeup(); lock2.reset(); diff --git a/tests/config/config.d/merge_tree.xml b/tests/config/config.d/merge_tree.xml index bf2da9b09a2..bee9812274c 100644 --- a/tests/config/config.d/merge_tree.xml +++ b/tests/config/config.d/merge_tree.xml @@ -3,4 +3,6 @@ 1 8 + + 1 diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index a3e7d6e4b8b..39d66d857ff 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -517,7 +517,7 @@ def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_delete_{suff}', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 3 SECOND - SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100 + SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100, remove_empty_parts=0 """.format( suff=num_run, replica=node.name ) @@ -529,7 +529,7 @@ def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_group_by_{suff}', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 3 SECOND GROUP BY id SET val = sum(val) - SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100 + SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100, remove_empty_parts=0 """.format( suff=num_run, replica=node.name ) @@ -541,7 +541,7 @@ def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_where_{suff}', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 3 SECOND DELETE WHERE id % 2 = 1 - SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100 + SETTINGS max_number_of_merges_with_ttl_in_pool=100, max_replicated_merges_with_ttl_in_queue=100, remove_empty_parts=0 """.format( suff=num_run, replica=node.name ) From 179b6aca6aec1962fe3690d037aa0dae27531f5d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 17:52:57 +0200 Subject: [PATCH 1040/2223] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0a61369e163..5b7616d5f28 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2846,6 +2846,8 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo } LOG_DEBUG(log, "Copied {} queue entries, {} entries ignored", total_entries_to_copy, source_queue.size() - total_entries_to_copy); + LOG_TRACE(log, "Parts in ZooKeeper after mimic: {}", fmt::join(zookeeper->getChildren(replica_path + "/parts"), ", ")); + LOG_TRACE(log, "Enqueued fetches after mimic: {}", fmt::join(created_get_parts, ", ")); } From 163b2f32da72a66d44967439446331b9943361f3 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Jun 2023 18:13:46 +0200 Subject: [PATCH 1041/2223] Update src/Common/AsynchronousMetrics.cpp --- src/Common/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index cf8d451385b..8cd33521cbb 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -948,7 +948,7 @@ void AsynchronousMetrics::update(TimePoint update_time) std::string line; readText(line, *cgroupcpu_max); - auto space = line.find_first_of(" "); + auto space = line.find(" "); if (line.rfind("max", space) == std::string::npos) { From 9d0a63bd9263ba9dc9f9181d2fdd9898d269530f Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 2 Jun 2023 18:15:20 +0200 Subject: [PATCH 1042/2223] Update src/Common/AsynchronousMetrics.cpp --- src/Common/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 8cd33521cbb..6821647a180 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -948,7 +948,7 @@ void AsynchronousMetrics::update(TimePoint update_time) std::string line; readText(line, *cgroupcpu_max); - auto space = line.find(" "); + auto space = line.find(' '); if (line.rfind("max", space) == std::string::npos) { From da4d55cdaf4e25e16ddbf9028e6c8f5d336c60f6 Mon Sep 17 00:00:00 2001 From: Valentin Alexeev Date: Fri, 2 Jun 2023 14:02:26 +0200 Subject: [PATCH 1043/2223] Additional error information when JSON is too large If a parser fails on a large JSON, then output the last position processed to allow review. --- src/Formats/JSONUtils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 284cffdb9d7..7b7c073b6b2 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -45,9 +45,9 @@ namespace JSONUtils const auto current_object_size = memory.size() + static_cast(pos - in.position()); if (min_bytes != 0 && current_object_size > 10 * min_bytes) throw ParsingException(ErrorCodes::INCORRECT_DATA, - "Size of JSON object is extremely large. Expected not greater than {} bytes, but current is {} bytes per row. " + "Size of JSON object at position {} is extremely large. Expected not greater than {} bytes, but current is {} bytes per row. " "Increase the value setting 'min_chunk_bytes_for_parallel_parsing' or check your data manually, " - "most likely JSON is malformed", min_bytes, current_object_size); + "most likely JSON is malformed", pos, min_bytes, current_object_size); if (quotes) { From 71c5b1d9c67b8a8fca252aa56932b13deeaa12f3 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 12:42:35 -0400 Subject: [PATCH 1044/2223] add svg function docs --- docs/en/sql-reference/functions/geo/svg.md | 52 ++++++++++++++++++++++ 1 file changed, 52 insertions(+) create mode 100644 docs/en/sql-reference/functions/geo/svg.md diff --git a/docs/en/sql-reference/functions/geo/svg.md b/docs/en/sql-reference/functions/geo/svg.md new file mode 100644 index 00000000000..9081ac71338 --- /dev/null +++ b/docs/en/sql-reference/functions/geo/svg.md @@ -0,0 +1,52 @@ +--- +slug: /en/sql-reference/functions/geo/svg +sidebar_label: SVG +title: "Functions for Generating SVG images from Geo data" +--- + +## Syntax + +``` sql +SVG(geometry,[style]) +``` + +### Parameters + +- `geometry` — Geo data +- `style` — Optional style name + +### Returned value + +- The SVG representation of the geometry: + - SVG circle + - SVG polygon + - SVG path + +Type: String + +## Examples + +### Circle +```sql +SELECT SVG((0., 0.)) +``` +```response + +``` + +### Polygon +```sql +SELECT SVG([(0., 0.), (10, 0), (10, 10), (0, 10)]) +``` +```response + +``` + +### Path +```sql +SELECT SVG([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]]) +``` +```response + +``` + From 516cda94eeb6c822b12697fd32921cc79ea97c15 Mon Sep 17 00:00:00 2001 From: Valentin Alexeev Date: Fri, 2 Jun 2023 17:14:21 +0200 Subject: [PATCH 1045/2223] Use in.count() instead of pos --- src/Formats/JSONUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 7b7c073b6b2..0aac72c68fe 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -47,7 +47,7 @@ namespace JSONUtils throw ParsingException(ErrorCodes::INCORRECT_DATA, "Size of JSON object at position {} is extremely large. Expected not greater than {} bytes, but current is {} bytes per row. " "Increase the value setting 'min_chunk_bytes_for_parallel_parsing' or check your data manually, " - "most likely JSON is malformed", pos, min_bytes, current_object_size); + "most likely JSON is malformed", in.count(), min_bytes, current_object_size); if (quotes) { From 9b8975194821fe44018ed5bcbc9d5ae088b970f5 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 12:52:06 -0400 Subject: [PATCH 1046/2223] add docs for parallel view processing --- docs/en/operations/settings/settings.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 6c951739d41..5730503a670 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4229,6 +4229,12 @@ Default value: `2000` If it's enabled, in hedged requests we can start new connection until receiving first data packet even if we have already made some progress (but progress haven't updated for `receive_data_timeout` timeout), otherwise we disable changing replica after the first time we made progress. +## parallel_view_processing + +Enables pushing to attached views concurrently instead of sequentially. + +Default value: `false`. + ## partial_result_on_first_cancel {#partial_result_on_first_cancel} When set to `true` and the user wants to interrupt a query (for example using `Ctrl+C` on the client), then the query continues execution only on data that was already read from the table. Afterwards, it will return a partial result of the query for the part of the table that was read. To fully stop the execution of a query without a partial result, the user should send 2 cancel requests. From d28b4181e94c5602b5512af8ed541dcc2a1a55f2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 2 Jun 2023 16:57:36 +0000 Subject: [PATCH 1047/2223] Add `SHOW USER PROCESSES` query --- src/Interpreters/InterpreterFactory.cpp | 6 ++++ .../InterpreterShowUserProcessesQuery.cpp | 18 +++++++++++ .../InterpreterShowUserProcessesQuery.h | 30 +++++++++++++++++ src/Parsers/ASTShowUserProcessesQuery.h | 17 ++++++++++ src/Parsers/ParserQueryWithOutput.cpp | 5 ++- src/Parsers/ParserShowUserProcessesQuery.h | 32 +++++++++++++++++++ 6 files changed, 107 insertions(+), 1 deletion(-) create mode 100644 src/Interpreters/InterpreterShowUserProcessesQuery.cpp create mode 100644 src/Interpreters/InterpreterShowUserProcessesQuery.h create mode 100644 src/Parsers/ASTShowUserProcessesQuery.h create mode 100644 src/Parsers/ParserShowUserProcessesQuery.h diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 9cd1f2a251c..c31e3801478 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -81,6 +82,7 @@ #include #include #include +#include #include #include #include @@ -266,6 +268,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Interpreters/InterpreterShowUserProcessesQuery.cpp b/src/Interpreters/InterpreterShowUserProcessesQuery.cpp new file mode 100644 index 00000000000..51287a7ad5b --- /dev/null +++ b/src/Interpreters/InterpreterShowUserProcessesQuery.cpp @@ -0,0 +1,18 @@ +#include + +#include +#include +#include + +#include + + +namespace DB +{ + +BlockIO InterpreterShowUserProcessesQuery::execute() +{ + return executeQuery("SELECT * FROM system.user_processes ORDER BY user DESC", getContext(), true); +} + +} diff --git a/src/Interpreters/InterpreterShowUserProcessesQuery.h b/src/Interpreters/InterpreterShowUserProcessesQuery.h new file mode 100644 index 00000000000..a1c385dc82f --- /dev/null +++ b/src/Interpreters/InterpreterShowUserProcessesQuery.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/** Return list of currently executing queries. +TODO(antaljanosbenjamin) + */ +class InterpreterShowUserProcessesQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterShowUserProcessesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + + /// We ignore the quota and limits here because execute() will rewrite a show query as a SELECT query and then + /// the SELECT query will checks the quota and limits. + bool ignoreQuota() const override { return true; } + bool ignoreLimits() const override { return true; } + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Parsers/ASTShowUserProcessesQuery.h b/src/Parsers/ASTShowUserProcessesQuery.h new file mode 100644 index 00000000000..cd522c152b6 --- /dev/null +++ b/src/Parsers/ASTShowUserProcessesQuery.h @@ -0,0 +1,17 @@ +#pragma once + +#include + + +namespace DB +{ + +struct ASTShowUserProcessesIDAndQueryNames +{ + static constexpr auto ID = "ShowUserProcesses"; + static constexpr auto Query = "SHOW USER PROCESSES"; +}; + +using ASTShowUserProcessesQuery = ASTQueryWithOutputImpl; + +} diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 6796f4528c4..d5293e5f709 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -61,6 +62,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserShowGrantsQuery show_grants_p; ParserShowPrivilegesQuery show_privileges_p; ParserExplainQuery explain_p(end, allow_settings_after_format_in_insert); + ParserShowUserProcessesQuery show_user_processes_p; ASTPtr query; @@ -88,7 +90,8 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec || show_access_p.parse(pos, query, expected) || show_access_entities_p.parse(pos, query, expected) || show_grants_p.parse(pos, query, expected) - || show_privileges_p.parse(pos, query, expected); + || show_privileges_p.parse(pos, query, expected) + || show_user_processes_p.parse(pos, query, expected); if (!parsed) return false; diff --git a/src/Parsers/ParserShowUserProcessesQuery.h b/src/Parsers/ParserShowUserProcessesQuery.h new file mode 100644 index 00000000000..be484e74d5d --- /dev/null +++ b/src/Parsers/ParserShowUserProcessesQuery.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/** Query SHOW USER PROCESSES + */ +class ParserShowUserProcessesQuery : public IParserBase +{ +protected: + const char * getName() const override { return "SHOW USER PROCESSES query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + auto query = std::make_shared(); + + if (!ParserKeyword("SHOW USER PROCESSES").ignore(pos, expected)) + return false; + + node = query; + + return true; + } +}; + +} From 96fe4b5107611a627b7981fdac2afe9304660e48 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 2 Jun 2023 16:57:46 +0000 Subject: [PATCH 1048/2223] Add tests --- .../02771_system_user_processes.reference | 5 ++--- .../02771_system_user_processes.sh | 19 +++++++++++-------- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02771_system_user_processes.reference b/tests/queries/0_stateless/02771_system_user_processes.reference index ab0ff41ddc5..8c8ca8abb52 100644 --- a/tests/queries/0_stateless/02771_system_user_processes.reference +++ b/tests/queries/0_stateless/02771_system_user_processes.reference @@ -1,6 +1,5 @@ +SHOW USER PROCESSES query succeeded! 0 0 -default -test_user_02771 default true true -test_user_02771 2 2 +2 2 diff --git a/tests/queries/0_stateless/02771_system_user_processes.sh b/tests/queries/0_stateless/02771_system_user_processes.sh index e8bf88a9fb2..910af4be9e2 100755 --- a/tests/queries/0_stateless/02771_system_user_processes.sh +++ b/tests/queries/0_stateless/02771_system_user_processes.sh @@ -4,12 +4,15 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS test_user_02771" -$CLICKHOUSE_CLIENT -q "CREATE USER test_user_02771" -$CLICKHOUSE_CLIENT -u test_user_02771 -q "SELECT * FROM system.numbers LIMIT 1" -$CLICKHOUSE_CLIENT -u test_user_02771 -q "SELECT * FROM system.numbers LIMIT 1" -$CLICKHOUSE_CLIENT -q "SELECT user FROM system.user_processes" -$CLICKHOUSE_CLIENT -q "SELECT user, toBool(ProfileEvents['SelectQuery'] > 0), toBool(ProfileEvents['Query'] > 0) FROM system.user_processes WHERE user='default'" -$CLICKHOUSE_CLIENT -q "SELECT user, ProfileEvents['SelectQuery'], ProfileEvents['Query'] FROM system.user_processes WHERE user='test_user_02771'" -$CLICKHOUSE_CLIENT -q "DROP USER test_user_02771" +USER_POSTFIX=`random_str 10` +USER="test_user_02771_$USER_POSTFIX" + +$CLICKHOUSE_CLIENT -q "SHOW USER PROCESSES" &>"${CLICKHOUSE_TMP}/test_output" && echo "SHOW USER PROCESSES query succeeded!" || cat "${CLICKHOUSE_TMP}/test_output" +$CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS $USER" +$CLICKHOUSE_CLIENT -q "CREATE USER $USER" +$CLICKHOUSE_CLIENT -u "$USER" -q "SELECT * FROM system.numbers LIMIT 1" +$CLICKHOUSE_CLIENT -u "$USER" -q "SELECT * FROM system.numbers LIMIT 1" +$CLICKHOUSE_CLIENT -q "SELECT user, toBool(ProfileEvents['SelectQuery'] > 0), toBool(ProfileEvents['Query'] > 0) FROM system.user_processes WHERE user='default'" +$CLICKHOUSE_CLIENT -q "SELECT ProfileEvents['SelectQuery'], ProfileEvents['Query'] FROM system.user_processes WHERE user='$USER'" +$CLICKHOUSE_CLIENT -q "DROP USER $USER" From 0d98a46326ca671cb0ff0540972c6eba5280d565 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Jun 2023 13:02:18 -0400 Subject: [PATCH 1049/2223] Fix KeyError in cherry-pick --- tests/ci/cherry_pick.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index d36315151aa..07cdcc76c3a 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -71,11 +71,11 @@ This pull-request will be merged automatically as it reaches the mergeable state ### If the PR was closed and then reopened If it stuck, check {pr_url} for `{backport_created_label}` and delete it if \ -necessary. Manually merging will do nothing, since `{label_backports_created}` \ +necessary. Manually merging will do nothing, since `{backport_created_label}` \ prevents the original PR {pr_url} from being processed. If you want to recreate the PR: delete the `{label_cherrypick}` label and delete this branch. -You may also need to delete the `{label_backports_created}` label from the original PR. +You may also need to delete the `{backport_created_label}` label from the original PR. """ BACKPORT_DESCRIPTION = """This pull-request is a last step of an automated \ backporting. From 423afec70542c266187d49cf571d5f6bb4324977 Mon Sep 17 00:00:00 2001 From: tpanetti Date: Fri, 2 Jun 2023 10:05:38 -0700 Subject: [PATCH 1050/2223] Change case and function name for MySQL Compatible types This changes the function name for MySQL Compatible types from getMySQLName to getSQLCompatibleName and changes the casing of the types to upper --- src/DataTypes/DataTypeAggregateFunction.h | 2 +- src/DataTypes/DataTypeArray.h | 4 +- src/DataTypes/DataTypeDate.h | 2 +- src/DataTypes/DataTypeDate32.h | 2 +- src/DataTypes/DataTypeDateTime.h | 2 +- src/DataTypes/DataTypeDateTime64.h | 2 +- src/DataTypes/DataTypeEnum.cpp | 2 +- src/DataTypes/DataTypeEnum.h | 2 +- src/DataTypes/DataTypeFixedString.h | 2 +- src/DataTypes/DataTypeFunction.h | 2 +- src/DataTypes/DataTypeIPv4andIPv6.h | 4 +- src/DataTypes/DataTypeInterval.h | 2 +- src/DataTypes/DataTypeLowCardinality.cpp | 2 +- src/DataTypes/DataTypeLowCardinality.h | 2 +- src/DataTypes/DataTypeMap.h | 2 +- src/DataTypes/DataTypeNothing.h | 2 +- src/DataTypes/DataTypeNullable.h | 2 +- src/DataTypes/DataTypeNumberBase.cpp | 28 +- src/DataTypes/DataTypeNumberBase.h | 2 +- src/DataTypes/DataTypeObject.h | 2 +- src/DataTypes/DataTypeSet.h | 2 +- src/DataTypes/DataTypeString.h | 2 +- src/DataTypes/DataTypeTuple.h | 2 +- src/DataTypes/DataTypeUUID.h | 2 +- src/DataTypes/DataTypesDecimal.h | 4 +- src/DataTypes/IDataType.h | 4 +- ...show_columns_mysql_compatibility.reference | 424 +++++++++--------- .../02775_show_columns_mysql_compatibility.sh | 6 +- 28 files changed, 260 insertions(+), 256 deletions(-) diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index 697be13652c..13ca3508580 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -45,7 +45,7 @@ public: String doGetName() const override; String getNameWithoutVersion() const; const char * getFamilyName() const override { return "AggregateFunction"; } - const char * getMySQLName() const override { return "text"; } + const char * getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; } Array getParameters() const { return parameters; } diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index b031f411975..528062b60be 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -30,9 +30,9 @@ public: { return "Array"; } - const char * getMySQLName() const override + const char * getSQLCompatibleName() const override { - return "text"; + return "TEXT"; } bool canBeInsideNullable() const override diff --git a/src/DataTypes/DataTypeDate.h b/src/DataTypes/DataTypeDate.h index 33bcb6123ff..7b622ae04a3 100644 --- a/src/DataTypes/DataTypeDate.h +++ b/src/DataTypes/DataTypeDate.h @@ -13,7 +13,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Date; } const char * getFamilyName() const override { return family_name; } - const char * getMySQLName() const override { return "date"; } + const char * getSQLCompatibleName() const override { return "DATE"; } bool canBeUsedAsVersion() const override { return true; } bool canBeInsideNullable() const override { return true; } diff --git a/src/DataTypes/DataTypeDate32.h b/src/DataTypes/DataTypeDate32.h index 56315f46e8c..65b0ec7407e 100644 --- a/src/DataTypes/DataTypeDate32.h +++ b/src/DataTypes/DataTypeDate32.h @@ -13,7 +13,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Date32; } const char * getFamilyName() const override { return family_name; } - const char * getMySQLName() const override { return "date"; } + const char * getSQLCompatibleName() const override { return "DATE"; } Field getDefault() const override { diff --git a/src/DataTypes/DataTypeDateTime.h b/src/DataTypes/DataTypeDateTime.h index c868f92c311..2facc758f90 100644 --- a/src/DataTypes/DataTypeDateTime.h +++ b/src/DataTypes/DataTypeDateTime.h @@ -36,7 +36,7 @@ public: static constexpr auto family_name = "DateTime"; const char * getFamilyName() const override { return family_name; } - const char * getMySQLName() const override { return "datetime"; } + const char * getSQLCompatibleName() const override { return "DATETIME"; } String doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::DateTime; } diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index 8d317bb9430..b836b84918f 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -28,7 +28,7 @@ public: DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info); const char * getFamilyName() const override { return family_name; } - const char * getMySQLName() const override { return "datetime"; } + const char * getSQLCompatibleName() const override { return "DATETIME"; } std::string doGetName() const override; TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index bfed4d4d5a2..24a3976179d 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -41,7 +41,7 @@ std::string DataTypeEnum::generateMySQLName(const Values & values) { WriteBufferFromOwnString out; - writeString("enum", out); + writeString("ENUM", out); writeChar('(', out); auto first = true; diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index c6e523adf96..2cdaa2db06c 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -54,7 +54,7 @@ public: std::string doGetName() const override { return type_name; } const char * getFamilyName() const override; - const char * getMySQLName() const override { return my_sql_type_name.c_str(); } + const char * getSQLCompatibleName() const override { return my_sql_type_name.c_str(); } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeFixedString.h b/src/DataTypes/DataTypeFixedString.h index eb09914ec9c..2900efd5a34 100644 --- a/src/DataTypes/DataTypeFixedString.h +++ b/src/DataTypes/DataTypeFixedString.h @@ -42,7 +42,7 @@ public: TypeIndex getTypeId() const override { return type_id; } const char * getFamilyName() const override { return "FixedString"; } - const char * getMySQLName() const override { return "text"; } + const char * getSQLCompatibleName() const override { return "TEXT"; } size_t getN() const { diff --git a/src/DataTypes/DataTypeFunction.h b/src/DataTypes/DataTypeFunction.h index f3423796126..df59f7738b2 100644 --- a/src/DataTypes/DataTypeFunction.h +++ b/src/DataTypes/DataTypeFunction.h @@ -24,7 +24,7 @@ public: std::string doGetName() const override; const char * getFamilyName() const override { return "Function"; } - const char * getMySQLName() const override { return "text"; } + const char * getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Function; } const DataTypes & getArgumentTypes() const diff --git a/src/DataTypes/DataTypeIPv4andIPv6.h b/src/DataTypes/DataTypeIPv4andIPv6.h index 8f7fe79793b..be0ebb90f3c 100644 --- a/src/DataTypes/DataTypeIPv4andIPv6.h +++ b/src/DataTypes/DataTypeIPv4andIPv6.h @@ -19,7 +19,7 @@ public: static constexpr auto type_id = TypeToTypeIndex; const char * getFamilyName() const override { return TypeName.data(); } - const char * getMySQLName() const override { return "text"; } + const char * getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return type_id; } @@ -61,7 +61,7 @@ public: static constexpr auto type_id = TypeToTypeIndex; const char * getFamilyName() const override { return TypeName.data(); } - const char * getMySQLName() const override { return "text"; } + const char * getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeInterval.h b/src/DataTypes/DataTypeInterval.h index 69a56e8aadd..ee2157431dd 100644 --- a/src/DataTypes/DataTypeInterval.h +++ b/src/DataTypes/DataTypeInterval.h @@ -26,7 +26,7 @@ public: std::string doGetName() const override { return fmt::format("Interval{}", kind.toString()); } const char * getFamilyName() const override { return "Interval"; } - const char * getMySQLName() const override { return "text"; } + const char * getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Interval; } bool equals(const IDataType & rhs) const override; diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index b1c32317015..e59613e6974 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -29,7 +29,7 @@ namespace ErrorCodes DataTypeLowCardinality::DataTypeLowCardinality(DataTypePtr dictionary_type_) : dictionary_type(std::move(dictionary_type_)), - mysql_name(dictionary_type->getMySQLName()) + mysql_name(dictionary_type->getSQLCompatibleName()) { auto inner_type = dictionary_type; if (dictionary_type->isNullable()) diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index bcc39f58ff7..4dee8565568 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -24,7 +24,7 @@ public: return "LowCardinality(" + dictionary_type->getName() + ")"; } const char * getFamilyName() const override { return "LowCardinality"; } - const char * getMySQLName() const override { return mysql_name.c_str(); } + const char * getSQLCompatibleName() const override { return mysql_name.c_str(); } TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; } diff --git a/src/DataTypes/DataTypeMap.h b/src/DataTypes/DataTypeMap.h index 526dc321f44..299119f1759 100644 --- a/src/DataTypes/DataTypeMap.h +++ b/src/DataTypes/DataTypeMap.h @@ -30,7 +30,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Map; } std::string doGetName() const override; const char * getFamilyName() const override { return "Map"; } - const char * getMySQLName() const override { return "json"; } + const char * getSQLCompatibleName() const override { return "JSON"; } bool canBeInsideNullable() const override { return false; } diff --git a/src/DataTypes/DataTypeNothing.h b/src/DataTypes/DataTypeNothing.h index fdef6026603..b35ced5dcb3 100644 --- a/src/DataTypes/DataTypeNothing.h +++ b/src/DataTypes/DataTypeNothing.h @@ -16,7 +16,7 @@ public: static constexpr bool is_parametric = false; const char * getFamilyName() const override { return "Nothing"; } - const char * getMySQLName() const override { return "text"; } + const char * getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Nothing; } diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index 64b201d32b2..b5fe1bb2dd9 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -16,7 +16,7 @@ public: explicit DataTypeNullable(const DataTypePtr & nested_data_type_); std::string doGetName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } const char * getFamilyName() const override { return "Nullable"; } - const char * getMySQLName() const override { return nested_data_type->getMySQLName(); } + const char * getSQLCompatibleName() const override { return nested_data_type->getSQLCompatibleName(); } TypeIndex getTypeId() const override { return TypeIndex::Nullable; } MutableColumnPtr createColumn() const override; diff --git a/src/DataTypes/DataTypeNumberBase.cpp b/src/DataTypes/DataTypeNumberBase.cpp index 7d200de7996..db654448e83 100644 --- a/src/DataTypes/DataTypeNumberBase.cpp +++ b/src/DataTypes/DataTypeNumberBase.cpp @@ -32,20 +32,20 @@ bool DataTypeNumberBase::isValueRepresentedByUnsignedInteger() const template const std::map DataTypeNumberBase::mysqlTypeMap = { - {"UInt8", "tinyint unsigned"}, - {"UInt16", "smallint unsigned"}, - {"UInt32", "mediumint unsigned"}, - {"UInt64", "bigint unsigned"}, - {"UInt128", "text"}, - {"UInt256", "text"}, - {"Int8", "tinyint"}, - {"Int16", "smallint"}, - {"Int32", "int"}, - {"Int64", "bigint"}, - {"Int128", "text"}, - {"Int256", "text"}, - {"Float32", "float"}, - {"Float64", "double"}, + {"UInt8", "TINYINT UNSIGNED"}, + {"UInt16", "SMALLINT UNSIGNED"}, + {"UInt32", "MEDIUMINT UNSIGNEd"}, + {"UInt64", "BIGINT UNSIGNED"}, + {"UInt128", "TEXT"}, + {"UInt256", "TEXT"}, + {"Int8", "TINYINT"}, + {"Int16", "SMALLINT"}, + {"Int32", "INT"}, + {"Int64", "BIGINT"}, + {"Int128", "TEXT"}, + {"Int256", "TEXT"}, + {"Float32", "FLOAT"}, + {"Float64", "DOUBLE"}, }; /// Explicit template instantiations - to avoid code bloat in headers. diff --git a/src/DataTypes/DataTypeNumberBase.h b/src/DataTypes/DataTypeNumberBase.h index b5c963cf245..1a855a974f0 100644 --- a/src/DataTypes/DataTypeNumberBase.h +++ b/src/DataTypes/DataTypeNumberBase.h @@ -27,7 +27,7 @@ public: using ColumnType = ColumnVector; const char * getFamilyName() const override { return TypeName.data(); } - const char * getMySQLName() const override { return mysqlTypeMap.at(TypeName.data()).c_str(); } + const char * getSQLCompatibleName() const override { return mysqlTypeMap.at(TypeName.data()).c_str(); } TypeIndex getTypeId() const override { return TypeToTypeIndex; } Field getDefault() const override; diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 8a2c36abcd7..618c7389758 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -23,7 +23,7 @@ public: DataTypeObject(const String & schema_format_, bool is_nullable_); const char * getFamilyName() const override { return "Object"; } - const char * getMySQLName() const override { return "json"; } + const char * getSQLCompatibleName() const override { return "JSON"; } String doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::Object; } diff --git a/src/DataTypes/DataTypeSet.h b/src/DataTypes/DataTypeSet.h index bdad638b5d5..916b4f071a5 100644 --- a/src/DataTypes/DataTypeSet.h +++ b/src/DataTypes/DataTypeSet.h @@ -15,7 +15,7 @@ class DataTypeSet final : public IDataTypeDummy public: static constexpr bool is_parametric = true; const char * getFamilyName() const override { return "Set"; } - const char * getMySQLName() const override { return "text"; } + const char * getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Set; } bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); } diff --git a/src/DataTypes/DataTypeString.h b/src/DataTypes/DataTypeString.h index bddfb4ae287..338b3846266 100644 --- a/src/DataTypes/DataTypeString.h +++ b/src/DataTypes/DataTypeString.h @@ -22,7 +22,7 @@ public: } // FIXME: string can contain arbitrary bytes, not only UTF-8 sequences - const char * getMySQLName() const override { return "blob"; } + const char * getSQLCompatibleName() const override { return "BLOB"; } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index d264cc97f60..93fa87b1332 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -33,7 +33,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Tuple; } std::string doGetName() const override; const char * getFamilyName() const override { return "Tuple"; } - const char * getMySQLName() const override { return "json"; } + const char * getSQLCompatibleName() const override { return "JSON"; } bool canBeInsideNullable() const override { return false; } bool supportsSparseSerialization() const override { return true; } diff --git a/src/DataTypes/DataTypeUUID.h b/src/DataTypes/DataTypeUUID.h index 4d54db42b45..bbf35074df3 100644 --- a/src/DataTypes/DataTypeUUID.h +++ b/src/DataTypes/DataTypeUUID.h @@ -18,7 +18,7 @@ public: static constexpr auto type_id = TypeIndex::UUID; const char * getFamilyName() const override { return "UUID"; } - const char * getMySQLName() const override { return "char"; } + const char * getSQLCompatibleName() const override { return "CHAR"; } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index 5c9405cb060..6f3bf582aeb 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -37,10 +37,10 @@ public: using Base::Base; static constexpr auto family_name = "Decimal"; - static constexpr auto mysql_name = "decimal"; + static constexpr auto mysql_name = "DECIMAL"; const char * getFamilyName() const override { return family_name; } - const char * getMySQLName() const override { return mysql_name; } + const char * getSQLCompatibleName() const override { return mysql_name; } std::string doGetName() const override; TypeIndex getTypeId() const override { return TypeToTypeIndex; } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 2bed18897ce..93fdbab05ef 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -83,7 +83,7 @@ public: /// Name of data type family (example: FixedString, Array). virtual const char * getFamilyName() const = 0; - virtual const char * getMySQLName() const = 0; + virtual const char * getSQLCompatibleName() const = 0; /// Data type id. It's used for runtime type checks. virtual TypeIndex getTypeId() const = 0; @@ -135,7 +135,7 @@ public: protected: virtual String doGetName() const { return getFamilyName(); } - virtual String doGetMySQLName() const { return getMySQLName(); } + virtual String doGetMySQLName() const { return getSQLCompatibleName(); } virtual SerializationPtr doGetDefaultSerialization() const = 0; public: diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference index 96e542611c6..1742cd9c90c 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference @@ -4,226 +4,226 @@ Create pseudo-random database name Create tab duplicate table Run MySQL test field type null key default extra -aggregate_function text 0 NULL -array_value text 0 NULL -boolean_value tinyint unsigned 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value datetime 0 NULL -datetime_value datetime 0 NULL -decimal_value decimal 0 NULL -enum_value enum('apple', 'banana', 'orange') 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -int32 int 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value json 0 NULL -low_cardinality blob 0 NULL -low_cardinality_date datetime 0 NULL -map_value json 0 NULL -nested.nested_int text 0 NULL -nested.nested_string text 0 NULL -nullable_value int 0 NULL -string_value blob 0 NULL -tuple_value json 0 NULL -uint64 bigint unsigned 0 PRI SOR NULL -uuid_value char 0 NULL +aggregate_function TEXT 0 NULL +array_value TEXT 0 NULL +boolean_value TINYINT UNSIGNED 0 NULL +date32_value DATE 0 NULL +date_value DATE 0 NULL +datetime64_value DATETIME 0 NULL +datetime_value DATETIME 0 NULL +decimal_value DECIMAL 0 NULL +enum_value ENUM('apple', 'banana', 'orange') 0 NULL +fixed_string_value TEXT 0 NULL +float32 FLOAT 0 NULL +float64 DOUBLE 0 NULL +int32 INT 0 NULL +ipv4_value TEXT 0 NULL +ipv6_value TEXT 0 NULL +json_value JSON 0 NULL +low_cardinality BLOB 0 NULL +low_cardinality_date DATETIME 0 NULL +map_value JSON 0 NULL +nested.nested_int TEXT 0 NULL +nested.nested_string TEXT 0 NULL +nullable_value INT 0 NULL +string_value BLOB 0 NULL +tuple_value JSON 0 NULL +uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uuid_value CHAR 0 NULL field type null key default extra -aggregate_function text 0 NULL -array_value text 0 NULL -boolean_value tinyint unsigned 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value datetime 0 NULL -datetime_value datetime 0 NULL -decimal_value decimal 0 NULL -enum_value enum('apple', 'banana', 'orange') 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -int32 int 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value json 0 NULL -low_cardinality blob 0 NULL -low_cardinality_date datetime 0 NULL -map_value json 0 NULL -nested.nested_int text 0 NULL -nested.nested_string text 0 NULL -nullable_value int 0 NULL -string_value blob 0 NULL -tuple_value json 0 NULL -uint64 bigint unsigned 0 PRI SOR NULL -uuid_value char 0 NULL +aggregate_function TEXT 0 NULL +array_value TEXT 0 NULL +boolean_value TINYINT UNSIGNED 0 NULL +date32_value DATE 0 NULL +date_value DATE 0 NULL +datetime64_value DATETIME 0 NULL +datetime_value DATETIME 0 NULL +decimal_value DECIMAL 0 NULL +enum_value ENUM('apple', 'banana', 'orange') 0 NULL +fixed_string_value TEXT 0 NULL +float32 FLOAT 0 NULL +float64 DOUBLE 0 NULL +int32 INT 0 NULL +ipv4_value TEXT 0 NULL +ipv6_value TEXT 0 NULL +json_value JSON 0 NULL +low_cardinality BLOB 0 NULL +low_cardinality_date DATETIME 0 NULL +map_value JSON 0 NULL +nested.nested_int TEXT 0 NULL +nested.nested_string TEXT 0 NULL +nullable_value INT 0 NULL +string_value BLOB 0 NULL +tuple_value JSON 0 NULL +uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uuid_value CHAR 0 NULL field type null key default extra collation comment privileges -aggregate_function text 0 NULL NULL -array_value text 0 NULL NULL -boolean_value tinyint unsigned 0 NULL NULL -date32_value date 0 NULL NULL -date_value date 0 NULL NULL -datetime64_value datetime 0 NULL NULL -datetime_value datetime 0 NULL NULL -decimal_value decimal 0 NULL NULL -enum_value enum('apple', 'banana', 'orange') 0 NULL NULL -fixed_string_value text 0 NULL NULL -float32 float 0 NULL NULL -float64 double 0 NULL NULL -int32 int 0 NULL NULL -ipv4_value text 0 NULL NULL -ipv6_value text 0 NULL NULL -json_value json 0 NULL NULL -low_cardinality blob 0 NULL NULL -low_cardinality_date datetime 0 NULL NULL -map_value json 0 NULL NULL -nested.nested_int text 0 NULL NULL -nested.nested_string text 0 NULL NULL -nullable_value int 0 NULL NULL -string_value blob 0 NULL NULL -tuple_value json 0 NULL NULL -uint64 bigint unsigned 0 PRI SOR NULL NULL -uuid_value char 0 NULL NULL +aggregate_function TEXT 0 NULL NULL +array_value TEXT 0 NULL NULL +boolean_value TINYINT UNSIGNED 0 NULL NULL +date32_value DATE 0 NULL NULL +date_value DATE 0 NULL NULL +datetime64_value DATETIME 0 NULL NULL +datetime_value DATETIME 0 NULL NULL +decimal_value DECIMAL 0 NULL NULL +enum_value ENUM('apple', 'banana', 'orange') 0 NULL NULL +fixed_string_value TEXT 0 NULL NULL +float32 FLOAT 0 NULL NULL +float64 DOUBLE 0 NULL NULL +int32 INT 0 NULL NULL +ipv4_value TEXT 0 NULL NULL +ipv6_value TEXT 0 NULL NULL +json_value JSON 0 NULL NULL +low_cardinality BLOB 0 NULL NULL +low_cardinality_date DATETIME 0 NULL NULL +map_value JSON 0 NULL NULL +nested.nested_int TEXT 0 NULL NULL +nested.nested_string TEXT 0 NULL NULL +nullable_value INT 0 NULL NULL +string_value BLOB 0 NULL NULL +tuple_value JSON 0 NULL NULL +uint64 BIGINT UNSIGNED 0 PRI SOR NULL NULL +uuid_value CHAR 0 NULL NULL field type null key default extra -int32 int 0 NULL -nested.nested_int text 0 NULL -uint64 bigint unsigned 0 PRI SOR NULL +int32 INT 0 NULL +nested.nested_int TEXT 0 NULL +uint64 BIGINT UNSIGNED 0 PRI SOR NULL field type null key default extra -aggregate_function text 0 NULL -array_value text 0 NULL -boolean_value tinyint unsigned 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value datetime 0 NULL -datetime_value datetime 0 NULL -decimal_value decimal 0 NULL -enum_value enum('apple', 'banana', 'orange') 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value json 0 NULL -low_cardinality blob 0 NULL -low_cardinality_date datetime 0 NULL -map_value json 0 NULL -nested.nested_string text 0 NULL -nullable_value int 0 NULL -string_value blob 0 NULL -tuple_value json 0 NULL -uuid_value char 0 NULL +aggregate_function TEXT 0 NULL +array_value TEXT 0 NULL +boolean_value TINYINT UNSIGNED 0 NULL +date32_value DATE 0 NULL +date_value DATE 0 NULL +datetime64_value DATETIME 0 NULL +datetime_value DATETIME 0 NULL +decimal_value DECIMAL 0 NULL +enum_value ENUM('apple', 'banana', 'orange') 0 NULL +fixed_string_value TEXT 0 NULL +float32 FLOAT 0 NULL +float64 DOUBLE 0 NULL +ipv4_value TEXT 0 NULL +ipv6_value TEXT 0 NULL +json_value JSON 0 NULL +low_cardinality BLOB 0 NULL +low_cardinality_date DATETIME 0 NULL +map_value JSON 0 NULL +nested.nested_string TEXT 0 NULL +nullable_value INT 0 NULL +string_value BLOB 0 NULL +tuple_value JSON 0 NULL +uuid_value CHAR 0 NULL field type null key default extra -int32 int 0 NULL -nested.nested_int text 0 NULL -uint64 bigint unsigned 0 PRI SOR NULL +int32 INT 0 NULL +nested.nested_int TEXT 0 NULL +uint64 BIGINT UNSIGNED 0 PRI SOR NULL field type null key default extra -aggregate_function text 0 NULL -array_value text 0 NULL -boolean_value tinyint unsigned 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value datetime 0 NULL -datetime_value datetime 0 NULL -decimal_value decimal 0 NULL -enum_value enum('apple', 'banana', 'orange') 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value json 0 NULL -low_cardinality blob 0 NULL -low_cardinality_date datetime 0 NULL -map_value json 0 NULL -nested.nested_string text 0 NULL -nullable_value int 0 NULL -string_value blob 0 NULL -tuple_value json 0 NULL -uuid_value char 0 NULL +aggregate_function TEXT 0 NULL +array_value TEXT 0 NULL +boolean_value TINYINT UNSIGNED 0 NULL +date32_value DATE 0 NULL +date_value DATE 0 NULL +datetime64_value DATETIME 0 NULL +datetime_value DATETIME 0 NULL +decimal_value DECIMAL 0 NULL +enum_value ENUM('apple', 'banana', 'orange') 0 NULL +fixed_string_value TEXT 0 NULL +float32 FLOAT 0 NULL +float64 DOUBLE 0 NULL +ipv4_value TEXT 0 NULL +ipv6_value TEXT 0 NULL +json_value JSON 0 NULL +low_cardinality BLOB 0 NULL +low_cardinality_date DATETIME 0 NULL +map_value JSON 0 NULL +nested.nested_string TEXT 0 NULL +nullable_value INT 0 NULL +string_value BLOB 0 NULL +tuple_value JSON 0 NULL +uuid_value CHAR 0 NULL field type null key default extra -int32 int 0 NULL -nested.nested_int text 0 NULL -uint64 bigint unsigned 0 PRI SOR NULL +int32 INT 0 NULL +nested.nested_int TEXT 0 NULL +uint64 BIGINT UNSIGNED 0 PRI SOR NULL field type null key default extra -aggregate_function text 0 NULL +aggregate_function TEXT 0 NULL field type null key default extra -aggregate_function text 0 NULL -array_value text 0 NULL -boolean_value tinyint unsigned 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value datetime 0 NULL -datetime_value datetime 0 NULL -decimal_value decimal 0 NULL -enum_value enum('apple', 'banana', 'orange') 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -int32 int 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value json 0 NULL -low_cardinality blob 0 NULL -low_cardinality_date datetime 0 NULL -map_value json 0 NULL -nested.nested_int text 0 NULL -nested.nested_string text 0 NULL -nullable_value int 0 NULL -string_value blob 0 NULL -tuple_value json 0 NULL -uint64 bigint unsigned 0 PRI SOR NULL -uuid_value char 0 NULL +aggregate_function TEXT 0 NULL +array_value TEXT 0 NULL +boolean_value TINYINT UNSIGNED 0 NULL +date32_value DATE 0 NULL +date_value DATE 0 NULL +datetime64_value DATETIME 0 NULL +datetime_value DATETIME 0 NULL +decimal_value DECIMAL 0 NULL +enum_value ENUM('apple', 'banana', 'orange') 0 NULL +fixed_string_value TEXT 0 NULL +float32 FLOAT 0 NULL +float64 DOUBLE 0 NULL +int32 INT 0 NULL +ipv4_value TEXT 0 NULL +ipv6_value TEXT 0 NULL +json_value JSON 0 NULL +low_cardinality BLOB 0 NULL +low_cardinality_date DATETIME 0 NULL +map_value JSON 0 NULL +nested.nested_int TEXT 0 NULL +nested.nested_string TEXT 0 NULL +nullable_value INT 0 NULL +string_value BLOB 0 NULL +tuple_value JSON 0 NULL +uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uuid_value CHAR 0 NULL field type null key default extra -aggregate_function text 0 NULL -array_value text 0 NULL -boolean_value tinyint unsigned 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value datetime 0 NULL -datetime_value datetime 0 NULL -decimal_value decimal 0 NULL -enum_value enum('apple', 'banana', 'orange') 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -int32 int 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value json 0 NULL -low_cardinality blob 0 NULL -low_cardinality_date datetime 0 NULL -map_value json 0 NULL -nested.nested_int text 0 NULL -nested.nested_string text 0 NULL -nullable_value int 0 NULL -string_value blob 0 NULL -tuple_value json 0 NULL -uint64 bigint unsigned 0 PRI SOR NULL -uuid_value char 0 NULL +aggregate_function TEXT 0 NULL +array_value TEXT 0 NULL +boolean_value TINYINT UNSIGNED 0 NULL +date32_value DATE 0 NULL +date_value DATE 0 NULL +datetime64_value DATETIME 0 NULL +datetime_value DATETIME 0 NULL +decimal_value DECIMAL 0 NULL +enum_value ENUM('apple', 'banana', 'orange') 0 NULL +fixed_string_value TEXT 0 NULL +float32 FLOAT 0 NULL +float64 DOUBLE 0 NULL +int32 INT 0 NULL +ipv4_value TEXT 0 NULL +ipv6_value TEXT 0 NULL +json_value JSON 0 NULL +low_cardinality BLOB 0 NULL +low_cardinality_date DATETIME 0 NULL +map_value JSON 0 NULL +nested.nested_int TEXT 0 NULL +nested.nested_string TEXT 0 NULL +nullable_value INT 0 NULL +string_value BLOB 0 NULL +tuple_value JSON 0 NULL +uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uuid_value CHAR 0 NULL field type null key default extra -aggregate_function text 0 NULL -array_value text 0 NULL -boolean_value tinyint unsigned 0 NULL -date32_value date 0 NULL -date_value date 0 NULL -datetime64_value datetime 0 NULL -datetime_value datetime 0 NULL -decimal_value decimal 0 NULL -enum_value enum('apple', 'banana', 'orange') 0 NULL -fixed_string_value text 0 NULL -float32 float 0 NULL -float64 double 0 NULL -int32 int 0 NULL -ipv4_value text 0 NULL -ipv6_value text 0 NULL -json_value json 0 NULL -low_cardinality blob 0 NULL -low_cardinality_date datetime 0 NULL -map_value json 0 NULL -nested.nested_int text 0 NULL -nested.nested_string text 0 NULL -nullable_value int 0 NULL -string_value blob 0 NULL -tuple_value json 0 NULL -uint64 bigint unsigned 0 PRI SOR NULL -uuid_value char 0 NULL +aggregate_function TEXT 0 NULL +array_value TEXT 0 NULL +boolean_value TINYINT UNSIGNED 0 NULL +date32_value DATE 0 NULL +date_value DATE 0 NULL +datetime64_value DATETIME 0 NULL +datetime_value DATETIME 0 NULL +decimal_value DECIMAL 0 NULL +enum_value ENUM('apple', 'banana', 'orange') 0 NULL +fixed_string_value TEXT 0 NULL +float32 FLOAT 0 NULL +float64 DOUBLE 0 NULL +int32 INT 0 NULL +ipv4_value TEXT 0 NULL +ipv6_value TEXT 0 NULL +json_value JSON 0 NULL +low_cardinality BLOB 0 NULL +low_cardinality_date DATETIME 0 NULL +map_value JSON 0 NULL +nested.nested_int TEXT 0 NULL +nested.nested_string TEXT 0 NULL +nullable_value INT 0 NULL +string_value BLOB 0 NULL +tuple_value JSON 0 NULL +uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uuid_value CHAR 0 NULL diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh index a446c6e817e..fd1ad92f060 100755 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh @@ -1,4 +1,8 @@ -#!/bin/bash +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh # This script tests the MySQL compatibility of the SHOW COLUMNS command in ClickHouse USER="default" From 09b5b0c3f7a1265e5b21f2a818ec05c9afdc48e4 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 13:31:02 -0400 Subject: [PATCH 1051/2223] add word --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2feb7981fcc..0455556ae96 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1453,6 +1453,7 @@ gRPC gccMurmurHash gcem generateRandom +generateRandomStructure generateULID generateUUIDv geoDistance From 85ded501d798a067fd9d3b1bdd0e2d6d8cbcc14b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 2 Jun 2023 20:02:14 +0200 Subject: [PATCH 1052/2223] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2feb7981fcc..0455556ae96 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1453,6 +1453,7 @@ gRPC gccMurmurHash gcem generateRandom +generateRandomStructure generateULID generateUUIDv geoDistance From a3ed86a52879367308f4425bef5617f98486a1bb Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Jun 2023 14:36:44 -0400 Subject: [PATCH 1053/2223] Documentation --- .../utilities/clickhouse-keeper-client.md | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) create mode 100644 docs/en/operations/utilities/clickhouse-keeper-client.md diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md new file mode 100644 index 00000000000..77f816fe428 --- /dev/null +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -0,0 +1,53 @@ +--- +slug: /en/operations/utilities/clickhouse-keeper-client +sidebar_label: clickhouse-keeper-client +--- + +# clickhouse-keeper-client + +A client application to interact with clickhouse-keeper by its native protocol. + +## Keys {#clickhouse-keeper-client} + +- `-q QUERY`, `--query=QUERY` — Query to execute. If this parameter is not passed, `clickhouse-keeper-client` will start in interactive mode. +- `-h HOST`, `--host=HOST` — Server host. Default value: `localhost`. +- `-p N`, `--port=N` — Server port. Default value: 2181 +- `--connection-timeout=TIMEOUT` — Set connection timeout in seconds. Default value: 10s. +- `--session-timeout=TIMEOUT` — Set session timeout in seconds. Default value: 10s. +- `--operation-timeout=TIMEOUT` — Set operation timeout in seconds. Default value: 10s. +- `--history-file=FILE_PATH` — Set path of history file. Default value: `~/.keeper-client-history`. +- `--help` — Shows the help message. + +## Example {#clickhouse-keeper-client-example} + +```bash +./clickhouse-keeper-client -h localhost:2181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30 +Connected to ZooKeeper at [::1]:2181 with session_id 137 +/ :) ls +keeper foo bar +/ :) cd keeper +/keeper :) ls +api_version +/keeper :) cd api_version +/keeper/api_version :) ls + +/keeper/api_version :) cd xyz +Path /keeper/api_version/xyz does not exists +/keeper/api_version :) cd ../../ +/ :) ls +keeper foo bar +/ :) get keeper/api_version +2 +``` + +## Commands {#clickhouse-keeper-client-commands} + +- `ls [path]` -- Lists the nodes for the given path (default: cwd) +- `cd [path]` -- Change the working path (default `.`) +- `set [version]` -- Updates the node's value. Only update if version matches (default: -1) +- `create ` -- Creates new node +- `get ` -- Returns the node's value +- `remove ` -- Remove the node +- `rmr ` -- Recursively deletes path. Confirmation required +- `flwc ` -- Executes four-letter-word command +- `help` -- Prints this message From 991d1b97fc1b0959d1cb1659ee46a893b693716f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 2 Jun 2023 20:48:31 +0200 Subject: [PATCH 1054/2223] less traces in logs --- src/IO/WriteBufferFromS3.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 462cf2674c3..210cea02a36 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -245,10 +245,8 @@ WriteBufferFromS3::~WriteBufferFromS3() LOG_INFO(log, "WriteBufferFromS3 is not finalized in destructor. " "It could be if an exception occurs. File is not written to S3. " - "{}. " - "Stack trace: {}", - getLogDetails(), - StackTrace().toString()); + "{}.", + getLogDetails()); } task_tracker->safeWaitAll(); From 50654435dc1cf6ac826d08d28adf2e669250d5ec Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 2 Jun 2023 19:36:37 +0000 Subject: [PATCH 1055/2223] Implement endianness-independent serialization for UUID --- .../Serializations/SerializationUUID.cpp | 16 ++--- src/IO/ReadHelpers.cpp | 72 +++++++++---------- src/IO/ReadHelpers.h | 15 ++-- src/IO/WriteHelpers.cpp | 38 +++++++--- src/IO/WriteHelpers.h | 7 +- .../Formats/Impl/AvroRowInputFormat.cpp | 3 +- .../Formats/Impl/AvroRowOutputFormat.cpp | 5 +- 7 files changed, 75 insertions(+), 81 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationUUID.cpp b/src/DataTypes/Serializations/SerializationUUID.cpp index ee1327ef094..13313111b2b 100644 --- a/src/DataTypes/Serializations/SerializationUUID.cpp +++ b/src/DataTypes/Serializations/SerializationUUID.cpp @@ -51,19 +51,11 @@ void SerializationUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & ist { assertChar('\'', istr); char * next_pos = find_first_symbols<'\\', '\''>(istr.position(), istr.buffer().end()); - size_t len = next_pos - istr.position(); - if ((len == 32) && (istr.position()[32] == '\'')) + const auto len = next_pos - istr.position(); + if ((len == 32 || len == 36) && istr.position()[len] == '\'') { - parseUUIDWithoutSeparator( - reinterpret_cast(istr.position()), std::reverse_iterator(reinterpret_cast(&uuid) + 16)); - istr.ignore(33); - fast = true; - } - else if ((len == 36) && (istr.position()[36] == '\'')) - { - parseUUID( - reinterpret_cast(istr.position()), std::reverse_iterator(reinterpret_cast(&uuid) + 16)); - istr.ignore(37); + uuid = parseUUID(std::span(reinterpret_cast(istr.position()), len)); + istr.ignore(len + 1); fast = true; } else diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 99d25ee6613..a85a057f2b3 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -46,48 +46,40 @@ inline void parseHex(IteratorSrc src, IteratorDst dst) dst[dst_pos] = unhex2(reinterpret_cast(&src[src_pos])); } -void parseUUID(const UInt8 * src36, UInt8 * dst16) +UUID parseUUID(std::span src) { - /// If string is not like UUID - implementation specific behaviour. + UUID uuid; + const auto * src_ptr = src.data(); + auto * dst = reinterpret_cast(&uuid); + if (const auto size = src.size(); size == 36) + { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + parseHex<4>(src_ptr, dst); + parseHex<2>(src_ptr + 9, dst + 4); + parseHex<2>(src_ptr + 14, dst + 6); + parseHex<2>(src_ptr + 19, dst + 8); + parseHex<6>(src_ptr + 24, dst + 10); +#else + const std::reverse_iterator dst_it(dst + sizeof(UUID)); + /// FIXME This code looks like trash. + parseHex<4>(src_ptr, dst + 8); + parseHex<2>(src_ptr + 9, dst + 12); + parseHex<2>(src_ptr + 14, dst + 14); + parseHex<2>(src_ptr + 19, dst); + parseHex<6>(src_ptr + 24, dst + 2); +#endif + } + else if (size == 32) + { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + parseHex<16>(src_ptr, dst); +#else + parseHex<8>(src_ptr, dst + 8); + parseHex<8>(src_ptr + 16, dst); +#endif + } - parseHex<4>(&src36[0], &dst16[0]); - parseHex<2>(&src36[9], &dst16[4]); - parseHex<2>(&src36[14], &dst16[6]); - parseHex<2>(&src36[19], &dst16[8]); - parseHex<6>(&src36[24], &dst16[10]); -} - -void parseUUIDWithoutSeparator(const UInt8 * src36, UInt8 * dst16) -{ - /// If string is not like UUID - implementation specific behaviour. - - parseHex<16>(&src36[0], &dst16[0]); -} - -/** Function used when byte ordering is important when parsing uuid - * ex: When we create an UUID type - */ -void parseUUID(const UInt8 * src36, std::reverse_iterator dst16) -{ - /// If string is not like UUID - implementation specific behaviour. - - /// FIXME This code looks like trash. - parseHex<4>(&src36[0], dst16 + 8); - parseHex<2>(&src36[9], dst16 + 12); - parseHex<2>(&src36[14], dst16 + 14); - parseHex<2>(&src36[19], dst16); - parseHex<6>(&src36[24], dst16 + 2); -} - -/** Function used when byte ordering is important when parsing uuid - * ex: When we create an UUID type - */ -void parseUUIDWithoutSeparator(const UInt8 * src36, std::reverse_iterator dst16) -{ - /// If string is not like UUID - implementation specific behaviour. - - parseHex<8>(&src36[0], dst16 + 8); - parseHex<8>(&src36[16], dst16); + return uuid; } void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 32338552b66..7e293944d19 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -623,12 +624,6 @@ struct NullOutput void push_back(char) {} /// NOLINT }; -void parseUUID(const UInt8 * src36, UInt8 * dst16); -void parseUUIDWithoutSeparator(const UInt8 * src36, UInt8 * dst16); -void parseUUID(const UInt8 * src36, std::reverse_iterator dst16); -void parseUUIDWithoutSeparator(const UInt8 * src36, std::reverse_iterator dst16); - - template ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf); @@ -770,6 +765,9 @@ inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf) return readDateTextImpl(date, buf); } +/// If string is not like UUID - implementation specific behaviour. +UUID parseUUID(std::span src); + template inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf) { @@ -797,12 +795,9 @@ inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf) return ReturnType(false); } } - - parseUUID(reinterpret_cast(s), std::reverse_iterator(reinterpret_cast(&uuid) + 16)); } - else - parseUUIDWithoutSeparator(reinterpret_cast(s), std::reverse_iterator(reinterpret_cast(&uuid) + 16)); + uuid = parseUUID({reinterpret_cast(s), size}); return ReturnType(true); } else diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index a0eceddc6f6..6023d4c9d5b 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -23,17 +23,35 @@ void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes) /** Function used when byte ordering is important when parsing uuid * ex: When we create an UUID type */ -void formatUUID(std::reverse_iterator src16, UInt8 * dst36) +std::array formatUUID(const UUID & uuid) { - formatHex(src16 + 8, &dst36[0], 4); - dst36[8] = '-'; - formatHex(src16 + 12, &dst36[9], 2); - dst36[13] = '-'; - formatHex(src16 + 14, &dst36[14], 2); - dst36[18] = '-'; - formatHex(src16, &dst36[19], 2); - dst36[23] = '-'; - formatHex(src16 + 2, &dst36[24], 6); + std::array dst; + const auto * src_ptr = reinterpret_cast(&uuid); + auto * dst_ptr = dst.data(); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + formatHex(src_ptr, dst_ptr, 4); + dst[8] = '-'; + formatHex(src_ptr + 4, dst_ptr + 9, 2); + dst[13] = '-'; + formatHex(src_ptr + 6, dst_ptr + 14, 2); + dst[18] = '-'; + formatHex(src_ptr + 8, dst_ptr + 19, 2); + dst[23] = '-'; + formatHex(src_ptr + 10, dst_ptr + 24, 6); +#else + const std::reverse_iterator src_it(src_ptr + 16); + formatHex(src_it + 8, dst_ptr, 4); + dst[8] = '-'; + formatHex(src_it + 12, dst_ptr + 9, 2); + dst[13] = '-'; + formatHex(src_it + 14, dst_ptr + 14, 2); + dst[18] = '-'; + formatHex(src_it, dst_ptr + 19, 2); + dst[23] = '-'; + formatHex(src_it + 2, dst_ptr + 24, 6); +#endif + + return dst; } void writeIPv4Text(const IPv4 & ip, WriteBuffer & buf) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index cdbc952690c..923684c4249 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -625,13 +625,12 @@ inline void writeXMLStringForTextElement(std::string_view s, WriteBuffer & buf) writeXMLStringForTextElement(s.data(), s.data() + s.size(), buf); } -void formatUUID(std::reverse_iterator src16, UInt8 * dst36); +std::array formatUUID(const UUID & uuid); inline void writeUUIDText(const UUID & uuid, WriteBuffer & buf) { - char s[36]; - formatUUID(std::reverse_iterator(reinterpret_cast(&uuid) + 16), reinterpret_cast(s)); - buf.write(s, sizeof(s)); + const auto text = formatUUID(uuid); + buf.write(text.data(), text.size()); } void writeIPv4Text(const IPv4 & ip, WriteBuffer & buf); diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index c2602a4d1d5..974b198a483 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -256,8 +256,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro if (tmp.length() != 36) throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", tmp); - UUID uuid; - parseUUID(reinterpret_cast(tmp.data()), std::reverse_iterator(reinterpret_cast(&uuid) + 16)); + const auto uuid = parseUUID({reinterpret_cast(tmp.data()), tmp.length()}); assert_cast(column).insertValue(uuid); return true; }; diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index c743b2c1766..2b163164d56 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -329,9 +329,8 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF return {schema, [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { const auto & uuid = assert_cast(column).getElement(row_num); - std::array s; - formatUUID(std::reverse_iterator(reinterpret_cast(&uuid) + 16), s.data()); - encoder.encodeBytes(reinterpret_cast(s.data()), s.size()); + const auto text = formatUUID(uuid); + encoder.encodeBytes(reinterpret_cast(text.data()), text.size()); }}; } case TypeIndex::Array: From 87eaaa0f7bf43a7145c24e726af8b3b912f38eea Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 16:30:18 -0400 Subject: [PATCH 1056/2223] address review comments --- .../table-engines/integrations/embedded-rocksdb.md | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index dab741a9f63..6664b6a4613 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -127,15 +127,17 @@ A special `direct` join with EmbeddedRocksDB tables is supported. This direct join avoids forming a hash table in memory and accesses the data directly from the EmbeddedRocksDB. +With large joins you may see much lower memory usage with direct joins +because the hash table is not created. + To enable direct joins: ```sql -SET join_algorithm = 'direct' +SET join_algorithm = 'direct, hash' ``` :::tip -When the `join_algorithm` is set to `direct`, direct joins will be used -when possible. However, direct joins are not used for RIGHT or FULL JOINs. -ClickHouse will choose another join algorithm when direct joins are not possible. +When the `join_algorithm` is set to `direct, hash`, direct joins will be used +when possible, and hash otherwise. ::: #### Example @@ -205,3 +207,6 @@ ORDER BY key ASC └─────┴─────────┴────────┴────────┘ ``` +### More information on Joins +- [`join_algorithm` setting](/docs/en/operations/settings/settings.md#settings-join_algorithm) +- [JOIN clause](/docs/en/sql-reference/statements/select/join.md) From 4506299d73a3fbf8fc9446b3eed05fe4d5553c23 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 30 May 2023 20:53:45 +0200 Subject: [PATCH 1057/2223] impl --- docker/test/performance-comparison/report.py | 4 +++- tests/ci/performance_comparison_check.py | 8 +++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 214f2d550b4..a1f2eb9d9ec 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -626,7 +626,9 @@ if args.report == "main": message_array.append(str(faster_queries) + " faster") if slower_queries: - if slower_queries > 3: + # This threshold should be synchronized with the value in https://github.com/ClickHouse/ClickHouse/blob/master/tests/ci/performance_comparison_check.py#L225 + # False positives rate should be < 1%: https://shorturl.at/CDEK8 + if slower_queries > 5: status = "failure" message_array.append(str(slower_queries) + " slower") diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index bf5704f31bd..1baf547816f 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -219,6 +219,12 @@ if __name__ == "__main__": except Exception: traceback.print_exc() + def too_many_slow(msg): + match = re.search("(|.* )(\d+) slower.*", msg) + # This threshold should be synchronized with the value in https://github.com/ClickHouse/ClickHouse/blob/master/docker/test/performance-comparison/report.py#L629 + threshold = 5 + return int(match.group(2).strip()) > threshold if match else False + # Try to fetch status from the report. status = "" message = "" @@ -236,7 +242,7 @@ if __name__ == "__main__": # TODO: Remove me, always green mode for the first time, unless errors status = "success" - if "errors" in message.lower(): + if "errors" in message.lower() or too_many_slow(message.lower()): status = "failure" # TODO: Remove until here except Exception: From b091d85bb1b38bd2bcccf8ed1c1588a56e06b7a7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Jun 2023 23:30:47 +0200 Subject: [PATCH 1058/2223] Remove strange object storage methods --- .../AzureBlobStorage/AzureObjectStorage.cpp | 70 ++++++++++++------- .../AzureBlobStorage/AzureObjectStorage.h | 4 +- .../Cached/CachedObjectStorage.cpp | 4 +- .../Cached/CachedObjectStorage.h | 2 +- .../DiskObjectStorageMetadata.cpp | 8 +-- .../DiskObjectStorageMetadata.h | 4 +- ...jectStorageRemoteMetadataRestoreHelper.cpp | 12 ++-- src/Disks/ObjectStorages/IObjectStorage.cpp | 26 +++++-- src/Disks/ObjectStorages/IObjectStorage.h | 68 +++++++----------- .../MetadataStorageFromDisk.cpp | 4 +- .../MetadataStorageFromPlainObjectStorage.cpp | 52 +++++++------- .../ObjectStorages/S3/S3ObjectStorage.cpp | 68 +++++------------- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 7 +- 13 files changed, 156 insertions(+), 173 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index c4dd0161c70..0044f465081 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -67,6 +67,49 @@ bool AzureObjectStorage::exists(const StoredObject & object) const return false; } +void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const +{ + auto client_ptr = client.get(); + + /// What a shame, no Exists method... + Azure::Storage::Blobs::ListBlobsOptions options; + options.Prefix = path; + if (max_keys) + options.PageSizeHint = max_keys; + else + options.PageSizeHint = settings.get()->list_object_keys_size; + Azure::Storage::Blobs::ListBlobsPagedResponse blob_list_response; + + while (true) + { + blob_list_response = client_ptr->ListBlobs(options); + auto blobs_list = blob_list_response.Blobs; + + for (const auto & blob : blobs_list) + { + children.emplace_back( + blob.Name, + ObjectMetadata{ + static_cast(blob.BlobSize), + blob.Details.LastModified.time_since_epoch().count(), + {}}); + } + + if (max_keys) + { + int keys_left = max_keys - static_cast(children.size()); + if (keys_left <= 0) + break; + options.PageSizeHint = keys_left; + } + + if (blob_list_response.HasPage()) + options.ContinuationToken = blob_list_response.NextPageToken; + else + break; + } +} + std::unique_ptr AzureObjectStorage::readObject( /// NOLINT const StoredObject & object, const ReadSettings & read_settings, @@ -146,33 +189,6 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO patchSettings(write_settings)); } -void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const -{ - auto client_ptr = client.get(); - - Azure::Storage::Blobs::ListBlobsOptions blobs_list_options; - blobs_list_options.Prefix = path; - if (max_keys) - blobs_list_options.PageSizeHint = max_keys; - else - blobs_list_options.PageSizeHint = settings.get()->list_object_keys_size; - - auto blobs_list_response = client_ptr->ListBlobs(blobs_list_options); - for (;;) - { - auto blobs_list = blobs_list_response.Blobs; - - for (const auto & blob : blobs_list) - children.emplace_back(blob.Name, blob.BlobSize); - - if (max_keys && children.size() >= static_cast(max_keys)) - break; - if (!blobs_list_response.HasPage()) - break; - blobs_list_response.MoveToNextPage(); - } -} - /// Remove file. Throws exception if file doesn't exists or it's a directory. void AzureObjectStorage::removeObject(const StoredObject & object) { diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 0c2aecd5c62..a36a03bcda4 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -58,6 +58,8 @@ public: AzureClientPtr && client_, SettingsPtr && settings_); + void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } std::string getName() const override { return "AzureObjectStorage"; } @@ -84,8 +86,6 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; - /// Remove file. Throws exception if file doesn't exists or it's a directory. void removeObject(const StoredObject & object) override; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index acf9430e85c..1d24d9d5411 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -201,9 +201,9 @@ std::unique_ptr CachedObjectStorage::cloneObjectStorage( return object_storage->cloneObjectStorage(new_namespace, config, config_prefix, context); } -void CachedObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const +void CachedObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { - object_storage->findAllFiles(path, children, max_keys); + object_storage->listObjects(path, children, max_keys); } ObjectMetadata CachedObjectStorage::getObjectMetadata(const std::string & path) const diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index f8e346e1aed..b5186d39c32 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -71,7 +71,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; ObjectMetadata getObjectMetadata(const std::string & path) const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index c3284b635da..19d5a8e3567 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -54,7 +54,7 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) assertChar('\n', buf); storage_objects[i].relative_path = object_relative_path; - storage_objects[i].bytes_size = object_size; + storage_objects[i].metadata.size_bytes = object_size; } readIntText(ref_count, buf); @@ -93,9 +93,9 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const writeIntText(total_size, buf); writeChar('\n', buf); - for (const auto & [object_relative_path, object_size] : storage_objects) + for (const auto & [object_relative_path, object_metadata] : storage_objects) { - writeIntText(object_size, buf); + writeIntText(object_metadata.size_bytes, buf); writeChar('\t', buf); writeEscapedString(object_relative_path, buf); writeChar('\n', buf); @@ -139,7 +139,7 @@ DiskObjectStorageMetadata::DiskObjectStorageMetadata( void DiskObjectStorageMetadata::addObject(const String & path, size_t size) { total_size += size; - storage_objects.emplace_back(path, size); + storage_objects.emplace_back(path, ObjectMetadata{size, {}, {}}); } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index a2d0653e4aa..6dced85d0b1 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -21,7 +21,7 @@ private: const std::string & common_metadata_path; /// Relative paths of blobs. - RelativePathsWithSize storage_objects; + RelativePathsWithMetadata storage_objects; const std::string object_storage_root_path; @@ -63,7 +63,7 @@ public: return object_storage_root_path; } - RelativePathsWithSize getBlobsRelativePaths() const + RelativePathsWithMetadata getBlobsRelativePaths() const { return storage_objects; } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index 4cca89b9a4f..74d1698bf01 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -356,7 +356,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * LOG_INFO(disk->log, "Starting restore files for disk {}", disk->name); std::vector> results; - auto restore_files = [this, &source_object_storage, &restore_information, &results](const RelativePathsWithSize & objects) + auto restore_files = [this, &source_object_storage, &restore_information, &results](const RelativePathsWithMetadata & objects) { std::vector keys_names; for (const auto & object : objects) @@ -389,8 +389,8 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * return true; }; - RelativePathsWithSize children; - source_object_storage->findAllFiles(restore_information.source_path, children, /* max_keys= */ 0); + RelativePathsWithMetadata children; + source_object_storage->listObjects(restore_information.source_path, children, /* max_keys= */ 0); restore_files(children); @@ -472,7 +472,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject || disk->object_storage_root_path != restore_information.source_path; std::set renames; - auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const RelativePathsWithSize & objects) + auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const RelativePathsWithMetadata & objects) { const String rename = "rename"; const String hardlink = "hardlink"; @@ -539,8 +539,8 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject return true; }; - RelativePathsWithSize children; - source_object_storage->findAllFiles(restore_information.source_path + "operations/", children, /* max_keys= */ 0); + RelativePathsWithMetadata children; + source_object_storage->listObjects(restore_information.source_path + "operations/", children, /* max_keys= */ 0); restore_file_operations(children); if (restore_information.detached) diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index a810db0cdf8..a5903f9d429 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -16,15 +16,29 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void IObjectStorage::findAllFiles(const std::string &, RelativePathsWithSize &, int) const +bool IObjectStorage::existsOrHasAnyChild(const std::string & path) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "findAllFiles() is not supported"); + RelativePathsWithMetadata files; + listObjects(path, files, 1); + return !files.empty(); } -void IObjectStorage::getDirectoryContents(const std::string &, - RelativePathsWithSize &, - std::vector &) const + +void IObjectStorage::listObjects(const std::string &, RelativePathsWithMetadata &, int) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getDirectoryContents() is not supported"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listObjects() is not supported"); +} + + +std::optional IObjectStorage::tryGetObjectMetadata(const std::string & path) const +{ + try + { + return getObjectMetadata(path); + } + catch (...) + { + return {}; + } } ThreadPool & IObjectStorage::getThreadPoolWriter() diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 8babb2fbf1a..3a0bf1834a1 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -30,19 +30,6 @@ class WriteBufferFromFileBase; using ObjectAttributes = std::map; -struct RelativePathWithSize -{ - String relative_path; - size_t bytes_size; - - RelativePathWithSize() = default; - - RelativePathWithSize(const String & relative_path_, size_t bytes_size_) - : relative_path(relative_path_), bytes_size(bytes_size_) {} -}; -using RelativePathsWithSize = std::vector; - - struct ObjectMetadata { uint64_t size_bytes; @@ -50,6 +37,22 @@ struct ObjectMetadata std::optional attributes; }; +struct RelativePathWithMetadata +{ + String relative_path; + ObjectMetadata metadata{}; + + RelativePathWithMetadata() = default; + + RelativePathWithMetadata(const String & relative_path_, const ObjectMetadata & metadata_) + : relative_path(relative_path_), metadata(metadata_) + {} +}; + +using RelativePathsWithMetadata = std::vector; + + + /// Base class for all object storages which implement some subset of ordinary filesystem operations. /// /// Examples of object storages are S3, Azure Blob Storage, HDFS. @@ -65,36 +68,17 @@ public: /// Object exists or not virtual bool exists(const StoredObject & object) const = 0; - /// List all objects with specific prefix. - /// - /// For example if you do this over filesystem, you should skip folders and - /// return files only, so something like on local filesystem: - /// - /// find . -type f - /// - /// @param children - out files (relative paths) with their sizes. - /// @param max_keys - return not more then max_keys children - /// NOTE: max_keys is not the same as list_object_keys_size (disk property) - /// - if max_keys is set not more then max_keys keys should be returned - /// - however list_object_keys_size determine the size of the batch and should return all keys - /// - /// NOTE: It makes sense only for real object storages (S3, Azure), since - /// it is used only for one of the following: - /// - send_metadata (to restore metadata) - /// - see DiskObjectStorage::restoreMetadataIfNeeded() - /// - MetadataStorageFromPlainObjectStorage - only for s3_plain disk - virtual void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const; + /// Object exists or any child on the specified path exists. + /// We have this method because object storages are flat for example + /// /a/b/c/d may exist but /a/b/c may not. So this method will return true for + /// /, /a, /a/b, /a/b/c, /a/b/c/d while exists will return true only for /a/b/c/d + virtual bool existsOrHasAnyChild(const std::string & path) const; - /// Analog of directory content for object storage (object storage does not - /// have "directory" definition, but it can be emulated with usage of - /// "delimiter"), so this is analog of: - /// - /// find . -maxdepth 1 $path - /// - /// Return files in @files and directories in @directories - virtual void getDirectoryContents(const std::string & path, - RelativePathsWithSize & files, - std::vector & directories) const; + virtual void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const; + + /// Get object metadata if supported. It should be possible to receive + /// at least size of object + virtual std::optional tryGetObjectMetadata(const std::string & path) const; /// Get object metadata if supported. It should be possible to receive /// at least size of object diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp index 6adf24b5bda..9461a82845f 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp @@ -142,10 +142,10 @@ StoredObjects MetadataStorageFromDisk::getStorageObjects(const std::string & pat object_storage_paths.reserve(object_storage_relative_paths.size()); /// Relative paths -> absolute. - for (auto & [object_relative_path, size] : object_storage_relative_paths) + for (auto & [object_relative_path, object_meta] : object_storage_relative_paths) { auto object_path = fs::path(metadata->getBlobsCommonPrefix()) / object_relative_path; - StoredObject object{ object_path, size, path }; + StoredObject object{ object_path, object_meta.size_bytes, path }; object_storage_paths.push_back(object); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index a680a344746..2459fa38da3 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -39,11 +39,10 @@ std::filesystem::path MetadataStorageFromPlainObjectStorage::getAbsolutePath(con bool MetadataStorageFromPlainObjectStorage::exists(const std::string & path) const { - RelativePathsWithSize children; /// NOTE: exists() cannot be used here since it works only for existing /// key, and does not work for some intermediate path. - object_storage->findAllFiles(getAbsolutePath(path), children, 1); - return !children.empty(); + std::string abs_path = getAbsolutePath(path); + return object_storage->existsOrHasAnyChild(abs_path); } bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) const @@ -55,44 +54,47 @@ bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) con bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path) const { std::string directory = getAbsolutePath(path); - trimRight(directory); - directory += "/"; + if (!directory.ends_with('/')) + directory += '/'; - /// NOTE: This check is far from ideal, since it work only if the directory - /// really has files, and has excessive API calls - RelativePathsWithSize files; - std::vector directories; - object_storage->getDirectoryContents(directory, files, directories); - return !files.empty() || !directories.empty(); + RelativePathsWithMetadata files; + object_storage->listObjects(directory, files, 1); + return !files.empty(); } uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const { - RelativePathsWithSize children; - object_storage->findAllFiles(getAbsolutePath(path), children, 1); - if (children.empty()) - return 0; - if (children.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "findAllFiles() return multiple paths ({}) for {}", children.size(), path); - return children.front().bytes_size; + RelativePathsWithMetadata children; + auto metadata = object_storage->tryGetObjectMetadata(getAbsolutePath(path)); + if (metadata) + return metadata->size_bytes; + return 0; } std::vector MetadataStorageFromPlainObjectStorage::listDirectory(const std::string & path) const { - RelativePathsWithSize files; - std::vector directories; - object_storage->getDirectoryContents(getAbsolutePath(path), files, directories); + RelativePathsWithMetadata files; + std::string abs_path = getAbsolutePath(path); + if (!abs_path.ends_with('/')) + abs_path += '/'; + + object_storage->listObjects(abs_path, files, 0); std::vector result; for (const auto & path_size : files) + { result.push_back(path_size.relative_path); - for (const auto & directory : directories) - result.push_back(directory); + } + for (auto & row : result) { - chassert(row.starts_with(object_storage_root_path)); - row.erase(0, object_storage_root_path.size()); + chassert(row.starts_with(abs_path)); + row.erase(0, abs_path.size()); + auto slash_pos = row.find_first_of('/'); + if (slash_pos != std::string::npos) + row.erase(slash_pos, row.size() - slash_pos); } + return result; } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 79e9e1141bb..fc7ca4c35b5 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -183,7 +183,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN disk_write_settings); } -void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const +void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { auto settings_ptr = s3_settings.get(); auto client_ptr = client.get(); @@ -211,7 +211,7 @@ void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSi break; for (const auto & object : objects) - children.emplace_back(object.GetKey(), object.GetSize()); + children.emplace_back(object.GetKey(), ObjectMetadata{static_cast(object.GetSize()), object.GetLastModified().Millis() / 1000, {}}); if (max_keys) { @@ -225,54 +225,6 @@ void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSi } while (outcome.GetResult().GetIsTruncated()); } -void S3ObjectStorage::getDirectoryContents(const std::string & path, - RelativePathsWithSize & files, - std::vector & directories) const -{ - auto settings_ptr = s3_settings.get(); - auto client_ptr = client.get(); - - S3::ListObjectsV2Request request; - request.SetBucket(bucket); - /// NOTE: if you do "ls /foo" instead of "ls /foo/" over S3 with this API - /// it will return only "/foo" itself without any underlying nodes. - if (path.ends_with("/")) - request.SetPrefix(path); - else - request.SetPrefix(path + "/"); - request.SetMaxKeys(settings_ptr->list_object_keys_size); - request.SetDelimiter("/"); - - Aws::S3::Model::ListObjectsV2Outcome outcome; - do - { - ProfileEvents::increment(ProfileEvents::S3ListObjects); - ProfileEvents::increment(ProfileEvents::DiskS3ListObjects); - outcome = client_ptr->ListObjectsV2(request); - throwIfError(outcome); - - auto result = outcome.GetResult(); - auto result_objects = result.GetContents(); - auto result_common_prefixes = result.GetCommonPrefixes(); - - if (result_objects.empty() && result_common_prefixes.empty()) - break; - - for (const auto & object : result_objects) - files.emplace_back(object.GetKey(), object.GetSize()); - - for (const auto & common_prefix : result_common_prefixes) - { - std::string directory = common_prefix.GetPrefix(); - /// Make it compatible with std::filesystem::path::filename() - trimRight(directory, '/'); - directories.emplace_back(directory); - } - - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - } while (outcome.GetResult().GetIsTruncated()); -} - void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exists) { auto client_ptr = client.get(); @@ -359,6 +311,22 @@ void S3ObjectStorage::removeObjectsIfExist(const StoredObjects & objects) removeObjectsImpl(objects, true); } +std::optional S3ObjectStorage::tryGetObjectMetadata(const std::string & path) const +{ + auto settings_ptr = s3_settings.get(); + auto object_info = S3::getObjectInfo(*client.get(), bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); + + if (object_info.size == 0 && object_info.last_modification_time == 0 && object_info.metadata.empty()) + return {}; + + ObjectMetadata result; + result.size_bytes = object_info.size; + result.last_modified = object_info.last_modification_time; + result.attributes = object_info.metadata; + + return result; +} + ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 70ed899586e..b0eb01aec0d 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -100,10 +100,7 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; - void getDirectoryContents(const std::string & path, - RelativePathsWithSize & files, - std::vector & directories) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; /// Uses `DeleteObjectRequest`. void removeObject(const StoredObject & object) override; @@ -121,6 +118,8 @@ public: ObjectMetadata getObjectMetadata(const std::string & path) const override; + std::optional tryGetObjectMetadata(const std::string & path) const override; + void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, From c73c836f9dfc74dbf5e9d213843c03a97e2a5735 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Jun 2023 23:33:04 +0200 Subject: [PATCH 1059/2223] Better --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 2459fa38da3..1131ec313e4 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -86,6 +86,7 @@ std::vector MetadataStorageFromPlainObjectStorage::listDirectory(co result.push_back(path_size.relative_path); } + std::unordered_set duplicates_filter; for (auto & row : result) { chassert(row.starts_with(abs_path)); @@ -93,9 +94,10 @@ std::vector MetadataStorageFromPlainObjectStorage::listDirectory(co auto slash_pos = row.find_first_of('/'); if (slash_pos != std::string::npos) row.erase(slash_pos, row.size() - slash_pos); + duplicates_filter.insert(row); } - return result; + return std::vector(duplicates_filter.begin(), duplicates_filter.end()); } DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(const std::string & path) const From 530f743ed062157cb0fc74d3bbc0bf51b186c0b5 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 2 Jun 2023 23:41:25 +0200 Subject: [PATCH 1060/2223] Fix Object data type for StorageDistributed --- src/Storages/StorageDistributed.cpp | 3 ++- src/Storages/StorageDummy.cpp | 3 ++- src/Storages/StorageDummy.h | 9 ++++++++- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 43b1333413e..b9625ce2ab7 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -948,7 +948,8 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto resolved_remote_storage_id = remote_storage_id; if (remote_storage_id.hasDatabase()) resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); - auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns()); + + auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns(), distributed_storage_snapshot->object_columns); auto table_node = std::make_shared(std::move(storage), query_context); if (table_expression_modifiers) diff --git a/src/Storages/StorageDummy.cpp b/src/Storages/StorageDummy.cpp index e5f3b0b4d8e..4f2fb3883bf 100644 --- a/src/Storages/StorageDummy.cpp +++ b/src/Storages/StorageDummy.cpp @@ -9,8 +9,9 @@ namespace DB { -StorageDummy::StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_) +StorageDummy::StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_, ColumnsDescription object_columns_) : IStorage(table_id_) + , object_columns(std::move(object_columns_)) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); diff --git a/src/Storages/StorageDummy.h b/src/Storages/StorageDummy.h index a7beef9d531..2f9a8beb4d0 100644 --- a/src/Storages/StorageDummy.h +++ b/src/Storages/StorageDummy.h @@ -11,7 +11,7 @@ namespace DB class StorageDummy : public IStorage { public: - StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_); + StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_, ColumnsDescription object_columns_ = {}); std::string getName() const override { return "StorageDummy"; } @@ -22,6 +22,11 @@ public: bool supportsDynamicSubcolumns() const override { return true; } bool canMoveConditionsToPrewhere() const override { return false; } + StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr /*query_context*/) const override + { + return std::make_shared(*this, metadata_snapshot, object_columns); + } + QueryProcessingStage::Enum getQueryProcessingStage( ContextPtr local_context, QueryProcessingStage::Enum to_stage, @@ -37,6 +42,8 @@ public: QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams) override; +private: + const ColumnsDescription object_columns; }; class ReadFromDummy : public SourceStepWithFilter From e548dce123debf4864348d606629f90844b5e5f8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 3 Jun 2023 00:08:47 +0200 Subject: [PATCH 1061/2223] fix --- tests/ci/performance_comparison_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index 1baf547816f..41ace95c350 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -220,7 +220,7 @@ if __name__ == "__main__": traceback.print_exc() def too_many_slow(msg): - match = re.search("(|.* )(\d+) slower.*", msg) + match = re.search(r"(|.* )(\d+) slower.*", msg) # This threshold should be synchronized with the value in https://github.com/ClickHouse/ClickHouse/blob/master/docker/test/performance-comparison/report.py#L629 threshold = 5 return int(match.group(2).strip()) > threshold if match else False From 30be723a9aca0a585e92dbaee71d15905d6dc490 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Jun 2023 05:11:02 +0200 Subject: [PATCH 1062/2223] Fix bad code around metadata in RocksDB --- src/Common/ProfileEvents.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 9f4fc2d135b..2e0c4b82bba 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -429,10 +429,10 @@ The server successfully detected this situation and will download merged part fr M(AggregationPreallocatedElementsInHashTables, "How many elements were preallocated in hash tables for aggregation.") \ M(AggregationHashTablesInitializedAsTwoLevel, "How many hash tables were inited as two-level for aggregation.") \ \ - M(MergeTreeMetadataCacheGet, "Number of rocksdb reads(used for merge tree metadata cache)") \ - M(MergeTreeMetadataCachePut, "Number of rocksdb puts(used for merge tree metadata cache)") \ - M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \ - M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheGet, "Number of rocksdb reads (used for merge tree metadata cache)") \ + M(MergeTreeMetadataCachePut, "Number of rocksdb puts (used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes (used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks (used for merge tree metadata cache)") \ M(MergeTreeMetadataCacheHit, "Number of times the read of meta file was done from MergeTree metadata cache") \ M(MergeTreeMetadataCacheMiss, "Number of times the read of meta file was not done from MergeTree metadata cache") \ \ From ccba3500dd92eaecd38a56d92b09336af26f371c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Jun 2023 07:13:26 +0300 Subject: [PATCH 1063/2223] Update StoragePostgreSQL.cpp --- src/Storages/StoragePostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index ab6660abe00..431285da60d 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -247,6 +247,7 @@ public: if (nested_type->isNullable()) nested_type = static_cast(nested_type.get())->getNestedType(); + /// UUIDs inside arrays are expected to be unquoted in PostgreSQL. const bool quoted = !isUUID(nested_type); writeChar('{', ostr); From 7a7e03a2ffbd879afd5971de6de13c7919a89157 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Jun 2023 14:16:59 +0300 Subject: [PATCH 1064/2223] Function if constant folding --- src/Functions/if.cpp | 25 +++++++++++++++++++ .../25337_if_constant_folding.reference | 5 ++++ .../0_stateless/25337_if_constant_folding.sql | 1 + 3 files changed, 31 insertions(+) create mode 100644 tests/queries/0_stateless/25337_if_constant_folding.reference create mode 100644 tests/queries/0_stateless/25337_if_constant_folding.sql diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 93bdf406f9d..d00e83c4eb7 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -1116,6 +1116,31 @@ public: return res; } + + ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override + { + const ColumnWithTypeAndName & arg_cond = arguments[0]; + if (!arg_cond.column || !isColumnConst(*arg_cond.column)) { + return {}; + } + + const ColumnConst * cond_const_col = checkAndGetColumnConst>(arg_cond.column.get()); + bool condition_value = cond_const_col->getValue(); + + const ColumnWithTypeAndName & arg_then = arguments[1]; + const ColumnWithTypeAndName & arg_else = arguments[2]; + const ColumnWithTypeAndName & potential_const_column = condition_value ? arg_then : arg_else; + + if (!potential_const_column.column || !isColumnConst(*potential_const_column.column)) + return {}; + + auto result = castColumn(potential_const_column, result_type); + if (!isColumnConst(*result)) { + return {}; + } + + return result; + } }; } diff --git a/tests/queries/0_stateless/25337_if_constant_folding.reference b/tests/queries/0_stateless/25337_if_constant_folding.reference new file mode 100644 index 00000000000..9dfcf39f5a7 --- /dev/null +++ b/tests/queries/0_stateless/25337_if_constant_folding.reference @@ -0,0 +1,5 @@ +0 +1 +2 +3 +4 diff --git a/tests/queries/0_stateless/25337_if_constant_folding.sql b/tests/queries/0_stateless/25337_if_constant_folding.sql new file mode 100644 index 00000000000..1610465021b --- /dev/null +++ b/tests/queries/0_stateless/25337_if_constant_folding.sql @@ -0,0 +1 @@ +SELECT cast(number, if(1 = 1, 'UInt64', toString(number))) FROM numbers(5); From 894457d6e1cf7f005e1eb3cae57dded0dd8c4699 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 3 Jun 2023 14:36:52 +0200 Subject: [PATCH 1065/2223] Style fxi --- src/Disks/ObjectStorages/IObjectStorage.h | 1 - .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 5 ----- 2 files changed, 6 deletions(-) diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 3a0bf1834a1..28de80a88cd 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -52,7 +52,6 @@ struct RelativePathWithMetadata using RelativePathsWithMetadata = std::vector; - /// Base class for all object storages which implement some subset of ordinary filesystem operations. /// /// Examples of object storages are S3, Azure Blob Storage, HDFS. diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 1131ec313e4..c119e9f3adc 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -10,11 +10,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - MetadataStorageFromPlainObjectStorage::MetadataStorageFromPlainObjectStorage( ObjectStoragePtr object_storage_, const std::string & object_storage_root_path_) From 13a122697139f80f34ca006f691fd1f4f20e8528 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 16:09:05 +0200 Subject: [PATCH 1066/2223] Fix nested distributed SELECT in WITH clause For the CSE (common scalar expression, form of CTE) form of WITH it will set max_result_rows to 1, since there should not be more rows, but this will be applied for the DESC TABLE as well (service distributed query that required to obtain the structure). Note, that it is a problem only for nested distributed queries because getSubqueryContext() from InterpreterSelectQuery.cpp resets those limits as well, but this does not helps, for the nested DESC since it is executed on the remote node that has max_result_rows=1. Signed-off-by: Azat Khuzhin --- src/Storages/getStructureOfRemoteTable.cpp | 9 +++++++++ .../0_stateless/02768_cse_nested_distributed.reference | 3 +++ .../queries/0_stateless/02768_cse_nested_distributed.sql | 5 +++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02768_cse_nested_distributed.reference create mode 100644 tests/queries/0_stateless/02768_cse_nested_distributed.sql diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index b2737249166..e5fc01be9f4 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -60,6 +60,15 @@ ColumnsDescription getStructureOfRemoteTableInShard( ColumnsDescription res; auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef(), table_id); + /// Ignore limit for result number of rows (that could be set during handling CSE/CTE), + /// since this is a service query and should not lead to query failure. + { + Settings new_settings = new_context->getSettings(); + new_settings.max_result_rows = 0; + new_settings.max_result_bytes = 0; + new_context->setSettings(new_settings); + } + /// Expect only needed columns from the result of DESC TABLE. NOTE 'comment' column is ignored for compatibility reasons. Block sample_block { diff --git a/tests/queries/0_stateless/02768_cse_nested_distributed.reference b/tests/queries/0_stateless/02768_cse_nested_distributed.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/02768_cse_nested_distributed.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/02768_cse_nested_distributed.sql b/tests/queries/0_stateless/02768_cse_nested_distributed.sql new file mode 100644 index 00000000000..90e526c0d01 --- /dev/null +++ b/tests/queries/0_stateless/02768_cse_nested_distributed.sql @@ -0,0 +1,5 @@ +with (select count() > 0 from remote('127.2', system.settings)) as s select s; +-- nested +with (select count() > 0 from remote('127.2', remote('127.2', system.settings))) as s select s; +-- nested via view() +with (select count() > 0 from remote('127.2', view(select count() from remote('127.2', system.settings)))) as s select s; From e28dfb7ea851844c943a4dbab33dcf6d2f468f4e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Jun 2023 21:44:31 +0300 Subject: [PATCH 1067/2223] Updated tests --- tests/queries/0_stateless/00835_if_generic_case.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00835_if_generic_case.reference b/tests/queries/0_stateless/00835_if_generic_case.reference index 84c036b17ca..8f9c3f6ef13 100644 --- a/tests/queries/0_stateless/00835_if_generic_case.reference +++ b/tests/queries/0_stateless/00835_if_generic_case.reference @@ -3,7 +3,7 @@ 2000-01-01 00:00:00 2000-01-02 2000-01-02 00:00:00 2000-01-01 00:00:00 2000-01-02 2000-01-02 00:00:00 2000-01-01 00:00:00 2000-01-02 2000-01-02 00:00:00 -2000-01-01 00:00:00 2000-01-02 2000-01-02 +2000-01-01 00:00:00 2000-01-02 2000-01-02 00:00:00 2000-01-01 00:00:00 2000-01-02 2000-01-02 2000-01-01 00:00:00 2000-01-02 2000-01-02 2000-01-01 00:00:00 2000-01-02 2000-01-01 00:00:00 From e61131c4bb7fda6a6883c4b3946e6d5862547728 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Jun 2023 21:45:53 +0300 Subject: [PATCH 1068/2223] QueryNode small fix --- src/Analyzer/QueryNode.cpp | 20 +++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 4c10d76690a..51e3dac781d 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -202,15 +202,16 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && - cte_name == rhs_typed.cte_name && - projection_columns == rhs_typed.projection_columns && is_distinct == rhs_typed.is_distinct && is_limit_with_ties == rhs_typed.is_limit_with_ties && is_group_by_with_totals == rhs_typed.is_group_by_with_totals && is_group_by_with_rollup == rhs_typed.is_group_by_with_rollup && is_group_by_with_cube == rhs_typed.is_group_by_with_cube && is_group_by_with_grouping_sets == rhs_typed.is_group_by_with_grouping_sets && - is_group_by_all == rhs_typed.is_group_by_all; + is_group_by_all == rhs_typed.is_group_by_all && + cte_name == rhs_typed.cte_name && + projection_columns == rhs_typed.projection_columns && + settings_changes == rhs_typed.settings_changes; } void QueryNode::updateTreeHashImpl(HashState & state) const @@ -239,6 +240,18 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(is_group_by_with_cube); state.update(is_group_by_with_grouping_sets); state.update(is_group_by_all); + + state.update(settings_changes.size()); + + for (const auto & setting_change : settings_changes) + { + state.update(setting_change.name.size()); + state.update(setting_change.name); + + auto setting_change_value_dump = setting_change.value.dump(); + state.update(setting_change_value_dump.size()); + state.update(setting_change_value_dump); + } } QueryTreeNodePtr QueryNode::cloneImpl() const @@ -256,6 +269,7 @@ QueryTreeNodePtr QueryNode::cloneImpl() const result_query_node->is_group_by_all = is_group_by_all; result_query_node->cte_name = cte_name; result_query_node->projection_columns = projection_columns; + result_query_node->settings_changes = settings_changes; return result_query_node; } From 0f37be549236f4b891ec324abd73adfdcc68f0b0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 17:20:07 +0200 Subject: [PATCH 1069/2223] Fix description for BrokenDistributedFilesToInsert Signed-off-by: Azat Khuzhin --- src/Common/CurrentMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 82d68ca8185..63a23c30f84 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -146,7 +146,7 @@ M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ M(DistributedFilesToInsert, "Number of pending files to process for asynchronous insertion into Distributed tables. Number of files for every shard is summed.") \ - M(BrokenDistributedFilesToInsert, "Number of files for asynchronous insertion into Distributed tables that has been marked as broken. This metric will starts from 0 on start. Number of files for every shard is summed.") \ + M(BrokenDistributedFilesToInsert, "Number of files for asynchronous insertion into Distributed tables that has been marked as broken. Number of files for every shard is summed.") \ M(TablesToDropQueueSize, "Number of dropped tables, that are waiting for background data removal.") \ M(MaxDDLEntryID, "Max processed DDL entry of DDLWorker.") \ M(MaxPushedDDLEntryID, "Max DDL entry of DDLWorker that pushed to zookeeper.") \ From 69aec7af9bfa475bbc6581f4c582eff6db04d3de Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 17:23:39 +0200 Subject: [PATCH 1070/2223] Add new metrics BrokenDistributedBytesToInsert/DistributedBytesToInsert Useful to see at the server status overall. Signed-off-by: Azat Khuzhin --- src/Common/CurrentMetrics.cpp | 2 ++ .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 9 +++++++++ .../Distributed/DistributedAsyncInsertDirectoryQueue.h | 2 ++ 3 files changed, 13 insertions(+) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 63a23c30f84..edfbbe17600 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -145,6 +145,8 @@ M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool.") \ M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ + M(DistributedBytesToInsert, "Number of pending bytes to process for asynchronous insertion into Distributed tables. Number of bytes for every shard is summed.") \ + M(BrokenDistributedBytesToInsert, "Number of bytes for asynchronous insertion into Distributed tables that has been marked as broken. Number of bytes for every shard is summed.") \ M(DistributedFilesToInsert, "Number of pending files to process for asynchronous insertion into Distributed tables. Number of files for every shard is summed.") \ M(BrokenDistributedFilesToInsert, "Number of files for asynchronous insertion into Distributed tables that has been marked as broken. Number of files for every shard is summed.") \ M(TablesToDropQueueSize, "Number of dropped tables, that are waiting for background data removal.") \ diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 9a9a6651bc4..d8d9a0c9d1e 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -35,6 +35,8 @@ namespace CurrentMetrics extern const Metric DistributedSend; extern const Metric DistributedFilesToInsert; extern const Metric BrokenDistributedFilesToInsert; + extern const Metric DistributedBytesToInsert; + extern const Metric BrokenDistributedBytesToInsert; } namespace fs = std::filesystem; @@ -138,7 +140,9 @@ DistributedAsyncInsertDirectoryQueue::DistributedAsyncInsertDirectoryQueue( , max_sleep_time(storage.getDistributedSettingsRef().monitor_max_sleep_time_ms.totalMilliseconds()) , log(&Poco::Logger::get(getLoggerName())) , monitor_blocker(monitor_blocker_) + , metric_pending_bytes(CurrentMetrics::DistributedBytesToInsert, 0) , metric_pending_files(CurrentMetrics::DistributedFilesToInsert, 0) + , metric_broken_bytes(CurrentMetrics::BrokenDistributedBytesToInsert, 0) , metric_broken_files(CurrentMetrics::BrokenDistributedFilesToInsert, 0) { fs::create_directory(broken_path); @@ -357,6 +361,7 @@ void DistributedAsyncInsertDirectoryQueue::initializeFilesFromDisk() LOG_TRACE(log, "Files set to {}", pending_files.size()); LOG_TRACE(log, "Bytes set to {}", bytes_count); + metric_pending_bytes.changeTo(bytes_count); metric_pending_files.changeTo(pending_files.size()); status.files_count = pending_files.size(); status.bytes_count = bytes_count; @@ -380,6 +385,7 @@ void DistributedAsyncInsertDirectoryQueue::initializeFilesFromDisk() LOG_TRACE(log, "Broken bytes set to {}", broken_bytes_count); metric_broken_files.changeTo(broken_files); + metric_broken_bytes.changeTo(broken_bytes_count); status.broken_files_count = broken_files; status.broken_bytes_count = broken_bytes_count; } @@ -520,6 +526,7 @@ bool DistributedAsyncInsertDirectoryQueue::addFileAndSchedule(const std::string { std::lock_guard lock(status_mutex); metric_pending_files.add(); + metric_pending_bytes.add(file_size); status.bytes_count += file_size; ++status.files_count; } @@ -679,6 +686,7 @@ void DistributedAsyncInsertDirectoryQueue::markAsBroken(const std::string & file status.broken_bytes_count += file_size; metric_broken_files.add(); + metric_broken_bytes.add(file_size); } fs::rename(file_path, broken_file_path); @@ -692,6 +700,7 @@ void DistributedAsyncInsertDirectoryQueue::markAsSend(const std::string & file_p { std::lock_guard status_lock(status_mutex); metric_pending_files.sub(); + metric_pending_bytes.sub(file_size); --status.files_count; status.bytes_count -= file_size; } diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h index de8bb813824..9a8a235e265 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h @@ -149,7 +149,9 @@ private: BackgroundSchedulePoolTaskHolder task_handle; + CurrentMetrics::Increment metric_pending_bytes; CurrentMetrics::Increment metric_pending_files; + CurrentMetrics::Increment metric_broken_bytes; CurrentMetrics::Increment metric_broken_files; }; From 009fe3d25e8a755e86d45084f2af5784f4463523 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 3 Jun 2023 21:32:29 +0200 Subject: [PATCH 1071/2223] Add profile events for eviction --- src/Common/ProfileEvents.cpp | 3 +++ src/Interpreters/Cache/FileCache.cpp | 14 +++++++++++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 9f4fc2d135b..8146a5017ad 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -381,6 +381,9 @@ The server successfully detected this situation and will download merged part fr M(CachedWriteBufferCacheWriteBytes, "Bytes written from source (remote fs, etc) to filesystem cache") \ M(CachedWriteBufferCacheWriteMicroseconds, "Time spent writing data into filesystem cache") \ \ + M(FilesystemCacheEvictedBytes, "Number of bytes evicted from filesystem cache") \ + M(FilesystemCacheEvictedFileSegments, "Number of file segments evicted from filesystem cache") \ + \ M(RemoteFSSeeks, "Total number of seeks for async buffer") \ M(RemoteFSPrefetches, "Number of prefetches made with asynchronous reading from remote filesystem") \ M(RemoteFSCancelledPrefetches, "Number of cancelled prefecthes (because of seek)") \ diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index ba160a31b73..79a9765108f 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -17,6 +17,12 @@ namespace fs = std::filesystem; +namespace ProfileEvents +{ + extern const Event FilesystemCacheEvictedBytes; + extern const Event FilesystemCacheEvictedFileSegments; +} + namespace { @@ -643,7 +649,9 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) return PriorityIterationResult::CONTINUE; } - /// TODO: we can resize if partially downloaded instead. + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + locked_key.removeFileSegment(segment->offset(), segment->lock()); return PriorityIterationResult::REMOVE_AND_CONTINUE; } @@ -721,6 +729,10 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) chassert(candidate->releasable()); const auto * segment = candidate->file_segment.get(); + + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + locked_key->removeFileSegment(segment->offset(), segment->lock()); segment->getQueueIterator()->remove(cache_lock); From a0df8566051a4c0c163dae14b5e41b7d83b7540c Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 3 Jun 2023 23:11:03 +0200 Subject: [PATCH 1072/2223] Able to insert --- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 17 +++++++++----- src/Storages/StorageAzure.cpp | 4 ++-- src/Storages/StorageAzure.h | 22 +++++++++++++------ 3 files changed, 28 insertions(+), 15 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 1e06490b5bc..1b62b5fdb05 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -57,14 +57,22 @@ void validateContainerName(const String & container_name) AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { - String storage_account_url = config.getString(config_prefix + ".storage_account_url"); - validateStorageAccountUrl(storage_account_url); + std::string storage_url; + if (config.has(config_prefix + ".storage_account_url")) + { + storage_url = config.getString(config_prefix + ".storage_account_url"); + validateStorageAccountUrl(storage_url); + } + else + { + storage_url = config.getString(config_prefix + ".connection_string"); + } String container_name = config.getString(config_prefix + ".container_name", "default-container"); validateContainerName(container_name); std::optional container_already_exists {}; if (config.has(config_prefix + ".container_already_exists")) container_already_exists = {config.getBool(config_prefix + ".container_already_exists")}; - return {storage_account_url, container_name, container_already_exists}; + return {storage_url, container_name, container_already_exists}; } @@ -136,10 +144,7 @@ std::unique_ptr getAzureBlobContainerClient( /// If container_already_exists is not set (in config), ignore already exists error. /// (Conflict - The specified container already exists) if (!endpoint.container_already_exists.has_value() && e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict) - { - tryLogCurrentException("Container already exists, returning the existing container"); return getAzureBlobStorageClientWithAuth(final_url, container_name, config, config_prefix); - } throw; } } diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 30fd3fcbe95..fd250a128c9 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -236,7 +236,7 @@ AzureClientPtr StorageAzure::createClient(StorageAzure::Configuration configurat StorageAzure::StorageAzure( const Configuration & configuration_, std::unique_ptr && object_storage_, - ContextPtr context_, + ContextPtr, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -252,7 +252,7 @@ StorageAzure::StorageAzure( , partition_by(partition_by_) { FormatFactory::instance().checkFormatName(configuration.format); - context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(configuration.getConnectionURL())); + //context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(configuration.getConnectionURL())); StorageInMemoryMetadata storage_metadata; if (columns_.empty()) diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index b99df2e89a5..6bf18d91265 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -18,6 +18,11 @@ struct AzureSimpleAccountConfiguration std::string storage_account_url; }; +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + using AzureConnectionString = std::string; using AzureCredentials = std::variant; @@ -85,13 +90,16 @@ public: } Pipe read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - size_t num_streams) override; + const Names &, + const StorageSnapshotPtr &, + SelectQueryInfo &, + ContextPtr, + QueryProcessingStage::Enum, + size_t, + size_t) override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Read not implemented"); + } SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr context) override; From 57070227a8468f68f9caede43b4d93ad286a2d00 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 3 Jun 2023 23:44:19 +0200 Subject: [PATCH 1073/2223] Fxi --- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp | 4 +++- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 0044f465081..0358b4e915a 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -91,7 +91,9 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith blob.Name, ObjectMetadata{ static_cast(blob.BlobSize), - blob.Details.LastModified.time_since_epoch().count(), + Poco::Timestamp::fromEpochTime( + std::chrono::duration_cast( + blob.Details.LastModified.time_since_epoch()).count()), {}}); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index fc7ca4c35b5..6e63efcc1e3 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -211,7 +211,7 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet break; for (const auto & object : objects) - children.emplace_back(object.GetKey(), ObjectMetadata{static_cast(object.GetSize()), object.GetLastModified().Millis() / 1000, {}}); + children.emplace_back(object.GetKey(), ObjectMetadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}}); if (max_keys) { From 6f01e2ad9fd25d3d09828d7fb8847d7e730ccb3f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Jun 2023 02:49:45 +0200 Subject: [PATCH 1074/2223] Fix incorrect constant folding --- src/Parsers/ExpressionElementParsers.cpp | 6 ++- ..._formatting_and_constant_folding.reference | 4 ++ ...istent_formatting_and_constant_folding.sql | 41 +++++++++++++++++++ 3 files changed, 50 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.reference create mode 100644 tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.sql diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 28cef51e571..3a7e8790bb4 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -829,7 +829,11 @@ static bool parseNumber(char * buffer, size_t size, bool negative, int base, Fie if (pos_integer == buffer + size && errno != ERANGE && (!negative || uint_value <= (1ULL << 63))) { - if (negative) + /// -0 should be still parsed as UInt instead of Int, + /// because otherwise it is not preserved during formatting-parsing roundtrip + /// (the signedness is lost during formatting) + + if (negative && uint_value != 0) res = static_cast(-uint_value); else res = uint_value; diff --git a/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.reference b/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.reference new file mode 100644 index 00000000000..39d23aa2298 --- /dev/null +++ b/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.reference @@ -0,0 +1,4 @@ +0 UInt8 -1 Int8 -0 Float64 +0 +0 +0 diff --git a/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.sql b/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.sql new file mode 100644 index 00000000000..31278862ab8 --- /dev/null +++ b/tests/queries/0_stateless/02782_inconsistent_formatting_and_constant_folding.sql @@ -0,0 +1,41 @@ +SELECT -0, toTypeName(-0), -1, toTypeName(-1), -0., toTypeName(-0.); + +DROP TABLE IF EXISTS t4; +DROP TABLE IF EXISTS t7; + +create table t4 (c26 String) engine = Log; +create view t7 as select max(ref_3.c26) as c_2_c46_1 from t4 as ref_3; + +select + c_7_c4585_14 as c_4_c4593_5 + from + (select + avg(0) as c_7_c4572_1, + max(-0) as c_7_c4585_14 + from + t7 as ref_0 + group by ref_0.c_2_c46_1) as subq_0 +where c_4_c4593_5 <= multiIf(true, 1, exp10(c_4_c4593_5) <= 1, 1, 1); + +select x as c + from + (select 1 AS k, + max(0) as a, + max(-0) as x + from + t7 GROUP BY k) +where NOT ignore(c); + +SELECT x +FROM +( + SELECT + avg(0) AS c_7_c4572_1, + max(-0) AS x + FROM t7 AS ref_0 + GROUP BY ref_0.c_2_c46_1 +) +WHERE x <= multiIf(true, 1, exp10(x) <= 1, 1, 1); + +DROP TABLE t7; +DROP TABLE t4; From c62558f982366ba2b2bc03cd410bf70840358fcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Jun 2023 04:44:51 +0200 Subject: [PATCH 1075/2223] Control memory usage in generateRandom --- src/Storages/StorageGenerateRandom.cpp | 85 ++++++++++++++++++- ...0416_pocopatch_progress_in_http_headers.sh | 1 - 2 files changed, 83 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index e48d3187cb2..293beca9c24 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -29,6 +29,7 @@ #include #include +#include #include #include @@ -81,6 +82,66 @@ void fillBufferWithRandomData(char * __restrict data, size_t limit, size_t size_ } +size_t estimateValueSize( + const DataTypePtr type, + UInt64 max_array_length, + UInt64 max_string_length) +{ + if (type->haveMaximumSizeOfValue()) + return type->getMaximumSizeOfValueInMemory(); + + TypeIndex idx = type->getTypeId(); + + switch (idx) + { + case TypeIndex::String: + { + return max_string_length + sizeof(size_t) + 1; + } + + /// The logic in this function should reflect the logic of fillColumnWithRandomData. + case TypeIndex::Array: + { + auto nested_type = typeid_cast(*type).getNestedType(); + return sizeof(size_t) + estimateValueSize(nested_type, max_array_length / 2, max_string_length); + } + + case TypeIndex::Map: + { + const DataTypePtr & nested_type = typeid_cast(*type).getNestedType(); + return sizeof(size_t) + estimateValueSize(nested_type, max_array_length / 2, max_string_length); + } + + case TypeIndex::Tuple: + { + auto elements = typeid_cast(type.get())->getElements(); + const size_t tuple_size = elements.size(); + size_t res = 0; + + for (size_t i = 0; i < tuple_size; ++i) + res += estimateValueSize(elements[i], max_array_length, max_string_length); + + return res; + } + + case TypeIndex::Nullable: + { + auto nested_type = typeid_cast(*type).getNestedType(); + return 1 + estimateValueSize(nested_type, max_array_length, max_string_length); + } + + case TypeIndex::LowCardinality: + { + auto nested_type = typeid_cast(*type).getDictionaryType(); + return sizeof(size_t) + estimateValueSize(nested_type, max_array_length, max_string_length); + } + + default: + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "The 'GenerateRandom' is not implemented for type {}", type->getName()); + } +} + + ColumnPtr fillColumnWithRandomData( const DataTypePtr type, UInt64 limit, @@ -192,7 +253,8 @@ ColumnPtr fillColumnWithRandomData( offsets[i] = offset; } - auto data_column = fillColumnWithRandomData(nested_type, offset, max_array_length, max_string_length, rng, context); + /// This division by two makes the size growth subexponential on depth. + auto data_column = fillColumnWithRandomData(nested_type, offset, max_array_length / 2, max_string_length, rng, context); return ColumnArray::create(data_column, std::move(offsets_column)); } @@ -200,7 +262,7 @@ ColumnPtr fillColumnWithRandomData( case TypeIndex::Map: { const DataTypePtr & nested_type = typeid_cast(*type).getNestedType(); - auto nested_column = fillColumnWithRandomData(nested_type, limit, max_array_length, max_string_length, rng, context); + auto nested_column = fillColumnWithRandomData(nested_type, limit, max_array_length / 2, max_string_length, rng, context); return ColumnMap::create(nested_column); } @@ -597,6 +659,25 @@ Pipe StorageGenerateRandom::read( block_header.insert({std::move(column), name_type.type, name_type.name}); } + /// Correction of block size for wide tables. + size_t preferred_block_size_bytes = context->getSettingsRef().preferred_block_size_bytes; + if (preferred_block_size_bytes) + { + size_t estimated_row_size_bytes = estimateValueSize(std::make_shared(block_header.getDataTypes()), max_array_length, max_string_length); + + size_t estimated_block_size_bytes = 0; + if (common::mulOverflow(max_block_size, estimated_row_size_bytes, estimated_block_size_bytes)) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large estimated block size in GenerateRandom table: its estimation leads to 64bit overflow"); + chassert(estimated_block_size_bytes != 0); + + if (estimated_block_size_bytes > preferred_block_size_bytes) + { + max_block_size = static_cast(max_block_size * (static_cast(preferred_block_size_bytes) / estimated_block_size_bytes)); + if (max_block_size == 0) + max_block_size = 1; + } + } + /// Will create more seed values for each source from initial seed. pcg64 generate(random_seed); diff --git a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh index 6e9814cbca8..b2189ab0cc2 100755 --- a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh +++ b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh @@ -5,7 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' -# This test will fail with external poco (progress not supported) ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&enable_http_compression=1" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d From 7ceedbd1806348f3af80d135430db5ca245a341a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Jun 2023 04:45:55 +0200 Subject: [PATCH 1076/2223] Update tests --- tests/queries/0_stateless/02539_generate_random_map.reference | 4 ++-- .../0_stateless/02586_generate_random_structure.reference | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02539_generate_random_map.reference b/tests/queries/0_stateless/02539_generate_random_map.reference index c0dc175c3cd..25e7f55667e 100644 --- a/tests/queries/0_stateless/02539_generate_random_map.reference +++ b/tests/queries/0_stateless/02539_generate_random_map.reference @@ -1,2 +1,2 @@ -1 -20 +0 +10 diff --git a/tests/queries/0_stateless/02586_generate_random_structure.reference b/tests/queries/0_stateless/02586_generate_random_structure.reference index d2929fb4564..abe168c59e8 100644 --- a/tests/queries/0_stateless/02586_generate_random_structure.reference +++ b/tests/queries/0_stateless/02586_generate_random_structure.reference @@ -13,7 +13,7 @@ c3.e6 Array(Int256) c4 FixedString(183) c5 IPv4 c6 UInt256 -Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:777622572,102:-1122882357,62:1647813163,-94:2094022166},{-32:1448633509},{},{},{34:1536340393,19:-2049677851,74:65643868,-46:-1990799930,97:-531041081,46:-2634833,14:1581632600,89:-771229823,-105:1238603584},{47:1458809010,109:1640682510,86:1945730198,85:1505847247,35:-35189402}] [153363749503.3642648494826450951141750747382772821825909005880434540971999557,79828591186.7378041015337066268618633118713347614941338787453473118807106292,81672688565.9633830721322966111551266731935181670389237071708068971548883315,573768486971.1812413548839655834002608768736215115033958693122764224003897029,-393925092368.4893467278351090742501814120269109477445490969167853713051140487,46027399426.0865278566391382610843315130162915324295037009704113636499519839] [755855942,1804001770,-78103159,-866181765,731736602,-79599206] [5253556148991564114,4681434929596395351,-7302160004580855709,-3686747220178471318,6288582051009949273,646864891160092871] [17035203905051045016266537043565487029724162173062647021612805252288722534904,-42105881403933504641593145676742477006499618886131028341247993701618141933523,45346626822580305846120377917274679004279343244238782744860626882886217433843,-3660165069803677989574889324494857545543653453780976182221584349306428201647,-23316760935816288837287058499520670431785615691220162210524162590241529297823,6184785563808848524970564618169964412151721224362412457508264894603779018817] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 -Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:777622572,102:-1122882357,62:1647813163,-94:2094022166},{-32:1448633509},{},{},{34:1536340393,19:-2049677851,74:65643868,-46:-1990799930,97:-531041081,46:-2634833,14:1581632600,89:-771229823,-105:1238603584},{47:1458809010,109:1640682510,86:1945730198,85:1505847247,35:-35189402}] [153363749503.3642648494826450951141750747382772821825909005880434540971999557,79828591186.7378041015337066268618633118713347614941338787453473118807106292,81672688565.9633830721322966111551266731935181670389237071708068971548883315,573768486971.1812413548839655834002608768736215115033958693122764224003897029,-393925092368.4893467278351090742501814120269109477445490969167853713051140487,46027399426.0865278566391382610843315130162915324295037009704113636499519839] [755855942,1804001770,-78103159,-866181765,731736602,-79599206] [5253556148991564114,4681434929596395351,-7302160004580855709,-3686747220178471318,6288582051009949273,646864891160092871] [17035203905051045016266537043565487029724162173062647021612805252288722534904,-42105881403933504641593145676742477006499618886131028341247993701618141933523,45346626822580305846120377917274679004279343244238782744860626882886217433843,-3660165069803677989574889324494857545543653453780976182221584349306428201647,-23316760935816288837287058499520670431785615691220162210524162590241529297823,6184785563808848524970564618169964412151721224362412457508264894603779018817] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 +Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:237920722},{102:1831835481},{},{},{62:-1960618666,-94:-1219892650},{-32:777622572}] [-431683920736.2529974565693898437068698973660186760023819586689844248966618581,59756148020.9162673945900094878560562068677573399624031604279613893604002735,586508082308.9120376291637372686770291666437007489038136467093669000000245094,-542964250950.8072472621084212227729061994250148872708538789242780170470779236,102348230986.0528243362965635624540408239649901816289081513971979913007157412,856260707339.0308016734722383288485766011293159337170215574648625303012038641] [700742145,583743175,-694731143,205377506,-47073316,524775483] [-1523104801326101990,-3813742700509249040,-198797568561120929,1360848130483946585,-2034445041726915230,6445693522245970031] [-42294009754395695750286152418877456026522055877244789929409825751148328749462,-30114637036117781224059264373564439361653611308928312514969460679379590706382,50790691897328530213645537587874962516097580703236937570724165115560305762147,-53982023979693597997184389721196431225054557132927198244547868871851761566844,-4746897427675718862552189488292169089453556841811840545196357111533622948298,-12014200750706123405006110782843469166121588186681927916239818910819762049960] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 +Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10','e1V0','e1V10','e1V14','e1V10','e1V14'] [{-13:237920722},{102:1831835481},{},{},{62:-1960618666,-94:-1219892650},{-32:777622572}] [-431683920736.2529974565693898437068698973660186760023819586689844248966618581,59756148020.9162673945900094878560562068677573399624031604279613893604002735,586508082308.9120376291637372686770291666437007489038136467093669000000245094,-542964250950.8072472621084212227729061994250148872708538789242780170470779236,102348230986.0528243362965635624540408239649901816289081513971979913007157412,856260707339.0308016734722383288485766011293159337170215574648625303012038641] [700742145,583743175,-694731143,205377506,-47073316,524775483] [-1523104801326101990,-3813742700509249040,-198797568561120929,1360848130483946585,-2034445041726915230,6445693522245970031] [-42294009754395695750286152418877456026522055877244789929409825751148328749462,-30114637036117781224059264373564439361653611308928312514969460679379590706382,50790691897328530213645537587874962516097580703236937570724165115560305762147,-53982023979693597997184389721196431225054557132927198244547868871851761566844,-4746897427675718862552189488292169089453556841811840545196357111533622948298,-12014200750706123405006110782843469166121588186681927916239818910819762049960] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 Tb#yV[>M*ܨ(OR8V1n)H}C\'I7tqnV)䳆qLPoRg<{3iH_m!q\'G 127.48.9.45 ['e1V10'] [{}] [825002272867.1157788721157301271303736024856710948164507982705676578804195475] [1865150610] [7514464811443271056] [33504961604882608369857530219353040639899064613284394558131808339620328539033] ڡ|A"x>rwzZ:j8tZD"Tu2h!WIytPa|\'yofFO\0Ֆ6\fIrESacW<~e lT>P3})w%4@_2N"ІXp$^ҘͰ\04@n\b\r4H 16.177.117.209 7882774382721411359365561736453116698030365959050344381263687375357052837130 c1 LowCardinality(Nullable(UInt64)), c2 Date32, c3 LowCardinality(Nullable(Float64)), c4 Int256, c5 Date32 From 53ec091c8d0b8aae36b2ee533f77b6cecc8dadf5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Jun 2023 05:00:29 +0200 Subject: [PATCH 1077/2223] Disable skim (Rust library) under memory sanitizer --- rust/skim/CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/rust/skim/CMakeLists.txt b/rust/skim/CMakeLists.txt index 1e7a43aba7c..c2e406ec12f 100644 --- a/rust/skim/CMakeLists.txt +++ b/rust/skim/CMakeLists.txt @@ -14,6 +14,11 @@ if (OS_FREEBSD) return() endif() +if (SANITIZE STREQUAL "memory") + message(STATUS "skim is disabled under memory sanitizer, because the interop is not instrumented properly") + return() +endif() + clickhouse_import_crate(MANIFEST_PATH Cargo.toml) # -Wno-dollar-in-identifier-extension: cxx bridge complies names with '$' From 60c2245da4c1cb23a7e62088b6c409afe941bd56 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Jun 2023 10:58:05 +0200 Subject: [PATCH 1078/2223] Make 01565_query_loop_after_client_error slightly more robust MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit CI founds one failure of this test [1], which is quite interesting, the test itself should not takes too long: 2023.06.04 05:01:36.889144 [ 42287 ] {} DynamicQueryHandler: Request URI: /?query=SELECT+%27Running+test+stateless%2F01565_query_loop_after_client_error.expect+from+pid%3D1053%27&database=system&connect_timeout=30&receive_timeout=30&send_timeout=30&http_connection_timeout=30&http_receive_timeout=30&http_send_timeout=30&output_format_parallel_formatting=0 ... 2023.06.04 05:01:37.364595 [ 42844 ] {fa79939f-1fa0-4f3b-b599-fd2269122d6a} executeQuery: (from [::1]:40264) (comment: 01565_query_loop_after_client_error.expect) INSERT INTO t01565(c0, c1) VALUES (stage: Complete) 2023.06.04 05:01:37.366485 [ 42844 ] {fa79939f-1fa0-4f3b-b599-fd2269122d6a} TCPHandler: Change cancellation status from NOT_CANCELLED to FULLY_CANCELLED. Log message: Received 'Cancel' packet from the client, canceling the query. 2023.06.04 05:01:37.366810 [ 42844 ] {fa79939f-1fa0-4f3b-b599-fd2269122d6a} TCPHandler: Processed in 0.002539914 sec. But for the last INSERT the client itself works **very slow**, which seems was the reason why "\r" had been sent too early and was never interpreted: send: sending "INSERT INTO t01565(c0, c1) VALUES ('1', 1) ;\n" to { exp4 } expect: does " " (spawn_id exp4) match glob pattern "INSERT"? no I expect: does " \u001b[21GI\u001b[0m\u001b[J\u001b[22G" (spawn_id exp4) match glob pattern "INSERT"? no  expect: does " \u001b[21GI\u001b[0m\u001b[J\u001b[22G\u001b[21G" (spawn_id exp4) match glob pattern "INSERT"? no INSERT INTO t expect: does " \u001b[21GI\u001b[0m\u001b[J\u001b[22G\u001b[21GINSERT INTO t\u001b[0m" (spawn_id exp4) match glob pattern "INSERT"? yes expect: set expect_out(0,string) "INSERT" expect: set expect_out(spawn_id) "exp4" expect: set expect_out(buffer) " \u001b[21GI\u001b[0m\u001b[J\u001b[22G\u001b[21GINSERT" send: sending "\r" to { exp4 } expect: does " INTO t\u001b[0m" (spawn_id exp4) match glob pattern "Ok."? no  expect: does " INTO t\u001b[0m\u001b[J" (spawn_id exp4) match glob pattern "Ok."? no  expect: does " INTO t\u001b[0m\u001b[J\u001b[34G" (spawn_id exp4) match glob pattern "Ok."? no  expect: does " INTO t\u001b[0m\u001b[J\u001b[34G\u001b[21G" (spawn_id exp4) match glob pattern "Ok."? no INSERT INTO t01565(c0, c1) VALUES ('1', 1) ;  expect: does " INTO t\u001b[0m\u001b[J\u001b[34G\u001b[21G\u001b[JINSERT INTO t01565\u001b[0;22;33m(\u001b[0mc0\u001b[0;1m,\u001b[0m c1\u001b[0;22;33m)\u001b[0m VALUES \u001b[0;22;33m(\u001b[0;22;36m'1'\u001b[0;1m,\u001b[0m \u001b[0;22;32m1\u001b[0;22;33m)\u001b[0m \u001b[0;1m;\u001b[0m\r\n\r\n\u001b[0m\u001b[1G" (spawn_id exp4) match glob pattern "Ok."? no expect: timed out Here you can see that it matched "INSERT" and then expect receive data from the client almost byte by byte. So I hope that expecting the last part of the query should fix the problem. [1]: https://s3.amazonaws.com/clickhouse-test-reports/50429/228ebab86db95dca1e29967061d245985bc86a0f/stateless_tests__release__s3_storage__[2_2].html Signed-off-by: Azat Khuzhin --- .../01565_query_loop_after_client_error.expect | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01565_query_loop_after_client_error.expect b/tests/queries/0_stateless/01565_query_loop_after_client_error.expect index 0faf8f0192b..e718fd99b7f 100755 --- a/tests/queries/0_stateless/01565_query_loop_after_client_error.expect +++ b/tests/queries/0_stateless/01565_query_loop_after_client_error.expect @@ -24,24 +24,24 @@ expect "\n:) " send -- "DROP TABLE IF EXISTS t01565;\n" # NOTE: this is important for -mn mode, you should send "\r" only after reading echoed command -expect "DROP" +expect "\r\n" send -- "\r" expect "\nOk." expect "\n:)" send -- "CREATE TABLE t01565 (c0 String, c1 Int32) ENGINE = Memory() ;\n" -expect "CREATE" +expect "\r\n" send -- "\r" expect "\nOk." expect "\n:) " send -- "INSERT INTO t01565(c0, c1) VALUES (\"1\",1) ;\n" -expect "INSERT" +expect "\r\n" send -- "\r" expect "\n:) " send -- "INSERT INTO t01565(c0, c1) VALUES ('1', 1) ;\n" -expect "INSERT" +expect "\r\n" send -- "\r" expect "\nOk." expect "\n:) " From db806bd394c7b7dfe42f225f3c1ad7b1be1f2ea9 Mon Sep 17 00:00:00 2001 From: auxten Date: Sun, 4 Jun 2023 17:44:29 +0800 Subject: [PATCH 1079/2223] Resize underlying vector only pos_offset == vector.size() --- src/IO/WriteBufferFromVector.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index 4b2a3581625..c793a34b406 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -86,7 +86,10 @@ private: size_t old_size = vector.size(); /// pos may not be equal to vector.data() + old_size, because WriteBuffer::next() can be used to flush data size_t pos_offset = pos - reinterpret_cast(vector.data()); - vector.resize(old_size * size_multiplier); + if (pos_offset == vector.size()) + { + vector.resize(old_size * size_multiplier); + } internal_buffer = Buffer(reinterpret_cast(vector.data() + pos_offset), reinterpret_cast(vector.data() + vector.size())); working_buffer = internal_buffer; } From 97bd3f048316b421966b2e4d6ced8258808fdefe Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 22 May 2023 23:25:59 +0800 Subject: [PATCH 1080/2223] Proper mutation of skip indices and projections --- src/Interpreters/MutationsInterpreter.cpp | 93 ++++- src/Interpreters/MutationsInterpreter.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 25 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 14 +- src/Storages/MergeTree/MergeTreeData.h | 6 +- .../MergeTree/MergeTreeDataWriter.cpp | 20 +- src/Storages/MergeTree/MutateTask.cpp | 366 +++++++++--------- src/Storages/StorageInMemoryMetadata.cpp | 16 +- src/Storages/StorageInMemoryMetadata.h | 9 +- ...ith_skip_indices_and_projections.reference | 0 ...part_with_skip_indices_and_projections.sql | 31 ++ 12 files changed, 359 insertions(+), 224 deletions(-) create mode 100644 tests/queries/0_stateless/02763_mutate_compact_part_with_skip_indices_and_projections.reference create mode 100644 tests/queries/0_stateless/02763_mutate_compact_part_with_skip_indices_and_projections.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 713ebade1d5..791018a3f38 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -109,13 +109,16 @@ QueryTreeNodePtr prepareQueryAffectedQueryTree(const std::vector & has_index_or_projection) { NameSet new_updated_columns = updated_columns; ColumnDependencies dependencies; while (!new_updated_columns.empty()) { - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_index_or_projection); new_updated_columns.clear(); for (const auto & dependency : new_dependencies) { @@ -288,6 +291,11 @@ bool MutationsInterpreter::Source::materializeTTLRecalculateOnly() const return data && data->getSettings()->materialize_ttl_recalculate_only; } +bool MutationsInterpreter::Source::hasIndexOrProjection(const String & file_name) const +{ + return part && part->checksums.has(file_name); +} + static Names getAvailableColumnsWithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage) { auto all_columns = metadata_snapshot->getColumns().getNamesOfPhysical(); @@ -524,8 +532,54 @@ void MutationsInterpreter::prepare(bool dry_run) validateUpdateColumns(source, metadata_snapshot, updated_columns, column_to_affected_materialized); } + for (const auto & [_, names] : column_to_affected_materialized) + updated_columns.insert(names.begin(), names.end()); + + std::function has_index_or_projection + = [&](const String & file_name) { return source.hasIndexOrProjection(file_name); }; + if (settings.recalculate_dependencies_of_updated_columns) - dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns); + dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns, has_index_or_projection); + + for (const auto & index : metadata_snapshot->getSecondaryIndices()) + { + if (source.hasIndexOrProjection("skp_idx_" + index.name + ".idx") || source.hasIndexOrProjection("skp_idx_" + index.name + ".idx2")) + { + // If some dependent columns gets mutated + bool mutate = false; + const auto & index_cols = index.expression->getRequiredColumns(); + for (const auto & col : index_cols) + { + if (updated_columns.contains(col)) + { + mutate = true; + break; + } + } + if (mutate) + materialized_indices.insert(index.name); + } + } + + for (const auto & projection : metadata_snapshot->getProjections()) + { + if (source.hasIndexOrProjection(projection.getDirectoryName())) + { + // If some dependent columns gets mutated + bool mutate = false; + const auto & projection_cols = projection.required_columns; + for (const auto & col : projection_cols) + { + if (updated_columns.contains(col)) + { + mutate = true; + break; + } + } + if (mutate) + materialized_projections.insert(projection.name); + } + } std::vector read_columns; /// First, break a sequence of commands into stages. @@ -680,20 +734,27 @@ void MutationsInterpreter::prepare(bool dry_run) if (it == std::cend(indices_desc)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown index: {}", command.index_name); - auto query = (*it).expression_list_ast->clone(); - auto syntax_result = TreeRewriter(context).analyze(query, all_columns); - const auto required_columns = syntax_result->requiredSourceColumns(); - for (const auto & column : required_columns) - dependencies.emplace(column, ColumnDependency::SKIP_INDEX); - materialized_indices.emplace(command.index_name); + if (!source.hasIndexOrProjection("skp_idx_" + it->name + ".idx") + && !source.hasIndexOrProjection("skp_idx_" + it->name + ".idx2")) + { + auto query = (*it).expression_list_ast->clone(); + auto syntax_result = TreeRewriter(context).analyze(query, all_columns); + const auto required_columns = syntax_result->requiredSourceColumns(); + for (const auto & column : required_columns) + dependencies.emplace(column, ColumnDependency::SKIP_INDEX); + materialized_indices.emplace(command.index_name); + } } else if (command.type == MutationCommand::MATERIALIZE_PROJECTION) { mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); const auto & projection = projections_desc.get(command.projection_name); - for (const auto & column : projection.required_columns) - dependencies.emplace(column, ColumnDependency::PROJECTION); - materialized_projections.emplace(command.projection_name); + if (!source.hasIndexOrProjection(projection.getDirectoryName())) + { + for (const auto & column : projection.required_columns) + dependencies.emplace(column, ColumnDependency::PROJECTION); + materialized_projections.emplace(command.projection_name); + } } else if (command.type == MutationCommand::DROP_INDEX) { @@ -712,7 +773,8 @@ void MutationsInterpreter::prepare(bool dry_run) { // just recalculate ttl_infos without remove expired data auto all_columns_vec = all_columns.getNames(); - auto new_dependencies = metadata_snapshot->getColumnDependencies(NameSet(all_columns_vec.begin(), all_columns_vec.end()), false); + auto new_dependencies = metadata_snapshot->getColumnDependencies( + NameSet(all_columns_vec.begin(), all_columns_vec.end()), false, has_index_or_projection); for (const auto & dependency : new_dependencies) { if (dependency.kind == ColumnDependency::TTL_EXPRESSION) @@ -737,7 +799,8 @@ void MutationsInterpreter::prepare(bool dry_run) } auto all_columns_vec = all_columns.getNames(); - auto all_dependencies = getAllColumnDependencies(metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end())); + auto all_dependencies = getAllColumnDependencies( + metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end()), has_index_or_projection); for (const auto & dependency : all_dependencies) { @@ -746,7 +809,7 @@ void MutationsInterpreter::prepare(bool dry_run) } /// Recalc only skip indices and projections of columns which could be updated by TTL. - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_index_or_projection); for (const auto & dependency : new_dependencies) { if (dependency.kind == ColumnDependency::SKIP_INDEX || dependency.kind == ColumnDependency::PROJECTION) diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 49ba07641d9..d783b503531 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -120,6 +120,7 @@ public: bool supportsLightweightDelete() const; bool hasLightweightDeleteMask() const; bool materializeTTLRecalculateOnly() const; + bool hasIndexOrProjection(const String & file_name) const; void read( Stage & first_stage, diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index d27b03fff44..ca814a2afd5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -632,7 +632,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks if (!parent_part) { loadTTLInfos(); - loadProjections(require_columns_checksums, check_consistency); + loadProjections(require_columns_checksums, check_consistency, false /* if_not_loaded */); } if (check_consistency) @@ -690,13 +690,13 @@ void IMergeTreeDataPart::addProjectionPart( const String & projection_name, std::shared_ptr && projection_part) { - /// Here should be a check that projection we are trying to add - /// does not exist, but unfortunately this check fails in tests. - /// TODO: fix. + if (hasProjection(projection_name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection part {} in part {} is already loaded. This is a bug", projection_name, name); + projection_parts[projection_name] = std::move(projection_part); } -void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency) +void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency, bool if_not_loaded) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); for (const auto & projection : metadata_snapshot->projections) @@ -704,9 +704,18 @@ void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool ch auto path = projection.name + ".proj"; if (getDataPartStorage().exists(path)) { - auto part = getProjectionPartBuilder(projection.name).withPartFormatFromDisk().build(); - part->loadColumnsChecksumsIndexes(require_columns_checksums, check_consistency); - addProjectionPart(projection.name, std::move(part)); + if (hasProjection(projection.name)) + { + if (!if_not_loaded) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Projection part {} in part {} is already loaded. This is a bug", projection.name, name); + } + else + { + auto part = getProjectionPartBuilder(projection.name).withPartFormatFromDisk().build(); + part->loadColumnsChecksumsIndexes(require_columns_checksums, check_consistency); + addProjectionPart(projection.name, std::move(part)); + } } } } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 411de3af982..b6b6d8c6693 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -388,7 +388,7 @@ public: bool hasProjection(const String & projection_name) const { return projection_parts.contains(projection_name); } - void loadProjections(bool require_columns_checksums, bool check_consistency); + void loadProjections(bool require_columns_checksums, bool check_consistency, bool if_not_loaded = false); /// Return set of metadata file names without checksums. For example, /// columns.txt or checksums.txt itself. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 32665429051..0115ce07b2c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -605,14 +605,14 @@ namespace ExpressionActionsPtr getCombinedIndicesExpression( const KeyDescription & key, - const IndicesDescription & indices, + const MergeTreeIndices & indices, const ColumnsDescription & columns, ContextPtr context) { ASTPtr combined_expr_list = key.expression_list_ast->clone(); for (const auto & index : indices) - for (const auto & index_expr : index.expression_list_ast->children) + for (const auto & index_expr : index->index.expression_list_ast->children) combined_expr_list->children.push_back(index_expr->clone()); auto syntax_result = TreeRewriter(context).analyze(combined_expr_list, columns.getAllPhysical()); @@ -644,14 +644,16 @@ DataTypes MergeTreeData::getMinMaxColumnsTypes(const KeyDescription & partition_ return {}; } -ExpressionActionsPtr MergeTreeData::getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const +ExpressionActionsPtr +MergeTreeData::getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot, const MergeTreeIndices & indices) const { - return getCombinedIndicesExpression(metadata_snapshot->getPrimaryKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), getContext()); + return getCombinedIndicesExpression(metadata_snapshot->getPrimaryKey(), indices, metadata_snapshot->getColumns(), getContext()); } -ExpressionActionsPtr MergeTreeData::getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const +ExpressionActionsPtr +MergeTreeData::getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot, const MergeTreeIndices & indices) const { - return getCombinedIndicesExpression(metadata_snapshot->getSortingKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), getContext()); + return getCombinedIndicesExpression(metadata_snapshot->getSortingKey(), indices, metadata_snapshot->getColumns(), getContext()); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 1c41de6fa19..6fd9d223f32 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -956,8 +956,10 @@ public: /// Get column types required for partition key static DataTypes getMinMaxColumnsTypes(const KeyDescription & partition_key); - ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; - ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; + ExpressionActionsPtr + getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot, const MergeTreeIndices & indices) const; + ExpressionActionsPtr + getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot, const MergeTreeIndices & indices) const; /// Get compression codec for part according to TTL rules and /// section from config.xml. diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index db486b163eb..6ff4d6be870 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -398,9 +398,11 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( temp_part.temporary_directory_lock = data.getTemporaryPartDirectoryHolder(part_dir); + auto indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices()); + /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) - data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot)->execute(block); + data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot, indices)->execute(block); Names sort_columns = metadata_snapshot->getSortingKeyColumns(); SortDescription sort_description; @@ -517,10 +519,16 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); - const auto & index_factory = MergeTreeIndexFactory::instance(); - auto out = std::make_unique(new_data_part, metadata_snapshot, columns, - index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, - context->getCurrentTransaction(), false, false, context->getWriteSettings()); + auto out = std::make_unique( + new_data_part, + metadata_snapshot, + columns, + indices, + compression_codec, + context->getCurrentTransaction(), + false, + false, + context->getWriteSettings()); out->writeWithPermutation(block, perm_ptr); @@ -606,7 +614,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) - data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot)->execute(block); + data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot, {})->execute(block); Names sort_columns = metadata_snapshot->getSortingKeyColumns(); SortDescription sort_description; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 76096d00641..d65897ac97d 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -200,8 +200,7 @@ static void splitAndModifyMutationCommands( { for_file_renames.push_back(command); } - /// If we don't have this column in source part, than we don't need - /// to materialize it + /// If we don't have this column in source part, we don't need to materialize it. else if (part_columns.has(command.column_name)) { if (command.type == MutationCommand::Type::READ_COLUMN) @@ -438,51 +437,13 @@ static ExecuteTTLType shouldExecuteTTL(const StorageMetadataPtr & metadata_snaps } -/// Get skip indices, that should exists in the resulting data part. -static MergeTreeIndices getIndicesForNewDataPart( - const IndicesDescription & all_indices, - const MutationCommands & commands_for_removes) -{ - NameSet removed_indices; - for (const auto & command : commands_for_removes) - if (command.type == MutationCommand::DROP_INDEX) - removed_indices.insert(command.column_name); - - MergeTreeIndices new_indices; - for (const auto & index : all_indices) - if (!removed_indices.contains(index.name)) - new_indices.push_back(MergeTreeIndexFactory::instance().get(index)); - - return new_indices; -} - -static std::vector getProjectionsForNewDataPart( - const ProjectionsDescription & all_projections, - const MutationCommands & commands_for_removes) -{ - NameSet removed_projections; - for (const auto & command : commands_for_removes) - if (command.type == MutationCommand::DROP_PROJECTION) - removed_projections.insert(command.column_name); - - std::vector new_projections; - for (const auto & projection : all_projections) - if (!removed_projections.contains(projection.name)) - new_projections.push_back(&projection); - - return new_projections; -} - - /// Return set of indices which should be recalculated during mutation also /// wraps input stream into additional expression stream static std::set getIndicesToRecalculate( QueryPipelineBuilder & builder, - const NameSet & updated_columns, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, - const NameSet & materialized_indices, - const MergeTreeData::DataPartPtr & source_part) + const NameSet & materialized_indices) { /// Checks if columns used in skipping indexes modified. const auto & index_factory = MergeTreeIndexFactory::instance(); @@ -492,11 +453,7 @@ static std::set getIndicesToRecalculate( for (const auto & index : indices) { - bool has_index = - source_part->checksums.has(INDEX_FILE_PREFIX + index.name + ".idx") || - source_part->checksums.has(INDEX_FILE_PREFIX + index.name + ".idx2"); - // If we ask to materialize and it already exists - if (!has_index && materialized_indices.contains(index.name)) + if (materialized_indices.contains(index.name)) { if (indices_to_recalc.insert(index_factory.get(index)).second) { @@ -505,26 +462,6 @@ static std::set getIndicesToRecalculate( indices_recalc_expr_list->children.push_back(expr->clone()); } } - // If some dependent columns gets mutated - else - { - bool mutate = false; - const auto & index_cols = index.expression->getRequiredColumns(); - for (const auto & col : index_cols) - { - if (updated_columns.contains(col)) - { - mutate = true; - break; - } - } - if (mutate && indices_to_recalc.insert(index_factory.get(index)).second) - { - ASTPtr expr_list = index.expression_list_ast->clone(); - for (const auto & expr : expr_list->children) - indices_recalc_expr_list->children.push_back(expr->clone()); - } - } } if (!indices_to_recalc.empty() && builder.initialized()) @@ -545,37 +482,15 @@ static std::set getIndicesToRecalculate( return indices_to_recalc; } -std::set getProjectionsToRecalculate( - const NameSet & updated_columns, +static std::set getProjectionsToRecalculate( const StorageMetadataPtr & metadata_snapshot, - const NameSet & materialized_projections, - const MergeTreeData::DataPartPtr & source_part) + const NameSet & materialized_projections) { - /// Checks if columns used in projections modified. std::set projections_to_recalc; for (const auto & projection : metadata_snapshot->getProjections()) { - // If we ask to materialize and it doesn't exist - if (!source_part->checksums.has(projection.name + ".proj") && materialized_projections.contains(projection.name)) - { + if (materialized_projections.contains(projection.name)) projections_to_recalc.insert(&projection); - } - else - { - // If some dependent columns gets mutated - bool mutate = false; - const auto & projection_cols = projection.required_columns; - for (const auto & col : projection_cols) - { - if (updated_columns.contains(col)) - { - mutate = true; - break; - } - } - if (mutate) - projections_to_recalc.insert(&projection); - } } return projections_to_recalc; } @@ -618,33 +533,6 @@ static NameSet collectFilesToSkip( /// Do not hardlink this file because it's always rewritten at the end of mutation. files_to_skip.insert(IMergeTreeDataPart::SERIALIZATION_FILE_NAME); - auto new_stream_counts = getStreamCounts(new_part, new_part->getColumns().getNames()); - auto source_updated_stream_counts = getStreamCounts(source_part, updated_header.getNames()); - auto new_updated_stream_counts = getStreamCounts(new_part, updated_header.getNames()); - - /// Skip all modified files in new part. - for (const auto & [stream_name, _] : new_updated_stream_counts) - { - files_to_skip.insert(stream_name + ".bin"); - files_to_skip.insert(stream_name + mrk_extension); - } - - /// Skip files that we read from source part and do not write in new part. - /// E.g. ALTER MODIFY from LowCardinality(String) to String. - for (const auto & [stream_name, _] : source_updated_stream_counts) - { - /// If we read shared stream and do not write it - /// (e.g. while ALTER MODIFY COLUMN from array of Nested type to String), - /// we need to hardlink its files, because they will be lost otherwise. - bool need_hardlink = new_updated_stream_counts[stream_name] == 0 && new_stream_counts[stream_name] != 0; - - if (!need_hardlink) - { - files_to_skip.insert(stream_name + ".bin"); - files_to_skip.insert(stream_name + mrk_extension); - } - } - for (const auto & index : indices_to_recalc) { /// Since MinMax index has .idx2 extension, we need to add correct extension. @@ -655,6 +543,36 @@ static NameSet collectFilesToSkip( for (const auto & projection : projections_to_recalc) files_to_skip.insert(projection->getDirectoryName()); + if (isWidePart(source_part)) + { + auto new_stream_counts = getStreamCounts(new_part, new_part->getColumns().getNames()); + auto source_updated_stream_counts = getStreamCounts(source_part, updated_header.getNames()); + auto new_updated_stream_counts = getStreamCounts(new_part, updated_header.getNames()); + + /// Skip all modified files in new part. + for (const auto & [stream_name, _] : new_updated_stream_counts) + { + files_to_skip.insert(stream_name + ".bin"); + files_to_skip.insert(stream_name + mrk_extension); + } + + /// Skip files that we read from source part and do not write in new part. + /// E.g. ALTER MODIFY from LowCardinality(String) to String. + for (const auto & [stream_name, _] : source_updated_stream_counts) + { + /// If we read shared stream and do not write it + /// (e.g. while ALTER MODIFY COLUMN from array of Nested type to String), + /// we need to hardlink its files, because they will be lost otherwise. + bool need_hardlink = new_updated_stream_counts[stream_name] == 0 && new_stream_counts[stream_name] != 0; + + if (!need_hardlink) + { + files_to_skip.insert(stream_name + ".bin"); + files_to_skip.insert(stream_name + mrk_extension); + } + } + } + return files_to_skip; } @@ -701,57 +619,60 @@ static NameToNameVector collectFilesForRenames( if (source_part->checksums.has(command.column_name + ".proj")) add_rename(command.column_name + ".proj", ""); } - else if (command.type == MutationCommand::Type::DROP_COLUMN) + else if (isWidePart(source_part)) { - ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) + if (command.type == MutationCommand::Type::DROP_COLUMN) { - String stream_name = ISerialization::getFileNameForStream({command.column_name, command.data_type}, substream_path); - /// Delete files if they are no longer shared with another column. - if (--stream_counts[stream_name] == 0) + ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) { - add_rename(stream_name + ".bin", ""); - add_rename(stream_name + mrk_extension, ""); - } - }; + String stream_name = ISerialization::getFileNameForStream({command.column_name, command.data_type}, substream_path); + /// Delete files if they are no longer shared with another column. + if (--stream_counts[stream_name] == 0) + { + add_rename(stream_name + ".bin", ""); + add_rename(stream_name + mrk_extension, ""); + } + }; - if (auto serialization = source_part->tryGetSerialization(command.column_name)) - serialization->enumerateStreams(callback); - } - else if (command.type == MutationCommand::Type::RENAME_COLUMN) - { - String escaped_name_from = escapeForFileName(command.column_name); - String escaped_name_to = escapeForFileName(command.rename_to); - - ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) + if (auto serialization = source_part->tryGetSerialization(command.column_name)) + serialization->enumerateStreams(callback); + } + else if (command.type == MutationCommand::Type::RENAME_COLUMN) { - String stream_from = ISerialization::getFileNameForStream(command.column_name, substream_path); - String stream_to = boost::replace_first_copy(stream_from, escaped_name_from, escaped_name_to); + String escaped_name_from = escapeForFileName(command.column_name); + String escaped_name_to = escapeForFileName(command.rename_to); - if (stream_from != stream_to) + ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) { - add_rename(stream_from + ".bin", stream_to + ".bin"); - add_rename(stream_from + mrk_extension, stream_to + mrk_extension); - } - }; + String stream_from = ISerialization::getFileNameForStream(command.column_name, substream_path); + String stream_to = boost::replace_first_copy(stream_from, escaped_name_from, escaped_name_to); - if (auto serialization = source_part->tryGetSerialization(command.column_name)) - serialization->enumerateStreams(callback); - } - else if (command.type == MutationCommand::Type::READ_COLUMN) - { - /// Remove files for streams that exist in source_part, - /// but were removed in new_part by MODIFY COLUMN from - /// type with higher number of streams (e.g. LowCardinality -> String). + if (stream_from != stream_to) + { + add_rename(stream_from + ".bin", stream_to + ".bin"); + add_rename(stream_from + mrk_extension, stream_to + mrk_extension); + } + }; - auto old_streams = getStreamCounts(source_part, source_part->getColumns().getNames()); - auto new_streams = getStreamCounts(new_part, source_part->getColumns().getNames()); - - for (const auto & [old_stream, _] : old_streams) + if (auto serialization = source_part->tryGetSerialization(command.column_name)) + serialization->enumerateStreams(callback); + } + else if (command.type == MutationCommand::Type::READ_COLUMN) { - if (!new_streams.contains(old_stream) && --stream_counts[old_stream] == 0) + /// Remove files for streams that exist in source_part, + /// but were removed in new_part by MODIFY COLUMN from + /// type with higher number of streams (e.g. LowCardinality -> String). + + auto old_streams = getStreamCounts(source_part, source_part->getColumns().getNames()); + auto new_streams = getStreamCounts(new_part, source_part->getColumns().getNames()); + + for (const auto & [old_stream, _] : old_streams) { - add_rename(old_stream + ".bin", ""); - add_rename(old_stream + mrk_extension, ""); + if (!new_streams.contains(old_stream) && --stream_counts[old_stream] == 0) + { + add_rename(old_stream + ".bin", ""); + add_rename(old_stream + mrk_extension, ""); + } } } } @@ -851,11 +772,8 @@ void finalizeMutatedPart( new_data_part->minmax_idx = source_part->minmax_idx; new_data_part->modification_time = time(nullptr); - /// This line should not be here because at that moment - /// of executing of mutation all projections should be loaded. - /// But unfortunately without it some tests fail. - /// TODO: fix. - new_data_part->loadProjections(false, false); + /// Load rest projections which are hardlinked + new_data_part->loadProjections(false, false, true /* if_not_loaded */); /// All information about sizes is stored in checksums. /// It doesn't make sense to touch filesystem for sizes. @@ -917,9 +835,9 @@ struct MutationContext std::vector projections_to_build; IMergeTreeDataPart::MinMaxIndexPtr minmax_idx{nullptr}; - NameSet updated_columns; std::set indices_to_recalc; std::set projections_to_recalc; + MergeTreeData::DataPart::Checksums existing_indices_checksums; NameSet files_to_skip; NameToNameVector files_to_rename; @@ -1331,10 +1249,102 @@ private: /// (which is locked in shared mode when input streams are created) and when inserting new data /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus /// deadlock is impossible. - ctx->compression_codec = ctx->data->getCompressionCodecForPart(ctx->source_part->getBytesOnDisk(), ctx->source_part->ttl_infos, ctx->time_of_mutation); + ctx->compression_codec + = ctx->data->getCompressionCodecForPart(ctx->source_part->getBytesOnDisk(), ctx->source_part->ttl_infos, ctx->time_of_mutation); - auto skip_part_indices = MutationHelpers::getIndicesForNewDataPart(ctx->metadata_snapshot->getSecondaryIndices(), ctx->for_file_renames); - ctx->projections_to_build = MutationHelpers::getProjectionsForNewDataPart(ctx->metadata_snapshot->getProjections(), ctx->for_file_renames); + NameSet entries_to_hardlink; + + NameSet removed_indices; + for (const auto & command : ctx->for_file_renames) + { + if (command.type == MutationCommand::DROP_INDEX) + removed_indices.insert(command.column_name); + } + + const auto & indices = ctx->metadata_snapshot->getSecondaryIndices(); + MergeTreeIndices skip_indices; + for (const auto & idx : indices) + { + if (removed_indices.contains(idx.name)) + continue; + + if (ctx->materialized_indices.contains(idx.name)) + skip_indices.push_back(MergeTreeIndexFactory::instance().get(idx)); + + auto hardlink_index = [&](const String & idx_name) + { + if (ctx->source_part->checksums.has(idx_name)) + { + auto it = ctx->source_part->checksums.files.find(idx_name); + if (it != ctx->source_part->checksums.files.end()) + { + entries_to_hardlink.insert(idx_name); + ctx->existing_indices_checksums.addFile(idx_name, it->second.file_size, it->second.file_hash); + } + } + }; + hardlink_index(INDEX_FILE_PREFIX + idx.name + ".idx"); + hardlink_index(INDEX_FILE_PREFIX + idx.name + ".idx2"); + } + + NameSet removed_projections; + for (const auto & command : ctx->for_file_renames) + { + if (command.type == MutationCommand::DROP_PROJECTION) + removed_projections.insert(command.column_name); + } + + const auto & projections = ctx->metadata_snapshot->getProjections(); + for (const auto & projection : projections) + { + if (removed_projections.contains(projection.name)) + continue; + + if (ctx->materialized_projections.contains(projection.name)) + ctx->projections_to_build.push_back(&projection); + + if (ctx->source_part->checksums.has(projection.getDirectoryName())) + entries_to_hardlink.insert(projection.getDirectoryName()); + } + + NameSet hardlinked_files; + /// Create hardlinks for unchanged files + for (auto it = ctx->source_part->getDataPartStorage().iterate(); it->isValid(); it->next()) + { + if (!entries_to_hardlink.contains(it->name())) + continue; + + if (it->isFile()) + { + ctx->new_data_part->getDataPartStorage().createHardLinkFrom( + ctx->source_part->getDataPartStorage(), it->name(), it->name()); + hardlinked_files.insert(it->name()); + } + else + { + // it's a projection part directory + ctx->new_data_part->getDataPartStorage().createProjection(it->name()); + + auto projection_data_part_storage_src = ctx->source_part->getDataPartStorage().getProjection(it->name()); + auto projection_data_part_storage_dst = ctx->new_data_part->getDataPartStorage().getProjection(it->name()); + + for (auto p_it = projection_data_part_storage_src->iterate(); p_it->isValid(); p_it->next()) + { + projection_data_part_storage_dst->createHardLinkFrom( + *projection_data_part_storage_src, p_it->name(), p_it->name()); + + auto file_name_with_projection_prefix = fs::path(projection_data_part_storage_src->getPartDirectory()) / p_it->name(); + hardlinked_files.insert(file_name_with_projection_prefix); + } + } + } + + /// Tracking of hardlinked files required for zero-copy replication. + /// We don't remove them when we delete last copy of source part because + /// new part can use them. + ctx->hardlinked_files.source_table_shared_id = ctx->source_part->storage.getTableSharedID(); + ctx->hardlinked_files.source_part_name = ctx->source_part->name; + ctx->hardlinked_files.hardlinks_from_source_part = std::move(hardlinked_files); if (!ctx->mutating_pipeline_builder.initialized()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot mutate part columns with uninitialized mutations stream. It's a bug"); @@ -1343,8 +1353,8 @@ private: if (ctx->metadata_snapshot->hasPrimaryKey() || ctx->metadata_snapshot->hasSecondaryIndices()) { - builder.addTransform( - std::make_shared(builder.getHeader(), ctx->data->getPrimaryKeyAndSkipIndicesExpression(ctx->metadata_snapshot))); + builder.addTransform(std::make_shared( + builder.getHeader(), ctx->data->getPrimaryKeyAndSkipIndicesExpression(ctx->metadata_snapshot, skip_indices))); builder.addTransform(std::make_shared(builder.getHeader())); } @@ -1361,7 +1371,7 @@ private: ctx->new_data_part, ctx->metadata_snapshot, ctx->new_data_part->getColumns(), - skip_part_indices, + skip_indices, ctx->compression_codec, ctx->txn, /*reset_columns=*/ true, @@ -1381,10 +1391,12 @@ private: void finalize() { ctx->new_data_part->minmax_idx = std::move(ctx->minmax_idx); + ctx->new_data_part->loadProjections(false, false, true /* if_not_loaded */); ctx->mutating_executor.reset(); ctx->mutating_pipeline.reset(); - static_pointer_cast(ctx->out)->finalizePart(ctx->new_data_part, ctx->need_sync); + static_pointer_cast(ctx->out)->finalizePart( + ctx->new_data_part, ctx->need_sync, nullptr, &ctx->existing_indices_checksums); ctx->out.reset(); } @@ -1530,7 +1542,7 @@ private: /// new part can use them. ctx->hardlinked_files.source_table_shared_id = ctx->source_part->storage.getTableSharedID(); ctx->hardlinked_files.source_part_name = ctx->source_part->name; - ctx->hardlinked_files.hardlinks_from_source_part = hardlinked_files; + ctx->hardlinked_files.hardlinks_from_source_part = std::move(hardlinked_files); (*ctx->mutate_entry)->columns_written = ctx->storage_columns.size() - ctx->updated_header.columns(); @@ -1878,14 +1890,10 @@ bool MutateTask::prepare() } else /// TODO: check that we modify only non-key columns in this case. { - /// We will modify only some of the columns. Other columns and key values can be copied as-is. - for (const auto & name_type : ctx->updated_header.getNamesAndTypesList()) - ctx->updated_columns.emplace(name_type.name); - ctx->indices_to_recalc = MutationHelpers::getIndicesToRecalculate( - ctx->mutating_pipeline_builder, ctx->updated_columns, ctx->metadata_snapshot, ctx->context, ctx->materialized_indices, ctx->source_part); - ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( - ctx->updated_columns, ctx->metadata_snapshot, ctx->materialized_projections, ctx->source_part); + ctx->mutating_pipeline_builder, ctx->metadata_snapshot, ctx->context, ctx->materialized_indices); + + ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate(ctx->metadata_snapshot, ctx->materialized_projections); ctx->files_to_skip = MutationHelpers::collectFilesToSkip( ctx->source_part, diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 45abd4bebef..afe75349864 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -236,7 +236,10 @@ bool StorageInMemoryMetadata::hasAnyGroupByTTL() const return !table_ttl.group_by_ttl.empty(); } -ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet & updated_columns, bool include_ttl_target) const +ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( + const NameSet & updated_columns, + bool include_ttl_target, + const std::function & has_indice_or_projection) const { if (updated_columns.empty()) return {}; @@ -264,10 +267,16 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet }; for (const auto & index : getSecondaryIndices()) - add_dependent_columns(index.expression, indices_columns); + { + if (has_indice_or_projection("skp_idx_" + index.name + ".idx") || has_indice_or_projection("skp_idx_" + index.name + ".idx2")) + add_dependent_columns(index.expression, indices_columns); + } for (const auto & projection : getProjections()) - add_dependent_columns(&projection, projections_columns); + { + if (has_indice_or_projection(projection.getDirectoryName())) + add_dependent_columns(&projection, projections_columns); + } auto add_for_rows_ttl = [&](const auto & expression, auto & to_set) { @@ -312,7 +321,6 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet res.emplace(column, ColumnDependency::TTL_TARGET); return res; - } Block StorageInMemoryMetadata::getSampleBlockInsertable() const diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 25618c5b03f..4ed7eb8bf29 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -147,9 +147,12 @@ struct StorageInMemoryMetadata TTLDescriptions getGroupByTTLs() const; bool hasAnyGroupByTTL() const; - /// Returns columns, which will be needed to calculate dependencies (skip - /// indices, TTL expressions) if we update @updated_columns set of columns. - ColumnDependencies getColumnDependencies(const NameSet & updated_columns, bool include_ttl_target) const; + /// Returns columns, which will be needed to calculate dependencies (skip indices, projections, + /// TTL expressions) if we update @updated_columns set of columns. + ColumnDependencies getColumnDependencies( + const NameSet & updated_columns, + bool include_ttl_target, + const std::function & has_indice_or_projection) const; /// Block with ordinary + materialized columns. Block getSampleBlock() const; diff --git a/tests/queries/0_stateless/02763_mutate_compact_part_with_skip_indices_and_projections.reference b/tests/queries/0_stateless/02763_mutate_compact_part_with_skip_indices_and_projections.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02763_mutate_compact_part_with_skip_indices_and_projections.sql b/tests/queries/0_stateless/02763_mutate_compact_part_with_skip_indices_and_projections.sql new file mode 100644 index 00000000000..bb9825fe5a0 --- /dev/null +++ b/tests/queries/0_stateless/02763_mutate_compact_part_with_skip_indices_and_projections.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test ( col1 Int64, dt Date ) ENGINE = MergeTree PARTITION BY dt ORDER BY tuple(); + +INSERT INTO test FORMAT Values (1, today()); + +ALTER TABLE test ADD COLUMN col2 String; + +ALTER TABLE test ADD INDEX i1 (col1, col2) TYPE set(100) GRANULARITY 1; + +ALTER TABLE test MATERIALIZE INDEX i1; + +ALTER TABLE test ADD COLUMN col3 String; + +ALTER TABLE test DROP COLUMN col3; + +DROP TABLE IF EXISTS test; + +CREATE TABLE test ( col1 Int64, dt Date ) ENGINE = MergeTree PARTITION BY dt ORDER BY tuple(); + +INSERT INTO test FORMAT Values (1, today()); + +ALTER TABLE test ADD COLUMN col2 String; + +ALTER TABLE test ADD PROJECTION p1 ( SELECT col2, sum(col1) GROUP BY col2 ); + +ALTER TABLE test MATERIALIZE PROJECTION p1; + +ALTER TABLE test ADD COLUMN col3 String; + +ALTER TABLE test DROP COLUMN col3; From 92b2200c55f27eefceb610535e813e62bd49ffce Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 23 May 2023 20:46:50 +0800 Subject: [PATCH 1081/2223] mutation stages can be empty --- src/Interpreters/MutationsInterpreter.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 791018a3f38..1059e1fdae5 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -901,6 +901,11 @@ void MutationsInterpreter::prepare(bool dry_run) } } + /// Stages might be empty when we materialize skip indices or projections which don't add any + /// column dependencies. + if (stages.empty()) + stages.emplace_back(context); + is_prepared = true; prepareMutationStages(stages, dry_run); } From 24e015b961189e1b2202cfdf5fefe6c9e5904e71 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 4 Jun 2023 13:34:20 +0200 Subject: [PATCH 1082/2223] Fix --- src/Interpreters/Cache/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 8a24a4fe5ee..7afd17d32d1 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -650,7 +650,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) } ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->getDownloadedSize(false)); locked_key.removeFileSegment(segment->offset(), segment->lock()); return PriorityIterationResult::REMOVE_AND_CONTINUE; From 46cbdeeb7e7975b25de35fb75912da3a7ece21ec Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 4 Jun 2023 15:02:46 +0300 Subject: [PATCH 1083/2223] Fixed tests --- src/Functions/if.cpp | 6 ++---- ...olding.reference => 02771_if_constant_folding.reference} | 0 ...f_constant_folding.sql => 02771_if_constant_folding.sql} | 0 3 files changed, 2 insertions(+), 4 deletions(-) rename tests/queries/0_stateless/{25337_if_constant_folding.reference => 02771_if_constant_folding.reference} (100%) rename tests/queries/0_stateless/{25337_if_constant_folding.sql => 02771_if_constant_folding.sql} (100%) diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index d00e83c4eb7..8d43b3a4ca3 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -1120,9 +1120,8 @@ public: ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override { const ColumnWithTypeAndName & arg_cond = arguments[0]; - if (!arg_cond.column || !isColumnConst(*arg_cond.column)) { + if (!arg_cond.column || !isColumnConst(*arg_cond.column)) return {}; - } const ColumnConst * cond_const_col = checkAndGetColumnConst>(arg_cond.column.get()); bool condition_value = cond_const_col->getValue(); @@ -1135,9 +1134,8 @@ public: return {}; auto result = castColumn(potential_const_column, result_type); - if (!isColumnConst(*result)) { + if (!isColumnConst(*result)) return {}; - } return result; } diff --git a/tests/queries/0_stateless/25337_if_constant_folding.reference b/tests/queries/0_stateless/02771_if_constant_folding.reference similarity index 100% rename from tests/queries/0_stateless/25337_if_constant_folding.reference rename to tests/queries/0_stateless/02771_if_constant_folding.reference diff --git a/tests/queries/0_stateless/25337_if_constant_folding.sql b/tests/queries/0_stateless/02771_if_constant_folding.sql similarity index 100% rename from tests/queries/0_stateless/25337_if_constant_folding.sql rename to tests/queries/0_stateless/02771_if_constant_folding.sql From e24c9267bc12026ff3acde78675464a17f6d5cc1 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 4 Jun 2023 20:06:27 +0800 Subject: [PATCH 1084/2223] fix --- src/Interpreters/MutationsInterpreter.cpp | 68 +++++++++-------------- src/Storages/MergeTree/MutateTask.cpp | 39 +++++++------ 2 files changed, 47 insertions(+), 60 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 1059e1fdae5..25bb3fc5e82 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -532,55 +532,12 @@ void MutationsInterpreter::prepare(bool dry_run) validateUpdateColumns(source, metadata_snapshot, updated_columns, column_to_affected_materialized); } - for (const auto & [_, names] : column_to_affected_materialized) - updated_columns.insert(names.begin(), names.end()); - std::function has_index_or_projection = [&](const String & file_name) { return source.hasIndexOrProjection(file_name); }; if (settings.recalculate_dependencies_of_updated_columns) dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns, has_index_or_projection); - for (const auto & index : metadata_snapshot->getSecondaryIndices()) - { - if (source.hasIndexOrProjection("skp_idx_" + index.name + ".idx") || source.hasIndexOrProjection("skp_idx_" + index.name + ".idx2")) - { - // If some dependent columns gets mutated - bool mutate = false; - const auto & index_cols = index.expression->getRequiredColumns(); - for (const auto & col : index_cols) - { - if (updated_columns.contains(col)) - { - mutate = true; - break; - } - } - if (mutate) - materialized_indices.insert(index.name); - } - } - - for (const auto & projection : metadata_snapshot->getProjections()) - { - if (source.hasIndexOrProjection(projection.getDirectoryName())) - { - // If some dependent columns gets mutated - bool mutate = false; - const auto & projection_cols = projection.required_columns; - for (const auto & col : projection_cols) - { - if (updated_columns.contains(col)) - { - mutate = true; - break; - } - } - if (mutate) - materialized_projections.insert(projection.name); - } - } - std::vector read_columns; /// First, break a sequence of commands into stages. for (auto & command : commands) @@ -869,6 +826,31 @@ void MutationsInterpreter::prepare(bool dry_run) for (const auto & column : changed_columns) stages.back().column_to_updated.emplace( column, std::make_shared(column)); + + for (const auto & index : metadata_snapshot->getSecondaryIndices()) + { + if (source.hasIndexOrProjection("skp_idx_" + index.name + ".idx") + || source.hasIndexOrProjection("skp_idx_" + index.name + ".idx2")) + { + const auto & index_cols = index.expression->getRequiredColumns(); + bool changed = std::any_of( + index_cols.begin(), index_cols.end(), [&](const auto & col) { return changed_columns.contains(col); }); + if (changed) + materialized_indices.insert(index.name); + } + } + + for (const auto & projection : metadata_snapshot->getProjections()) + { + if (source.hasIndexOrProjection(projection.getDirectoryName())) + { + const auto & projection_cols = projection.required_columns; + bool changed = std::any_of( + projection_cols.begin(), projection_cols.end(), [&](const auto & col) { return changed_columns.contains(col); }); + if (changed) + materialized_projections.insert(projection.name); + } + } } if (!unchanged_columns.empty()) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index d65897ac97d..7031027002d 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1269,22 +1269,23 @@ private: continue; if (ctx->materialized_indices.contains(idx.name)) - skip_indices.push_back(MergeTreeIndexFactory::instance().get(idx)); - - auto hardlink_index = [&](const String & idx_name) { - if (ctx->source_part->checksums.has(idx_name)) + skip_indices.push_back(MergeTreeIndexFactory::instance().get(idx)); + } + else + { + auto prefix = fmt::format("{}{}.", INDEX_FILE_PREFIX, idx.name); + auto it = ctx->source_part->checksums.files.upper_bound(prefix); + while (it != ctx->source_part->checksums.files.end()) { - auto it = ctx->source_part->checksums.files.find(idx_name); - if (it != ctx->source_part->checksums.files.end()) - { - entries_to_hardlink.insert(idx_name); - ctx->existing_indices_checksums.addFile(idx_name, it->second.file_size, it->second.file_hash); - } + if (!startsWith(it->first, prefix)) + break; + + entries_to_hardlink.insert(it->first); + ctx->existing_indices_checksums.addFile(it->first, it->second.file_size, it->second.file_hash); + ++it; } - }; - hardlink_index(INDEX_FILE_PREFIX + idx.name + ".idx"); - hardlink_index(INDEX_FILE_PREFIX + idx.name + ".idx2"); + } } NameSet removed_projections; @@ -1301,10 +1302,14 @@ private: continue; if (ctx->materialized_projections.contains(projection.name)) + { ctx->projections_to_build.push_back(&projection); - - if (ctx->source_part->checksums.has(projection.getDirectoryName())) - entries_to_hardlink.insert(projection.getDirectoryName()); + } + else + { + if (ctx->source_part->checksums.has(projection.getDirectoryName())) + entries_to_hardlink.insert(projection.getDirectoryName()); + } } NameSet hardlinked_files; @@ -1354,7 +1359,7 @@ private: if (ctx->metadata_snapshot->hasPrimaryKey() || ctx->metadata_snapshot->hasSecondaryIndices()) { builder.addTransform(std::make_shared( - builder.getHeader(), ctx->data->getPrimaryKeyAndSkipIndicesExpression(ctx->metadata_snapshot, skip_indices))); + builder.getHeader(), ctx->data->getPrimaryKeyAndSkipIndicesExpression(ctx->metadata_snapshot, skip_indices))); builder.addTransform(std::make_shared(builder.getHeader())); } From 63eab2783e4e0c23fdbe3e5e233b4e3c0773f40a Mon Sep 17 00:00:00 2001 From: Sergey Kazmin <43613813+yerseg@users.noreply.github.com> Date: Fri, 2 Jun 2023 17:06:19 +0000 Subject: [PATCH 1085/2223] Make typeid_cast for pointers noexcept --- src/Common/typeid_cast.h | 38 ++++++++++++-------------------------- 1 file changed, 12 insertions(+), 26 deletions(-) diff --git a/src/Common/typeid_cast.h b/src/Common/typeid_cast.h index baee3aaf632..e134aac09e4 100644 --- a/src/Common/typeid_cast.h +++ b/src/Common/typeid_cast.h @@ -25,14 +25,14 @@ namespace DB */ template requires std::is_reference_v -To typeid_cast(From & from) +To typeid_cast(From & from) noexcept(false) { try { if ((typeid(From) == typeid(To)) || (typeid(from) == typeid(To))) return static_cast(from); } - catch (const std::exception & e) + catch (const std::bad_typeid & e) { throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR); } @@ -44,19 +44,12 @@ To typeid_cast(From & from) template requires std::is_pointer_v -To typeid_cast(From * from) +To typeid_cast(From * from) noexcept { - try - { - if ((typeid(From) == typeid(std::remove_pointer_t)) || (from && typeid(*from) == typeid(std::remove_pointer_t))) - return static_cast(from); - else - return nullptr; - } - catch (const std::exception & e) - { - throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR); - } + if ((typeid(From) == typeid(std::remove_pointer_t)) || (from && typeid(*from) == typeid(std::remove_pointer_t))) + return static_cast(from); + else + return nullptr; } namespace detail @@ -79,17 +72,10 @@ inline constexpr bool is_shared_ptr_v = is_shared_ptr::value; template requires detail::is_shared_ptr_v -To typeid_cast(const std::shared_ptr & from) +To typeid_cast(const std::shared_ptr & from) noexcept { - try - { - if ((typeid(From) == typeid(typename To::element_type)) || (from && typeid(*from) == typeid(typename To::element_type))) - return std::static_pointer_cast(from); - else - return nullptr; - } - catch (const std::exception & e) - { - throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR); - } + if ((typeid(From) == typeid(typename To::element_type)) || (from && typeid(*from) == typeid(typename To::element_type))) + return std::static_pointer_cast(from); + else + return nullptr; } From a3a12834671c95914103e59c24cf68fadc40f68f Mon Sep 17 00:00:00 2001 From: Sergey Kazmin <43613813+yerseg@users.noreply.github.com> Date: Sat, 3 Jun 2023 19:36:11 +0300 Subject: [PATCH 1086/2223] remove try-catch from the impl of typeid_cast for refs --- src/Common/typeid_cast.h | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/src/Common/typeid_cast.h b/src/Common/typeid_cast.h index e134aac09e4..f1ded97a9f1 100644 --- a/src/Common/typeid_cast.h +++ b/src/Common/typeid_cast.h @@ -27,15 +27,8 @@ template requires std::is_reference_v To typeid_cast(From & from) noexcept(false) { - try - { - if ((typeid(From) == typeid(To)) || (typeid(from) == typeid(To))) - return static_cast(from); - } - catch (const std::bad_typeid & e) - { - throw DB::Exception::createDeprecated(e.what(), DB::ErrorCodes::LOGICAL_ERROR); - } + if ((typeid(From) == typeid(To)) || (typeid(from) == typeid(To))) + return static_cast(from); throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Bad cast from type {} to {}", demangle(typeid(from).name()), demangle(typeid(To).name())); From 0f4dd26cebbcf9201124287e8d31acda92a9e9f7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 4 Jun 2023 16:03:44 +0200 Subject: [PATCH 1087/2223] Add async iteration to object storage --- src/Common/CurrentMetrics.cpp | 4 + .../AzureBlobStorage/AzureObjectStorage.cpp | 69 ++++++++++++++++++ .../AzureBlobStorage/AzureObjectStorage.h | 2 + src/Disks/ObjectStorages/IObjectStorage.cpp | 9 +++ src/Disks/ObjectStorages/IObjectStorage.h | 7 ++ .../ObjectStorages/ObjectStorageIterator.cpp | 20 +++++ .../ObjectStorages/ObjectStorageIterator.h | 53 ++++++++++++++ .../ObjectStorageIteratorAsync.cpp | 64 ++++++++++++++++ .../ObjectStorageIteratorAsync.h | 58 +++++++++++++++ .../ObjectStorages/S3/S3ObjectStorage.cpp | 73 +++++++++++++++++++ src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 2 + 11 files changed, 361 insertions(+) create mode 100644 src/Disks/ObjectStorages/ObjectStorageIterator.cpp create mode 100644 src/Disks/ObjectStorages/ObjectStorageIterator.h create mode 100644 src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp create mode 100644 src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 82d68ca8185..4c858ee788d 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -131,6 +131,10 @@ M(DistributedInsertThreadsActive, "Number of threads used for INSERT into Distributed running a task.") \ M(StorageS3Threads, "Number of threads in the StorageS3 thread pool.") \ M(StorageS3ThreadsActive, "Number of threads in the StorageS3 thread pool running a task.") \ + M(ObjectStorageS3Threads, "Number of threads in the S3ObjectStorage thread pool.") \ + M(ObjectStorageS3ThreadsActive, "Number of threads in the S3ObjectStorage thread pool running a task.") \ + M(ObjectStorageAzureThreads, "Number of threads in the AzureObjectStorage thread pool.") \ + M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \ M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \ M(MergeTreePartsLoaderThreadsActive, "Number of threads in the MergeTree parts loader thread pool running a task.") \ M(MergeTreePartsCleanerThreads, "Number of threads in the MergeTree parts cleaner thread pool.") \ diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 0358b4e915a..23a0da39dd3 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -11,9 +11,16 @@ #include #include +#include #include #include +namespace CurrentMetrics +{ + extern const Metric ObjectStorageAzureThreads; + extern const Metric ObjectStorageAzureThreadsActive; + +} namespace DB { @@ -26,6 +33,60 @@ namespace ErrorCodes } +namespace +{ + +class AzureIteratorAsync final : public IObjectStorageIteratorAsync +{ +public: + AzureIteratorAsync( + const std::string & path_prefix, + std::shared_ptr client_, + size_t max_list_size) + : IObjectStorageIteratorAsync( + CurrentMetrics::ObjectStorageAzureThreads, + CurrentMetrics::ObjectStorageAzureThreadsActive, + "ListObjectAzure") + , client(client_) + { + + options.Prefix = path_prefix; + options.PageSizeHint = static_cast(max_list_size); + } + +private: + bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) override + { + auto outcome = client->ListBlobs(options); + auto blob_list_response = client->ListBlobs(options); + auto blobs_list = blob_list_response.Blobs; + + for (const auto & blob : blobs_list) + { + batch.emplace_back( + blob.Name, + ObjectMetadata{ + static_cast(blob.BlobSize), + Poco::Timestamp::fromEpochTime( + std::chrono::duration_cast( + blob.Details.LastModified.time_since_epoch()).count()), + {}}); + } + + options.ContinuationToken = blob_list_response.NextPageToken; + if (blob_list_response.HasPage()) + return true; + + return false; + } + + std::shared_ptr client; + Azure::Storage::Blobs::ListBlobsOptions options; +}; + +} + + AzureObjectStorage::AzureObjectStorage( const String & name_, AzureClientPtr && client_, @@ -67,6 +128,14 @@ bool AzureObjectStorage::exists(const StoredObject & object) const return false; } +ObjectStorageIteratorPtr AzureObjectStorage::iterate(const std::string & path_prefix) const +{ + auto settings_ptr = settings.get(); + auto client_ptr = client.get(); + + return std::make_shared(path_prefix, client_ptr, settings_ptr->list_object_keys_size); +} + void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { auto client_ptr = client.get(); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index a36a03bcda4..5b08ceb80e3 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -60,6 +60,8 @@ public: void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const override; + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } std::string getName() const override { return "AzureObjectStorage"; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index a5903f9d429..ea22294224c 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -29,6 +30,14 @@ void IObjectStorage::listObjects(const std::string &, RelativePathsWithMetadata } +ObjectStorageIteratorPtr IObjectStorage::iterate(const std::string & path_prefix) const +{ + RelativePathsWithMetadata files; + listObjects(path_prefix, files, 0); + + return std::make_shared(std::move(files)); +} + std::optional IObjectStorage::tryGetObjectMetadata(const std::string & path) const { try diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 28de80a88cd..32f9d1ba764 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -20,6 +20,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -51,6 +54,8 @@ struct RelativePathWithMetadata using RelativePathsWithMetadata = std::vector; +class IObjectStorageIterator; +using ObjectStorageIteratorPtr = std::shared_ptr; /// Base class for all object storages which implement some subset of ordinary filesystem operations. /// @@ -75,6 +80,8 @@ public: virtual void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const; + virtual ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const; + /// Get object metadata if supported. It should be possible to receive /// at least size of object virtual std::optional tryGetObjectMetadata(const std::string & path) const; diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.cpp b/src/Disks/ObjectStorages/ObjectStorageIterator.cpp new file mode 100644 index 00000000000..188b743958c --- /dev/null +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.cpp @@ -0,0 +1,20 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +RelativePathWithMetadata ObjectStorageIteratorFromList::current() const +{ + if (!isValid()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); + + return *batch_iterator; +} + +} diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.h b/src/Disks/ObjectStorages/ObjectStorageIterator.h new file mode 100644 index 00000000000..c3afd395a74 --- /dev/null +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class IObjectStorageIterator +{ +public: + virtual void next() = 0; + virtual bool isValid() const = 0; + virtual RelativePathWithMetadata current() const = 0; + virtual size_t getAccumulatedSize() const = 0; + + virtual ~IObjectStorageIterator() = default; +}; + +using ObjectStorageIteratorPtr = std::shared_ptr; + +class ObjectStorageIteratorFromList : public IObjectStorageIterator +{ +public: + explicit ObjectStorageIteratorFromList(RelativePathsWithMetadata && batch_) + : batch(std::move(batch_)) + , batch_iterator(batch.begin()) + { + } + + void next() override + { + if (isValid()) + ++batch_iterator; + } + + bool isValid() const override + { + return batch_iterator != batch.end(); + } + + RelativePathWithMetadata current() const override; + + size_t getAccumulatedSize() const override + { + return batch.size(); + } +private: + RelativePathsWithMetadata batch; + RelativePathsWithMetadata::iterator batch_iterator; +}; + +} diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp new file mode 100644 index 00000000000..766071cf815 --- /dev/null +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -0,0 +1,64 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +void IObjectStorageIteratorAsync::next() +{ + std::lock_guard lock(mutex); + + if (current_batch_iterator != current_batch.end()) + { + ++current_batch_iterator; + } + else if (!is_finished) + { + if (outcome_future.valid()) + { + BatchAndHasNext next_batch = outcome_future.get(); + current_batch = std::move(next_batch.batch); + accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); + current_batch_iterator = current_batch.begin(); + if (next_batch.has_next) + outcome_future = scheduleBatch(); + else + is_finished = true; + } + } +} + +std::future IObjectStorageIteratorAsync::scheduleBatch() +{ + return list_objects_scheduler([this] + { + BatchAndHasNext result; + result.has_next = getBatchAndCheckNext(result.batch); + return result; + }, Priority{}); +} + + +bool IObjectStorageIteratorAsync::isValid() const +{ + return current_batch_iterator != current_batch.end(); +} + +RelativePathWithMetadata IObjectStorageIteratorAsync::current() const +{ + if (!isValid()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); + + return *current_batch_iterator; +} + +size_t IObjectStorageIteratorAsync::getAccumulatedSize() const +{ + return accumulated_size.load(std::memory_order_relaxed); +} + +} diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h new file mode 100644 index 00000000000..81ba9bce137 --- /dev/null +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -0,0 +1,58 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class IObjectStorageIteratorAsync : public IObjectStorageIterator +{ +public: + IObjectStorageIteratorAsync( + CurrentMetrics::Metric threads_metric, + CurrentMetrics::Metric threads_active_metric, + const std::string & thread_name) + : list_objects_pool(threads_metric, threads_active_metric, 1) + , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, thread_name)) + { + } + + void next() override; + bool isValid() const override; + RelativePathWithMetadata current() const override; + size_t getAccumulatedSize() const override; + + ~IObjectStorageIteratorAsync() override + { + list_objects_pool.wait(); + } + +protected: + + virtual bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) = 0; + + struct BatchAndHasNext + { + RelativePathsWithMetadata batch; + bool has_next; + }; + + std::future scheduleBatch(); + + bool is_finished{false}; + + std::mutex mutex; + ThreadPool list_objects_pool; + ThreadPoolCallbackRunner list_objects_scheduler; + std::future outcome_future; + RelativePathsWithMetadata current_batch; + RelativePathsWithMetadata::iterator current_batch_iterator; + std::atomic accumulated_size = 0; +}; + + +} diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 6e63efcc1e3..d19be20f920 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -3,6 +3,7 @@ #if USE_AWS_S3 #include +#include #include #include @@ -33,6 +34,13 @@ namespace ProfileEvents extern const Event DiskS3ListObjects; } +namespace CurrentMetrics +{ + extern const Metric ObjectStorageS3Threads; + extern const Metric ObjectStorageS3ThreadsActive; +} + + namespace DB { @@ -84,6 +92,62 @@ void logIfError(const Aws::Utils::Outcome & response, std::functi } +namespace +{ + +class S3IteratorAsync final : public IObjectStorageIteratorAsync +{ +public: + S3IteratorAsync( + const std::string & bucket, + const std::string & path_prefix, + std::shared_ptr client_, + size_t max_list_size) + : IObjectStorageIteratorAsync( + CurrentMetrics::ObjectStorageS3Threads, + CurrentMetrics::ObjectStorageS3ThreadsActive, + "ListObjectS3") + , client(client_) + { + request.SetBucket(bucket); + request.SetPrefix(path_prefix); + request.SetMaxKeys(static_cast(max_list_size)); + } + +private: + bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) override + { + ProfileEvents::increment(ProfileEvents::S3ListObjects); + + bool result = false; + auto outcome = client->ListObjectsV2(request); + /// Outcome failure will be handled on the caller side. + if (outcome.IsSuccess()) + { + auto objects = outcome.GetResult().GetContents(); + + result = !objects.empty(); + + for (const auto & object : objects) + batch.emplace_back(object.GetKey(), ObjectMetadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}}); + + if (result) + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + + return result; + } + + throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", + quoteString(request.GetBucket()), quoteString(request.GetPrefix()), + backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); + } + + std::shared_ptr client; + S3::ListObjectsV2Request request; +}; + +} + bool S3ObjectStorage::exists(const StoredObject & object) const { auto settings_ptr = s3_settings.get(); @@ -183,6 +247,15 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN disk_write_settings); } + +ObjectStorageIteratorPtr S3ObjectStorage::iterate(const std::string & path_prefix) const +{ + auto settings_ptr = s3_settings.get(); + auto client_ptr = client.get(); + + return std::make_shared(bucket, path_prefix, client_ptr, settings_ptr->list_object_keys_size); +} + void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { auto settings_ptr = s3_settings.get(); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index b0eb01aec0d..072e1354d38 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -102,6 +102,8 @@ public: void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const override; + /// Uses `DeleteObjectRequest`. void removeObject(const StoredObject & object) override; From 4bb44c7c72417a6e7a5f2ec7e1651b4360f9956e Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Sun, 4 Jun 2023 23:06:21 +0800 Subject: [PATCH 1088/2223] 1. skip extract darwin toolchain in builder when uncessary 2. update MacOSX SDK version in toolchain readme to match in builder --- cmake/toolchain/darwin-x86_64/README.txt | 4 ++-- docker/packager/binary/build.sh | 8 +++++--- docker/packager/packager | 2 ++ 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/cmake/toolchain/darwin-x86_64/README.txt b/cmake/toolchain/darwin-x86_64/README.txt index 65c9aba5be6..90ada960bfa 100644 --- a/cmake/toolchain/darwin-x86_64/README.txt +++ b/cmake/toolchain/darwin-x86_64/README.txt @@ -1,2 +1,2 @@ -wget https://github.com/phracker/MacOSX-SDKs/releases/download/10.15/MacOSX10.15.sdk.tar.xz -tar xJf MacOSX10.15.sdk.tar.xz --strip-components=1 +wget https://github.com/phracker/MacOSX-SDKs/releases/download/11.3/MacOSX11.0.sdk.tar.xz +tar xJf MacOSX11.0.sdk.tar.xz --strip-components=1 diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 2cd0a011013..ee1011a9cd5 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -11,9 +11,11 @@ ccache_status () { [ -O /build ] || git config --global --add safe.directory /build -mkdir -p /build/cmake/toolchain/darwin-x86_64 -tar xJf /MacOSX11.0.sdk.tar.xz -C /build/cmake/toolchain/darwin-x86_64 --strip-components=1 -ln -sf darwin-x86_64 /build/cmake/toolchain/darwin-aarch64 +if [ "$EXTRACT_TOOLCHAIN_DARWIN" = "1" ];then + mkdir -p /build/cmake/toolchain/darwin-x86_64 + tar xJf /MacOSX11.0.sdk.tar.xz -C /build/cmake/toolchain/darwin-x86_64 --strip-components=1 + ln -sf darwin-x86_64 /build/cmake/toolchain/darwin-aarch64 +fi # Uncomment to debug ccache. Don't put ccache log in /output right away, or it # will be confusingly packed into the "performance" package. diff --git a/docker/packager/packager b/docker/packager/packager index a894fe2d8e9..1b3df858cd2 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -167,6 +167,7 @@ def parse_env_variables( cmake_flags.append( "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/darwin/toolchain-x86_64.cmake" ) + result.append("EXTRACT_TOOLCHAIN_DARWIN=1") elif is_cross_darwin_arm: cc = compiler[: -len(DARWIN_ARM_SUFFIX)] cmake_flags.append("-DCMAKE_AR:FILEPATH=/cctools/bin/aarch64-apple-darwin-ar") @@ -181,6 +182,7 @@ def parse_env_variables( cmake_flags.append( "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/darwin/toolchain-aarch64.cmake" ) + result.append("EXTRACT_TOOLCHAIN_DARWIN=1") elif is_cross_arm: cc = compiler[: -len(ARM_SUFFIX)] cmake_flags.append( From 34c4b89b161df4427912d402fa6aee4a6821dde1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 4 Jun 2023 16:23:34 +0000 Subject: [PATCH 1089/2223] fix backward compatibility for IP types hashing in aggregate functions --- base/base/IPv4andIPv6.h | 24 +++++++++++++------ .../AggregateFunctionMap.cpp | 3 +++ src/AggregateFunctions/AggregateFunctionMap.h | 3 +++ .../AggregateFunctionUniq.cpp | 9 +++++++ .../AggregateFunctionUniq.h | 12 +++++----- src/Core/Types_fwd.h | 2 +- src/DataTypes/DataTypeMap.cpp | 3 ++- 7 files changed, 41 insertions(+), 15 deletions(-) diff --git a/base/base/IPv4andIPv6.h b/base/base/IPv4andIPv6.h index 7b745ec7b84..4aee2329572 100644 --- a/base/base/IPv4andIPv6.h +++ b/base/base/IPv4andIPv6.h @@ -7,16 +7,17 @@ namespace DB { - using IPv4 = StrongTypedef; + struct IPv4 : StrongTypedef + { + using StrongTypedef::StrongTypedef; + using StrongTypedef::operator=; + constexpr explicit IPv4(UInt64 value): StrongTypedef(static_cast(value)) {} + }; struct IPv6 : StrongTypedef { - constexpr IPv6() = default; - constexpr explicit IPv6(const UInt128 & x) : StrongTypedef(x) {} - constexpr explicit IPv6(UInt128 && x) : StrongTypedef(std::move(x)) {} - - IPv6 & operator=(const UInt128 & rhs) { StrongTypedef::operator=(rhs); return *this; } - IPv6 & operator=(UInt128 && rhs) { StrongTypedef::operator=(std::move(rhs)); return *this; } + using StrongTypedef::StrongTypedef; + using StrongTypedef::operator=; bool operator<(const IPv6 & rhs) const { @@ -62,4 +63,13 @@ namespace std return std::hash()(x.toUnderType()); } }; + + template <> + struct hash + { + size_t operator()(const DB::IPv4 & x) const + { + return std::hash()(x.toUnderType()); + } + }; } diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index f6100602f3f..38e4f49d9a2 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -100,6 +100,9 @@ public: return std::make_shared>(nested_function, arguments); case TypeIndex::UUID: return std::make_shared>(nested_function, arguments); + case TypeIndex::IPv4: + return std::make_shared>(nested_function, arguments); + case TypeIndex::IPv6: case TypeIndex::FixedString: case TypeIndex::String: return std::make_shared>(nested_function, arguments); diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 55f6611974e..4a4ae92735b 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -19,6 +19,7 @@ #include #include #include "DataTypes/Serializations/ISerialization.h" +#include #include "base/types.h" #include #include "AggregateFunctions/AggregateFunctionFactory.h" @@ -147,6 +148,8 @@ public: StringRef key_ref; if (key_type->getTypeId() == TypeIndex::FixedString) key_ref = assert_cast(key_column).getDataAt(offset + i); + else if (key_type->getTypeId() == TypeIndex::IPv6) + key_ref = assert_cast(key_column).getDataAt(offset + i); else key_ref = assert_cast(key_column).getDataAt(offset + i); diff --git a/src/AggregateFunctions/AggregateFunctionUniq.cpp b/src/AggregateFunctions/AggregateFunctionUniq.cpp index bf998c5ee9e..f5147daa97b 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -60,6 +61,10 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument return std::make_shared>(argument_types); else if (which.isUUID()) return std::make_shared>(argument_types); + else if (which.isIPv4()) + return std::make_shared>(argument_types); + else if (which.isIPv6()) + return std::make_shared>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) @@ -109,6 +114,10 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument return std::make_shared>>(argument_types); else if (which.isUUID()) return std::make_shared>>(argument_types); + else if (which.isIPv4()) + return std::make_shared>>(argument_types); + else if (which.isIPv6()) + return std::make_shared>>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index c782b9314fd..0524dd53ec0 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -248,17 +248,17 @@ struct Adder AggregateFunctionUniqUniquesHashSetData> || std::is_same_v>) { const auto & column = *columns[0]; - if constexpr (!std::is_same_v) + if constexpr (std::is_same_v || std::is_same_v) + { + StringRef value = column.getDataAt(row_num); + data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); + } + else { using ValueType = typename decltype(data.set)::value_type; const auto & value = assert_cast &>(column).getElement(row_num); data.set.insert(static_cast(AggregateFunctionUniqTraits::hash(value))); } - else - { - StringRef value = column.getDataAt(row_num); - data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); - } } else if constexpr (std::is_same_v>) { diff --git a/src/Core/Types_fwd.h b/src/Core/Types_fwd.h index 3db1127fe5a..a59e4b6eab8 100644 --- a/src/Core/Types_fwd.h +++ b/src/Core/Types_fwd.h @@ -27,7 +27,7 @@ namespace DB using UUID = StrongTypedef; -using IPv4 = StrongTypedef; +struct IPv4; struct IPv6; diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index 0f5d97e6761..90561857fad 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -69,7 +69,7 @@ void DataTypeMap::assertKeyType() const if (!checkKeyType(key_type)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type of Map key must be a type, that can be represented by integer " - "or String or FixedString (possibly LowCardinality) or UUID," + "or String or FixedString (possibly LowCardinality) or UUID or IPv6," " but {} given", key_type->getName()); } @@ -120,6 +120,7 @@ bool DataTypeMap::checkKeyType(DataTypePtr key_type) else if (!key_type->isValueRepresentedByInteger() && !isStringOrFixedString(*key_type) && !WhichDataType(key_type).isNothing() + && !WhichDataType(key_type).isIPv6() && !WhichDataType(key_type).isUUID()) { return false; From 8d8d06285ff6d931d6a1ee069c9a5885dc09264a Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 4 Jun 2023 18:59:22 +0200 Subject: [PATCH 1090/2223] Add integration test --- src/Storages/StorageAzure.cpp | 51 +++++++- .../__init__.py | 1 + .../test_storage_azure_blob_storage/test.py | 122 ++++++++++++++++++ 3 files changed, 170 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_storage_azure_blob_storage/__init__.py create mode 100644 tests/integration/test_storage_azure_blob_storage/test.py diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index fd250a128c9..7b4bc9e6769 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -115,6 +115,7 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C else { configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); if (!is_format_arg(sixth_arg)) @@ -216,18 +217,58 @@ AzureClientPtr StorageAzure::createClient(StorageAzure::Configuration configurat if (configuration.is_connection_string) { result = std::make_unique(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container)); + result->CreateIfNotExists(); } else { if (configuration.account_name.has_value() && configuration.account_key.has_value()) { auto storage_shared_key_credential = std::make_shared(*configuration.account_name, *configuration.account_key); - result = std::make_unique(configuration.connection_url, storage_shared_key_credential); + auto blob_service_client = std::make_unique(configuration.connection_url, storage_shared_key_credential); + try + { + result = std::make_unique(blob_service_client->CreateBlobContainer(configuration.container).Value); + } + catch (const Azure::Storage::StorageException & e) + { + if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict) + { + auto final_url = configuration.connection_url + + (configuration.connection_url.back() == '/' ? "" : "/") + + configuration.container; + + result = std::make_unique(final_url, storage_shared_key_credential); + } + else + { + throw; + } + } } + else + { + auto managed_identity_credential = std::make_shared(); + auto blob_service_client = std::make_unique(configuration.connection_url, managed_identity_credential); + try + { + result = std::make_unique(blob_service_client->CreateBlobContainer(configuration.container).Value); + } + catch (const Azure::Storage::StorageException & e) + { + if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict) + { + auto final_url = configuration.connection_url + + (configuration.connection_url.back() == '/' ? "" : "/") + + configuration.container; - auto managed_identity_credential = std::make_shared(); - - result = std::make_unique(configuration.connection_url, managed_identity_credential); + result = std::make_unique(final_url, managed_identity_credential); + } + else + { + throw; + } + } + } } return result; @@ -466,12 +507,14 @@ SinkToStoragePtr StorageAzure::write(const ASTPtr & query, const StorageMetadata if (!truncate_in_insert && object_storage->exists(StoredObject(configuration.blob_path))) { + if (local_context->getSettingsRef().s3_create_new_file_on_insert) { size_t index = configuration.blobs_paths.size(); const auto & first_key = configuration.blobs_paths[0]; auto pos = first_key.find_first_of('.'); String new_key; + do { new_key = first_key.substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : first_key.substr(pos)); diff --git a/tests/integration/test_storage_azure_blob_storage/__init__.py b/tests/integration/test_storage_azure_blob_storage/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_storage_azure_blob_storage/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py new file mode 100644 index 00000000000..94b059fe4fe --- /dev/null +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -0,0 +1,122 @@ +#!/usr/bin/env python3 + +import gzip +import json +import logging +import os +import io +import random +import threading +import time + +from azure.storage.blob import BlobServiceClient +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.network import PartitionManager +from helpers.mock_servers import start_mock_servers +from helpers.test_tools import exec_query_with_retry + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node", + with_azurite=True, + ) + cluster.start() + + yield cluster + finally: + cluster.shutdown() + +def azure_query(node, query, try_num=3, settings={}): + for i in range(try_num): + try: + return node.query(query, settings=settings) + except Exception as ex: + retriable_errors = [ + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response" + ] + retry = False + for error in retriable_errors: + if error in str(ex): + retry = True + print(f"Try num: {i}. Having retriable error: {ex}") + break + if not retry or i == try_num - 1: + raise Exception(ex) + continue + +def get_azure_file_content(filename): + container_name = "cont" + connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + container_client = blob_service_client.get_container_client(container_name) + blob_client = container_client.get_blob_client(filename) + download_stream = blob_client.download_blob() + return download_stream.readall().decode('utf-8') + +def test_create_table_connection_string(cluster): + node = cluster.instances["node"] + azure_query(node, "CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = Azure('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', 'cont', 'test_create_connection_string', 'CSV')") + +def test_create_table_account_string(cluster): + node = cluster.instances["node"] + azure_query(node, "CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')") + +def test_simple_write_account_string(cluster): + node = cluster.instances["node"] + azure_query(node, "CREATE TABLE test_simple_write (key UInt64, data String) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')") + azure_query(node, "INSERT INTO test_simple_write VALUES (1, 'a')") + print(get_azure_file_content('test_simple_write.csv')) + assert get_azure_file_content('test_simple_write.csv') == '1,"a"\n' + +def test_simple_write_connection_string(cluster): + node = cluster.instances["node"] + azure_query(node, "CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = Azure('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_c.csv', 'CSV')") + azure_query(node, "INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") + print(get_azure_file_content('test_simple_write_c.csv')) + assert get_azure_file_content('test_simple_write_c.csv') == '1,"a"\n' + + +def test_partition_by(cluster): + node = cluster.instances["node"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + partition_by = "column3" + values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" + filename = "test_{_partition_id}.csv" + + azure_query(node, f"CREATE TABLE test_partitioned_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}") + azure_query(node, f"INSERT INTO test_partitioned_write VALUES {values}") + + assert "1,2,3\n" == get_azure_file_content("test_3.csv") + assert "3,2,1\n" == get_azure_file_content("test_1.csv") + assert "78,43,45\n" == get_azure_file_content("test_45.csv") + + +def test_partition_by_string_column(cluster): + node = cluster.instances["node"] + table_format = "col_num UInt32, col_str String" + partition_by = "col_str" + values = "(1, 'foo/bar'), (3, 'йцук'), (78, '你好')" + filename = "test_{_partition_id}.csv" + azure_query(node, f"CREATE TABLE test_partitioned_string_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}") + azure_query(node, f"INSERT INTO test_partitioned_string_write VALUES {values}") + + assert '1,"foo/bar"\n' == get_azure_file_content("test_foo/bar.csv") + assert '3,"йцук"\n' == get_azure_file_content("test_йцук.csv") + assert '78,"你好"\n' == get_azure_file_content("test_你好.csv") + + +def test_partition_by_const_column(cluster): + node = cluster.instances["node"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" + partition_by = "'88'" + values_csv = "1,2,3\n3,2,1\n78,43,45\n" + filename = "test_{_partition_id}.csv" + azure_query(node, f"CREATE TABLE test_partitioned_const_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}") + azure_query(node, f"INSERT INTO test_partitioned_const_write VALUES {values}") + assert values_csv == get_azure_file_content("test_88.csv") From a66f68e5df5584cf08a42ff6dd12b4e935f2cb3a Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 5 Jun 2023 01:48:13 +0800 Subject: [PATCH 1091/2223] fix again --- src/Interpreters/MutationsInterpreter.cpp | 58 ++++++++++++----------- 1 file changed, 31 insertions(+), 27 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 25bb3fc5e82..25c52ad8925 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -38,6 +38,7 @@ #include #include #include +#include namespace DB @@ -804,10 +805,10 @@ void MutationsInterpreter::prepare(bool dry_run) /// We care about affected indices and projections because we also need to rewrite them /// when one of index columns updated or filtered with delete. /// The same about columns, that are needed for calculation of TTL expressions. + NameSet changed_columns; + NameSet unchanged_columns; if (!dependencies.empty()) { - NameSet changed_columns; - NameSet unchanged_columns; for (const auto & dependency : dependencies) { if (dependency.isReadOnly()) @@ -826,31 +827,6 @@ void MutationsInterpreter::prepare(bool dry_run) for (const auto & column : changed_columns) stages.back().column_to_updated.emplace( column, std::make_shared(column)); - - for (const auto & index : metadata_snapshot->getSecondaryIndices()) - { - if (source.hasIndexOrProjection("skp_idx_" + index.name + ".idx") - || source.hasIndexOrProjection("skp_idx_" + index.name + ".idx2")) - { - const auto & index_cols = index.expression->getRequiredColumns(); - bool changed = std::any_of( - index_cols.begin(), index_cols.end(), [&](const auto & col) { return changed_columns.contains(col); }); - if (changed) - materialized_indices.insert(index.name); - } - } - - for (const auto & projection : metadata_snapshot->getProjections()) - { - if (source.hasIndexOrProjection(projection.getDirectoryName())) - { - const auto & projection_cols = projection.required_columns; - bool changed = std::any_of( - projection_cols.begin(), projection_cols.end(), [&](const auto & col) { return changed_columns.contains(col); }); - if (changed) - materialized_projections.insert(projection.name); - } - } } if (!unchanged_columns.empty()) @@ -883,6 +859,34 @@ void MutationsInterpreter::prepare(bool dry_run) } } + for (const auto & index : metadata_snapshot->getSecondaryIndices()) + { + if (source.hasIndexOrProjection("skp_idx_" + index.name + ".idx") || source.hasIndexOrProjection("skp_idx_" + index.name + ".idx2")) + { + const auto & index_cols = index.expression->getRequiredColumns(); + bool changed = std::any_of( + index_cols.begin(), + index_cols.end(), + [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); + if (changed) + materialized_indices.insert(index.name); + } + } + + for (const auto & projection : metadata_snapshot->getProjections()) + { + if (source.hasIndexOrProjection(projection.getDirectoryName())) + { + const auto & projection_cols = projection.required_columns; + bool changed = std::any_of( + projection_cols.begin(), + projection_cols.end(), + [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); + if (changed) + materialized_projections.insert(projection.name); + } + } + /// Stages might be empty when we materialize skip indices or projections which don't add any /// column dependencies. if (stages.empty()) From 2cc457141ed83a50c7a6e4dc395325c6fd4a898d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 15:32:54 -0300 Subject: [PATCH 1092/2223] clean documentation of ip4 ip6 from domains --- docs/en/interfaces/formats.md | 34 +++++++++---------- docs/en/operations/system-tables/query_log.md | 4 +-- .../system-tables/query_thread_log.md | 4 +-- .../operations/system-tables/session_log.md | 2 +- .../operations/system-tables/zookeeper_log.md | 2 +- docs/en/sql-reference/data-types/index.md | 2 +- .../data-types/{domains => }/ipv4.md | 27 +++------------ .../data-types/{domains => }/ipv6.md | 29 +++------------- .../functions/ip-address-functions.md | 6 ++-- docs/redirects.txt | 10 +++--- 10 files changed, 41 insertions(+), 79 deletions(-) rename docs/en/sql-reference/data-types/{domains => }/ipv4.md (60%) rename docs/en/sql-reference/data-types/{domains => }/ipv6.md (61%) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 2ab9e8caec4..d75fb32b571 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1292,8 +1292,8 @@ For output it uses the following correspondence between ClickHouse types and BSO | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `\x04` array | | [Named Tuple](/docs/en/sql-reference/data-types/tuple.md) | `\x03` document | | [Map](/docs/en/sql-reference/data-types/map.md) | `\x03` document | -| [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `\x10` int32 | -| [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `\x05` binary, `\x00` binary subtype | +| [IPv4](/docs/en/sql-reference/data-types/ipv4.md) | `\x10` int32 | +| [IPv6](/docs/en/sql-reference/data-types/ipv6.md) | `\x05` binary, `\x00` binary subtype | For input it uses the following correspondence between BSON types and ClickHouse types: @@ -1303,7 +1303,7 @@ For input it uses the following correspondence between BSON types and ClickHouse | `\x02` string | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | | `\x03` document | [Map](/docs/en/sql-reference/data-types/map.md)/[Named Tuple](/docs/en/sql-reference/data-types/tuple.md) | | `\x04` array | [Array](/docs/en/sql-reference/data-types/array.md)/[Tuple](/docs/en/sql-reference/data-types/tuple.md) | -| `\x05` binary, `\x00` binary subtype | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md)/[IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | +| `\x05` binary, `\x00` binary subtype | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md)/[IPv6](/docs/en/sql-reference/data-types/ipv6.md) | | `\x05` binary, `\x02` old binary subtype | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | | `\x05` binary, `\x03` old uuid subtype | [UUID](/docs/en/sql-reference/data-types/uuid.md) | | `\x05` binary, `\x04` uuid subtype | [UUID](/docs/en/sql-reference/data-types/uuid.md) | @@ -1313,7 +1313,7 @@ For input it uses the following correspondence between BSON types and ClickHouse | `\x0A` null value | [NULL](/docs/en/sql-reference/data-types/nullable.md) | | `\x0D` JavaScript code | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | | `\x0E` symbol | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | -| `\x10` int32 | [Int32/UInt32](/docs/en/sql-reference/data-types/int-uint.md)/[Decimal32](/docs/en/sql-reference/data-types/decimal.md)/[IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md)/[Enum8/Enum16](/docs/en/sql-reference/data-types/enum.md) | +| `\x10` int32 | [Int32/UInt32](/docs/en/sql-reference/data-types/int-uint.md)/[Decimal32](/docs/en/sql-reference/data-types/decimal.md)/[IPv4](/docs/en/sql-reference/data-types/ipv4.md)/[Enum8/Enum16](/docs/en/sql-reference/data-types/enum.md) | | `\x12` int64 | [Int64/UInt64](/docs/en/sql-reference/data-types/int-uint.md)/[Decimal64](/docs/en/sql-reference/data-types/decimal.md)/[DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | Other BSON types are not supported. Also, it performs conversion between different integer types (for example, you can insert BSON int32 value into ClickHouse UInt8). @@ -1663,8 +1663,8 @@ The table below shows supported data types and how they match ClickHouse [data t | `ENUM` | [Enum(8/16)](/docs/en/sql-reference/data-types/enum.md) | `ENUM` | | `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | | `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | -| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `UINT32` | -| `DATA` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `DATA` | +| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/ipv4.md) | `UINT32` | +| `DATA` | [IPv6](/docs/en/sql-reference/data-types/ipv6.md) | `DATA` | | `DATA` | [Int128/UInt128/Int256/UInt256](/docs/en/sql-reference/data-types/int-uint.md) | `DATA` | | `DATA` | [Decimal128/Decimal256](/docs/en/sql-reference/data-types/decimal.md) | `DATA` | | `STRUCT(entries LIST(STRUCT(key Key, value Value)))` | [Map](/docs/en/sql-reference/data-types/map.md) | `STRUCT(entries LIST(STRUCT(key Key, value Value)))` | @@ -1866,8 +1866,8 @@ The table below shows supported data types and how they match ClickHouse [data t | `long (timestamp-millis)` \** | [DateTime64(3)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-millis)` \** | | `long (timestamp-micros)` \** | [DateTime64(6)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \** | | `bytes (decimal)` \** | [DateTime64(N)](/docs/en/sql-reference/data-types/datetime.md) | `bytes (decimal)` \** | -| `int` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `int` | -| `fixed(16)` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `fixed(16)` | +| `int` | [IPv4](/docs/en/sql-reference/data-types/ipv4.md) | `int` | +| `fixed(16)` | [IPv6](/docs/en/sql-reference/data-types/ipv6.md) | `fixed(16)` | | `bytes (decimal)` \** | [Decimal(P, S)](/docs/en/sql-reference/data-types/decimal.md) | `bytes (decimal)` \** | | `string (uuid)` \** | [UUID](/docs/en/sql-reference/data-types/uuid.md) | `string (uuid)` \** | | `fixed(16)` | [Int128/UInt128](/docs/en/sql-reference/data-types/int-uint.md) | `fixed(16)` | @@ -2001,9 +2001,9 @@ The table below shows supported data types and how they match ClickHouse [data t | `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | | `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | | `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` | -| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `UINT32` | -| `FIXED_LENGTH_BYTE_ARRAY`, `BINARY` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `FIXED_LENGTH_BYTE_ARRAY` | -| `FIXED_LENGTH_BYTE_ARRAY`, `BINARY` | [Int128/UInt128/Int256/UInt256](/docs/en/sql-reference/data-types/int-uint.md) | `FIXED_LENGTH_BYTE_ARRAY` | +| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/ipv4.md) | `UINT32` | +| `FIXED_LENGTH_BYTE_ARRAY`, `BINARY` | [IPv6](/docs/en/sql-reference/data-types/ipv6.md) | `FIXED_LENGTH_BYTE_ARRAY` | +| `FIXED_LENGTH_BYTE_ARRAY`, `BINARY` | [Int128/UInt128/Int256/UInt256](/docs/en/sql-reference/data-types/int-uint.md) | `FIXED_LENGTH_BYTE_ARRAY` | Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` and `Map` types also can be nested. @@ -2057,7 +2057,7 @@ Special format for reading Parquet file metadata (https://parquet.apache.org/doc - logical_type - column logical type - compression - compression used for this column - total_uncompressed_size - total uncompressed bytes size of the column, calculated as the sum of total_uncompressed_size of the column from all row groups - - total_compressed_size - total compressed bytes size of the column, calculated as the sum of total_compressed_size of the column from all row groups + - total_compressed_size - total compressed bytes size of the column, calculated as the sum of total_compressed_size of the column from all row groups - space_saved - percent of space saved by compression, calculated as (1 - total_compressed_size/total_uncompressed_size). - encodings - the list of encodings used for this column - row_groups - the list of row groups metadata with the next structure: @@ -2204,9 +2204,9 @@ The table below shows supported data types and how they match ClickHouse [data t | `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | | `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | | `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` | -| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `UINT32` | -| `FIXED_SIZE_BINARY`, `BINARY` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `FIXED_SIZE_BINARY` | -| `FIXED_SIZE_BINARY`, `BINARY` | [Int128/UInt128/Int256/UInt256](/docs/en/sql-reference/data-types/int-uint.md) | `FIXED_SIZE_BINARY` | +| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/ipv4.md) | `UINT32` | +| `FIXED_SIZE_BINARY`, `BINARY` | [IPv6](/docs/en/sql-reference/data-types/ipv6.md) | `FIXED_SIZE_BINARY` | +| `FIXED_SIZE_BINARY`, `BINARY` | [Int128/UInt128/Int256/UInt256](/docs/en/sql-reference/data-types/int-uint.md) | `FIXED_SIZE_BINARY` | Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` and `Map` types also can be nested. @@ -2272,7 +2272,7 @@ The table below shows supported data types and how they match ClickHouse [data t | `Struct` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `Struct` | | `Map` | [Map](/docs/en/sql-reference/data-types/map.md) | `Map` | | `Int` | [IPv4](/docs/en/sql-reference/data-types/int-uint.md) | `Int` | -| `Binary` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `Binary` | +| `Binary` | [IPv6](/docs/en/sql-reference/data-types/ipv6.md) | `Binary` | | `Binary` | [Int128/UInt128/Int256/UInt256](/docs/en/sql-reference/data-types/int-uint.md) | `Binary` | | `Binary` | [Decimal256](/docs/en/sql-reference/data-types/decimal.md) | `Binary` | @@ -2485,7 +2485,7 @@ ClickHouse supports reading and writing [MessagePack](https://msgpack.org/) data | `uint 64` | [DateTime64](/docs/en/sql-reference/data-types/datetime.md) | `uint 64` | | `fixarray`, `array 16`, `array 32` | [Array](/docs/en/sql-reference/data-types/array.md)/[Tuple](/docs/en/sql-reference/data-types/tuple.md) | `fixarray`, `array 16`, `array 32` | | `fixmap`, `map 16`, `map 32` | [Map](/docs/en/sql-reference/data-types/map.md) | `fixmap`, `map 16`, `map 32` | -| `uint 32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `uint 32` | +| `uint 32` | [IPv4](/docs/en/sql-reference/data-types/ipv4.md) | `uint 32` | | `bin 8` | [String](/docs/en/sql-reference/data-types/string.md) | `bin 8` | | `int 8` | [Enum8](/docs/en/sql-reference/data-types/enum.md) | `int 8` | | `bin 8` | [(U)Int128/(U)Int256](/docs/en/sql-reference/data-types/int-uint.md) | `bin 8` | diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 71e1452cef1..b9fdd19c643 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -71,11 +71,11 @@ Columns: - 0 — Query was initiated by another query as part of distributed query execution. - `user` ([String](../../sql-reference/data-types/string.md)) — Name of the user who initiated the current query. - `query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the query. -- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP address that was used to make the query. +- `address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP address that was used to make the query. - `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — The client port that was used to make the query. - `initial_user` ([String](../../sql-reference/data-types/string.md)) — Name of the user who ran the initial query (for distributed query execution). - `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the initial query (for distributed query execution). -- `initial_address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP address that the parent query was launched from. +- `initial_address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP address that the parent query was launched from. - `initial_port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — The client port that was used to make the parent query. - `initial_query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Initial query starting time (for distributed query execution). - `initial_query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Initial query starting time with microseconds precision (for distributed query execution). diff --git a/docs/en/operations/system-tables/query_thread_log.md b/docs/en/operations/system-tables/query_thread_log.md index cdd23bb15db..a6d5632ade9 100644 --- a/docs/en/operations/system-tables/query_thread_log.md +++ b/docs/en/operations/system-tables/query_thread_log.md @@ -40,11 +40,11 @@ Columns: - 0 — Query was initiated by another query for distributed query execution. - `user` ([String](../../sql-reference/data-types/string.md)) — Name of the user who initiated the current query. - `query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the query. -- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP address that was used to make the query. +- `address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP address that was used to make the query. - `port` ([UInt16](../../sql-reference/data-types/int-uint.md#uint-ranges)) — The client port that was used to make the query. - `initial_user` ([String](../../sql-reference/data-types/string.md)) — Name of the user who ran the initial query (for distributed query execution). - `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the initial query (for distributed query execution). -- `initial_address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP address that the parent query was launched from. +- `initial_address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP address that the parent query was launched from. - `initial_port` ([UInt16](../../sql-reference/data-types/int-uint.md#uint-ranges)) — The client port that was used to make the parent query. - `interface` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Interface that the query was initiated from. Possible values: - 1 — TCP. diff --git a/docs/en/operations/system-tables/session_log.md b/docs/en/operations/system-tables/session_log.md index 661d34677e4..5b1a2b2a489 100644 --- a/docs/en/operations/system-tables/session_log.md +++ b/docs/en/operations/system-tables/session_log.md @@ -28,7 +28,7 @@ Columns: - `profiles` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — The list of profiles set for all roles and/or users. - `roles` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — The list of roles to which the profile is applied. - `settings` ([Array](../../sql-reference/data-types/array.md)([Tuple](../../sql-reference/data-types/tuple.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md), [String](../../sql-reference/data-types/string.md)))) — Settings that were changed when the client logged in/out. -- `client_address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — The IP address that was used to log in/out. +- `client_address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — The IP address that was used to log in/out. - `client_port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — The client port that was used to log in/out. - `interface` ([Enum8](../../sql-reference/data-types/enum.md)) — The interface from which the login was initiated. Possible values: - `TCP` diff --git a/docs/en/operations/system-tables/zookeeper_log.md b/docs/en/operations/system-tables/zookeeper_log.md index b7cc4e22cd6..dce5be29f62 100644 --- a/docs/en/operations/system-tables/zookeeper_log.md +++ b/docs/en/operations/system-tables/zookeeper_log.md @@ -15,7 +15,7 @@ Columns with request parameters: - `Finalize` — The connection is lost, no response was received. - `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the event happened. - `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time when the event happened. -- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP address of ZooKeeper server that was used to make the request. +- `address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP address of ZooKeeper server that was used to make the request. - `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — The port of ZooKeeper server that was used to make the request. - `session_id` ([Int64](../../sql-reference/data-types/int-uint.md)) — The session ID that the ZooKeeper server sets for each connection. - `xid` ([Int32](../../sql-reference/data-types/int-uint.md)) — The ID of the request within the session. This is usually a sequential request number. It is the same for the request row and the paired `response`/`finalize` row. diff --git a/docs/en/sql-reference/data-types/index.md b/docs/en/sql-reference/data-types/index.md index 508307a0543..ffd063590fa 100644 --- a/docs/en/sql-reference/data-types/index.md +++ b/docs/en/sql-reference/data-types/index.md @@ -28,6 +28,6 @@ ClickHouse data types include: - **Nested data structures**: A [`Nested` data structure](./nested-data-structures/index.md) is like a table inside a cell - **Tuples**: A [`Tuple` of elements](./tuple.md), each having an individual type. - **Nullable**: [`Nullable`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column settings its default value for the data type) -- **IP addresses**: use [`IPv4`](./domains/ipv4.md) and [`IPv6`](./domains/ipv6.md) to efficiently store IP addresses +- **IP addresses**: use [`IPv4`](./ipv4.md) and [`IPv6`](./ipv6.md) to efficiently store IP addresses - **Geo types**: for [geographical data](./geo.md), including `Point`, `Ring`, `Polygon` and `MultiPolygon` - **Special data types**: including [`Expression`](./special-data-types/expression.md), [`Set`](./special-data-types/set.md), [`Nothing`](./special-data-types/nothing.md) and [`Interval`](./special-data-types/interval.md) diff --git a/docs/en/sql-reference/data-types/domains/ipv4.md b/docs/en/sql-reference/data-types/ipv4.md similarity index 60% rename from docs/en/sql-reference/data-types/domains/ipv4.md rename to docs/en/sql-reference/data-types/ipv4.md index b34814211fc..288806f47b3 100644 --- a/docs/en/sql-reference/data-types/domains/ipv4.md +++ b/docs/en/sql-reference/data-types/ipv4.md @@ -1,12 +1,12 @@ --- -slug: /en/sql-reference/data-types/domains/ipv4 +slug: /en/sql-reference/data-types/ipv4 sidebar_position: 59 sidebar_label: IPv4 --- ## IPv4 -`IPv4` is a domain based on `UInt32` type and serves as a typed replacement for storing IPv4 values. It provides compact storage with the human-friendly input-output format and column type information on inspection. +IPv4 addresses. Stored in 4 bytes as UInt32. ### Basic Usage @@ -57,25 +57,6 @@ SELECT toTypeName(from), hex(from) FROM hits LIMIT 1; └──────────────────┴───────────┘ ``` -Domain values are not implicitly convertible to types other than `UInt32`. -If you want to convert `IPv4` value to a string, you have to do that explicitly with `IPv4NumToString()` function: +**See Also** -``` sql -SELECT toTypeName(s), IPv4NumToString(from) as s FROM hits LIMIT 1; -``` - - ┌─toTypeName(IPv4NumToString(from))─┬─s──────────────┐ - │ String │ 183.247.232.58 │ - └───────────────────────────────────┴────────────────┘ - -Or cast to a `UInt32` value: - -``` sql -SELECT toTypeName(i), CAST(from as UInt32) as i FROM hits LIMIT 1; -``` - -``` text -┌─toTypeName(CAST(from, 'UInt32'))─┬──────────i─┐ -│ UInt32 │ 3086477370 │ -└──────────────────────────────────┴────────────┘ -``` +- [Functions for Working with IPv4 and IPv6 Addresses](../functions/ip-address-functions.md) diff --git a/docs/en/sql-reference/data-types/domains/ipv6.md b/docs/en/sql-reference/data-types/ipv6.md similarity index 61% rename from docs/en/sql-reference/data-types/domains/ipv6.md rename to docs/en/sql-reference/data-types/ipv6.md index dcb22e3cb6d..284a1f80854 100644 --- a/docs/en/sql-reference/data-types/domains/ipv6.md +++ b/docs/en/sql-reference/data-types/ipv6.md @@ -1,12 +1,12 @@ --- -slug: /en/sql-reference/data-types/domains/ipv6 +slug: /en/sql-reference/data-types/ipv6 sidebar_position: 60 sidebar_label: IPv6 --- ## IPv6 -`IPv6` is a domain based on `FixedString(16)` type and serves as a typed replacement for storing IPv6 values. It provides compact storage with the human-friendly input-output format and column type information on inspection. +IPv6 addresses. Stored in 16 bytes as UInt128. ### Basic Usage @@ -57,27 +57,6 @@ SELECT toTypeName(from), hex(from) FROM hits LIMIT 1; └──────────────────┴──────────────────────────────────┘ ``` -Domain values are not implicitly convertible to types other than `FixedString(16)`. -If you want to convert `IPv6` value to a string, you have to do that explicitly with `IPv6NumToString()` function: +**See Also** -``` sql -SELECT toTypeName(s), IPv6NumToString(from) as s FROM hits LIMIT 1; -``` - -``` text -┌─toTypeName(IPv6NumToString(from))─┬─s─────────────────────────────┐ -│ String │ 2001:44c8:129:2632:33:0:252:2 │ -└───────────────────────────────────┴───────────────────────────────┘ -``` - -Or cast to a `FixedString(16)` value: - -``` sql -SELECT toTypeName(i), CAST(from as FixedString(16)) as i FROM hits LIMIT 1; -``` - -``` text -┌─toTypeName(CAST(from, 'FixedString(16)'))─┬─i───────┐ -│ FixedString(16) │ ��� │ -└───────────────────────────────────────────┴─────────┘ -``` +- [Functions for Working with IPv4 and IPv6 Addresses](../functions/ip-address-functions.md) diff --git a/docs/en/sql-reference/functions/ip-address-functions.md b/docs/en/sql-reference/functions/ip-address-functions.md index 0dc1db1161b..33c788a632e 100644 --- a/docs/en/sql-reference/functions/ip-address-functions.md +++ b/docs/en/sql-reference/functions/ip-address-functions.md @@ -248,7 +248,7 @@ SELECT IPv6CIDRToRange(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32); ## toIPv4(string) -An alias to `IPv4StringToNum()` that takes a string form of IPv4 address and returns value of [IPv4](../../sql-reference/data-types/domains/ipv4.md) type, which is binary equal to value returned by `IPv4StringToNum()`. +An alias to `IPv4StringToNum()` that takes a string form of IPv4 address and returns value of [IPv4](../../sql-reference/data-types/ipv4.md) type, which is binary equal to value returned by `IPv4StringToNum()`. ``` sql WITH @@ -296,7 +296,7 @@ Same as `toIPv6`, but if the IPv6 address has an invalid format, it returns null ## toIPv6 -Converts a string form of IPv6 address to [IPv6](../../sql-reference/data-types/domains/ipv6.md) type. If the IPv6 address has an invalid format, returns an empty value. +Converts a string form of IPv6 address to [IPv6](../../sql-reference/data-types/ipv6.md) type. If the IPv6 address has an invalid format, returns an empty value. Similar to [IPv6StringToNum](#ipv6stringtonums) function, which converts IPv6 address to binary format. If the input string contains a valid IPv4 address, then the IPv6 equivalent of the IPv4 address is returned. @@ -315,7 +315,7 @@ toIPv6(string) - IP address. -Type: [IPv6](../../sql-reference/data-types/domains/ipv6.md). +Type: [IPv6](../../sql-reference/data-types/ipv6.md). **Examples** diff --git a/docs/redirects.txt b/docs/redirects.txt index cea138f7237..3abc8df2b7f 100644 --- a/docs/redirects.txt +++ b/docs/redirects.txt @@ -14,7 +14,7 @@ data_types/datetime.md sql-reference/data-types/datetime.md data_types/datetime64.md sql-reference/data-types/datetime64.md data_types/decimal.md sql-reference/data-types/decimal.md data_types/domains/ipv4.md sql-reference/data-types/domains/ipv4.md -data_types/domains/ipv6.md sql-reference/data-types/domains/ipv6.md +data_types/domains/ipv6.md sql-reference/data-types/ipv6.md data_types/domains/overview.md sql-reference/data-types/domains/overview.md data_types/enum.md sql-reference/data-types/enum.md data_types/fixedstring.md sql-reference/data-types/fixedstring.md @@ -162,7 +162,7 @@ interfaces/third-party/client_libraries.md interfaces/third-party/client-librari interfaces/third-party_client_libraries.md interfaces/third-party/client-libraries.md interfaces/third-party_gui.md interfaces/third-party/gui.md interfaces/third_party/index.md interfaces/third-party/index.md -introduction/index.md +introduction/index.md introduction/distinctive_features.md introduction/distinctive-features.md introduction/features_considered_disadvantages.md introduction/distinctive-features.md introduction/possible_silly_questions.md faq/general.md @@ -305,8 +305,10 @@ sql_reference/data_types/datetime.md sql-reference/data-types/datetime.md sql_reference/data_types/datetime64.md sql-reference/data-types/datetime64.md sql_reference/data_types/decimal.md sql-reference/data-types/decimal.md sql_reference/data_types/domains/index.md sql-reference/data-types/domains/index.md -sql_reference/data_types/domains/ipv4.md sql-reference/data-types/domains/ipv4.md -sql_reference/data_types/domains/ipv6.md sql-reference/data-types/domains/ipv6.md +sql_reference/data_types/domains/ipv4.md sql-reference/data-types/ipv4.md +sql_reference/data-types/domains/ipv4.md sql-reference/data-types/ipv4.md +sql_reference/data_types/domains/ipv6.md sql-reference/data-types/ipv6.md +sql_reference/data-types/domains/ipv6.md sql-reference/data-types/ipv6.md sql_reference/data_types/domains/overview.md sql-reference/data-types/domains/overview.md sql_reference/data_types/enum.md sql-reference/data-types/enum.md sql_reference/data_types/fixedstring.md sql-reference/data-types/fixedstring.md From 2923d57757cd28dde82d8edd762c1912129a68e4 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 15:37:32 -0300 Subject: [PATCH 1093/2223] Update redirects.txt --- docs/redirects.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/redirects.txt b/docs/redirects.txt index 3abc8df2b7f..98d6f6b8f7c 100644 --- a/docs/redirects.txt +++ b/docs/redirects.txt @@ -13,7 +13,7 @@ data_types/date.md sql-reference/data-types/date.md data_types/datetime.md sql-reference/data-types/datetime.md data_types/datetime64.md sql-reference/data-types/datetime64.md data_types/decimal.md sql-reference/data-types/decimal.md -data_types/domains/ipv4.md sql-reference/data-types/domains/ipv4.md +data_types/domains/ipv4.md sql-reference/data-types/ipv4.md data_types/domains/ipv6.md sql-reference/data-types/ipv6.md data_types/domains/overview.md sql-reference/data-types/domains/overview.md data_types/enum.md sql-reference/data-types/enum.md From 6fe0aa531e28c27287fb5b6f57536ede0016b20a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 16:18:53 -0300 Subject: [PATCH 1094/2223] Update summap.md --- .../aggregate-functions/reference/summap.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/summap.md b/docs/en/sql-reference/aggregate-functions/reference/summap.md index 1acfde3783a..d63e8b81716 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/summap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/summap.md @@ -5,7 +5,9 @@ sidebar_position: 141 # sumMap -Syntax: `sumMap(key, value)` or `sumMap(Tuple(key, value))` +Syntax: [sumMap(key , value )](../../data-types/array.md) or [sumMap(Tuple(key , value ))](../../data-types/tuple.md) + +Alias: `sumMappedArrays`. Totals the `value` array according to the keys specified in the `key` array. @@ -27,6 +29,7 @@ CREATE TABLE sum_map( ), statusMapTuple Tuple(Array(Int32), Array(Int32)) ) ENGINE = Log; + INSERT INTO sum_map VALUES ('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10], ([1, 2, 3], [10, 10, 10])), ('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10], ([3, 4, 5], [10, 10, 10])), @@ -47,3 +50,7 @@ GROUP BY timeslot │ 2000-01-01 00:01:00 │ ([4,5,6,7,8],[10,10,20,10,10]) │ ([4,5,6,7,8],[10,10,20,10,10]) │ └─────────────────────┴──────────────────────────────────────────────┴────────────────────────────────┘ ``` + +**See Also** + +- [-Map combinator for Map datatype](../combinators.md#-map) From 424a043c326cb8451bfc0da23fc2d1df2385fd4e Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 16:22:26 -0300 Subject: [PATCH 1095/2223] Update map.md --- docs/en/sql-reference/data-types/map.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/sql-reference/data-types/map.md b/docs/en/sql-reference/data-types/map.md index 0ea183d73d8..e0c8b98f9f8 100644 --- a/docs/en/sql-reference/data-types/map.md +++ b/docs/en/sql-reference/data-types/map.md @@ -108,6 +108,7 @@ Result: - [map()](../../sql-reference/functions/tuple-map-functions.md#function-map) function - [CAST()](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function +- [-Map combinator for Map datatype](../aggregate-functions/combinators.md#-map) ## Related content From 7c64b1f26634acc8e93dd65c00b8b0dfb6419f1d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 16:32:22 -0300 Subject: [PATCH 1096/2223] Update combinators.md --- .../aggregate-functions/combinators.md | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index e1db5d8d23e..fd693430064 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -30,7 +30,34 @@ Example 2: `uniqArray(arr)` – Counts the number of unique elements in all ‘a The -Map suffix can be appended to any aggregate function. This will create an aggregate function which gets Map type as an argument, and aggregates values of each key of the map separately using the specified aggregate function. The result is also of a Map type. -Examples: `sumMap(map(1,1))`, `avgMap(map('a', 1))`. +**Example** + +```sql +CREATE TABLE map_map( + date Date, + timeslot DateTime, + status Map(String, UInt64) +) ENGINE = Log; + +INSERT INTO map_map VALUES + ('2000-01-01', '2000-01-01 00:00:00', (['a', 'b', 'c'], [10, 10, 10])), + ('2000-01-01', '2000-01-01 00:00:00', (['c', 'd', 'e'], [10, 10, 10])), + ('2000-01-01', '2000-01-01 00:01:00', (['d', 'e', 'f'], [10, 10, 10])), + ('2000-01-01', '2000-01-01 00:01:00', (['f', 'g', 'g'], [10, 10, 10])); + +SELECT + timeslot, + sumMap(status), + avgMap(status), + minMap(status) +FROM map_map +GROUP BY timeslot; + +┌────────────timeslot─┬─sumMap(status)───────────────────────┬─avgMap(status)───────────────────────┬─minMap(status)───────────────────────┐ +│ 2000-01-01 00:00:00 │ {'a':10,'b':10,'c':20,'d':10,'e':10} │ {'a':10,'b':10,'c':10,'d':10,'e':10} │ {'a':10,'b':10,'c':10,'d':10,'e':10} │ +│ 2000-01-01 00:01:00 │ {'d':10,'e':10,'f':20,'g':20} │ {'d':10,'e':10,'f':10,'g':10} │ {'d':10,'e':10,'f':10,'g':10} │ +└─────────────────────┴──────────────────────────────────────┴──────────────────────────────────────┴──────────────────────────────────────┘ +``` ## -SimpleState From d81884be82bf32440cbaae3f63349716898aa1e3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:13:25 -0300 Subject: [PATCH 1097/2223] Update summap.md --- docs/en/sql-reference/aggregate-functions/reference/summap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/summap.md b/docs/en/sql-reference/aggregate-functions/reference/summap.md index d63e8b81716..0725b80c2a2 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/summap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/summap.md @@ -5,7 +5,7 @@ sidebar_position: 141 # sumMap -Syntax: [sumMap(key , value )](../../data-types/array.md) or [sumMap(Tuple(key , value ))](../../data-types/tuple.md) +Syntax: `sumMap(key , value )` [Array](../../data-types/array.md) or `sumMap(Tuple(key , value ))` [Tuple](../../data-types/tuple.md) Alias: `sumMappedArrays`. From d93ee14e328ce0c8026a1d94a4233b9905fed793 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:15:16 -0300 Subject: [PATCH 1098/2223] Update summap.md --- docs/en/sql-reference/aggregate-functions/reference/summap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/summap.md b/docs/en/sql-reference/aggregate-functions/reference/summap.md index 0725b80c2a2..aa12fad9559 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/summap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/summap.md @@ -5,7 +5,7 @@ sidebar_position: 141 # sumMap -Syntax: `sumMap(key , value )` [Array](../../data-types/array.md) or `sumMap(Tuple(key , value ))` [Tuple](../../data-types/tuple.md) +Syntax: `sumMap(key <[Array](../../data-types/array.md)>, value <[Array](../../data-types/array.md)>)` or `sumMap([Tuple](../../data-types/tuple.md)(key , value ))` Alias: `sumMappedArrays`. From 7ca719c73a41f54c3b2032297e8d0b3f9f6dd7a2 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:19:07 -0300 Subject: [PATCH 1099/2223] Update summap.md --- docs/en/sql-reference/aggregate-functions/reference/summap.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/summap.md b/docs/en/sql-reference/aggregate-functions/reference/summap.md index aa12fad9559..baa25edc250 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/summap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/summap.md @@ -5,7 +5,9 @@ sidebar_position: 141 # sumMap -Syntax: `sumMap(key <[Array](../../data-types/array.md)>, value <[Array](../../data-types/array.md)>)` or `sumMap([Tuple](../../data-types/tuple.md)(key , value ))` +Syntax: `sumMap(key , value )` [Array type](../../data-types/array.md) or `sumMap(Tuple(key , value ))` [Tuple type](../../data-types/tuple.md). + +Arguments: Alias: `sumMappedArrays`. From 0d9728410fd7631ff0e0755c749024de5b7ffdb1 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:31:12 -0300 Subject: [PATCH 1100/2223] Update first_value.md --- .../reference/first_value.md | 35 ++++++++++++++++--- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/first_value.md b/docs/en/sql-reference/aggregate-functions/reference/first_value.md index f343ca3f66c..15e0b113afd 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/first_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/first_value.md @@ -6,24 +6,32 @@ sidebar_position: 7 # first_value Selects the first encountered value, similar to `any`, but could accept NULL. +Mostly it should be used with [Window Functions](../../window-functions.md). +Without Window Functions the result will be random if the source stream is not ordered. ## examples ```sql -insert into test_data (a,b) values (1,null), (2,3), (4, 5), (6,null) +CREATE TABLE test_data +( + a Int64, + b Nullable(Int64) +) +ENGINE = Memory; + +INSERT INTO test_data (a, b) FORMAT Values (1,null), (2,3), (4, 5), (6,null); ``` ### example1 The NULL value is ignored at default. ```sql -select first_value(b) from test_data +select first_value(b) from test_data; ``` ```text ┌─first_value_ignore_nulls(b)─┐ │ 3 │ └─────────────────────────────┘ - ``` ### example2 @@ -36,7 +44,6 @@ select first_value(b) ignore nulls from test_data ┌─first_value_ignore_nulls(b)─┐ │ 3 │ └─────────────────────────────┘ - ``` ### example3 @@ -46,10 +53,28 @@ select first_value(b) respect nulls from test_data ``` ```text - ┌─first_value_respect_nulls(b)─┐ │ ᴺᵁᴸᴸ │ └──────────────────────────────┘ ``` +### example4 +Stabilized result using the sub-query with `ORDER BY`. +```sql +SELECT + first_value_respect_nulls(b), + first_value(b) +FROM +( + SELECT * + FROM test_data + ORDER BY a ASC +) +``` + +```text +┌─first_value_respect_nulls(b)─┬─first_value(b)─┐ +│ ᴺᵁᴸᴸ │ 3 │ +└──────────────────────────────┴────────────────┘ +``` From 2e26e84909098c97eab553eb7fcfa98b5a92bfae Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:34:31 -0300 Subject: [PATCH 1101/2223] Update last_value.md --- .../reference/last_value.md | 32 +++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/last_value.md b/docs/en/sql-reference/aggregate-functions/reference/last_value.md index 7b6e14e4a55..77b4f3d1b60 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/last_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/last_value.md @@ -6,12 +6,20 @@ sidebar_position: 8 # last_value Selects the last encountered value, similar to `anyLast`, but could accept NULL. - +Mostly it should be used with [Window Functions](../../window-functions.md). +Without Window Functions the result will be random if the source stream is not ordered. ## examples ```sql -insert into test_data (a,b) values (1,null), (2,3), (4, 5), (6,null) +CREATE TABLE test_data +( + a Int64, + b Nullable(Int64) +) +ENGINE = Memory; + +INSERT INTO test_data (a, b) Values (1,null), (2,3), (4, 5), (6,null) ``` ### example1 @@ -50,4 +58,24 @@ select last_value(b) respect nulls from test_data └─────────────────────────────┘ ``` +### example4 +Stabilized result using the sub-query with `ORDER BY`. +```sql +SELECT + last_value_respect_nulls(b), + last_value(b) +FROM +( + SELECT * + FROM test_data + ORDER BY a ASC +) +``` + +```text +┌─last_value_respect_nulls(b)─┬─last_value(b)─┐ +│ ᴺᵁᴸᴸ │ 5 │ +└─────────────────────────────┴───────────────┘ +``` + From bd7a593dd3ae4447f23c8658a04fb79d164b9d84 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:34:46 -0300 Subject: [PATCH 1102/2223] Update first_value.md --- .../sql-reference/aggregate-functions/reference/first_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/first_value.md b/docs/en/sql-reference/aggregate-functions/reference/first_value.md index 15e0b113afd..6b764ec5739 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/first_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/first_value.md @@ -19,7 +19,7 @@ CREATE TABLE test_data ) ENGINE = Memory; -INSERT INTO test_data (a, b) FORMAT Values (1,null), (2,3), (4, 5), (6,null); +INSERT INTO test_data (a, b) Values (1,null), (2,3), (4, 5), (6,null); ``` ### example1 From fb21a6907a3eef7a054b44f9c81c5bc3a05f5cb7 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:38:05 -0300 Subject: [PATCH 1103/2223] Update last_value.md --- .../sql-reference/aggregate-functions/reference/last_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/last_value.md b/docs/en/sql-reference/aggregate-functions/reference/last_value.md index 77b4f3d1b60..21a86a5f130 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/last_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/last_value.md @@ -6,7 +6,7 @@ sidebar_position: 8 # last_value Selects the last encountered value, similar to `anyLast`, but could accept NULL. -Mostly it should be used with [Window Functions](../../window-functions.md). +Mostly it should be used with [Window Functions](../../window-functions/index.md). Without Window Functions the result will be random if the source stream is not ordered. ## examples From bcd89cbbf28c79e831f3e39a422319e5b6ea5915 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 17:38:38 -0300 Subject: [PATCH 1104/2223] Update first_value.md --- .../sql-reference/aggregate-functions/reference/first_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/first_value.md b/docs/en/sql-reference/aggregate-functions/reference/first_value.md index 6b764ec5739..c1965b23fe3 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/first_value.md +++ b/docs/en/sql-reference/aggregate-functions/reference/first_value.md @@ -6,7 +6,7 @@ sidebar_position: 7 # first_value Selects the first encountered value, similar to `any`, but could accept NULL. -Mostly it should be used with [Window Functions](../../window-functions.md). +Mostly it should be used with [Window Functions](../../window-functions/index.md). Without Window Functions the result will be random if the source stream is not ordered. ## examples From bf127f4e1e3a08de7ae822d0b53d25ad80899efa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Jun 2023 21:31:43 +0200 Subject: [PATCH 1105/2223] MSan support for Rust Previously you have to unpoison memory from the Rust, however Rust does supports MSan, so let's simply use it. But for this we need nightly Rust and recompile standard library. Signed-off-by: Azat Khuzhin --- docker/packager/binary/Dockerfile | 4 +++- rust/.cargo/config.toml.in | 7 +++++++ rust/BLAKE3/include/blake3.h | 2 -- rust/BLAKE3/src/lib.rs | 25 ------------------------- rust/CMakeLists.txt | 10 ++++++++++ src/Functions/FunctionsHashing.h | 7 +------ 6 files changed, 21 insertions(+), 34 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index fa860b2207f..dd21c8552d3 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -46,10 +46,12 @@ ENV CXX=clang++-${LLVM_VERSION} # Rust toolchain and libraries ENV RUSTUP_HOME=/rust/rustup ENV CARGO_HOME=/rust/cargo -ENV PATH="/rust/cargo/env:${PATH}" ENV PATH="/rust/cargo/bin:${PATH}" RUN curl https://sh.rustup.rs -sSf | bash -s -- -y && \ chmod 777 -R /rust && \ + rustup toolchain install nightly && \ + rustup default nightly && \ + rustup component add rust-src && \ rustup target add aarch64-unknown-linux-gnu && \ rustup target add x86_64-apple-darwin && \ rustup target add x86_64-unknown-freebsd && \ diff --git a/rust/.cargo/config.toml.in b/rust/.cargo/config.toml.in index a1dd966117b..db83145d449 100644 --- a/rust/.cargo/config.toml.in +++ b/rust/.cargo/config.toml.in @@ -1,3 +1,10 @@ [env] CFLAGS = "@RUST_CFLAGS@" CXXFLAGS = "@RUST_CXXFLAGS@" + +[build] +rustflags = @RUSTFLAGS@ +rustdocflags = @RUSTFLAGS@ + +[unstable] +@RUST_CARGO_BUILD_STD@ diff --git a/rust/BLAKE3/include/blake3.h b/rust/BLAKE3/include/blake3.h index 85572506d43..5dc7d5bd902 100644 --- a/rust/BLAKE3/include/blake3.h +++ b/rust/BLAKE3/include/blake3.h @@ -8,8 +8,6 @@ extern "C" { char *blake3_apply_shim(const char *begin, uint32_t _size, uint8_t *out_char_data); -char *blake3_apply_shim_msan_compat(const char *begin, uint32_t size, uint8_t *out_char_data); - void blake3_free_char_pointer(char *ptr_to_free); } // extern "C" diff --git a/rust/BLAKE3/src/lib.rs b/rust/BLAKE3/src/lib.rs index 2b54787589f..011145d2f71 100644 --- a/rust/BLAKE3/src/lib.rs +++ b/rust/BLAKE3/src/lib.rs @@ -3,7 +3,6 @@ extern crate libc; use std::ffi::{CStr, CString}; use std::os::raw::c_char; -use std::mem; #[no_mangle] pub unsafe extern "C" fn blake3_apply_shim( @@ -24,30 +23,6 @@ pub unsafe extern "C" fn blake3_apply_shim( std::ptr::null_mut() } -#[no_mangle] -pub unsafe extern "C" fn blake3_apply_shim_msan_compat( - mut begin: *const c_char, - size: u32, - out_char_data: *mut u8, -) -> *mut c_char { - if begin.is_null() { - let err_str = CString::new("input was a null pointer").unwrap(); - return err_str.into_raw(); - } - libc::memset(out_char_data as *mut libc::c_void, 0, mem::size_of::()); - let mut hasher = blake3::Hasher::new(); - let mut vec = Vec::::new(); - for _ in 0..size { - vec.push(*begin as u8); - begin = begin.add(1); - } - let input_res = vec.as_mut_slice(); - hasher.update(input_res); - let mut reader = hasher.finalize_xof(); - reader.fill(std::slice::from_raw_parts_mut(out_char_data, blake3::OUT_LEN)); - std::ptr::null_mut() -} - // Freeing memory according to docs: https://doc.rust-lang.org/std/ffi/struct.CString.html#method.into_raw #[no_mangle] pub unsafe extern "C" fn blake3_free_char_pointer(ptr_to_free: *mut c_char) { diff --git a/rust/CMakeLists.txt b/rust/CMakeLists.txt index d229894791a..6700ead9786 100644 --- a/rust/CMakeLists.txt +++ b/rust/CMakeLists.txt @@ -14,8 +14,18 @@ macro(configure_rustc) set(RUST_CFLAGS "${RUST_CFLAGS} --sysroot ${CMAKE_SYSROOT}") endif() + set(RUSTFLAGS "[]") + set(RUST_CARGO_BUILD_STD "") + # For more info: https://doc.rust-lang.org/beta/unstable-book/compiler-flags/sanitizer.html#memorysanitizer + if (SANITIZE STREQUAL "memory") + set(RUST_CARGO_BUILD_STD "build-std = [\"std\", \"panic_abort\", \"core\", \"alloc\"]") + set(RUSTFLAGS "[\"-Zsanitizer=memory\", \"-Zsanitizer-memory-track-origins\"]") + endif() + message(STATUS "RUST_CFLAGS: ${RUST_CFLAGS}") message(STATUS "RUST_CXXFLAGS: ${RUST_CXXFLAGS}") + message(STATUS "RUSTFLAGS: ${RUSTFLAGS}") + message(STATUS "RUST_CARGO_BUILD_STD: ${RUST_CARGO_BUILD_STD}") # NOTE: requires RW access for the source dir configure_file("${CMAKE_CURRENT_SOURCE_DIR}/.cargo/config.toml.in" "${CMAKE_CURRENT_SOURCE_DIR}/.cargo/config.toml" @ONLY) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 32e3fbbd4ea..a4d4fbd085d 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -816,12 +816,7 @@ struct ImplBLAKE3 #else static void apply(const char * begin, const size_t size, unsigned char* out_char_data) { -# if defined(MEMORY_SANITIZER) - auto err_msg = blake3_apply_shim_msan_compat(begin, safe_cast(size), out_char_data); - __msan_unpoison(out_char_data, length); -# else - auto err_msg = blake3_apply_shim(begin, safe_cast(size), out_char_data); -# endif + auto err_msg = blake3_apply_shim(begin, safe_cast(size), out_char_data); if (err_msg != nullptr) { auto err_st = std::string(err_msg); From 045573e92565a6a58a7eae80cc11bb686807e9ee Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 18:10:34 -0300 Subject: [PATCH 1106/2223] Update datetime64.md --- .../en/sql-reference/data-types/datetime64.md | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index 2d4035831fa..da3d9dc4f65 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -63,7 +63,7 @@ SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Asia/ ``` text ┌───────────────timestamp─┬─event_id─┐ -│ 2019-01-01 00:00:00.000 │ 2 │ +│ 2019-01-01 00:00:00.000 │ 3 │ └─────────────────────────┴──────────┘ ``` @@ -75,8 +75,8 @@ SELECT * FROM dt WHERE timestamp = toDateTime64(1546300800.123, 3); ``` text ┌───────────────timestamp─┬─event_id─┐ -│ 2019-01-01 00:00:00.123 │ 1 │ -│ 2019-01-01 00:00:00.123 │ 2 │ +│ 2019-01-01 03:00:00.123 │ 1 │ +│ 2019-01-01 03:00:00.123 │ 2 │ └─────────────────────────┴──────────┘ ``` @@ -91,7 +91,7 @@ SELECT toDateTime64(now(), 3, 'Asia/Istanbul') AS column, toTypeName(column) AS ``` text ┌──────────────────column─┬─x──────────────────────────────┐ -│ 2019-10-16 04:12:04.000 │ DateTime64(3, 'Asia/Istanbul') │ +│ 2023-06-05 00:09:52.000 │ DateTime64(3, 'Asia/Istanbul') │ └─────────────────────────┴────────────────────────────────┘ ``` @@ -100,13 +100,14 @@ SELECT toDateTime64(now(), 3, 'Asia/Istanbul') AS column, toTypeName(column) AS ``` sql SELECT toDateTime64(timestamp, 3, 'Europe/London') as lon_time, -toDateTime64(timestamp, 3, 'Asia/Istanbul') as mos_time +toDateTime64(timestamp, 3, 'Asia/Istanbul') as istanbul_time FROM dt; ``` ``` text -┌───────────────lon_time──┬────────────────mos_time─┐ -│ 2019-01-01 00:00:00.000 │ 2019-01-01 03:00:00.000 │ +┌────────────────lon_time─┬───────────istanbul_time─┐ +│ 2019-01-01 00:00:00.123 │ 2019-01-01 03:00:00.123 │ +│ 2019-01-01 00:00:00.123 │ 2019-01-01 03:00:00.123 │ │ 2018-12-31 21:00:00.000 │ 2019-01-01 00:00:00.000 │ └─────────────────────────┴─────────────────────────┘ ``` @@ -115,10 +116,9 @@ FROM dt; - [Type conversion functions](../../sql-reference/functions/type-conversion-functions.md) - [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md) -- [Functions for working with arrays](../../sql-reference/functions/array-functions.md) -- [The `date_time_input_format` setting](../../operations/settings/settings.md#settings-date_time_input_format) -- [The `date_time_output_format` setting](../../operations/settings/settings.md#settings-date_time_output_format) +- [The `date_time_input_format` setting](../../operations/settings/formats.md#date_time_input_format) +- [The `date_time_output_format` setting](../../operations/settings/formats.md#date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) -- [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime) +- [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-for-working-with-dates-and-times) - [`Date` data type](../../sql-reference/data-types/date.md) - [`DateTime` data type](../../sql-reference/data-types/datetime.md) From 136efd68257ccaac503b2e47957803e283165afc Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 18:58:24 -0300 Subject: [PATCH 1107/2223] Update datetime64.md --- docs/en/sql-reference/data-types/datetime64.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index da3d9dc4f65..793691850b1 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -116,8 +116,8 @@ FROM dt; - [Type conversion functions](../../sql-reference/functions/type-conversion-functions.md) - [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md) -- [The `date_time_input_format` setting](../../operations/settings/formats.md#date_time_input_format) -- [The `date_time_output_format` setting](../../operations/settings/formats.md#date_time_output_format) +- [The `date_time_input_format` setting](../../operations/settings/settings-formats.md#date_time_input_format) +- [The `date_time_output_format` setting](../../operations/settings/settings-formats.md#date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-for-working-with-dates-and-times) - [`Date` data type](../../sql-reference/data-types/date.md) From aa35689cb10dbdbab0c8475a7f92b8978e6eb6b8 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 19:39:30 -0300 Subject: [PATCH 1108/2223] fix links in other lang-s --- docs/ru/operations/system-tables/query_log.md | 4 ++-- docs/ru/operations/system-tables/query_thread_log.md | 4 ++-- docs/ru/operations/system-tables/session_log.md | 2 +- docs/ru/operations/system-tables/zookeeper_log.md | 2 +- docs/ru/sql-reference/data-types/{domains => }/ipv4.md | 2 +- docs/ru/sql-reference/data-types/{domains => }/ipv6.md | 2 +- docs/ru/sql-reference/functions/ip-address-functions.md | 4 ++-- docs/zh/operations/system-tables/query_log.md | 4 ++-- docs/zh/operations/system-tables/query_thread_log.md | 4 ++-- docs/zh/operations/system-tables/zookeeper_log.md | 2 +- docs/zh/sql-reference/data-types/{domains => }/ipv4.md | 2 +- docs/zh/sql-reference/data-types/{domains => }/ipv6.md | 2 +- 12 files changed, 17 insertions(+), 17 deletions(-) rename docs/ru/sql-reference/data-types/{domains => }/ipv4.md (98%) rename docs/ru/sql-reference/data-types/{domains => }/ipv6.md (98%) rename docs/zh/sql-reference/data-types/{domains => }/ipv4.md (98%) rename docs/zh/sql-reference/data-types/{domains => }/ipv6.md (98%) diff --git a/docs/ru/operations/system-tables/query_log.md b/docs/ru/operations/system-tables/query_log.md index a55528bd829..8f858c14fb1 100644 --- a/docs/ru/operations/system-tables/query_log.md +++ b/docs/ru/operations/system-tables/query_log.md @@ -69,11 +69,11 @@ ClickHouse не удаляет данные из таблица автомати - 0 — запрос был инициирован другим запросом при выполнении распределенного запроса. - `user` ([String](../../sql-reference/data-types/string.md)) — пользователь, запустивший текущий запрос. - `query_id` ([String](../../sql-reference/data-types/string.md)) — ID запроса. -- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP адрес, с которого пришел запрос. +- `address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP адрес, с которого пришел запрос. - `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — порт, с которого клиент сделал запрос - `initial_user` ([String](../../sql-reference/data-types/string.md)) — пользователь, запустивший первоначальный запрос (для распределенных запросов). - `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — ID родительского запроса. -- `initial_address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP адрес, с которого пришел родительский запрос. +- `initial_address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP адрес, с которого пришел родительский запрос. - `initial_port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — порт, с которого клиент сделал родительский запрос. - `initial_query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время начала обработки запроса (для распределенных запросов). - `initial_query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — время начала обработки запроса с точностью до микросекунд (для распределенных запросов). diff --git a/docs/ru/operations/system-tables/query_thread_log.md b/docs/ru/operations/system-tables/query_thread_log.md index c9aabb02cad..1a256e1657a 100644 --- a/docs/ru/operations/system-tables/query_thread_log.md +++ b/docs/ru/operations/system-tables/query_thread_log.md @@ -39,11 +39,11 @@ ClickHouse не удаляет данные из таблицы автомати - 0 — запрос был инициирован другим запросом при распределенном запросе. - `user` ([String](../../sql-reference/data-types/string.md)) — пользователь, запустивший текущий запрос. - `query_id` ([String](../../sql-reference/data-types/string.md)) — ID запроса. -- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP адрес, с которого пришел запрос. +- `address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP адрес, с которого пришел запрос. - `port` ([UInt16](../../sql-reference/data-types/int-uint.md#uint-ranges)) — порт, с которого пришел запрос. - `initial_user` ([String](../../sql-reference/data-types/string.md)) — пользователь, запустивший первоначальный запрос (для распределенных запросов). - `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — ID родительского запроса. -- `initial_address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP адрес, с которого пришел родительский запрос. +- `initial_address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP адрес, с которого пришел родительский запрос. - `initial_port` ([UInt16](../../sql-reference/data-types/int-uint.md#uint-ranges)) — порт, пришел родительский запрос. - `interface` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — интерфейс, с которого ушёл запрос. Возможные значения: - 1 — TCP. diff --git a/docs/ru/operations/system-tables/session_log.md b/docs/ru/operations/system-tables/session_log.md index 1f313e7815a..5849cb51ab4 100644 --- a/docs/ru/operations/system-tables/session_log.md +++ b/docs/ru/operations/system-tables/session_log.md @@ -27,7 +27,7 @@ slug: /ru/operations/system-tables/session_log - `profiles` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — список профилей, установленных для всех ролей и (или) пользователей. - `roles` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — список ролей, к которым применяется данный профиль. - `settings` ([Array](../../sql-reference/data-types/array.md)([Tuple](../../sql-reference/data-types/tuple.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md), [String](../../sql-reference/data-types/string.md)))) — настройки, которые были изменены при входе или выходе клиента из системы. -- `client_address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP-адрес, который использовался для входа или выхода из системы. +- `client_address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP-адрес, который использовался для входа или выхода из системы. - `client_port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — порт клиента, который использовался для входа или выхода из системы. - `interface` ([Enum8](../../sql-reference/data-types/enum.md)) — интерфейс, с которого был инициирован вход в систему. Возможные значения: - `TCP` diff --git a/docs/ru/operations/system-tables/zookeeper_log.md b/docs/ru/operations/system-tables/zookeeper_log.md index ccbdd5110ad..9874cb3a269 100644 --- a/docs/ru/operations/system-tables/zookeeper_log.md +++ b/docs/ru/operations/system-tables/zookeeper_log.md @@ -15,7 +15,7 @@ slug: /ru/operations/system-tables/zookeeper_log - `Finalize` — соединение разорвано, ответ не получен. - `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата, когда произошло событие. - `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — дата и время, когда произошло событие. -- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP адрес сервера ZooKeeper, с которого был сделан запрос. +- `address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — IP адрес сервера ZooKeeper, с которого был сделан запрос. - `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — порт сервера ZooKeeper, с которого был сделан запрос. - `session_id` ([Int64](../../sql-reference/data-types/int-uint.md)) — идентификатор сессии, который сервер ZooKeeper создает для каждого соединения. - `xid` ([Int32](../../sql-reference/data-types/int-uint.md)) — идентификатор запроса внутри сессии. Обычно это последовательный номер запроса, одинаковый у строки запроса и у парной строки `response`/`finalize`. diff --git a/docs/ru/sql-reference/data-types/domains/ipv4.md b/docs/ru/sql-reference/data-types/ipv4.md similarity index 98% rename from docs/ru/sql-reference/data-types/domains/ipv4.md rename to docs/ru/sql-reference/data-types/ipv4.md index 57a19e282ae..8d308785eea 100644 --- a/docs/ru/sql-reference/data-types/domains/ipv4.md +++ b/docs/ru/sql-reference/data-types/ipv4.md @@ -1,5 +1,5 @@ --- -slug: /ru/sql-reference/data-types/domains/ipv4 +slug: /ru/sql-reference/data-types/ipv4 sidebar_position: 59 sidebar_label: IPv4 --- diff --git a/docs/ru/sql-reference/data-types/domains/ipv6.md b/docs/ru/sql-reference/data-types/ipv6.md similarity index 98% rename from docs/ru/sql-reference/data-types/domains/ipv6.md rename to docs/ru/sql-reference/data-types/ipv6.md index fdfb26f68c1..808068ce90a 100644 --- a/docs/ru/sql-reference/data-types/domains/ipv6.md +++ b/docs/ru/sql-reference/data-types/ipv6.md @@ -1,5 +1,5 @@ --- -slug: /ru/sql-reference/data-types/domains/ipv6 +slug: /ru/sql-reference/data-types/ipv6 sidebar_position: 60 sidebar_label: IPv6 --- diff --git a/docs/ru/sql-reference/functions/ip-address-functions.md b/docs/ru/sql-reference/functions/ip-address-functions.md index 96d4b737c88..d1a72b82b67 100644 --- a/docs/ru/sql-reference/functions/ip-address-functions.md +++ b/docs/ru/sql-reference/functions/ip-address-functions.md @@ -265,7 +265,7 @@ SELECT ## toIPv6 {#toipv6string} -Приводит строку с адресом в формате IPv6 к типу [IPv6](../../sql-reference/data-types/domains/ipv6.md). Возвращает пустое значение, если входящая строка не является корректным IP адресом. +Приводит строку с адресом в формате IPv6 к типу [IPv6](../../sql-reference/data-types/ipv6.md). Возвращает пустое значение, если входящая строка не является корректным IP адресом. Похоже на функцию [IPv6StringToNum](#ipv6stringtonums), которая представляет адрес IPv6 в двоичном виде. Если входящая строка содержит корректный IPv4 адрес, функция возвращает его IPv6 эквивалент. @@ -284,7 +284,7 @@ toIPv6(string) - IP адрес. -Тип: [IPv6](../../sql-reference/data-types/domains/ipv6.md). +Тип: [IPv6](../../sql-reference/data-types/ipv6.md). **Примеры** diff --git a/docs/zh/operations/system-tables/query_log.md b/docs/zh/operations/system-tables/query_log.md index 7149282dfcc..0ba669906cb 100644 --- a/docs/zh/operations/system-tables/query_log.md +++ b/docs/zh/operations/system-tables/query_log.md @@ -60,11 +60,11 @@ ClickHouse不会自动从表中删除数据。更多详情请看 [introduction]( - 0 — 由另一个查询发起的,作为分布式查询的一部分. - `user` ([String](../../sql-reference/data-types/string.md)) — 发起查询的用户. - `query_id` ([String](../../sql-reference/data-types/string.md)) — 查询ID. -- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — 发起查询的客户端IP地址. +- `address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — 发起查询的客户端IP地址. - `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — 发起查询的客户端端口. - `initial_user` ([String](../../sql-reference/data-types/string.md)) — 初始查询的用户名(用于分布式查询执行). - `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — 运行初始查询的ID(用于分布式查询执行). -- `initial_address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — 运行父查询的IP地址. +- `initial_address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — 运行父查询的IP地址. - `initial_port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — 发起父查询的客户端端口. - `interface` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 发起查询的接口. 可能的值: - 1 — TCP. diff --git a/docs/zh/operations/system-tables/query_thread_log.md b/docs/zh/operations/system-tables/query_thread_log.md index 8a41c1501a6..c4b7e2f1043 100644 --- a/docs/zh/operations/system-tables/query_thread_log.md +++ b/docs/zh/operations/system-tables/query_thread_log.md @@ -36,11 +36,11 @@ ClickHouse不会自动从表中删除数据。 欲了解更多详情,请参照 - 0 — 由其他查询发起的分布式查询。 - `user` ([字符串](../../sql-reference/data-types/string.md)) — 发起查询的用户名。 - `query_id` ([字符串](../../sql-reference/data-types/string.md)) — 查询的ID。 -- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — 发起查询的IP地址。 +- `address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — 发起查询的IP地址。 - `port` ([UInt16](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 发起查询的端口。 - `initial_user` ([字符串](../../sql-reference/data-types/string.md)) — 首次发起查询的用户名(对于分布式查询)。 - `initial_query_id` ([字符串](../../sql-reference/data-types/string.md)) — 首次发起查询的ID(对于分布式查询)。 -- `initial_address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — 发起该查询的父查询IP地址。 +- `initial_address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — 发起该查询的父查询IP地址。 - `initial_port` ([UInt16](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 发起该查询的父查询端口。 - `interface` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 发起查询的界面,可能的值: - 1 — TCP. diff --git a/docs/zh/operations/system-tables/zookeeper_log.md b/docs/zh/operations/system-tables/zookeeper_log.md index 59dcdaecdc1..ebc51a2e79d 100644 --- a/docs/zh/operations/system-tables/zookeeper_log.md +++ b/docs/zh/operations/system-tables/zookeeper_log.md @@ -15,7 +15,7 @@ slug: /zh/operations/system-tables/zookeeper_log - `Finalize` — 连接丢失, 未收到响应. - `event_date` ([Date](../../sql-reference/data-types/date.md)) — 事件发生的日期. - `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — 事件发生的日期和时间. -- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — 用于发出请求的 ZooKeeper 服务器的 IP 地址. +- `address` ([IPv6](../../sql-reference/data-types/ipv6.md)) — 用于发出请求的 ZooKeeper 服务器的 IP 地址. - `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — 用于发出请求的 ZooKeeper 服务器的端口. - `session_id` ([Int64](../../sql-reference/data-types/int-uint.md)) — ZooKeeper 服务器为每个连接设置的会话 ID. - `xid` ([Int32](../../sql-reference/data-types/int-uint.md)) — 会话中请求的 ID. 这通常是一个连续的请求编号. 请求行和配对的 `response`/`finalize` 行相同. diff --git a/docs/zh/sql-reference/data-types/domains/ipv4.md b/docs/zh/sql-reference/data-types/ipv4.md similarity index 98% rename from docs/zh/sql-reference/data-types/domains/ipv4.md rename to docs/zh/sql-reference/data-types/ipv4.md index 69e17b2f617..b89af974b87 100644 --- a/docs/zh/sql-reference/data-types/domains/ipv4.md +++ b/docs/zh/sql-reference/data-types/ipv4.md @@ -1,5 +1,5 @@ --- -slug: /zh/sql-reference/data-types/domains/ipv4 +slug: /zh/sql-reference/data-types/ipv4 --- ## IPv4 {#ipv4} diff --git a/docs/zh/sql-reference/data-types/domains/ipv6.md b/docs/zh/sql-reference/data-types/ipv6.md similarity index 98% rename from docs/zh/sql-reference/data-types/domains/ipv6.md rename to docs/zh/sql-reference/data-types/ipv6.md index 9dd88692c37..3896bb873d8 100644 --- a/docs/zh/sql-reference/data-types/domains/ipv6.md +++ b/docs/zh/sql-reference/data-types/ipv6.md @@ -1,5 +1,5 @@ --- -slug: /zh/sql-reference/data-types/domains/ipv6 +slug: /zh/sql-reference/data-types/ipv6 --- ## IPv6 {#ipv6} From 2e187e0a0eae7f0109c6af30bd6baad0e75c9b71 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 4 Jun 2023 20:12:35 -0300 Subject: [PATCH 1109/2223] try to fix redirect --- docs/redirects.txt | 2 -- docs/ru/sql-reference/data-types/ipv4.md | 27 +++--------------------- 2 files changed, 3 insertions(+), 26 deletions(-) diff --git a/docs/redirects.txt b/docs/redirects.txt index 98d6f6b8f7c..ddfc66aa48b 100644 --- a/docs/redirects.txt +++ b/docs/redirects.txt @@ -306,9 +306,7 @@ sql_reference/data_types/datetime64.md sql-reference/data-types/datetime64.md sql_reference/data_types/decimal.md sql-reference/data-types/decimal.md sql_reference/data_types/domains/index.md sql-reference/data-types/domains/index.md sql_reference/data_types/domains/ipv4.md sql-reference/data-types/ipv4.md -sql_reference/data-types/domains/ipv4.md sql-reference/data-types/ipv4.md sql_reference/data_types/domains/ipv6.md sql-reference/data-types/ipv6.md -sql_reference/data-types/domains/ipv6.md sql-reference/data-types/ipv6.md sql_reference/data_types/domains/overview.md sql-reference/data-types/domains/overview.md sql_reference/data_types/enum.md sql-reference/data-types/enum.md sql_reference/data_types/fixedstring.md sql-reference/data-types/fixedstring.md diff --git a/docs/ru/sql-reference/data-types/ipv4.md b/docs/ru/sql-reference/data-types/ipv4.md index 8d308785eea..5cb977c64c9 100644 --- a/docs/ru/sql-reference/data-types/ipv4.md +++ b/docs/ru/sql-reference/data-types/ipv4.md @@ -6,7 +6,7 @@ sidebar_label: IPv4 ## IPv4 {#ipv4} -`IPv4` — это домен, базирующийся на типе данных `UInt32` предназначенный для хранения адресов IPv4. Он обеспечивает компактное хранение данных с удобным для человека форматом ввода-вывода, и явно отображаемым типом данных в структуре таблицы. +IPv4-адреса. Хранится в 4 байтах как UInt32. ### Применение {#primenenie} @@ -57,27 +57,6 @@ SELECT toTypeName(from), hex(from) FROM hits LIMIT 1; └──────────────────┴───────────┘ ``` -Значения с доменным типом данных не преобразуются неявно в другие типы данных, кроме `UInt32`. -Если необходимо преобразовать значение типа `IPv4` в строку, то это необходимо делать явно с помощью функции `IPv4NumToString()`: +**См. также** -``` sql -SELECT toTypeName(s), IPv4NumToString(from) AS s FROM hits LIMIT 1; -``` - -``` text -┌─toTypeName(IPv4NumToString(from))─┬─s──────────────┐ -│ String │ 183.247.232.58 │ -└───────────────────────────────────┴────────────────┘ -``` - -Или приводить к типу данных `UInt32`: - -``` sql -SELECT toTypeName(i), CAST(from AS UInt32) AS i FROM hits LIMIT 1; -``` - -``` text -┌─toTypeName(CAST(from, 'UInt32'))─┬──────────i─┐ -│ UInt32 │ 3086477370 │ -└──────────────────────────────────┴────────────┘ -``` +- [Functions for Working with IPv4 and IPv6 Addresses](../functions/ip-address-functions.md) From e5c95add52ed86c56249fe85d8f7c02132736ae3 Mon Sep 17 00:00:00 2001 From: auxten Date: Mon, 5 Jun 2023 08:43:55 +0800 Subject: [PATCH 1110/2223] use old_size Co-authored-by: Alexey Milovidov --- src/IO/WriteBufferFromVector.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index c793a34b406..a2ecc34f1ab 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -86,7 +86,7 @@ private: size_t old_size = vector.size(); /// pos may not be equal to vector.data() + old_size, because WriteBuffer::next() can be used to flush data size_t pos_offset = pos - reinterpret_cast(vector.data()); - if (pos_offset == vector.size()) + if (pos_offset == old_size) { vector.resize(old_size * size_multiplier); } From 4234c4f36addd2607ecc16131ec67ef1089d10ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Jun 2023 02:51:11 +0200 Subject: [PATCH 1111/2223] Remove flaky test --- tests/integration/test_merge_tree_s3/test.py | 25 -------------------- 1 file changed, 25 deletions(-) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 7730bfcf7b2..2ccd517923a 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -739,31 +739,6 @@ def test_cache_with_full_disk_space(cluster, node_name): check_no_objects_after_drop(cluster, node_name=node_name) -@pytest.mark.parametrize("node_name", ["node"]) -def test_store_cleanup_disk_s3(cluster, node_name): - node = cluster.instances[node_name] - node.query("DROP TABLE IF EXISTS s3_test SYNC") - node.query( - "CREATE TABLE s3_test UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" - ) - node.query("INSERT INTO s3_test SELECT 1") - - node.stop_clickhouse(kill=True) - path_to_data = "/var/lib/clickhouse/" - node.exec_in_container(["rm", f"{path_to_data}/metadata/default/s3_test.sql"]) - node.start_clickhouse() - - node.wait_for_log_line( - "Removing unused directory", timeout=90, look_behind_lines=1000 - ) - node.wait_for_log_line("directories from store") - node.query( - "CREATE TABLE s3_test UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" - ) - node.query("INSERT INTO s3_test SELECT 1") - check_no_objects_after_drop(cluster) - - @pytest.mark.parametrize("node_name", ["node"]) def test_cache_setting_compatibility(cluster, node_name): node = cluster.instances[node_name] From 47379ac03965f4834bf6aaa00ce777dec731a3c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Jun 2023 03:58:42 +0300 Subject: [PATCH 1112/2223] Update build.sh --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index ee1011a9cd5..c0803c74147 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -11,7 +11,7 @@ ccache_status () { [ -O /build ] || git config --global --add safe.directory /build -if [ "$EXTRACT_TOOLCHAIN_DARWIN" = "1" ];then +if [ "$EXTRACT_TOOLCHAIN_DARWIN" = "1" ]; then mkdir -p /build/cmake/toolchain/darwin-x86_64 tar xJf /MacOSX11.0.sdk.tar.xz -C /build/cmake/toolchain/darwin-x86_64 --strip-components=1 ln -sf darwin-x86_64 /build/cmake/toolchain/darwin-aarch64 From 5fc8838b04d37d26207fff488bd60127f9eedaa8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Jun 2023 04:58:29 +0300 Subject: [PATCH 1113/2223] Update KeyCondition.cpp --- src/Storages/MergeTree/KeyCondition.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 923e5237420..16bd555092e 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -943,6 +943,13 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & return {field.columns, field.row_idx, result_idx}; } +/** When table's key has expression with these functions from a column, + * and when a column in a query is compared with a constant, such as: + * CREATE TABLE (x String) ORDER BY toDate(x) + * SELECT ... WHERE x LIKE 'Hello%' + * we want to apply the function to the constant for index analysis, + * but should modify it to pass on unparseable values. + */ static std::set date_time_parsing_functions = { "toDate", "toDate32", From 3c5bd78856d1848cd457a30c2d8320b3f65a41d9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Jun 2023 06:13:39 +0200 Subject: [PATCH 1114/2223] Fix typo --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index ab5820be90a..02ef7e6bebd 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -948,7 +948,7 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & * CREATE TABLE (x String) ORDER BY toDate(x) * SELECT ... WHERE x LIKE 'Hello%' * we want to apply the function to the constant for index analysis, - * but should modify it to pass on unparseable values. + * but should modify it to pass on unparsable values. */ static std::set date_time_parsing_functions = { "toDate", From c0f162c5b67b112691b8ef805bf2a56060441a0b Mon Sep 17 00:00:00 2001 From: johanngan Date: Fri, 26 May 2023 15:56:40 -0500 Subject: [PATCH 1115/2223] Add dictGetAll function for RegExpTreeDictionary This function outputs an array of attribute values from all regexp nodes that matched in a regexp tree dictionary. An optional final argument can be passed to limit the array size. --- docs/en/sql-reference/dictionaries/index.md | 63 ++++++++- .../functions/ext-dict-functions.md | 78 +++++++++++ src/Dictionaries/IDictionary.h | 45 +++++++ src/Dictionaries/RegExpTreeDictionary.cpp | 122 +++++++++++++++--- src/Dictionaries/RegExpTreeDictionary.h | 49 ++++++- .../FunctionsExternalDictionaries.cpp | 15 +++ src/Functions/FunctionsExternalDictionaries.h | 77 ++++++++--- ...04_regexp_dictionary_yaml_source.reference | 6 + .../02504_regexp_dictionary_yaml_source.sh | 48 +++++++ 9 files changed, 459 insertions(+), 44 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 43e9300c1ae..6c3d80683db 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2280,7 +2280,7 @@ This config consists of a list of regular expression tree nodes. Each node has t - The value of an attribute may contain **back references**, referring to capture groups of the matched regular expression. In the example, the value of attribute `version` in the first node consists of a back-reference `\1` to capture group `(\d+[\.\d]*)` in the regular expression. Back-reference numbers range from 1 to 9 and are written as `$1` or `\1` (for number 1). The back reference is replaced by the matched capture group during query execution. - **child nodes**: a list of children of a regexp tree node, each of which has its own attributes and (potentially) children nodes. String matching proceeds in a depth-first fashion. If a string matches a regexp node, the dictionary checks if it also matches the nodes' child nodes. If that is the case, the attributes of the deepest matching node are assigned. Attributes of a child node overwrite equally named attributes of parent nodes. The name of child nodes in YAML files can be arbitrary, e.g. `versions` in above example. -Regexp tree dictionaries only allow access using the functions `dictGet` and `dictGetOrDefault`. +Regexp tree dictionaries only allow access using the functions `dictGet`, `dictGetOrDefault`, and `dictGetAll`. Example: @@ -2300,6 +2300,67 @@ In this case, we first match the regular expression `\d+/tclwebkit(?:\d+[\.\d]*) With a powerful YAML configure file, we can use a regexp tree dictionaries as a user agent string parser. We support [uap-core](https://github.com/ua-parser/uap-core) and demonstrate how to use it in the functional test [02504_regexp_dictionary_ua_parser](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh) +#### Collecting Attribute Values + +Sometimes it is useful to return values from multiple regular expressions that matched, rather than just the value of a leaf node. In these cases, the specialized [`dictGetAll`](../../sql-reference/functions/ext-dict-functions.md#dictgetall) function can be used. If a node has an attribute value of type `T`, `dictGetAll` will return an `Array(T)` containing zero or more values. + +By default, the number of matches returned per key is unbounded. A bound can be passed as an optional fourth argument to `dictGetAll`. The array is populated in _topological order_, meaning that child nodes come before parent nodes, and sibling nodes follow the ordering in the source. + +Example: + +```sql +CREATE DICTIONARY regexp_dict +( + regexp String, + tag String, + topological_index Int64, + captured Nullable(String), + parent String +) +PRIMARY KEY(regexp) +SOURCE(YAMLRegExpTree(PATH '/var/lib/clickhouse/user_files/regexp_tree.yaml')) +LAYOUT(regexp_tree) +LIFETIME(0) +``` + +```yaml +# /var/lib/clickhouse/user_files/regexp_tree.yaml +- regexp: 'clickhouse\.com' + tag: 'ClickHouse' + topological_index: 1 + paths: + - regexp: 'clickhouse\.com/docs(.*)' + tag: 'ClickHouse Documentation' + topological_index: 0 + captured: '\1' + parent: 'ClickHouse' + +- regexp: '/docs(/|$)' + tag: 'Documentation' + topological_index: 2 + +- regexp: 'github.com' + tag: 'GitHub' + topological_index: 3 + captured: 'NULL' +``` + +```sql +CREATE TABLE urls (url String) ENGINE=MergeTree ORDER BY url; +INSERT INTO urls VALUES ('clickhouse.com'), ('clickhouse.com/docs/en'), ('github.com/clickhouse/tree/master/docs'); +SELECT url, dictGetAll('regexp_dict', ('tag', 'topological_index', 'captured', 'parent'), url, 2) FROM urls; +``` + +Result: + +```text +┌─url────────────────────────────────────┬─dictGetAll('regexp_dict', ('tag', 'topological_index', 'captured', 'parent'), url, 2)─┐ +│ clickhouse.com │ (['ClickHouse'],[1],[],[]) │ +│ clickhouse.com/docs/en │ (['ClickHouse Documentation','ClickHouse'],[0,1],['/en'],['ClickHouse']) │ +│ github.com/clickhouse/tree/master/docs │ (['Documentation','GitHub'],[2,3],[NULL],[]) │ +└────────────────────────────────────────┴───────────────────────────────────────────────────────────────────────────────────────┘ +``` + ### Use Regular Expression Tree Dictionary in ClickHouse Cloud Above used `YAMLRegExpTree` source works in ClickHouse Open Source but not in ClickHouse Cloud. To use regexp tree dictionaries in ClickHouse could, first create a regexp tree dictionary from a YAML file locally in ClickHouse Open Source, then dump this dictionary into a CSV file using the `dictionary` table function and the [INTO OUTFILE](../statements/select/into-outfile.md) clause. diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 7d8aa2c0390..284d6d80405 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -403,6 +403,84 @@ SELECT dictGetDescendants('hierarchy_flat_dictionary', number, 1) FROM system.nu └────────────────────────────────────────────────────────────┘ ``` + +## dictGetAll + +Retrieves the attribute values of all nodes that matched each key in a [regular expression tree dictionary](../../sql-reference/dictionaries/index.md#regexp-tree-dictionary). + +Besides returning values of type `Array(T)` instead of `T`, this function behaves similarly to [`dictGet`](#dictget-dictgetordefault-dictgetornull). + +**Syntax** + +``` sql +dictGetAll('dict_name', attr_names, id_expr[, limit]) +``` + +**Arguments** + +- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal). +- `attr_names` — Name of the column of the dictionary, [String literal](../../sql-reference/syntax.md#syntax-string-literal), or tuple of column names, [Tuple](../../sql-reference/data-types/tuple.md)([String literal](../../sql-reference/syntax.md#syntax-string-literal)). +- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning array of dictionary key-type value or [Tuple](../../sql-reference/data-types/tuple.md)-type value depending on the dictionary configuration. +- `limit` - Maximum length for each value array returned. When truncating, child nodes are given precedence over parent nodes, and otherwise the defined list order for the regexp tree dictionary is respected. If unspecified, array length is unlimited. + +**Returned value** + +- If ClickHouse parses the attribute successfully in the attribute’s data type as defined in the dictionary, returns an array of dictionary attribute values that correspond to `id_expr` for each attribute specified by `attr_names`. + +- If there is no key corresponding to `id_expr` in the dictionary, then an empty array is returned. + +ClickHouse throws an exception if it cannot parse the value of the attribute or the value does not match the attribute data type. + +**Example** + +Consider the following regexp tree dictionary: + +```sql +CREATE DICTIONARY regexp_dict +( + regexp String, + tag String +) +PRIMARY KEY(regexp) +SOURCE(YAMLRegExpTree(PATH '/var/lib/clickhouse/user_files/regexp_tree.yaml')) +LAYOUT(regexp_tree) +... +``` + +```yaml +# /var/lib/clickhouse/user_files/regexp_tree.yaml +- regexp: 'foo' + tag: 'foo_attr' +- regexp: 'bar' + tag: 'bar_attr' +- regexp: 'baz' + tag: 'baz_attr' +``` + +Get all matching values: + +```sql +SELECT dictGetAll('regexp_dict', 'tag', 'foobarbaz'); +``` + +```text +┌─dictGetAll('regexp_dict', 'tag', 'foobarbaz')─┐ +│ ['foo_attr','bar_attr','baz_attr'] │ +└───────────────────────────────────────────────┘ +``` + +Get up to 2 matching values: + +```sql +SELECT dictGetAll('regexp_dict', 'tag', 'foobarbaz', 2); +``` + +```text +┌─dictGetAll('regexp_dict', 'tag', 'foobarbaz', 2)─┐ +│ ['foo_attr','bar_attr'] │ +└──────────────────────────────────────────────────┘ +``` + ## Other Functions ClickHouse supports specialized functions that convert dictionary attribute values to a specific data type regardless of the dictionary configuration. diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index ee18e8b9a7e..f1834b4b129 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -207,6 +207,51 @@ public: return result; } + /** + * Analogous to getColumn, but for dictGetAll + */ + virtual ColumnPtr getColumnAllValues( + const std::string & attribute_name [[maybe_unused]], + const DataTypePtr & result_type [[maybe_unused]], + const Columns & key_columns [[maybe_unused]], + const DataTypes & key_types [[maybe_unused]], + const ColumnPtr & default_values_column [[maybe_unused]], + size_t limit [[maybe_unused]]) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Method getColumnAllValues is not supported for {} dictionary.", + getDictionaryID().getNameForLogs()); + } + + /** + * Analogous to getColumns, but for dictGetAll + */ + virtual Columns getColumnsAllValues( + const Strings & attribute_names, + const DataTypes & result_types, + const Columns & key_columns, + const DataTypes & key_types, + const Columns & default_values_columns, + size_t limit) const + { + size_t attribute_names_size = attribute_names.size(); + + Columns result; + result.reserve(attribute_names_size); + + for (size_t i = 0; i < attribute_names_size; ++i) + { + const auto & attribute_name = attribute_names[i]; + const auto & result_type = result_types[i]; + const auto & default_values_column = default_values_columns[i]; + + result.emplace_back(getColumnAllValues( + attribute_name, result_type, key_columns, key_types, default_values_column, limit)); + } + + return result; + } + /** Subclass must validate key columns and key types and return ColumnUInt8 that * is bitmask representation of is key in dictionary or not. * If key is in dictionary then value of associated row will be 1, otherwise 0. diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 9841cadcdca..8d0af9b0abf 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -70,7 +70,7 @@ namespace explicit StringPiece(int ref_) : ref_num(ref_) {} }; - Field parseStringToField(const String & raw, DataTypePtr data_type) + Field parseStringToField(const String & raw, const DataTypePtr data_type) try { ReadBufferFromString buffer(raw); @@ -419,6 +419,65 @@ RegExpTreeDictionary::RegExpTreeDictionary( calculateBytesAllocated(); } +// Thin wrapper around unordered_map that manages the collection of attribute values subject to the +// behavior specified by collect_values_limit +class RegExpTreeDictionary::AttributeCollector : public std::unordered_map +{ +private: + std::optional collect_values_limit; // std::nullopt means single-value mode, i.e. don't collect + size_t n_full_attributes; + +public: + explicit AttributeCollector(std::optional collect_values_limit_) + : collect_values_limit(collect_values_limit_), n_full_attributes(0) + { + } + + constexpr bool collecting() const { return collect_values_limit != std::nullopt; } + + // Add a name-value pair to the collection if there's space + void add(const String & attr_name, Field field) + { + if (collect_values_limit) + { + if (!this->contains(attr_name)) + (*this)[attr_name] = Array(); + + Array & values = (*this)[attr_name].safeGet(); + if (values.size() < *collect_values_limit) + { + values.push_back(std::move(field)); + if (values.size() == *collect_values_limit) + n_full_attributes++; + } + } + else if (!this->contains(attr_name)) + { + (*this)[attr_name] = std::move(field); + n_full_attributes++; + } + } + + // Checks if no more values can be added for a given attribute + inline bool full(const String & attr_name) const + { + if (collect_values_limit) + { + auto it = this->find(attr_name); + if (it == this->end()) + return false; + return it->second.safeGet().size() >= *collect_values_limit; + } + else + { + return this->contains(attr_name); + } + } + + // Returns the number of full attributes + inline size_t attributesFull() const { return n_full_attributes; } +}; + std::pair processBackRefs(const String & data, const re2_st::RE2 & searcher, const std::vector & pieces) { re2_st::StringPiece haystack(data.data(), data.size()); @@ -442,7 +501,7 @@ std::pair processBackRefs(const String & data, const re2_st::RE2 & // The return value means whether we finish collecting. bool RegExpTreeDictionary::setAttributes( UInt64 id, - std::unordered_map & attributes_to_set, + AttributeCollector & attributes_to_set, const String & data, std::unordered_set & visited_nodes, const std::unordered_map & attributes, @@ -451,34 +510,43 @@ bool RegExpTreeDictionary::setAttributes( { if (visited_nodes.contains(id)) - return attributes_to_set.size() == attributes.size(); + return attributes_to_set.attributesFull() == attributes.size(); visited_nodes.emplace(id); const auto & node_attributes = regex_nodes.at(id)->attributes; for (const auto & [name_, value] : node_attributes) { - if (!attributes.contains(name_) || attributes_to_set.contains(name_)) + if (!attributes.contains(name_) || attributes_to_set.full(name_)) continue; + if (value.containsBackRefs()) { auto [updated_str, use_default] = processBackRefs(data, regex_nodes.at(id)->searcher, value.pieces); if (use_default) { - DefaultValueProvider default_value(attributes.at(name_).null_value, defaults.at(name_)); - attributes_to_set[name_] = default_value.getDefaultValue(key_index); + // Back-ref processing failed. + // - If not collecting values, set the default value immediately while we're still on this node. + // Otherwise, a value from a different node could take its place before we set it to the default value post-walk. + // - If collecting values, don't add anything. If we find no other matches for this attribute, + // then we'll set its value to the default Array value later. + if (!attributes_to_set.collecting()) + { + DefaultValueProvider default_value(attributes.at(name_).null_value, defaults.at(name_)); + attributes_to_set.add(name_, default_value.getDefaultValue(key_index)); + } } else - attributes_to_set[name_] = parseStringToField(updated_str, attributes.at(name_).type); + attributes_to_set.add(name_, parseStringToField(updated_str, attributes.at(name_).type)); } else - attributes_to_set[name_] = value.field; + attributes_to_set.add(name_, value.field); } auto parent_id = regex_nodes.at(id)->parent_id; if (parent_id > 0) setAttributes(parent_id, attributes_to_set, data, visited_nodes, attributes, defaults, key_index); - /// if all the attributes have set, the walking through can be stopped. - return attributes_to_set.size() == attributes.size(); + /// if all attributes are full, we can stop walking the tree + return attributes_to_set.attributesFull() == attributes.size(); } /// a temp struct to store all the matched result. @@ -550,7 +618,8 @@ std::unordered_map RegExpTreeDictionary::match( const ColumnString::Chars & keys_data, const ColumnString::Offsets & keys_offsets, const std::unordered_map & attributes, - const std::unordered_map & defaults) const + const std::unordered_map & defaults, + std::optional collect_values_limit) const { #if USE_VECTORSCAN @@ -573,7 +642,7 @@ std::unordered_map RegExpTreeDictionary::match( /// initialize columns for (const auto & [name_, attr] : attributes) { - auto col_ptr = attr.type->createColumn(); + auto col_ptr = (collect_values_limit ? std::make_shared(attr.type) : attr.type)->createColumn(); col_ptr->reserve(keys_offsets.size()); columns[name_] = std::move(col_ptr); } @@ -630,11 +699,11 @@ std::unordered_map RegExpTreeDictionary::match( match_result.sort(); /// Walk through the regex tree util all attributes are set; - std::unordered_map attributes_to_set; + AttributeCollector attributes_to_set{collect_values_limit}; std::unordered_set visited_nodes; /// Some node matches but its parents cannot match. In this case we must regard this node unmatched. - auto is_invalid = [&](UInt64 id) + auto is_valid = [&](UInt64 id) { while (id) { @@ -650,7 +719,7 @@ std::unordered_map RegExpTreeDictionary::match( for (auto item : match_result.matched_idx_sorted_list) { UInt64 id = item.second; - if (!is_invalid(id)) + if (!is_valid(id)) continue; if (visited_nodes.contains(id)) continue; @@ -663,7 +732,8 @@ std::unordered_map RegExpTreeDictionary::match( if (attributes_to_set.contains(name_)) continue; - DefaultValueProvider default_value(attr.null_value, defaults.at(name_)); + DefaultValueProvider default_value( + collect_values_limit ? DataTypeArray(attr.type).getDefault() : attr.null_value, defaults.at(name_)); columns[name_]->insert(default_value.getDefaultValue(key_idx)); } @@ -727,12 +797,13 @@ Pipe RegExpTreeDictionary::read(const Names & , size_t max_block_size, size_t) c return Pipe(std::make_shared(std::move(result))); } -Columns RegExpTreeDictionary::getColumns( +Columns RegExpTreeDictionary::getColumnsImpl( const Strings & attribute_names, const DataTypes & result_types, const Columns & key_columns, const DataTypes & key_types, - const Columns & default_values_columns) const + const Columns & default_values_columns, + std::optional collect_values_limit) const { /// valid check if (key_columns.size() != 1) @@ -746,7 +817,17 @@ Columns RegExpTreeDictionary::getColumns( for (size_t i = 0; i < attribute_names.size(); i++) { - const auto & attribute = structure.getAttribute(attribute_names[i], result_types[i]); + DataTypePtr attribute_type = result_types[i]; + if (collect_values_limit) + { + if (!WhichDataType(attribute_type).isArray()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Expected Array result type for attribute `{}`, got `{}`", + attribute_names[i], + attribute_type->getName()); + attribute_type = assert_cast(*attribute_type).getNestedType(); + } + const auto & attribute = structure.getAttribute(attribute_names[i], attribute_type); attributes.emplace(attribute.name, attribute); defaults[attribute.name] = default_values_columns[i]; } @@ -757,7 +838,8 @@ Columns RegExpTreeDictionary::getColumns( key_column->getChars(), key_column->getOffsets(), attributes, - defaults); + defaults, + collect_values_limit); Columns result; for (const String & name_ : attribute_names) diff --git a/src/Dictionaries/RegExpTreeDictionary.h b/src/Dictionaries/RegExpTreeDictionary.h index 683588e688f..30966184eb6 100644 --- a/src/Dictionaries/RegExpTreeDictionary.h +++ b/src/Dictionaries/RegExpTreeDictionary.h @@ -101,16 +101,50 @@ public: const Columns & key_columns, const DataTypes & key_types, const ColumnPtr & default_values_column) const override - { - return getColumns(Strings({attribute_name}), DataTypes({result_type}), key_columns, key_types, Columns({default_values_column}))[0]; - } + { + return getColumns(Strings({attribute_name}), DataTypes({result_type}), key_columns, key_types, Columns({default_values_column}))[0]; + } Columns getColumns( const Strings & attribute_names, const DataTypes & result_types, const Columns & key_columns, const DataTypes & key_types, - const Columns & default_values_columns) const override; + const Columns & default_values_columns) const override + { + return getColumnsImpl(attribute_names, result_types, key_columns, key_types, default_values_columns, std::nullopt); + } + + ColumnPtr getColumnAllValues( + const std::string & attribute_name, + const DataTypePtr & result_type, + const Columns & key_columns, + const DataTypes & key_types, + const ColumnPtr & default_values_column, + size_t limit) const override + { + return getColumnsAllValues( + Strings({attribute_name}), DataTypes({result_type}), key_columns, key_types, Columns({default_values_column}), limit)[0]; + } + + Columns getColumnsAllValues( + const Strings & attribute_names, + const DataTypes & result_types, + const Columns & key_columns, + const DataTypes & key_types, + const Columns & default_values_columns, + size_t limit) const override + { + return getColumnsImpl(attribute_names, result_types, key_columns, key_types, default_values_columns, limit); + } + + Columns getColumnsImpl( + const Strings & attribute_names, + const DataTypes & result_types, + const Columns & key_columns, + const DataTypes & key_types, + const Columns & default_values_columns, + std::optional collect_values_limit) const; private: const DictionaryStructure structure; @@ -137,11 +171,14 @@ private: const ColumnString::Chars & keys_data, const ColumnString::Offsets & keys_offsets, const std::unordered_map & attributes, - const std::unordered_map & defaults) const; + const std::unordered_map & defaults, + std::optional collect_values_limit) const; + + class AttributeCollector; bool setAttributes( UInt64 id, - std::unordered_map & attributes_to_set, + AttributeCollector & attributes_to_set, const String & data, std::unordered_set & visited_nodes, const std::unordered_map & attributes, diff --git a/src/Functions/FunctionsExternalDictionaries.cpp b/src/Functions/FunctionsExternalDictionaries.cpp index 70b1e3cc861..9fa08c82d41 100644 --- a/src/Functions/FunctionsExternalDictionaries.cpp +++ b/src/Functions/FunctionsExternalDictionaries.cpp @@ -45,11 +45,26 @@ Accepts 3 parameters: Returned value: value of the dictionary attribute parsed in the attribute’s data type if key is found, otherwise NULL. Throws an exception if cannot parse the value of the attribute or the value does not match the attribute data type. +)" }; + + constexpr auto dict_get_all_description { R"( +Retrieves all values from a dictionary corresponding to the given key values. + +Accepts 3 or 4 parameters: +-- name of the dictionary; +-- name of the column of the dictionary or tuple of column names; +-- key value - expression returning dictionary key-type value or tuple-type value - depending on the dictionary configuration; +-- [optional] maximum number of values to return for each attribute; + +Returned value: array of dictionary attribute values parsed in the attribute's data type if key is found, otherwise empty array. + +Throws an exception if cannot parse the value of the attribute, the value does not match the attribute data type, or the dictionary doesn't support this function. )" }; factory.registerFunction>(FunctionDocumentation{ .description=fmt::format(dict_get_description, "attribute’s data type") }); factory.registerFunction>(FunctionDocumentation{ .description=fmt::format(dict_get_or_default_description, "attribute’s data type") }); factory.registerFunction(FunctionDocumentation{ .description=dict_get_or_null_description }); + factory.registerFunction>(FunctionDocumentation{ .description=dict_get_all_description }); factory.registerFunction(FunctionDocumentation{ .description=fmt::format(dict_get_description, "UInt8") }); factory.registerFunction(FunctionDocumentation{ .description=fmt::format(dict_get_description, "UInt16") }); diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 97d85f384bc..e4529ff1765 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -296,7 +296,8 @@ private: enum class DictionaryGetFunctionType { get, - getOrDefault + getOrDefault, + getAll }; /// This variant of function derives the result type automatically. @@ -304,7 +305,10 @@ template class FunctionDictGetNoType final : public IFunction { public: - static constexpr auto name = dictionary_get_function_type == DictionaryGetFunctionType::get ? "dictGet" : "dictGetOrDefault"; + // Kind of gross but we need a static field called "name" for FunctionFactory::registerFunction, and this is the easiest way + static constexpr auto name = (dictionary_get_function_type == DictionaryGetFunctionType::get) + ? "dictGet" + : ((dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault) ? "dictGetOrDefault" : "dictGetAll"); static FunctionPtr create(ContextPtr context) { @@ -321,7 +325,13 @@ public: bool useDefaultImplementationForConstants() const final { return true; } bool useDefaultImplementationForNulls() const final { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const final + { + if constexpr (dictionary_get_function_type == DictionaryGetFunctionType::getAll) + return {0, 1, 3}; + else + return {0, 1}; + } bool isDeterministic() const override { return false; } @@ -360,6 +370,15 @@ public: } bool key_is_nullable = arguments[2].type->isNullable(); + if (dictionary_get_function_type == DictionaryGetFunctionType::getAll) + { + if (key_is_nullable) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function {} does not support nullable keys", getName()); + + // Wrap all the attribute types in Array() + for (auto it = attribute_types.begin(); it != attribute_types.end(); ++it) + *it = std::make_shared(*it); + } if (attribute_types.size() > 1) { if (key_is_nullable) @@ -424,6 +443,7 @@ public: } Columns default_cols; + size_t collect_values_limit = std::numeric_limits::max(); if (dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault) { @@ -464,6 +484,19 @@ public: } else { + if (dictionary_get_function_type == DictionaryGetFunctionType::getAll && current_arguments_index < arguments.size()) + { + auto limit_col = arguments[current_arguments_index].column; + if (!limit_col || !isColumnConst(*limit_col)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of fourth argument of function {}. Expected const unsigned integer.", + arguments[current_arguments_index].type->getName(), + getName()); + + collect_values_limit = limit_col->getUInt(0); + ++current_arguments_index; + } + for (size_t i = 0; i < attribute_names.size(); ++i) default_cols.emplace_back(nullptr); } @@ -549,7 +582,8 @@ public: attribute_type = attribute_types.front(); } - auto result_column = executeDictionaryRequest(dictionary, attribute_names, key_columns, key_types, attribute_type, default_cols); + auto result_column = executeDictionaryRequest( + dictionary, attribute_names, key_columns, key_types, attribute_type, default_cols, collect_values_limit); if (key_is_nullable) result_column = wrapInNullable(result_column, {arguments[2]}, result_type, input_rows_count); @@ -565,7 +599,8 @@ private: const Columns & key_columns, const DataTypes & key_types, const DataTypePtr & result_type, - const Columns & default_cols) const + const Columns & default_cols, + size_t collect_values_limit) const { ColumnPtr result; @@ -573,23 +608,31 @@ private: { const auto & result_tuple_type = assert_cast(*result_type); - Columns result_columns = dictionary->getColumns( - attribute_names, - result_tuple_type.getElements(), - key_columns, - key_types, - default_cols); + Columns result_columns; + if (dictionary_get_function_type == DictionaryGetFunctionType::getAll) + { + result_columns = dictionary->getColumnsAllValues( + attribute_names, result_tuple_type.getElements(), key_columns, key_types, default_cols, collect_values_limit); + } + else + { + result_columns + = dictionary->getColumns(attribute_names, result_tuple_type.getElements(), key_columns, key_types, default_cols); + } result = ColumnTuple::create(std::move(result_columns)); } else { - result = dictionary->getColumn( - attribute_names[0], - result_type, - key_columns, - key_types, - default_cols.front()); + if (dictionary_get_function_type == DictionaryGetFunctionType::getAll) + { + result = dictionary->getColumnAllValues( + attribute_names[0], result_type, key_columns, key_types, default_cols.front(), collect_values_limit); + } + else + { + result = dictionary->getColumn(attribute_names[0], result_type, key_columns, key_types, default_cols.front()); + } } return result; diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference index dfcd170e8f4..437012dd516 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference +++ b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference @@ -5,3 +5,9 @@ ('BlackBerry WebKit','10.0') ('BlackBerry WebKit','1.0') (true,'61f0c404-5cb3-11e7-907b-a6006ad3dba0','2023-01-01','2023-01-01 01:01:01',[1,2,3,-1,-2,-3]) +(['ClickHouse'],[1],[],[]) +(['ClickHouse'],[1],[],[]) +(['ClickHouse Documentation','ClickHouse','Documentation'],[0,1,2],['/en'],['ClickHouse']) +(['ClickHouse Documentation','ClickHouse'],[0,1],['/en'],['ClickHouse']) +(['Documentation','GitHub'],[2,3],[NULL],[]) +(['Documentation','GitHub'],[2,3],[NULL],[]) diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh index 1b5a9cdeea4..ac0793460a9 100755 --- a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh +++ b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh @@ -128,9 +128,57 @@ LAYOUT(regexp_tree); select dictGet('regexp_dict2', ('col_bool','col_uuid', 'col_date', 'col_datetime', 'col_array'), 'abc'); " +cat > "$yaml" < Date: Sun, 4 Jun 2023 19:48:14 -0500 Subject: [PATCH 1116/2223] Review comments: Use constexpr-if in more places Also add a comment about the apparent lack of type checking on the limit column. --- src/Functions/FunctionsExternalDictionaries.h | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index e4529ff1765..db6529da73c 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -370,7 +370,7 @@ public: } bool key_is_nullable = arguments[2].type->isNullable(); - if (dictionary_get_function_type == DictionaryGetFunctionType::getAll) + if constexpr (dictionary_get_function_type == DictionaryGetFunctionType::getAll) { if (key_is_nullable) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function {} does not support nullable keys", getName()); @@ -487,6 +487,7 @@ public: if (dictionary_get_function_type == DictionaryGetFunctionType::getAll && current_arguments_index < arguments.size()) { auto limit_col = arguments[current_arguments_index].column; + // The getUInt later attempts to cast and throws on a type mismatch, so skip actual type checking here if (!limit_col || !isColumnConst(*limit_col)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of fourth argument of function {}. Expected const unsigned integer.", @@ -609,7 +610,7 @@ private: const auto & result_tuple_type = assert_cast(*result_type); Columns result_columns; - if (dictionary_get_function_type == DictionaryGetFunctionType::getAll) + if constexpr (dictionary_get_function_type == DictionaryGetFunctionType::getAll) { result_columns = dictionary->getColumnsAllValues( attribute_names, result_tuple_type.getElements(), key_columns, key_types, default_cols, collect_values_limit); @@ -624,7 +625,7 @@ private: } else { - if (dictionary_get_function_type == DictionaryGetFunctionType::getAll) + if constexpr (dictionary_get_function_type == DictionaryGetFunctionType::getAll) { result = dictionary->getColumnAllValues( attribute_names[0], result_type, key_columns, key_types, default_cols.front(), collect_values_limit); From 5e1c93c9c819a1d5819ab742fe4981199c621462 Mon Sep 17 00:00:00 2001 From: johanngan Date: Sun, 4 Jun 2023 21:09:41 -0500 Subject: [PATCH 1117/2223] Add dictGetAll to spell-check dictionary --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 0455556ae96..d6cef1883f4 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1342,6 +1342,7 @@ detectLanguageUnknown determinator deterministically dictGet +dictGetAll dictGetChildren dictGetDescendant dictGetHierarchy From f1058d2d9d2201f21882b487499ea4f4212fec0b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 5 Jun 2023 09:51:16 +0300 Subject: [PATCH 1118/2223] Revert "Disable skim (Rust library) under memory sanitizer" --- rust/skim/CMakeLists.txt | 5 ----- 1 file changed, 5 deletions(-) diff --git a/rust/skim/CMakeLists.txt b/rust/skim/CMakeLists.txt index c2e406ec12f..1e7a43aba7c 100644 --- a/rust/skim/CMakeLists.txt +++ b/rust/skim/CMakeLists.txt @@ -14,11 +14,6 @@ if (OS_FREEBSD) return() endif() -if (SANITIZE STREQUAL "memory") - message(STATUS "skim is disabled under memory sanitizer, because the interop is not instrumented properly") - return() -endif() - clickhouse_import_crate(MANIFEST_PATH Cargo.toml) # -Wno-dollar-in-identifier-extension: cxx bridge complies names with '$' From c860db0fb77ad247e39dec10b3419ab1cb2b05e3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 5 Jun 2023 10:32:46 +0300 Subject: [PATCH 1119/2223] Fixed tests --- src/Functions/if.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 8d43b3a4ca3..65e2212e894 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -1124,6 +1124,9 @@ public: return {}; const ColumnConst * cond_const_col = checkAndGetColumnConst>(arg_cond.column.get()); + if (!cond_const_col) + return {}; + bool condition_value = cond_const_col->getValue(); const ColumnWithTypeAndName & arg_then = arguments[1]; From 4225cab2e8203b45e17188da28c3f6a1a330878c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Jun 2023 08:34:25 +0000 Subject: [PATCH 1120/2223] Rewrite bugprone shell script command --- tests/queries/0_stateless/02771_system_user_processes.sh | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02771_system_user_processes.sh b/tests/queries/0_stateless/02771_system_user_processes.sh index 910af4be9e2..f0e5b2a6987 100755 --- a/tests/queries/0_stateless/02771_system_user_processes.sh +++ b/tests/queries/0_stateless/02771_system_user_processes.sh @@ -7,7 +7,13 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) USER_POSTFIX=`random_str 10` USER="test_user_02771_$USER_POSTFIX" -$CLICKHOUSE_CLIENT -q "SHOW USER PROCESSES" &>"${CLICKHOUSE_TMP}/test_output" && echo "SHOW USER PROCESSES query succeeded!" || cat "${CLICKHOUSE_TMP}/test_output" +if $CLICKHOUSE_CLIENT -q "SHOW USER PROCESSES" &>"${CLICKHOUSE_TMP}/test_output" +then + echo "SHOW USER PROCESSES query succeeded!" +else + cat "${CLICKHOUSE_TMP}/test_output" +fi + $CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS $USER" $CLICKHOUSE_CLIENT -q "CREATE USER $USER" $CLICKHOUSE_CLIENT -u "$USER" -q "SELECT * FROM system.numbers LIMIT 1" From 3657ef05fffa722115becb7f7e8937a3a472625d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:26:41 +0000 Subject: [PATCH 1121/2223] Cosmetics: Fix indentation --- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 30 +++++++++++--------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 3b1a41eb85d..b349c0567ef 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -14,19 +14,23 @@ namespace DB // mainly for serialization and deserialization of the index namespace ApproximateNearestNeighbour { - using AnnoyIndexThreadedBuildPolicy = ::Annoy::AnnoyIndexMultiThreadedBuildPolicy; - // TODO: Support different metrics. List of available metrics can be taken from here: - // https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 - template - class AnnoyIndex : public ::Annoy::AnnoyIndex - { - using Base = ::Annoy::AnnoyIndex; - public: - explicit AnnoyIndex(const uint64_t dim) : Base::AnnoyIndex(dim) {} - void serialize(WriteBuffer& ostr) const; - void deserialize(ReadBuffer& istr); - uint64_t getNumOfDimensions() const; - }; + +using AnnoyIndexThreadedBuildPolicy = ::Annoy::AnnoyIndexMultiThreadedBuildPolicy; + +// TODO: Support different metrics. List of available metrics can be taken from here: +// https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 +template +class AnnoyIndex : public ::Annoy::AnnoyIndex +{ + using Base = ::Annoy::AnnoyIndex; + +public: + explicit AnnoyIndex(const uint64_t dim) : Base::AnnoyIndex(dim) {} + void serialize(WriteBuffer& ostr) const; + void deserialize(ReadBuffer& istr); + uint64_t getNumOfDimensions() const; +}; + } template From 32756292309182bfa2ddf59213c1628a22651e26 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:31:34 +0000 Subject: [PATCH 1122/2223] Cosmetics: Remove parentheses in single statement if/for/while --- src/Storages/MergeTree/CommonANNIndexes.cpp | 94 ------------------- .../MergeTree/MergeTreeIndexAnnoy.cpp | 31 +----- 2 files changed, 4 insertions(+), 121 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 4b360e029e5..f0c6f256f73 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -35,17 +35,11 @@ void extractTargetVectorFromLiteral(ANN::ANNQueryInformation::Embedding & target for (const auto & value : literal.value()) { if (value.tryGet(float_element_of_target_vector)) - { target.emplace_back(float_element_of_target_vector); - } else if (value.tryGet(int_element_of_target_vector)) - { target.emplace_back(static_cast(int_element_of_target_vector)); - } else - { throw Exception(ErrorCodes::INCORRECT_QUERY, "Wrong type of elements in target vector. Only float or int are supported."); - } } } @@ -74,9 +68,7 @@ ANNCondition::ANNCondition(const SelectQueryInfo & query_info, bool ANNCondition::alwaysUnknownOrTrue(String metric_name) const { if (!index_is_useful) - { return true; // Query isn't supported - } // If query is supported, check metrics for match return !(castMetricFromStringToType(metric_name) == query_information->metric); } @@ -85,72 +77,56 @@ float ANNCondition::getComparisonDistanceForWhereQuery() const { if (index_is_useful && query_information.has_value() && query_information->query_type == ANNQueryInformation::Type::Where) - { return query_information->distance; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported method for this query type"); } UInt64 ANNCondition::getLimit() const { if (index_is_useful && query_information.has_value()) - { return query_information->limit; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "No LIMIT section in query, not supported"); } std::vector ANNCondition::getTargetVector() const { if (index_is_useful && query_information.has_value()) - { return query_information->target; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Target vector was requested for useless or uninitialized index."); } size_t ANNCondition::getNumOfDimensions() const { if (index_is_useful && query_information.has_value()) - { return query_information->target.size(); - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of dimensions was requested for useless or uninitialized index."); } String ANNCondition::getColumnName() const { if (index_is_useful && query_information.has_value()) - { return query_information->column_name; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Column name was requested for useless or uninitialized index."); } ANNQueryInformation::Metric ANNCondition::getMetricType() const { if (index_is_useful && query_information.has_value()) - { return query_information->metric; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Metric name was requested for useless or uninitialized index."); } float ANNCondition::getPValueForLpDistance() const { if (index_is_useful && query_information.has_value()) - { return query_information->p_for_lp_dist; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "P from LPDistance was requested for useless or uninitialized index."); } ANNQueryInformation::Type ANNCondition::getQueryType() const { if (index_is_useful && query_information.has_value()) - { return query_information->query_type; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Query type was requested for useless or uninitialized index."); } @@ -171,24 +147,16 @@ bool ANNCondition::checkQueryStructure(const SelectQueryInfo & query) const auto & select = query.query->as(); if (select.prewhere()) // If query has PREWHERE clause - { traverseAST(select.prewhere(), rpn_prewhere_clause); - } if (select.where()) // If query has WHERE clause - { traverseAST(select.where(), rpn_where_clause); - } if (select.limitLength()) // If query has LIMIT clause - { traverseAtomAST(select.limitLength(), rpn_limit); - } if (select.orderBy()) // If query has ORDERBY clause - { traverseOrderByAST(select.orderBy(), rpn_order_by_clause); - } // Reverse RPNs for conveniences during parsing std::reverse(rpn_prewhere_clause.begin(), rpn_prewhere_clause.end()); @@ -203,29 +171,21 @@ bool ANNCondition::checkQueryStructure(const SelectQueryInfo & query) // Query without a LIMIT clause or with a limit greater than a restriction is not supported if (!limit_is_valid || limit_restriction < limit) - { return false; - } // Search type query in both sections isn't supported if (prewhere_is_valid && where_is_valid) - { return false; - } // Search type should be in WHERE or PREWHERE clause if (prewhere_is_valid || where_is_valid) - { query_information = std::move(prewhere_is_valid ? prewhere_info : where_info); - } if (order_by_is_valid) { // Query with valid where and order by type is not supported if (query_information.has_value()) - { return false; - } query_information = std::move(order_by_info); } @@ -244,17 +204,13 @@ void ANNCondition::traverseAST(const ASTPtr & node, RPN & rpn) const ASTs & children = func->arguments->children; // Traverse children nodes for (const auto& child : children) - { traverseAST(child, rpn); - } } RPNElement element; // Get the data behind node if (!traverseAtomAST(node, element)) - { element.function = RPNElement::FUNCTION_UNKNOWN; - } rpn.emplace_back(std::move(element)); } @@ -273,32 +229,20 @@ bool ANNCondition::traverseAtomAST(const ASTPtr & node, RPNElement & out) function->name == "cosineDistance" || function->name == "dotProduct" || function->name == "LpDistance") - { out.function = RPNElement::FUNCTION_DISTANCE; - } else if (function->name == "tuple") - { out.function = RPNElement::FUNCTION_TUPLE; - } else if (function->name == "array") - { out.function = RPNElement::FUNCTION_ARRAY; - } else if (function->name == "less" || function->name == "greater" || function->name == "lessOrEquals" || function->name == "greaterOrEquals") - { out.function = RPNElement::FUNCTION_COMPARISON; - } else if (function->name == "_CAST") - { out.function = RPNElement::FUNCTION_CAST; - } else - { return false; - } return true; } @@ -378,12 +322,8 @@ bool ANNCondition::tryCastToConstType(const ASTPtr & node, RPNElement & out) void ANNCondition::traverseOrderByAST(const ASTPtr & node, RPN & rpn) { if (const auto * expr_list = node->as()) - { if (const auto * order_by_element = expr_list->children.front()->as()) - { traverseAST(order_by_element->children.front(), rpn); - } - } } // Returns true and stores ANNQueryInformation if the query has valid WHERE clause @@ -395,17 +335,13 @@ bool ANNCondition::matchRPNWhere(RPN & rpn, ANNQueryInformation & expr) // WHERE section must have at least 5 expressions // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(TargetVector(floats)) if (rpn.size() < 5) - { return false; - } auto iter = rpn.begin(); // Query starts from operator less if (iter->function != RPNElement::FUNCTION_COMPARISON) - { return false; - } const bool greater_case = iter->func_name == "greater" || iter->func_name == "greaterOrEquals"; const bool less_case = iter->func_name == "less" || iter->func_name == "lessOrEquals"; @@ -415,9 +351,7 @@ bool ANNCondition::matchRPNWhere(RPN & rpn, ANNQueryInformation & expr) if (less_case) { if (iter->function != RPNElement::FUNCTION_FLOAT_LITERAL) - { return false; - } expr.distance = getFloatOrIntLiteralOrPanic(iter); if (expr.distance < 0) @@ -427,22 +361,16 @@ bool ANNCondition::matchRPNWhere(RPN & rpn, ANNQueryInformation & expr) } else if (!greater_case) - { return false; - } auto end = rpn.end(); if (!matchMainParts(iter, end, expr)) - { return false; - } if (greater_case) { if (expr.target.size() < 2) - { return false; - } expr.distance = expr.target.back(); if (expr.distance < 0) throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", expr.distance); @@ -461,9 +389,7 @@ bool ANNCondition::matchRPNOrderBy(RPN & rpn, ANNQueryInformation & expr) // ORDER BY clause must have at least 3 expressions if (rpn.size() < 3) - { return false; - } auto iter = rpn.begin(); auto end = rpn.end(); @@ -490,9 +416,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en // Matches DistanceFunc->[Column]->[Tuple(array)Func]->TargetVector(floats)->[Column] if (iter->function != RPNElement::FUNCTION_DISTANCE) - { return false; - } expr.metric = castMetricFromStringToType(iter->func_name); ++iter; @@ -501,9 +425,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en { if (iter->function != RPNElement::FUNCTION_FLOAT_LITERAL && iter->function != RPNElement::FUNCTION_INT_LITERAL) - { return false; - } expr.p_for_lp_dist = getFloatOrIntLiteralOrPanic(iter); ++iter; } @@ -516,9 +438,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en } if (iter->function == RPNElement::FUNCTION_TUPLE || iter->function == RPNElement::FUNCTION_ARRAY) - { ++iter; - } if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { @@ -539,9 +459,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en ++iter; /// Cast should be made to array or tuple if (!iter->func_name.starts_with("Array") && !iter->func_name.starts_with("Tuple")) - { return false; - } ++iter; if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { @@ -554,31 +472,23 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en ++iter; } else - { return false; - } } while (iter != end) { if (iter->function == RPNElement::FUNCTION_FLOAT_LITERAL || iter->function == RPNElement::FUNCTION_INT_LITERAL) - { expr.target.emplace_back(getFloatOrIntLiteralOrPanic(iter)); - } else if (iter->function == RPNElement::FUNCTION_IDENTIFIER) { if (identifier_found) - { return false; - } expr.column_name = std::move(iter->identifier.value()); identifier_found = true; } else - { return false; - } ++iter; } @@ -591,13 +501,9 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en float ANNCondition::getFloatOrIntLiteralOrPanic(const RPN::iterator& iter) { if (iter->float_literal.has_value()) - { return iter->float_literal.value(); - } if (iter->int_literal.has_value()) - { return static_cast(iter->int_literal.value()); - } throw Exception(ErrorCodes::INCORRECT_QUERY, "Wrong parsed AST in buildRPN\n"); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index f64d6104ac6..0b7e1f29f03 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -219,17 +219,11 @@ bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const { if (distance_name == "L2Distance") - { return getUsefulRangesImpl<::Annoy::Euclidean>(idx_granule); - } else if (distance_name == "cosineDistance") - { return getUsefulRangesImpl<::Annoy::Angular>(idx_granule); - } else - { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); - } } @@ -297,26 +291,18 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { if (distance_name == "L2Distance") - { return std::make_shared >(index.name, index.sample_block); - } - if (distance_name == "cosineDistance") - { + else if (distance_name == "cosineDistance") return std::make_shared >(index.name, index.sample_block); - } throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { if (distance_name == "L2Distance") - { return std::make_shared >(index.name, index.sample_block, number_of_trees); - } if (distance_name == "cosineDistance") - { return std::make_shared >(index.name, index.sample_block, number_of_trees); - } throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } @@ -331,16 +317,10 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) uint64_t param = 100; String distance_name = "L2Distance"; if (!index.arguments.empty() && !index.arguments[0].tryGet(param)) - { if (!index.arguments[0].tryGet(distance_name)) - { throw Exception(ErrorCodes::INCORRECT_DATA, "Can't parse first argument"); - } - } if (index.arguments.size() > 1 && !index.arguments[1].tryGet(distance_name)) - { throw Exception(ErrorCodes::INCORRECT_DATA, "Can't parse second argument"); - } return std::make_shared(index, param, distance_name); } @@ -381,18 +361,14 @@ static void assertIndexColumnsType(const Block & header) void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { if (index.arguments.size() > 2) - { throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters"); - } + if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64 && index.arguments[0].getType() != Field::Types::String) - { throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String."); - } + if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - { throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String."); - } if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); @@ -401,4 +377,5 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) } } + #endif // ENABLE_ANNOY From edad92a7f224732c52f9d57d062e46c23396d19a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:36:27 +0000 Subject: [PATCH 1123/2223] Cosmetics: Minor aesthetic fixes --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 45 ++++++++++--------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 20 +++------ 2 files changed, 30 insertions(+), 35 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 0b7e1f29f03..b31779ff71c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -18,10 +18,10 @@ namespace DB namespace ApproximateNearestNeighbour { -template -void AnnoyIndex::serialize(WriteBuffer& ostr) const +template +void AnnoyIndex::serialize(WriteBuffer& ostr) const { - assert(Base::_built); + chassert(Base::_built); writeIntBinary(Base::_s, ostr); writeIntBinary(Base::_n_items, ostr); writeIntBinary(Base::_n_nodes, ostr); @@ -32,10 +32,10 @@ void AnnoyIndex::serialize(WriteBuffer& ostr) const ostr.write(reinterpret_cast(Base::_nodes), Base::_s * Base::_n_nodes); } -template -void AnnoyIndex::deserialize(ReadBuffer& istr) +template +void AnnoyIndex::deserialize(ReadBuffer& istr) { - assert(!Base::_built); + chassert(!Base::_built); readIntBinary(Base::_s, istr); readIntBinary(Base::_n_items, istr); readIntBinary(Base::_n_nodes, istr); @@ -54,8 +54,8 @@ void AnnoyIndex::deserialize(ReadBuffer& istr) Base::_built = true; } -template -uint64_t AnnoyIndex::getNumOfDimensions() const +template +uint64_t AnnoyIndex::getNumOfDimensions() const { return Base::get_f(); } @@ -84,16 +84,16 @@ template MergeTreeIndexGranuleAnnoy::MergeTreeIndexGranuleAnnoy( const String & index_name_, const Block & index_sample_block_, - AnnoyIndexPtr index_base_) + AnnoyIndexPtr index_) : index_name(index_name_) , index_sample_block(index_sample_block_) - , index(std::move(index_base_)) + , index(std::move(index_)) {} template void MergeTreeIndexGranuleAnnoy::serializeBinary(WriteBuffer & ostr) const { - /// number of dimensions is required in the constructor, + /// Number of dimensions is required in the index constructor, /// so it must be written and read separately from the other part writeIntBinary(index->getNumOfDimensions(), ostr); // write dimension index->serialize(ostr); @@ -123,7 +123,7 @@ MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy::getGranuleAndR { // NOLINTNEXTLINE(*) index->build(static_cast(number_of_trees), /*number_of_threads=*/1); - auto granule = std::make_shared >(index_name, index_sample_block, index); + auto granule = std::make_shared>(index_name, index_sample_block, index); index = nullptr; return granule; } @@ -202,7 +202,8 @@ MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( const SelectQueryInfo & query, ContextPtr context, const String& distance_name_) - : condition(query, context), distance_name(distance_name_) + : condition(query, context) + , distance_name(distance_name_) {} @@ -232,15 +233,16 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI { UInt64 limit = condition.getLimit(); UInt64 index_granularity = condition.getIndexGranularity(); - std::optional comp_dist = condition.getQueryType() == ApproximateNearestNeighbour::ANNQueryInformation::Type::Where ? - std::optional(condition.getComparisonDistanceForWhereQuery()) : std::nullopt; + std::optional comp_dist = condition.getQueryType() == ApproximateNearestNeighbour::ANNQueryInformation::Type::Where + ? std::optional(condition.getComparisonDistanceForWhereQuery()) + : std::nullopt; if (comp_dist && comp_dist.value() < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); std::vector target_vec = condition.getTargetVector(); - auto granule = std::dynamic_pointer_cast >(idx_granule); + auto granule = std::dynamic_pointer_cast>(idx_granule); if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); @@ -291,18 +293,19 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { if (distance_name == "L2Distance") - return std::make_shared >(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block); else if (distance_name == "cosineDistance") - return std::make_shared >(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { + /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 if (distance_name == "L2Distance") - return std::make_shared >(index.name, index.sample_block, number_of_trees); + return std::make_shared>(index.name, index.sample_block, number_of_trees); if (distance_name == "cosineDistance") - return std::make_shared >(index.name, index.sample_block, number_of_trees); + return std::make_shared>(index.name, index.sample_block, number_of_trees); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } @@ -378,4 +381,4 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) } -#endif // ENABLE_ANNOY +#endif diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index b349c0567ef..d591187fc64 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -15,17 +15,13 @@ namespace DB namespace ApproximateNearestNeighbour { -using AnnoyIndexThreadedBuildPolicy = ::Annoy::AnnoyIndexMultiThreadedBuildPolicy; - -// TODO: Support different metrics. List of available metrics can be taken from here: -// https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 template -class AnnoyIndex : public ::Annoy::AnnoyIndex +class AnnoyIndex : public ::Annoy::AnnoyIndex { - using Base = ::Annoy::AnnoyIndex; + using Base = ::Annoy::AnnoyIndex; public: - explicit AnnoyIndex(const uint64_t dim) : Base::AnnoyIndex(dim) {} + explicit AnnoyIndex(uint64_t dim) : Base::AnnoyIndex(dim) {} void serialize(WriteBuffer& ostr) const; void deserialize(ReadBuffer& istr); uint64_t getNumOfDimensions() const; @@ -40,10 +36,7 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule using AnnoyIndexPtr = std::shared_ptr; MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_); - MergeTreeIndexGranuleAnnoy( - const String & index_name_, - const Block & index_sample_block_, - AnnoyIndexPtr index_base_); + MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexPtr index_); ~MergeTreeIndexGranuleAnnoy() override = default; @@ -118,8 +111,7 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator() const override; - MergeTreeIndexConditionPtr createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const override; + MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; } @@ -131,4 +123,4 @@ private: } -#endif // ENABLE_ANNOY +#endif From 1018677f464756f0fcb4a1c26e7143e07d30dfe7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:39:04 +0000 Subject: [PATCH 1124/2223] Cosmetics: Move ctors into cpp file --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 12 ++++++++++++ src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 8 ++------ 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index b31779ff71c..e82f279c412 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -18,6 +18,12 @@ namespace DB namespace ApproximateNearestNeighbour { +template +AnnoyIndex::AnnoyIndex(uint64_t dim) + : Base::AnnoyIndex(dim) +{ +} + template void AnnoyIndex::serialize(WriteBuffer& ostr) const { @@ -290,6 +296,12 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI return result_vector; } +MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t number_of_trees_, const String& distance_name_) + : IMergeTreeIndex(index_) + , number_of_trees(number_of_trees_) + , distance_name(distance_name_) +{} + MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { if (distance_name == "L2Distance") diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index d591187fc64..2ccb3527b18 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -21,7 +21,7 @@ class AnnoyIndex : public ::Annoy::AnnoyIndex; public: - explicit AnnoyIndex(uint64_t dim) : Base::AnnoyIndex(dim) {} + explicit AnnoyIndex(uint64_t dim); void serialize(WriteBuffer& ostr) const; void deserialize(ReadBuffer& istr); uint64_t getNumOfDimensions() const; @@ -100,11 +100,7 @@ class MergeTreeIndexAnnoy : public IMergeTreeIndex { public: - MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t number_of_trees_, const String& distance_name_) - : IMergeTreeIndex(index_) - , number_of_trees(number_of_trees_) - , distance_name(distance_name_) - {} + MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t number_of_trees_, const String& distance_name_); ~MergeTreeIndexAnnoy() override = default; From 4631595cf658ae007e70b563a4631b2a95fd2439 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:44:20 +0000 Subject: [PATCH 1125/2223] Cosmetics: number_of_trees --> trees --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 14 +++++++------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 8 ++++---- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index e82f279c412..946889066b1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -118,17 +118,17 @@ template MergeTreeIndexAggregatorAnnoy::MergeTreeIndexAggregatorAnnoy( const String & index_name_, const Block & index_sample_block_, - uint64_t number_of_trees_) + uint64_t trees_) : index_name(index_name_) , index_sample_block(index_sample_block_) - , number_of_trees(number_of_trees_) + , trees(trees_) {} template MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy::getGranuleAndReset() { // NOLINTNEXTLINE(*) - index->build(static_cast(number_of_trees), /*number_of_threads=*/1); + index->build(static_cast(trees), /*number_of_threads=*/1); auto granule = std::make_shared>(index_name, index_sample_block, index); index = nullptr; return granule; @@ -296,9 +296,9 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI return result_vector; } -MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t number_of_trees_, const String& distance_name_) +MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String& distance_name_) : IMergeTreeIndex(index_) - , number_of_trees(number_of_trees_) + , trees(trees_) , distance_name(distance_name_) {} @@ -315,9 +315,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 if (distance_name == "L2Distance") - return std::make_shared>(index.name, index.sample_block, number_of_trees); + return std::make_shared>(index.name, index.sample_block, trees); if (distance_name == "cosineDistance") - return std::make_shared>(index.name, index.sample_block, number_of_trees); + return std::make_shared>(index.name, index.sample_block, trees); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 2ccb3527b18..2c41cf457ce 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -56,7 +56,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator using AnnoyIndex = ApproximateNearestNeighbour::AnnoyIndex; using AnnoyIndexPtr = std::shared_ptr; - MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, uint64_t number_of_trees); + MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, uint64_t trees); ~MergeTreeIndexAggregatorAnnoy() override = default; bool empty() const override { return !index || index->get_n_items() == 0; } @@ -65,7 +65,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator String index_name; Block index_sample_block; - const uint64_t number_of_trees; + const uint64_t trees; AnnoyIndexPtr index; }; @@ -100,7 +100,7 @@ class MergeTreeIndexAnnoy : public IMergeTreeIndex { public: - MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t number_of_trees_, const String& distance_name_); + MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String& distance_name_); ~MergeTreeIndexAnnoy() override = default; @@ -112,7 +112,7 @@ public: bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; } private: - const uint64_t number_of_trees; + const uint64_t trees; const String distance_name; }; From a8bf7af2918bf3101600cec4465c20e3a3faec0b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:47:05 +0000 Subject: [PATCH 1126/2223] Cosmetics: Move ErrorCodes to top --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 946889066b1..54df6f46ef2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -15,6 +15,16 @@ namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int INCORRECT_DATA; + extern const int INCORRECT_NUMBER_OF_COLUMNS; + extern const int INCORRECT_QUERY; + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; +} + namespace ApproximateNearestNeighbour { @@ -69,16 +79,6 @@ uint64_t AnnoyIndex::getNumOfDimensions() const } -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int INCORRECT_DATA; - extern const int INCORRECT_NUMBER_OF_COLUMNS; - extern const int INCORRECT_QUERY; - extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; -} - template MergeTreeIndexGranuleAnnoy::MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_) : index_name(index_name_) From f8c1f2bd33898e2fbc1bb4a963458a781778f722 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:48:12 +0000 Subject: [PATCH 1127/2223] Cosmetics: Remove absolute namespace qualification of Annoy library internals --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 54df6f46ef2..716624b7453 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -226,9 +226,9 @@ bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const { if (distance_name == "L2Distance") - return getUsefulRangesImpl<::Annoy::Euclidean>(idx_granule); + return getUsefulRangesImpl(idx_granule); else if (distance_name == "cosineDistance") - return getUsefulRangesImpl<::Annoy::Angular>(idx_granule); + return getUsefulRangesImpl(idx_granule); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } @@ -305,9 +305,9 @@ MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64 MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { if (distance_name == "L2Distance") - return std::make_shared>(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block); else if (distance_name == "cosineDistance") - return std::make_shared>(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } @@ -315,9 +315,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 if (distance_name == "L2Distance") - return std::make_shared>(index.name, index.sample_block, trees); + return std::make_shared>(index.name, index.sample_block, trees); if (distance_name == "cosineDistance") - return std::make_shared>(index.name, index.sample_block, trees); + return std::make_shared>(index.name, index.sample_block, trees); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } From e373cf682ade101bcc2fd288263e547690834ff8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:50:59 +0000 Subject: [PATCH 1128/2223] Cosmetics: Unwrap Annoy index from nested namespace --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 4 ---- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 11 ++--------- 2 files changed, 2 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 716624b7453..cbeb6540721 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -25,8 +25,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -namespace ApproximateNearestNeighbour -{ template AnnoyIndex::AnnoyIndex(uint64_t dim) @@ -76,8 +74,6 @@ uint64_t AnnoyIndex::getNumOfDimensions() const return Base::get_f(); } -} - template MergeTreeIndexGranuleAnnoy::MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 2c41cf457ce..44f5f8d8eb7 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -10,11 +10,6 @@ namespace DB { -// auxiliary namespace for working with spotify-annoy library -// mainly for serialization and deserialization of the index -namespace ApproximateNearestNeighbour -{ - template class AnnoyIndex : public ::Annoy::AnnoyIndex { @@ -27,12 +22,10 @@ public: uint64_t getNumOfDimensions() const; }; -} - template struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule { - using AnnoyIndex = ApproximateNearestNeighbour::AnnoyIndex; + using AnnoyIndex = AnnoyIndex; using AnnoyIndexPtr = std::shared_ptr; MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_); @@ -53,7 +46,7 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule template struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator { - using AnnoyIndex = ApproximateNearestNeighbour::AnnoyIndex; + using AnnoyIndex = AnnoyIndex; using AnnoyIndexPtr = std::shared_ptr; MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, uint64_t trees); From dc88d4e7422e9b0c19873297a9b7e2d3107ed052 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:53:07 +0000 Subject: [PATCH 1129/2223] Cosmetics: Factorize repeated typedefs into a single typedef --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 15 ++++++--------- 2 files changed, 10 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index cbeb6540721..58c59028075 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -86,7 +86,7 @@ template MergeTreeIndexGranuleAnnoy::MergeTreeIndexGranuleAnnoy( const String & index_name_, const Block & index_sample_block_, - AnnoyIndexPtr index_) + AnnoyIndexPtr index_) : index_name(index_name_) , index_sample_block(index_sample_block_) , index(std::move(index_)) @@ -106,7 +106,7 @@ void MergeTreeIndexGranuleAnnoy::deserializeBinary(ReadBuffer & istr, { uint64_t dimension; readIntBinary(dimension, istr); - index = std::make_shared(dimension); + index = std::make_shared>(dimension); index->deserialize(istr); } @@ -164,7 +164,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t if (offsets[i + 1] - offsets[i] != size) throw Exception(ErrorCodes::INCORRECT_DATA, "Arrays should have same length"); - index = std::make_shared(size); + index = std::make_shared>(size); index->add_item(index->get_n_items(), array.data()); /// add all rows from 1 to num_rows - 1 (this is the same as the beginning of the last element) @@ -190,7 +190,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t } assert(!data.empty()); if (!index) - index = std::make_shared(data[0].size()); + index = std::make_shared>(data[0].size()); for (const auto& item : data) index->add_item(index->get_n_items(), item.data()); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 44f5f8d8eb7..bb0d1883fc2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -22,14 +22,14 @@ public: uint64_t getNumOfDimensions() const; }; +template +using AnnoyIndexPtr = std::shared_ptr>; + template struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule { - using AnnoyIndex = AnnoyIndex; - using AnnoyIndexPtr = std::shared_ptr; - MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_); - MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexPtr index_); + MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexPtr index_); ~MergeTreeIndexGranuleAnnoy() override = default; @@ -40,15 +40,12 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule String index_name; Block index_sample_block; - AnnoyIndexPtr index; + AnnoyIndexPtr index; }; template struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator { - using AnnoyIndex = AnnoyIndex; - using AnnoyIndexPtr = std::shared_ptr; - MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, uint64_t trees); ~MergeTreeIndexAggregatorAnnoy() override = default; @@ -59,7 +56,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator String index_name; Block index_sample_block; const uint64_t trees; - AnnoyIndexPtr index; + AnnoyIndexPtr index; }; From 594572b0de48f7bd3be39f3abcb057f708b7fcf9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:55:01 +0000 Subject: [PATCH 1130/2223] Cosmetics: AnnoyIndex --> AnnoyIndexWithSerialization --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 16 ++++++++-------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 12 ++++++------ 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 58c59028075..60b9efcaf67 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -27,13 +27,13 @@ namespace ErrorCodes template -AnnoyIndex::AnnoyIndex(uint64_t dim) +AnnoyIndexWithSerialization::AnnoyIndexWithSerialization(uint64_t dim) : Base::AnnoyIndex(dim) { } template -void AnnoyIndex::serialize(WriteBuffer& ostr) const +void AnnoyIndexWithSerialization::serialize(WriteBuffer& ostr) const { chassert(Base::_built); writeIntBinary(Base::_s, ostr); @@ -47,7 +47,7 @@ void AnnoyIndex::serialize(WriteBuffer& ostr) const } template -void AnnoyIndex::deserialize(ReadBuffer& istr) +void AnnoyIndexWithSerialization::deserialize(ReadBuffer& istr) { chassert(!Base::_built); readIntBinary(Base::_s, istr); @@ -69,7 +69,7 @@ void AnnoyIndex::deserialize(ReadBuffer& istr) } template -uint64_t AnnoyIndex::getNumOfDimensions() const +uint64_t AnnoyIndexWithSerialization::getNumOfDimensions() const { return Base::get_f(); } @@ -86,7 +86,7 @@ template MergeTreeIndexGranuleAnnoy::MergeTreeIndexGranuleAnnoy( const String & index_name_, const Block & index_sample_block_, - AnnoyIndexPtr index_) + AnnoyIndexWithSerializationPtr index_) : index_name(index_name_) , index_sample_block(index_sample_block_) , index(std::move(index_)) @@ -106,7 +106,7 @@ void MergeTreeIndexGranuleAnnoy::deserializeBinary(ReadBuffer & istr, { uint64_t dimension; readIntBinary(dimension, istr); - index = std::make_shared>(dimension); + index = std::make_shared>(dimension); index->deserialize(istr); } @@ -164,7 +164,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t if (offsets[i + 1] - offsets[i] != size) throw Exception(ErrorCodes::INCORRECT_DATA, "Arrays should have same length"); - index = std::make_shared>(size); + index = std::make_shared>(size); index->add_item(index->get_n_items(), array.data()); /// add all rows from 1 to num_rows - 1 (this is the same as the beginning of the last element) @@ -190,7 +190,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t } assert(!data.empty()); if (!index) - index = std::make_shared>(data[0].size()); + index = std::make_shared>(data[0].size()); for (const auto& item : data) index->add_item(index->get_n_items(), item.data()); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index bb0d1883fc2..c5520ab5673 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -11,25 +11,25 @@ namespace DB { template -class AnnoyIndex : public ::Annoy::AnnoyIndex +class AnnoyIndexWithSerialization : public ::Annoy::AnnoyIndex { using Base = ::Annoy::AnnoyIndex; public: - explicit AnnoyIndex(uint64_t dim); + explicit AnnoyIndexWithSerialization(uint64_t dim); void serialize(WriteBuffer& ostr) const; void deserialize(ReadBuffer& istr); uint64_t getNumOfDimensions() const; }; template -using AnnoyIndexPtr = std::shared_ptr>; +using AnnoyIndexWithSerializationPtr = std::shared_ptr>; template struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule { MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_); - MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexPtr index_); + MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexWithSerializationPtr index_); ~MergeTreeIndexGranuleAnnoy() override = default; @@ -40,7 +40,7 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule String index_name; Block index_sample_block; - AnnoyIndexPtr index; + AnnoyIndexWithSerializationPtr index; }; template @@ -56,7 +56,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator String index_name; Block index_sample_block; const uint64_t trees; - AnnoyIndexPtr index; + AnnoyIndexWithSerializationPtr index; }; From c5ededdc5badab8fd18c6a321008157f2d1bbad5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:03:51 +0000 Subject: [PATCH 1131/2223] Cosmetics: Switch arguments in MTIConditionAnnoy ctor --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 60b9efcaf67..b7346e540d2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -202,8 +202,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( const IndexDescription & /*index*/, const SelectQueryInfo & query, - ContextPtr context, - const String& distance_name_) + const String& distance_name_, + ContextPtr context) : condition(query, context) , distance_name(distance_name_) {} @@ -320,7 +320,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition( const SelectQueryInfo & query, ContextPtr context) const { - return std::make_shared(index, query, context, distance_name); + return std::make_shared(index, query, distance_name, context); }; MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index c5520ab5673..9ebaa335542 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -66,8 +66,8 @@ public: MergeTreeIndexConditionAnnoy( const IndexDescription & index, const SelectQueryInfo & query, - ContextPtr context, - const String& distance_name); + const String& distance_name, + ContextPtr context); bool alwaysUnknownOrTrue() const override; From 3b77e4090221b9977fc98bef1e7fac5cf035eb6a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:05:25 +0000 Subject: [PATCH 1132/2223] Cosmetics: Remove dots from exception messages --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index b7346e540d2..e13ea91e56c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -345,7 +345,7 @@ static void assertIndexColumnsType(const Block & header) if (!WhichDataType(nested_type_index).isFloat32()) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported.", + "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", column_data_type_ptr->getName()); } else if (const auto * tuple_type = typeid_cast(column_data_type_ptr.get())) @@ -357,14 +357,14 @@ static void assertIndexColumnsType(const Block & header) if (!WhichDataType(nested_type_index).isFloat32()) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported.", + "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", column_data_type_ptr->getName()); } } else throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported.", + "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", column_data_type_ptr->getName()); } @@ -376,10 +376,10 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64 && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String."); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String"); if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String."); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String"); if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); From 423f69228239503b420153054a7c878b14aa2f47 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:07:43 +0000 Subject: [PATCH 1133/2223] Cosmetics: Remove unnecessary toString() --- src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexFullText.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexInverted.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexMinMax.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp index fe5a2a861f6..ef98accfbc6 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp @@ -42,7 +42,7 @@ void MergeTreeIndexAggregatorBloomFilter::update(const Block & block, size_t * p { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", toString(*pos), toString(block.rows())); + "Position: {}, Block rows: {}.", *pos, block.rows()); Block granule_index_block; size_t max_read_rows = std::min(block.rows() - *pos, limit); diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index e13ea91e56c..133c0a9a58a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -137,7 +137,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t throw Exception( ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. Position: {}, Block rows: {}.", - toString(*pos), toString(block.rows())); + *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); if (rows_read == 0) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 06fddd51cb8..b15bf4d6811 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -92,7 +92,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", toString(*pos), toString(block.rows())); + "Position: {}, Block rows: {}.", *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index baa11368c8b..e19187646cd 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -123,7 +123,7 @@ void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", toString(*pos), toString(block.rows())); + "Position: {}, Block rows: {}.", *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); auto row_id = store->getNextRowIDRange(rows_read); diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp index d80f7521430..3b011837cb3 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp @@ -122,7 +122,7 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", toString(*pos), toString(block.rows())); + "Position: {}, Block rows: {}.", *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 901636a2de9..120b3e43472 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -146,7 +146,7 @@ void MergeTreeIndexAggregatorSet::update(const Block & block, size_t * pos, size { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", toString(*pos), toString(block.rows())); + "Position: {}, Block rows: {}.", *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); From 7608e08eed5cb80f6be097ca572827a5090a0469 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:09:03 +0000 Subject: [PATCH 1134/2223] Cosmetics: more constness --- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 9ebaa335542..cde61af2891 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -38,8 +38,8 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule bool empty() const override { return !index.get(); } - String index_name; - Block index_sample_block; + const String index_name; + const Block index_sample_block; AnnoyIndexWithSerializationPtr index; }; @@ -53,8 +53,8 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator MergeTreeIndexGranulePtr getGranuleAndReset() override; void update(const Block & block, size_t * pos, size_t limit) override; - String index_name; - Block index_sample_block; + const String index_name; + const Block index_sample_block; const uint64_t trees; AnnoyIndexWithSerializationPtr index; }; @@ -81,7 +81,7 @@ private: template std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; - ApproximateNearestNeighbour::ANNCondition condition; + const ApproximateNearestNeighbour::ANNCondition condition; const String distance_name; }; From a3d4ede26cc57b80a74061072a8b4e7fbe2832bf Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:11:10 +0000 Subject: [PATCH 1135/2223] Cosmetics: Update exception messages --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 133c0a9a58a..cd094bbeeac 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -144,7 +144,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t return; if (index_sample_block.columns() > 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Only one column is supported"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column"); auto index_column_name = index_sample_block.getByPosition(0).name; const auto & column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); @@ -162,7 +162,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t size_t size = offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) if (offsets[i + 1] - offsets[i] != size) - throw Exception(ErrorCodes::INCORRECT_DATA, "Arrays should have same length"); + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); index = std::make_shared>(size); From 2b74daaa1744588c44414f5e87578107fd0e1b84 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:12:29 +0000 Subject: [PATCH 1136/2223] Cosmetics: make input switch a bit more idiomatic --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index cd094bbeeac..048200eb57f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -148,8 +148,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t auto index_column_name = index_sample_block.getByPosition(0).name; const auto & column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); - const auto & column_array = typeid_cast(column_cut.get()); - if (column_array) + + if (const auto & column_array = typeid_cast(column_cut.get())) { const auto & data = column_array->getData(); const auto & array = typeid_cast(data).getData(); @@ -171,14 +171,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t for (size_t current_row = 1; current_row < num_rows; ++current_row) index->add_item(index->get_n_items(), &array[offsets[current_row - 1]]); } - else + else if (const auto & column_tuple = typeid_cast(column_cut.get())) { - /// Other possible type of column is Tuple - const auto & column_tuple = typeid_cast(column_cut.get()); - - if (!column_tuple) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Wrong type was given to index."); - const auto & columns = column_tuple->getColumns(); std::vector> data{column_tuple->size(), std::vector()}; @@ -194,6 +188,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t for (const auto& item : data) index->add_item(index->get_n_items(), item.data()); } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array or Tuple column"); *pos += rows_read; } From 68ad903f4c9a6bc04f6847d586dd6e8ff08c4aae Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:13:22 +0000 Subject: [PATCH 1137/2223] Cosmetics: unglue * and & --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 048200eb57f..80c56a299be 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -149,10 +149,10 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t auto index_column_name = index_sample_block.getByPosition(0).name; const auto & column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); - if (const auto & column_array = typeid_cast(column_cut.get())) + if (const auto & column_array = typeid_cast(column_cut.get())) { const auto & data = column_array->getData(); - const auto & array = typeid_cast(data).getData(); + const auto & array = typeid_cast(data).getData(); if (array.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); const auto & offsets = column_array->getOffsets(); @@ -171,21 +171,21 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t for (size_t current_row = 1; current_row < num_rows; ++current_row) index->add_item(index->get_n_items(), &array[offsets[current_row - 1]]); } - else if (const auto & column_tuple = typeid_cast(column_cut.get())) + else if (const auto & column_tuple = typeid_cast(column_cut.get())) { const auto & columns = column_tuple->getColumns(); std::vector> data{column_tuple->size(), std::vector()}; - for (const auto& column : columns) + for (const auto & column : columns) { - const auto& pod_array = typeid_cast(column.get())->getData(); + const auto & pod_array = typeid_cast(column.get())->getData(); for (size_t i = 0; i < pod_array.size(); ++i) data[i].push_back(pod_array[i]); } assert(!data.empty()); if (!index) index = std::make_shared>(data[0].size()); - for (const auto& item : data) + for (const auto & item : data) index->add_item(index->get_n_items(), item.data()); } else From 15c9e235c44785a8dc67544728d607271fe10436 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:19:44 +0000 Subject: [PATCH 1138/2223] Cosmetics: add some comments + minor changes --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 80c56a299be..ea284892754 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -140,6 +140,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); + if (rows_read == 0) return; @@ -153,10 +154,12 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t { const auto & data = column_array->getData(); const auto & array = typeid_cast(data).getData(); + if (array.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); + const auto & offsets = column_array->getOffsets(); - size_t num_rows = offsets.size(); + const size_t num_rows = offsets.size(); /// Check all sizes are the same size_t size = offsets[0]; @@ -166,8 +169,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t index = std::make_shared>(size); + /// Add all rows of block index->add_item(index->get_n_items(), array.data()); - /// add all rows from 1 to num_rows - 1 (this is the same as the beginning of the last element) for (size_t current_row = 1; current_row < num_rows; ++current_row) index->add_item(index->get_n_items(), &array[offsets[current_row - 1]]); } @@ -175,6 +178,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t { const auto & columns = column_tuple->getColumns(); + /// TODO check if calling index->add_item() directly on the block's tuples is faster than materializing everything std::vector> data{column_tuple->size(), std::vector()}; for (const auto & column : columns) { @@ -182,9 +186,12 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t for (size_t i = 0; i < pod_array.size(); ++i) data[i].push_back(pod_array[i]); } - assert(!data.empty()); - if (!index) - index = std::make_shared>(data[0].size()); + + if (data.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Tuple has 0 rows, {} rows expected", rows_read); + + index = std::make_shared>(data[0].size()); + for (const auto & item : data) index->add_item(index->get_n_items(), item.data()); } From 828155ebefde62b610e9866a45a6d7fd71eb14b7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:21:47 +0000 Subject: [PATCH 1139/2223] Cosmetics: Move assertIndexColumnsType() into annoyIndexValidator() --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 36 ++++++++----------- 1 file changed, 15 insertions(+), 21 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index ea284892754..636fd384248 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -338,9 +338,22 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) return std::make_shared(index, param, distance_name); } -static void assertIndexColumnsType(const Block & header) +void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { - DataTypePtr column_data_type_ptr = header.getDataTypes()[0]; + if (index.arguments.size() > 2) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters"); + + if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64 + && index.arguments[0].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String"); + + if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String"); + + if (index.column_names.size() != 1 || index.data_types.size() != 1) + throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); + + DataTypePtr column_data_type_ptr = index.sample_block.getDataTypes()[0]; if (const auto * array_type = typeid_cast(column_data_type_ptr.get())) { @@ -369,25 +382,6 @@ static void assertIndexColumnsType(const Block & header) ErrorCodes::ILLEGAL_COLUMN, "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", column_data_type_ptr->getName()); - -} - -void annoyIndexValidator(const IndexDescription & index, bool /* attach */) -{ - if (index.arguments.size() > 2) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters"); - - if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64 - && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String"); - - if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String"); - - if (index.column_names.size() != 1 || index.data_types.size() != 1) - throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); - - assertIndexColumnsType(index.sample_block); } } From 0854d913723d967e07105857924f788817ac48c5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:27:01 +0000 Subject: [PATCH 1140/2223] Cosmetics: Rename variable --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 636fd384248..d75f4978f08 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -353,35 +353,35 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); - DataTypePtr column_data_type_ptr = index.sample_block.getDataTypes()[0]; + DataTypePtr data_type = index.sample_block.getDataTypes()[0]; - if (const auto * array_type = typeid_cast(column_data_type_ptr.get())) + if (const auto * data_type_array = typeid_cast(data_type.get())) { - TypeIndex nested_type_index = array_type->getNestedType()->getTypeId(); + TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) throw Exception( ErrorCodes::ILLEGAL_COLUMN, "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - column_data_type_ptr->getName()); + data_type->getName()); } - else if (const auto * tuple_type = typeid_cast(column_data_type_ptr.get())) + else if (const auto * data_type_tuple = typeid_cast(data_type.get())) { - const DataTypes & nested_types = tuple_type->getElements(); - for (const auto & type : nested_types) + const DataTypes & inner_types = data_type_tuple->getElements(); + for (const auto & inner_type : inner_types) { - TypeIndex nested_type_index = type->getTypeId(); + TypeIndex nested_type_index = inner_type->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - column_data_type_ptr->getName()); + "Unexpected inner_type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", + data_type->getName()); } } else throw Exception( ErrorCodes::ILLEGAL_COLUMN, "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - column_data_type_ptr->getName()); + data_type->getName()); } } From 62c8b9a7a11c0a3aae75a10a2a0ad61db5d63b55 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:28:20 +0000 Subject: [PATCH 1141/2223] Cosmetics: Factorize throw into lambda --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index d75f4978f08..d35f435c391 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -353,16 +353,21 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); + auto throw_unsupported_underlying_column_exception = [](DataTypePtr data_type) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32). Given type: {}", + data_type->getName()); + }; + DataTypePtr data_type = index.sample_block.getDataTypes()[0]; if (const auto * data_type_array = typeid_cast(data_type.get())) { TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - data_type->getName()); + throw_unsupported_underlying_column_exception(data_type); } else if (const auto * data_type_tuple = typeid_cast(data_type.get())) { @@ -371,17 +376,11 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { TypeIndex nested_type_index = inner_type->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Unexpected inner_type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - data_type->getName()); + throw_unsupported_underlying_column_exception(data_type); } } else - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - data_type->getName()); + throw_unsupported_underlying_column_exception(data_type); } } From 5d871c7fa09637eb89b69680ba5e5d256bddbdd7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:29:51 +0000 Subject: [PATCH 1142/2223] Cosmetics: +comments --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index d35f435c391..8f0cad48dc0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -340,6 +340,8 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { + /// Check number and type of Annoy index arguments: + if (index.arguments.size() > 2) throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters"); @@ -350,9 +352,13 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String"); + /// Check that the index is created on a single column + if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); + /// Check data type of indexed column: + auto throw_unsupported_underlying_column_exception = [](DataTypePtr data_type) { throw Exception( From f577bf35fc53262290e3c6e18352d1a446cbb642 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:35:38 +0000 Subject: [PATCH 1143/2223] Simplify Annoy parameterization --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 25 ++++--- tests/queries/0_stateless/02354_annoy.sh | 73 +------------------ .../0_stateless/02354_annoy_index.reference | 1 + .../queries/0_stateless/02354_annoy_index.sql | 26 +++++++ 4 files changed, 42 insertions(+), 83 deletions(-) create mode 100644 tests/queries/0_stateless/02354_annoy_index.reference create mode 100644 tests/queries/0_stateless/02354_annoy_index.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 8f0cad48dc0..dc353c97143 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -328,14 +328,16 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition( MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) { - uint64_t param = 100; + uint64_t trees = 100; String distance_name = "L2Distance"; - if (!index.arguments.empty() && !index.arguments[0].tryGet(param)) - if (!index.arguments[0].tryGet(distance_name)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Can't parse first argument"); - if (index.arguments.size() > 1 && !index.arguments[1].tryGet(distance_name)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Can't parse second argument"); - return std::make_shared(index, param, distance_name); + + if (!index.arguments.empty()) + distance_name = index.arguments[0].get(); + + if (index.arguments.size() > 1) + trees = index.arguments[1].get(); + + return std::make_shared(index, trees, distance_name); } void annoyIndexValidator(const IndexDescription & index, bool /* attach */) @@ -345,12 +347,11 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (index.arguments.size() > 2) throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters"); - if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64 - && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String"); + if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance function argument of Annoy index must be of type String"); - if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String"); + if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Number of trees argument of Annoy index must be UInt64"); /// Check that the index is created on a single column diff --git a/tests/queries/0_stateless/02354_annoy.sh b/tests/queries/0_stateless/02354_annoy.sh index 87258debf0f..1031ea81946 100755 --- a/tests/queries/0_stateless/02354_annoy.sh +++ b/tests/queries/0_stateless/02354_annoy.sh @@ -91,7 +91,7 @@ CREATE TABLE 02354_annoy_cosine ( id Int32, embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy(100, 'cosineDistance') GRANULARITY 1 + INDEX annoy_index embedding TYPE annoy('cosineDistance', 100) GRANULARITY 1 ) ENGINE = MergeTree ORDER BY id @@ -120,7 +120,7 @@ CREATE TABLE 02354_annoy_cosine ( id Int32, embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy(100, 'cosineDistance') GRANULARITY 1 + INDEX annoy_index embedding TYPE annoy('cosineDistance', 100) GRANULARITY 1 ) ENGINE = MergeTree ORDER BY id @@ -141,72 +141,3 @@ ORDER BY cosineDistance(embedding, [0.0, 0.0, 10.0]) LIMIT 3; DROP TABLE IF EXISTS 02354_annoy_cosine; " | grep "annoy_index" - -# # Check that weird base columns are rejected -$CLICKHOUSE_CLIENT -nm --allow_experimental_annoy_index=1 -q " -DROP TABLE IF EXISTS 02354_annoy; - --- Index spans >1 column - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Array(Float32), - INDEX annoy_index (embedding, id) TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 7 } - --- Index must be created on Array(Float32) or Tuple(Float32) - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Float32, - INDEX annoy_index embedding TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } - - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Array(Float64), - INDEX annoy_index embedding TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Tuple(Float32, Float64), - INDEX annoy_index embedding TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Array(LowCardinality(Float32)), - INDEX annoy_index embedding TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Array(Nullable(Float32)), - INDEX annoy_index embedding TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 }" diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference new file mode 100644 index 00000000000..2d162500f67 --- /dev/null +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -0,0 +1 @@ +Negative tests diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql new file mode 100644 index 00000000000..8df9af1ee73 --- /dev/null +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -0,0 +1,26 @@ +-- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check + +SET allow_experimental_annoy_index = 1; + +DROP TABLE IF EXISTS tab; + +SELECT 'Negative tests'; + +-- must have at most 2 arguments +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } + +-- first argument must be UInt64 +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } + +-- 2nd argument must be String +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('L2Distance', 'not an UInt64')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } + +-- must be created on single column +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index (embedding, id) TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } + +-- must be created on Array/Tuple(Float32) columns +SET allow_suspicious_low_cardinality_types = 1; +CREATE TABLE tab(id Int32, embedding Float32, INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, embedding Array(Float64), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, embedding LowCardinality(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, embedding Nullable(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } From 03b6856556c67fe1c0f7c1df0b28b19556fd3fcc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:39:57 +0000 Subject: [PATCH 1144/2223] Cosmetics: distance_name --> distance_function --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 41 +++++++++---------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 8 ++-- 2 files changed, 24 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index dc353c97143..a9679453655 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -205,31 +205,31 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( const IndexDescription & /*index*/, const SelectQueryInfo & query, - const String& distance_name_, + const String & distance_function_, ContextPtr context) : condition(query, context) - , distance_name(distance_name_) + , distance_function(distance_function_) {} -bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr /* idx_granule */) const +bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr /*idx_granule*/) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "mayBeTrueOnGranule is not supported for ANN skip indexes"); } bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const { - return condition.alwaysUnknownOrTrue(distance_name); + return condition.alwaysUnknownOrTrue(distance_function); } std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const { - if (distance_name == "L2Distance") + if (distance_function == "L2Distance") return getUsefulRangesImpl(idx_granule); - else if (distance_name == "cosineDistance") + else if (distance_function == "cosineDistance") return getUsefulRangesImpl(idx_granule); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); } @@ -295,49 +295,48 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI return result_vector; } -MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String& distance_name_) +MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String & distance_function_) : IMergeTreeIndex(index_) , trees(trees_) - , distance_name(distance_name_) + , distance_function(distance_function_) {} MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { - if (distance_name == "L2Distance") + if (distance_function == "L2Distance") return std::make_shared>(index.name, index.sample_block); - else if (distance_name == "cosineDistance") + else if (distance_function == "cosineDistance") return std::make_shared>(index.name, index.sample_block); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); } MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 - if (distance_name == "L2Distance") + if (distance_function == "L2Distance") return std::make_shared>(index.name, index.sample_block, trees); - if (distance_name == "cosineDistance") + if (distance_function == "cosineDistance") return std::make_shared>(index.name, index.sample_block, trees); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); } -MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const +MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const { - return std::make_shared(index, query, distance_name, context); + return std::make_shared(index, query, distance_function, context); }; MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) { uint64_t trees = 100; - String distance_name = "L2Distance"; + String distance_function = "L2Distance"; if (!index.arguments.empty()) - distance_name = index.arguments[0].get(); + distance_function = index.arguments[0].get(); if (index.arguments.size() > 1) trees = index.arguments[1].get(); - return std::make_shared(index, trees, distance_name); + return std::make_shared(index, trees, distance_function); } void annoyIndexValidator(const IndexDescription & index, bool /* attach */) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index cde61af2891..9741412e3fa 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -66,7 +66,7 @@ public: MergeTreeIndexConditionAnnoy( const IndexDescription & index, const SelectQueryInfo & query, - const String& distance_name, + const String& distance_function, ContextPtr context); bool alwaysUnknownOrTrue() const override; @@ -82,7 +82,7 @@ private: std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; const ApproximateNearestNeighbour::ANNCondition condition; - const String distance_name; + const String distance_function; }; @@ -90,7 +90,7 @@ class MergeTreeIndexAnnoy : public IMergeTreeIndex { public: - MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String& distance_name_); + MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String & distance_function_); ~MergeTreeIndexAnnoy() override = default; @@ -103,7 +103,7 @@ public: private: const uint64_t trees; - const String distance_name; + const String distance_function; }; From 18304f5aeff627b47965ffd5c07ae6c5d61bface Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:51:21 +0000 Subject: [PATCH 1145/2223] Check distance function in CREATE TABLE instead of first INSERT --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 13 +++++++++++-- tests/queries/0_stateless/02354_annoy_index.sql | 3 +++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index a9679453655..12446623c30 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -307,7 +307,7 @@ MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const return std::make_shared>(index.name, index.sample_block); else if (distance_function == "cosineDistance") return std::make_shared>(index.name, index.sample_block); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); + std::unreachable(); } MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const @@ -317,7 +317,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const return std::make_shared>(index.name, index.sample_block, trees); if (distance_function == "cosineDistance") return std::make_shared>(index.name, index.sample_block, trees); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); + std::unreachable(); } MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const @@ -357,6 +357,15 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); + /// Check that a supported metric was passed as first argument + + if (!index.arguments.empty()) + { + String distance_name = index.arguments[0].get(); + if (distance_name != "L2Distance" && distance_name != "cosineDistance") + throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index supports only distance functions 'L2Distance' and 'cosineDistance'. Given distance function: {}", distance_name); + } + /// Check data type of indexed column: auto throw_unsupported_underlying_column_exception = [](DataTypePtr data_type) diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 8df9af1ee73..3a5fb6817ff 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -24,3 +24,6 @@ CREATE TABLE tab(id Int32, embedding Float32, INDEX annoy_index embedding TYPE a CREATE TABLE tab(id Int32, embedding Array(Float64), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, embedding LowCardinality(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, embedding Nullable(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } + +-- reject unsupported distance functions +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } From d3158a28770339f36f9c0e69acb6d271ced351f6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:56:27 +0000 Subject: [PATCH 1146/2223] Cosmetics: Consolidate parameters --- src/Core/Settings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6a0833aef60..8055e9b4880 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -719,7 +719,6 @@ class IColumn; \ M(Bool, parallelize_output_from_storages, true, "Parallelize output for reading step from storage. It allows parallelizing query processing right after reading from storage if possible", 0) \ M(String, insert_deduplication_token, "", "If not empty, used for duplicate detection instead of data digest", 0) \ - M(String, ann_index_select_query_params, "", "Parameters passed to ANN indexes in SELECT queries, the format is 'param1=x, param2=y, ...'", 0) \ M(Bool, count_distinct_optimization, false, "Rewrite count distinct to subquery of group by", 0) \ M(Bool, throw_if_no_data_to_insert, true, "Enables or disables empty INSERTs, enabled by default", 0) \ M(Bool, compatibility_ignore_auto_increment_in_create_table, false, "Ignore AUTO_INCREMENT keyword in column declaration if true, otherwise return error. It simplifies migration from MySQL", 0) \ @@ -742,7 +741,8 @@ class IColumn; M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions (hashid, etc)", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ - M(UInt64, max_limit_for_ann_queries, 1000000, "Maximum limit value for using ANN indexes is used to prevent memory overflow in search queries for indexes", 0) \ + M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexs.", 0) \ + M(String, ann_index_select_query_params, "", "Parameters passed to ANN indexes in SELECT queries, the format is 'param1=x, param2=y, ...'", 0) \ M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \ 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) \ From a973ac5dbb99e7fc624742b34fc507935cd792e4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:03:50 +0000 Subject: [PATCH 1147/2223] Replace weird generic ANN setting by Annoy-specific parameter --- .../mergetree-family/annindexes.md | 3 ++- src/Core/Settings.h | 2 +- src/Storages/MergeTree/CommonANNIndexes.cpp | 1 - src/Storages/MergeTree/CommonANNIndexes.h | 7 ------ .../MergeTree/MergeTreeIndexAnnoy.cpp | 18 +++----------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 5 ++-- .../0_stateless/02354_annoy_index.reference | 12 ++++++++++ .../queries/0_stateless/02354_annoy_index.sql | 24 +++++++++++++++++++ 8 files changed, 45 insertions(+), 27 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 03617a1a709..9b4de150235 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -135,7 +135,8 @@ ORDER BY id; Annoy supports `L2Distance` and `cosineDistance`. -In the `SELECT` in the settings (`ann_index_select_query_params`) you can specify the size of the internal buffer (more details in the description above or in the [original repository](https://github.com/spotify/annoy)). During the query it will inspect up to `search_k` nodes which defaults to `n_trees * n` if not provided. `search_k` gives you a run-time trade-off between better accuracy and speed. +Setting `search_k` (default `LIMIT * NumTrees`) determines how many nodes the Annoy index will inspect during SELECT queries. The setting +can be used to balance performance and accuracy at runtime. __Example__: ``` sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8055e9b4880..3e10f48a2fb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -742,7 +742,7 @@ class IColumn; M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexs.", 0) \ - M(String, ann_index_select_query_params, "", "Parameters passed to ANN indexes in SELECT queries, the format is 'param1=x, param2=y, ...'", 0) \ + M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \ M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \ 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) \ diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index f0c6f256f73..4748c869f83 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -60,7 +60,6 @@ namespace ApproximateNearestNeighbour ANNCondition::ANNCondition(const SelectQueryInfo & query_info, ContextPtr context) : block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, - ann_index_select_query_params{context->getSettings().get("ann_index_select_query_params").get()}, index_granularity{context->getMergeTreeSettings().get("index_granularity").get()}, limit_restriction{context->getSettings().get("max_limit_for_ann_queries").get()}, index_is_useful{checkQueryStructure(query_info)} {} diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index fefb9584863..4253bce703a 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -82,8 +82,6 @@ struct ANNQueryInformation * spaceDimension(which is targetVector's components count) * column * objects count from LIMIT clause(for both queries) - * settings str, if query has settings section with new 'ann_index_select_query_params' value, - than you can get the new value(empty by default) calling method getSettingsStr * queryHasOrderByClause and queryHasWhereClause return true if query matches the type Search query type is also recognized for PREWHERE clause @@ -121,11 +119,7 @@ public: // length's value from LIMIT clause UInt64 getLimit() const; - // value of 'ann_index_select_query_params' if have in SETTINGS clause, empty string otherwise - String getParamsStr() const { return ann_index_select_query_params; } - private: - struct RPNElement { enum Function @@ -217,7 +211,6 @@ private: std::optional query_information; // Get from settings ANNIndex parameters - String ann_index_select_query_params; UInt64 index_granularity; /// only queries with a lower limit can be considered to avoid memory overflow UInt64 limit_restriction; diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 12446623c30..6ffb7aecb7f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -209,6 +210,7 @@ MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( ContextPtr context) : condition(query, context) , distance_function(distance_function_) + , search_k(context->getSettings().get("annoy_index_search_k_nodes").get()) {} @@ -264,21 +266,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI neighbors.reserve(limit); distances.reserve(limit); - int k_search = -1; - String params_str = condition.getParamsStr(); - if (!params_str.empty()) - { - try - { - /// k_search=... (algorithm will inspect up to search_k nodes which defaults to n_trees * n if not provided) - k_search = std::stoi(params_str.data() + 9); - } - catch (...) - { - throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting of the annoy index should be int"); - } - } - annoy->get_nns_by_vector(target_vec.data(), limit, k_search, &neighbors, &distances); + annoy->get_nns_by_vector(target_vec.data(), limit, static_cast(search_k), &neighbors, &distances); std::unordered_set granule_numbers; for (size_t i = 0; i < neighbors.size(); ++i) { diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 9741412e3fa..fbc6b21fa6b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -69,20 +69,21 @@ public: const String& distance_function, ContextPtr context); + ~MergeTreeIndexConditionAnnoy() override = default; + bool alwaysUnknownOrTrue() const override; bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override; std::vector getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const override; - ~MergeTreeIndexConditionAnnoy() override = default; - private: template std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; const ApproximateNearestNeighbour::ANNCondition condition; const String distance_function; + const Int64 search_k; }; diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 2d162500f67..7da442cb905 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -1 +1,13 @@ +parameter annoy_index_search_k_nodes +parameter max_limit_for_ann_queries +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 Negative tests diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 3a5fb6817ff..3590b7d316e 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -4,6 +4,30 @@ SET allow_experimental_annoy_index = 1; DROP TABLE IF EXISTS tab; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; +-- SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); + +SELECT 'parameter annoy_index_search_k_nodes'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [5.3, 7.3, 2.1]) +LIMIT 5 +SETTINGS annoy_index_search_k_nodes=0; -- searches zero nodes --> no results + +SELECT 'parameter max_limit_for_ann_queries'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [5.3, 7.3, 2.1]) +LIMIT 5 +SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index + +DROP TABLE tab; + +DROP TABLE IF EXISTS tab; + SELECT 'Negative tests'; -- must have at most 2 arguments From 6d3431d2ff0325282ea373bfeafed6a2e1946577 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:12:40 +0000 Subject: [PATCH 1148/2223] Cosmetics: Sort includes --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 6ffb7aecb7f..fe3ea322b91 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -2,15 +2,15 @@ #include +#include #include #include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include namespace DB From f800940639bbe490e619594db62a83f7f8f2f80d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:13:33 +0000 Subject: [PATCH 1149/2223] Cosmetics: Shuffle statements --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 16b27c2c820..aa340d6afc1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1686,7 +1686,6 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( { if (index_mark != index_range.begin || !granule || last_index_mark != index_range.begin) granule = reader.read(); - const auto * gin_filter_condition = dynamic_cast(&*condition); // Cast to Ann condition auto ann_condition = std::dynamic_pointer_cast(condition); if (ann_condition != nullptr) @@ -1714,6 +1713,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( } bool result = false; + const auto * gin_filter_condition = dynamic_cast(&*condition); if (!gin_filter_condition) result = condition->mayBeTrueOnGranule(granule); else From 660760782ad6357c34f55113f3a9522e94b30dd3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:16:03 +0000 Subject: [PATCH 1150/2223] Rewrite ANN docs --- .../mergetree-family/annindexes.md | 140 ++++++++++-------- 1 file changed, 78 insertions(+), 62 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 9b4de150235..0cc1cff2dad 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -1,78 +1,89 @@ # Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex} -The main task that indexes achieve is to quickly find nearest neighbors for multidimensional data. An example of such a problem can be finding similar pictures (texts) for a given picture (text). That problem can be reduced to finding the nearest [embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning). They can be created from data using [UDF](/docs/en/sql-reference/functions/index.md/#executable-user-defined-functions). +Nearest neighborhood search refers to the problem of finding the point(s) with the smallest distance to a given point in an n-dimensional +space. Since exact search is in practice usually typically too slow, the task is often solved with approximate algorithms. A popular use +case of of neighbor search is finding similar pictures (texts) for a given picture (text). Pictures (texts) can be decomposed into +[embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning), and instead of +comparing pictures (texts) pixel-by-pixel (character-by-character), only the embeddings are compared. -The next queries find the closest neighbors in N-dimensional space using the L2 (Euclidean) distance: -``` sql -SELECT * -FROM table_name -WHERE L2Distance(Column, Point) < MaxDistance +In terms of SQL, the problem can be expressed as follows: + +``` sql +SELECT * +FROM table +WHERE L2Distance(column, Point) < MaxDistance LIMIT N ``` -``` sql -SELECT * -FROM table_name -ORDER BY L2Distance(Column, Point) +``` sql +SELECT * +FROM table +ORDER BY L2Distance(column, Point) LIMIT N ``` -But it will take some time for execution because of the long calculation of the distance between `TargetEmbedding` and all other vectors. This is where ANN indexes can help. They store a compact approximation of the search space (e.g. using clustering, search trees, etc.) and are able to compute approximate neighbors quickly. + +The queries are expensive because the L2 distance (Euclidean distance) between all points in `column` and `Point` must be computed. To speed this process up, ANN indexes store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer quickly. ## Indexes Structure -Approximate Nearest Neighbor Search Indexes (`ANNIndexes`) are similar to skip indexes. They are constructed by some granules and determine which of them should be skipped. Compared to skip indices, ANN indices use their results not only to skip some group of granules, but also to select particular granules from a set of granules. +Approximate Nearest Neighbor Search Indexes (or `ANNIndexes`) are similar to skip indexes. They are constructed over granules and determine which granules can be skipped. Compared to skip indices, ANN indices are not only able to skip granules, they can also to select particular granules from a set of granules. -`ANNIndexes` are designed to speed up two types of queries: +`ANNIndexes` support two types of queries: -- ###### Type 1: Where - ``` sql - SELECT * - FROM table_name - WHERE DistanceFunction(Column, Point) < MaxDistance +- WHERE queries: + ``` sql + SELECT * + FROM table + WHERE DistanceFunction(column, Point) < MaxDistance LIMIT N ``` -- ###### Type 2: Order by + +- ORDER BY queries: ``` sql - SELECT * - FROM table_name [WHERE ...] - ORDER BY DistanceFunction(Column, Point) + SELECT * + FROM table [WHERE ...] + ORDER BY DistanceFunction(column, Point) LIMIT N ``` -In these queries, `DistanceFunction` is selected from [distance functions](/docs/en/sql-reference/functions/distance-functions.md). `Point` is a known vector (something like `(0.1, 0.1, ... )`). To avoid writing large vectors, use [client parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters). `Value` - a float value that will bound the neighbourhood. +`DistanceFunction` is a [distance functions](/docs/en/sql-reference/functions/distance-functions.md), `Point` is a given vector (e.g. `(0.17, 0.33, ...)`) and `MaxDistance` is a float value which restricts the size of the neighbourhood. -:::note -ANN index can't speed up query that satisfies both types (`where + order by`, only one of them). All queries must have the limit, as algorithms are used to find nearest neighbors and need a specific number of them. -::: +To avoid writing large vectors, you can also use [query parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. -:::note -Indexes are applied only to queries with a limit less than the `max_limit_for_ann_queries` setting. This helps to avoid memory overflows in queries with a large limit. `max_limit_for_ann_queries` setting can be changed if you know you can provide enough memory. The default value is `1000000`. -::: +```bash +clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(embedding, {vec: Array(Float32)}) < 1.0" +``` -Both types of queries are handled the same way. The indexes get `n` neighbors (where `n` is taken from the `LIMIT` clause) and work with them. In `ORDER BY` query they remember the numbers of all parts of the granule that have at least one of neighbor. In `WHERE` query they remember only those parts that satisfy the requirements. +ANN index cannot speed up query that contain both `WHERE` and `ORDER BY`. Queries must have a limit, as the approximate algorithms used to determine the nearest neighbors require a specific number of them. + +Indexes are only used for queries with a `LIMIT` value smaller than setting `max_limit_for_ann_queries` (default: 1 million rows). This helps to prevent memory overflows in queries with a large limit. + +Both types of queries are processed similarly. The indexes are passed the number of neighbors `N`. In `ORDER BY` query they remember the numbers of all parts of the granule that have at least one of neighbor. In `WHERE` query they remember only those parts that satisfy the requirements. -## Create table with ANNIndex +## Creating Tables with an ANN Index -This feature is disabled by default. To enable it, set `allow_experimental_annoy_index` to 1. Also, this feature is disabled on ARM, due to likely problems with the algorithm. +As long as ANN indexes are experimental, you first need to `SET allow_experimental_annoy_index = 1`. + +Syntax: ```sql -CREATE TABLE t +CREATE TABLE table ( `id` Int64, - `data` Tuple(Float32, Float32, Float32), - INDEX ann_index_name data TYPE ann_index_type(ann_index_parameters) GRANULARITY N + `embedding` Tuple(Float32, Float32, Float32), + INDEX embedding TYPE () GRANULARITY N ) ENGINE = MergeTree ORDER BY id; ``` ```sql -CREATE TABLE t +CREATE TABLE table ( `id` Int64, - `data` Array(Float32), - INDEX ann_index_name data TYPE ann_index_type(ann_index_parameters) GRANULARITY N + `embedding` Array(Float32), + INDEX embedding TYPE () GRANULARITY N ) ENGINE = MergeTree ORDER BY id; @@ -80,69 +91,74 @@ ORDER BY id; With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyper parameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes. -As the indexes are built only during insertions into table, `INSERT` and `OPTIMIZE` queries are slower than for ordinary table. At this stage indexes remember all the information about the given data. ANNIndexes should be used if you have immutable or rarely changed data and many read requests. - -You can create your table with index which uses certain algorithm. Now only indices based on the following algorithms are supported: +Note that ANN indexes are built during column insertion and merge, i.e. `INSERT` and `OPTIMIZE` statements are slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changing data in conjunction with many read requests. # Index list + - [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy) # Annoy {#annoy} -Implementation of the algorithm was taken from [this repository](https://github.com/spotify/annoy). + +(currently disabled on ARM due to problems with the algorithm) + +This ANN index type implements [Annoy indexes](https://github.com/spotify/annoy). Short description of the algorithm: The algorithm recursively divides in half all space by random linear surfaces (lines in 2D, planes in 3D etc.). Thus it makes tree of polyhedrons and points that they contains. Repeating the operation several times for greater accuracy it creates a forest. To find K Nearest Neighbours it goes down through the trees and fills the buffer of closest points using the priority queue of polyhedrons. Next, it sorts buffer and return the nearest K points. -__Examples__: +Examples: + ```sql -CREATE TABLE t +CREATE TABLE table ( id Int64, - data Tuple(Float32, Float32, Float32), - INDEX ann_index_name data TYPE annoy(NumTrees, DistanceName) GRANULARITY N + embedding Tuple(Float32, Float32, Float32), + INDEX embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N ) ENGINE = MergeTree ORDER BY id; ``` ```sql -CREATE TABLE t +CREATE TABLE table ( id Int64, - data Array(Float32), - INDEX ann_index_name data TYPE annoy(NumTrees, DistanceName) GRANULARITY N + embedding Array(Float32), + INDEX embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N ) ENGINE = MergeTree ORDER BY id; ``` :::note -Table with array field will work faster, but all arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(data) = 256`. +Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(embedding) = 256`. ::: -Parameter `NumTrees` is the number of trees which the algorithm will create. The bigger it is, the slower (approximately linear) it works (in both `CREATE` and `SELECT` requests), but the better accuracy you get (adjusted for randomness). By default it is set to `100`. Parameter `DistanceName` is name of distance function. By default it is set to `L2Distance`. It can be set without changing first parameter, for example +Parameter `DistanceName` is name of a distance function with default `L2Distance`. Parameter `NumTrees` (default: 100) is the number of trees which the algorithm will create. Higher values of `NumTree` mean slower `CREATE` and `SELECT` statements (approximately linearly), but increase the accuracy of search results. + ```sql -CREATE TABLE t +CREATE TABLE table ( id Int64, - data Array(Float32), - INDEX ann_index_name data TYPE annoy('cosineDistance') GRANULARITY N + embedding Array(Float32), + INDEX ann_index_name embedding TYPE annoy('cosineDistance') GRANULARITY N ) ENGINE = MergeTree ORDER BY id; ``` -Annoy supports `L2Distance` and `cosineDistance`. +Annoy currently supports `L2Distance` and `cosineDistance` as distance functions. -Setting `search_k` (default `LIMIT * NumTrees`) determines how many nodes the Annoy index will inspect during SELECT queries. The setting -can be used to balance performance and accuracy at runtime. +Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many nodes are inspected during SELECTs. It can be used to +balance runtime and accuracy at runtime. + +Example: -__Example__: ``` sql -SELECT * -FROM table_name [WHERE ...] -ORDER BY L2Distance(Column, Point) +SELECT * +FROM table_name [WHERE ...] +ORDER BY L2Distance(column, Point) LIMIT N -SETTING ann_index_select_query_params=`k_search=100` +SETTINGS annoy_index_search_k_nodes=100 ``` From 662b0fb822e6e297c1e490fef8e2c09e22f4af09 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:31:33 +0000 Subject: [PATCH 1151/2223] Cosmetics: Unwrap common ANN code from nested namespace --- src/Storages/MergeTree/CommonANNIndexes.cpp | 75 +++++++++---------- src/Storages/MergeTree/CommonANNIndexes.h | 27 +++---- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreeIndexAnnoy.cpp | 3 +- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 4 +- 5 files changed, 50 insertions(+), 61 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 4748c869f83..5c42774fb24 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -24,10 +24,8 @@ namespace ErrorCodes namespace { -namespace ANN = ApproximateNearestNeighbour; - template -void extractTargetVectorFromLiteral(ANN::ANNQueryInformation::Embedding & target, Literal literal) +void extractTargetVectorFromLiteral(ApproximateNearestNeighborInformation::Embedding & target, Literal literal) { Float64 float_element_of_target_vector; Int64 int_element_of_target_vector; @@ -43,28 +41,25 @@ void extractTargetVectorFromLiteral(ANN::ANNQueryInformation::Embedding & target } } -ANN::ANNQueryInformation::Metric castMetricFromStringToType(String metric_name) +ApproximateNearestNeighborInformation::Metric castMetricFromStringToType(String metric_name) { if (metric_name == "L2Distance") - return ANN::ANNQueryInformation::Metric::L2; + return ApproximateNearestNeighborInformation::Metric::L2; if (metric_name == "LpDistance") - return ANN::ANNQueryInformation::Metric::Lp; - return ANN::ANNQueryInformation::Metric::Unknown; + return ApproximateNearestNeighborInformation::Metric::Lp; + return ApproximateNearestNeighborInformation::Metric::Unknown; } } -namespace ApproximateNearestNeighbour -{ - -ANNCondition::ANNCondition(const SelectQueryInfo & query_info, +ApproximateNearestNeighborCondition::ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context) : block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, index_granularity{context->getMergeTreeSettings().get("index_granularity").get()}, limit_restriction{context->getSettings().get("max_limit_for_ann_queries").get()}, index_is_useful{checkQueryStructure(query_info)} {} -bool ANNCondition::alwaysUnknownOrTrue(String metric_name) const +bool ApproximateNearestNeighborCondition::alwaysUnknownOrTrue(String metric_name) const { if (!index_is_useful) return true; // Query isn't supported @@ -72,64 +67,64 @@ bool ANNCondition::alwaysUnknownOrTrue(String metric_name) const return !(castMetricFromStringToType(metric_name) == query_information->metric); } -float ANNCondition::getComparisonDistanceForWhereQuery() const +float ApproximateNearestNeighborCondition::getComparisonDistanceForWhereQuery() const { if (index_is_useful && query_information.has_value() - && query_information->query_type == ANNQueryInformation::Type::Where) + && query_information->query_type == ApproximateNearestNeighborInformation::Type::Where) return query_information->distance; throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported method for this query type"); } -UInt64 ANNCondition::getLimit() const +UInt64 ApproximateNearestNeighborCondition::getLimit() const { if (index_is_useful && query_information.has_value()) return query_information->limit; throw Exception(ErrorCodes::LOGICAL_ERROR, "No LIMIT section in query, not supported"); } -std::vector ANNCondition::getTargetVector() const +std::vector ApproximateNearestNeighborCondition::getTargetVector() const { if (index_is_useful && query_information.has_value()) return query_information->target; throw Exception(ErrorCodes::LOGICAL_ERROR, "Target vector was requested for useless or uninitialized index."); } -size_t ANNCondition::getNumOfDimensions() const +size_t ApproximateNearestNeighborCondition::getNumOfDimensions() const { if (index_is_useful && query_information.has_value()) return query_information->target.size(); throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of dimensions was requested for useless or uninitialized index."); } -String ANNCondition::getColumnName() const +String ApproximateNearestNeighborCondition::getColumnName() const { if (index_is_useful && query_information.has_value()) return query_information->column_name; throw Exception(ErrorCodes::LOGICAL_ERROR, "Column name was requested for useless or uninitialized index."); } -ANNQueryInformation::Metric ANNCondition::getMetricType() const +ApproximateNearestNeighborInformation::Metric ApproximateNearestNeighborCondition::getMetricType() const { if (index_is_useful && query_information.has_value()) return query_information->metric; throw Exception(ErrorCodes::LOGICAL_ERROR, "Metric name was requested for useless or uninitialized index."); } -float ANNCondition::getPValueForLpDistance() const +float ApproximateNearestNeighborCondition::getPValueForLpDistance() const { if (index_is_useful && query_information.has_value()) return query_information->p_for_lp_dist; throw Exception(ErrorCodes::LOGICAL_ERROR, "P from LPDistance was requested for useless or uninitialized index."); } -ANNQueryInformation::Type ANNCondition::getQueryType() const +ApproximateNearestNeighborInformation::Type ApproximateNearestNeighborCondition::getQueryType() const { if (index_is_useful && query_information.has_value()) return query_information->query_type; throw Exception(ErrorCodes::LOGICAL_ERROR, "Query type was requested for useless or uninitialized index."); } -bool ANNCondition::checkQueryStructure(const SelectQueryInfo & query) +bool ApproximateNearestNeighborCondition::checkQueryStructure(const SelectQueryInfo & query) { // RPN-s for different sections of the query RPN rpn_prewhere_clause; @@ -138,9 +133,9 @@ bool ANNCondition::checkQueryStructure(const SelectQueryInfo & query) RPNElement rpn_limit; UInt64 limit; - ANNQueryInformation prewhere_info; - ANNQueryInformation where_info; - ANNQueryInformation order_by_info; + ApproximateNearestNeighborInformation prewhere_info; + ApproximateNearestNeighborInformation where_info; + ApproximateNearestNeighborInformation order_by_info; // Build rpns for query sections const auto & select = query.query->as(); @@ -195,7 +190,7 @@ bool ANNCondition::checkQueryStructure(const SelectQueryInfo & query) return query_information.has_value(); } -void ANNCondition::traverseAST(const ASTPtr & node, RPN & rpn) +void ApproximateNearestNeighborCondition::traverseAST(const ASTPtr & node, RPN & rpn) { // If the node is ASTFunction, it may have children nodes if (const auto * func = node->as()) @@ -214,7 +209,7 @@ void ANNCondition::traverseAST(const ASTPtr & node, RPN & rpn) rpn.emplace_back(std::move(element)); } -bool ANNCondition::traverseAtomAST(const ASTPtr & node, RPNElement & out) +bool ApproximateNearestNeighborCondition::traverseAtomAST(const ASTPtr & node, RPNElement & out) { // Match Functions if (const auto * function = node->as()) @@ -259,7 +254,7 @@ bool ANNCondition::traverseAtomAST(const ASTPtr & node, RPNElement & out) return tryCastToConstType(node, out); } -bool ANNCondition::tryCastToConstType(const ASTPtr & node, RPNElement & out) +bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node, RPNElement & out) { Field const_value; DataTypePtr const_type; @@ -318,18 +313,18 @@ bool ANNCondition::tryCastToConstType(const ASTPtr & node, RPNElement & out) return false; } -void ANNCondition::traverseOrderByAST(const ASTPtr & node, RPN & rpn) +void ApproximateNearestNeighborCondition::traverseOrderByAST(const ASTPtr & node, RPN & rpn) { if (const auto * expr_list = node->as()) if (const auto * order_by_element = expr_list->children.front()->as()) traverseAST(order_by_element->children.front(), rpn); } -// Returns true and stores ANNQueryInformation if the query has valid WHERE clause -bool ANNCondition::matchRPNWhere(RPN & rpn, ANNQueryInformation & expr) +// Returns true and stores ApproximateNearestNeighborInformation if the query has valid WHERE clause +bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & expr) { /// Fill query type field - expr.query_type = ANNQueryInformation::Type::Where; + expr.query_type = ApproximateNearestNeighborInformation::Type::Where; // WHERE section must have at least 5 expressions // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(TargetVector(floats)) @@ -381,10 +376,10 @@ bool ANNCondition::matchRPNWhere(RPN & rpn, ANNQueryInformation & expr) } // Returns true and stores ANNExpr if the query has valid ORDERBY clause -bool ANNCondition::matchRPNOrderBy(RPN & rpn, ANNQueryInformation & expr) +bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & expr) { /// Fill query type field - expr.query_type = ANNQueryInformation::Type::OrderBy; + expr.query_type = ApproximateNearestNeighborInformation::Type::OrderBy; // ORDER BY clause must have at least 3 expressions if (rpn.size() < 3) @@ -393,11 +388,11 @@ bool ANNCondition::matchRPNOrderBy(RPN & rpn, ANNQueryInformation & expr) auto iter = rpn.begin(); auto end = rpn.end(); - return ANNCondition::matchMainParts(iter, end, expr); + return ApproximateNearestNeighborCondition::matchMainParts(iter, end, expr); } // Returns true and stores Length if we have valid LIMIT clause in query -bool ANNCondition::matchRPNLimit(RPNElement & rpn, UInt64 & limit) +bool ApproximateNearestNeighborCondition::matchRPNLimit(RPNElement & rpn, UInt64 & limit) { if (rpn.function == RPNElement::FUNCTION_INT_LITERAL) { @@ -409,7 +404,7 @@ bool ANNCondition::matchRPNLimit(RPNElement & rpn, UInt64 & limit) } /* Matches dist function, target vector, column name */ -bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ANNQueryInformation & expr) +bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & expr) { bool identifier_found = false; @@ -420,7 +415,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en expr.metric = castMetricFromStringToType(iter->func_name); ++iter; - if (expr.metric == ANN::ANNQueryInformation::Metric::Lp) + if (expr.metric == ApproximateNearestNeighborInformation::Metric::Lp) { if (iter->function != RPNElement::FUNCTION_FLOAT_LITERAL && iter->function != RPNElement::FUNCTION_INT_LITERAL) @@ -497,7 +492,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en } // Gets float or int from AST node -float ANNCondition::getFloatOrIntLiteralOrPanic(const RPN::iterator& iter) +float ApproximateNearestNeighborCondition::getFloatOrIntLiteralOrPanic(const RPN::iterator& iter) { if (iter->float_literal.has_value()) return iter->float_literal.value(); @@ -507,5 +502,3 @@ float ANNCondition::getFloatOrIntLiteralOrPanic(const RPN::iterator& iter) } } - -} diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 4253bce703a..37695586515 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -9,9 +9,6 @@ namespace DB { -namespace ApproximateNearestNeighbour -{ - /** * Queries for Approximate Nearest Neighbour Search * have similar structure: @@ -25,7 +22,7 @@ namespace ApproximateNearestNeighbour * 1) p for LpDistance function * 2) distance to compare with (only for where queries) */ -struct ANNQueryInformation +struct ApproximateNearestNeighborInformation { using Embedding = std::vector; @@ -51,7 +48,7 @@ struct ANNQueryInformation }; /** - Class ANNCondition, is responsible for recognizing special query types which + Class ApproximateNearestNeighborCondition, is responsible for recognizing special query types which can be speeded up by ANN Indexes. It parses the SQL query and checks if it matches ANNIndexes. The recognizing method - alwaysUnknownOrTrue returns false if we can speed up the query, and true otherwise. @@ -87,10 +84,10 @@ struct ANNQueryInformation Search query type is also recognized for PREWHERE clause */ -class ANNCondition +class ApproximateNearestNeighborCondition { public: - ANNCondition(const SelectQueryInfo & query_info, + ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context); // false if query can be speeded up, true otherwise @@ -107,12 +104,12 @@ public: String getColumnName() const; - ANNQueryInformation::Metric getMetricType() const; + ApproximateNearestNeighborInformation::Metric getMetricType() const; // the P- value if the metric is 'LpDistance' float getPValueForLpDistance() const; - ANNQueryInformation::Type getQueryType() const; + ApproximateNearestNeighborInformation::Type getQueryType() const; UInt64 getIndexGranularity() const { return index_granularity; } @@ -191,16 +188,16 @@ private: void traverseOrderByAST(const ASTPtr & node, RPN & rpn); // Returns true and stores ANNExpr if the query has valid WHERE section - static bool matchRPNWhere(RPN & rpn, ANNQueryInformation & expr); + static bool matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & expr); // Returns true and stores ANNExpr if the query has valid ORDERBY section - static bool matchRPNOrderBy(RPN & rpn, ANNQueryInformation & expr); + static bool matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & expr); // Returns true and stores Length if we have valid LIMIT clause in query static bool matchRPNLimit(RPNElement & rpn, UInt64 & limit); /* Matches dist function, target vector, column name */ - static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ANNQueryInformation & expr); + static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & expr); // Gets float or int from AST node static float getFloatOrIntLiteralOrPanic(const RPN::iterator& iter); @@ -208,7 +205,7 @@ private: Block block_with_constants; // true if we have one of two supported query types - std::optional query_information; + std::optional query_information; // Get from settings ANNIndex parameters UInt64 index_granularity; @@ -218,12 +215,10 @@ private: }; // condition interface for Ann indexes. Returns vector of indexes of ranges in granule which are useful for query. -class IMergeTreeIndexConditionAnn : public IMergeTreeIndexCondition +class IMergeTreeIndexConditionApproximateNearestNeighbor : public IMergeTreeIndexCondition { public: virtual std::vector getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const = 0; }; } - -} diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index aa340d6afc1..3d290ea12ac 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1687,7 +1687,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( if (index_mark != index_range.begin || !granule || last_index_mark != index_range.begin) granule = reader.read(); // Cast to Ann condition - auto ann_condition = std::dynamic_pointer_cast(condition); + auto ann_condition = std::dynamic_pointer_cast(condition); if (ann_condition != nullptr) { // vector of indexes of useful ranges diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index fe3ea322b91..190f76fba5e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -240,7 +240,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI { UInt64 limit = condition.getLimit(); UInt64 index_granularity = condition.getIndexGranularity(); - std::optional comp_dist = condition.getQueryType() == ApproximateNearestNeighbour::ANNQueryInformation::Type::Where + std::optional comp_dist = condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where ? std::optional(condition.getComparisonDistanceForWhereQuery()) : std::nullopt; @@ -267,6 +267,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI distances.reserve(limit); annoy->get_nns_by_vector(target_vec.data(), limit, static_cast(search_k), &neighbors, &distances); + std::unordered_set granule_numbers; for (size_t i = 0; i < neighbors.size(); ++i) { diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index fbc6b21fa6b..95041ea31fb 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -60,7 +60,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator }; -class MergeTreeIndexConditionAnnoy final : public ApproximateNearestNeighbour::IMergeTreeIndexConditionAnn +class MergeTreeIndexConditionAnnoy final : public IMergeTreeIndexConditionApproximateNearestNeighbor { public: MergeTreeIndexConditionAnnoy( @@ -81,7 +81,7 @@ private: template std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; - const ApproximateNearestNeighbour::ANNCondition condition; + const ApproximateNearestNeighborCondition condition; const String distance_function; const Int64 search_k; }; From 999e4c33065279f4337387ece0343da36ca03098 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:37:20 +0000 Subject: [PATCH 1152/2223] Cosmetics: Less generic variable naming --- src/Storages/MergeTree/CommonANNIndexes.cpp | 52 ++++++++++----------- src/Storages/MergeTree/CommonANNIndexes.h | 6 +-- 2 files changed, 29 insertions(+), 29 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 5c42774fb24..669055068b8 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -321,10 +321,10 @@ void ApproximateNearestNeighborCondition::traverseOrderByAST(const ASTPtr & node } // Returns true and stores ApproximateNearestNeighborInformation if the query has valid WHERE clause -bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & expr) +bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field - expr.query_type = ApproximateNearestNeighborInformation::Type::Where; + ann_info.query_type = ApproximateNearestNeighborInformation::Type::Where; // WHERE section must have at least 5 expressions // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(TargetVector(floats)) @@ -347,9 +347,9 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe if (iter->function != RPNElement::FUNCTION_FLOAT_LITERAL) return false; - expr.distance = getFloatOrIntLiteralOrPanic(iter); - if (expr.distance < 0) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", expr.distance); + ann_info.distance = getFloatOrIntLiteralOrPanic(iter); + if (ann_info.distance < 0) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", ann_info.distance); ++iter; @@ -358,17 +358,17 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe return false; auto end = rpn.end(); - if (!matchMainParts(iter, end, expr)) + if (!matchMainParts(iter, end, ann_info)) return false; if (greater_case) { - if (expr.target.size() < 2) + if (ann_info.target.size() < 2) return false; - expr.distance = expr.target.back(); - if (expr.distance < 0) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", expr.distance); - expr.target.pop_back(); + ann_info.distance = ann_info.target.back(); + if (ann_info.distance < 0) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", ann_info.distance); + ann_info.target.pop_back(); } // query is ok @@ -376,10 +376,10 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe } // Returns true and stores ANNExpr if the query has valid ORDERBY clause -bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & expr) +bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field - expr.query_type = ApproximateNearestNeighborInformation::Type::OrderBy; + ann_info.query_type = ApproximateNearestNeighborInformation::Type::OrderBy; // ORDER BY clause must have at least 3 expressions if (rpn.size() < 3) @@ -388,7 +388,7 @@ bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, Approximate auto iter = rpn.begin(); auto end = rpn.end(); - return ApproximateNearestNeighborCondition::matchMainParts(iter, end, expr); + return ApproximateNearestNeighborCondition::matchMainParts(iter, end, ann_info); } // Returns true and stores Length if we have valid LIMIT clause in query @@ -404,7 +404,7 @@ bool ApproximateNearestNeighborCondition::matchRPNLimit(RPNElement & rpn, UInt64 } /* Matches dist function, target vector, column name */ -bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & expr) +bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info) { bool identifier_found = false; @@ -412,22 +412,22 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c if (iter->function != RPNElement::FUNCTION_DISTANCE) return false; - expr.metric = castMetricFromStringToType(iter->func_name); + ann_info.metric = castMetricFromStringToType(iter->func_name); ++iter; - if (expr.metric == ApproximateNearestNeighborInformation::Metric::Lp) + if (ann_info.metric == ApproximateNearestNeighborInformation::Metric::Lp) { if (iter->function != RPNElement::FUNCTION_FLOAT_LITERAL && iter->function != RPNElement::FUNCTION_INT_LITERAL) return false; - expr.p_for_lp_dist = getFloatOrIntLiteralOrPanic(iter); + ann_info.p_for_lp_dist = getFloatOrIntLiteralOrPanic(iter); ++iter; } if (iter->function == RPNElement::FUNCTION_IDENTIFIER) { identifier_found = true; - expr.column_name = std::move(iter->identifier.value()); + ann_info.column_name = std::move(iter->identifier.value()); ++iter; } @@ -436,13 +436,13 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extractTargetVectorFromLiteral(expr.target, iter->tuple_literal); + extractTargetVectorFromLiteral(ann_info.target, iter->tuple_literal); ++iter; } if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extractTargetVectorFromLiteral(expr.target, iter->array_literal); + extractTargetVectorFromLiteral(ann_info.target, iter->array_literal); ++iter; } @@ -457,12 +457,12 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c ++iter; if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extractTargetVectorFromLiteral(expr.target, iter->tuple_literal); + extractTargetVectorFromLiteral(ann_info.target, iter->tuple_literal); ++iter; } else if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extractTargetVectorFromLiteral(expr.target, iter->array_literal); + extractTargetVectorFromLiteral(ann_info.target, iter->array_literal); ++iter; } else @@ -473,12 +473,12 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c { if (iter->function == RPNElement::FUNCTION_FLOAT_LITERAL || iter->function == RPNElement::FUNCTION_INT_LITERAL) - expr.target.emplace_back(getFloatOrIntLiteralOrPanic(iter)); + ann_info.target.emplace_back(getFloatOrIntLiteralOrPanic(iter)); else if (iter->function == RPNElement::FUNCTION_IDENTIFIER) { if (identifier_found) return false; - expr.column_name = std::move(iter->identifier.value()); + ann_info.column_name = std::move(iter->identifier.value()); identifier_found = true; } else @@ -488,7 +488,7 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c } // Final checks of correctness - return identifier_found && !expr.target.empty(); + return identifier_found && !ann_info.target.empty(); } // Gets float or int from AST node diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 37695586515..0b207585048 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -188,16 +188,16 @@ private: void traverseOrderByAST(const ASTPtr & node, RPN & rpn); // Returns true and stores ANNExpr if the query has valid WHERE section - static bool matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & expr); + static bool matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & ann_info); // Returns true and stores ANNExpr if the query has valid ORDERBY section - static bool matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & expr); + static bool matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & ann_info); // Returns true and stores Length if we have valid LIMIT clause in query static bool matchRPNLimit(RPNElement & rpn, UInt64 & limit); /* Matches dist function, target vector, column name */ - static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & expr); + static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info); // Gets float or int from AST node static float getFloatOrIntLiteralOrPanic(const RPN::iterator& iter); From 567d54a26848ba0f5bf4ad38d226def02a92280b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:38:00 +0000 Subject: [PATCH 1153/2223] Cosmetics: more constness --- src/Storages/MergeTree/CommonANNIndexes.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 0b207585048..68ed217ef45 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -208,9 +208,9 @@ private: std::optional query_information; // Get from settings ANNIndex parameters - UInt64 index_granularity; + const UInt64 index_granularity; /// only queries with a lower limit can be considered to avoid memory overflow - UInt64 limit_restriction; + const UInt64 limit_restriction; bool index_is_useful = false; }; From 6fe208832d6c3ba8340fd73d5728efad567188ba Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:42:46 +0000 Subject: [PATCH 1154/2223] Cosmetics: target vector --> reference vector --- src/Storages/MergeTree/CommonANNIndexes.cpp | 48 +++++++++---------- src/Storages/MergeTree/CommonANNIndexes.h | 22 ++++----- .../MergeTree/MergeTreeIndexAnnoy.cpp | 6 +-- 3 files changed, 38 insertions(+), 38 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 669055068b8..20707a148ae 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -25,19 +25,19 @@ namespace { template -void extractTargetVectorFromLiteral(ApproximateNearestNeighborInformation::Embedding & target, Literal literal) +void extraceReferenceVectorFromLiteral(ApproximateNearestNeighborInformation::Embedding & reference_vector, Literal literal) { - Float64 float_element_of_target_vector; - Int64 int_element_of_target_vector; + Float64 float_element_of_reference_vector; + Int64 int_element_of_reference_vector; for (const auto & value : literal.value()) { - if (value.tryGet(float_element_of_target_vector)) - target.emplace_back(float_element_of_target_vector); - else if (value.tryGet(int_element_of_target_vector)) - target.emplace_back(static_cast(int_element_of_target_vector)); + if (value.tryGet(float_element_of_reference_vector)) + reference_vector.emplace_back(float_element_of_reference_vector); + else if (value.tryGet(int_element_of_reference_vector)) + reference_vector.emplace_back(static_cast(int_element_of_reference_vector)); else - throw Exception(ErrorCodes::INCORRECT_QUERY, "Wrong type of elements in target vector. Only float or int are supported."); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Wrong type of elements in reference vector. Only float or int are supported."); } } @@ -82,17 +82,17 @@ UInt64 ApproximateNearestNeighborCondition::getLimit() const throw Exception(ErrorCodes::LOGICAL_ERROR, "No LIMIT section in query, not supported"); } -std::vector ApproximateNearestNeighborCondition::getTargetVector() const +std::vector ApproximateNearestNeighborCondition::getReferenceVector() const { if (index_is_useful && query_information.has_value()) - return query_information->target; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Target vector was requested for useless or uninitialized index."); + return query_information->reference_vector; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Reference vector was requested for useless or uninitialized index."); } size_t ApproximateNearestNeighborCondition::getNumOfDimensions() const { if (index_is_useful && query_information.has_value()) - return query_information->target.size(); + return query_information->reference_vector.size(); throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of dimensions was requested for useless or uninitialized index."); } @@ -327,7 +327,7 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe ann_info.query_type = ApproximateNearestNeighborInformation::Type::Where; // WHERE section must have at least 5 expressions - // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(TargetVector(floats)) + // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(ReferenceVector(floats)) if (rpn.size() < 5) return false; @@ -363,12 +363,12 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe if (greater_case) { - if (ann_info.target.size() < 2) + if (ann_info.reference_vector.size() < 2) return false; - ann_info.distance = ann_info.target.back(); + ann_info.distance = ann_info.reference_vector.back(); if (ann_info.distance < 0) throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", ann_info.distance); - ann_info.target.pop_back(); + ann_info.reference_vector.pop_back(); } // query is ok @@ -403,12 +403,12 @@ bool ApproximateNearestNeighborCondition::matchRPNLimit(RPNElement & rpn, UInt64 return false; } -/* Matches dist function, target vector, column name */ +/* Matches dist function, referencer vector, column name */ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info) { bool identifier_found = false; - // Matches DistanceFunc->[Column]->[Tuple(array)Func]->TargetVector(floats)->[Column] + // Matches DistanceFunc->[Column]->[Tuple(array)Func]->ReferenceVector(floats)->[Column] if (iter->function != RPNElement::FUNCTION_DISTANCE) return false; @@ -436,13 +436,13 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extractTargetVectorFromLiteral(ann_info.target, iter->tuple_literal); + extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); ++iter; } if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extractTargetVectorFromLiteral(ann_info.target, iter->array_literal); + extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); ++iter; } @@ -457,12 +457,12 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c ++iter; if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extractTargetVectorFromLiteral(ann_info.target, iter->tuple_literal); + extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); ++iter; } else if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extractTargetVectorFromLiteral(ann_info.target, iter->array_literal); + extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); ++iter; } else @@ -473,7 +473,7 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c { if (iter->function == RPNElement::FUNCTION_FLOAT_LITERAL || iter->function == RPNElement::FUNCTION_INT_LITERAL) - ann_info.target.emplace_back(getFloatOrIntLiteralOrPanic(iter)); + ann_info.reference_vector.emplace_back(getFloatOrIntLiteralOrPanic(iter)); else if (iter->function == RPNElement::FUNCTION_IDENTIFIER) { if (identifier_found) @@ -488,7 +488,7 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c } // Final checks of correctness - return identifier_found && !ann_info.target.empty(); + return identifier_found && !ann_info.reference_vector.empty(); } // Gets float or int from AST node diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 68ed217ef45..9c075139707 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -12,7 +12,7 @@ namespace DB /** * Queries for Approximate Nearest Neighbour Search * have similar structure: - * 1) target vector from which all distances are calculated + * 1) reference vector from which all distances are calculated * 2) metric name (e.g L2Distance, LpDistance, etc.) * 3) name of column with embeddings * 4) type of query @@ -27,7 +27,7 @@ struct ApproximateNearestNeighborInformation using Embedding = std::vector; // Extracted data from valid query - Embedding target; + Embedding reference_vector; enum class Metric { Unknown, @@ -56,14 +56,14 @@ struct ApproximateNearestNeighborInformation There are two main patterns of queries being supported 1) Search query type - SELECT * FROM * WHERE DistanceFunc(column, target_vector) < floatLiteral LIMIT count + SELECT * FROM * WHERE DistanceFunc(column, reference) < floatLiteral LIMIT count 2) OrderBy query type - SELECT * FROM * WHERE * ORDERBY DistanceFunc(column, target_vector) LIMIT count + SELECT * FROM * WHERE * ORDERBY DistanceFunc(column, reference) LIMIT count *Query without LIMIT count is not supported* - target_vector(should have float coordinates) examples: + reference(should have float coordinates) examples: tuple(0.1, 0.1, ...., 0.1) or (0.1, 0.1, ...., 0.1) [the word tuple is not needed] @@ -72,11 +72,11 @@ struct ApproximateNearestNeighborInformation returns true. From matching query it extracts - * targetVector + * referenceVector * metricName(DistanceFunction) * dimension size if query uses LpDistance * distance to compare(ONLY for search types, otherwise you get exception) - * spaceDimension(which is targetVector's components count) + * spaceDimension(which is reference vector's components count) * column * objects count from LIMIT clause(for both queries) * queryHasOrderByClause and queryHasWhereClause return true if query matches the type @@ -96,10 +96,10 @@ public: // returns the distance to compare with for search query float getComparisonDistanceForWhereQuery() const; - // distance should be calculated regarding to targetVector - std::vector getTargetVector() const; + // distance should be calculated regarding to reference vector + std::vector getReferenceVector() const; - // targetVector dimension size + // reference vector's dimension size size_t getNumOfDimensions() const; String getColumnName() const; @@ -196,7 +196,7 @@ private: // Returns true and stores Length if we have valid LIMIT clause in query static bool matchRPNLimit(RPNElement & rpn, UInt64 & limit); - /* Matches dist function, target vector, column name */ + /* Matches dist function, reference vector, column name */ static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info); // Gets float or int from AST node diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 190f76fba5e..9dcfd421ba5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -247,7 +247,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI if (comp_dist && comp_dist.value() < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); - std::vector target_vec = condition.getTargetVector(); + std::vector reference_vector = condition.getReferenceVector(); auto granule = std::dynamic_pointer_cast>(idx_granule); if (granule == nullptr) @@ -260,13 +260,13 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI "does not match with the dimension in the index ({})", toString(condition.getNumOfDimensions()), toString(annoy->getNumOfDimensions())); - /// neighbors contain indexes of dots which were closest to target vector + /// neighbors contain indexes of dots which were closest to the reference vector std::vector neighbors; std::vector distances; neighbors.reserve(limit); distances.reserve(limit); - annoy->get_nns_by_vector(target_vec.data(), limit, static_cast(search_k), &neighbors, &distances); + annoy->get_nns_by_vector(reference_vector.data(), limit, static_cast(search_k), &neighbors, &distances); std::unordered_set granule_numbers; for (size_t i = 0; i < neighbors.size(); ++i) From 8cc382121a3b7580136eaf7514957dd0e08f3283 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:48:36 +0000 Subject: [PATCH 1155/2223] Cosmetics: Update comments --- src/Storages/MergeTree/CommonANNIndexes.cpp | 57 ++++---- src/Storages/MergeTree/CommonANNIndexes.h | 152 ++++++++++---------- 2 files changed, 103 insertions(+), 106 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 20707a148ae..f4a0e9bf728 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -126,7 +126,7 @@ ApproximateNearestNeighborInformation::Type ApproximateNearestNeighborCondition: bool ApproximateNearestNeighborCondition::checkQueryStructure(const SelectQueryInfo & query) { - // RPN-s for different sections of the query + /// RPN-s for different sections of the query RPN rpn_prewhere_clause; RPN rpn_where_clause; RPN rpn_order_by_clause; @@ -137,47 +137,50 @@ bool ApproximateNearestNeighborCondition::checkQueryStructure(const SelectQueryI ApproximateNearestNeighborInformation where_info; ApproximateNearestNeighborInformation order_by_info; - // Build rpns for query sections + /// Build rpns for query sections const auto & select = query.query->as(); - if (select.prewhere()) // If query has PREWHERE clause + /// If query has PREWHERE clause + if (select.prewhere()) traverseAST(select.prewhere(), rpn_prewhere_clause); - if (select.where()) // If query has WHERE clause + /// If query has WHERE clause + if (select.where()) traverseAST(select.where(), rpn_where_clause); - if (select.limitLength()) // If query has LIMIT clause + /// If query has LIMIT clause + if (select.limitLength()) traverseAtomAST(select.limitLength(), rpn_limit); if (select.orderBy()) // If query has ORDERBY clause traverseOrderByAST(select.orderBy(), rpn_order_by_clause); - // Reverse RPNs for conveniences during parsing + /// Reverse RPNs for conveniences during parsing std::reverse(rpn_prewhere_clause.begin(), rpn_prewhere_clause.end()); std::reverse(rpn_where_clause.begin(), rpn_where_clause.end()); std::reverse(rpn_order_by_clause.begin(), rpn_order_by_clause.end()); - // Match rpns with supported types and extract information + /// Match rpns with supported types and extract information const bool prewhere_is_valid = matchRPNWhere(rpn_prewhere_clause, prewhere_info); const bool where_is_valid = matchRPNWhere(rpn_where_clause, where_info); const bool order_by_is_valid = matchRPNOrderBy(rpn_order_by_clause, order_by_info); const bool limit_is_valid = matchRPNLimit(rpn_limit, limit); - // Query without a LIMIT clause or with a limit greater than a restriction is not supported + /// Query without a LIMIT clause or with a limit greater than a restriction is not supported if (!limit_is_valid || limit_restriction < limit) return false; - // Search type query in both sections isn't supported + /// Search type query in both sections isn't supported if (prewhere_is_valid && where_is_valid) return false; - // Search type should be in WHERE or PREWHERE clause + /// Search type should be in WHERE or PREWHERE clause if (prewhere_is_valid || where_is_valid) query_information = std::move(prewhere_is_valid ? prewhere_info : where_info); if (order_by_is_valid) { - // Query with valid where and order by type is not supported + /// Query with valid where and order by type is not supported if (query_information.has_value()) return false; @@ -202,7 +205,7 @@ void ApproximateNearestNeighborCondition::traverseAST(const ASTPtr & node, RPN & } RPNElement element; - // Get the data behind node + /// Get the data behind node if (!traverseAtomAST(node, element)) element.function = RPNElement::FUNCTION_UNKNOWN; @@ -211,10 +214,10 @@ void ApproximateNearestNeighborCondition::traverseAST(const ASTPtr & node, RPN & bool ApproximateNearestNeighborCondition::traverseAtomAST(const ASTPtr & node, RPNElement & out) { - // Match Functions + /// Match Functions if (const auto * function = node->as()) { - // Set the name + /// Set the name out.func_name = function->name; if (function->name == "L1Distance" || @@ -240,7 +243,7 @@ bool ApproximateNearestNeighborCondition::traverseAtomAST(const ASTPtr & node, R return true; } - // Match identifier + /// Match identifier else if (const auto * identifier = node->as()) { out.function = RPNElement::FUNCTION_IDENTIFIER; @@ -250,7 +253,7 @@ bool ApproximateNearestNeighborCondition::traverseAtomAST(const ASTPtr & node, R return true; } - // Check if we have constants behind the node + /// Check if we have constants behind the node return tryCastToConstType(node, out); } @@ -320,20 +323,20 @@ void ApproximateNearestNeighborCondition::traverseOrderByAST(const ASTPtr & node traverseAST(order_by_element->children.front(), rpn); } -// Returns true and stores ApproximateNearestNeighborInformation if the query has valid WHERE clause +/// Returns true and stores ApproximateNearestNeighborInformation if the query has valid WHERE clause bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field ann_info.query_type = ApproximateNearestNeighborInformation::Type::Where; - // WHERE section must have at least 5 expressions - // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(ReferenceVector(floats)) + /// WHERE section must have at least 5 expressions + /// Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(ReferenceVector(floats)) if (rpn.size() < 5) return false; auto iter = rpn.begin(); - // Query starts from operator less + /// Query starts from operator less if (iter->function != RPNElement::FUNCTION_COMPARISON) return false; @@ -371,11 +374,11 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe ann_info.reference_vector.pop_back(); } - // query is ok + /// query is ok return true; } -// Returns true and stores ANNExpr if the query has valid ORDERBY clause +/// Returns true and stores ANNExpr if the query has valid ORDERBY clause bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field @@ -391,7 +394,7 @@ bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, Approximate return ApproximateNearestNeighborCondition::matchMainParts(iter, end, ann_info); } -// Returns true and stores Length if we have valid LIMIT clause in query +/// Returns true and stores Length if we have valid LIMIT clause in query bool ApproximateNearestNeighborCondition::matchRPNLimit(RPNElement & rpn, UInt64 & limit) { if (rpn.function == RPNElement::FUNCTION_INT_LITERAL) @@ -403,12 +406,12 @@ bool ApproximateNearestNeighborCondition::matchRPNLimit(RPNElement & rpn, UInt64 return false; } -/* Matches dist function, referencer vector, column name */ +/// Matches dist function, referencer vector, column name bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info) { bool identifier_found = false; - // Matches DistanceFunc->[Column]->[Tuple(array)Func]->ReferenceVector(floats)->[Column] + /// Matches DistanceFunc->[Column]->[Tuple(array)Func]->ReferenceVector(floats)->[Column] if (iter->function != RPNElement::FUNCTION_DISTANCE) return false; @@ -487,11 +490,11 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c ++iter; } - // Final checks of correctness + /// Final checks of correctness return identifier_found && !ann_info.reference_vector.empty(); } -// Gets float or int from AST node +/// Gets float or int from AST node float ApproximateNearestNeighborCondition::getFloatOrIntLiteralOrPanic(const RPN::iterator& iter) { if (iter->float_literal.has_value()) diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 9c075139707..2ac9384a884 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -9,31 +9,31 @@ namespace DB { -/** - * Queries for Approximate Nearest Neighbour Search - * have similar structure: - * 1) reference vector from which all distances are calculated - * 2) metric name (e.g L2Distance, LpDistance, etc.) - * 3) name of column with embeddings - * 4) type of query - * 5) Number of elements, that should be taken (limit) - * - * And two optional parameters: - * 1) p for LpDistance function - * 2) distance to compare with (only for where queries) - */ +/// Approximate Nearest Neighbour queries have a similar structure: +/// - reference vector from which all distances are calculated +/// - metric name (e.g L2Distance, LpDistance, etc.) +/// - name of column with embeddings +/// - type of query +/// - maximum number of returned elements (LIMIT) +/// +/// And two optional parameters: +/// - p for LpDistance function +/// - distance to compare with (only for where queries) +/// +/// This struct holds all these components. struct ApproximateNearestNeighborInformation { using Embedding = std::vector; - - // Extracted data from valid query Embedding reference_vector; + enum class Metric { Unknown, L2, Lp - } metric; + }; + Metric metric; + String column_name; UInt64 limit; @@ -41,79 +41,71 @@ struct ApproximateNearestNeighborInformation { OrderBy, Where - } query_type; + }; + Type query_type; float p_for_lp_dist = -1.0; float distance = -1.0; }; -/** - Class ApproximateNearestNeighborCondition, is responsible for recognizing special query types which - can be speeded up by ANN Indexes. It parses the SQL query and checks - if it matches ANNIndexes. The recognizing method - alwaysUnknownOrTrue - returns false if we can speed up the query, and true otherwise. - It has only one argument, name of the metric with which index was built. - There are two main patterns of queries being supported - - 1) Search query type - SELECT * FROM * WHERE DistanceFunc(column, reference) < floatLiteral LIMIT count - - 2) OrderBy query type - SELECT * FROM * WHERE * ORDERBY DistanceFunc(column, reference) LIMIT count - - *Query without LIMIT count is not supported* - - reference(should have float coordinates) examples: - tuple(0.1, 0.1, ...., 0.1) or (0.1, 0.1, ...., 0.1) - [the word tuple is not needed] - - If the query matches one of these two types, than the class extracts useful information - from the query. If the query has both 1 and 2 types, than we can't speed and alwaysUnknownOrTrue - returns true. - - From matching query it extracts - * referenceVector - * metricName(DistanceFunction) - * dimension size if query uses LpDistance - * distance to compare(ONLY for search types, otherwise you get exception) - * spaceDimension(which is reference vector's components count) - * column - * objects count from LIMIT clause(for both queries) - * queryHasOrderByClause and queryHasWhereClause return true if query matches the type - - Search query type is also recognized for PREWHERE clause -*/ +// Class ANNCondition, is responsible for recognizing if the query is an ANN queries which can utilize ANN indexes. It parses the SQL query +/// and checks if it matches ANNIndexes. Method alwaysUnknownOrTrue returns false if we can speed up the query, and true otherwise. It has +/// only one argument, the name of the metric with which index was built. Two main patterns of queries are supported +/// +/// - 1. WHERE queries: +/// SELECT * FROM * WHERE DistanceFunc(column, reference_vector) < floatLiteral LIMIT count +/// +/// - 2. ORDER BY queries: +/// SELECT * FROM * WHERE * ORDER BY DistanceFunc(column, reference_vector) LIMIT count +/// +/// Queries without LIMIT count are not supported +/// If the query is both of type 1. and 2., than we can't use the index and alwaysUnknownOrTrue returns true. +/// reference_vector should have float coordinates, e.g. (0.2, 0.1, .., 0.5) +/// +/// If the query matches one of these two types, then this class extracts the main information needed for ANN indexes from the query. +/// +/// From matching query it extracts +/// - referenceVector +/// - metricName(DistanceFunction) +/// - dimension size if query uses LpDistance +/// - distance to compare(ONLY for search types, otherwise you get exception) +/// - spaceDimension(which is referenceVector's components count) +/// - column +/// - objects count from LIMIT clause(for both queries) +/// - queryHasOrderByClause and queryHasWhereClause return true if query matches the type +/// +/// Search query type is also recognized for PREWHERE clause class ApproximateNearestNeighborCondition { public: ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context); - // false if query can be speeded up, true otherwise + /// Returns false if query can be speeded up by an ANN index, true otherwise. bool alwaysUnknownOrTrue(String metric_name) const; - // returns the distance to compare with for search query + /// Returns the distance to compare with for search query float getComparisonDistanceForWhereQuery() const; - // distance should be calculated regarding to reference vector + /// Distance should be calculated regarding to referenceVector std::vector getReferenceVector() const; - // reference vector's dimension size + /// Reference vector's dimension size size_t getNumOfDimensions() const; String getColumnName() const; ApproximateNearestNeighborInformation::Metric getMetricType() const; - // the P- value if the metric is 'LpDistance' + /// The P- value if the metric is 'LpDistance' float getPValueForLpDistance() const; ApproximateNearestNeighborInformation::Type getQueryType() const; UInt64 getIndexGranularity() const { return index_granularity; } - // length's value from LIMIT clause + /// Length's value from LIMIT clause UInt64 getLimit() const; private: @@ -121,7 +113,7 @@ private: { enum Function { - // DistanceFunctions + /// DistanceFunctions FUNCTION_DISTANCE, //tuple(0.1, ..., 0.1) @@ -130,31 +122,31 @@ private: //array(0.1, ..., 0.1) FUNCTION_ARRAY, - // Operators <, >, <=, >= + /// Operators <, >, <=, >= FUNCTION_COMPARISON, - // Numeric float value + /// Numeric float value FUNCTION_FLOAT_LITERAL, - // Numeric int value + /// Numeric int value FUNCTION_INT_LITERAL, - // Column identifier + /// Column identifier FUNCTION_IDENTIFIER, - // Unknown, can be any value + /// Unknown, can be any value FUNCTION_UNKNOWN, - // (0.1, ...., 0.1) vector without word 'tuple' + /// (0.1, ...., 0.1) vector without word 'tuple' FUNCTION_LITERAL_TUPLE, - // [0.1, ...., 0.1] vector without word 'array' + /// [0.1, ...., 0.1] vector without word 'array' FUNCTION_LITERAL_ARRAY, - // if client parameters are used, cast will always be in the query + /// if client parameters are used, cast will always be in the query FUNCTION_CAST, - // name of type in cast function + /// name of type in cast function FUNCTION_STRING_LITERAL, }; @@ -178,33 +170,33 @@ private: bool checkQueryStructure(const SelectQueryInfo & query); - // Util functions for the traversal of AST, parses AST and builds rpn + /// Util functions for the traversal of AST, parses AST and builds rpn void traverseAST(const ASTPtr & node, RPN & rpn); - // Return true if we can identify our node type + /// Return true if we can identify our node type bool traverseAtomAST(const ASTPtr & node, RPNElement & out); - // Checks if the AST stores ConstType expression + /// Checks if the AST stores ConstType expression bool tryCastToConstType(const ASTPtr & node, RPNElement & out); - // Traverses the AST of ORDERBY section + /// Traverses the AST of ORDERBY section void traverseOrderByAST(const ASTPtr & node, RPN & rpn); - // Returns true and stores ANNExpr if the query has valid WHERE section + /// Returns true and stores ANNExpr if the query has valid WHERE section static bool matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & ann_info); - // Returns true and stores ANNExpr if the query has valid ORDERBY section + /// Returns true and stores ANNExpr if the query has valid ORDERBY section static bool matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & ann_info); - // Returns true and stores Length if we have valid LIMIT clause in query + /// Returns true and stores Length if we have valid LIMIT clause in query static bool matchRPNLimit(RPNElement & rpn, UInt64 & limit); /* Matches dist function, reference vector, column name */ static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info); - // Gets float or int from AST node + /// Gets float or int from AST node static float getFloatOrIntLiteralOrPanic(const RPN::iterator& iter); Block block_with_constants; - // true if we have one of two supported query types + /// true if we have one of two supported query types std::optional query_information; // Get from settings ANNIndex parameters @@ -214,10 +206,12 @@ private: bool index_is_useful = false; }; -// condition interface for Ann indexes. Returns vector of indexes of ranges in granule which are useful for query. + +/// Common interface of ANN indexes. class IMergeTreeIndexConditionApproximateNearestNeighbor : public IMergeTreeIndexCondition { public: + /// Returns vector of indexes of ranges in granule which are useful for query. virtual std::vector getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const = 0; }; From 6580d2c326021cf476a274bce79b0b5d82bb92e4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:52:14 +0000 Subject: [PATCH 1156/2223] Cosmetics: castMetricFromStringToType --> stringToMetric --- src/Storages/MergeTree/CommonANNIndexes.cpp | 24 +-- src/Storages/MergeTree/CommonANNIndexes.h | 2 +- .../queries/0_stateless/02354_annoy.reference | 26 ---- tests/queries/0_stateless/02354_annoy.sh | 143 ------------------ 4 files changed, 14 insertions(+), 181 deletions(-) delete mode 100644 tests/queries/0_stateless/02354_annoy.reference delete mode 100755 tests/queries/0_stateless/02354_annoy.sh diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index f4a0e9bf728..2e2eb4e19ea 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -41,30 +41,32 @@ void extraceReferenceVectorFromLiteral(ApproximateNearestNeighborInformation::Em } } -ApproximateNearestNeighborInformation::Metric castMetricFromStringToType(String metric_name) +ApproximateNearestNeighborInformation::Metric stringToMetric(std::string_view metric) { - if (metric_name == "L2Distance") + if (metric == "L2Distance") return ApproximateNearestNeighborInformation::Metric::L2; - if (metric_name == "LpDistance") + else if (metric == "LpDistance") return ApproximateNearestNeighborInformation::Metric::Lp; - return ApproximateNearestNeighborInformation::Metric::Unknown; + else + return ApproximateNearestNeighborInformation::Metric::Unknown; } } ApproximateNearestNeighborCondition::ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context) : - block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, - index_granularity{context->getMergeTreeSettings().get("index_granularity").get()}, - limit_restriction{context->getSettings().get("max_limit_for_ann_queries").get()}, - index_is_useful{checkQueryStructure(query_info)} {} + block_with_constants(KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)), + index_granularity(context->getMergeTreeSettings().get("index_granularity").get()), + limit_restriction(context->getSettings().get("max_limit_for_ann_queries").get()), + index_is_useful(checkQueryStructure(query_info)) +{} -bool ApproximateNearestNeighborCondition::alwaysUnknownOrTrue(String metric_name) const +bool ApproximateNearestNeighborCondition::alwaysUnknownOrTrue(String metric) const { if (!index_is_useful) return true; // Query isn't supported // If query is supported, check metrics for match - return !(castMetricFromStringToType(metric_name) == query_information->metric); + return !(stringToMetric(metric) == query_information->metric); } float ApproximateNearestNeighborCondition::getComparisonDistanceForWhereQuery() const @@ -415,7 +417,7 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c if (iter->function != RPNElement::FUNCTION_DISTANCE) return false; - ann_info.metric = castMetricFromStringToType(iter->func_name); + ann_info.metric = stringToMetric(iter->func_name); ++iter; if (ann_info.metric == ApproximateNearestNeighborInformation::Metric::Lp) diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 2ac9384a884..6b094a40b26 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -83,7 +83,7 @@ public: ContextPtr context); /// Returns false if query can be speeded up by an ANN index, true otherwise. - bool alwaysUnknownOrTrue(String metric_name) const; + bool alwaysUnknownOrTrue(String metric) const; /// Returns the distance to compare with for search query float getComparisonDistanceForWhereQuery() const; diff --git a/tests/queries/0_stateless/02354_annoy.reference b/tests/queries/0_stateless/02354_annoy.reference deleted file mode 100644 index 38678fb67c9..00000000000 --- a/tests/queries/0_stateless/02354_annoy.reference +++ /dev/null @@ -1,26 +0,0 @@ -1 [0,0,10] -2 [0,0,10.5] -3 [0,0,9.5] -4 [0,0,9.7] -5 [0,0,10.2] -1 [0,0,10] -5 [0,0,10.2] -4 [0,0,9.7] -1 [0,0,10] -2 [0,0,10.5] -3 [0,0,9.5] -4 [0,0,9.7] -5 [0,0,10.2] -1 [0,0,10] -5 [0,0,10.2] -4 [0,0,9.7] - Name: annoy_index - Name: annoy_index -1 [0,0,10] -2 [0.2,0,10] -3 [-0.3,0,10] -1 [0,0,10] -2 [0.2,0,10] -3 [-0.3,0,10] - Name: annoy_index - Name: annoy_index diff --git a/tests/queries/0_stateless/02354_annoy.sh b/tests/queries/0_stateless/02354_annoy.sh deleted file mode 100755 index 1031ea81946..00000000000 --- a/tests/queries/0_stateless/02354_annoy.sh +++ /dev/null @@ -1,143 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# Check that index works correctly for L2Distance and with client parameters -$CLICKHOUSE_CLIENT -nm --allow_experimental_annoy_index=1 -q " -DROP TABLE IF EXISTS 02354_annoy_l2; - -CREATE TABLE 02354_annoy_l2 -( - id Int32, - embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy() GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; - -INSERT INTO 02354_annoy_l2 VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); - -SELECT * -FROM 02354_annoy_l2 -WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 5; - -SELECT * -FROM 02354_annoy_l2 -ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; - -SET param_02354_target_vector='[0.0, 0.0, 10.0]'; - -SELECT * -FROM 02354_annoy_l2 -WHERE L2Distance(embedding, {02354_target_vector: Array(Float32)}) < 1.0 -LIMIT 5; - -SELECT * -FROM 02354_annoy_l2 -ORDER BY L2Distance(embedding, {02354_target_vector: Array(Float32)}) -LIMIT 3; - -SELECT * -FROM 02354_annoy_l2 -ORDER BY L2Distance(embedding, [0.0, 0.0]) -LIMIT 3; -- { serverError 80 } - - -DROP TABLE IF EXISTS 02354_annoy_l2; -" - -# Check that indexes are used -$CLICKHOUSE_CLIENT -nm --allow_experimental_annoy_index=1 -q " -DROP TABLE IF EXISTS 02354_annoy_l2; - -CREATE TABLE 02354_annoy_l2 -( - id Int32, - embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy() GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; - -INSERT INTO 02354_annoy_l2 VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); - -EXPLAIN indexes=1 -SELECT * -FROM 02354_annoy_l2 -WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 5; - -EXPLAIN indexes=1 -SELECT * -FROM 02354_annoy_l2 -ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; -DROP TABLE IF EXISTS 02354_annoy_l2; -" | grep "annoy_index" - - -# # Check that index works correctly for cosineDistance -$CLICKHOUSE_CLIENT -nm --allow_experimental_annoy_index=1 -q " -DROP TABLE IF EXISTS 02354_annoy_cosine; - -CREATE TABLE 02354_annoy_cosine -( - id Int32, - embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy('cosineDistance', 100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; - -INSERT INTO 02354_annoy_cosine VALUES (1, [0.0, 0.0, 10.0]), (2, [0.2, 0.0, 10.0]), (3, [-0.3, 0.0, 10.0]), (4, [0.5, 0.0, 10.1]), (5, [0.8, 0.0, 10.0]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); - -SELECT * -FROM 02354_annoy_cosine -WHERE cosineDistance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 3; - -SELECT * -FROM 02354_annoy_cosine -ORDER BY cosineDistance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; - -DROP TABLE IF EXISTS 02354_annoy_cosine; -" - -# # Check that indexes are used -$CLICKHOUSE_CLIENT -nm --allow_experimental_annoy_index=1 -q " -DROP TABLE IF EXISTS 02354_annoy_cosine; - -CREATE TABLE 02354_annoy_cosine -( - id Int32, - embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy('cosineDistance', 100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; - -INSERT INTO 02354_annoy_cosine VALUES (1, [0.0, 0.0, 10.0]), (2, [0.2, 0.0, 10.0]), (3, [-0.3, 0.0, 10.0]), (4, [0.5, 0.0, 10.1]), (5, [0.8, 0.0, 10.0]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); - -EXPLAIN indexes=1 -SELECT * -FROM 02354_annoy_cosine -WHERE cosineDistance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 3; - -EXPLAIN indexes=1 -SELECT * -FROM 02354_annoy_cosine -ORDER BY cosineDistance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; -DROP TABLE IF EXISTS 02354_annoy_cosine; -" | grep "annoy_index" From ee5b49c3fd71fddae0ca1d0e2ee96f3dadfb82f1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:57:17 +0000 Subject: [PATCH 1157/2223] Consolidate Annoy index tests --- .../0_stateless/02354_annoy_index.reference | 41 +++++++++++++++++++ .../queries/0_stateless/02354_annoy_index.sql | 34 +++++++++++++++ 2 files changed, 75 insertions(+) diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 7da442cb905..5f3b523fbe4 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -1,3 +1,44 @@ +WHERE type, L2Distance +1 [0,0,10] +2 [0,0,10.5] +3 [0,0,9.5] +4 [0,0,9.7] +5 [0,0,10.2] +ORDER BY type, L2Distance +1 [0,0,10] +5 [0,0,10.2] +4 [0,0,9.7] +Reference ARRAYs with non-matching dimension are rejected +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 1/1 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 1/1 parameter annoy_index_search_k_nodes parameter max_limit_for_ann_queries Expression (Projection) diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 3590b7d316e..3e1c176f3f6 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -9,6 +9,40 @@ CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding -- SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); +SELECT 'WHERE type, L2Distance'; +SELECT * +FROM tab +WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 +LIMIT 5; + +SELECT 'ORDER BY type, L2Distance'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) +LIMIT 3; + + +SELECT 'Reference ARRAYs with non-matching dimension are rejected'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [0.0, 0.0]) +LIMIT 3; -- { serverError INCORRECT_QUERY } + + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 +LIMIT 5; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) +LIMIT 3; + SELECT 'parameter annoy_index_search_k_nodes'; SELECT * FROM tab From 55256f4664c6226174fc249a652336b8d14a5251 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 22:12:54 +0000 Subject: [PATCH 1158/2223] Cosmetics: Fix typo --- src/Storages/MergeTree/CommonANNIndexes.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 2e2eb4e19ea..45d86e54ea2 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -25,7 +25,7 @@ namespace { template -void extraceReferenceVectorFromLiteral(ApproximateNearestNeighborInformation::Embedding & reference_vector, Literal literal) +void extractReferenceVectorFromLiteral(ApproximateNearestNeighborInformation::Embedding & reference_vector, Literal literal) { Float64 float_element_of_reference_vector; Int64 int_element_of_reference_vector; @@ -441,13 +441,13 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); + extractReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); ++iter; } if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); + extractReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); ++iter; } @@ -462,12 +462,12 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c ++iter; if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); + extractReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); ++iter; } else if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); + extractReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); ++iter; } else From 879b70a594fdc42421f04ff1f757a518b67c6956 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 26 May 2023 08:50:36 +0000 Subject: [PATCH 1159/2223] CommonANNIndexes.h/cpp --> ApproximateNearestNeighborIndexesCommon.h/cpp --- ...es.cpp => ApproximateNearestNeighborIndexesCommon.cpp} | 8 +++----- ...ndexes.h => ApproximateNearestNeighborIndexesCommon.h} | 0 src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 2 +- 4 files changed, 5 insertions(+), 7 deletions(-) rename src/Storages/MergeTree/{CommonANNIndexes.cpp => ApproximateNearestNeighborIndexesCommon.cpp} (99%) rename src/Storages/MergeTree/{CommonANNIndexes.h => ApproximateNearestNeighborIndexesCommon.h} (100%) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp similarity index 99% rename from src/Storages/MergeTree/CommonANNIndexes.cpp rename to src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index 45d86e54ea2..4f0bcd7ff81 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -1,17 +1,15 @@ -#include -#include +#include +#include #include #include #include #include #include #include - +#include #include -#include - namespace DB { diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h similarity index 100% rename from src/Storages/MergeTree/CommonANNIndexes.h rename to src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 3d290ea12ac..fb11fabfac1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -45,7 +45,7 @@ #include -#include +#include namespace CurrentMetrics { diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 95041ea31fb..0ff676cf11f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -2,7 +2,7 @@ #ifdef ENABLE_ANNOY -#include +#include #include #include From 8213e366fae75c7079f814944d163e2e5d9b9b54 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 26 May 2023 08:57:35 +0000 Subject: [PATCH 1160/2223] Cosmetics: Remove absolute namespace qualification of Annoy library internals, pt. II --- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 0ff676cf11f..2c78ff536ae 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -11,9 +11,9 @@ namespace DB { template -class AnnoyIndexWithSerialization : public ::Annoy::AnnoyIndex +class AnnoyIndexWithSerialization : public Annoy::AnnoyIndex { - using Base = ::Annoy::AnnoyIndex; + using Base = Annoy::AnnoyIndex; public: explicit AnnoyIndexWithSerialization(uint64_t dim); From 0c26123fd5cf1b848a14eff86d0e5a97461d4c16 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 26 May 2023 09:39:06 +0000 Subject: [PATCH 1161/2223] Cosmetics: limit_restriction --> max_limit_for_ann_queries --- .../ApproximateNearestNeighborIndexesCommon.cpp | 13 ++++++------- .../ApproximateNearestNeighborIndexesCommon.h | 5 ++--- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index 4f0bcd7ff81..252035f3335 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -51,12 +51,11 @@ ApproximateNearestNeighborInformation::Metric stringToMetric(std::string_view me } -ApproximateNearestNeighborCondition::ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, - ContextPtr context) : - block_with_constants(KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)), - index_granularity(context->getMergeTreeSettings().get("index_granularity").get()), - limit_restriction(context->getSettings().get("max_limit_for_ann_queries").get()), - index_is_useful(checkQueryStructure(query_info)) +ApproximateNearestNeighborCondition::ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context) + : block_with_constants(KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)) + , index_granularity(context->getMergeTreeSettings().get("index_granularity").get()) + , max_limit_for_ann_queries(context->getSettings().get("max_limit_for_ann_queries").get()) + , index_is_useful(checkQueryStructure(query_info)) {} bool ApproximateNearestNeighborCondition::alwaysUnknownOrTrue(String metric) const @@ -167,7 +166,7 @@ bool ApproximateNearestNeighborCondition::checkQueryStructure(const SelectQueryI const bool limit_is_valid = matchRPNLimit(rpn_limit, limit); /// Query without a LIMIT clause or with a limit greater than a restriction is not supported - if (!limit_is_valid || limit_restriction < limit) + if (!limit_is_valid || max_limit_for_ann_queries < limit) return false; /// Search type query in both sections isn't supported diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h index 6b094a40b26..513a86c42d8 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h @@ -79,8 +79,7 @@ struct ApproximateNearestNeighborInformation class ApproximateNearestNeighborCondition { public: - ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, - ContextPtr context); + ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context); /// Returns false if query can be speeded up by an ANN index, true otherwise. bool alwaysUnknownOrTrue(String metric) const; @@ -202,7 +201,7 @@ private: // Get from settings ANNIndex parameters const UInt64 index_granularity; /// only queries with a lower limit can be considered to avoid memory overflow - const UInt64 limit_restriction; + const UInt64 max_limit_for_ann_queries; bool index_is_useful = false; }; From 146668a850234394d34c03c2d0246bbed7855549 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 May 2023 15:34:43 +0000 Subject: [PATCH 1162/2223] Cosmetics: query_type --> type --- .../ApproximateNearestNeighborIndexesCommon.cpp | 8 ++++---- .../ApproximateNearestNeighborIndexesCommon.h | 10 ++++++++-- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index 252035f3335..c47e53788a7 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -69,7 +69,7 @@ bool ApproximateNearestNeighborCondition::alwaysUnknownOrTrue(String metric) con float ApproximateNearestNeighborCondition::getComparisonDistanceForWhereQuery() const { if (index_is_useful && query_information.has_value() - && query_information->query_type == ApproximateNearestNeighborInformation::Type::Where) + && query_information->type == ApproximateNearestNeighborInformation::Type::Where) return query_information->distance; throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported method for this query type"); } @@ -119,7 +119,7 @@ float ApproximateNearestNeighborCondition::getPValueForLpDistance() const ApproximateNearestNeighborInformation::Type ApproximateNearestNeighborCondition::getQueryType() const { if (index_is_useful && query_information.has_value()) - return query_information->query_type; + return query_information->type; throw Exception(ErrorCodes::LOGICAL_ERROR, "Query type was requested for useless or uninitialized index."); } @@ -326,7 +326,7 @@ void ApproximateNearestNeighborCondition::traverseOrderByAST(const ASTPtr & node bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field - ann_info.query_type = ApproximateNearestNeighborInformation::Type::Where; + ann_info.type = ApproximateNearestNeighborInformation::Type::Where; /// WHERE section must have at least 5 expressions /// Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(ReferenceVector(floats)) @@ -381,7 +381,7 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field - ann_info.query_type = ApproximateNearestNeighborInformation::Type::OrderBy; + ann_info.type = ApproximateNearestNeighborInformation::Type::OrderBy; // ORDER BY clause must have at least 3 expressions if (rpn.size() < 3) diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h index 513a86c42d8..4fb95c3f492 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h @@ -42,7 +42,7 @@ struct ApproximateNearestNeighborInformation OrderBy, Where }; - Type query_type; + Type type; float p_for_lp_dist = -1.0; float distance = -1.0; @@ -150,7 +150,11 @@ private: }; explicit RPNElement(Function function_ = FUNCTION_UNKNOWN) - : function(function_), func_name("Unknown"), float_literal(std::nullopt), identifier(std::nullopt) {} + : function(function_) + , func_name("Unknown") + , float_literal(std::nullopt) + , identifier(std::nullopt) + {} Function function; String func_name; @@ -200,8 +204,10 @@ private: // Get from settings ANNIndex parameters const UInt64 index_granularity; + /// only queries with a lower limit can be considered to avoid memory overflow const UInt64 max_limit_for_ann_queries; + bool index_is_useful = false; }; From c94ec9f5ddb8d047c370749f3249374fb746ed06 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 May 2023 15:35:14 +0000 Subject: [PATCH 1163/2223] Cosmetics: Fix typo --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3e10f48a2fb..df4cf5ff087 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -741,7 +741,7 @@ class IColumn; M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions (hashid, etc)", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ - M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexs.", 0) \ + M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \ M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \ From 4e9a5331781166160feaf71ca8944a4f1b733846 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 09:53:20 +0000 Subject: [PATCH 1164/2223] Cosmetics: condition --> ann_condition --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 20 +++++++++---------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 2 +- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 9dcfd421ba5..9afaba6e521 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -208,12 +208,11 @@ MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( const SelectQueryInfo & query, const String & distance_function_, ContextPtr context) - : condition(query, context) + : ann_condition(query, context) , distance_function(distance_function_) , search_k(context->getSettings().get("annoy_index_search_k_nodes").get()) {} - bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr /*idx_granule*/) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "mayBeTrueOnGranule is not supported for ANN skip indexes"); @@ -221,7 +220,7 @@ bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr / bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const { - return condition.alwaysUnknownOrTrue(distance_function); + return ann_condition.alwaysUnknownOrTrue(distance_function); } std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const @@ -234,20 +233,19 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndex throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); } - template std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const { - UInt64 limit = condition.getLimit(); - UInt64 index_granularity = condition.getIndexGranularity(); - std::optional comp_dist = condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where - ? std::optional(condition.getComparisonDistanceForWhereQuery()) + UInt64 limit = ann_condition.getLimit(); + UInt64 index_granularity = ann_condition.getIndexGranularity(); + std::optional comp_dist = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where + ? std::optional(ann_condition.getComparisonDistanceForWhereQuery()) : std::nullopt; if (comp_dist && comp_dist.value() < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); - std::vector reference_vector = condition.getReferenceVector(); + std::vector reference_vector = ann_condition.getReferenceVector(); auto granule = std::dynamic_pointer_cast>(idx_granule); if (granule == nullptr) @@ -255,10 +253,10 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI auto annoy = granule->index; - if (condition.getNumOfDimensions() != annoy->getNumOfDimensions()) + if (ann_condition.getNumOfDimensions() != annoy->getNumOfDimensions()) throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) " "does not match with the dimension in the index ({})", - toString(condition.getNumOfDimensions()), toString(annoy->getNumOfDimensions())); + toString(ann_condition.getNumOfDimensions()), toString(annoy->getNumOfDimensions())); /// neighbors contain indexes of dots which were closest to the reference vector std::vector neighbors; diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 2c78ff536ae..457a505d909 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -81,7 +81,7 @@ private: template std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; - const ApproximateNearestNeighborCondition condition; + const ApproximateNearestNeighborCondition ann_condition; const String distance_function; const Int64 search_k; }; From 397715bfa501e89de741e2b8edfbafdd0cede707 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 09:55:37 +0000 Subject: [PATCH 1165/2223] Cosmetics: comp_dist --> comparison_distance --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 9afaba6e521..7808f07511b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -238,11 +238,11 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI { UInt64 limit = ann_condition.getLimit(); UInt64 index_granularity = ann_condition.getIndexGranularity(); - std::optional comp_dist = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where + std::optional comparison_distance = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where ? std::optional(ann_condition.getComparisonDistanceForWhereQuery()) : std::nullopt; - if (comp_dist && comp_dist.value() < 0) + if (comparison_distance && comparison_distance.value() < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); std::vector reference_vector = ann_condition.getReferenceVector(); @@ -269,7 +269,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI std::unordered_set granule_numbers; for (size_t i = 0; i < neighbors.size(); ++i) { - if (comp_dist && distances[i] > comp_dist) + if (comparison_distance && distances[i] > comparison_distance) continue; granule_numbers.insert(neighbors[i] / index_granularity); } From 06329fb08b8418e347d5c64a37beeb51d5504376 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 08:35:59 +0000 Subject: [PATCH 1166/2223] Minor: Make unique by sort/erase (should be a bit faster) --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 31 +++++++++---------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 2 +- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 7808f07511b..d1715b2c4c1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -148,8 +148,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t if (index_sample_block.columns() > 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column"); - auto index_column_name = index_sample_block.getByPosition(0).name; - const auto & column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); + const String & index_column_name = index_sample_block.getByPosition(0).name; + ColumnPtr column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); if (const auto & column_array = typeid_cast(column_cut.get())) { @@ -204,7 +204,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( - const IndexDescription & /*index*/, + const IndexDescription & /*index_description*/, const SelectQueryInfo & query, const String & distance_function_, ContextPtr context) @@ -251,35 +251,34 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); - auto annoy = granule->index; + const AnnoyIndexWithSerializationPtr & annoy = granule->index; if (ann_condition.getNumOfDimensions() != annoy->getNumOfDimensions()) throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) " - "does not match with the dimension in the index ({})", - toString(ann_condition.getNumOfDimensions()), toString(annoy->getNumOfDimensions())); + "does not match the dimension in the index ({})", + ann_condition.getNumOfDimensions(), annoy->getNumOfDimensions()); - /// neighbors contain indexes of dots which were closest to the reference vector - std::vector neighbors; + std::vector neighbors; /// indexes of dots which were closest to the reference vector std::vector distances; neighbors.reserve(limit); distances.reserve(limit); annoy->get_nns_by_vector(reference_vector.data(), limit, static_cast(search_k), &neighbors, &distances); - std::unordered_set granule_numbers; + std::vector granule_numbers; + granule_numbers.reserve(neighbors.size()); for (size_t i = 0; i < neighbors.size(); ++i) { if (comparison_distance && distances[i] > comparison_distance) continue; - granule_numbers.insert(neighbors[i] / index_granularity); + granule_numbers.push_back(neighbors[i] / index_granularity); } - std::vector result_vector; - result_vector.reserve(granule_numbers.size()); - for (auto granule_number : granule_numbers) - result_vector.push_back(granule_number); + /// make unique + std::sort(granule_numbers.begin(), granule_numbers.end()); + granule_numbers.erase(std::unique(granule_numbers.begin(), granule_numbers.end()), granule_numbers.end()); - return result_vector; + return granule_numbers; } MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String & distance_function_) @@ -302,7 +301,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 if (distance_function == "L2Distance") return std::make_shared>(index.name, index.sample_block, trees); - if (distance_function == "cosineDistance") + else if (distance_function == "cosineDistance") return std::make_shared>(index.name, index.sample_block, trees); std::unreachable(); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 457a505d909..bca06edd0f8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -64,7 +64,7 @@ class MergeTreeIndexConditionAnnoy final : public IMergeTreeIndexConditionApprox { public: MergeTreeIndexConditionAnnoy( - const IndexDescription & index, + const IndexDescription & index_description, const SelectQueryInfo & query, const String& distance_function, ContextPtr context); From ece96f54e96b526693e58b859c9c835f17eff5f4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Jun 2023 09:47:58 +0000 Subject: [PATCH 1167/2223] Fix tests --- src/Coordination/KeeperSnapshotManager.cpp | 8 +++----- src/Coordination/tests/gtest_coordination.cpp | 2 ++ 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 44e990c7b95..d10df0fd785 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -62,7 +62,7 @@ namespace std::string getSnapshotFileName(uint64_t up_to_log_idx, bool compress_zstd) { - auto base = std::string{"snapshot_"} + std::to_string(up_to_log_idx) + ".bin"; + auto base = fmt::format("snapshot_{}.bin", up_to_log_idx); if (compress_zstd) base += ".zstd"; return base; @@ -567,10 +567,8 @@ KeeperSnapshotManager::KeeperSnapshotManager( continue; } - if (clean_incomplete_file(it->path())) - continue; - - snapshot_files.push_back(it->path()); + if (it->name().starts_with("snapshot_") && !clean_incomplete_file(it->path())) + snapshot_files.push_back(it->path()); } for (const auto & snapshot_file : snapshot_files) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index ff7d545ecdd..50e81eca8ca 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1944,6 +1944,8 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) changelog1.end_of_append_batch(0, 0); } + waitDurableLogs(changelog1); + DB::KeeperLogStore changelog2( DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog2.init(0, 3); From 22110ac7427abc4feac120a1cfa5cd29850def8b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Jun 2023 11:59:49 +0200 Subject: [PATCH 1168/2223] Fix exception message --- src/Interpreters/Cache/Metadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 9dff77e2af8..d0780202121 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -374,7 +374,7 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm fs::remove(path); } else if (file_segment->downloaded_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path); file_segment->detach(segment_lock, *this); return key_metadata->erase(it); From 3082029406846dd92f1d9018156fc0bf9fee8d7c Mon Sep 17 00:00:00 2001 From: ismailakpolat Date: Mon, 5 Jun 2023 13:03:17 +0300 Subject: [PATCH 1169/2223] Update rabbitmq.md Duplicate parameter name in definition --- docs/en/engines/table-engines/integrations/rabbitmq.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 08062278904..7620cd22767 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -42,7 +42,6 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [rabbitmq_queue_consume = false,] [rabbitmq_address = '',] [rabbitmq_vhost = '/',] - [rabbitmq_queue_consume = false,] [rabbitmq_username = '',] [rabbitmq_password = '',] [rabbitmq_commit_on_select = false,] From a224c8936ccebc243ab82b7338ddd93a10f3c099 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Jun 2023 10:18:07 +0000 Subject: [PATCH 1170/2223] Fix minor issues in documentation --- docs/en/operations/system-tables/processes.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/system-tables/processes.md b/docs/en/operations/system-tables/processes.md index 2e729920ed0..ffa37357053 100644 --- a/docs/en/operations/system-tables/processes.md +++ b/docs/en/operations/system-tables/processes.md @@ -10,14 +10,14 @@ Columns: - `user` (String) – The user who made the query. Keep in mind that for distributed processing, queries are sent to remote servers under the `default` user. The field contains the username for a specific query, not for a query that this query initiated. - `address` (String) – The IP address the request was made from. The same for distributed processing. To track where a distributed query was originally made from, look at `system.processes` on the query requestor server. - `elapsed` (Float64) – The time in seconds since request execution started. -- `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. -- `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. +- `read_rows` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. +- `read_bytes` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. -- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting. +- `memory_usage` (Int64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting. - `query` (String) – The query text. For `INSERT`, it does not include the data to insert. - `query_id` (String) – Query ID, if defined. -- `is_cancelled` (Int8) – Was query cancelled. -- `is_all_data_sent` (Int8) – Was all data sent to the client (in other words query had been finished on the server). +- `is_cancelled` (UInt8) – Was query cancelled. +- `is_all_data_sent` (UInt8) – Was all data sent to the client (in other words query had been finished on the server). ```sql SELECT * FROM system.processes LIMIT 10 FORMAT Vertical; From 256f713d6b75bf971203f24bb6db0fcab6fa9aec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Jun 2023 10:18:25 +0000 Subject: [PATCH 1171/2223] Add docs for `system.user_processes` --- .../system-tables/user_processes.md | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 docs/en/operations/system-tables/user_processes.md diff --git a/docs/en/operations/system-tables/user_processes.md b/docs/en/operations/system-tables/user_processes.md new file mode 100644 index 00000000000..a9b97390ed6 --- /dev/null +++ b/docs/en/operations/system-tables/user_processes.md @@ -0,0 +1,28 @@ +--- +slug: /en/operations/system-tables/user_processes +--- +# user_processes + +This system table is used for implementing the `SHOW USER PROCESSES` query. + +Columns: + +- `user` ([String](../../sql-reference/data-types/string.md)) — User name. +- `memory_usage` ([Int64](../../sql-reference/data-types/int-uint#int-ranges)) – Sum of RAM used by all processes of the user. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting. +- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint#int-ranges)) — The peak of memory usage of the user. It can be reset when no queries are run for the user. +- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/map)) – Summary of ProfileEvents that measure different metrics for the user. The description of them could be found in the table [system.events](../../operations/system-tables/events.md#system_tables-events) + +```sql +SELECT * FROM system.user_processes LIMIT 10 FORMAT Vertical; +``` + +```response +Row 1: +────── +user: default +memory_usage: 9832 +peak_memory_usage: 9832 +ProfileEvents: {'Query':5,'SelectQuery':5,'QueriesWithSubqueries':38,'SelectQueriesWithSubqueries':38,'QueryTimeMicroseconds':842048,'SelectQueryTimeMicroseconds':842048,'ReadBufferFromFileDescriptorRead':6,'ReadBufferFromFileDescriptorReadBytes':234,'IOBufferAllocs':3,'IOBufferAllocBytes':98493,'ArenaAllocChunks':283,'ArenaAllocBytes':1482752,'FunctionExecute':670,'TableFunctionExecute':16,'DiskReadElapsedMicroseconds':19,'NetworkSendElapsedMicroseconds':684,'NetworkSendBytes':139498,'SelectedRows':6076,'SelectedBytes':685802,'ContextLock':1140,'RWLockAcquiredReadLocks':193,'RWLockReadersWaitMilliseconds':4,'RealTimeMicroseconds':1585163,'UserTimeMicroseconds':889767,'SystemTimeMicroseconds':13630,'SoftPageFaults':1947,'OSCPUWaitMicroseconds':6,'OSCPUVirtualTimeMicroseconds':903251,'OSReadChars':28631,'OSWriteChars':28888,'QueryProfilerRuns':3,'LogTrace':79,'LogDebug':24} + +1 row in set. Elapsed: 0.010 sec. +``` From e140cad10c0e0ab1a3a291b0ceef37a72bcb4295 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Jun 2023 10:18:36 +0000 Subject: [PATCH 1172/2223] Clean up includes --- src/Storages/System/StorageSystemUserProcesses.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/System/StorageSystemUserProcesses.cpp b/src/Storages/System/StorageSystemUserProcesses.cpp index 5973f9e2af3..de34fede0ac 100644 --- a/src/Storages/System/StorageSystemUserProcesses.cpp +++ b/src/Storages/System/StorageSystemUserProcesses.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include @@ -10,8 +9,6 @@ #include #include #include -#include -#include namespace DB From cdb5997339f20f27d6c6ad628c341579ffcf9264 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 10:22:34 +0000 Subject: [PATCH 1173/2223] Cosmetics: add assert --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 -- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 4 +++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index fb11fabfac1..c6b7232be4a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1693,9 +1693,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( // vector of indexes of useful ranges auto result = ann_condition->getUsefulRanges(granule); if (result.empty()) - { ++granules_dropped; - } for (auto range : result) { diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index d1715b2c4c1..cf7fbb3bab3 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -251,7 +251,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); - const AnnoyIndexWithSerializationPtr & annoy = granule->index; + AnnoyIndexWithSerializationPtr annoy = granule->index; if (ann_condition.getNumOfDimensions() != annoy->getNumOfDimensions()) throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) " @@ -265,6 +265,8 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI annoy->get_nns_by_vector(reference_vector.data(), limit, static_cast(search_k), &neighbors, &distances); + chassert(neighbors.size() == distances.size()); + std::vector granule_numbers; granule_numbers.reserve(neighbors.size()); for (size_t i = 0; i < neighbors.size(); ++i) From 7d659fdca2bf80c6f045333f531a8a4f058d33d8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 10:26:45 +0000 Subject: [PATCH 1174/2223] Tests: Decrease granule size --- tests/queries/0_stateless/02354_annoy_index.reference | 10 +++++----- tests/queries/0_stateless/02354_annoy_index.sql | 7 +------ 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 5f3b523fbe4..5b37c32d914 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -17,12 +17,12 @@ Expression ((Projection + Before ORDER BY)) PrimaryKey Condition: true Parts: 1/1 - Granules: 1/1 + Granules: 3/3 Skip Name: annoy_index Description: annoy GRANULARITY 1 Parts: 1/1 - Granules: 1/1 + Granules: 1/3 ORDER BY type, L2Distance, check that index is used Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) @@ -33,12 +33,12 @@ Expression (Projection) PrimaryKey Condition: true Parts: 1/1 - Granules: 1/1 + Granules: 3/3 Skip Name: annoy_index Description: annoy GRANULARITY 1 Parts: 1/1 - Granules: 1/1 + Granules: 3/3 parameter annoy_index_search_k_nodes parameter max_limit_for_ann_queries Expression (Projection) @@ -50,5 +50,5 @@ Expression (Projection) PrimaryKey Condition: true Parts: 1/1 - Granules: 1/1 + Granules: 3/3 Negative tests diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 3e1c176f3f6..2c40653667d 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -3,10 +3,7 @@ SET allow_experimental_annoy_index = 1; DROP TABLE IF EXISTS tab; - -DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; --- SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); SELECT 'WHERE type, L2Distance'; @@ -21,14 +18,12 @@ FROM tab ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) LIMIT 3; - SELECT 'Reference ARRAYs with non-matching dimension are rejected'; SELECT * FROM tab ORDER BY L2Distance(embedding, [0.0, 0.0]) LIMIT 3; -- { serverError INCORRECT_QUERY } - SELECT 'WHERE type, L2Distance, check that index is used'; EXPLAIN indexes=1 SELECT * From de503dc5d20bb8ddd8816c9639686c2c6089fdbe Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 10:49:45 +0000 Subject: [PATCH 1175/2223] Add tests for tuple + add tests for custom distance metric / tree count --- .../0_stateless/02354_annoy_index.reference | 67 ++++++++++++++- .../queries/0_stateless/02354_annoy_index.sql | 81 +++++++++++++++++-- 2 files changed, 140 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 5b37c32d914..5bd1377d6f4 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -1,3 +1,4 @@ +--- Test with Array --- WHERE type, L2Distance 1 [0,0,10] 2 [0,0,10.5] @@ -51,4 +52,68 @@ Expression (Projection) Condition: true Parts: 1/1 Granules: 3/3 -Negative tests +--- Test with Tuple --- +WHERE type, L2Distance +1 (0,0,10) +2 (0,0,10.5) +3 (0,0,9.5) +4 (0,0,9.7) +5 (0,0,10.2) +ORDER BY type, L2Distance +1 (0,0,10) +5 (0,0,10.2) +4 (0,0,9.7) +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 1/3 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 3/3 +parameter annoy_index_search_k_nodes +parameter max_limit_for_ann_queries +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 +--- Test alternative metric (cosine distance) and non-default NumTrees --- +WHERE type, L2Distance +1 [0,0,10] +2 [0,0,10.5] +3 [0,0,9.5] +4 [0,0,9.7] +5 [0,0,10.2] +ORDER BY type, L2Distance +1 [0,0,10] +5 [0,0,10.2] +4 [0,0,9.7] +--- Negative tests --- diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 2c40653667d..170c048d420 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -2,6 +2,8 @@ SET allow_experimental_annoy_index = 1; +SELECT '--- Test with Array ---'; + DROP TABLE IF EXISTS tab; CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); @@ -55,19 +57,87 @@ SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index DROP TABLE tab; -DROP TABLE IF EXISTS tab; +SELECT '--- Test with Tuple ---'; -SELECT 'Negative tests'; +CREATE TABLE tab(id Int32, embedding Tuple(Float32, Float32, Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; +INSERT INTO tab VALUES (1, (0.0, 0.0, 10.0)), (2, (0.0, 0.0, 10.5)), (3, (0.0, 0.0, 9.5)), (4, (0.0, 0.0, 9.7)), (5, (0.0, 0.0, 10.2)), (6, (10.0, 0.0, 0.0)), (7, (9.5, 0.0, 0.0)), (8, (9.7, 0.0, 0.0)), (9, (10.2, 0.0, 0.0)), (10, (10.5, 0.0, 0.0)), (11, (0.0, 10.0, 0.0)), (12, (0.0, 9.5, 0.0)), (13, (0.0, 9.7, 0.0)), (14, (0.0, 10.2, 0.0)), (15, (0.0, 10.5, 0.0)); + +SELECT 'WHERE type, L2Distance'; +SELECT * +FROM tab +WHERE L2Distance(embedding, (0.0, 0.0, 10.0)) < 1.0 +LIMIT 5; + +SELECT 'ORDER BY type, L2Distance'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, (0.0, 0.0, 10.0)) +LIMIT 3; + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(embedding, (0.0, 0.0, 10.0)) < 1.0 +LIMIT 5; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(embedding, (0.0, 0.0, 10.0)) +LIMIT 3; + +SELECT 'parameter annoy_index_search_k_nodes'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, (5.3, 7.3, 2.1)) +LIMIT 5 +SETTINGS annoy_index_search_k_nodes=0; -- searches zero nodes --> no results + +SELECT 'parameter max_limit_for_ann_queries'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(embedding, (5.3, 7.3, 2.1)) +LIMIT 5 +SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index + +DROP TABLE tab; + +SELECT '--- Test alternative metric (cosine distance) and non-default NumTrees ---'; + +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('cosineDistance', 200)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); + +SELECT 'WHERE type, L2Distance'; +SELECT * +FROM tab +WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 +LIMIT 5; + +SELECT 'ORDER BY type, L2Distance'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) +LIMIT 3; + +DROP TABLE tab; + +SELECT '--- Negative tests ---'; -- must have at most 2 arguments CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } --- first argument must be UInt64 +-- first argument (distance_function) must be String CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } --- 2nd argument must be String +-- 2nd argument (number of trees) must be UInt64 CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('L2Distance', 'not an UInt64')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +-- reject unsupported distance functions +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } + -- must be created on single column CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index (embedding, id) TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } @@ -77,6 +147,3 @@ CREATE TABLE tab(id Int32, embedding Float32, INDEX annoy_index embedding TYPE a CREATE TABLE tab(id Int32, embedding Array(Float64), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, embedding LowCardinality(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, embedding Nullable(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } - --- reject unsupported distance functions -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } From 8f60423daa828d9f4ce8f13b90e1c42de22f9edd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 10:55:05 +0000 Subject: [PATCH 1176/2223] Cosmetics: more constness --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 12 ++++++------ src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 5 +---- 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index cf7fbb3bab3..72dd92ead5e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -236,22 +236,22 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndex template std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const { - UInt64 limit = ann_condition.getLimit(); - UInt64 index_granularity = ann_condition.getIndexGranularity(); - std::optional comparison_distance = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where + const UInt64 limit = ann_condition.getLimit(); + const UInt64 index_granularity = ann_condition.getIndexGranularity(); + const std::optional comparison_distance = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where ? std::optional(ann_condition.getComparisonDistanceForWhereQuery()) : std::nullopt; if (comparison_distance && comparison_distance.value() < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); - std::vector reference_vector = ann_condition.getReferenceVector(); + const std::vector reference_vector = ann_condition.getReferenceVector(); - auto granule = std::dynamic_pointer_cast>(idx_granule); + const auto granule = std::dynamic_pointer_cast>(idx_granule); if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); - AnnoyIndexWithSerializationPtr annoy = granule->index; + const AnnoyIndexWithSerializationPtr annoy = granule->index; if (ann_condition.getNumOfDimensions() != annoy->getNumOfDimensions()) throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) " diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index bca06edd0f8..5204ff07b27 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -66,15 +66,13 @@ public: MergeTreeIndexConditionAnnoy( const IndexDescription & index_description, const SelectQueryInfo & query, - const String& distance_function, + const String & distance_function, ContextPtr context); ~MergeTreeIndexConditionAnnoy() override = default; bool alwaysUnknownOrTrue() const override; - bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override; - std::vector getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const override; private: @@ -97,7 +95,6 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator() const override; - MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; } From d9a6e36685f71ed592af24a7e951272a688b9eea Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 5 Jun 2023 11:02:23 +0000 Subject: [PATCH 1177/2223] Add comments --- src/Analyzer/TableNode.cpp | 3 +++ src/Planner/PlannerJoinTree.cpp | 3 +++ src/Planner/Utils.cpp | 3 +++ src/Storages/StorageDistributed.cpp | 5 +++-- 4 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index 17d12bd6afa..f899c1ae6fe 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -91,6 +91,9 @@ ASTPtr TableNode::toASTImpl(const ConvertToASTOptions & /* options */) const if (!temporary_table_name.empty()) return std::make_shared(temporary_table_name); + // In case of cross-replication we don't know what database is used for the table. + // `storage_id.hasDatabase()` can return false only on the initiator node. + // Each shard will use the default database (in the case of cross-replication shards may have different defaults). if (!storage_id.hasDatabase()) return std::make_shared(storage_id.getTableName()); return std::make_shared(storage_id.getDatabaseName(), storage_id.getTableName()); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4a1708f96d3..8bd674f5b67 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -106,6 +106,9 @@ void checkAccessRights(const TableNode & table_node, const Names & column_names, storage_id.getFullTableName()); } + // In case of cross-replication we don't know what database is used for the table. + // `storage_id.hasDatabase()` can return false only on the initiator node. + // Each shard will use the default database (in the case of cross-replication shards may have different defaults). if (storage_id.hasDatabase()) query_context->checkAccess(AccessType::SELECT, storage_id, column_names); } diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index cd4fb9182e9..94f3bbf6440 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -107,6 +107,9 @@ Block buildCommonHeaderForUnion(const Blocks & queries_headers, SelectUnionMode ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node) { auto & query_node_typed = query_node->as(); + + // In case of cross-replication we don't know what database is used for the table. + // Each shard will use the default database (in the case of cross-replication shards may have different defaults). auto result_ast = query_node_typed.toAST({ .fully_qualified_identifiers = false }); while (true) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b9625ce2ab7..969c117cb28 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,8 +30,6 @@ #include #include #include -#include "Analyzer/IQueryTreeNode.h" -#include "Analyzer/MatcherNode.h" #include #include @@ -946,6 +944,9 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, else { auto resolved_remote_storage_id = remote_storage_id; + // In case of cross-replication we don't know what database is used for the table. + // `storage_id.hasDatabase()` can return false only on the initiator node. + // Each shard will use the default database (in the case of cross-replication shards may have different defaults). if (remote_storage_id.hasDatabase()) resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); From f3a8517a447daeafd6c5bad0b819a2122cf6161c Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 5 Jun 2023 13:07:07 +0200 Subject: [PATCH 1178/2223] Fix --- src/Interpreters/Cache/FileCache.cpp | 2 +- src/Interpreters/Cache/Metadata.cpp | 10 ++++++++++ src/Interpreters/Cache/Metadata.h | 2 +- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 65dca790183..5ccbe6ad72d 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -123,7 +123,7 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: auto add_to_result = [&](const FileSegmentMetadata & file_segment_metadata) { FileSegmentPtr file_segment; - if (file_segment_metadata.valid()) + if (!file_segment_metadata.evicting()) { file_segment = file_segment_metadata.file_segment; if (file_segment->isDownloaded()) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 843ffd45b63..fea552c4071 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -346,6 +346,16 @@ void LockedKey::removeAllReleasable() ++it; continue; } + else if (it->second.evicting()) + { + /// File segment is currently a removal candidate, + /// we do not know if it will be removed or not yet, + /// but its size is currently accounted as potentially removed, + /// so if we remove file segment now, we break the freeable_count + /// calculation in tryReserve. + ++it; + continue; + } auto file_segment = it->second->file_segment; it = removeFileSegment(file_segment->offset(), file_segment->lock()); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 2e015b07ed0..4732123fabc 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -22,7 +22,7 @@ struct FileSegmentMetadata : private boost::noncopyable size_t size() const; - bool valid() const { return !removal_candidate.load(); } + bool evicting() const { return !removal_candidate.load(); } Priority::Iterator getQueueIterator() const { return file_segment->getQueueIterator(); } From 2866bac08932a841595fb788e84bc9be3b07a32f Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 5 Jun 2023 14:03:19 +0200 Subject: [PATCH 1179/2223] Add named collections and remove host filter support --- src/Storages/StorageAzure.cpp | 78 ++++++++++++++++++- src/Storages/StorageAzure.h | 9 +-- .../configs/named_collections.xml | 15 ++++ .../test_storage_azure_blob_storage/test.py | 14 ++++ 4 files changed, 106 insertions(+), 10 deletions(-) create mode 100644 tests/integration/test_storage_azure_blob_storage/configs/named_collections.xml diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 7b4bc9e6769..8ce8a923c33 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -18,6 +18,7 @@ #include #include +#include #include #include #include @@ -45,11 +46,61 @@ namespace ErrorCodes extern const int DATABASE_ACCESS_DENIED; } +namespace +{ + +static const std::unordered_set required_configuration_keys = { + "blob_path", + "container", +}; + +static const std::unordered_set optional_configuration_keys = { + "format", + "compression", + "compression_method", + "account_name", + "account_key", + "connection_string", + "storage_account_url", +}; + bool isConnectionString(const std::string & candidate) { return candidate.starts_with("DefaultEndpointsProtocol"); } + +void processNamedCollectionResult(StorageAzure::Configuration & configuration, const NamedCollection & collection) +{ + validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); + + if (collection.has("connection_string")) + { + configuration.connection_url = collection.get("connection_string"); + configuration.is_connection_string = true; + } + + if (collection.has("storage_account_url")) + { + configuration.connection_url = collection.get("storage_account_url"); + configuration.is_connection_string = false; + } + + configuration.container = collection.get("container"); + configuration.blob_path = collection.get("blob_path"); + + if (collection.has("account_name")) + configuration.account_name = collection.get("account_name"); + + if (collection.has("account_key")) + configuration.account_key = collection.get("account_key"); + + configuration.format = collection.getOrDefault("format", configuration.format); + configuration.compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); +} + +} + StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file) { StorageAzure::Configuration configuration; @@ -57,6 +108,19 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C /// Supported signatures: /// /// Azure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) + /// + + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + { + processNamedCollectionResult(configuration, *named_collection); + + configuration.blobs_paths = {configuration.blob_path}; + + if (configuration.format == "auto" && get_format_from_file) + configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); + + return configuration; + } if (engine_args.size() < 3 || engine_args.size() > 7) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, @@ -274,10 +338,20 @@ AzureClientPtr StorageAzure::createClient(StorageAzure::Configuration configurat return result; } +Poco::URI StorageAzure::Configuration::getConnectionURL() const +{ + if (!is_connection_string) + return Poco::URI(connection_url); + + auto parsed_connection_string = Azure::Storage::_internal::ParseConnectionString(connection_url); + return Poco::URI(parsed_connection_string.BlobServiceUrl.GetAbsoluteUrl()); +} + + StorageAzure::StorageAzure( const Configuration & configuration_, std::unique_ptr && object_storage_, - ContextPtr, + ContextPtr context, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -293,7 +367,7 @@ StorageAzure::StorageAzure( , partition_by(partition_by_) { FormatFactory::instance().checkFormatName(configuration.format); - //context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(configuration.getConnectionURL())); + context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.getConnectionURL()); StorageInMemoryMetadata storage_metadata; if (columns_.empty()) diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index 6bf18d91265..03a9abf2ce6 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -51,13 +51,7 @@ public: return blobs_paths.back().find(PARTITION_ID_WILDCARD) != String::npos; } - std::string getConnectionURL() const - { - if (!is_connection_string) - return connection_url; - - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Connection string not implemented yet"); - } + Poco::URI getConnectionURL() const; std::string connection_url; bool is_connection_string; @@ -121,7 +115,6 @@ private: const bool distributed_processing; std::optional format_settings; ASTPtr partition_by; - }; } diff --git a/tests/integration/test_storage_azure_blob_storage/configs/named_collections.xml b/tests/integration/test_storage_azure_blob_storage/configs/named_collections.xml new file mode 100644 index 00000000000..dc70895bc05 --- /dev/null +++ b/tests/integration/test_storage_azure_blob_storage/configs/named_collections.xml @@ -0,0 +1,15 @@ + + + + DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1; + cont + test_simple_write_named.csv + CSV + + + http://azurite1:10000/devstoreaccount1 + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + + + diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 94b059fe4fe..11404602f58 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -23,6 +23,7 @@ def cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "node", + main_configs=["configs/named_collections.xml"], with_azurite=True, ) cluster.start() @@ -80,6 +81,19 @@ def test_simple_write_connection_string(cluster): print(get_azure_file_content('test_simple_write_c.csv')) assert get_azure_file_content('test_simple_write_c.csv') == '1,"a"\n' +def test_simple_write_named_collection_1(cluster): + node = cluster.instances["node"] + azure_query(node, "CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = Azure(azure_conf1)") + azure_query(node, "INSERT INTO test_simple_write_named_collection_1 VALUES (1, 'a')") + print(get_azure_file_content('test_simple_write_named.csv')) + assert get_azure_file_content('test_simple_write_named.csv') == '1,"a"\n' + +def test_simple_write_named_collection_2(cluster): + node = cluster.instances["node"] + azure_query(node, "CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')") + azure_query(node, "INSERT INTO test_simple_write_named_collection_2 VALUES (1, 'a')") + print(get_azure_file_content('test_simple_write_named_2.csv')) + assert get_azure_file_content('test_simple_write_named_2.csv') == '1,"a"\n' def test_partition_by(cluster): node = cluster.instances["node"] From 1713dbe6318b310ba4414e8f8804fe5e0c8a155f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Jun 2023 12:07:05 +0000 Subject: [PATCH 1180/2223] Grant select on system tables to test user --- tests/queries/0_stateless/02771_system_user_processes.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02771_system_user_processes.sh b/tests/queries/0_stateless/02771_system_user_processes.sh index f0e5b2a6987..8e2fbfb5287 100755 --- a/tests/queries/0_stateless/02771_system_user_processes.sh +++ b/tests/queries/0_stateless/02771_system_user_processes.sh @@ -16,6 +16,7 @@ fi $CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS $USER" $CLICKHOUSE_CLIENT -q "CREATE USER $USER" +$CLICKHOUSE_CLIENT -q "GRANT SELECT ON system.* TO $USER" $CLICKHOUSE_CLIENT -u "$USER" -q "SELECT * FROM system.numbers LIMIT 1" $CLICKHOUSE_CLIENT -u "$USER" -q "SELECT * FROM system.numbers LIMIT 1" $CLICKHOUSE_CLIENT -q "SELECT user, toBool(ProfileEvents['SelectQuery'] > 0), toBool(ProfileEvents['Query'] > 0) FROM system.user_processes WHERE user='default'" From c8d85a43c70f96de5c5ce922a43897425566aa00 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 5 Jun 2023 12:22:32 +0000 Subject: [PATCH 1181/2223] Fix unit tests --- src/Analyzer/ColumnNode.cpp | 4 ++-- src/Analyzer/IQueryTreeNode.h | 5 ++++- src/Planner/Utils.cpp | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index dd41522ac7d..3d9f5d1640e 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -96,7 +96,7 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const std::vector column_identifier_parts; auto column_source = getColumnSourceOrNull(); - if (column_source) + if (column_source && options.fully_qualified_identifiers) { auto node_type = column_source->getNodeType(); if (node_type == QueryTreeNodeType::TABLE || @@ -117,7 +117,7 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const else { const auto & table_storage_id = table_node->getStorageID(); - if (table_storage_id.hasDatabase() && options.fully_qualified_identifiers) + if (table_storage_id.hasDatabase() && options.qualify_indentifiers_with_database) column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() }; else column_identifier_parts = { table_storage_id.getTableName() }; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 351d03bc8cb..763963b734a 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -187,10 +187,13 @@ public: /// Identifiers are fully qualified (`database.table.column`), otherwise names are just column names (`column`) bool fully_qualified_identifiers = true; + + /// Identifiers are qualified but database name is not added (`table.column`) if set to false. + bool qualify_indentifiers_with_database = true; }; /// Convert query tree to AST - ASTPtr toAST(const ConvertToASTOptions & options = { .add_cast_for_constants = true, .fully_qualified_identifiers = true }) const; + ASTPtr toAST(const ConvertToASTOptions & options = { .add_cast_for_constants = true, .fully_qualified_identifiers = true, .qualify_indentifiers_with_database = true }) const; /// Convert query tree to AST and then format it for error message. String formatConvertedASTForErrorMessage() const; diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 94f3bbf6440..733db0f00bc 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -110,7 +110,7 @@ ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node) // In case of cross-replication we don't know what database is used for the table. // Each shard will use the default database (in the case of cross-replication shards may have different defaults). - auto result_ast = query_node_typed.toAST({ .fully_qualified_identifiers = false }); + auto result_ast = query_node_typed.toAST({ .qualify_indentifiers_with_database = false }); while (true) { From 90e9df9109433971d70d544e848eeead361b96f8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Jun 2023 12:27:46 +0000 Subject: [PATCH 1182/2223] Revert "Add `SHOW USER PROCESSES` query" This reverts commit d28b4181e94c5602b5512af8ed541dcc2a1a55f2. --- src/Interpreters/InterpreterFactory.cpp | 6 ---- .../InterpreterShowUserProcessesQuery.cpp | 18 ----------- .../InterpreterShowUserProcessesQuery.h | 30 ----------------- src/Parsers/ASTShowUserProcessesQuery.h | 17 ---------- src/Parsers/ParserQueryWithOutput.cpp | 5 +-- src/Parsers/ParserShowUserProcessesQuery.h | 32 ------------------- 6 files changed, 1 insertion(+), 107 deletions(-) delete mode 100644 src/Interpreters/InterpreterShowUserProcessesQuery.cpp delete mode 100644 src/Interpreters/InterpreterShowUserProcessesQuery.h delete mode 100644 src/Parsers/ASTShowUserProcessesQuery.h delete mode 100644 src/Parsers/ParserShowUserProcessesQuery.h diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index c31e3801478..9cd1f2a251c 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -23,7 +23,6 @@ #include #include #include -#include #include #include #include @@ -82,7 +81,6 @@ #include #include #include -#include #include #include #include @@ -268,10 +266,6 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } - else if (query->as()) - { - return std::make_unique(query, context); - } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Interpreters/InterpreterShowUserProcessesQuery.cpp b/src/Interpreters/InterpreterShowUserProcessesQuery.cpp deleted file mode 100644 index 51287a7ad5b..00000000000 --- a/src/Interpreters/InterpreterShowUserProcessesQuery.cpp +++ /dev/null @@ -1,18 +0,0 @@ -#include - -#include -#include -#include - -#include - - -namespace DB -{ - -BlockIO InterpreterShowUserProcessesQuery::execute() -{ - return executeQuery("SELECT * FROM system.user_processes ORDER BY user DESC", getContext(), true); -} - -} diff --git a/src/Interpreters/InterpreterShowUserProcessesQuery.h b/src/Interpreters/InterpreterShowUserProcessesQuery.h deleted file mode 100644 index a1c385dc82f..00000000000 --- a/src/Interpreters/InterpreterShowUserProcessesQuery.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Return list of currently executing queries. -TODO(antaljanosbenjamin) - */ -class InterpreterShowUserProcessesQuery : public IInterpreter, WithMutableContext -{ -public: - InterpreterShowUserProcessesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) - : WithMutableContext(context_), query_ptr(query_ptr_) {} - - BlockIO execute() override; - - /// We ignore the quota and limits here because execute() will rewrite a show query as a SELECT query and then - /// the SELECT query will checks the quota and limits. - bool ignoreQuota() const override { return true; } - bool ignoreLimits() const override { return true; } - -private: - ASTPtr query_ptr; -}; - -} diff --git a/src/Parsers/ASTShowUserProcessesQuery.h b/src/Parsers/ASTShowUserProcessesQuery.h deleted file mode 100644 index cd522c152b6..00000000000 --- a/src/Parsers/ASTShowUserProcessesQuery.h +++ /dev/null @@ -1,17 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -struct ASTShowUserProcessesIDAndQueryNames -{ - static constexpr auto ID = "ShowUserProcesses"; - static constexpr auto Query = "SHOW USER PROCESSES"; -}; - -using ASTShowUserProcessesQuery = ASTQueryWithOutputImpl; - -} diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index d5293e5f709..6796f4528c4 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -62,7 +61,6 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserShowGrantsQuery show_grants_p; ParserShowPrivilegesQuery show_privileges_p; ParserExplainQuery explain_p(end, allow_settings_after_format_in_insert); - ParserShowUserProcessesQuery show_user_processes_p; ASTPtr query; @@ -90,8 +88,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec || show_access_p.parse(pos, query, expected) || show_access_entities_p.parse(pos, query, expected) || show_grants_p.parse(pos, query, expected) - || show_privileges_p.parse(pos, query, expected) - || show_user_processes_p.parse(pos, query, expected); + || show_privileges_p.parse(pos, query, expected); if (!parsed) return false; diff --git a/src/Parsers/ParserShowUserProcessesQuery.h b/src/Parsers/ParserShowUserProcessesQuery.h deleted file mode 100644 index be484e74d5d..00000000000 --- a/src/Parsers/ParserShowUserProcessesQuery.h +++ /dev/null @@ -1,32 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -/** Query SHOW USER PROCESSES - */ -class ParserShowUserProcessesQuery : public IParserBase -{ -protected: - const char * getName() const override { return "SHOW USER PROCESSES query"; } - - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override - { - auto query = std::make_shared(); - - if (!ParserKeyword("SHOW USER PROCESSES").ignore(pos, expected)) - return false; - - node = query; - - return true; - } -}; - -} From 28eb9562b8a8904941a4b59e8ecf1c5b97aa70cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Jun 2023 12:29:11 +0000 Subject: [PATCH 1183/2223] Remove the usage of `SHOW USER PROCESSES` from tests --- .../0_stateless/02771_system_user_processes.reference | 1 - tests/queries/0_stateless/02771_system_user_processes.sh | 7 ------- 2 files changed, 8 deletions(-) diff --git a/tests/queries/0_stateless/02771_system_user_processes.reference b/tests/queries/0_stateless/02771_system_user_processes.reference index 8c8ca8abb52..a55207ff3f4 100644 --- a/tests/queries/0_stateless/02771_system_user_processes.reference +++ b/tests/queries/0_stateless/02771_system_user_processes.reference @@ -1,4 +1,3 @@ -SHOW USER PROCESSES query succeeded! 0 0 default true true diff --git a/tests/queries/0_stateless/02771_system_user_processes.sh b/tests/queries/0_stateless/02771_system_user_processes.sh index 8e2fbfb5287..c680283d36e 100755 --- a/tests/queries/0_stateless/02771_system_user_processes.sh +++ b/tests/queries/0_stateless/02771_system_user_processes.sh @@ -7,13 +7,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) USER_POSTFIX=`random_str 10` USER="test_user_02771_$USER_POSTFIX" -if $CLICKHOUSE_CLIENT -q "SHOW USER PROCESSES" &>"${CLICKHOUSE_TMP}/test_output" -then - echo "SHOW USER PROCESSES query succeeded!" -else - cat "${CLICKHOUSE_TMP}/test_output" -fi - $CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS $USER" $CLICKHOUSE_CLIENT -q "CREATE USER $USER" $CLICKHOUSE_CLIENT -q "GRANT SELECT ON system.* TO $USER" From fe1354f22184181c3ed996e0928509917bdc5f7d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 5 Jun 2023 12:32:08 +0000 Subject: [PATCH 1184/2223] Analyzer: Do not apply Query Tree optimizations on shards --- src/Interpreters/InterpreterSelectQueryAnalyzer.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 98f70c25dcd..4f2f05dc7eb 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -135,7 +135,8 @@ QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query, QueryTreePassManager query_tree_pass_manager(context); addQueryTreePasses(query_tree_pass_manager); - if (select_query_options.ignore_ast_optimizations) + if (select_query_options.ignore_ast_optimizations + || context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) query_tree_pass_manager.run(query_tree, 1 /*up_to_pass_index*/); else query_tree_pass_manager.run(query_tree); From 5cb2d8b4e2849b015682ea05f2105258162f3335 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Jun 2023 12:32:25 +0000 Subject: [PATCH 1185/2223] Update failing tests --- tests/queries/0_stateless/01399_http_request_headers.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01399_http_request_headers.reference b/tests/queries/0_stateless/01399_http_request_headers.reference index 90a10a9818d..92ea6606a12 100644 --- a/tests/queries/0_stateless/01399_http_request_headers.reference +++ b/tests/queries/0_stateless/01399_http_request_headers.reference @@ -6,6 +6,7 @@ Code: 516 1 Code: 516 processes +processes Code: 81 [1] Code: 73 From c3d6e4c9155b22fe24018ad0099eef2ccd787f5b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 5 Jun 2023 12:36:19 +0000 Subject: [PATCH 1186/2223] Fix docs --- docs/en/operations/system-tables/user_processes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/user_processes.md b/docs/en/operations/system-tables/user_processes.md index a9b97390ed6..94c153fb683 100644 --- a/docs/en/operations/system-tables/user_processes.md +++ b/docs/en/operations/system-tables/user_processes.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/user_processes --- # user_processes -This system table is used for implementing the `SHOW USER PROCESSES` query. +This system table can be used to get overview of memory usage and ProfileEvents of users. Columns: From 4f0adf5f61fca0162184e5d4858c75fdb0a10e2e Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Jun 2023 12:40:54 +0000 Subject: [PATCH 1187/2223] Better support for avro decimals --- .../Formats/Impl/AvroRowInputFormat.cpp | 5 +++-- .../0_stateless/02782_avro_decimals.reference | 13 +++++++++++++ tests/queries/0_stateless/02782_avro_decimals.sh | 10 ++++++++++ tests/queries/0_stateless/data_avro/decimals.avro | Bin 0 -> 295 bytes 4 files changed, 26 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02782_avro_decimals.reference create mode 100755 tests/queries/0_stateless/02782_avro_decimals.sh create mode 100644 tests/queries/0_stateless/data_avro/decimals.avro diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index c2602a4d1d5..267f9e522e2 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -176,14 +176,15 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No { static constexpr size_t field_type_size = sizeof(typename DecimalType::FieldType); decoder.decodeString(tmp); - if (tmp.size() != field_type_size) + if (tmp.size() > field_type_size) throw ParsingException( ErrorCodes::CANNOT_PARSE_UUID, - "Cannot parse type {}, expected binary data with size {}, got {}", + "Cannot parse type {}, expected binary data with size less then {}, got {}", target_type->getName(), field_type_size, tmp.size()); + tmp = std::string(field_type_size - tmp.size(), '\0') + tmp; typename DecimalType::FieldType field; ReadBufferFromString buf(tmp); readBinaryBigEndian(field.value, buf); diff --git a/tests/queries/0_stateless/02782_avro_decimals.reference b/tests/queries/0_stateless/02782_avro_decimals.reference new file mode 100644 index 00000000000..ed46f1c3758 --- /dev/null +++ b/tests/queries/0_stateless/02782_avro_decimals.reference @@ -0,0 +1,13 @@ +d Decimal(14, 4) +0 +1 +1.1 +12.12 +123.123 +1234.1234 +12345.1234 +123456.1234 +1234567.1234 +12345678.1234 +123456789.1234 +1234567890.1234 diff --git a/tests/queries/0_stateless/02782_avro_decimals.sh b/tests/queries/0_stateless/02782_avro_decimals.sh new file mode 100755 index 00000000000..5b754965806 --- /dev/null +++ b/tests/queries/0_stateless/02782_avro_decimals.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "desc file('$CUR_DIR/data_avro/decimals.avro')" +$CLICKHOUSE_LOCAL -q "select * from file('$CUR_DIR/data_avro/decimals.avro')" + diff --git a/tests/queries/0_stateless/data_avro/decimals.avro b/tests/queries/0_stateless/data_avro/decimals.avro new file mode 100644 index 0000000000000000000000000000000000000000..5c29ac235d59003696735c8c8092eed5bcce41b2 GIT binary patch literal 295 zcmeZI%3@>@Nh~YM*GtY%NloU+E6vFf1M`cMGg5OCFECXrB@y Date: Mon, 5 Jun 2023 12:43:38 +0000 Subject: [PATCH 1188/2223] Fix exception message --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 267f9e522e2..ae65960a372 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -179,7 +179,7 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No if (tmp.size() > field_type_size) throw ParsingException( ErrorCodes::CANNOT_PARSE_UUID, - "Cannot parse type {}, expected binary data with size less then {}, got {}", + "Cannot parse type {}, expected binary data with size equal to or less than {}, got {}", target_type->getName(), field_type_size, tmp.size()); From aa20935cb9913d628df08e80b793ad853363203a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Jun 2023 12:45:14 +0000 Subject: [PATCH 1189/2223] Better --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index ae65960a372..201845177a5 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -183,8 +183,10 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No target_type->getName(), field_type_size, tmp.size()); + else if (tmp.size() != field_type_size) + /// Add padding with 0-bytes. + tmp = std::string(field_type_size - tmp.size(), '\0') + tmp; - tmp = std::string(field_type_size - tmp.size(), '\0') + tmp; typename DecimalType::FieldType field; ReadBufferFromString buf(tmp); readBinaryBigEndian(field.value, buf); From bc8ee56a19489bd2d42fd06fdca82ff1948236ca Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 5 Jun 2023 14:46:52 +0200 Subject: [PATCH 1190/2223] Support settings, test truncate --- src/Core/Settings.h | 5 +++ src/Storages/StorageAzure.cpp | 37 ++++++++++++++++--- src/Storages/StorageAzure.h | 8 ++++ .../test_storage_azure_blob_storage/test.py | 9 +++++ 4 files changed, 53 insertions(+), 6 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6a0833aef60..a484e8e816d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -81,7 +81,9 @@ class IColumn; M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3 s3_min_upload_part_size multiplied by s3_upload_part_size_multiply_factor.", 0) \ M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited. You ", 0) \ M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ + M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ + M(UInt64, azure_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ @@ -90,8 +92,11 @@ class IColumn; M(UInt64, s3_max_put_rps, 0, "Limit on S3 PUT request per second rate before throttling. Zero means unlimited.", 0) \ M(UInt64, s3_max_put_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_put_rps`", 0) \ M(UInt64, s3_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ + M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ + M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ + M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 8ce8a923c33..3d519ade3dc 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -255,9 +255,15 @@ void registerStorageAzure(StorageFactory & factory) if (args.storage_def->partition_by) partition_by = args.storage_def->partition_by->clone(); + const auto & context_settings = args.getContext()->getSettingsRef(); + auto settings = std::make_unique(); + settings->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; + settings->max_single_read_retries = context_settings.azure_max_single_read_retries; + settings->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); + return std::make_shared( std::move(configuration), - std::make_unique("AzureStorage", std::move(client), std::make_unique()), + std::make_unique("AzureStorage", std::move(client), std::move(settings)), args.getContext(), args.table_id, args.columns, @@ -395,7 +401,6 @@ StorageAzure::StorageAzure( void StorageAzure::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { - if (configuration.withGlobs()) { throw Exception( @@ -577,12 +582,12 @@ SinkToStoragePtr StorageAzure::write(const ASTPtr & query, const StorageMetadata throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "Azure key '{}' contains globs, so the table is in readonly mode", configuration.blob_path); - bool truncate_in_insert = local_context->getSettingsRef().s3_truncate_on_insert; + bool truncate_in_insert = local_context->getSettingsRef().azure_truncate_on_insert; if (!truncate_in_insert && object_storage->exists(StoredObject(configuration.blob_path))) { - if (local_context->getSettingsRef().s3_create_new_file_on_insert) + if (local_context->getSettingsRef().azure_create_new_file_on_insert) { size_t index = configuration.blobs_paths.size(); const auto & first_key = configuration.blobs_paths[0]; @@ -603,8 +608,8 @@ SinkToStoragePtr StorageAzure::write(const ASTPtr & query, const StorageMetadata throw Exception( ErrorCodes::BAD_ARGUMENTS, "Object in bucket {} with key {} already exists. " - "If you want to overwrite it, enable setting s3_truncate_on_insert, if you " - "want to create a new file on each insert, enable setting s3_create_new_file_on_insert", + "If you want to overwrite it, enable setting azure_truncate_on_insert, if you " + "want to create a new file on each insert, enable setting azure_create_new_file_on_insert", configuration.container, configuration.blobs_paths.back()); } } @@ -630,6 +635,26 @@ bool StorageAzure::supportsPartitionBy() const return true; } +bool StorageAzure::supportsSubcolumns() const +{ + return FormatFactory::instance().checkIfFormatSupportsSubcolumns(configuration.format); +} + +bool StorageAzure::supportsSubsetOfColumns() const +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); +} + +bool StorageAzure::prefersLargeBlocks() const +{ + return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration.format); +} + +bool StorageAzure::parallelizeOutputAfterReading(ContextPtr context) const +{ + return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration.format, context); +} + } #endif diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index 03a9abf2ce6..255d7e713e7 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -103,6 +103,14 @@ public: bool supportsPartitionBy() const override; + bool supportsSubcolumns() const override; + + bool supportsSubsetOfColumns() const override; + + bool prefersLargeBlocks() const override; + + bool parallelizeOutputAfterReading(ContextPtr context) const override; + static SchemaCache & getSchemaCache(const ContextPtr & ctx); private: diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 11404602f58..e78fa185b17 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -134,3 +134,12 @@ def test_partition_by_const_column(cluster): azure_query(node, f"CREATE TABLE test_partitioned_const_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}") azure_query(node, f"INSERT INTO test_partitioned_const_write VALUES {values}") assert values_csv == get_azure_file_content("test_88.csv") + +def test_truncate(cluster): + node = cluster.instances["node"] + azure_query(node, "CREATE TABLE test_truncate (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_truncate.csv', format='CSV')") + azure_query(node, "INSERT INTO test_truncate VALUES (1, 'a')") + assert get_azure_file_content('test_truncate.csv') == '1,"a"\n' + azure_query(node, "TRUNCATE TABLE test_truncate") + with pytest.raises(Exception): + print(get_azure_file_content('test_truncate.csv')) From 638775e5802df2393c020d9409855ab58d7ff54b Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 5 Jun 2023 12:48:54 +0000 Subject: [PATCH 1191/2223] Analyzer: fix 01487_distributed_in_not_default_db --- tests/broken_tests.txt | 1 - .../0_stateless/01487_distributed_in_not_default_db.sql | 4 ++++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index 02935712325..da2493faa1e 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -37,7 +37,6 @@ 01319_optimize_skip_unused_shards_nesting 01353_low_cardinality_join_types 01455_shard_leaf_max_rows_bytes_to_read -01487_distributed_in_not_default_db 01495_subqueries_in_with_statement 01504_rocksdb 01527_dist_sharding_key_dictGet_reload diff --git a/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql b/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql index ccd2c571290..cd027530ac8 100644 --- a/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql +++ b/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql @@ -25,6 +25,10 @@ CREATE TABLE d AS t ENGINE = Distributed(test_cluster_two_shards_different_datab USE test_01487; DROP DATABASE test_01487; +-- After the default database is dropped QueryAnalysisPass cannot process the following SELECT query. +-- That query is invalid on the initiator node. +set allow_experimental_analyzer = 0; + SELECT * FROM main_01487.d WHERE value IN (SELECT l.value FROM l) ORDER BY value; USE main_01487; From c4e1dc55458cfc53ad3eee1406e6bac61bfe32d7 Mon Sep 17 00:00:00 2001 From: Daniel Kutenin Date: Mon, 5 Jun 2023 14:10:53 +0100 Subject: [PATCH 1192/2223] Enable FAST_DEC_LOOP for Arm LZ4 to get 5% of decomp speed It's disabled for clang because of mobile https://github.com/lz4/lz4/blob/e82198428c8061372d5adef1f9bfff4203f6081e/lib/lz4.c#L471 --- contrib/lz4-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/lz4-cmake/CMakeLists.txt b/contrib/lz4-cmake/CMakeLists.txt index 94def029410..c0fd574134f 100644 --- a/contrib/lz4-cmake/CMakeLists.txt +++ b/contrib/lz4-cmake/CMakeLists.txt @@ -12,6 +12,7 @@ add_library (_lz4 ${SRCS}) add_library (ch_contrib::lz4 ALIAS _lz4) target_compile_definitions (_lz4 PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS=1) +target_compile_definitions (_lz4 PUBLIC LZ4_FAST_DEC_LOOP=1) if (SANITIZE STREQUAL "undefined") target_compile_options (_lz4 PRIVATE -fno-sanitize=undefined) endif () From 4c88b7bbb725245060067b101593688f470ee399 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 13:13:49 +0000 Subject: [PATCH 1193/2223] Further improve ANN index docs --- .../mergetree-family/annindexes.md | 152 +++++++++--------- 1 file changed, 72 insertions(+), 80 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 0cc1cff2dad..58655c11321 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -22,15 +22,53 @@ ORDER BY L2Distance(column, Point) LIMIT N ``` -The queries are expensive because the L2 distance (Euclidean distance) between all points in `column` and `Point` must be computed. To speed this process up, ANN indexes store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer quickly. +The queries are expensive because the L2 (Euclidean) distance between `Point` and all points in `column` and must be computed. To speed this process up, Approximate Nearest Neighbor Search Indexes (ANN indexes) store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer quickly. -## Indexes Structure +# Creating ANN Indexes -Approximate Nearest Neighbor Search Indexes (or `ANNIndexes`) are similar to skip indexes. They are constructed over granules and determine which granules can be skipped. Compared to skip indices, ANN indices are not only able to skip granules, they can also to select particular granules from a set of granules. +As long as ANN indexes are experimental, you first need to `SET allow_experimental_annoy_index = 1`. -`ANNIndexes` support two types of queries: +Syntax to create an ANN index over an `Array` column: + +```sql +CREATE TABLE table +( + `id` Int64, + `embedding` Array(Float32), + INDEX embedding TYPE () GRANULARITY +) +ENGINE = MergeTree +ORDER BY id; +``` + +Syntax to create an ANN index over a `Tuple` column: + +```sql +CREATE TABLE table +( + `id` Int64, + `embedding` Tuple(Float32[, Float32[, ...]]), + INDEX embedding TYPE () GRANULARITY +) +ENGINE = MergeTree +ORDER BY id; +``` + +ANN indexes are built during column insertion and merge and `INSERT` and `OPTIMIZE` statements will slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively comparatively many more read requests than write requests. + +Similar to regular skip indexes, ANN indexes are constructed over granules and each indexed block consists of `GRANULARITY = `-many +granules. For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`, +then each indexed block will consist of 16384 rows. However, unlike skip indexes, ANN indexes are not only able to skip the entire indexed +block, they are able to skip individual granules in indexed blocks. As a result, the `GRANULARITY` parameter has a different meaning in ANN +indexes than in normal skip indexes. Basically, the bigger `GRANULARITY` is chosen, the more data is provided to a single ANN index, and the +higher the chance that with the right hyper parameters, the index will remember the data structure better. + +# Using ANN Indexes + +ANN indexes support two types of queries: - WHERE queries: + ``` sql SELECT * FROM table @@ -39,86 +77,40 @@ Approximate Nearest Neighbor Search Indexes (or `ANNIndexes`) are similar to ski ``` - ORDER BY queries: + ``` sql SELECT * - FROM table [WHERE ...] + FROM table + [WHERE ...] ORDER BY DistanceFunction(column, Point) LIMIT N ``` -`DistanceFunction` is a [distance functions](/docs/en/sql-reference/functions/distance-functions.md), `Point` is a given vector (e.g. `(0.17, 0.33, ...)`) and `MaxDistance` is a float value which restricts the size of the neighbourhood. +`DistanceFunction` is a [distance function](/docs/en/sql-reference/functions/distance-functions.md), `Point` is a reference vector (e.g. `(0.17, 0.33, ...)`) and `MaxDistance` is a floating point value which restricts the size of the neighbourhood. -To avoid writing large vectors, you can also use [query parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. +:::tip +To avoid writing out large vectors, you can use [query parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. ```bash clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(embedding, {vec: Array(Float32)}) < 1.0" ``` +::: -ANN index cannot speed up query that contain both `WHERE` and `ORDER BY`. Queries must have a limit, as the approximate algorithms used to determine the nearest neighbors require a specific number of them. +ANN indexes cannot speed up queries that contain both a `WHERE DistanceFunction(column, Point) < MaxDistance` and an `ORDER BY DistanceFunction(column, Point)` clause. Also, the approximate algorithms used to determine the nearest neighbors require a limit, hence queries that use an ANN index must have a `LIMIT` clause. -Indexes are only used for queries with a `LIMIT` value smaller than setting `max_limit_for_ann_queries` (default: 1 million rows). This helps to prevent memory overflows in queries with a large limit. +An ANN index is only used if the query has a `LIMIT` value smaller than setting `max_limit_for_ann_queries` (default: 1 million rows). This is a safety measure which helps to avoid large memory consumption by external libraries for approximate neighbor search. -Both types of queries are processed similarly. The indexes are passed the number of neighbors `N`. In `ORDER BY` query they remember the numbers of all parts of the granule that have at least one of neighbor. In `WHERE` query they remember only those parts that satisfy the requirements. - - -## Creating Tables with an ANN Index - -As long as ANN indexes are experimental, you first need to `SET allow_experimental_annoy_index = 1`. - -Syntax: - -```sql -CREATE TABLE table -( - `id` Int64, - `embedding` Tuple(Float32, Float32, Float32), - INDEX embedding TYPE () GRANULARITY N -) -ENGINE = MergeTree -ORDER BY id; -``` - -```sql -CREATE TABLE table -( - `id` Int64, - `embedding` Array(Float32), - INDEX embedding TYPE () GRANULARITY N -) -ENGINE = MergeTree -ORDER BY id; -``` - -With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyper parameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes. - -Note that ANN indexes are built during column insertion and merge, i.e. `INSERT` and `OPTIMIZE` statements are slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changing data in conjunction with many read requests. - -# Index list +# Available ANN Indexes - [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy) -# Annoy {#annoy} +## Annoy {#annoy} -(currently disabled on ARM due to problems with the algorithm) +(currently disabled on ARM due to memory safety problems with the algorithm) -This ANN index type implements [Annoy indexes](https://github.com/spotify/annoy). +This type of ANN index implements [the Annoy algorithm](https://github.com/spotify/annoy) which uses a recursive division of the space in random linear surfaces (lines in 2D, planes in 3D etc.). -Short description of the algorithm: -The algorithm recursively divides in half all space by random linear surfaces (lines in 2D, planes in 3D etc.). Thus it makes tree of polyhedrons and points that they contains. Repeating the operation several times for greater accuracy it creates a forest. -To find K Nearest Neighbours it goes down through the trees and fills the buffer of closest points using the priority queue of polyhedrons. Next, it sorts buffer and return the nearest K points. - -Examples: - -```sql -CREATE TABLE table -( - id Int64, - embedding Tuple(Float32, Float32, Float32), - INDEX embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N -) -ENGINE = MergeTree -ORDER BY id; -``` +Syntax to create a Annoy index over a `Array` column: ```sql CREATE TABLE table @@ -131,26 +123,26 @@ ENGINE = MergeTree ORDER BY id; ``` +Syntax to create a Annoy index over a `Tuple` column: + +```sql +CREATE TABLE table +( + id Int64, + embedding Tuple(Float32[, Float32[, ...]]), + INDEX embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N +) +ENGINE = MergeTree +ORDER BY id; +``` + +Parameter `DistanceName` is name of a distance function (default `L2Distance`). Annoy currently supports `L2Distance` and `cosineDistance` as distance functions. Parameter `NumTrees` (default: 100) is the number of trees which the algorithm will create. Higher values of `NumTree` mean slower `CREATE` and `SELECT` statements (approximately linearly), but increase the accuracy of search results. + :::note Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(embedding) = 256`. ::: -Parameter `DistanceName` is name of a distance function with default `L2Distance`. Parameter `NumTrees` (default: 100) is the number of trees which the algorithm will create. Higher values of `NumTree` mean slower `CREATE` and `SELECT` statements (approximately linearly), but increase the accuracy of search results. - -```sql -CREATE TABLE table -( - id Int64, - embedding Array(Float32), - INDEX ann_index_name embedding TYPE annoy('cosineDistance') GRANULARITY N -) -ENGINE = MergeTree -ORDER BY id; -``` - -Annoy currently supports `L2Distance` and `cosineDistance` as distance functions. - -Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many nodes are inspected during SELECTs. It can be used to +Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. It can be used to balance runtime and accuracy at runtime. Example: From 5fb4f1fc614a749bd0706e5dcd952224e821bf77 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 5 Jun 2023 06:55:34 -0700 Subject: [PATCH 1194/2223] Implement review comments --- .../ReservoirSamplerDeterministic.h | 2 +- src/Common/TransformEndianness.hpp | 35 ++++++++++++------- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/src/AggregateFunctions/ReservoirSamplerDeterministic.h index 9dea821e839..b1a39a5dcc5 100644 --- a/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -178,7 +178,7 @@ public: void write(DB::WriteBuffer & buf) const { - const auto size = samples.size(); + const size_t size = samples.size(); writeBinaryLittleEndian(size, buf); writeBinaryLittleEndian(total_values, buf); diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 17cf441d17f..228490d24a1 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -8,18 +8,35 @@ namespace DB { template - requires is_big_int_v +requires std::is_integral_v +inline void transformEndianness(T & value) +{ + if constexpr (endian != std::endian::native) + value = std::byteswap(value); +} + +template +requires is_big_int_v inline void transformEndianness(T & x) { if constexpr (std::endian::native != endian) { - std::ranges::transform(x.items, std::begin(x.items), [](auto& item) { return std::byteswap(item); }); - std::ranges::reverse(x.items); + auto & items = x.items; + std::transform(std::begin(items), std::end(items), std::begin(items), [](auto & item) { return std::byteswap(item); }); + std::reverse(std::begin(items), std::end(items)); } } template - requires is_decimal || std::is_floating_point_v +requires is_decimal +inline void transformEndianness(T & x) +{ + if constexpr (std::endian::native != endian) + transformEndianness(x.value); +} + +template +requires std::is_floating_point_v inline void transformEndianness(T & value) { if constexpr (std::endian::native != endian) @@ -30,15 +47,7 @@ inline void transformEndianness(T & value) } template - requires std::is_integral_v && (sizeof(T) <= 8) -inline void transformEndianness(T & value) -{ - if constexpr (endian != std::endian::native) - value = std::byteswap(value); -} - -template - requires std::is_scoped_enum_v +requires std::is_scoped_enum_v inline void transformEndianness(T & x) { using UnderlyingType = std::underlying_type_t; From 50430ed304d8a4d4ce2bafc66bc1b7b74afec678 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 5 Jun 2023 06:55:52 -0700 Subject: [PATCH 1195/2223] Configure rule for concepts requires clause --- .clang-format | 1 + 1 file changed, 1 insertion(+) diff --git a/.clang-format b/.clang-format index 2da3911dced..893d9c613f1 100644 --- a/.clang-format +++ b/.clang-format @@ -74,6 +74,7 @@ ConstructorInitializerIndentWidth: 4 ContinuationIndentWidth: 4 DerivePointerAlignment: false DisableFormat: false +IndentRequiresClause: false IndentWidth: 4 IndentWrappedFunctionNames: false MacroBlockBegin: '' From 2b3db1d33c8be5dcfef3ba189a292d469f0f9676 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Jun 2023 17:05:06 +0200 Subject: [PATCH 1196/2223] Update Metadata.cpp --- src/Interpreters/Cache/Metadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index fea552c4071..5b6561a665e 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -346,7 +346,7 @@ void LockedKey::removeAllReleasable() ++it; continue; } - else if (it->second.evicting()) + else if (it->second->evicting()) { /// File segment is currently a removal candidate, /// we do not know if it will be removed or not yet, From 3938309374cef05afb618c36d932bd380abb1651 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 5 Jun 2023 08:18:03 -0700 Subject: [PATCH 1197/2223] Implement review comments --- .../Serializations/SerializationUUID.cpp | 2 +- src/IO/ReadHelpers.cpp | 27 ++++++++++--------- src/IO/ReadHelpers.h | 1 - src/IO/WriteHelpers.cpp | 25 ++++++++--------- src/IO/WriteHelpers.h | 7 +++-- .../Formats/Impl/AvroRowInputFormat.cpp | 2 +- .../Formats/Impl/AvroRowOutputFormat.cpp | 4 +-- 7 files changed, 35 insertions(+), 33 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationUUID.cpp b/src/DataTypes/Serializations/SerializationUUID.cpp index 13313111b2b..76be273d7dc 100644 --- a/src/DataTypes/Serializations/SerializationUUID.cpp +++ b/src/DataTypes/Serializations/SerializationUUID.cpp @@ -51,7 +51,7 @@ void SerializationUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & ist { assertChar('\'', istr); char * next_pos = find_first_symbols<'\\', '\''>(istr.position(), istr.buffer().end()); - const auto len = next_pos - istr.position(); + const size_t len = next_pos - istr.position(); if ((len == 32 || len == 36) && istr.position()[len] == '\'') { uuid = parseUUID(std::span(reinterpret_cast(istr.position()), len)); diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index a85a057f2b3..99b3e4b514b 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -31,6 +31,7 @@ namespace ErrorCodes extern const int CANNOT_PARSE_QUOTED_STRING; extern const int CANNOT_PARSE_DATETIME; extern const int CANNOT_PARSE_DATE; + extern const int CANNOT_PARSE_UUID; extern const int INCORRECT_DATA; extern const int ATTEMPT_TO_READ_AFTER_EOF; extern const int LOGICAL_ERROR; @@ -51,33 +52,35 @@ UUID parseUUID(std::span src) UUID uuid; const auto * src_ptr = src.data(); auto * dst = reinterpret_cast(&uuid); - if (const auto size = src.size(); size == 36) + const auto size = src.size(); + if (size == 36) { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - parseHex<4>(src_ptr, dst); - parseHex<2>(src_ptr + 9, dst + 4); - parseHex<2>(src_ptr + 14, dst + 6); - parseHex<2>(src_ptr + 19, dst + 8); - parseHex<6>(src_ptr + 24, dst + 10); -#else +#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ const std::reverse_iterator dst_it(dst + sizeof(UUID)); - /// FIXME This code looks like trash. parseHex<4>(src_ptr, dst + 8); parseHex<2>(src_ptr + 9, dst + 12); parseHex<2>(src_ptr + 14, dst + 14); parseHex<2>(src_ptr + 19, dst); parseHex<6>(src_ptr + 24, dst + 2); +#else + parseHex<4>(src_ptr, dst); + parseHex<2>(src_ptr + 9, dst + 4); + parseHex<2>(src_ptr + 14, dst + 6); + parseHex<2>(src_ptr + 19, dst + 8); + parseHex<6>(src_ptr + 24, dst + 10); #endif } else if (size == 32) { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - parseHex<16>(src_ptr, dst); -#else +#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ parseHex<8>(src_ptr, dst + 8); parseHex<8>(src_ptr + 16, dst); +#else + parseHex<16>(src_ptr, dst); #endif } + else + throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Unexpected length when trying to parse UUID ({})", size); return uuid; } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 7e293944d19..804dab16db9 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -765,7 +765,6 @@ inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf) return readDateTextImpl(date, buf); } -/// If string is not like UUID - implementation specific behaviour. UUID parseUUID(std::span src); template diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index 6023d4c9d5b..4f1a95181d4 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -20,25 +20,12 @@ void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes) } } -/** Function used when byte ordering is important when parsing uuid - * ex: When we create an UUID type - */ std::array formatUUID(const UUID & uuid) { std::array dst; const auto * src_ptr = reinterpret_cast(&uuid); auto * dst_ptr = dst.data(); -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - formatHex(src_ptr, dst_ptr, 4); - dst[8] = '-'; - formatHex(src_ptr + 4, dst_ptr + 9, 2); - dst[13] = '-'; - formatHex(src_ptr + 6, dst_ptr + 14, 2); - dst[18] = '-'; - formatHex(src_ptr + 8, dst_ptr + 19, 2); - dst[23] = '-'; - formatHex(src_ptr + 10, dst_ptr + 24, 6); -#else +#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ const std::reverse_iterator src_it(src_ptr + 16); formatHex(src_it + 8, dst_ptr, 4); dst[8] = '-'; @@ -49,6 +36,16 @@ std::array formatUUID(const UUID & uuid) formatHex(src_it, dst_ptr + 19, 2); dst[23] = '-'; formatHex(src_it + 2, dst_ptr + 24, 6); +#else + formatHex(src_ptr, dst_ptr, 4); + dst[8] = '-'; + formatHex(src_ptr + 4, dst_ptr + 9, 2); + dst[13] = '-'; + formatHex(src_ptr + 6, dst_ptr + 14, 2); + dst[18] = '-'; + formatHex(src_ptr + 8, dst_ptr + 19, 2); + dst[23] = '-'; + formatHex(src_ptr + 10, dst_ptr + 24, 6); #endif return dst; diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 923684c4249..056c2ca1b50 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -625,12 +625,15 @@ inline void writeXMLStringForTextElement(std::string_view s, WriteBuffer & buf) writeXMLStringForTextElement(s.data(), s.data() + s.size(), buf); } +/// @brief Serialize `uuid` into an array of characters in big-endian byte order. +/// @param uuid UUID to serialize. +/// @return Array of characters in big-endian byte order. std::array formatUUID(const UUID & uuid); inline void writeUUIDText(const UUID & uuid, WriteBuffer & buf) { - const auto text = formatUUID(uuid); - buf.write(text.data(), text.size()); + const auto serialized_uuid = formatUUID(uuid); + buf.write(serialized_uuid.data(), serialized_uuid.size()); } void writeIPv4Text(const IPv4 & ip, WriteBuffer & buf); diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 974b198a483..a4d4e374f4f 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -256,7 +256,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro if (tmp.length() != 36) throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", tmp); - const auto uuid = parseUUID({reinterpret_cast(tmp.data()), tmp.length()}); + const UUID uuid = parseUUID({reinterpret_cast(tmp.data()), tmp.length()}); assert_cast(column).insertValue(uuid); return true; }; diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index 2b163164d56..f0985e7cffc 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -329,8 +329,8 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF return {schema, [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { const auto & uuid = assert_cast(column).getElement(row_num); - const auto text = formatUUID(uuid); - encoder.encodeBytes(reinterpret_cast(text.data()), text.size()); + const auto serialized_uuid = formatUUID(uuid); + encoder.encodeBytes(reinterpret_cast(serialized_uuid.data()), serialized_uuid.size()); }}; } case TypeIndex::Array: From 33e51d4f3b25aa1af5dedf751b5fec5229dc6eac Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Jun 2023 15:22:04 +0000 Subject: [PATCH 1198/2223] Add setting to limit the number of bytes to read in schema inference --- docs/en/interfaces/schema-inference.md | 16 +++++++------ .../operations/settings/settings-formats.md | 6 +++++ src/Core/Settings.h | 1 + src/Formats/EscapingRuleUtils.cpp | 3 ++- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 3 ++- src/Formats/ReadSchemaUtils.cpp | 14 +++++++---- src/Processors/Formats/ISchemaReader.cpp | 24 +++++++++++-------- src/Processors/Formats/ISchemaReader.h | 9 +++++-- .../Impl/JSONColumnsBlockInputFormat.cpp | 2 +- .../Impl/JSONColumnsBlockInputFormatBase.cpp | 12 ++++++---- .../Impl/JSONColumnsBlockInputFormatBase.h | 16 +++++++++++-- .../JSONCompactColumnsBlockInputFormat.cpp | 2 +- ...ytes_to_read_in_schema_inference.reference | 1 + ..._max_bytes_to_read_in_schema_inference.sql | 4 ++++ 15 files changed, 80 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/02783_max_bytes_to_read_in_schema_inference.reference create mode 100644 tests/queries/0_stateless/02783_max_bytes_to_read_in_schema_inference.sql diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index c448d0aee47..a757a032b7d 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -329,8 +329,8 @@ SELECT count() FROM system.schema_inference_cache WHERE storage='S3' ## Text formats {#text-formats} For text formats, ClickHouse reads the data row by row, extracts column values according to the format, -and then uses some recursive parsers and heuristics to determine the type for each value. The maximum number of rows read from the data in schema inference -is controlled by the setting `input_format_max_rows_to_read_for_schema_inference` with default value 25000. +and then uses some recursive parsers and heuristics to determine the type for each value. The maximum number of rows and bytes read from the data in schema inference +is controlled by the settings `input_format_max_rows_to_read_for_schema_inference` (25000 by default) and `input_format_max_bytes_to_read_for_schema_inference` (32Mb by default). By default, all inferred types are [Nullable](../sql-reference/data-types/nullable.md), but you can change this by setting `schema_inference_make_columns_nullable` (see examples in the [settings](#settings-for-text-formats) section). ### JSON formats {#json-formats} @@ -1144,13 +1144,15 @@ Line: value_1=2, value_2="Some string 2", value_3="[4, 5, NULL]"$$) ### Settings for text formats {#settings-for-text-formats} -#### input_format_max_rows_to_read_for_schema_inference +#### input_format_max_rows_to_read_for_schema_inference/input_format_max_bytes_to_read_for_schema_inference -This setting controls the maximum number of rows to be read while schema inference. -The more rows are read, the more time is spent on schema inference, but the greater the chance to +These settings control the amount of data to be read while schema inference. +The more rows/bytes are read, the more time is spent on schema inference, but the greater the chance to correctly determine the types (especially when the data contains a lot of nulls). -Default value: `25000`. +Default values: +- `25000` for `input_format_max_rows_to_read_for_schema_inference`. +- `33554432` (32 Mb) for `input_format_max_bytes_to_read_for_schema_inference`. #### column_names_for_schema_inference @@ -1623,7 +1625,7 @@ In schema inference for CapnProto format ClickHouse uses the following type matc ## Strong-typed binary formats {#strong-typed-binary-formats} In such formats, each serialized value contains information about its type (and possibly about its name), but there is no information about the whole table. -In schema inference for such formats, ClickHouse reads data row by row (up to `input_format_max_rows_to_read_for_schema_inference` rows) and extracts +In schema inference for such formats, ClickHouse reads data row by row (up to `input_format_max_rows_to_read_for_schema_inference` rows or `input_format_max_bytes_to_read_for_schema_inference` bytes) and extracts the type (and possibly name) for each value from the data and then converts these types to ClickHouse types. ### MsgPack {#msgpack} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 65038d3a256..e4a8c916bcf 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -137,6 +137,12 @@ The maximum rows of data to read for automatic schema inference. Default value: `25'000`. +## input_format_max_bytes_to_read_for_schema_inference {#input_format_max_bytes_to_read_for_schema_inference} + +The maximum amount of data in bytes to read for automatic schema inference. + +Default value: `33554432` (32 Mb). + ## column_names_for_schema_inference {#column_names_for_schema_inference} The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...' diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 67c92a0be8b..f1e6c518f30 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -844,6 +844,7 @@ class IColumn; M(UInt64, input_format_msgpack_number_of_columns, 0, "The number of columns in inserted MsgPack data. Used for automatic schema inference from data.", 0) \ M(MsgPackUUIDRepresentation, output_format_msgpack_uuid_representation, FormatSettings::MsgPackUUIDRepresentation::EXT, "The way how to output UUID in MsgPack format.", 0) \ M(UInt64, input_format_max_rows_to_read_for_schema_inference, 25000, "The maximum rows of data to read for automatic schema inference", 0) \ + M(UInt64, input_format_max_bytes_to_read_for_schema_inference, 32 * 1024 * 1024, "The maximum bytes of data to read for automatic schema inference", 0) \ M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 804f32e4b46..9f744218da2 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -408,9 +408,10 @@ DataTypes getDefaultDataTypeForEscapingRules(const std::vectormax_rows_to_read_for_schema_inference : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference; + size_t max_bytes_to_read = format_settings ? format_settings->max_bytes_to_read_for_schema_inference + : context->getSettingsRef().input_format_max_bytes_to_read_for_schema_inference; size_t iterations = 0; ColumnsDescription cached_columns; while (true) @@ -120,7 +122,7 @@ ColumnsDescription readSchemaFromFormat( try { schema_reader = FormatFactory::instance().getSchemaReader(format_name, *buf, context, format_settings); - schema_reader->setMaxRowsToRead(max_rows_to_read); + schema_reader->setMaxRowsAndBytesToRead(max_rows_to_read, max_bytes_to_read); names_and_types = schema_reader->readSchema(); break; } @@ -132,10 +134,14 @@ ColumnsDescription readSchemaFromFormat( size_t rows_read = schema_reader->getNumRowsRead(); assert(rows_read <= max_rows_to_read); max_rows_to_read -= schema_reader->getNumRowsRead(); - if (rows_read != 0 && max_rows_to_read == 0) + size_t bytes_read = buf->count(); + /// We could exceed max_bytes_to_read a bit to complete row parsing. + max_bytes_to_read -= std::min(bytes_read, max_bytes_to_read); + if (rows_read != 0 && (max_rows_to_read == 0 || max_bytes_to_read == 0)) { - exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting " - "input_format_max_rows_to_read_for_schema_inference"; + exception_message += "\nTo increase the maximum number of rows/bytes to read for structure determination, use setting " + "input_format_max_rows_to_read_for_schema_inference/input_format_max_bytes_to_read_for_schema_inference"; + if (iterations > 1) { exception_messages += "\n" + exception_message; diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index c96cb373a2d..9f26a3543d0 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -57,11 +57,15 @@ void checkFinalInferredType( } IIRowSchemaReader::IIRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, DataTypePtr default_type_) - : ISchemaReader(in_), default_type(default_type_), hints_str(format_settings_.schema_inference_hints), format_settings(format_settings_) + : ISchemaReader(in_) + , max_rows_to_read(format_settings_.max_rows_to_read_for_schema_inference) + , max_bytes_to_read(format_settings_.max_bytes_to_read_for_schema_inference) + , default_type(default_type_) + , hints_str(format_settings_.schema_inference_hints) + , format_settings(format_settings_) { } - void IIRowSchemaReader::setContext(ContextPtr & context) { ColumnsDescription columns; @@ -99,11 +103,11 @@ IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & form NamesAndTypesList IRowSchemaReader::readSchema() { - if (max_rows_to_read == 0) + if (max_rows_to_read == 0 || max_bytes_to_read == 0) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Cannot read rows to determine the schema, the maximum number of rows to read is set to 0. " - "Most likely setting input_format_max_rows_to_read_for_schema_inference is set to 0"); + "Cannot read rows to determine the schema, the maximum number of rows (or bytes) to read is set to 0. " + "Most likely setting input_format_max_rows_to_read_for_schema_inference or input_format_max_bytes_to_read_for_schema_inference is set to 0"); DataTypes data_types = readRowAndGetDataTypes(); @@ -143,7 +147,7 @@ NamesAndTypesList IRowSchemaReader::readSchema() data_types[i] = hint_it->second; } - for (rows_read = 1; rows_read < max_rows_to_read; ++rows_read) + for (rows_read = 1; rows_read < max_rows_to_read && in.count() < max_bytes_to_read; ++rows_read) { DataTypes new_data_types = readRowAndGetDataTypes(); if (new_data_types.empty()) @@ -220,11 +224,11 @@ IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, const For NamesAndTypesList IRowWithNamesSchemaReader::readSchema() { - if (max_rows_to_read == 0) + if (max_rows_to_read == 0 || max_bytes_to_read == 0) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Cannot read rows to determine the schema, the maximum number of rows to read is set to 0. " - "Most likely setting input_format_max_rows_to_read_for_schema_inference is set to 0"); + "Cannot read rows to determine the schema, the maximum number of rows (or bytes) to read is set to 0. " + "Most likely setting input_format_max_rows_to_read_for_schema_inference or input_format_max_bytes_to_read_for_schema_inference is set to 0"); bool eof = false; auto names_and_types = readRowAndGetNamesAndDataTypes(eof); @@ -245,7 +249,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema() names_order.push_back(name); } - for (rows_read = 1; rows_read < max_rows_to_read; ++rows_read) + for (rows_read = 1; rows_read < max_rows_to_read && in.count() < max_bytes_to_read; ++rows_read) { auto new_names_and_types = readRowAndGetNamesAndDataTypes(eof); if (eof) diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h index 78b34a07840..40702198a57 100644 --- a/src/Processors/Formats/ISchemaReader.h +++ b/src/Processors/Formats/ISchemaReader.h @@ -32,7 +32,7 @@ public: virtual bool needContext() const { return false; } virtual void setContext(ContextPtr &) {} - virtual void setMaxRowsToRead(size_t) {} + virtual void setMaxRowsAndBytesToRead(size_t, size_t) {} virtual size_t getNumRowsRead() const { return 0; } virtual ~ISchemaReader() = default; @@ -54,12 +54,17 @@ public: virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type); protected: - void setMaxRowsToRead(size_t max_rows) override { max_rows_to_read = max_rows; } + void setMaxRowsAndBytesToRead(size_t max_rows, size_t max_bytes) override + { + max_rows_to_read = max_rows; + max_bytes_to_read = max_bytes; + } size_t getNumRowsRead() const override { return rows_read; } virtual void transformFinalTypeIfNeeded(DataTypePtr &) {} size_t max_rows_to_read; + size_t max_bytes_to_read; size_t rows_read = 0; DataTypePtr default_type; String hints_str; diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormat.cpp b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormat.cpp index 8d4c4b0c6cf..3d003658e64 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormat.cpp @@ -55,7 +55,7 @@ void registerJSONColumnsSchemaReader(FormatFactory & factory) ); factory.registerAdditionalInfoForSchemaCacheGetter("JSONColumns", [](const FormatSettings & settings) { - return getAdditionalFormatInfoByEscapingRule(settings, FormatSettings::EscapingRule::JSON); + return getAdditionalFormatInfoForAllRowBasedFormats(settings) + getAdditionalFormatInfoByEscapingRule(settings, FormatSettings::EscapingRule::JSON); }); } diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp index 2e264c59f56..84a07ebc8fb 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp @@ -176,6 +176,8 @@ JSONColumnsSchemaReaderBase::JSONColumnsSchemaReaderBase( , hints_str(format_settings_.schema_inference_hints) , reader(std::move(reader_)) , column_names_from_settings(splitColumnNames(format_settings_.column_names_for_schema_inference)) + , max_rows_to_read(format_settings_.max_rows_to_read_for_schema_inference) + , max_bytes_to_read(format_settings_.max_bytes_to_read_for_schema_inference) { } @@ -196,12 +198,12 @@ void JSONColumnsSchemaReaderBase::transformTypesIfNeeded(DataTypePtr & type, Dat NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema() { - size_t total_rows_read = 0; std::unordered_map names_to_types; std::vector names_order; /// Read data block by block and determine the type for each column - /// until max_rows_to_read_for_schema_inference is reached. - while (total_rows_read < format_settings.max_rows_to_read_for_schema_inference) + /// until max_rows_to_read/max_bytes_to_read is reached. + /// Note that we can exceed max_bytes_to_read to compete block parsing. + while (total_rows_read < max_rows_to_read && in.count() < max_bytes_to_read) { if (in.eof()) break; @@ -268,7 +270,7 @@ NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema() return result; } -DataTypePtr JSONColumnsSchemaReaderBase::readColumnAndGetDataType(const String & column_name, size_t & rows_read, size_t max_rows_to_read) +DataTypePtr JSONColumnsSchemaReaderBase::readColumnAndGetDataType(const String & column_name, size_t & rows_read, size_t max_rows) { /// Check for empty column. if (reader->checkColumnEnd()) @@ -279,7 +281,7 @@ DataTypePtr JSONColumnsSchemaReaderBase::readColumnAndGetDataType(const String & do { /// If we reached max_rows_to_read, skip the rest part of this column. - if (rows_read == max_rows_to_read) + if (rows_read == max_rows) { reader->skipColumn(); break; diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h index 2babc0734f9..886c8841540 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h @@ -82,11 +82,19 @@ public: bool needContext() const override { return !hints_str.empty(); } void setContext(ContextPtr & ctx) override; + void setMaxRowsAndBytesToRead(size_t max_rows, size_t max_bytes) override + { + max_rows_to_read = max_rows; + max_bytes_to_read = max_bytes; + } + + size_t getNumRowsRead() const override { return total_rows_read; } + private: NamesAndTypesList readSchema() override; - /// Read whole column in the block (up to max_rows_to_read rows) and extract the data type. - DataTypePtr readColumnAndGetDataType(const String & column_name, size_t & rows_read, size_t max_rows_to_read); + /// Read whole column in the block (up to max_rows rows) and extract the data type. + DataTypePtr readColumnAndGetDataType(const String & column_name, size_t & rows_read, size_t max_rows); const FormatSettings format_settings; String hints_str; @@ -95,6 +103,10 @@ private: std::unique_ptr reader; Names column_names_from_settings; JSONInferenceInfo inference_info; + + size_t total_rows_read = 0; + size_t max_rows_to_read; + size_t max_bytes_to_read; }; } diff --git a/src/Processors/Formats/Impl/JSONCompactColumnsBlockInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactColumnsBlockInputFormat.cpp index ade18d21892..09df7beaa73 100644 --- a/src/Processors/Formats/Impl/JSONCompactColumnsBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactColumnsBlockInputFormat.cpp @@ -53,7 +53,7 @@ void registerJSONCompactColumnsSchemaReader(FormatFactory & factory) ); factory.registerAdditionalInfoForSchemaCacheGetter("JSONCompactColumns", [](const FormatSettings & settings) { - auto result = getAdditionalFormatInfoByEscapingRule(settings, FormatSettings::EscapingRule::JSON); + auto result = getAdditionalFormatInfoForAllRowBasedFormats(settings) + getAdditionalFormatInfoByEscapingRule(settings, FormatSettings::EscapingRule::JSON); return result + fmt::format(", column_names_for_schema_inference={}", settings.column_names_for_schema_inference); }); } diff --git a/tests/queries/0_stateless/02783_max_bytes_to_read_in_schema_inference.reference b/tests/queries/0_stateless/02783_max_bytes_to_read_in_schema_inference.reference new file mode 100644 index 00000000000..d45098ddc0f --- /dev/null +++ b/tests/queries/0_stateless/02783_max_bytes_to_read_in_schema_inference.reference @@ -0,0 +1 @@ +a Nullable(Int64) diff --git a/tests/queries/0_stateless/02783_max_bytes_to_read_in_schema_inference.sql b/tests/queries/0_stateless/02783_max_bytes_to_read_in_schema_inference.sql new file mode 100644 index 00000000000..9dbf176472d --- /dev/null +++ b/tests/queries/0_stateless/02783_max_bytes_to_read_in_schema_inference.sql @@ -0,0 +1,4 @@ +set input_format_max_rows_to_read_for_schema_inference=2; +desc format('JSONEachRow', '{"a" : null}, {"a" : 42}') settings input_format_max_bytes_to_read_for_schema_inference=10; -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +desc format('JSONEachRow', '{"a" : null}, {"a" : 42}') settings input_format_max_bytes_to_read_for_schema_inference=20; + From 79cbebaf0dcc98b947a78dcfa490493cf18f076b Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 5 Jun 2023 15:34:04 +0000 Subject: [PATCH 1199/2223] Remove unnecessary conditional expression --- src/Common/TransformEndianness.hpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 228490d24a1..4d690d75d9e 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -31,8 +31,7 @@ template requires is_decimal inline void transformEndianness(T & x) { - if constexpr (std::endian::native != endian) - transformEndianness(x.value); + transformEndianness(x.value); } template From 0832cb2d7a3bdd2c5e4721c8d8d71a14ce4485ee Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Jun 2023 17:51:12 +0200 Subject: [PATCH 1200/2223] Update Metadata.h --- src/Interpreters/Cache/Metadata.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 4732123fabc..64f91595822 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -22,7 +22,7 @@ struct FileSegmentMetadata : private boost::noncopyable size_t size() const; - bool evicting() const { return !removal_candidate.load(); } + bool evicting() const { return removal_candidate.load(); } Priority::Iterator getQueueIterator() const { return file_segment->getQueueIterator(); } From 654aee209f616aeef350e39cc3c3909862fa14e2 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 5 Jun 2023 11:55:04 -0400 Subject: [PATCH 1201/2223] add video --- .../mergetree-family/invertedindexes.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index 31f5a87a2b6..db3d6d0a479 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -15,6 +15,18 @@ tokenized cells of the string column. For example, the string cell "I will be a " wi", "wil", "ill", "ll ", "l b", " be" etc. The more fine-granular the input strings are tokenized, the bigger but also the more useful the resulting inverted index will be. +
+ +
+ :::note Inverted indexes are experimental and should not be used in production environments yet. They may change in the future in backward-incompatible ways, for example with respect to their DDL/DQL syntax or performance/compression characteristics. From 67af505ed63fc49d253f67f75b814dcf551e3a2c Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Jun 2023 17:04:55 +0000 Subject: [PATCH 1202/2223] Respect setting input_format_as_default in schema inference --- src/Processors/Formats/ISchemaReader.cpp | 6 ++++++ .../02784_schema_inference_null_as_default.reference | 9 +++++++++ .../02784_schema_inference_null_as_default.sql | 7 +++++++ 3 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02784_schema_inference_null_as_default.reference create mode 100644 tests/queries/0_stateless/02784_schema_inference_null_as_default.sql diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index c96cb373a2d..0cb6499f423 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -54,6 +55,11 @@ void checkFinalInferredType( if (settings.schema_inference_make_columns_nullable) type = makeNullableRecursively(type); + /// In case when data for some column could contain nulls and regular values, + /// resulting inferred type is Nullable. + /// If input_format_null_as_default is enabled, we should remove Nullable type. + else if (settings.null_as_default) + type = removeNullable(type); } IIRowSchemaReader::IIRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, DataTypePtr default_type_) diff --git a/tests/queries/0_stateless/02784_schema_inference_null_as_default.reference b/tests/queries/0_stateless/02784_schema_inference_null_as_default.reference new file mode 100644 index 00000000000..c83819ab2e2 --- /dev/null +++ b/tests/queries/0_stateless/02784_schema_inference_null_as_default.reference @@ -0,0 +1,9 @@ +x Nullable(Int64) +\N +42 +x Nullable(Int64) +\N +42 +x Int64 +0 +42 diff --git a/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql b/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql new file mode 100644 index 00000000000..9c9f99d8283 --- /dev/null +++ b/tests/queries/0_stateless/02784_schema_inference_null_as_default.sql @@ -0,0 +1,7 @@ +desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=1; +select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=1; +desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=0; +select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=0; +desc format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=1; +select * from format(JSONEachRow, '{"x" : null}, {"x" : 42}') settings schema_inference_make_columns_nullable=0, input_format_null_as_default=1; + From 028e48dfa716c7e7aa9f5e3df56adb563d653b02 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Jun 2023 17:33:10 +0000 Subject: [PATCH 1203/2223] Update docs --- docs/en/interfaces/schema-inference.md | 24 ++++++++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index c448d0aee47..bef858eaba0 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -1192,7 +1192,7 @@ DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : nul #### schema_inference_make_columns_nullable Controls making inferred types `Nullable` in schema inference for formats without information about nullability. -If the setting is enabled, all inferred type will be `Nullable`, if disabled, the inferred type will be `Nullable` only if the column contains `NULL` in a sample that is parsed during schema inference. +If the setting is enabled, all inferred type will be `Nullable`, if disabled, the inferred type will be `Nullable` only if `input_format_null_as_default` is disabled and the column contains `NULL` in a sample that is parsed during schema inference. Enabled by default. @@ -1215,7 +1215,8 @@ DESC format(JSONEachRow, $$ └─────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` ```sql -SET schema_inference_make_columns_nullable = 0 +SET schema_inference_make_columns_nullable = 0; +SET input_format_null_as_default = 0; DESC format(JSONEachRow, $$ {"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]} {"id" : 2, "age" : 19, "name" : "Alan", "status" : "married", "hobbies" : ["tennis", "art"]} @@ -1232,6 +1233,25 @@ DESC format(JSONEachRow, $$ └─────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` +```sql +SET schema_inference_make_columns_nullable = 0; +SET input_format_null_as_default = 1; +DESC format(JSONEachRow, $$ + {"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]} + {"id" : 2, "age" : 19, "name" : "Alan", "status" : "married", "hobbies" : ["tennis", "art"]} + $$) +``` +```response + +┌─name────┬─type──────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐ +│ id │ Int64 │ │ │ │ │ │ +│ age │ Int64 │ │ │ │ │ │ +│ name │ String │ │ │ │ │ │ +│ status │ String │ │ │ │ │ │ +│ hobbies │ Array(String) │ │ │ │ │ │ +└─────────┴───────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +``` + #### input_format_try_infer_integers If enabled, ClickHouse will try to infer integers instead of floats in schema inference for text formats. From 35439a8b06501460fe9162e09eae0fa9b334d1a1 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 5 Jun 2023 10:47:52 -0700 Subject: [PATCH 1204/2223] Use reverse iterator for little-endian version --- src/IO/ReadHelpers.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 99b3e4b514b..1bd67e240c9 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -53,15 +53,18 @@ UUID parseUUID(std::span src) const auto * src_ptr = src.data(); auto * dst = reinterpret_cast(&uuid); const auto size = src.size(); + +#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ + const std::reverse_iterator dst_it(dst + sizeof(UUID)); +#endif if (size == 36) { #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ - const std::reverse_iterator dst_it(dst + sizeof(UUID)); - parseHex<4>(src_ptr, dst + 8); - parseHex<2>(src_ptr + 9, dst + 12); - parseHex<2>(src_ptr + 14, dst + 14); - parseHex<2>(src_ptr + 19, dst); - parseHex<6>(src_ptr + 24, dst + 2); + parseHex<4>(src_ptr, dst_it + 8); + parseHex<2>(src_ptr + 9, dst_it + 12); + parseHex<2>(src_ptr + 14, dst_it + 14); + parseHex<2>(src_ptr + 19, dst_it); + parseHex<6>(src_ptr + 24, dst_it + 2); #else parseHex<4>(src_ptr, dst); parseHex<2>(src_ptr + 9, dst + 4); @@ -73,8 +76,8 @@ UUID parseUUID(std::span src) else if (size == 32) { #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ - parseHex<8>(src_ptr, dst + 8); - parseHex<8>(src_ptr + 16, dst); + parseHex<8>(src_ptr, dst_it + 8); + parseHex<8>(src_ptr + 16, dst_it); #else parseHex<16>(src_ptr, dst); #endif From ad85faabd1941b891b406225ecf7c6b568c8328f Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 5 Jun 2023 16:09:53 +0000 Subject: [PATCH 1205/2223] Fix test --- tests/integration/test_storage_mongodb/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index e6e77c64515..6ce71fb91fa 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -71,6 +71,7 @@ def test_simple_select(started_cluster): simple_mongo_table.drop() +@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) def test_simple_select_from_view(started_cluster): mongo_connection = get_mongo_connection(started_cluster) db = mongo_connection["test"] @@ -86,7 +87,7 @@ def test_simple_select_from_view(started_cluster): node = started_cluster.instances["node"] node.query( - "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo2:27017', 'test', 'simple_table_view', 'root', 'clickhouse')" + "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table_view', 'root', 'clickhouse')" ) assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" From e8c6c7967b38ebdd467fb9c04966419731b5a689 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Jun 2023 18:21:40 +0000 Subject: [PATCH 1206/2223] Move attaching gdb to separate lib --- docker/test/stateless/run.sh | 41 +++-------------------------------- tests/ci/attach_gdb.lib | 42 ++++++++++++++++++++++++++++++++++++ tests/ci/stress_tests.lib | 41 +++-------------------------------- 3 files changed, 48 insertions(+), 76 deletions(-) create mode 100644 tests/ci/attach_gdb.lib diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index dfee7d84cde..df650b37cc6 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -15,6 +15,8 @@ dpkg -i package_folder/clickhouse-client_*.deb ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test +source /usr/share/clickhouse-test/ci/attach_gdb.lib + # install test configs /usr/share/clickhouse-test/config/install.sh @@ -85,44 +87,7 @@ fi sleep 5 -# Set follow-fork-mode to parent, because we attach to clickhouse-server, not to watchdog -# and clickhouse-server can do fork-exec, for example, to run some bridge. -# Do not set nostop noprint for all signals, because some it may cause gdb to hang, -# explicitly ignore non-fatal signals that are used by server. -# Number of SIGRTMIN can be determined only in runtime. -RTMIN=$(kill -l SIGRTMIN) -echo " -set follow-fork-mode parent -handle SIGHUP nostop noprint pass -handle SIGINT nostop noprint pass -handle SIGQUIT nostop noprint pass -handle SIGPIPE nostop noprint pass -handle SIGTERM nostop noprint pass -handle SIGUSR1 nostop noprint pass -handle SIGUSR2 nostop noprint pass -handle SIG$RTMIN nostop noprint pass -info signals -continue -backtrace full -thread apply all backtrace full -info registers -disassemble /s -up -disassemble /s -up -disassemble /s -p \"done\" -detach -quit -" > script.gdb - -# FIXME Hung check may work incorrectly because of attached gdb -# 1. False positives are possible -# 2. We cannot attach another gdb to get stacktraces if some queries hung -gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log & -sleep 5 -# gdb will send SIGSTOP, spend some time loading debug info and then send SIGCONT, wait for it (up to send_timeout, 300s) -time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||: +attach_gdb_to_clickhouse function run_tests() { diff --git a/tests/ci/attach_gdb.lib b/tests/ci/attach_gdb.lib new file mode 100644 index 00000000000..2df6243f796 --- /dev/null +++ b/tests/ci/attach_gdb.lib @@ -0,0 +1,42 @@ +#!/bin/bash + +function attach_gdb_to_clickhouse() +{ + # Set follow-fork-mode to parent, because we attach to clickhouse-server, not to watchdog + # and clickhouse-server can do fork-exec, for example, to run some bridge. + # Do not set nostop noprint for all signals, because some it may cause gdb to hang, + # explicitly ignore non-fatal signals that are used by server. + # Number of SIGRTMIN can be determined only in runtime. + RTMIN=$(kill -l SIGRTMIN) + echo " +set follow-fork-mode parent +handle SIGHUP nostop noprint pass +handle SIGINT nostop noprint pass +handle SIGQUIT nostop noprint pass +handle SIGPIPE nostop noprint pass +handle SIGTERM nostop noprint pass +handle SIGUSR1 nostop noprint pass +handle SIGUSR2 nostop noprint pass +handle SIG$RTMIN nostop noprint pass +info signals +continue +backtrace full +thread apply all backtrace full +info registers +disassemble /s +up +disassemble /s +up +disassemble /s +p \"done\" +detach +quit +" > script.gdb + + # FIXME Hung check may work incorrectly because of attached gdb + # We cannot attach another gdb to get stacktraces if some queries hung + gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log & + sleep 5 + # gdb will send SIGSTOP, spend some time loading debug info and then send SIGCONT, wait for it (up to send_timeout, 300s) + time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||: +} diff --git a/tests/ci/stress_tests.lib b/tests/ci/stress_tests.lib index 04df50b3248..2b8ac77b952 100644 --- a/tests/ci/stress_tests.lib +++ b/tests/ci/stress_tests.lib @@ -9,6 +9,8 @@ FAIL="\tFAIL\t\\N\t" FAILURE_CONTEXT_LINES=100 FAILURE_CONTEXT_MAX_LINE_WIDTH=300 +source attach_gdb.lib + function escaped() { # That's the simplest way I found to escape a string in bash. Yep, bash is the most convenient programming language. @@ -184,44 +186,7 @@ function start() counter=$((counter + 1)) done - # Set follow-fork-mode to parent, because we attach to clickhouse-server, not to watchdog - # and clickhouse-server can do fork-exec, for example, to run some bridge. - # Do not set nostop noprint for all signals, because some it may cause gdb to hang, - # explicitly ignore non-fatal signals that are used by server. - # Number of SIGRTMIN can be determined only in runtime. - RTMIN=$(kill -l SIGRTMIN) - echo " -set follow-fork-mode parent -handle SIGHUP nostop noprint pass -handle SIGINT nostop noprint pass -handle SIGQUIT nostop noprint pass -handle SIGPIPE nostop noprint pass -handle SIGTERM nostop noprint pass -handle SIGUSR1 nostop noprint pass -handle SIGUSR2 nostop noprint pass -handle SIG$RTMIN nostop noprint pass -info signals -continue -backtrace full -thread apply all backtrace full -info registers -disassemble /s -up -disassemble /s -up -disassemble /s -p \"done\" -detach -quit -" > script.gdb - - # FIXME Hung check may work incorrectly because of attached gdb - # 1. False positives are possible - # 2. We cannot attach another gdb to get stacktraces if some queries hung - gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log & - sleep 5 - # gdb will send SIGSTOP, spend some time loading debug info and then send SIGCONT, wait for it (up to send_timeout, 300s) - time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||: + attach_gdb_to_clickhouse } function check_server_start() From dedb9067ce695bc8324997484aa627722a64ebbd Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 5 Jun 2023 20:36:17 +0200 Subject: [PATCH 1207/2223] WIP : Azure Table Function, added read and StorageAzureSource --- src/Common/ProfileEvents.cpp | 3 + src/Interpreters/ActionsDAG.cpp | 2 + src/Storages/StorageAzure.cpp | 116 ++++++++++++++++- src/Storages/StorageAzure.h | 39 +++++- src/TableFunctions/CMakeLists.txt | 2 +- src/TableFunctions/ITableFunctionCluster.h | 1 + src/TableFunctions/TableFunctionAzure.cpp | 118 ++++++++++++++++++ src/TableFunctions/TableFunctionAzure.h | 72 +++++++++++ src/TableFunctions/registerTableFunctions.cpp | 6 + src/TableFunctions/registerTableFunctions.h | 4 + 10 files changed, 354 insertions(+), 9 deletions(-) create mode 100644 src/TableFunctions/TableFunctionAzure.cpp create mode 100644 src/TableFunctions/TableFunctionAzure.h diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index fdee9902634..3cc41c1972d 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -348,6 +348,9 @@ The server successfully detected this situation and will download merged part fr M(S3PutObject, "Number of S3 API PutObject calls.") \ M(S3GetObject, "Number of S3 API GetObject calls.") \ \ + M(AzureDeleteObjects, "Number of S3 API DeleteObject(s) calls.") \ + M(AzureListObjects, "Number of S3 API ListObjects calls.") \ + \ M(DiskS3DeleteObjects, "Number of DiskS3 API DeleteObject(s) calls.") \ M(DiskS3CopyObject, "Number of DiskS3 API CopyObject calls.") \ M(DiskS3ListObjects, "Number of DiskS3 API ListObjects calls.") \ diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index cbf6cc1cbe3..94bdca60e69 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -598,6 +598,8 @@ Block ActionsDAG::updateHeader(Block header) const } ColumnsWithTypeAndName result_columns; + + result_columns.reserve(outputs.size()); struct Frame diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 30fd3fcbe95..683da3a9825 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -32,6 +33,9 @@ #include #include +#include +#include + using namespace Azure::Storage::Blobs; @@ -52,6 +56,8 @@ bool isConnectionString(const std::string & candidate) StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file) { + LOG_INFO(&Poco::Logger::get("StorageAzure"), "get_format_from_file = {}", get_format_from_file); + StorageAzure::Configuration configuration; /// Supported signatures: @@ -74,6 +80,11 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); + LOG_INFO(&Poco::Logger::get("StorageAzure"), "connection_url = {}", configuration.connection_url); + LOG_INFO(&Poco::Logger::get("StorageAzure"), "container = {}", configuration.container); + LOG_INFO(&Poco::Logger::get("StorageAzure"), "blobpath = {}", configuration.blob_path); + + auto is_format_arg = [] (const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); @@ -81,6 +92,7 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C if (engine_args.size() == 4) { + //'c1 UInt64, c2 UInt64 auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); if (is_format_arg(fourth_arg)) { @@ -143,8 +155,13 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C configuration.blobs_paths = {configuration.blob_path}; - if (configuration.format == "auto" && get_format_from_file) - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); + + LOG_INFO(&Poco::Logger::get("StorageAzure"), "get_format_from_file = {}", get_format_from_file); + +// if (configuration.format == "auto" && get_format_from_file) +// configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); + + configuration.format = "TSV"; return configuration; } @@ -215,6 +232,7 @@ AzureClientPtr StorageAzure::createClient(StorageAzure::Configuration configurat if (configuration.is_connection_string) { + LOG_INFO(&Poco::Logger::get("StorageAzure"), "createClient is_connection_string "); result = std::make_unique(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container)); } else @@ -228,8 +246,14 @@ AzureClientPtr StorageAzure::createClient(StorageAzure::Configuration configurat auto managed_identity_credential = std::make_shared(); result = std::make_unique(configuration.connection_url, managed_identity_credential); + + LOG_INFO(&Poco::Logger::get("StorageAzure"), "createClient account_name & account_key "); } + + + + return result; } @@ -251,15 +275,13 @@ StorageAzure::StorageAzure( , format_settings(format_settings_) , partition_by(partition_by_) { - FormatFactory::instance().checkFormatName(configuration.format); +// FormatFactory::instance().checkFormatName(configuration.format); context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(configuration.getConnectionURL())); StorageInMemoryMetadata storage_metadata; if (columns_.empty()) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Schema inference is not supported yet"); - //auto columns = getTableStructureFromDataImpl(configuration, format_settings, context_); - //storage_metadata.setColumns(columns); } else storage_metadata.setColumns(columns_); @@ -268,11 +290,28 @@ StorageAzure::StorageAzure( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + StoredObjects objects; + for (const auto & key : configuration.blobs_paths) + objects.emplace_back(key); + + for (auto obj : objects) + { + LOG_INFO(&Poco::Logger::get("StorageAzure"), "constructor obj.remote_paths = {}", obj.remote_path); + if (object_storage->exists(obj)) + { + LOG_INFO(&Poco::Logger::get("StorageAzure"), "constructor exists obj.remote_paths = {}", obj.remote_path); +// auto read_buffer = object_storage->readObject(obj); +// LOG_INFO(&Poco::Logger::get("StorageAzure"), "constructor read size obj.remote_paths = {} , size = {}", obj.remote_path, read_buffer->getFileSize()); + } + } + + auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, {"_file", std::make_shared(std::make_shared())}}; auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList(); + virtual_columns = getVirtualsForStorage(columns, default_virtuals); for (const auto & column : virtual_columns) virtual_block.insert({column.type->createColumn(), column.type, column.name}); @@ -435,6 +474,35 @@ private: } + +Pipe StorageAzure::read( + const Names & /*column_names*/ , + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /*query_info*/, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t /*num_streams*/) +{ + Pipes pipes; + + StoredObjects objects; + for (const auto & key : configuration.blobs_paths) + objects.emplace_back(key); + + auto reader = object_storage->readObjects(objects); + auto block_for_format = storage_snapshot->metadata->getSampleBlock(); + + for (auto col : block_for_format.getColumns()) + LOG_INFO(&Poco::Logger::get("StorageAzure"), "read col = {}",col->getName()); + + + pipes.emplace_back(std::make_shared(std::move(reader), context, block_for_format, max_block_size)); + + + return Pipe::unitePipes(std::move(pipes)); +} + SinkToStoragePtr StorageAzure::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { auto sample_block = metadata_snapshot->getSampleBlock(); @@ -513,6 +581,44 @@ bool StorageAzure::supportsPartitionBy() const return true; } + +StorageAzureSource::StorageAzureSource (std::unique_ptr && read_buffer_, ContextPtr context_, + const Block & sample_block_,UInt64 max_block_size_) + :ISource(Block()) + , WithContext(context_) + , read_buffer(std::move(read_buffer_)) + , sample_block(sample_block_) + , max_block_size(max_block_size_) +{ + auto format = "TSV"; + + auto input_format = FormatFactory::instance().getInput( + format, *read_buffer, sample_block, getContext(), max_block_size); + + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); + + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + reader = std::make_unique(*pipeline); +} + + +Chunk StorageAzureSource::generate() +{ + Chunk chunk; + if (reader->pull(chunk)) + { + LOG_INFO(&Poco::Logger::get("StorageAzureSource"), "pulled chunk rows = {}", chunk.getNumRows()); + + } + return chunk; +} + +String StorageAzureSource::getName() const +{ + return "StorageAzureSource"; +} + } #endif diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index b99df2e89a5..b93501ce2f2 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -48,10 +48,9 @@ public: std::string getConnectionURL() const { - if (!is_connection_string) +// if (!is_connection_string) return connection_url; - - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Connection string not implemented yet"); + //throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Connection string not implemented yet"); } std::string connection_url; @@ -78,6 +77,11 @@ public: static StorageAzure::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file = true); static AzureClientPtr createClient(StorageAzure::Configuration configuration); + static AzureObjectStorage::SettingsPtr createSettings(StorageAzure::Configuration configuration); + static ColumnsDescription getTableStructureFromData( + const StorageAzure::Configuration & configuration, + const std::optional & format_settings, + ContextPtr ctx); String getName() const override { @@ -114,6 +118,35 @@ private: std::optional format_settings; ASTPtr partition_by; + static ColumnsDescription getTableStructureFromDataImpl( + const Configuration & configuration, + const std::optional & format_settings, + ContextPtr ctx); + +}; + +class StorageAzureSource : public ISource, WithContext +{ +public: + StorageAzureSource (std::unique_ptr && read_buffer_, ContextPtr context_, const Block & sample_block_, UInt64 max_block_size_); + ~StorageAzureSource() override {} + + Chunk generate() override; + String getName() const override; + + +private: +// std::unique_ptr read_buffer; + + String path; + std::unique_ptr read_buffer; +// std::unique_ptr read_buf; + std::unique_ptr pipeline; + std::unique_ptr reader; + Block sample_block; + UInt64 max_block_size; + +// void createReader(); }; } diff --git a/src/TableFunctions/CMakeLists.txt b/src/TableFunctions/CMakeLists.txt index b1fa61a72ee..3544c5bf8b4 100644 --- a/src/TableFunctions/CMakeLists.txt +++ b/src/TableFunctions/CMakeLists.txt @@ -17,5 +17,5 @@ add_library(clickhouse_table_functions ${clickhouse_table_functions_sources}) target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms) if (TARGET ch_contrib::hivemetastore) - target_link_libraries(clickhouse_table_functions PRIVATE ch_contrib::hivemetastore ch_contrib::hdfs ch_contrib::parquet) + target_link_libraries(clickhouse_table_functions PRIVATE ch_contrib::hivemetastore ch_contrib::hdfs ch_contrib::parquet ch_contrib::azure_sdk) endif () diff --git a/src/TableFunctions/ITableFunctionCluster.h b/src/TableFunctions/ITableFunctionCluster.h index ad88d7b54f0..f68558596ca 100644 --- a/src/TableFunctions/ITableFunctionCluster.h +++ b/src/TableFunctions/ITableFunctionCluster.h @@ -4,6 +4,7 @@ #include #include +#include #include #include #include diff --git a/src/TableFunctions/TableFunctionAzure.cpp b/src/TableFunctions/TableFunctionAzure.cpp new file mode 100644 index 00000000000..eb67ed9a983 --- /dev/null +++ b/src/TableFunctions/TableFunctionAzure.cpp @@ -0,0 +1,118 @@ +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +//#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "registerTableFunctions.h" +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; +} + + +void TableFunctionAzure::parseArgumentsImpl(ASTs & args, const ContextPtr & context) +{ + if (args.size() != 5) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + configuration.connection_url = checkAndGetLiteralArgument(args[0], "connection_url"); + configuration.container = checkAndGetLiteralArgument(args[1], "container"); + configuration.blob_path = checkAndGetLiteralArgument(args[2], "blob_path"); + configuration.format = checkAndGetLiteralArgument(args[3], "format"); + configuration.structure = checkAndGetLiteralArgument(args[4], "structure"); +} + +void TableFunctionAzure::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + LOG_INFO(&Poco::Logger::get("TableFunctionAzure"), "parseArguments = {}", ast_function->dumpTree()); + + ASTs & args_func = ast_function->children; + + if (args_func.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); + + auto & args = args_func.at(0)->children; + + parseArgumentsImpl(args, context); +} + +ColumnsDescription TableFunctionAzure::getActualTableStructure(ContextPtr context) const +{ + return parseColumnsListFromString(configuration.structure, context); +} + +bool TableFunctionAzure::supportsReadingSubsetOfColumns() +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); +} + +StoragePtr TableFunctionAzure::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +{ + LOG_INFO(&Poco::Logger::get("TableFunctionAzure"), "executeImpl = {}", table_name); + + ColumnsDescription columns; + columns = parseColumnsListFromString(configuration.structure, context); + + configuration.is_connection_string = true; + configuration.blobs_paths = {configuration.blob_path}; + + auto client = StorageAzure::createClient(configuration); + + StoragePtr storage = std::make_shared( + configuration, + std::make_unique(table_name, std::move(client), std::make_unique()), + context, + StorageID(getDatabaseName(), table_name), + columns, + ConstraintsDescription{}, + String{}, + /// No format_settings for table function Azure + std::nullopt, nullptr); + + storage->startup(); + + return storage; +} + +void registerTableFunctionAzure(TableFunctionFactory & factory) +{ + factory.registerFunction( + {.documentation + = {.description=R"(The table function can be used to read the data stored on Azure Blob Storage.)", + .examples{{"azure_blob", "SELECT * FROM azure_blob(connection, container, blob_path, format, structure)", ""}}}, + .allow_readonly = false}); +} + +} + +#endif diff --git a/src/TableFunctions/TableFunctionAzure.h b/src/TableFunctions/TableFunctionAzure.h new file mode 100644 index 00000000000..a6fb5415113 --- /dev/null +++ b/src/TableFunctions/TableFunctionAzure.h @@ -0,0 +1,72 @@ +#pragma once + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include + + +namespace DB +{ + +class Context; + +/* AzureBlob(source, [access_key_id, secret_access_key,] [format, structure, compression]) - creates a temporary storage for a file in AzureBlob. + */ +class TableFunctionAzure : public ITableFunction +{ +public: + static constexpr auto name = "azure_blob"; + static constexpr auto signature = "- connection_url, container, blob, format, structure\n"; + + static size_t getMaxNumberOfArguments() { return 5; } + + 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"; } + + void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } + + bool supportsReadingSubsetOfColumns() override; + + std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override + { + return {"_path", "_file"}; + } + + virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); + + static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context); + +protected: + + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns) const override; + + const char * getStorageTypeName() const override { return "Azure"; } + + ColumnsDescription getActualTableStructure(ContextPtr context) const override; + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + mutable StorageAzure::Configuration configuration; + ColumnsDescription structure_hint; +}; + +} + +#endif diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 4f3411df4c5..e0114368e44 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -71,6 +71,12 @@ void registerTableFunctions() registerTableFunctionFormat(factory); registerTableFunctionExplain(factory); + +#if USE_AZURE_BLOB_STORAGE + registerTableFunctionAzure(factory); +#endif + + } } diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index c51522a5e99..fa4fec2b03a 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -69,6 +69,10 @@ void registerTableFunctionFormat(TableFunctionFactory & factory); void registerTableFunctionExplain(TableFunctionFactory & factory); +#if USE_AZURE_BLOB_STORAGE +void registerTableFunctionAzure(TableFunctionFactory & factory); +#endif + void registerTableFunctions(); } From 59095c445d7a48ae12ac48c8748d4b48699a1274 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 5 Jun 2023 20:44:20 +0200 Subject: [PATCH 1208/2223] Revert "Revert "make filter push down through cross join"" --- .../Optimizations/filterPushDown.cpp | 6 +++--- .../01763_filter_push_down_bugs.reference | 19 +++++++++++++++++++ .../01763_filter_push_down_bugs.sql | 19 +++++++++++++++++++ 3 files changed, 41 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 37bc894339f..db29038999b 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -272,7 +272,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { /// If totals step has HAVING expression, skip it for now. /// TODO: - /// We can merge HAVING expression with current filer. + /// We can merge HAVING expression with current filter. /// Also, we can push down part of HAVING which depend only on aggregation keys. if (totals_having->getActions()) return 0; @@ -323,9 +323,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin(); - /// Only inner and left(/right) join are supported. Other types may generate default values for left table keys. + /// Only inner, cross and left(/right) join are supported. Other types may generate default values for left table keys. /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (table_join.kind() != JoinKind::Inner && table_join.kind() != kind) + if (table_join.kind() != JoinKind::Inner && table_join.kind() != JoinKind::Cross && table_join.kind() != kind) return 0; bool is_left = kind == JoinKind::Left; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 5aa2e645509..7df35e2948d 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -6,3 +6,22 @@ String1_0 String2_0 String3_0 String4_0 1 String1_0 String2_0 String3_0 String4_0 1 1 [0,1,2] 1 +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + Join (JOIN FillRightFirst) + Filter (( + Before JOIN)) + ReadFromMergeTree (default.t1) + Indexes: + PrimaryKey + Keys: + id + Condition: (id in [101, 101]) + Parts: 1/1 + Granules: 1/1 + Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) + ReadFromMergeTree (default.t2) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 1058bf75144..2ee249b5ce7 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -38,6 +38,25 @@ DROP TABLE IF EXISTS Test; select x, y from (select [0, 1, 2] as y, 1 as a, 2 as b) array join y as x where a = 1 and b = 2 and (x = 1 or x != 1) and x = 1; +DROP TABLE IF EXISTS t; create table t(a UInt8) engine=MergeTree order by a; insert into t select * from numbers(2); select a from t t1 join t t2 on t1.a = t2.a where t1.a; +DROP TABLE IF EXISTS t; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id; +CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time; + +insert into t1 values (101, '2023-05-28 00:00:00'), (102, '2023-05-28 00:00:00'); +insert into t2 values ('2023-05-31 00:00:00'); + +EXPLAIN indexes=1 SELECT id, delete_time FROM t1 + CROSS JOIN ( + SELECT delete_time + FROM t2 +) AS d WHERE create_time < delete_time AND id = 101; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; From 4da37a731962ab452ef6a7fb75025f0a6f4e2a51 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:19:50 -0300 Subject: [PATCH 1209/2223] Update argmax.md --- .../aggregate-functions/reference/argmax.md | 60 ++++++++++++++++++- 1 file changed, 59 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 65c43ab04c0..9aaa35dc6d8 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -5,7 +5,8 @@ sidebar_position: 106 # argMax -Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. +Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. +Both parts the `arg` and the `max` behave as aggregate functions, they skip `Null` during processing and return not-Null values if not-Null values are available. **Syntax** @@ -49,3 +50,60 @@ Result: │ director │ └──────────────────────┘ ``` + +**Extended example** + +```sql +CREATE TABLE test +( + a Nullable(String), + b Nullable(Int64) +) +ENGINE = Memory AS +SELECT * +FROM values(('a', 1), ('b', 2), ('c', 2), (NULL, 3), (NULL, NULL), ('d', NULL)); + +select * from test; +┌─a────┬────b─┐ +│ a │ 1 │ +│ b │ 2 │ +│ c │ 2 │ +│ ᴺᵁᴸᴸ │ 3 │ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +│ d │ ᴺᵁᴸᴸ │ +└──────┴──────┘ + +select argMax(a, b), max(b) from test; +┌─argMax(a, b)─┬─max(b)─┐ +│ b │ 3 │ -- argMax = b because it the first not-Null value, max(b) is from another row! +└──────────────┴────────┘ + +select argMax(tuple(a), b) from test; +┌─argMax(tuple(a), b)─┐ +│ (NULL) │ -- Tuple allows to get Null value. +└─────────────────────┘ + +select (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB from test; +┌─argMaxA─┬─argMaxB─┐ +│ ᴺᵁᴸᴸ │ 3 │ -- you can use Tuple and get both (all - tuple(*) ) columns for the according max(b) +└─────────┴─────────┘ + +select argMax(a, b), max(b) from test where a is Null and b is Null; +┌─argMax(a, b)─┬─max(b)─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- Nulls are not skipped because only Null values are available +└──────────────┴────────┘ + +select argMax(a, (b,a)) from test; +┌─argMax(a, tuple(b, a))─┐ +│ c │ -- There are two rows with b=2, Tuple in the `Max` allows to get not the first `arg` +└────────────────────────┘ + +select argMax(a, tuple(b)) from test; +┌─argMax(a, tuple(b))─┐ +│ b │ -- Tuple can be used `Max` to not skip Nulls in `Max` +└─────────────────────┘ +``` + +**See also** + +- [Tuple](../../sql-reference/data-types/tuple.md) From 6741a6d7c81ab9a17042170c182327295a6de356 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:21:09 -0300 Subject: [PATCH 1210/2223] Update argmax.md --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 9aaa35dc6d8..7800e90eec7 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -106,4 +106,4 @@ select argMax(a, tuple(b)) from test; **See also** -- [Tuple](../../sql-reference/data-types/tuple.md) +- [Tuple](../../data-types/tuple.md) From 113ce8c7574f3e8348cf3ba7981e9a837460ff8b Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:43:07 -0300 Subject: [PATCH 1211/2223] Update argmin.md --- .../aggregate-functions/reference/argmin.md | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index a7c21e3f15b..7972bdf84b8 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -6,6 +6,7 @@ sidebar_position: 105 # argMin Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered. +Both parts the `arg` and the `min` behave as [aggregate functions](../aggregate-functions/index.md), they both [skip `Null`](../aggregate-functions/index.md#null-processing) during processing and return not-Null values if not-Null values are available. **Syntax** @@ -49,3 +50,65 @@ Result: │ worker │ └──────────────────────┘ ``` + +**Extended example** + +```sql +CREATE TABLE test +( + a Nullable(String), + b Nullable(Int64) +) +ENGINE = Memory AS +SELECT * +FROM values((NULL, 0), ('a', 1), ('b', 2), ('c', 2), (NULL, NULL), ('d', NULL)); + +select * from test; +┌─a────┬────b─┐ +│ ᴺᵁᴸᴸ │ 0 │ +│ a │ 1 │ +│ b │ 2 │ +│ c │ 2 │ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +│ d │ ᴺᵁᴸᴸ │ +└──────┴──────┘ + +select argMin(a, b), min(b) from test; +┌─argMin(a, b)─┬─min(b)─┐ +│ a │ 0 │ -- argMin = a because it the first not-Null value, min(b) is from another row! +└──────────────┴────────┘ + +select argMin(tuple(a), b) from test; +┌─argMin(tuple(a), b)─┐ +│ (NULL) │ -- Tuple allows to get Null value. +└─────────────────────┘ + +select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; +┌─argMinA─┬─argMinB─┐ +│ ᴺᵁᴸᴸ │ 0 │ -- you can use Tuple and get both (all - tuple(*) ) columns for the according max(b) +└─────────┴─────────┘ + +select argMin(a, b), min(b) from test where a is Null and b is Null; +┌─argMin(a, b)─┬─min(b)─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- Nulls are not skipped because only Null values are available +└──────────────┴────────┘ + +select argMin(a, (b, a)), min(tuple(b, a)) from test; +┌─argMin(a, tuple(b, a))─┬─min(tuple(b, a))─┐ +│ d │ (NULL,NULL) │ 'd' is the first Not null value for the min +└────────────────────────┴──────────────────┘ + +select argMin((a, b), (b, a)), min(tuple(b, a)) from test; +┌─argMin(tuple(a, b), tuple(b, a))─┬─min(tuple(b, a))─┐ +│ (NULL,NULL) │ (NULL,NULL) │ +└──────────────────────────────────┴──────────────────┘ + +select argMin(a, tuple(b)) from test; +┌─argMax(a, tuple(b))─┐ +│ b │ -- Tuple can be used in `Min` to not skip Nulls in `Min` +└─────────────────────┘ +``` + +**See also** + +- [Tuple](../../data-types/tuple.md) From 40986539775f5d74659521000fe4aa05ca47b06b Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:43:47 -0300 Subject: [PATCH 1212/2223] Update argmin.md --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 7972bdf84b8..a481157784b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -6,7 +6,7 @@ sidebar_position: 105 # argMin Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `min` behave as [aggregate functions](../aggregate-functions/index.md), they both [skip `Null`](../aggregate-functions/index.md#null-processing) during processing and return not-Null values if not-Null values are available. +Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not-Null values if not-Null values are available. **Syntax** From a175e4628e640376bc49ceabcebb278159e6d55f Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:44:42 -0300 Subject: [PATCH 1213/2223] Update argmax.md --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 7800e90eec7..a736804c8dc 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -6,7 +6,7 @@ sidebar_position: 106 # argMax Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `max` behave as aggregate functions, they skip `Null` during processing and return not-Null values if not-Null values are available. +Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not-Null values if not-Null values are available. **Syntax** @@ -100,7 +100,7 @@ select argMax(a, (b,a)) from test; select argMax(a, tuple(b)) from test; ┌─argMax(a, tuple(b))─┐ -│ b │ -- Tuple can be used `Max` to not skip Nulls in `Max` +│ b │ -- Tuple can be used in `Max` to not skip Nulls in `Max` └─────────────────────┘ ``` From 3e444790af079c3d486ac943b29d4900f4d0576f Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:52:51 -0300 Subject: [PATCH 1214/2223] Update index.md --- docs/en/sql-reference/aggregate-functions/index.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 8951ac4ee6a..019e1cab873 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -72,3 +72,15 @@ FROM t_null_big │ 2.3333333333333335 │ 1.4 │ └────────────────────┴─────────────────────┘ ``` + +Also you can use [Tuple](../data-types/tuple.md) to change NULL skipping behavior. + +```sql +select groupArray(b), groupArray(tuple(b)) from t_null_big; +┌─groupArray(b)─┬─groupArray(tuple(b))────────┐ +│ [2,2,3] │ [(2),(NULL),(2),(3),(NULL)] │ +└───────────────┴─────────────────────────────┘ +``` + + + From b4c0d68d0f7a17702d074a9a2f216bbda524c94d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:53:37 -0300 Subject: [PATCH 1215/2223] Update index.md --- docs/en/sql-reference/aggregate-functions/index.md | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 019e1cab873..a3808335168 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -76,10 +76,11 @@ FROM t_null_big Also you can use [Tuple](../data-types/tuple.md) to change NULL skipping behavior. ```sql -select groupArray(b), groupArray(tuple(b)) from t_null_big; -┌─groupArray(b)─┬─groupArray(tuple(b))────────┐ -│ [2,2,3] │ [(2),(NULL),(2),(3),(NULL)] │ -└───────────────┴─────────────────────────────┘ +select groupArray(b), groupArray(tuple(b)).1 from t_null_big; + +┌─groupArray(b)─┬─tupleElement(groupArray(tuple(b)), 1)─┐ +│ [2,2,3] │ [2,NULL,2,3,NULL] │ +└───────────────┴───────────────────────────────────────┘ ``` From e37cd36db7316c5d2a90df8ca8d8bee8fa016e4a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:57:28 -0300 Subject: [PATCH 1216/2223] Update argmin.md --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index a481157784b..067c81f56cf 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -85,7 +85,7 @@ select argMin(tuple(a), b) from test; select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; ┌─argMinA─┬─argMinB─┐ -│ ᴺᵁᴸᴸ │ 0 │ -- you can use Tuple and get both (all - tuple(*) ) columns for the according max(b) +│ ᴺᵁᴸᴸ │ 0 │ -- you can use Tuple and get both (all - tuple(*)) columns for the according max(b) └─────────┴─────────┘ select argMin(a, b), min(b) from test where a is Null and b is Null; @@ -95,7 +95,7 @@ select argMin(a, b), min(b) from test where a is Null and b is Null; select argMin(a, (b, a)), min(tuple(b, a)) from test; ┌─argMin(a, tuple(b, a))─┬─min(tuple(b, a))─┐ -│ d │ (NULL,NULL) │ 'd' is the first Not null value for the min +│ d │ (NULL,NULL) │ -- 'd' is the first Not null value for the min └────────────────────────┴──────────────────┘ select argMin((a, b), (b, a)), min(tuple(b, a)) from test; From 15fcad190933aa7b885bdeacb9b2f277104433ac Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:57:42 -0300 Subject: [PATCH 1217/2223] Update argmax.md --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index a736804c8dc..8a84f361589 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -85,7 +85,7 @@ select argMax(tuple(a), b) from test; select (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB from test; ┌─argMaxA─┬─argMaxB─┐ -│ ᴺᵁᴸᴸ │ 3 │ -- you can use Tuple and get both (all - tuple(*) ) columns for the according max(b) +│ ᴺᵁᴸᴸ │ 3 │ -- you can use Tuple and get both (all - tuple(*)) columns for the according max(b) └─────────┴─────────┘ select argMax(a, b), max(b) from test where a is Null and b is Null; From 3023eb73d02fe7825626a1b5767827432f1513c1 Mon Sep 17 00:00:00 2001 From: Misz606 <113922942+Misz606@users.noreply.github.com> Date: Mon, 5 Jun 2023 21:02:27 +0100 Subject: [PATCH 1218/2223] Update aggregatingmergetree.md Grammatical update in docs --- .../table-engines/mergetree-family/aggregatingmergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index 2b8b43802ea..62191d9b5e4 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -109,7 +109,7 @@ INSERT INTO test.visits (StartDate, CounterID, Sign, UserID) VALUES (1667446031, 1, 6, 3) ``` -The data are inserted in both the table and the materialized view `test.mv_visits`. +The data is inserted in both the table and the materialized view `test.mv_visits`. To get the aggregated data, we need to execute a query such as `SELECT ... GROUP BY ...` from the materialized view `test.mv_visits`: From fd39616e780ac216fd07aa2625119a65c85661ee Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 5 Jun 2023 22:09:32 +0200 Subject: [PATCH 1219/2223] suppress some tests for analyzer --- tests/queries/0_stateless/01479_cross_join_9855.sql | 4 ++-- tests/queries/0_stateless/01763_filter_push_down_bugs.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01479_cross_join_9855.sql b/tests/queries/0_stateless/01479_cross_join_9855.sql index 6dc76f22057..9dcf209a1cd 100644 --- a/tests/queries/0_stateless/01479_cross_join_9855.sql +++ b/tests/queries/0_stateless/01479_cross_join_9855.sql @@ -2,8 +2,8 @@ SET cross_to_inner_join_rewrite = 1; SELECT count() FROM numbers(4) AS n1, numbers(3) AS n2 -WHERE n1.number > (select avg(n.number) from numbers(3) n); +WHERE n1.number > (select avg(n.number) from numbers(3) n) SETTINGS allow_experimental_analyzer=0; SELECT count() FROM numbers(4) AS n1, numbers(3) AS n2, numbers(6) AS n3 -WHERE n1.number > (select avg(n.number) from numbers(3) n); +WHERE n1.number > (select avg(n.number) from numbers(3) n) SETTINGS allow_experimental_analyzer=0; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 2ee249b5ce7..5f7f4379714 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -56,7 +56,7 @@ EXPLAIN indexes=1 SELECT id, delete_time FROM t1 CROSS JOIN ( SELECT delete_time FROM t2 -) AS d WHERE create_time < delete_time AND id = 101; +) AS d WHERE create_time < delete_time AND id = 101 SETTINGS allow_experimental_analyzer=0; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From aa6f4e43c5ea0fa4d6c84f121eba766d89a6efdf Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 5 Jun 2023 23:15:13 +0200 Subject: [PATCH 1220/2223] Fixed COLUMN_NOT_FOUND in block issue --- src/Storages/StorageAzure.cpp | 35 ++++++++++++++--------- src/Storages/StorageAzure.h | 3 +- src/Storages/StorageS3.cpp | 3 ++ src/TableFunctions/TableFunctionAzure.cpp | 5 ++-- src/TableFunctions/TableFunctionS3.cpp | 5 ++++ 5 files changed, 34 insertions(+), 17 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 683da3a9825..76e9130bda3 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -476,7 +476,7 @@ private: Pipe StorageAzure::read( - const Names & /*column_names*/ , + const Names & column_names , const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & /*query_info*/, ContextPtr context, @@ -491,13 +491,11 @@ Pipe StorageAzure::read( objects.emplace_back(key); auto reader = object_storage->readObjects(objects); - auto block_for_format = storage_snapshot->metadata->getSampleBlock(); - - for (auto col : block_for_format.getColumns()) - LOG_INFO(&Poco::Logger::get("StorageAzure"), "read col = {}",col->getName()); + auto columns_description = storage_snapshot->getDescriptionForColumns(column_names); + auto block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); - pipes.emplace_back(std::make_shared(std::move(reader), context, block_for_format, max_block_size)); + pipes.emplace_back(std::make_shared(std::move(reader), context, block_for_format, max_block_size, columns_description)); return Pipe::unitePipes(std::move(pipes)); @@ -583,12 +581,13 @@ bool StorageAzure::supportsPartitionBy() const StorageAzureSource::StorageAzureSource (std::unique_ptr && read_buffer_, ContextPtr context_, - const Block & sample_block_,UInt64 max_block_size_) - :ISource(Block()) + const Block & sample_block_,UInt64 max_block_size_, const ColumnsDescription & columns_) + :ISource(sample_block_) , WithContext(context_) , read_buffer(std::move(read_buffer_)) , sample_block(sample_block_) , max_block_size(max_block_size_) + , columns_desc(columns_) { auto format = "TSV"; @@ -598,6 +597,13 @@ StorageAzureSource::StorageAzureSource (std::unique_ptr QueryPipelineBuilder builder; builder.init(Pipe(input_format)); + if (columns_desc.hasDefaults()) + { + builder.addSimpleTransform( + [&](const Block & header) + { return std::make_shared(header, columns_desc, *input_format, getContext()); }); + } + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); reader = std::make_unique(*pipeline); } @@ -605,13 +611,16 @@ StorageAzureSource::StorageAzureSource (std::unique_ptr Chunk StorageAzureSource::generate() { - Chunk chunk; - if (reader->pull(chunk)) + while(true) { - LOG_INFO(&Poco::Logger::get("StorageAzureSource"), "pulled chunk rows = {}", chunk.getNumRows()); - + Chunk chunk; + if (reader->pull(chunk)) + { + LOG_INFO(&Poco::Logger::get("StorageAzureSource"), "pulled chunk rows = {}", chunk.getNumRows()); + } + return chunk; } - return chunk; +// return {}; } String StorageAzureSource::getName() const diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index b93501ce2f2..61237fcc2f9 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -128,7 +128,7 @@ private: class StorageAzureSource : public ISource, WithContext { public: - StorageAzureSource (std::unique_ptr && read_buffer_, ContextPtr context_, const Block & sample_block_, UInt64 max_block_size_); + StorageAzureSource (std::unique_ptr && read_buffer_, ContextPtr context_, const Block & sample_block_, UInt64 max_block_size_, const ColumnsDescription & columns_); ~StorageAzureSource() override {} Chunk generate() override; @@ -145,6 +145,7 @@ private: std::unique_ptr reader; Block sample_block; UInt64 max_block_size; + ColumnsDescription columns_desc; // void createReader(); }; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2d8aaec0f07..7d6254b2551 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -957,6 +957,9 @@ StorageS3::StorageS3( {"_file", std::make_shared(std::make_shared())}}; auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList(); + + LOG_INFO(&Poco::Logger::get("StorageS3"), "constructor columns = {}", columns.toString()); + virtual_columns = getVirtualsForStorage(columns, default_virtuals); for (const auto & column : virtual_columns) virtual_block.insert({column.type->createColumn(), column.type, column.name}); diff --git a/src/TableFunctions/TableFunctionAzure.cpp b/src/TableFunctions/TableFunctionAzure.cpp index eb67ed9a983..f565a365a13 100644 --- a/src/TableFunctions/TableFunctionAzure.cpp +++ b/src/TableFunctions/TableFunctionAzure.cpp @@ -54,7 +54,8 @@ void TableFunctionAzure::parseArgumentsImpl(ASTs & args, const ContextPtr & cont void TableFunctionAzure::parseArguments(const ASTPtr & ast_function, ContextPtr context) { - LOG_INFO(&Poco::Logger::get("TableFunctionAzure"), "parseArguments = {}", ast_function->dumpTree()); + /// Clone ast function, because we can modify its arguments like removing headers. + auto ast_copy = ast_function->clone(); ASTs & args_func = ast_function->children; @@ -78,8 +79,6 @@ bool TableFunctionAzure::supportsReadingSubsetOfColumns() StoragePtr TableFunctionAzure::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - LOG_INFO(&Poco::Logger::get("TableFunctionAzure"), "executeImpl = {}", table_name); - ColumnsDescription columns; columns = parseColumnsListFromString(configuration.structure, context); diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index c8cc0cddd30..7f283afd6b4 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -294,6 +294,8 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String & ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) const { + LOG_INFO(&Poco::Logger::get("TableFunctionS3"), "getActualTableStructure configuration.structure = {} ",configuration.structure); + if (configuration.structure == "auto") { context->checkAccess(getSourceAccessType()); @@ -319,6 +321,9 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context else if (!structure_hint.empty()) columns = structure_hint; + LOG_INFO(&Poco::Logger::get("TableFunctionS3"), "executeImpl structre = {} structure_hint = {} ",configuration.structure, structure_hint.getAll().toString()); + + StoragePtr storage = std::make_shared( configuration, context, From bd6b0ff1c005aaa8f976103762e6848e3b3448d8 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 5 Jun 2023 23:32:26 +0200 Subject: [PATCH 1221/2223] Updated to read only 1st object --- src/Storages/StorageAzure.cpp | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 76e9130bda3..c8d13cbf242 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -490,13 +490,16 @@ Pipe StorageAzure::read( for (const auto & key : configuration.blobs_paths) objects.emplace_back(key); - auto reader = object_storage->readObjects(objects); - auto columns_description = storage_snapshot->getDescriptionForColumns(column_names); - auto block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); + if (objects.size() > 1) + { + auto reader = object_storage->readObject(objects[0]); + auto columns_description = storage_snapshot->getDescriptionForColumns(column_names); + auto block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); - pipes.emplace_back(std::make_shared(std::move(reader), context, block_for_format, max_block_size, columns_description)); - + pipes.emplace_back( + std::make_shared(std::move(reader), context, block_for_format, max_block_size, columns_description)); + } return Pipe::unitePipes(std::move(pipes)); } @@ -592,7 +595,9 @@ StorageAzureSource::StorageAzureSource (std::unique_ptr auto format = "TSV"; auto input_format = FormatFactory::instance().getInput( - format, *read_buffer, sample_block, getContext(), max_block_size); + format, *read_buffer, sample_block, getContext(), max_block_size, + FormatSettings(), std::nullopt, std::nullopt, + true); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -611,6 +616,7 @@ StorageAzureSource::StorageAzureSource (std::unique_ptr Chunk StorageAzureSource::generate() { + LOG_INFO(&Poco::Logger::get("StorageAzureSource"), "generate"); while(true) { Chunk chunk; From 495482cdb2b6a6a2d272c50bb3995b0409f7fb91 Mon Sep 17 00:00:00 2001 From: tpanetti Date: Mon, 5 Jun 2023 15:22:29 -0700 Subject: [PATCH 1222/2223] Refactor ClickHouse->MySQL Type conversion and add configuration setting to trigger type conversion --- src/Core/Settings.h | 1 + src/DataTypes/DataTypeAggregateFunction.h | 2 +- src/DataTypes/DataTypeArray.h | 2 +- src/DataTypes/DataTypeDate.h | 2 +- src/DataTypes/DataTypeDate32.h | 2 +- src/DataTypes/DataTypeDateTime.h | 2 +- src/DataTypes/DataTypeDateTime64.h | 2 +- src/DataTypes/DataTypeEnum.cpp | 1 - src/DataTypes/DataTypeEnum.h | 3 +- src/DataTypes/DataTypeFixedString.h | 3 +- src/DataTypes/DataTypeFunction.h | 2 +- src/DataTypes/DataTypeIPv4andIPv6.h | 4 +- src/DataTypes/DataTypeInterval.h | 2 +- src/DataTypes/DataTypeLowCardinality.cpp | 3 +- src/DataTypes/DataTypeLowCardinality.h | 3 +- src/DataTypes/DataTypeMap.h | 2 +- src/DataTypes/DataTypeNothing.h | 2 +- src/DataTypes/DataTypeNullable.h | 2 +- src/DataTypes/DataTypeNumberBase.cpp | 67 +++++-- src/DataTypes/DataTypeNumberBase.h | 4 +- src/DataTypes/DataTypeObject.h | 2 +- src/DataTypes/DataTypeSet.h | 2 +- src/DataTypes/DataTypeString.h | 3 +- src/DataTypes/DataTypeTuple.h | 2 +- src/DataTypes/DataTypeUUID.h | 2 +- src/DataTypes/DataTypesDecimal.cpp | 5 + src/DataTypes/DataTypesDecimal.h | 3 +- src/DataTypes/IDataType.h | 12 +- src/Storages/System/StorageSystemColumns.cpp | 11 +- ...show_columns_mysql_compatibility.reference | 187 +++++++++++++++--- .../02775_show_columns_mysql_compatibility.sh | 31 ++- 31 files changed, 278 insertions(+), 93 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 67c92a0be8b..1ce30ff121f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -190,6 +190,7 @@ class IColumn; M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ + M(Bool, output_format_mysql_types, false, "Use MySQL converted types when connected via MySQL compatibility", 0) \ \ M(UInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \ \ diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index 13ca3508580..83c9f10f407 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -45,7 +45,7 @@ public: String doGetName() const override; String getNameWithoutVersion() const; const char * getFamilyName() const override { return "AggregateFunction"; } - const char * getSQLCompatibleName() const override { return "TEXT"; } + String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; } Array getParameters() const { return parameters; } diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 528062b60be..2714ca1d023 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -30,7 +30,7 @@ public: { return "Array"; } - const char * getSQLCompatibleName() const override + String getSQLCompatibleName() const override { return "TEXT"; } diff --git a/src/DataTypes/DataTypeDate.h b/src/DataTypes/DataTypeDate.h index 7b622ae04a3..0d557cad5f0 100644 --- a/src/DataTypes/DataTypeDate.h +++ b/src/DataTypes/DataTypeDate.h @@ -13,7 +13,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Date; } const char * getFamilyName() const override { return family_name; } - const char * getSQLCompatibleName() const override { return "DATE"; } + String getSQLCompatibleName() const override { return "DATE"; } bool canBeUsedAsVersion() const override { return true; } bool canBeInsideNullable() const override { return true; } diff --git a/src/DataTypes/DataTypeDate32.h b/src/DataTypes/DataTypeDate32.h index 65b0ec7407e..0879a404179 100644 --- a/src/DataTypes/DataTypeDate32.h +++ b/src/DataTypes/DataTypeDate32.h @@ -13,7 +13,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Date32; } const char * getFamilyName() const override { return family_name; } - const char * getSQLCompatibleName() const override { return "DATE"; } + String getSQLCompatibleName() const override { return "DATE"; } Field getDefault() const override { diff --git a/src/DataTypes/DataTypeDateTime.h b/src/DataTypes/DataTypeDateTime.h index 2facc758f90..edc8b016490 100644 --- a/src/DataTypes/DataTypeDateTime.h +++ b/src/DataTypes/DataTypeDateTime.h @@ -36,7 +36,7 @@ public: static constexpr auto family_name = "DateTime"; const char * getFamilyName() const override { return family_name; } - const char * getSQLCompatibleName() const override { return "DATETIME"; } + String getSQLCompatibleName() const override { return "DATETIME"; } String doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::DateTime; } diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index b836b84918f..e786cc09f28 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -28,7 +28,7 @@ public: DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info); const char * getFamilyName() const override { return family_name; } - const char * getSQLCompatibleName() const override { return "DATETIME"; } + String getSQLCompatibleName() const override { return "DATETIME"; } std::string doGetName() const override; TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index 24a3976179d..1750ae785bf 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -90,7 +90,6 @@ template DataTypeEnum::DataTypeEnum(const Values & values_) : EnumValues(values_) , type_name(generateName(this->getValues())) - , my_sql_type_name(generateMySQLName(this->getValues())) { } diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index 2cdaa2db06c..d148f753c82 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -45,7 +45,6 @@ public: private: std::string type_name; - std::string my_sql_type_name; static std::string generateName(const Values & values); static std::string generateMySQLName(const Values & values); @@ -54,7 +53,7 @@ public: std::string doGetName() const override { return type_name; } const char * getFamilyName() const override; - const char * getSQLCompatibleName() const override { return my_sql_type_name.c_str(); } + String getSQLCompatibleName() const override { return generateMySQLName(this->getValues()); } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeFixedString.h b/src/DataTypes/DataTypeFixedString.h index 2900efd5a34..22ec793208d 100644 --- a/src/DataTypes/DataTypeFixedString.h +++ b/src/DataTypes/DataTypeFixedString.h @@ -42,7 +42,8 @@ public: TypeIndex getTypeId() const override { return type_id; } const char * getFamilyName() const override { return "FixedString"; } - const char * getSQLCompatibleName() const override { return "TEXT"; } + /// Use TEXT for compatibility with MySQL to allow arbitrary bytes. + String getSQLCompatibleName() const override { return "TEXT"; } size_t getN() const { diff --git a/src/DataTypes/DataTypeFunction.h b/src/DataTypes/DataTypeFunction.h index df59f7738b2..b57c0587dde 100644 --- a/src/DataTypes/DataTypeFunction.h +++ b/src/DataTypes/DataTypeFunction.h @@ -24,7 +24,7 @@ public: std::string doGetName() const override; const char * getFamilyName() const override { return "Function"; } - const char * getSQLCompatibleName() const override { return "TEXT"; } + String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Function; } const DataTypes & getArgumentTypes() const diff --git a/src/DataTypes/DataTypeIPv4andIPv6.h b/src/DataTypes/DataTypeIPv4andIPv6.h index be0ebb90f3c..487ce04f67c 100644 --- a/src/DataTypes/DataTypeIPv4andIPv6.h +++ b/src/DataTypes/DataTypeIPv4andIPv6.h @@ -19,7 +19,7 @@ public: static constexpr auto type_id = TypeToTypeIndex; const char * getFamilyName() const override { return TypeName.data(); } - const char * getSQLCompatibleName() const override { return "TEXT"; } + String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return type_id; } @@ -61,7 +61,7 @@ public: static constexpr auto type_id = TypeToTypeIndex; const char * getFamilyName() const override { return TypeName.data(); } - const char * getSQLCompatibleName() const override { return "TEXT"; } + String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeInterval.h b/src/DataTypes/DataTypeInterval.h index ee2157431dd..7de56c13b56 100644 --- a/src/DataTypes/DataTypeInterval.h +++ b/src/DataTypes/DataTypeInterval.h @@ -26,7 +26,7 @@ public: std::string doGetName() const override { return fmt::format("Interval{}", kind.toString()); } const char * getFamilyName() const override { return "Interval"; } - const char * getSQLCompatibleName() const override { return "TEXT"; } + String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Interval; } bool equals(const IDataType & rhs) const override; diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index e59613e6974..8293455cabc 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -28,8 +28,7 @@ namespace ErrorCodes } DataTypeLowCardinality::DataTypeLowCardinality(DataTypePtr dictionary_type_) - : dictionary_type(std::move(dictionary_type_)), - mysql_name(dictionary_type->getSQLCompatibleName()) + : dictionary_type(std::move(dictionary_type_)) { auto inner_type = dictionary_type; if (dictionary_type->isNullable()) diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index 4dee8565568..f6d8d07a312 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -11,7 +11,6 @@ class DataTypeLowCardinality : public IDataType { private: DataTypePtr dictionary_type; - std::string mysql_name; public: @@ -24,7 +23,7 @@ public: return "LowCardinality(" + dictionary_type->getName() + ")"; } const char * getFamilyName() const override { return "LowCardinality"; } - const char * getSQLCompatibleName() const override { return mysql_name.c_str(); } + String getSQLCompatibleName() const override { return dictionary_type->getSQLCompatibleName(); } TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; } diff --git a/src/DataTypes/DataTypeMap.h b/src/DataTypes/DataTypeMap.h index 299119f1759..294c5d7ac77 100644 --- a/src/DataTypes/DataTypeMap.h +++ b/src/DataTypes/DataTypeMap.h @@ -30,7 +30,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Map; } std::string doGetName() const override; const char * getFamilyName() const override { return "Map"; } - const char * getSQLCompatibleName() const override { return "JSON"; } + String getSQLCompatibleName() const override { return "JSON"; } bool canBeInsideNullable() const override { return false; } diff --git a/src/DataTypes/DataTypeNothing.h b/src/DataTypes/DataTypeNothing.h index b35ced5dcb3..c3a7e2d09f0 100644 --- a/src/DataTypes/DataTypeNothing.h +++ b/src/DataTypes/DataTypeNothing.h @@ -16,7 +16,7 @@ public: static constexpr bool is_parametric = false; const char * getFamilyName() const override { return "Nothing"; } - const char * getSQLCompatibleName() const override { return "TEXT"; } + String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Nothing; } diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index b5fe1bb2dd9..e3165414c07 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -16,7 +16,7 @@ public: explicit DataTypeNullable(const DataTypePtr & nested_data_type_); std::string doGetName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } const char * getFamilyName() const override { return "Nullable"; } - const char * getSQLCompatibleName() const override { return nested_data_type->getSQLCompatibleName(); } + String getSQLCompatibleName() const override { return nested_data_type->getSQLCompatibleName(); } TypeIndex getTypeId() const override { return TypeIndex::Nullable; } MutableColumnPtr createColumn() const override; diff --git a/src/DataTypes/DataTypeNumberBase.cpp b/src/DataTypes/DataTypeNumberBase.cpp index db654448e83..e4c0fb96483 100644 --- a/src/DataTypes/DataTypeNumberBase.cpp +++ b/src/DataTypes/DataTypeNumberBase.cpp @@ -11,6 +11,55 @@ Field DataTypeNumberBase::getDefault() const { return NearestFieldType(); } +template +String DataTypeNumberBase::getSQLCompatibleName() const +{ + if constexpr (std::is_same_v) + { + return "TINYINT"; + } + else if constexpr (std::is_same_v) + { + return "SMALLINT"; + } + else if constexpr (std::is_same_v) + { + return "INTEGER"; + } + else if constexpr (std::is_same_v) + { + return "BIGINT"; + } + else if constexpr (std::is_same_v) + { + return "TINYINT UNSIGNED"; + } + else if constexpr (std::is_same_v) + { + return "SMALLINT UNSIGNED"; + } + else if constexpr (std::is_same_v) + { + return "INTEGER UNSIGNED"; + } + else if constexpr (std::is_same_v) + { + return "BIGINT UNSIGNED"; + } + else if constexpr (std::is_same_v) + { + return "FLOAT"; + } + else if constexpr (std::is_same_v) + { + return "DOUBLE"; + } + /// Unsupported types are converted to TEXT + else + { + return "TEXT"; + } +} template MutableColumnPtr DataTypeNumberBase::createColumn() const @@ -30,24 +79,6 @@ bool DataTypeNumberBase::isValueRepresentedByUnsignedInteger() const return is_integer && is_unsigned_v; } -template -const std::map DataTypeNumberBase::mysqlTypeMap = { - {"UInt8", "TINYINT UNSIGNED"}, - {"UInt16", "SMALLINT UNSIGNED"}, - {"UInt32", "MEDIUMINT UNSIGNEd"}, - {"UInt64", "BIGINT UNSIGNED"}, - {"UInt128", "TEXT"}, - {"UInt256", "TEXT"}, - {"Int8", "TINYINT"}, - {"Int16", "SMALLINT"}, - {"Int32", "INT"}, - {"Int64", "BIGINT"}, - {"Int128", "TEXT"}, - {"Int256", "TEXT"}, - {"Float32", "FLOAT"}, - {"Float64", "DOUBLE"}, -}; - /// Explicit template instantiations - to avoid code bloat in headers. template class DataTypeNumberBase; template class DataTypeNumberBase; diff --git a/src/DataTypes/DataTypeNumberBase.h b/src/DataTypes/DataTypeNumberBase.h index 1a855a974f0..d902c62505e 100644 --- a/src/DataTypes/DataTypeNumberBase.h +++ b/src/DataTypes/DataTypeNumberBase.h @@ -20,14 +20,12 @@ public: static constexpr bool is_parametric = false; static constexpr auto family_name = TypeName; static constexpr auto type_id = TypeToTypeIndex; - // Create a map from the name of the type to the name of the type in MySQL. - static const std::map mysqlTypeMap; using FieldType = T; using ColumnType = ColumnVector; const char * getFamilyName() const override { return TypeName.data(); } - const char * getSQLCompatibleName() const override { return mysqlTypeMap.at(TypeName.data()).c_str(); } + String getSQLCompatibleName() const override; TypeIndex getTypeId() const override { return TypeToTypeIndex; } Field getDefault() const override; diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 618c7389758..2e1e5398f7e 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -23,7 +23,7 @@ public: DataTypeObject(const String & schema_format_, bool is_nullable_); const char * getFamilyName() const override { return "Object"; } - const char * getSQLCompatibleName() const override { return "JSON"; } + String getSQLCompatibleName() const override { return "JSON"; } String doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::Object; } diff --git a/src/DataTypes/DataTypeSet.h b/src/DataTypes/DataTypeSet.h index 916b4f071a5..d88d76b31be 100644 --- a/src/DataTypes/DataTypeSet.h +++ b/src/DataTypes/DataTypeSet.h @@ -15,7 +15,7 @@ class DataTypeSet final : public IDataTypeDummy public: static constexpr bool is_parametric = true; const char * getFamilyName() const override { return "Set"; } - const char * getSQLCompatibleName() const override { return "TEXT"; } + String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Set; } bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); } diff --git a/src/DataTypes/DataTypeString.h b/src/DataTypes/DataTypeString.h index 338b3846266..c39fa90f6e7 100644 --- a/src/DataTypes/DataTypeString.h +++ b/src/DataTypes/DataTypeString.h @@ -21,8 +21,7 @@ public: return "String"; } - // FIXME: string can contain arbitrary bytes, not only UTF-8 sequences - const char * getSQLCompatibleName() const override { return "BLOB"; } + String getSQLCompatibleName() const override { return "BLOB"; } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 93fa87b1332..ea05e6ae59b 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -33,7 +33,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Tuple; } std::string doGetName() const override; const char * getFamilyName() const override { return "Tuple"; } - const char * getSQLCompatibleName() const override { return "JSON"; } + String getSQLCompatibleName() const override { return "JSON"; } bool canBeInsideNullable() const override { return false; } bool supportsSparseSerialization() const override { return true; } diff --git a/src/DataTypes/DataTypeUUID.h b/src/DataTypes/DataTypeUUID.h index bbf35074df3..8664c3bcfd1 100644 --- a/src/DataTypes/DataTypeUUID.h +++ b/src/DataTypes/DataTypeUUID.h @@ -18,7 +18,7 @@ public: static constexpr auto type_id = TypeIndex::UUID; const char * getFamilyName() const override { return "UUID"; } - const char * getSQLCompatibleName() const override { return "CHAR"; } + String getSQLCompatibleName() const override { return "CHAR"; } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index 1c2a63371ee..fa044d4ac9c 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -28,6 +28,11 @@ std::string DataTypeDecimal::doGetName() const return fmt::format("Decimal({}, {})", this->precision, this->scale); } +template +std::string DataTypeDecimal::getSQLCompatibleName() const +{ + return fmt::format("DECIMAL({}, {})", this->precision, this->scale); +} template bool DataTypeDecimal::equals(const IDataType & rhs) const diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index 6f3bf582aeb..5e4cfab7928 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -37,10 +37,9 @@ public: using Base::Base; static constexpr auto family_name = "Decimal"; - static constexpr auto mysql_name = "DECIMAL"; const char * getFamilyName() const override { return family_name; } - const char * getSQLCompatibleName() const override { return mysql_name; } + String getSQLCompatibleName() const override; std::string doGetName() const override; TypeIndex getTypeId() const override { return TypeToTypeIndex; } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 93fdbab05ef..51a9ecef0cc 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -71,19 +71,12 @@ public: return doGetName(); } - /// MySQL equivalent Name of data type (examples: UInt64, Array(String)). - String getMySQLTypeName() const - { - if (custom_name) - return custom_name->getName(); - else - return doGetMySQLName(); - } DataTypePtr getPtr() const { return shared_from_this(); } /// Name of data type family (example: FixedString, Array). virtual const char * getFamilyName() const = 0; - virtual const char * getSQLCompatibleName() const = 0; + /// Name of corresponding data type in MySQL (exampe: Bigint, Blob, etc) + virtual String getSQLCompatibleName() const = 0; /// Data type id. It's used for runtime type checks. virtual TypeIndex getTypeId() const = 0; @@ -135,7 +128,6 @@ public: protected: virtual String doGetName() const { return getFamilyName(); } - virtual String doGetMySQLName() const { return getSQLCompatibleName(); } virtual SerializationPtr doGetDefaultSerialization() const = 0; public: diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index f391a392dbb..684c35709a4 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -74,7 +74,8 @@ public: : ISource(header_) , columns_mask(std::move(columns_mask_)), max_block_size(max_block_size_) , databases(std::move(databases_)), tables(std::move(tables_)), storages(std::move(storages_)) - , clientInfo(context->getClientInfo()) + , client_info_interface(context->getClientInfo().interface) + , use_mysql_types(context->getSettingsRef().output_format_mysql_types) , total_tables(tables->size()), access(context->getAccess()) , query_id(context->getCurrentQueryId()), lock_acquire_timeout(context->getSettingsRef().lock_acquire_timeout) { @@ -132,9 +133,10 @@ protected: auto get_type_name = [this](const IDataType& type) -> std::string { - if (clientInfo.interface == DB::ClientInfo::Interface::MYSQL) + // Check if the output_format_mysql_types setting is enabled and client is connected via MySQL protocol + if (use_mysql_types && client_info_interface == DB::ClientInfo::Interface::MYSQL) { - return type.getMySQLTypeName(); + return type.getSQLCompatibleName(); } else { @@ -293,7 +295,8 @@ private: ColumnPtr databases; ColumnPtr tables; Storages storages; - ClientInfo clientInfo; + ClientInfo::Interface client_info_interface; + bool use_mysql_types; size_t db_table_num = 0; size_t total_tables; std::shared_ptr access; diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference index 1742cd9c90c..68e7be9ae6f 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference @@ -4,6 +4,44 @@ Create pseudo-random database name Create tab duplicate table Run MySQL test field type null key default extra +aggregate_function AggregateFunction(sum, Int32) 0 NULL +array_value Array(Int32) 0 NULL +boolean_value UInt8 0 NULL +date32_value Date32 0 NULL +date_value Date 0 NULL +datetime64_value DateTime64(3) 0 NULL +datetime_value DateTime 0 NULL +decimal_value Decimal(10, 2) 0 NULL +enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3) 0 NULL +fixed_string_value FixedString(10) 0 NULL +float32 Float32 0 NULL +float64 Float64 0 NULL +int128 Int128 0 NULL +int16 Int16 0 NULL +int256 Int256 0 NULL +int32 Int32 0 NULL +int64 Int64 0 NULL +int8 Int8 0 NULL +ipv4_value IPv4 0 NULL +ipv6_value IPv6 0 NULL +json_value Object('json') 0 NULL +low_cardinality LowCardinality(String) 0 NULL +low_cardinality_date LowCardinality(DateTime) 0 NULL +map_value Map(String, Int32) 0 NULL +nested.nested_int Array(Int32) 0 NULL +nested.nested_string Array(String) 0 NULL +nint32 Nullable(Int32) 1 NULL +nullable_value Nullable(Int32) 1 NULL +string_value String 0 NULL +tuple_value Tuple(Int32, String) 0 NULL +uint128 UInt128 0 NULL +uint16 UInt16 0 NULL +uint256 UInt256 0 NULL +uint32 UInt32 0 NULL +uint64 UInt64 0 PRI SOR NULL +uint8 UInt8 0 NULL +uuid_value UUID 0 NULL +field type null key default extra aggregate_function TEXT 0 NULL array_value TEXT 0 NULL boolean_value TINYINT UNSIGNED 0 NULL @@ -11,12 +49,17 @@ date32_value DATE 0 NULL date_value DATE 0 NULL datetime64_value DATETIME 0 NULL datetime_value DATETIME 0 NULL -decimal_value DECIMAL 0 NULL +decimal_value DECIMAL(10, 2) 0 NULL enum_value ENUM('apple', 'banana', 'orange') 0 NULL fixed_string_value TEXT 0 NULL float32 FLOAT 0 NULL float64 DOUBLE 0 NULL -int32 INT 0 NULL +int128 TEXT 0 NULL +int16 SMALLINT 0 NULL +int256 TEXT 0 NULL +int32 INTEGER 0 NULL +int64 BIGINT 0 NULL +int8 TINYINT 0 NULL ipv4_value TEXT 0 NULL ipv6_value TEXT 0 NULL json_value JSON 0 NULL @@ -25,10 +68,16 @@ low_cardinality_date DATETIME 0 NULL map_value JSON 0 NULL nested.nested_int TEXT 0 NULL nested.nested_string TEXT 0 NULL -nullable_value INT 0 NULL +nint32 INTEGER 0 NULL +nullable_value INTEGER 0 NULL string_value BLOB 0 NULL tuple_value JSON 0 NULL +uint128 TEXT 0 NULL +uint16 SMALLINT UNSIGNED 0 NULL +uint256 TEXT 0 NULL +uint32 INTEGER UNSIGNED 0 NULL uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uint8 TINYINT UNSIGNED 0 NULL uuid_value CHAR 0 NULL field type null key default extra aggregate_function TEXT 0 NULL @@ -38,12 +87,17 @@ date32_value DATE 0 NULL date_value DATE 0 NULL datetime64_value DATETIME 0 NULL datetime_value DATETIME 0 NULL -decimal_value DECIMAL 0 NULL +decimal_value DECIMAL(10, 2) 0 NULL enum_value ENUM('apple', 'banana', 'orange') 0 NULL fixed_string_value TEXT 0 NULL float32 FLOAT 0 NULL float64 DOUBLE 0 NULL -int32 INT 0 NULL +int128 TEXT 0 NULL +int16 SMALLINT 0 NULL +int256 TEXT 0 NULL +int32 INTEGER 0 NULL +int64 BIGINT 0 NULL +int8 TINYINT 0 NULL ipv4_value TEXT 0 NULL ipv6_value TEXT 0 NULL json_value JSON 0 NULL @@ -52,10 +106,16 @@ low_cardinality_date DATETIME 0 NULL map_value JSON 0 NULL nested.nested_int TEXT 0 NULL nested.nested_string TEXT 0 NULL -nullable_value INT 0 NULL +nint32 INTEGER 0 NULL +nullable_value INTEGER 0 NULL string_value BLOB 0 NULL tuple_value JSON 0 NULL +uint128 TEXT 0 NULL +uint16 SMALLINT UNSIGNED 0 NULL +uint256 TEXT 0 NULL +uint32 INTEGER UNSIGNED 0 NULL uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uint8 TINYINT UNSIGNED 0 NULL uuid_value CHAR 0 NULL field type null key default extra collation comment privileges aggregate_function TEXT 0 NULL NULL @@ -65,12 +125,17 @@ date32_value DATE 0 NULL NULL date_value DATE 0 NULL NULL datetime64_value DATETIME 0 NULL NULL datetime_value DATETIME 0 NULL NULL -decimal_value DECIMAL 0 NULL NULL +decimal_value DECIMAL(10, 2) 0 NULL NULL enum_value ENUM('apple', 'banana', 'orange') 0 NULL NULL fixed_string_value TEXT 0 NULL NULL float32 FLOAT 0 NULL NULL float64 DOUBLE 0 NULL NULL -int32 INT 0 NULL NULL +int128 TEXT 0 NULL NULL +int16 SMALLINT 0 NULL NULL +int256 TEXT 0 NULL NULL +int32 INTEGER 0 NULL NULL +int64 BIGINT 0 NULL NULL +int8 TINYINT 0 NULL NULL ipv4_value TEXT 0 NULL NULL ipv6_value TEXT 0 NULL NULL json_value JSON 0 NULL NULL @@ -79,15 +144,32 @@ low_cardinality_date DATETIME 0 NULL NULL map_value JSON 0 NULL NULL nested.nested_int TEXT 0 NULL NULL nested.nested_string TEXT 0 NULL NULL -nullable_value INT 0 NULL NULL +nint32 INTEGER 0 NULL NULL +nullable_value INTEGER 0 NULL NULL string_value BLOB 0 NULL NULL tuple_value JSON 0 NULL NULL +uint128 TEXT 0 NULL NULL +uint16 SMALLINT UNSIGNED 0 NULL NULL +uint256 TEXT 0 NULL NULL +uint32 INTEGER UNSIGNED 0 NULL NULL uint64 BIGINT UNSIGNED 0 PRI SOR NULL NULL +uint8 TINYINT UNSIGNED 0 NULL NULL uuid_value CHAR 0 NULL NULL field type null key default extra -int32 INT 0 NULL +int128 TEXT 0 NULL +int16 SMALLINT 0 NULL +int256 TEXT 0 NULL +int32 INTEGER 0 NULL +int64 BIGINT 0 NULL +int8 TINYINT 0 NULL nested.nested_int TEXT 0 NULL +nint32 INTEGER 0 NULL +uint128 TEXT 0 NULL +uint16 SMALLINT UNSIGNED 0 NULL +uint256 TEXT 0 NULL +uint32 INTEGER UNSIGNED 0 NULL uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uint8 TINYINT UNSIGNED 0 NULL field type null key default extra aggregate_function TEXT 0 NULL array_value TEXT 0 NULL @@ -96,7 +178,7 @@ date32_value DATE 0 NULL date_value DATE 0 NULL datetime64_value DATETIME 0 NULL datetime_value DATETIME 0 NULL -decimal_value DECIMAL 0 NULL +decimal_value DECIMAL(10, 2) 0 NULL enum_value ENUM('apple', 'banana', 'orange') 0 NULL fixed_string_value TEXT 0 NULL float32 FLOAT 0 NULL @@ -108,14 +190,25 @@ low_cardinality BLOB 0 NULL low_cardinality_date DATETIME 0 NULL map_value JSON 0 NULL nested.nested_string TEXT 0 NULL -nullable_value INT 0 NULL +nullable_value INTEGER 0 NULL string_value BLOB 0 NULL tuple_value JSON 0 NULL uuid_value CHAR 0 NULL field type null key default extra -int32 INT 0 NULL +int128 TEXT 0 NULL +int16 SMALLINT 0 NULL +int256 TEXT 0 NULL +int32 INTEGER 0 NULL +int64 BIGINT 0 NULL +int8 TINYINT 0 NULL nested.nested_int TEXT 0 NULL +nint32 INTEGER 0 NULL +uint128 TEXT 0 NULL +uint16 SMALLINT UNSIGNED 0 NULL +uint256 TEXT 0 NULL +uint32 INTEGER UNSIGNED 0 NULL uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uint8 TINYINT UNSIGNED 0 NULL field type null key default extra aggregate_function TEXT 0 NULL array_value TEXT 0 NULL @@ -124,7 +217,7 @@ date32_value DATE 0 NULL date_value DATE 0 NULL datetime64_value DATETIME 0 NULL datetime_value DATETIME 0 NULL -decimal_value DECIMAL 0 NULL +decimal_value DECIMAL(10, 2) 0 NULL enum_value ENUM('apple', 'banana', 'orange') 0 NULL fixed_string_value TEXT 0 NULL float32 FLOAT 0 NULL @@ -136,14 +229,25 @@ low_cardinality BLOB 0 NULL low_cardinality_date DATETIME 0 NULL map_value JSON 0 NULL nested.nested_string TEXT 0 NULL -nullable_value INT 0 NULL +nullable_value INTEGER 0 NULL string_value BLOB 0 NULL tuple_value JSON 0 NULL uuid_value CHAR 0 NULL field type null key default extra -int32 INT 0 NULL +int128 TEXT 0 NULL +int16 SMALLINT 0 NULL +int256 TEXT 0 NULL +int32 INTEGER 0 NULL +int64 BIGINT 0 NULL +int8 TINYINT 0 NULL nested.nested_int TEXT 0 NULL +nint32 INTEGER 0 NULL +uint128 TEXT 0 NULL +uint16 SMALLINT UNSIGNED 0 NULL +uint256 TEXT 0 NULL +uint32 INTEGER UNSIGNED 0 NULL uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uint8 TINYINT UNSIGNED 0 NULL field type null key default extra aggregate_function TEXT 0 NULL field type null key default extra @@ -154,12 +258,17 @@ date32_value DATE 0 NULL date_value DATE 0 NULL datetime64_value DATETIME 0 NULL datetime_value DATETIME 0 NULL -decimal_value DECIMAL 0 NULL +decimal_value DECIMAL(10, 2) 0 NULL enum_value ENUM('apple', 'banana', 'orange') 0 NULL fixed_string_value TEXT 0 NULL float32 FLOAT 0 NULL float64 DOUBLE 0 NULL -int32 INT 0 NULL +int128 TEXT 0 NULL +int16 SMALLINT 0 NULL +int256 TEXT 0 NULL +int32 INTEGER 0 NULL +int64 BIGINT 0 NULL +int8 TINYINT 0 NULL ipv4_value TEXT 0 NULL ipv6_value TEXT 0 NULL json_value JSON 0 NULL @@ -168,10 +277,16 @@ low_cardinality_date DATETIME 0 NULL map_value JSON 0 NULL nested.nested_int TEXT 0 NULL nested.nested_string TEXT 0 NULL -nullable_value INT 0 NULL +nint32 INTEGER 0 NULL +nullable_value INTEGER 0 NULL string_value BLOB 0 NULL tuple_value JSON 0 NULL +uint128 TEXT 0 NULL +uint16 SMALLINT UNSIGNED 0 NULL +uint256 TEXT 0 NULL +uint32 INTEGER UNSIGNED 0 NULL uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uint8 TINYINT UNSIGNED 0 NULL uuid_value CHAR 0 NULL field type null key default extra aggregate_function TEXT 0 NULL @@ -181,12 +296,17 @@ date32_value DATE 0 NULL date_value DATE 0 NULL datetime64_value DATETIME 0 NULL datetime_value DATETIME 0 NULL -decimal_value DECIMAL 0 NULL +decimal_value DECIMAL(10, 2) 0 NULL enum_value ENUM('apple', 'banana', 'orange') 0 NULL fixed_string_value TEXT 0 NULL float32 FLOAT 0 NULL float64 DOUBLE 0 NULL -int32 INT 0 NULL +int128 TEXT 0 NULL +int16 SMALLINT 0 NULL +int256 TEXT 0 NULL +int32 INTEGER 0 NULL +int64 BIGINT 0 NULL +int8 TINYINT 0 NULL ipv4_value TEXT 0 NULL ipv6_value TEXT 0 NULL json_value JSON 0 NULL @@ -195,10 +315,16 @@ low_cardinality_date DATETIME 0 NULL map_value JSON 0 NULL nested.nested_int TEXT 0 NULL nested.nested_string TEXT 0 NULL -nullable_value INT 0 NULL +nint32 INTEGER 0 NULL +nullable_value INTEGER 0 NULL string_value BLOB 0 NULL tuple_value JSON 0 NULL +uint128 TEXT 0 NULL +uint16 SMALLINT UNSIGNED 0 NULL +uint256 TEXT 0 NULL +uint32 INTEGER UNSIGNED 0 NULL uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uint8 TINYINT UNSIGNED 0 NULL uuid_value CHAR 0 NULL field type null key default extra aggregate_function TEXT 0 NULL @@ -208,12 +334,17 @@ date32_value DATE 0 NULL date_value DATE 0 NULL datetime64_value DATETIME 0 NULL datetime_value DATETIME 0 NULL -decimal_value DECIMAL 0 NULL +decimal_value DECIMAL(10, 2) 0 NULL enum_value ENUM('apple', 'banana', 'orange') 0 NULL fixed_string_value TEXT 0 NULL float32 FLOAT 0 NULL float64 DOUBLE 0 NULL -int32 INT 0 NULL +int128 TEXT 0 NULL +int16 SMALLINT 0 NULL +int256 TEXT 0 NULL +int32 INTEGER 0 NULL +int64 BIGINT 0 NULL +int8 TINYINT 0 NULL ipv4_value TEXT 0 NULL ipv6_value TEXT 0 NULL json_value JSON 0 NULL @@ -222,8 +353,14 @@ low_cardinality_date DATETIME 0 NULL map_value JSON 0 NULL nested.nested_int TEXT 0 NULL nested.nested_string TEXT 0 NULL -nullable_value INT 0 NULL +nint32 INTEGER 0 NULL +nullable_value INTEGER 0 NULL string_value BLOB 0 NULL tuple_value JSON 0 NULL +uint128 TEXT 0 NULL +uint16 SMALLINT UNSIGNED 0 NULL +uint256 TEXT 0 NULL +uint32 INTEGER UNSIGNED 0 NULL uint64 BIGINT UNSIGNED 0 PRI SOR NULL +uint8 TINYINT UNSIGNED 0 NULL uuid_value CHAR 0 NULL diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh index fd1ad92f060..938102cb5fc 100755 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh @@ -17,15 +17,25 @@ ${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS tab" ${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde" ${CLICKHOUSE_LOCAL} --query "DROP TABLE IF EXISTS database_123456789abcde.tab" -#${CLICKHOUSE_LOCAL} --query "SET allow_suspicious_low_cardinality_types = 1;" echo "Create tab table " ${CLICKHOUSE_LOCAL} -n -q " SET allow_suspicious_low_cardinality_types=1; - SET allow_experimental_object_type =1; + SET allow_experimental_object_type=1; CREATE TABLE tab ( + uint8 UInt8, + uint16 UInt16, + uint32 UInt32, uint64 UInt64, - int32 Nullable(Int32), + uint128 UInt128, + uint256 UInt256, + int8 Int8, + int16 Int16, + int32 Int32, + int64 Int64, + int128 Int128, + int256 Int256, + nint32 Nullable(Int32), float32 Float32, float64 Float64, decimal_value Decimal(10, 2), @@ -67,8 +77,19 @@ ${CLICKHOUSE_LOCAL} -n -q " SET allow_experimental_object_type =1; CREATE TABLE database_123456789abcde.tab ( + uint8 UInt8, + uint16 UInt16, + uint32 UInt32, uint64 UInt64, - int32 Nullable(Int32), + uint128 UInt128, + uint256 UInt256, + int8 Int8, + int16 Int16, + int32 Int32, + int64 Int64, + int128 Int128, + int256 Int256, + nint32 Nullable(Int32), float32 Float32, float64 Float64, decimal_value Decimal(10, 2), @@ -105,6 +126,8 @@ TEMP_FILE=$(mktemp) cat < $TEMP_FILE SHOW COLUMNS FROM tab; +SET output_format_mysql_types=1; +SHOW COLUMNS FROM tab; SHOW EXTENDED COLUMNS FROM tab; SHOW FULL COLUMNS FROM tab; SHOW COLUMNS FROM tab LIKE '%int%'; From cf9936ad327b4f94903e5993a1189c149c9c61dd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 5 Jun 2023 23:58:43 +0000 Subject: [PATCH 1223/2223] Automatic style fix --- .../test_storage_azure_blob_storage/test.py | 91 ++++++++++++++----- 1 file changed, 68 insertions(+), 23 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index e78fa185b17..6a3c915cdc5 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -17,6 +17,7 @@ from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry + @pytest.fixture(scope="module") def cluster(): try: @@ -32,6 +33,7 @@ def cluster(): finally: cluster.shutdown() + def azure_query(node, query, try_num=3, settings={}): for i in range(try_num): try: @@ -50,6 +52,7 @@ def azure_query(node, query, try_num=3, settings={}): raise Exception(ex) continue + def get_azure_file_content(filename): container_name = "cont" connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" @@ -57,43 +60,72 @@ def get_azure_file_content(filename): container_client = blob_service_client.get_container_client(container_name) blob_client = container_client.get_blob_client(filename) download_stream = blob_client.download_blob() - return download_stream.readall().decode('utf-8') + return download_stream.readall().decode("utf-8") + def test_create_table_connection_string(cluster): node = cluster.instances["node"] - azure_query(node, "CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = Azure('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', 'cont', 'test_create_connection_string', 'CSV')") + azure_query( + node, + "CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = Azure('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', 'cont', 'test_create_connection_string', 'CSV')", + ) + def test_create_table_account_string(cluster): node = cluster.instances["node"] - azure_query(node, "CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')") + azure_query( + node, + "CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", + ) + def test_simple_write_account_string(cluster): node = cluster.instances["node"] - azure_query(node, "CREATE TABLE test_simple_write (key UInt64, data String) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')") + azure_query( + node, + "CREATE TABLE test_simple_write (key UInt64, data String) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", + ) azure_query(node, "INSERT INTO test_simple_write VALUES (1, 'a')") - print(get_azure_file_content('test_simple_write.csv')) - assert get_azure_file_content('test_simple_write.csv') == '1,"a"\n' + print(get_azure_file_content("test_simple_write.csv")) + assert get_azure_file_content("test_simple_write.csv") == '1,"a"\n' + def test_simple_write_connection_string(cluster): node = cluster.instances["node"] - azure_query(node, "CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = Azure('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_c.csv', 'CSV')") + azure_query( + node, + "CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = Azure('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_c.csv', 'CSV')", + ) azure_query(node, "INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") - print(get_azure_file_content('test_simple_write_c.csv')) - assert get_azure_file_content('test_simple_write_c.csv') == '1,"a"\n' + print(get_azure_file_content("test_simple_write_c.csv")) + assert get_azure_file_content("test_simple_write_c.csv") == '1,"a"\n' + def test_simple_write_named_collection_1(cluster): node = cluster.instances["node"] - azure_query(node, "CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = Azure(azure_conf1)") - azure_query(node, "INSERT INTO test_simple_write_named_collection_1 VALUES (1, 'a')") - print(get_azure_file_content('test_simple_write_named.csv')) - assert get_azure_file_content('test_simple_write_named.csv') == '1,"a"\n' + azure_query( + node, + "CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = Azure(azure_conf1)", + ) + azure_query( + node, "INSERT INTO test_simple_write_named_collection_1 VALUES (1, 'a')" + ) + print(get_azure_file_content("test_simple_write_named.csv")) + assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' + def test_simple_write_named_collection_2(cluster): node = cluster.instances["node"] - azure_query(node, "CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')") - azure_query(node, "INSERT INTO test_simple_write_named_collection_2 VALUES (1, 'a')") - print(get_azure_file_content('test_simple_write_named_2.csv')) - assert get_azure_file_content('test_simple_write_named_2.csv') == '1,"a"\n' + azure_query( + node, + "CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')", + ) + azure_query( + node, "INSERT INTO test_simple_write_named_collection_2 VALUES (1, 'a')" + ) + print(get_azure_file_content("test_simple_write_named_2.csv")) + assert get_azure_file_content("test_simple_write_named_2.csv") == '1,"a"\n' + def test_partition_by(cluster): node = cluster.instances["node"] @@ -102,7 +134,10 @@ def test_partition_by(cluster): values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" filename = "test_{_partition_id}.csv" - azure_query(node, f"CREATE TABLE test_partitioned_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}") + azure_query( + node, + f"CREATE TABLE test_partitioned_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + ) azure_query(node, f"INSERT INTO test_partitioned_write VALUES {values}") assert "1,2,3\n" == get_azure_file_content("test_3.csv") @@ -116,7 +151,10 @@ def test_partition_by_string_column(cluster): partition_by = "col_str" values = "(1, 'foo/bar'), (3, 'йцук'), (78, '你好')" filename = "test_{_partition_id}.csv" - azure_query(node, f"CREATE TABLE test_partitioned_string_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}") + azure_query( + node, + f"CREATE TABLE test_partitioned_string_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + ) azure_query(node, f"INSERT INTO test_partitioned_string_write VALUES {values}") assert '1,"foo/bar"\n' == get_azure_file_content("test_foo/bar.csv") @@ -131,15 +169,22 @@ def test_partition_by_const_column(cluster): partition_by = "'88'" values_csv = "1,2,3\n3,2,1\n78,43,45\n" filename = "test_{_partition_id}.csv" - azure_query(node, f"CREATE TABLE test_partitioned_const_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}") + azure_query( + node, + f"CREATE TABLE test_partitioned_const_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + ) azure_query(node, f"INSERT INTO test_partitioned_const_write VALUES {values}") assert values_csv == get_azure_file_content("test_88.csv") + def test_truncate(cluster): node = cluster.instances["node"] - azure_query(node, "CREATE TABLE test_truncate (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_truncate.csv', format='CSV')") + azure_query( + node, + "CREATE TABLE test_truncate (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_truncate.csv', format='CSV')", + ) azure_query(node, "INSERT INTO test_truncate VALUES (1, 'a')") - assert get_azure_file_content('test_truncate.csv') == '1,"a"\n' + assert get_azure_file_content("test_truncate.csv") == '1,"a"\n' azure_query(node, "TRUNCATE TABLE test_truncate") with pytest.raises(Exception): - print(get_azure_file_content('test_truncate.csv')) + print(get_azure_file_content("test_truncate.csv")) From 9f80900d6f587383780d2a40f8173093dce68a5a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 1 Jun 2023 22:02:17 +0000 Subject: [PATCH 1224/2223] Changes related to an internal feature --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp | 7 +++++-- src/Disks/ObjectStorages/Cached/CachedObjectStorage.h | 4 +++- src/IO/ReadSettings.h | 2 ++ src/Interpreters/Cache/FileSegment.h | 2 +- src/Interpreters/Cache/IFileCachePriority.h | 1 + src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp | 7 +++++++ src/Storages/MergeTree/DataPartStorageOnDiskBase.h | 1 + src/Storages/MergeTree/IDataPartStorage.h | 1 + .../configs/config.d/storage_conf.xml | 1 + .../configs/config.d/users.xml | 7 +++++++ .../test_replicated_merge_tree_s3_zero_copy/test.py | 8 ++++++-- 13 files changed, 37 insertions(+), 8 deletions(-) create mode 100644 tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/users.xml diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 7c497baa450..877d8ff9bb7 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -1219,7 +1219,7 @@ off_t CachedOnDiskReadBufferFromFile::getPosition() void CachedOnDiskReadBufferFromFile::assertCorrectness() const { - if (!CachedObjectStorage::canUseReadThroughCache() + if (!CachedObjectStorage::canUseReadThroughCache(settings) && !settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache usage is not allowed (query_id: {})", query_id); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 12fbbbcf747..04030fe5f8f 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -36,7 +36,7 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( with_cache = settings.remote_fs_cache && settings.enable_filesystem_cache - && (!query_id.empty() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache); + && (!query_id.empty() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache || !settings.avoid_readthrough_cache_outside_query_context); } SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object) diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 1d24d9d5411..3e73e45638b 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -57,7 +57,7 @@ ReadSettings CachedObjectStorage::patchSettings(const ReadSettings & read_settin ReadSettings modified_settings{read_settings}; modified_settings.remote_fs_cache = cache; - if (!canUseReadThroughCache()) + if (!canUseReadThroughCache(read_settings)) modified_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; return object_storage->patchSettings(modified_settings); @@ -227,8 +227,11 @@ String CachedObjectStorage::getObjectsNamespace() const return object_storage->getObjectsNamespace(); } -bool CachedObjectStorage::canUseReadThroughCache() +bool CachedObjectStorage::canUseReadThroughCache(const ReadSettings & settings) { + if (!settings.avoid_readthrough_cache_outside_query_context) + return true; + return CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() && !CurrentThread::getQueryId().empty(); diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index b5186d39c32..ba9fbd02d94 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -112,7 +112,9 @@ public: WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override; - static bool canUseReadThroughCache(); + const FileCacheSettings & getCacheSettings() const { return cache_settings; } + + static bool canUseReadThroughCache(const ReadSettings & settings); private: FileCache::Key getCacheKey(const std::string & path) const; diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index e43ecd7f275..dae4261e92c 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -99,6 +99,8 @@ struct ReadSettings bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false; bool enable_filesystem_cache_log = false; bool is_file_cache_persistent = false; /// Some files can be made non-evictable. + /// Don't populate cache when the read is not part of query execution (e.g. background thread). + bool avoid_readthrough_cache_outside_query_context = true; size_t filesystem_cache_max_download_size = (128UL * 1024 * 1024 * 1024); bool skip_download_if_exceeds_query_cache = true; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 163a15fcfda..75395a671f4 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -85,7 +85,7 @@ public: EMPTY, /** * A newly created file segment never has DOWNLOADING state until call to getOrSetDownloader - * because each cache user might acquire multiple file segments and reads them one by one, + * because each cache user might acquire multiple file segments and read them one by one, * so only user which actually needs to read this segment earlier than others - becomes a downloader. */ DOWNLOADING, diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index ad63dcc7ea5..93343398783 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -85,6 +85,7 @@ public: virtual void removeAll(const CacheGuard::Lock &) = 0; + /// From lowest to highest priority. virtual void iterate(IterateFunc && func, const CacheGuard::Lock &) = 0; private: diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index cfc3ff58f81..30776a8bc50 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -202,6 +202,13 @@ bool DataPartStorageOnDiskBase::isStoredOnRemoteDisk() const return volume->getDisk()->isRemote(); } +std::optional DataPartStorageOnDiskBase::getCacheName() const +{ + if (volume->getDisk()->supportsCache()) + return volume->getDisk()->getCacheName(); + return std::nullopt; +} + bool DataPartStorageOnDiskBase::supportZeroCopyReplication() const { return volume->getDisk()->supportZeroCopyReplication(); diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 6b27b7296fc..043953eb20c 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -36,6 +36,7 @@ public: std::string getDiskName() const override; std::string getDiskType() const override; bool isStoredOnRemoteDisk() const override; + std::optional getCacheName() const override; bool supportZeroCopyReplication() const override; bool supportParallelWrite() const override; bool isBroken() const override; diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index f160254350d..933c9bd9958 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -149,6 +149,7 @@ public: virtual std::string getDiskName() const = 0; virtual std::string getDiskType() const = 0; virtual bool isStoredOnRemoteDisk() const { return false; } + virtual std::optional getCacheName() const { return std::nullopt; } virtual bool supportZeroCopyReplication() const { return false; } virtual bool supportParallelWrite() const = 0; virtual bool isBroken() const = 0; diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml index 15239041478..96d59d5633e 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml @@ -12,6 +12,7 @@ s3 100000000 ./cache_s3/ + 1 diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/users.xml b/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/users.xml new file mode 100644 index 00000000000..5de169edc1e --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/users.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py index eca18820016..72a01d278d8 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py @@ -19,6 +19,7 @@ def cluster(): cluster.add_instance( "node1", main_configs=["configs/config.d/storage_conf.xml"], + user_configs=["configs/config.d/users.xml"], macros={"replica": "1"}, with_minio=True, with_zookeeper=True, @@ -26,12 +27,14 @@ def cluster(): cluster.add_instance( "node2", main_configs=["configs/config.d/storage_conf.xml"], + user_configs=["configs/config.d/users.xml"], macros={"replica": "2"}, with_zookeeper=True, ) cluster.add_instance( "node3", main_configs=["configs/config.d/storage_conf.xml"], + user_configs=["configs/config.d/users.xml"], macros={"replica": "3"}, with_zookeeper=True, ) @@ -74,7 +77,7 @@ def generate_values(date_str, count, sign=1): def create_table(cluster, additional_settings=None): create_table_statement = """ - CREATE TABLE s3_test ON CLUSTER cluster( + CREATE TABLE s3_test ON CLUSTER cluster ( dt Date, id Int64, data String, @@ -95,7 +98,8 @@ def create_table(cluster, additional_settings=None): def drop_table(cluster): yield for node in list(cluster.instances.values()): - node.query("DROP TABLE IF EXISTS s3_test") + node.query("DROP TABLE IF EXISTS s3_test SYNC") + node.query("DROP TABLE IF EXISTS test_drop_table SYNC") minio = cluster.minio_client # Remove extra objects to prevent tests cascade failing From 760483d8d3a012b4b6456b9bdf14afd2d052a514 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 6 Jun 2023 04:18:51 +0000 Subject: [PATCH 1225/2223] multiple fixes --- base/base/IPv4andIPv6.h | 4 ++- .../AggregateFunctionMap.cpp | 1 + src/AggregateFunctions/AggregateFunctionMap.h | 26 +++++++++++++++++++ .../AggregateFunctionUniq.cpp | 2 +- .../AggregateFunctionUniq.h | 12 +++++++++ src/IO/ReadHelpers.h | 14 ++++++++++ src/IO/WriteHelpers.h | 8 ++++++ 7 files changed, 65 insertions(+), 2 deletions(-) diff --git a/base/base/IPv4andIPv6.h b/base/base/IPv4andIPv6.h index 4aee2329572..e2f93b54124 100644 --- a/base/base/IPv4andIPv6.h +++ b/base/base/IPv4andIPv6.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -55,12 +56,13 @@ namespace DB namespace std { + /// For historical reasons we hash IPv6 as a FixedString(16) template <> struct hash { size_t operator()(const DB::IPv6 & x) const { - return std::hash()(x.toUnderType()); + return std::hash{}(std::string_view(reinterpret_cast(&x.toUnderType()), IPV6_BINARY_LENGTH)); } }; diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index 38e4f49d9a2..b957b541fe1 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -103,6 +103,7 @@ public: case TypeIndex::IPv4: return std::make_shared>(nested_function, arguments); case TypeIndex::IPv6: + return std::make_shared>(nested_function, arguments); case TypeIndex::FixedString: case TypeIndex::String: return std::make_shared>(nested_function, arguments); diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 4a4ae92735b..7b9bb088d8f 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -21,6 +21,7 @@ #include "DataTypes/Serializations/ISerialization.h" #include #include "base/types.h" +#include #include #include "AggregateFunctions/AggregateFunctionFactory.h" @@ -70,6 +71,31 @@ struct AggregateFunctionMapCombinatorData } }; +/// Specialization for IPv6 - for historical reasons it should be stored as FixedString(16) +template <> +struct AggregateFunctionMapCombinatorData +{ + struct IPv6Hash + { + using hash_type = std::hash; + using is_transparent = void; + + size_t operator()(const IPv6 & ip) const { return hash_type{}(ip); } + }; + + using SearchType = IPv6; + std::unordered_map> merged_maps; + + static void writeKey(const IPv6 & key, WriteBuffer & buf) + { + writeIPv6Binary(key, buf); + } + static void readKey(IPv6 & key, ReadBuffer & buf) + { + readIPv6Binary(key, buf); + } +}; + template class AggregateFunctionMap final : public IAggregateFunctionDataHelper, AggregateFunctionMap> diff --git a/src/AggregateFunctions/AggregateFunctionUniq.cpp b/src/AggregateFunctions/AggregateFunctionUniq.cpp index f5147daa97b..748a232641e 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -117,7 +117,7 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument else if (which.isIPv4()) return std::make_shared>>(argument_types); else if (which.isIPv6()) - return std::make_shared>>(argument_types); + return std::make_shared>>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index 0524dd53ec0..03d999b47e2 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -101,6 +101,18 @@ struct AggregateFunctionUniqHLL12Data static String getName() { return "uniqHLL12"; } }; +template <> +struct AggregateFunctionUniqHLL12Data +{ + using Set = HyperLogLogWithSmallSetOptimization; + Set set; + + constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_variadic = false; + + static String getName() { return "uniqHLL12"; } +}; + template struct AggregateFunctionUniqHLL12DataForVariadic { diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 32338552b66..1aa294f76bf 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -61,6 +61,7 @@ namespace ErrorCodes extern const int INCORRECT_DATA; extern const int TOO_LARGE_STRING_SIZE; extern const int TOO_LARGE_ARRAY_SIZE; + extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; } /// Helper functions for formatted input. @@ -136,6 +137,19 @@ inline void readStringBinary(std::string & s, ReadBuffer & buf, size_t max_strin buf.readStrict(s.data(), size); } +/// For historical reasons we store IPv6 as a String +inline void readIPv6Binary(IPv6 & ip, ReadBuffer & buf) +{ + size_t size = 0; + readVarUInt(size, buf); + + if (size != IPV6_BINARY_LENGTH) + throw Exception(ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH, + "Size of the string {} doesn't match size of binary IPv6 {}", size, IPV6_BINARY_LENGTH); + + buf.readStrict(reinterpret_cast(&ip.toUnderType()), size); +} + template void readVectorBinary(std::vector & v, ReadBuffer & buf) { diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index cdbc952690c..505a2f988f0 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -10,6 +10,7 @@ #include +#include "Common/formatIPv6.h" #include #include #include @@ -104,6 +105,13 @@ inline void writeStringBinary(const std::string & s, WriteBuffer & buf) buf.write(s.data(), s.size()); } +/// For historical reasons we store IPv6 as a String +inline void writeIPv6Binary(const IPv6 & ip, WriteBuffer & buf) +{ + writeVarUInt(IPV6_BINARY_LENGTH, buf); + buf.write(reinterpret_cast(&ip.toUnderType()), IPV6_BINARY_LENGTH); +} + inline void writeStringBinary(StringRef s, WriteBuffer & buf) { writeVarUInt(s.size, buf); From 6d25e5a0d75325ddeee2be0d689da6ea4395fccc Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 6 Jun 2023 07:37:14 +0300 Subject: [PATCH 1226/2223] Substitute missing year in parseDateTimeBestEffortImpl() --- src/IO/parseDateTimeBestEffort.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index b370bee6f3f..f753e3c0b4e 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -578,12 +578,16 @@ ReturnType parseDateTimeBestEffortImpl( if (!year && !month && !day_of_month && !has_time) return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: neither Date nor Time was parsed successfully"); - if (!year) - year = 2000; - if (!month) - month = 1; if (!day_of_month) day_of_month = 1; + if (!month) + month = 1; + if (!year) + { + time_t now = time(nullptr); + UInt16 curr_year = local_time_zone.toYear(now); + year = now < local_time_zone.makeDateTime(year, month, day_of_month, hour, minute, second) ? curr_year - 1 : curr_year; + } auto is_leap_year = (year % 400 == 0) || (year % 100 != 0 && year % 4 == 0); From 614451998d60d4ffcb1e06096ab64882dc97d7ea Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 6 Jun 2023 08:06:24 +0200 Subject: [PATCH 1227/2223] Fixed build by adding StorageAzureSource constructor and getHeader function implementation --- src/Storages/StorageAzure.cpp | 49 +++++++++++++++++++++++++++++++++++ src/Storages/StorageAzure.h | 16 ++++++------ 2 files changed, 57 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 166c1156c49..3e29fa43383 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -43,6 +43,12 @@ using namespace Azure::Storage::Blobs; +namespace CurrentMetrics +{ + extern const Metric ObjectStorageAzureThreads; + extern const Metric ObjectStorageAzureThreadsActive; +} + namespace DB { @@ -1038,6 +1044,49 @@ Chunk StorageAzureSource::generate() return {}; } +Block StorageAzureSource::getHeader(Block sample_block, const std::vector & requested_virtual_columns) +{ + for (const auto & virtual_column : requested_virtual_columns) + sample_block.insert({virtual_column.type->createColumn(), virtual_column.type, virtual_column.name}); + + return sample_block; +} + +StorageAzureSource::StorageAzureSource( + const std::vector & requested_virtual_columns_, + const String & format_, + String name_, + const Block & sample_block_, + ContextPtr context_, + std::optional format_settings_, + const ColumnsDescription & columns_, + UInt64 max_block_size_, + String compression_hint_, + AzureObjectStorage * object_storage_, + const String & container_, + std::shared_ptr file_iterator_) + :ISource(getHeader(sample_block_, requested_virtual_columns_)) + , WithContext(context_) + , requested_virtual_columns(requested_virtual_columns_) + , format(format_) + , name(std::move(name_)) + , sample_block(sample_block_) + , format_settings(format_settings_) + , columns_desc(columns_) + , max_block_size(max_block_size_) + , compression_hint(compression_hint_) + , object_storage(std::move(object_storage_)) + , container(container_) + , file_iterator(file_iterator_) + , create_reader_pool(CurrentMetrics::ObjectStorageAzureThreads, CurrentMetrics::ObjectStorageAzureThreadsActive, 1) + , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateAzureReader")) +{ + reader = createReader(); + if (reader) + reader_future = createReaderAsync(); +} + + StorageAzureSource::~StorageAzureSource() { create_reader_pool.wait(); diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index b7f5eba343f..168015cf5d9 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -198,17 +198,17 @@ public: static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); private: - String name; - String container; - String format; std::vector requested_virtual_columns; - ColumnsDescription columns_desc; - AzureObjectStorage * object_storage; - std::shared_ptr file_iterator; - UInt64 max_block_size; - String compression_hint; + String format; + String name; Block sample_block; std::optional format_settings; + ColumnsDescription columns_desc; + UInt64 max_block_size; + String compression_hint; + AzureObjectStorage * object_storage; + String container; + std::shared_ptr file_iterator; struct ReaderHolder { From 0bc31a72888da7ccf50caee845abef4141077220 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 6 Jun 2023 07:17:30 +0000 Subject: [PATCH 1228/2223] Fix jepsen runs in PRs --- tests/ci/jepsen_check.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 9d35d2d6e35..c21fafa2605 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -25,6 +25,7 @@ from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results from version_helper import get_version_from_repo +from build_check import get_release_or_pr JEPSEN_GROUP_NAME = "jepsen_group" @@ -210,12 +211,7 @@ if __name__ == "__main__": build_name = get_build_name_for_check(check_name) - if pr_info.number == 0: - version = get_version_from_repo() - release_or_pr = f"{version.major}.{version.minor}" - else: - # PR number for anything else - release_or_pr = str(pr_info.number) + release_or_pr, _ = get_release_or_pr(pr_info, get_version_from_repo()) # This check run separately from other checks because it requires exclusive # run (see .github/workflows/jepsen.yml) So we cannot add explicit From ce8b39487e5416b018ce9d03feef4a8114e04f9b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 09:55:50 +0200 Subject: [PATCH 1229/2223] Update docs/en/engines/table-engines/mergetree-family/annindexes.md Co-authored-by: Nikita Taranov --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 58655c11321..f600f9a015c 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -54,7 +54,7 @@ ENGINE = MergeTree ORDER BY id; ``` -ANN indexes are built during column insertion and merge and `INSERT` and `OPTIMIZE` statements will slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively comparatively many more read requests than write requests. +ANN indexes are built during column insertion and merge and `INSERT` and `OPTIMIZE` statements will be slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively there are much more read requests than write requests. Similar to regular skip indexes, ANN indexes are constructed over granules and each indexed block consists of `GRANULARITY = `-many granules. For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`, From 0b18b75bec6f29e687b6d9cce91fc2cc6c906221 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 08:06:03 +0000 Subject: [PATCH 1230/2223] Cosmetics: Use abbreviated syntax to read settings --- .../MergeTree/ApproximateNearestNeighborIndexesCommon.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index c47e53788a7..bf277c55863 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -53,8 +53,8 @@ ApproximateNearestNeighborInformation::Metric stringToMetric(std::string_view me ApproximateNearestNeighborCondition::ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context) : block_with_constants(KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)) - , index_granularity(context->getMergeTreeSettings().get("index_granularity").get()) - , max_limit_for_ann_queries(context->getSettings().get("max_limit_for_ann_queries").get()) + , index_granularity(context->getMergeTreeSettings().index_granularity) + , max_limit_for_ann_queries(context->getSettings().max_limit_for_ann_queries) , index_is_useful(checkQueryStructure(query_info)) {} diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 72dd92ead5e..0a2df639b69 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -210,7 +210,7 @@ MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( ContextPtr context) : ann_condition(query, context) , distance_function(distance_function_) - , search_k(context->getSettings().get("annoy_index_search_k_nodes").get()) + , search_k(context->getSettings().annoy_index_search_k_nodes) {} bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr /*idx_granule*/) const From 2c5c0c5c9f82963903a618e7686c39a59899f020 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 08:14:50 +0000 Subject: [PATCH 1231/2223] Cosmetics: Remove exception path for something checked elsewhere already --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 0a2df639b69..b15b1bb1a91 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -229,8 +229,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndex return getUsefulRangesImpl(idx_granule); else if (distance_function == "cosineDistance") return getUsefulRangesImpl(idx_granule); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); + std::unreachable(); } template From 3f77b778e368c1c5e3cd3012d54107193094502b Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 6 Jun 2023 11:42:45 +0300 Subject: [PATCH 1232/2223] Fix runtime bug --- src/IO/parseDateTimeBestEffort.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index f753e3c0b4e..6bdba251c36 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -586,7 +586,7 @@ ReturnType parseDateTimeBestEffortImpl( { time_t now = time(nullptr); UInt16 curr_year = local_time_zone.toYear(now); - year = now < local_time_zone.makeDateTime(year, month, day_of_month, hour, minute, second) ? curr_year - 1 : curr_year; + year = now < local_time_zone.makeDateTime(curr_year, month, day_of_month, hour, minute, second) ? curr_year - 1 : curr_year; } auto is_leap_year = (year % 400 == 0) || (year % 100 != 0 && year % 4 == 0); From 1506545db0f062787ac779f997e30b0e07a55736 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 11:47:29 +0200 Subject: [PATCH 1233/2223] Fix merge conflicts --- src/Storages/StorageAzure.cpp | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 3e29fa43383..1e128bfed66 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -230,13 +230,10 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C configuration.blobs_paths = {configuration.blob_path}; - LOG_INFO(&Poco::Logger::get("StorageAzure"), "get_format_from_file = {}", get_format_from_file); -// if (configuration.format == "auto" && get_format_from_file) -// configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); - - configuration.format = "TSV"; + if (configuration.format == "auto" && get_format_from_file) + configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); return configuration; } @@ -367,11 +364,6 @@ AzureClientPtr StorageAzure::createClient(StorageAzure::Configuration configurat } } } - auto managed_identity_credential = std::make_shared(); - - result = std::make_unique(configuration.connection_url, managed_identity_credential); - - LOG_INFO(&Poco::Logger::get("StorageAzure"), "createClient account_name & account_key "); } return result; @@ -613,7 +605,7 @@ Pipe StorageAzure::read( size_t num_streams) { if (partition_by && configuration.withWildcard()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned Azure storage is not implemented yet"); Pipes pipes; @@ -1079,7 +1071,7 @@ StorageAzureSource::StorageAzureSource( , container(container_) , file_iterator(file_iterator_) , create_reader_pool(CurrentMetrics::ObjectStorageAzureThreads, CurrentMetrics::ObjectStorageAzureThreadsActive, 1) - , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateAzureReader")) + , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "AzureReader")) { reader = createReader(); if (reader) From 5b3cece42eb540f5b05d04faa7fe3de7cd1ccb86 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 6 Jun 2023 12:23:00 +0200 Subject: [PATCH 1234/2223] Fix shellcheck --- docker/test/stateless/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index df650b37cc6..c0acb0291a4 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -15,6 +15,7 @@ dpkg -i package_folder/clickhouse-client_*.deb ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test +# shellcheck disable=SC1091 source /usr/share/clickhouse-test/ci/attach_gdb.lib # install test configs From 8028184e301f58aab5c4674226abbb39c5b8b745 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 12:27:40 +0200 Subject: [PATCH 1235/2223] Fix read --- src/Storages/StorageAzure.cpp | 41 ++----------------- src/Storages/StorageAzure.h | 1 - .../test_storage_azure_blob_storage/test.py | 13 ++++++ 3 files changed, 16 insertions(+), 39 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 1e128bfed66..2f0029947f5 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -1096,6 +1096,7 @@ StorageAzureSource::ReaderHolder StorageAzureSource::createReader() return {}; size_t object_size = info.size_bytes != 0 ? info.size_bytes : object_storage->getObjectMetadata(current_key).size_bytes; + LOG_DEBUG(log, "SIZE {}", object_size); auto compression_method = chooseCompressionMethod(current_key, compression_hint); auto read_buf = createAzureReadBuffer(current_key, object_size); @@ -1104,6 +1105,7 @@ StorageAzureSource::ReaderHolder StorageAzureSource::createReader() format_settings, std::nullopt, std::nullopt, /* is_remote_fs */ true, compression_method); + LOG_DEBUG(log, "FORMAT {}", format); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -1138,50 +1140,13 @@ std::unique_ptr StorageAzureSource::createAzureReadBuffer(const Stri if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); - return createAsyncAzureReadBuffer(key, read_settings, object_size); + return object_storage->readObjects({StoredObject(key)}, read_settings, {}, object_size); } return object_storage->readObject(StoredObject(key), read_settings, {}, object_size); } -std::unique_ptr StorageAzureSource::createAsyncAzureReadBuffer( - const String & key, const ReadSettings & read_settings, size_t object_size) -{ - auto context = getContext(); - auto read_buffer_creator = - [this, read_settings, object_size] - (const std::string & path, size_t read_until_position) -> std::unique_ptr - { - auto buffer = object_storage->readObject(StoredObject(path), read_settings, {}, object_size); - buffer->setReadUntilPosition(read_until_position); - return buffer; - }; - - auto s3_impl = std::make_unique( - std::move(read_buffer_creator), - StoredObjects{StoredObject{key, object_size}}, - read_settings, - /* cache_log */nullptr); - - auto modified_settings{read_settings}; - /// FIXME: Changing this setting to default value breaks something around parquet reading - modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size; - - auto & pool_reader = context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - auto async_reader = std::make_unique( - std::move(s3_impl), pool_reader, modified_settings, - context->getAsyncReadCounters(), context->getFilesystemReadPrefetchesLog()); - - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - return async_reader; -} - - } #endif diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index 168015cf5d9..f114184c336 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -274,7 +274,6 @@ private: std::future createReaderAsync(); std::unique_ptr createAzureReadBuffer(const String & key, size_t object_size); - std::unique_ptr createAsyncAzureReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); }; } diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 6a3c915cdc5..8fa87c4e286 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -188,3 +188,16 @@ def test_truncate(cluster): azure_query(node, "TRUNCATE TABLE test_truncate") with pytest.raises(Exception): print(get_azure_file_content("test_truncate.csv")) + + +def test_simple_read_write(cluster): + node = cluster.instances["node"] + azure_query( + node, + "CREATE TABLE test_simple_read_write (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_simple_read_write.csv', format='CSV')", + ) + + azure_query(node, "INSERT INTO test_simple_read_write VALUES (1, 'a')") + assert get_azure_file_content("test_simple_read_write.csv") == '1,"a"\n' + + print(azure_query(node, "SELECT * FROM test_simple_read_write")) From 3d99abee436b95fd16d8d4011283d12fb9aebb5f Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 12:40:28 +0200 Subject: [PATCH 1236/2223] Remove async reads --- src/Storages/StorageAzure.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 2f0029947f5..804476a1842 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -1131,17 +1131,17 @@ std::unique_ptr StorageAzureSource::createAzureReadBuffer(const Stri { auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); read_settings.enable_filesystem_cache = false; - auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; - const bool object_too_small = object_size <= 2 * download_buffer_size; + //auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; + //const bool object_too_small = object_size <= 2 * download_buffer_size; // Create a read buffer that will prefetch the first ~1 MB of the file. // When reading lots of tiny files, this prefetching almost doubles the throughput. // For bigger files, parallel reading is more useful. - if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - { - LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); - return object_storage->readObjects({StoredObject(key)}, read_settings, {}, object_size); - } + //if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) + //{ + // LOG_TRACE(log, "Downloading object {} of size {} from S3 with initial prefetch", key, object_size); + // return object_storage->readObjects({StoredObject(key)}, read_settings, {}, object_size); + //} return object_storage->readObject(StoredObject(key), read_settings, {}, object_size); } From 8d044e8880a588a7854a6f6e2513226a6335ff48 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Jun 2023 10:57:13 +0000 Subject: [PATCH 1237/2223] Increase max array size in group bitmap --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 2 +- tests/queries/0_stateless/02782_bitmap_overflow.reference | 0 tests/queries/0_stateless/02782_bitmap_overflow.sql | 2 ++ 3 files changed, 3 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02782_bitmap_overflow.reference create mode 100644 tests/queries/0_stateless/02782_bitmap_overflow.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index d99f0bf16ee..7ea1ebe7749 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -122,7 +122,7 @@ public: size_t size; readVarUInt(size, in); - static constexpr size_t max_size = 1_GiB; + static constexpr size_t max_size = 100_GiB; if (size == 0) throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect size (0) in groupBitmap."); diff --git a/tests/queries/0_stateless/02782_bitmap_overflow.reference b/tests/queries/0_stateless/02782_bitmap_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02782_bitmap_overflow.sql b/tests/queries/0_stateless/02782_bitmap_overflow.sql new file mode 100644 index 00000000000..656a3e7c144 --- /dev/null +++ b/tests/queries/0_stateless/02782_bitmap_overflow.sql @@ -0,0 +1,2 @@ +select unhex('0181808080908380808000')::AggregateFunction(groupBitmap, UInt64); -- {serverError TOO_LARGE_ARRAY_SIZE} + From d497562a07b64434dc0c6b892e666776c51fa693 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 13:01:21 +0200 Subject: [PATCH 1238/2223] Copy and paste --- .../IO/ReadBufferFromAzureBlobStorage.cpp | 60 +++++++++++++++++-- src/Disks/IO/ReadBufferFromAzureBlobStorage.h | 9 +++ src/Storages/StorageAzure.cpp | 2 +- .../test_storage_azure_blob_storage/test.py | 24 +++++++- 4 files changed, 89 insertions(+), 6 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 0f197c2ff06..a086eb0a6df 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -35,6 +35,7 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( size_t max_single_read_retries_, size_t max_single_download_retries_, bool use_external_buffer_, + bool restricted_seek_, size_t read_until_position_) : ReadBufferFromFileBase(use_external_buffer_ ? 0 : read_settings_.remote_fs_buffer_size, nullptr, 0) , blob_container_client(blob_container_client_) @@ -44,6 +45,7 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( , read_settings(read_settings_) , tmp_buffer_size(read_settings.remote_fs_buffer_size) , use_external_buffer(use_external_buffer_) + , restricted_seek(restricted_seek_) , read_until_position(read_until_position_) { if (!use_external_buffer) @@ -118,8 +120,17 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() off_t ReadBufferFromAzureBlobStorage::seek(off_t offset_, int whence) { - if (initialized) - throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Seek is allowed only before first read attempt from the buffer."); + if (offset_ == getPosition() && whence == SEEK_SET) + return offset_; + + if (initialized && restricted_seek) + { + throw Exception( + ErrorCodes::CANNOT_SEEK_THROUGH_FILE, + "Seek is allowed only before first read attempt from the buffer (current offset: " + "{}, new offset: {}, reading until position: {}, available: {})", + getPosition(), offset_, read_until_position, available()); + } if (whence != SEEK_SET) throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET mode is allowed."); @@ -127,8 +138,36 @@ off_t ReadBufferFromAzureBlobStorage::seek(off_t offset_, int whence) if (offset_ < 0) throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bounds. Offset: {}", offset_); - offset = offset_; + if (!restricted_seek) + { + if (!working_buffer.empty() + && static_cast(offset_) >= offset - working_buffer.size() + && offset_ < offset) + { + pos = working_buffer.end() - (offset - offset_); + assert(pos >= working_buffer.begin()); + assert(pos < working_buffer.end()); + return getPosition(); + } + + off_t position = getPosition(); + if (initialized && offset_ > position) + { + size_t diff = offset_ - position; + if (diff < read_settings.remote_read_min_bytes_for_seek) + { + ignore(diff); + return offset_; + } + } + + resetWorkingBuffer(); + if (initialized) + initialized = false; + } + + offset = offset_; return offset; } @@ -152,7 +191,8 @@ void ReadBufferFromAzureBlobStorage::initialize() download_options.Range = {static_cast(offset), length}; - blob_client = std::make_unique(blob_container_client->GetBlobClient(path)); + if (!blob_client) + blob_client = std::make_unique(blob_container_client->GetBlobClient(path)); size_t sleep_time_with_backoff_milliseconds = 100; for (size_t i = 0; i < max_single_download_retries; ++i) @@ -182,6 +222,18 @@ void ReadBufferFromAzureBlobStorage::initialize() initialized = true; } +size_t ReadBufferFromAzureBlobStorage::getFileSize() +{ + if (!blob_client) + blob_client = std::make_unique(blob_container_client->GetBlobClient(path)); + + if (file_size.has_value()) + return *file_size; + + file_size = blob_client->GetProperties().Value.BlobSize; + return *file_size; +} + } #endif diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h index 6164a005773..599ecba1dd1 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h @@ -24,6 +24,7 @@ public: size_t max_single_read_retries_, size_t max_single_download_retries_, bool use_external_buffer_ = false, + bool restricted_seek_ = false, size_t read_until_position_ = 0); off_t seek(off_t off, int whence) override; @@ -40,6 +41,8 @@ public: bool supportsRightBoundedReads() const override { return true; } + size_t getFileSize() override; + private: void initialize(); @@ -55,6 +58,12 @@ private: std::vector tmp_buffer; size_t tmp_buffer_size; bool use_external_buffer; + + /// There is different seek policy for disk seek and for non-disk seek + /// (non-disk seek is applied for seekable input formats: orc, arrow, parquet). + bool restricted_seek; + + off_t read_until_position = 0; off_t offset = 0; diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 804476a1842..d83dc90e6ed 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -453,7 +453,7 @@ void StorageAzure::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextP for (const auto & key : configuration.blobs_paths) objects.emplace_back(key); - object_storage->removeObjects(objects); + object_storage->removeObjectsIfExist(objects); } namespace diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 8fa87c4e286..bcf5d068057 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -199,5 +199,27 @@ def test_simple_read_write(cluster): azure_query(node, "INSERT INTO test_simple_read_write VALUES (1, 'a')") assert get_azure_file_content("test_simple_read_write.csv") == '1,"a"\n' - print(azure_query(node, "SELECT * FROM test_simple_read_write")) + assert azure_query(node, "SELECT * FROM test_simple_read_write") == "1\ta\n" + + +def test_create_new_files_on_insert(cluster): + + node = cluster.instances["node"] + + azure_query(node, f"create table test_multiple_inserts(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_parquet', format='Parquet')") + azure_query(node, "truncate table test_multiple_inserts") + azure_query(node, + f"insert into test_multiple_inserts select number, randomString(100) from numbers(10) settings azure_truncate_on_insert=1" + ) + azure_query(node, + f"insert into test_multiple_inserts select number, randomString(100) from numbers(20) settings azure_create_new_file_on_insert=1" + ) + azure_query(node, + f"insert into test_multiple_inserts select number, randomString(100) from numbers(30) settings azure_create_new_file_on_insert=1" + ) + + result = azure_query(node, f"select count() from test_multiple_inserts") + assert int(result) == 60 + + azure_query(node, f"drop table test_multiple_inserts") From cf886d8ced474a1a80a985587af251a39701d1b2 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 6 Jun 2023 11:08:21 +0000 Subject: [PATCH 1239/2223] Remove IsConvertible() --- src/Functions/DateTimeTransforms.h | 46 +++++++++++++++-------------- src/Functions/FunctionsConversion.h | 41 ------------------------- 2 files changed, 24 insertions(+), 63 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index d154dd9ffa2..823272e0324 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1444,31 +1444,33 @@ struct Transformer for (size_t i = 0; i < size; ++i) { - constexpr bool transformHasIsConvertible = requires(const Transform& t) + if constexpr (std::is_same_v + || std::is_same_v) { - t.IsConvertible(vec_from[i], time_zone); - }; + bool check_range_result = true; - if constexpr (transformHasIsConvertible) - { - if constexpr (std::is_same_v - || std::is_same_v) + if constexpr (std::is_same_v) { - bool checked = transform.IsConvertible(vec_from[i], time_zone); - if (!checked) + check_range_result = vec_from[i] >= 0 && vec_from[i] <= DATE_LUT_MAX_DAY_NUM; + } + else if constexpr (std::is_same_v) + { + check_range_result = vec_from[i] >= 0 && vec_from[i] <= 0xFFFFFFFFL; + } + + if (!check_range_result) + { + if (std::is_same_v) { - if (std::is_same_v) - { - vec_to[i] = 0; - if (vec_null_map_to) - (*vec_null_map_to)[i] = true; - continue; - } - else - { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Value in column {} cannot be safely converted into type {}", - TypeName, TypeName); - } + vec_to[i] = 0; + if (vec_null_map_to) + (*vec_null_map_to)[i] = true; + continue; + } + else + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Value in column {} cannot be safely converted into type {}", + TypeName, TypeName); } } } @@ -1488,7 +1490,7 @@ struct DateTimeTransformImpl static ColumnPtr execute( const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/, const Transform & transform = {}) { - using Op = Transformer; + using Op = Transformer; const ColumnPtr source_col = arguments[0].column; if (const auto * sources = checkAndGetColumn(source_col.get())) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 6aa5843ff65..3a8ddcc9094 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -365,22 +365,11 @@ template struct ConvertImpl -static bool CheckDateRange(const FromType & value) -{ - return value >= 0 && value <= DATE_LUT_MAX_DAY_NUM; -} - template struct ToDateTransform32Or64 { static constexpr auto name = "toDate"; - static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) - { - return CheckDateRange(from); - } - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { // since converting to Date, no need in values outside of default LUT range. @@ -395,11 +384,6 @@ struct ToDateTransform32Or64Signed { static constexpr auto name = "toDate"; - static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) - { - return CheckDateRange(from); - } - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { // TODO: decide narrow or extended range based on FromType @@ -417,11 +401,6 @@ struct ToDateTransform8Or16Signed { static constexpr auto name = "toDate"; - static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) - { - return CheckDateRange(from); - } - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { if (from < 0) @@ -518,22 +497,12 @@ template struct ConvertImpl struct ConvertImpl : DateTimeTransformImpl> {}; -template -static bool CheckDateTimeRange(const FromType & value) -{ - return value >= 0 && value <= 0xFFFFFFFFL; -} template struct ToDateTimeTransform64 { static constexpr auto name = "toDateTime"; - static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) - { - return CheckDateTimeRange(from); - } - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { return static_cast(std::min(time_t(from), time_t(0xFFFFFFFF))); @@ -545,11 +514,6 @@ struct ToDateTimeTransformSigned { static constexpr auto name = "toDateTime"; - static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) - { - return CheckDateTimeRange(from); - } - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { if (from < 0) @@ -563,11 +527,6 @@ struct ToDateTimeTransform64Signed { static constexpr auto name = "toDateTime"; - static NO_SANITIZE_UNDEFINED bool IsConvertible(const FromType & from, const DateLUTImpl &) - { - return CheckDateTimeRange(from); - } - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { if (from < 0) From 18decb090ca6f85826dcddde2d61b53fa460ee4c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 6 Jun 2023 11:11:50 +0000 Subject: [PATCH 1240/2223] Automatic style fix --- .../test_storage_azure_blob_storage/test.py | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index bcf5d068057..7e66b6fb198 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -204,19 +204,24 @@ def test_simple_read_write(cluster): def test_create_new_files_on_insert(cluster): - node = cluster.instances["node"] - azure_query(node, f"create table test_multiple_inserts(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_parquet', format='Parquet')") + azure_query( + node, + f"create table test_multiple_inserts(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_parquet', format='Parquet')", + ) azure_query(node, "truncate table test_multiple_inserts") - azure_query(node, - f"insert into test_multiple_inserts select number, randomString(100) from numbers(10) settings azure_truncate_on_insert=1" + azure_query( + node, + f"insert into test_multiple_inserts select number, randomString(100) from numbers(10) settings azure_truncate_on_insert=1", ) - azure_query(node, - f"insert into test_multiple_inserts select number, randomString(100) from numbers(20) settings azure_create_new_file_on_insert=1" + azure_query( + node, + f"insert into test_multiple_inserts select number, randomString(100) from numbers(20) settings azure_create_new_file_on_insert=1", ) - azure_query(node, - f"insert into test_multiple_inserts select number, randomString(100) from numbers(30) settings azure_create_new_file_on_insert=1" + azure_query( + node, + f"insert into test_multiple_inserts select number, randomString(100) from numbers(30) settings azure_create_new_file_on_insert=1", ) result = azure_query(node, f"select count() from test_multiple_inserts") From 6b41a02f7fbdd46ee2774651fea3518d26844407 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 6 Jun 2023 13:14:52 +0200 Subject: [PATCH 1241/2223] resolve tests --- src/Client/Suggest.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 4ffa828dd40..e249aa1bb04 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -101,7 +101,9 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti add_column("name", "columns", true, suggestion_limit); } - query = "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" + query + ") WHERE notEmpty(res)"; + /// FIXME: Forbid this query using new analyzer because of bug https://github.com/ClickHouse/ClickHouse/pull/50430#issuecomment-1576860893 + /// We should remove this restriction after resolving this bug. + query = "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" + query + ") WHERE notEmpty(res) SETTINGS allow_experimental_analyzer=0"; return query; } From e87348010d3e77d60b8ccd85e7bd4574bec9600b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 6 Jun 2023 14:42:56 +0200 Subject: [PATCH 1242/2223] Rework loading and removing of data parts for MergeTree tables. (#49474) Co-authored-by: Sergei Trifonov --- programs/local/LocalServer.cpp | 24 +- programs/server/Server.cpp | 53 +++- src/Backups/BackupIO_S3.cpp | 8 +- src/Common/CurrentMetrics.cpp | 2 + src/Core/ServerSettings.h | 4 +- src/Formats/FormatFactory.cpp | 2 +- src/IO/SharedThreadPools.cpp | 151 ++++++---- src/IO/SharedThreadPools.h | 74 +++-- src/Interpreters/threadPoolCallbackRunner.h | 3 + src/Storages/MergeTree/MergeTreeData.cpp | 284 +++++++----------- src/Storages/MergeTree/MergeTreeData.h | 6 +- src/Storages/MergeTree/MergeTreeSettings.h | 5 +- src/Storages/MergeTree/MergeTreeSource.cpp | 2 +- src/Storages/StorageS3.cpp | 2 +- .../System/StorageSystemDetachedParts.cpp | 2 +- .../00988_parallel_parts_removal.sql | 2 +- .../00989_parallel_parts_loading.sql | 2 +- .../01810_max_part_removal_threads_long.sh | 11 +- .../02432_s3_parallel_parts_cleanup.sql | 5 +- 19 files changed, 352 insertions(+), 290 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 96c1ca261b5..caca7cfb50d 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -130,15 +130,31 @@ void LocalServer::initialize(Poco::Util::Application & self) }); #endif - IOThreadPool::initialize( + getIOThreadPool().initialize( config().getUInt("max_io_thread_pool_size", 100), config().getUInt("max_io_thread_pool_free_size", 0), config().getUInt("io_thread_pool_queue_size", 10000)); - OutdatedPartsLoadingThreadPool::initialize( - config().getUInt("max_outdated_parts_loading_thread_pool_size", 16), + + const size_t active_parts_loading_threads = config().getUInt("max_active_parts_loading_thread_pool_size", 64); + getActivePartsLoadingThreadPool().initialize( + active_parts_loading_threads, 0, // We don't need any threads one all the parts will be loaded - config().getUInt("max_outdated_parts_loading_thread_pool_size", 16)); + active_parts_loading_threads); + + const size_t outdated_parts_loading_threads = config().getUInt("max_outdated_parts_loading_thread_pool_size", 32); + getOutdatedPartsLoadingThreadPool().initialize( + outdated_parts_loading_threads, + 0, // We don't need any threads one all the parts will be loaded + outdated_parts_loading_threads); + + getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(active_parts_loading_threads); + + const size_t cleanup_threads = config().getUInt("max_parts_cleaning_thread_pool_size", 128); + getPartsCleaningThreadPool().initialize( + cleanup_threads, + 0, // We don't need any threads one all the parts will be deleted + cleanup_threads); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 9eb3e6c9ebc..d0fc8aca5e8 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -683,21 +683,36 @@ try }); #endif - IOThreadPool::initialize( + getIOThreadPool().initialize( server_settings.max_io_thread_pool_size, server_settings.max_io_thread_pool_free_size, server_settings.io_thread_pool_queue_size); - BackupsIOThreadPool::initialize( + getBackupsIOThreadPool().initialize( server_settings.max_backups_io_thread_pool_size, server_settings.max_backups_io_thread_pool_free_size, server_settings.backups_io_thread_pool_queue_size); - OutdatedPartsLoadingThreadPool::initialize( + getActivePartsLoadingThreadPool().initialize( + server_settings.max_active_parts_loading_thread_pool_size, + 0, // We don't need any threads once all the parts will be loaded + server_settings.max_active_parts_loading_thread_pool_size); + + getOutdatedPartsLoadingThreadPool().initialize( server_settings.max_outdated_parts_loading_thread_pool_size, - 0, // We don't need any threads one all the parts will be loaded + 0, // We don't need any threads once all the parts will be loaded server_settings.max_outdated_parts_loading_thread_pool_size); + /// It could grow if we need to synchronously wait until all the data parts will be loaded. + getOutdatedPartsLoadingThreadPool().setMaxTurboThreads( + server_settings.max_active_parts_loading_thread_pool_size + ); + + getPartsCleaningThreadPool().initialize( + server_settings.max_parts_cleaning_thread_pool_size, + 0, // We don't need any threads one all the parts will be deleted + server_settings.max_parts_cleaning_thread_pool_size); + /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) { @@ -1226,6 +1241,36 @@ try global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size); global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size); + getIOThreadPool().reloadConfiguration( + server_settings.max_io_thread_pool_size, + server_settings.max_io_thread_pool_free_size, + server_settings.io_thread_pool_queue_size); + + getBackupsIOThreadPool().reloadConfiguration( + server_settings.max_backups_io_thread_pool_size, + server_settings.max_backups_io_thread_pool_free_size, + server_settings.backups_io_thread_pool_queue_size); + + getActivePartsLoadingThreadPool().reloadConfiguration( + server_settings.max_active_parts_loading_thread_pool_size, + 0, // We don't need any threads once all the parts will be loaded + server_settings.max_active_parts_loading_thread_pool_size); + + getOutdatedPartsLoadingThreadPool().reloadConfiguration( + server_settings.max_outdated_parts_loading_thread_pool_size, + 0, // We don't need any threads once all the parts will be loaded + server_settings.max_outdated_parts_loading_thread_pool_size); + + /// It could grow if we need to synchronously wait until all the data parts will be loaded. + getOutdatedPartsLoadingThreadPool().setMaxTurboThreads( + server_settings.max_active_parts_loading_thread_pool_size + ); + + getPartsCleaningThreadPool().reloadConfiguration( + server_settings.max_parts_cleaning_thread_pool_size, + 0, // We don't need any threads one all the parts will be deleted + server_settings.max_parts_cleaning_thread_pool_size); + if (config->has("resources")) { global_context->getResourceManager()->updateConfiguration(*config); diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index f1fd276e34b..967beba4bf5 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -161,7 +161,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s /* dest_key= */ blob_path[0], request_settings, object_attributes, - threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupReaderS3"), + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupReaderS3"), /* for_disk_s3= */ true); return file_size; @@ -212,7 +212,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src fs::path(s3_uri.key) / path_in_backup, request_settings, {}, - threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); return; /// copied! } } @@ -224,7 +224,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {}, - threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); } BackupWriterS3::~BackupWriterS3() = default; @@ -258,7 +258,7 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) DBMS_DEFAULT_BUFFER_SIZE, request_settings, std::nullopt, - threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3"), + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3"), write_settings); } diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 956487a300e..61725d079bf 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -137,6 +137,8 @@ M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \ M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \ M(MergeTreePartsLoaderThreadsActive, "Number of threads in the MergeTree parts loader thread pool running a task.") \ + M(MergeTreeOutdatedPartsLoaderThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ + M(MergeTreeOutdatedPartsLoaderThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ M(MergeTreePartsCleanerThreads, "Number of threads in the MergeTree parts cleaner thread pool.") \ M(MergeTreePartsCleanerThreadsActive, "Number of threads in the MergeTree parts cleaner thread pool running a task.") \ M(SystemReplicasThreads, "Number of threads in the system.replicas thread pool.") \ diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index cb43d62ecd1..1a9f226041b 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -21,7 +21,9 @@ namespace DB M(UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0) \ M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \ M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ - M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The maximum number of threads that would be used for loading outdated data parts on startup", 0) \ + M(UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0) \ + M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0) \ + M(UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0) \ M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 6f2974c49c6..39b28e025a6 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -364,7 +364,7 @@ std::unique_ptr FormatFactory::wrapReadBufferIfNeeded( settings.max_download_buffer_size); res = wrapInParallelReadBufferIfSupported( - buf, threadPoolCallbackRunner(IOThreadPool::get(), "ParallelRead"), + buf, threadPoolCallbackRunner(getIOThreadPool().get(), "ParallelRead"), max_download_threads, settings.max_download_buffer_size, file_size); } diff --git a/src/IO/SharedThreadPools.cpp b/src/IO/SharedThreadPools.cpp index b7b6aea1567..6a0e953f0ef 100644 --- a/src/IO/SharedThreadPools.cpp +++ b/src/IO/SharedThreadPools.cpp @@ -9,8 +9,12 @@ namespace CurrentMetrics extern const Metric IOThreadsActive; extern const Metric BackupsIOThreads; extern const Metric BackupsIOThreadsActive; - extern const Metric OutdatedPartsLoadingThreads; - extern const Metric OutdatedPartsLoadingThreadsActive; + extern const Metric MergeTreePartsLoaderThreads; + extern const Metric MergeTreePartsLoaderThreadsActive; + extern const Metric MergeTreePartsCleanerThreads; + extern const Metric MergeTreePartsCleanerThreadsActive; + extern const Metric MergeTreeOutdatedPartsLoaderThreads; + extern const Metric MergeTreeOutdatedPartsLoaderThreadsActive; } namespace DB @@ -21,88 +25,117 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -std::unique_ptr IOThreadPool::instance; -void IOThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size) +StaticThreadPool::StaticThreadPool( + const String & name_, + CurrentMetrics::Metric threads_metric_, + CurrentMetrics::Metric threads_active_metric_) + : name(name_) + , threads_metric(threads_metric_) + , threads_active_metric(threads_active_metric_) +{ +} + +void StaticThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size) { if (instance) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "The IO thread pool is initialized twice"); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is initialized twice", name); + /// By default enabling "turbo mode" won't affect the number of threads anyhow + max_threads_turbo = max_threads; + max_threads_normal = max_threads; instance = std::make_unique( - CurrentMetrics::IOThreads, - CurrentMetrics::IOThreadsActive, + threads_metric, + threads_active_metric, max_threads, max_free_threads, queue_size, /* shutdown_on_exception= */ false); } -ThreadPool & IOThreadPool::get() +void StaticThreadPool::reloadConfiguration(size_t max_threads, size_t max_free_threads, size_t queue_size) { if (!instance) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "The IO thread pool is not initialized"); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is not initialized", name); + + instance->setMaxThreads(turbo_mode_enabled > 0 ? max_threads_turbo : max_threads); + instance->setMaxFreeThreads(max_free_threads); + instance->setQueueSize(queue_size); +} + + +ThreadPool & StaticThreadPool::get() +{ + if (!instance) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is not initialized", name); return *instance; } -std::unique_ptr BackupsIOThreadPool::instance; - -void BackupsIOThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size) -{ - if (instance) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "The BackupsIO thread pool is initialized twice"); - } - - instance = std::make_unique( - CurrentMetrics::BackupsIOThreads, - CurrentMetrics::BackupsIOThreadsActive, - max_threads, - max_free_threads, - queue_size, - /* shutdown_on_exception= */ false); -} - -ThreadPool & BackupsIOThreadPool::get() +void StaticThreadPool::enableTurboMode() { if (!instance) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "The BackupsIO thread pool is not initialized"); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is not initialized", name); - return *instance; + std::lock_guard lock(mutex); + + ++turbo_mode_enabled; + if (turbo_mode_enabled == 1) + instance->setMaxThreads(max_threads_turbo); } -std::unique_ptr OutdatedPartsLoadingThreadPool::instance; - -void OutdatedPartsLoadingThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size) -{ - if (instance) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "The PartsLoadingThreadPool thread pool is initialized twice"); - } - - instance = std::make_unique( - CurrentMetrics::OutdatedPartsLoadingThreads, - CurrentMetrics::OutdatedPartsLoadingThreadsActive, - max_threads, - max_free_threads, - queue_size, - /* shutdown_on_exception= */ false); -} - -ThreadPool & OutdatedPartsLoadingThreadPool::get() +void StaticThreadPool::disableTurboMode() { if (!instance) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "The PartsLoadingThreadPool thread pool is not initialized"); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is not initialized", name); - return *instance; + std::lock_guard lock(mutex); + + --turbo_mode_enabled; + if (turbo_mode_enabled == 0) + instance->setMaxThreads(max_threads_normal); +} + +void StaticThreadPool::setMaxTurboThreads(size_t max_threads_turbo_) +{ + if (!instance) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is not initialized", name); + + std::lock_guard lock(mutex); + + max_threads_turbo = max_threads_turbo_; + if (turbo_mode_enabled > 0) + instance->setMaxThreads(max_threads_turbo); +} + +StaticThreadPool & getIOThreadPool() +{ + static StaticThreadPool instance("IOThreadPool", CurrentMetrics::IOThreads, CurrentMetrics::IOThreadsActive); + return instance; +} + +StaticThreadPool & getBackupsIOThreadPool() +{ + static StaticThreadPool instance("BackupsIOThreadPool", CurrentMetrics::BackupsIOThreads, CurrentMetrics::BackupsIOThreadsActive); + return instance; +} + +StaticThreadPool & getActivePartsLoadingThreadPool() +{ + static StaticThreadPool instance("MergeTreePartsLoaderThreadPool", CurrentMetrics::MergeTreePartsLoaderThreads, CurrentMetrics::MergeTreePartsLoaderThreadsActive); + return instance; +} + +StaticThreadPool & getPartsCleaningThreadPool() +{ + static StaticThreadPool instance("MergeTreePartsCleanerThreadPool", CurrentMetrics::MergeTreePartsCleanerThreads, CurrentMetrics::MergeTreePartsCleanerThreadsActive); + return instance; +} + +StaticThreadPool & getOutdatedPartsLoadingThreadPool() +{ + static StaticThreadPool instance("MergeTreeOutdatedPartsLoaderThreadPool", CurrentMetrics::MergeTreeOutdatedPartsLoaderThreads, CurrentMetrics::MergeTreeOutdatedPartsLoaderThreadsActive); + return instance; } } diff --git a/src/IO/SharedThreadPools.h b/src/IO/SharedThreadPools.h index 1b43dfe778c..188a2a4f003 100644 --- a/src/IO/SharedThreadPools.h +++ b/src/IO/SharedThreadPools.h @@ -1,48 +1,64 @@ #pragma once +#include #include +#include + #include #include +#include namespace DB { -/* - * ThreadPool used for the IO. - */ -class IOThreadPool +class StaticThreadPool { - static std::unique_ptr instance; - public: - static void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size); - static ThreadPool & get(); + StaticThreadPool( + const String & name_, + CurrentMetrics::Metric threads_metric_, + CurrentMetrics::Metric threads_active_metric_); + + ThreadPool & get(); + + void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size); + void reloadConfiguration(size_t max_threads, size_t max_free_threads, size_t queue_size); + + /// At runtime we can increase the number of threads up the specified limit + /// This is needed to utilize as much a possible resources to accomplish some task. + void setMaxTurboThreads(size_t max_threads_turbo_); + void enableTurboMode(); + void disableTurboMode(); + +private: + const String name; + const CurrentMetrics::Metric threads_metric; + const CurrentMetrics::Metric threads_active_metric; + + std::unique_ptr instance; + std::mutex mutex; + size_t max_threads_turbo = 0; + size_t max_threads_normal = 0; + /// If this counter is > 0 - this specific mode is enabled + size_t turbo_mode_enabled = 0; }; +/// ThreadPool used for the IO. +StaticThreadPool & getIOThreadPool(); -/* - * ThreadPool used for the Backup IO. - */ -class BackupsIOThreadPool -{ - static std::unique_ptr instance; +/// ThreadPool used for the Backup IO. +StaticThreadPool & getBackupsIOThreadPool(); -public: - static void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size); - static ThreadPool & get(); -}; +/// ThreadPool used for the loading of Outdated data parts for MergeTree tables. +StaticThreadPool & getActivePartsLoadingThreadPool(); +/// ThreadPool used for deleting data parts for MergeTree tables. +StaticThreadPool & getPartsCleaningThreadPool(); -/* - * ThreadPool used for the loading of Outdated data parts for MergeTree tables. - */ -class OutdatedPartsLoadingThreadPool -{ - static std::unique_ptr instance; - -public: - static void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size); - static ThreadPool & get(); -}; +/// This ThreadPool is used for the loading of Outdated data parts for MergeTree tables. +/// Normally we will just load Outdated data parts concurrently in background, but in +/// case when we need to synchronously wait for the loading to be finished, we can increase +/// the number of threads by calling enableTurboMode() :-) +StaticThreadPool & getOutdatedPartsLoadingThreadPool(); } diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index f7324bfafe6..eb90b61cf31 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -44,6 +44,9 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & auto future = task->get_future(); + /// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority". + /// Note: calling method scheduleOrThrowOnError in intentional, because we don't want to throw exceptions + /// in critical places where this callback runner is used (e.g. loading or deletion of parts) my_pool->scheduleOrThrowOnError([my_task = std::move(task)]{ (*my_task)(); }, priority); return future; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 32665429051..e806e1bb93f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -130,10 +130,6 @@ namespace ProfileEvents namespace CurrentMetrics { extern const Metric DelayedInserts; - extern const Metric MergeTreePartsLoaderThreads; - extern const Metric MergeTreePartsLoaderThreadsActive; - extern const Metric MergeTreePartsCleanerThreads; - extern const Metric MergeTreePartsCleanerThreadsActive; } @@ -1425,71 +1421,17 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPartWithRetries( UNREACHABLE(); } -std::vector MergeTreeData::loadDataPartsFromDisk( - ThreadPool & pool, - size_t num_parts, - std::queue & parts_queue, - const MergeTreeSettingsPtr & settings) +std::vector MergeTreeData::loadDataPartsFromDisk(PartLoadingTreeNodes & parts_to_load) { - /// Parallel loading of data parts. - pool.setMaxThreads(std::min(static_cast(settings->max_part_loading_threads), num_parts)); - size_t num_threads = pool.getMaxThreads(); - LOG_DEBUG(log, "Going to use {} threads to load parts", num_threads); + const size_t num_parts = parts_to_load.size(); - std::vector parts_per_thread(num_threads, num_parts / num_threads); - for (size_t i = 0ul; i < num_parts % num_threads; ++i) - ++parts_per_thread[i]; + LOG_DEBUG(log, "Will load {} number of parts using {} threads", num_parts, getActivePartsLoadingThreadPool().get().getMaxThreads()); - /// Prepare data parts for parallel loading. Threads will focus on given disk first, then steal - /// others' tasks when finish current disk part loading process. - std::vector threads_parts(num_threads); - std::set remaining_thread_parts; - std::queue threads_queue; + /// Shuffle all the parts randomly to possible speed up loading them from JBOD. + std::shuffle(parts_to_load.begin(), parts_to_load.end(), thread_local_rng); - for (size_t i = 0; i < num_threads; ++i) - { - remaining_thread_parts.insert(i); - threads_queue.push(i); - } - - while (!parts_queue.empty()) - { - assert(!threads_queue.empty()); - size_t i = threads_queue.front(); - auto & need_parts = parts_per_thread[i]; - assert(need_parts > 0); - - auto & thread_parts = threads_parts[i]; - auto & current_parts = parts_queue.front(); - assert(!current_parts.empty()); - - auto parts_to_grab = std::min(need_parts, current_parts.size()); - thread_parts.insert(thread_parts.end(), current_parts.end() - parts_to_grab, current_parts.end()); - current_parts.resize(current_parts.size() - parts_to_grab); - need_parts -= parts_to_grab; - - /// Before processing next thread, change disk if possible. - /// Different threads will likely start loading parts from different disk, - /// which may improve read parallelism for JBOD. - - /// If current disk still has some parts, push it to the tail. - if (!current_parts.empty()) - parts_queue.push(std::move(current_parts)); - - parts_queue.pop(); - - /// If current thread still want some parts, push it to the tail. - if (need_parts > 0) - threads_queue.push(i); - - threads_queue.pop(); - } - - assert(threads_queue.empty()); - assert(std::all_of(threads_parts.begin(), threads_parts.end(), [](const auto & parts) - { - return !parts.empty(); - })); + auto runner = threadPoolCallbackRunner(getActivePartsLoadingThreadPool().get(), "ActiveParts"); + std::vector> parts_futures; std::mutex part_select_mutex; std::mutex part_loading_mutex; @@ -1498,81 +1440,77 @@ std::vector MergeTreeData::loadDataPartsFromDisk( try { - for (size_t thread = 0; thread < num_threads; ++thread) + while (true) { - pool.scheduleOrThrowOnError([&, thread, thread_group = CurrentThread::getGroup()] + bool are_parts_to_load_empty = false; { - SCOPE_EXIT_SAFE( - if (thread_group) - CurrentThread::detachFromGroupIfNotDetached(); - ); - if (thread_group) - CurrentThread::attachToGroupIfDetached(thread_group); + std::lock_guard lock(part_select_mutex); + are_parts_to_load_empty = parts_to_load.empty(); + } - while (true) + if (are_parts_to_load_empty) + { + /// Wait for all scheduled tasks. + /// We have to use .get() method to rethrow any exception that could occur. + for (auto & future: parts_futures) + future.get(); + parts_futures.clear(); + /// At this point it is possible, that some other parts appeared in the queue for processing (parts_to_load), + /// because we added them from inside the pool. + /// So we need to recheck it. + } + + PartLoadingTree::NodePtr current_part; + { + std::lock_guard lock(part_select_mutex); + if (parts_to_load.empty()) + break; + + current_part = parts_to_load.back(); + parts_to_load.pop_back(); + } + + parts_futures.push_back(runner( + [&, part = std::move(current_part)]() { - PartLoadingTree::NodePtr thread_part; - size_t thread_idx = thread; - - { - std::lock_guard lock{part_select_mutex}; - - if (remaining_thread_parts.empty()) - return; - - /// Steal task if nothing to do - if (threads_parts[thread].empty()) - { - // Try random steal tasks from the next thread - std::uniform_int_distribution distribution(0, remaining_thread_parts.size() - 1); - auto it = remaining_thread_parts.begin(); - std::advance(it, distribution(thread_local_rng)); - thread_idx = *it; - } - - auto & thread_parts = threads_parts[thread_idx]; - thread_part = thread_parts.back(); - thread_parts.pop_back(); - if (thread_parts.empty()) - remaining_thread_parts.erase(thread_idx); - } - /// Pass a separate mutex to guard the set of parts, because this lambda /// is called concurrently but with already locked @data_parts_mutex. auto res = loadDataPartWithRetries( - thread_part->info, thread_part->name, thread_part->disk, + part->info, part->name, part->disk, DataPartState::Active, part_loading_mutex, loading_parts_initial_backoff_ms, loading_parts_max_backoff_ms, loading_parts_max_tries); - thread_part->is_loaded = true; + part->is_loaded = true; bool is_active_part = res.part->getState() == DataPartState::Active; /// If part is broken or duplicate or should be removed according to transaction /// and it has any covered parts then try to load them to replace this part. - if (!is_active_part && !thread_part->children.empty()) + if (!is_active_part && !part->children.empty()) { std::lock_guard lock{part_select_mutex}; - for (const auto & [_, node] : thread_part->children) - threads_parts[thread].push_back(node); - remaining_thread_parts.insert(thread); + for (const auto & [_, node] : part->children) + parts_to_load.push_back(node); } { std::lock_guard lock(part_loading_mutex); loaded_parts.push_back(std::move(res)); } - } - }); + }, Priority{0})); } } catch (...) { - /// If this is not done, then in case of an exception, tasks will be destroyed before the threads are completed, and it will be bad. - pool.wait(); + /// Wait for all scheduled tasks + /// A future becomes invalid after .get() call + /// + .wait() method is used not to throw any exception here. + for (auto & future: parts_futures) + if (future.valid()) + future.wait(); + throw; } - pool.wait(); return loaded_parts; } @@ -1679,9 +1617,12 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } } - ThreadPool pool(CurrentMetrics::MergeTreePartsLoaderThreads, CurrentMetrics::MergeTreePartsLoaderThreadsActive, disks.size()); + auto runner = threadPoolCallbackRunner(getActivePartsLoadingThreadPool().get(), "ActiveParts"); std::vector parts_to_load_by_disk(disks.size()); + std::vector> disks_futures; + disks_futures.reserve(disks.size()); + for (size_t i = 0; i < disks.size(); ++i) { const auto & disk_ptr = disks[i]; @@ -1690,7 +1631,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) auto & disk_parts = parts_to_load_by_disk[i]; - pool.scheduleOrThrowOnError([&, disk_ptr]() + disks_futures.push_back(runner([&, disk_ptr]() { for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next()) { @@ -1703,38 +1644,31 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if (auto part_info = MergeTreePartInfo::tryParsePartName(it->name(), format_version)) disk_parts.emplace_back(*part_info, it->name(), disk_ptr); } - }); + }, Priority{0})); } - pool.wait(); + /// For iteration to be completed + /// Any exception will be re-thrown. + for (auto & future : disks_futures) + future.get(); + disks_futures.clear(); PartLoadingTree::PartLoadingInfos parts_to_load; for (auto & disk_parts : parts_to_load_by_disk) std::move(disk_parts.begin(), disk_parts.end(), std::back_inserter(parts_to_load)); auto loading_tree = PartLoadingTree::build(std::move(parts_to_load)); - /// Collect parts by disks' names. - std::map disk_part_map; + + size_t num_parts = 0; + PartLoadingTreeNodes active_parts; /// Collect only "the most covering" parts from the top level of the tree. loading_tree.traverse(/*recursive=*/ false, [&](const auto & node) { - disk_part_map[node->disk->getName()].emplace_back(node); + active_parts.emplace_back(node); }); - size_t num_parts = 0; - std::queue parts_queue; - - for (auto & [disk_name, disk_parts] : disk_part_map) - { - LOG_INFO(log, "Found {} parts for disk '{}' to load", disk_parts.size(), disk_name); - - if (disk_parts.empty()) - continue; - - num_parts += disk_parts.size(); - parts_queue.push(std::move(disk_parts)); - } + num_parts += active_parts.size(); auto part_lock = lockParts(); LOG_TEST(log, "loadDataParts: clearing data_parts_indexes (had {} parts)", data_parts_indexes.size()); @@ -1754,7 +1688,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if (num_parts > 0) { - auto loaded_parts = loadDataPartsFromDisk(pool, num_parts, parts_queue, settings); + auto loaded_parts = loadDataPartsFromDisk(active_parts); for (const auto & res : loaded_parts) { @@ -1783,10 +1717,12 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if (settings->in_memory_parts_enable_wal) { - pool.setMaxThreads(disks.size()); std::vector disks_wal_parts(disks.size()); std::mutex wal_init_lock; + std::vector> wal_disks_futures; + wal_disks_futures.reserve(disks.size()); + for (size_t i = 0; i < disks.size(); ++i) { const auto & disk_ptr = disks[i]; @@ -1795,7 +1731,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) auto & disk_wal_parts = disks_wal_parts[i]; - pool.scheduleOrThrowOnError([&, disk_ptr]() + wal_disks_futures.push_back(runner([&, disk_ptr]() { for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next()) { @@ -1821,10 +1757,14 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) disk_wal_parts.push_back(std::move(part)); } } - }); + }, Priority{0})); } - pool.wait(); + /// For for iteration to be completed + /// Any exception will be re-thrown. + for (auto & future : wal_disks_futures) + future.get(); + wal_disks_futures.clear(); MutableDataPartsVector parts_from_wal; for (auto & disk_wal_parts : disks_wal_parts) @@ -1925,7 +1865,7 @@ try std::atomic_size_t num_loaded_parts = 0; - auto runner = threadPoolCallbackRunner(OutdatedPartsLoadingThreadPool::get(), "OutdatedParts"); + auto runner = threadPoolCallbackRunner(getOutdatedPartsLoadingThreadPool().get(), "OutdatedParts"); std::vector> parts_futures; while (true) @@ -1938,8 +1878,10 @@ try if (is_async && outdated_data_parts_loading_canceled) { /// Wait for every scheduled task + /// In case of any exception it will be re-thrown and server will be terminated. for (auto & future : parts_futures) - future.wait(); + future.get(); + parts_futures.clear(); LOG_DEBUG(log, "Stopped loading outdated data parts because task was canceled. " @@ -1973,7 +1915,7 @@ try /// Wait for every scheduled task for (auto & future : parts_futures) - future.wait(); + future.get(); LOG_DEBUG(log, "Loaded {} outdated data parts {}", num_loaded_parts, is_async ? "asynchronously" : "synchronously"); @@ -1999,6 +1941,13 @@ void MergeTreeData::waitForOutdatedPartsToBeLoaded() const TSA_NO_THREAD_SAFETY_ if (isStaticStorage()) return; + /// We need to load parts as fast as possible + getOutdatedPartsLoadingThreadPool().enableTurboMode(); + SCOPE_EXIT({ + /// Let's lower the number of threads e.g. for later ATTACH queries to behave as usual + getOutdatedPartsLoadingThreadPool().disableTurboMode(); + }); + LOG_TRACE(log, "Will wait for outdated data parts to be loaded"); std::unique_lock lock(outdated_data_parts_mutex); @@ -2420,20 +2369,15 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t } }; - if (settings->max_part_removal_threads <= 1 || parts_to_remove.size() <= settings->concurrent_part_removal_threshold) + if (parts_to_remove.size() <= settings->concurrent_part_removal_threshold) { remove_single_thread(); return; } /// Parallel parts removal. - size_t num_threads = settings->max_part_removal_threads; - if (!num_threads) - num_threads = getNumberOfPhysicalCPUCores() * 2; - num_threads = std::min(num_threads, parts_to_remove.size()); std::mutex part_names_mutex; - ThreadPool pool(CurrentMetrics::MergeTreePartsCleanerThreads, CurrentMetrics::MergeTreePartsCleanerThreadsActive, - num_threads, num_threads, /* unlimited queue size */ 0); + auto runner = threadPoolCallbackRunner(getPartsCleaningThreadPool().get(), "PartsCleaning"); /// This flag disallow straightforward concurrent parts removal. It's required only in case /// when we have parts on zero-copy disk + at least some of them were mutated. @@ -2453,27 +2397,27 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t LOG_DEBUG( log, "Removing {} parts from filesystem (concurrently): Parts: [{}]", parts_to_remove.size(), fmt::join(parts_to_remove, ", ")); + std::vector> parts_to_remove_futures; + parts_to_remove_futures.reserve(parts_to_remove.size()); + for (const DataPartPtr & part : parts_to_remove) { - pool.scheduleOrThrowOnError([&part, &part_names_mutex, part_names_succeed, thread_group = CurrentThread::getGroup()] + parts_to_remove_futures.push_back(runner([&part, &part_names_mutex, part_names_succeed, thread_group = CurrentThread::getGroup()] { - SCOPE_EXIT_SAFE( - if (thread_group) - CurrentThread::detachFromGroupIfNotDetached(); - ); - if (thread_group) - CurrentThread::attachToGroupIfDetached(thread_group); - asMutableDeletingPart(part)->remove(); if (part_names_succeed) { std::lock_guard lock(part_names_mutex); part_names_succeed->insert(part->name); } - }); + }, Priority{0})); } - pool.wait(); + /// Any exception will be re-thrown. + for (auto & future : parts_to_remove_futures) + future.get(); + parts_to_remove_futures.clear(); + return; } @@ -2544,20 +2488,15 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t return independent_ranges; }; - auto schedule_parts_removal = [this, &pool, &part_names_mutex, part_names_succeed]( + std::vector> part_removal_futures; + + auto schedule_parts_removal = [this, &runner, &part_names_mutex, part_names_succeed, &part_removal_futures]( const MergeTreePartInfo & range, DataPartsVector && parts_in_range) { /// Below, range should be captured by copy to avoid use-after-scope on exception from pool - pool.scheduleOrThrowOnError( - [this, range, &part_names_mutex, part_names_succeed, thread_group = CurrentThread::getGroup(), batch = std::move(parts_in_range)] + part_removal_futures.push_back(runner( + [this, range, &part_names_mutex, part_names_succeed, batch = std::move(parts_in_range)] { - SCOPE_EXIT_SAFE( - if (thread_group) - CurrentThread::detachFromGroupIfNotDetached(); - ); - if (thread_group) - CurrentThread::attachToGroupIfDetached(thread_group); - LOG_TRACE(log, "Removing {} parts in blocks range {}", batch.size(), range.getPartNameForLogs()); for (const auto & part : batch) @@ -2569,7 +2508,7 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t part_names_succeed->insert(part->name); } } - }); + }, Priority{0})); }; RemovalRanges independent_ranges = split_into_independent_ranges(parts_to_remove, /* split_times */ 0); @@ -2632,7 +2571,11 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t LOG_TRACE(log, "Will remove {} big parts separately: {}", excluded_parts.size(), fmt::join(excluded_parts, ", ")); independent_ranges = split_into_independent_ranges(excluded_parts, /* split_times */ 0); - pool.wait(); + + /// Any exception will be re-thrown. + for (auto & future : part_removal_futures) + future.get(); + part_removal_futures.clear(); for (size_t i = 0; i < independent_ranges.infos.size(); ++i) { @@ -2641,7 +2584,10 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t schedule_parts_removal(range, std::move(parts_in_range)); } - pool.wait(); + /// Any exception will be re-thrown. + for (auto & future : part_removal_futures) + future.get(); + part_removal_futures.clear(); if (parts_to_remove.size() != sum_of_ranges + excluded_parts.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 1c41de6fa19..2f254f9a787 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1519,11 +1519,7 @@ private: size_t max_backoff_ms, size_t max_tries); - std::vector loadDataPartsFromDisk( - ThreadPool & pool, - size_t num_parts, - std::queue & parts_queue, - const MergeTreeSettingsPtr & settings); + std::vector loadDataPartsFromDisk(PartLoadingTreeNodes & parts_to_load); void loadDataPartsFromWAL(MutableDataPartsVector & parts_from_wal); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 5ea99009756..33aea358078 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -143,8 +143,6 @@ struct Settings; M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ M(Bool, materialize_ttl_recalculate_only, false, "Only recalculate ttl info when MATERIALIZE TTL", 0) \ M(Bool, enable_mixed_granularity_parts, true, "Enable parts with adaptive and non adaptive granularity", 0) \ - M(MaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \ - M(MaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \ M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ M(UInt64, zero_copy_concurrent_part_removal_max_split_times, 5, "Max recursion depth for splitting independent Outdated parts ranges into smaller subranges (highly not recommended to change)", 0) \ M(Float, zero_copy_concurrent_part_removal_max_postpone_ratio, static_cast(0.05), "Max percentage of top level parts to postpone removal in order to get smaller independent ranges (highly not recommended to change)", 0) \ @@ -192,6 +190,9 @@ struct Settings; M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Obsolete setting, does nothing.", 0) \ M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Obsolete setting, does nothing.", 0) \ M(Bool, in_memory_parts_insert_sync, false, "Obsolete setting, does nothing.", 0) \ + M(MaxThreads, max_part_loading_threads, 0, "Obsolete setting, does nothing.", 0) \ + M(MaxThreads, max_part_removal_threads, 0, "Obsolete setting, does nothing.", 0) \ + /// Settings that should not change after the creation of a table. /// NOLINTNEXTLINE #define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \ diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index b65f044a13b..69fbdd5a64d 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -105,7 +105,7 @@ struct MergeTreeSource::AsyncReadingState AsyncReadingState() { control = std::make_shared(); - callback_runner = threadPoolCallbackRunner(IOThreadPool::get(), "MergeTreeRead"); + callback_runner = threadPoolCallbackRunner(getIOThreadPool().get(), "MergeTreeRead"); } ~AsyncReadingState() diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2d8aaec0f07..f1a7bcb71a2 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -766,7 +766,7 @@ public: DBMS_DEFAULT_BUFFER_SIZE, configuration_.request_settings, std::nullopt, - threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelWrite"), + threadPoolCallbackRunner(getIOThreadPool().get(), "S3ParallelWrite"), context->getWriteSettings()), compression_method, 3); diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 9f80b994051..97af4094e42 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -194,7 +194,7 @@ private: futures.push_back( scheduleFromThreadPool( std::move(worker), - IOThreadPool::get(), + getIOThreadPool().get(), "DP_BytesOnDisk")); } diff --git a/tests/queries/0_stateless/00988_parallel_parts_removal.sql b/tests/queries/0_stateless/00988_parallel_parts_removal.sql index bff9bbe6d8d..8f79276782b 100644 --- a/tests/queries/0_stateless/00988_parallel_parts_removal.sql +++ b/tests/queries/0_stateless/00988_parallel_parts_removal.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS mt; -CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS max_part_removal_threads = 16, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, old_parts_lifetime = 1, parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; +CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, old_parts_lifetime = 1, parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; SYSTEM STOP MERGES mt; diff --git a/tests/queries/0_stateless/00989_parallel_parts_loading.sql b/tests/queries/0_stateless/00989_parallel_parts_loading.sql index 13cd56e1924..a05515cf756 100644 --- a/tests/queries/0_stateless/00989_parallel_parts_loading.sql +++ b/tests/queries/0_stateless/00989_parallel_parts_loading.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS mt; -CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS max_part_loading_threads = 16, parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; +CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; SYSTEM STOP MERGES mt; diff --git a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh index f8f49816479..87153a4bd58 100755 --- a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh +++ b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh @@ -11,6 +11,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +# The number of threads removing data parts should be between 1 and 129. +# Because max_parts_cleaning_thread_pool_size is 128 by default + $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -nm -q "create database ordinary_$CLICKHOUSE_DATABASE engine=Ordinary" # MergeTree @@ -22,7 +25,7 @@ $CLICKHOUSE_CLIENT -nm -q """ Engine=MergeTree() order by key partition by key%100 - settings max_part_removal_threads=10, concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0; + settings concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0; insert into data_01810 select * from numbers(100); drop table data_01810 settings log_queries=1; @@ -30,7 +33,7 @@ $CLICKHOUSE_CLIENT -nm -q """ -- sometimes the same thread can be used to remove part, due to ThreadPool, -- hence we cannot compare strictly. - select throwIf(not(length(thread_ids) between 1 and 11)) + select throwIf(not(length(thread_ids) between 1 and 129)) from system.query_log where event_date >= yesterday() and @@ -49,7 +52,7 @@ $CLICKHOUSE_CLIENT -nm -q """ Engine=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rep_data_01810', '1') order by key partition by key%100 - settings max_part_removal_threads=10, concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0; + settings concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0; SET insert_keeper_max_retries=1000; SET insert_keeper_retry_max_backoff_ms=10; @@ -60,7 +63,7 @@ $CLICKHOUSE_CLIENT -nm -q """ -- sometimes the same thread can be used to remove part, due to ThreadPool, -- hence we cannot compare strictly. - select throwIf(not(length(thread_ids) between 1 and 11)) + select throwIf(not(length(thread_ids) between 1 and 129)) from system.query_log where event_date >= yesterday() and diff --git a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql index 88fb2cdf9b1..5b9342972f4 100644 --- a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql +++ b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql @@ -8,7 +8,7 @@ drop table if exists rmt2; -- Disable compact parts, because we need hardlinks in mutations. create table rmt (n int, m int, k int) engine=ReplicatedMergeTree('/test/02432/{database}', '1') order by tuple() settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1, - max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, + concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_replicated_merges_in_queue=0, max_replicated_mutations_in_queue=0, min_bytes_for_wide_part=0, min_rows_for_wide_part=0; insert into rmt(n, m) values (1, 42); @@ -38,7 +38,7 @@ select count(), sum(n), sum(m) from rmt; -- New table can assign merges/mutations and can remove old parts create table rmt2 (n int, m int, k String) engine=ReplicatedMergeTree('/test/02432/{database}', '2') order by tuple() settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1, - max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, + concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, min_bytes_for_wide_part=0, min_rows_for_wide_part=0, max_replicated_merges_in_queue=1, old_parts_lifetime=0; @@ -66,4 +66,3 @@ drop table rmt2; system flush logs; select count() > 0 from system.text_log where yesterday() <= event_date and logger_name like '%' || currentDatabase() || '%' and message like '%Removing % parts from filesystem (concurrently): Parts:%'; select count() > 1, countDistinct(thread_id) > 1 from system.text_log where yesterday() <= event_date and logger_name like '%' || currentDatabase() || '%' and message like '%Removing % parts in blocks range%'; - From 36d298ceef12daf6689ae648b7efdedf2ee83d79 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 14:45:58 +0200 Subject: [PATCH 1243/2223] Fix commit for DiskObjectStorage (#50599) --- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 3 ++- src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp | 2 -- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 129f1ab1ef7..005d115a277 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -596,7 +596,8 @@ void DiskObjectStorage::writeFileUsingBlobWritingFunction(const String & path, W { LOG_TEST(log, "Write file: {}", path); auto transaction = createObjectStorageTransaction(); - return transaction->writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function)); + transaction->writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function)); + transaction->commit(); } void DiskObjectStorage::applyNewSettings( diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 257a6fdf2ea..bd66ada492f 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -710,8 +710,6 @@ void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction( metadata_transaction->createMetadataFile(path, blob_name, object_size); else metadata_transaction->addBlobToMetadata(path, blob_name, object_size); - - metadata_transaction->commit(); } From f096bfcad29b95ea02f720f0becd5931c5b7eb37 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 6 Jun 2023 15:47:34 +0300 Subject: [PATCH 1244/2223] Adjust 00569_parse_date_time_best_effort and 01543_parse_datetime_besteffort_or_null_empty_string to the new feature --- .../00569_parse_date_time_best_effort.reference | 8 ++++---- .../0_stateless/00569_parse_date_time_best_effort.sql | 8 ++++---- ...543_parse_datetime_besteffort_or_null_empty_string.sql | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference b/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference index ad7c17b7717..0729a7628f2 100644 --- a/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference +++ b/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference @@ -2,8 +2,8 @@ 0 ᴺᵁᴸᴸ 1970-01-01 00:00:00 0000 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 00:00:00 2000-01-01 00:00:00 2000-01-01 00:00:00 - 01:00:00 2000-01-01 01:00:00 2000-01-01 01:00:00 + 2000-01-01 00:00:00 2000-01-01 00:00:00 2000-01-01 00:00:00 + 2000-01-01 01:00:00 2000-01-01 01:00:00 2000-01-01 01:00:00 02/01/17 010203 MSK 2017-01-01 22:02:03 2017-01-01 22:02:03 02/01/17 010203 MSK+0100 2017-01-01 21:02:03 2017-01-01 21:02:03 02/01/17 010203 UTC+0300 2017-01-01 22:02:03 2017-01-01 22:02:03 @@ -11,13 +11,13 @@ 02/01/1970 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 02/01/70 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50 - 17 Apr 2 1:2:3 2000-04-17 01:02:03 2000-04-17 01:02:03 + 17 Apr 2000 2 1:2:3 2000-04-17 01:02:03 2000-04-17 01:02:03 19700102 01:00:00 1970-01-02 01:00:00 1970-01-02 01:00:00 1970010201:00:00 ᴺᵁᴸᴸ 1970-01-01 00:00:00 19700102010203 1970-01-02 01:02:03 1970-01-02 01:02:03 19700102010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 1970/01/02 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 - 20 2000-01-20 00:00:00 2000-01-20 00:00:00 + 20 2000 2000-01-20 00:00:00 2000-01-20 00:00:00 201 ᴺᵁᴸᴸ 1970-01-01 00:00:00 20160101 2016-01-01 00:00:00 2016-01-01 00:00:00 2016-01-01 2016-01-01 00:00:00 2016-01-01 00:00:00 diff --git a/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql b/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql index 5f71efa1485..511addb4e4d 100644 --- a/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql +++ b/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql @@ -7,8 +7,8 @@ FROM SELECT arrayJoin([ '0', '0000', -'00:00:00', -'01:00:00', +'2000-01-01 00:00:00', +'2000-01-01 01:00:00', '02/01/17 010203 MSK', '02/01/17 010203 MSK+0100', '02/01/17 010203 UTC+0300', @@ -16,13 +16,13 @@ FROM '02/01/1970 010203Z', '02/01/70 010203Z', '11 Feb 2018 06:40:50 +0300', -'17 Apr 2 1:2:3', +'17 Apr 2000 2 1:2:3', '19700102 01:00:00', '1970010201:00:00', '19700102010203', '19700102010203Z', '1970/01/02 010203Z', -'20', +'20 2000', '201', '20160101', '2016-01-01', diff --git a/tests/queries/0_stateless/01543_parse_datetime_besteffort_or_null_empty_string.sql b/tests/queries/0_stateless/01543_parse_datetime_besteffort_or_null_empty_string.sql index ad14c4ede06..7098028963d 100644 --- a/tests/queries/0_stateless/01543_parse_datetime_besteffort_or_null_empty_string.sql +++ b/tests/queries/0_stateless/01543_parse_datetime_besteffort_or_null_empty_string.sql @@ -4,7 +4,7 @@ SELECT parseDateTimeBestEffortOrNull('2020-01-01 11:01:01 am'); SELECT parseDateTimeBestEffortOrNull('2020-01-01 11:01:01 pm'); SELECT parseDateTimeBestEffortOrNull('2020-01-01 12:01:01 am'); SELECT parseDateTimeBestEffortOrNull('2020-01-01 12:01:01 pm'); -SELECT parseDateTimeBestEffortOrNull('01:01:01'); +SELECT parseDateTimeBestEffortOrNull('2000-01-01 01:01:01'); SELECT parseDateTimeBestEffortOrNull('20100'); SELECT parseDateTimeBestEffortOrNull('0100:0100:0000'); SELECT parseDateTimeBestEffortOrNull('x'); From a1f3bd9e231ade5a88e3bb53dd4c96caa583d835 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 14:55:17 +0200 Subject: [PATCH 1245/2223] Fix reads --- .../AzureBlobStorage/AzureObjectStorage.cpp | 15 +- .../ObjectStorages/ObjectStorageIterator.cpp | 2 +- .../ObjectStorages/ObjectStorageIterator.h | 12 +- .../ObjectStorageIteratorAsync.cpp | 41 +++-- .../ObjectStorageIteratorAsync.h | 8 +- src/Storages/StorageAzure.cpp | 25 ++- .../test_storage_azure_blob_storage/test.py | 145 ++++++++++++++++++ 7 files changed, 216 insertions(+), 32 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 23a0da39dd3..07173e65448 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -52,15 +52,18 @@ public: options.Prefix = path_prefix; options.PageSizeHint = static_cast(max_list_size); + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "ITER PREFIX {}", path_prefix); } private: bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) override { + batch.clear(); auto outcome = client->ListBlobs(options); auto blob_list_response = client->ListBlobs(options); auto blobs_list = blob_list_response.Blobs; + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "BLOB LIST SIZE {}", blobs_list.size()); for (const auto & blob : blobs_list) { batch.emplace_back( @@ -73,11 +76,15 @@ private: {}}); } - options.ContinuationToken = blob_list_response.NextPageToken; - if (blob_list_response.HasPage()) - return true; + if (!blob_list_response.NextPageToken.HasValue() || blob_list_response.NextPageToken.Value().empty()) + { + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "RETURN FALSE {}", blobs_list.size()); + return false; + } - return false; + options.ContinuationToken = blob_list_response.NextPageToken; + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "RETURN TRUE {}", blobs_list.size()); + return true; } std::shared_ptr client; diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.cpp b/src/Disks/ObjectStorages/ObjectStorageIterator.cpp index 188b743958c..72ec6e0e500 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIterator.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.cpp @@ -9,7 +9,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -RelativePathWithMetadata ObjectStorageIteratorFromList::current() const +RelativePathWithMetadata ObjectStorageIteratorFromList::current() { if (!isValid()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.h b/src/Disks/ObjectStorages/ObjectStorageIterator.h index e562d92e1fb..2ff5ce60acc 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIterator.h +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.h @@ -11,9 +11,9 @@ class IObjectStorageIterator public: virtual void next() = 0; virtual void nextBatch() = 0; - virtual bool isValid() const = 0; - virtual RelativePathWithMetadata current() const = 0; - virtual RelativePathsWithMetadata currentBatch() const = 0; + virtual bool isValid() = 0; + virtual RelativePathWithMetadata current() = 0; + virtual RelativePathsWithMetadata currentBatch() = 0; virtual size_t getAccumulatedSize() const = 0; virtual ~IObjectStorageIterator() = default; @@ -41,14 +41,14 @@ public: batch_iterator = batch.end(); } - bool isValid() const override + bool isValid() override { return batch_iterator != batch.end(); } - RelativePathWithMetadata current() const override; + RelativePathWithMetadata current() override; - RelativePathsWithMetadata currentBatch() const override + RelativePathsWithMetadata currentBatch() override { return batch; } diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index c97a941f7be..fd6452b7c2a 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -1,5 +1,7 @@ #include +#include + namespace DB { @@ -13,17 +15,26 @@ void IObjectStorageIteratorAsync::nextBatch() std::lock_guard lock(mutex); if (!is_finished) { - if (outcome_future.valid()) + if (!is_initialized) { - BatchAndHasNext next_batch = outcome_future.get(); - current_batch = std::move(next_batch.batch); - accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); - current_batch_iterator = current_batch.begin(); - if (next_batch.has_next) - outcome_future = scheduleBatch(); - else - is_finished = true; + outcome_future = scheduleBatch(); + is_initialized = true; } + + BatchAndHasNext next_batch = outcome_future.get(); + current_batch = std::move(next_batch.batch); + accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); + current_batch_iterator = current_batch.begin(); + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "HAS NEXT {}", next_batch.has_next); + if (next_batch.has_next) + outcome_future = scheduleBatch(); + else + is_finished = true; + } + else + { + current_batch.clear(); + current_batch_iterator = current_batch.begin(); } } @@ -62,12 +73,15 @@ std::future IObjectStorageIterator } -bool IObjectStorageIteratorAsync::isValid() const +bool IObjectStorageIteratorAsync::isValid() { + if (!is_initialized) + nextBatch(); + return current_batch_iterator != current_batch.end(); } -RelativePathWithMetadata IObjectStorageIteratorAsync::current() const +RelativePathWithMetadata IObjectStorageIteratorAsync::current() { if (!isValid()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); @@ -76,9 +90,12 @@ RelativePathWithMetadata IObjectStorageIteratorAsync::current() const } -RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch() const +RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch() { std::lock_guard lock(mutex); + if (!isValid()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); + return current_batch; } diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h index 3f3f41e7e77..a2b06da9a91 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -19,14 +19,13 @@ public: : list_objects_pool(threads_metric, threads_active_metric, 1) , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, thread_name)) { - nextBatch(); } void next() override; void nextBatch() override; - bool isValid() const override; - RelativePathWithMetadata current() const override; - RelativePathsWithMetadata currentBatch() const override; + bool isValid() override; + RelativePathWithMetadata current() override; + RelativePathsWithMetadata currentBatch() override; size_t getAccumulatedSize() const override; ~IObjectStorageIteratorAsync() override @@ -46,6 +45,7 @@ protected: std::future scheduleBatch(); + bool is_initialized{false}; bool is_finished{false}; mutable std::mutex mutex; diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index d83dc90e6ed..e0a1d8c514e 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -858,6 +858,7 @@ StorageAzureSource::Iterator::Iterator( } else { + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "GLOBS BRANCH"); const String key_prefix = blob_path_with_globs->substr(0, blob_path_with_globs->find_first_of("*?{")); /// We don't have to list bucket, because there is no asterisks. @@ -868,7 +869,11 @@ StorageAzureSource::Iterator::Iterator( return; } + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "KEY PREFIX {}", key_prefix); object_storage_iterator = object_storage->iterate(key_prefix); + + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "BLOBS BLOBS{}", *blob_path_with_globs); + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "REGEXP PATTERN {}", makeRegexpPatternFromGlobs(*blob_path_with_globs)); matcher = std::make_unique(makeRegexpPatternFromGlobs(*blob_path_with_globs)); if (!matcher->ok()) @@ -898,27 +903,37 @@ RelativePathWithMetadata StorageAzureSource::Iterator::next() } else { + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "GLOBS IN NEXt"); if (!blobs_with_metadata || index >= blobs_with_metadata->size()) { + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "INITIALIZING BLOBS BATCH"); RelativePathsWithMetadata new_batch; while (new_batch.empty()) { if (object_storage_iterator->isValid()) { + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "ITERATOR VALID FETCHING BATCH"); new_batch = object_storage_iterator->currentBatch(); + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "BATCH SIZE {}", new_batch.size()); object_storage_iterator->nextBatch(); } else { + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "ITERATOR INVALID"); is_finished = true; return {}; } - for (auto it = new_batch.begin(); it != new_batch.end(); ++it) + for (auto it = new_batch.begin(); it != new_batch.end();) { + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "ITERATOR FILTER {} MATCH {}", it->relative_path, re2::RE2::FullMatch(it->relative_path, *matcher)); if (!recursive && !re2::RE2::FullMatch(it->relative_path, *matcher)) it = new_batch.erase(it); + else + ++it; } + + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "NEW BATCH AFTER FILTEr {}", new_batch.size()); } index.store(0, std::memory_order_relaxed); @@ -1092,11 +1107,11 @@ String StorageAzureSource::getName() const StorageAzureSource::ReaderHolder StorageAzureSource::createReader() { auto [current_key, info] = file_iterator->next(); + LOG_DEBUG(log, "KEY {} SIZE {}", current_key, info.size_bytes); if (current_key.empty()) return {}; size_t object_size = info.size_bytes != 0 ? info.size_bytes : object_storage->getObjectMetadata(current_key).size_bytes; - LOG_DEBUG(log, "SIZE {}", object_size); auto compression_method = chooseCompressionMethod(current_key, compression_hint); auto read_buf = createAzureReadBuffer(current_key, object_size); @@ -1134,9 +1149,9 @@ std::unique_ptr StorageAzureSource::createAzureReadBuffer(const Stri //auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; //const bool object_too_small = object_size <= 2 * download_buffer_size; - // Create a read buffer that will prefetch the first ~1 MB of the file. - // When reading lots of tiny files, this prefetching almost doubles the throughput. - // For bigger files, parallel reading is more useful. + ///// Create a read buffer that will prefetch the first ~1 MB of the file. + ///// When reading lots of tiny files, this prefetching almost doubles the throughput. + ///// For bigger files, parallel reading is more useful. //if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) //{ // LOG_TRACE(log, "Downloading object {} of size {} from S3 with initial prefetch", key, object_size); diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index bcf5d068057..b5cd7cb4566 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -62,6 +62,20 @@ def get_azure_file_content(filename): download_stream = blob_client.download_blob() return download_stream.readall().decode("utf-8") +def put_azure_file_content(filename, data): + container_name = "cont" + connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + try: + container_client = blob_service_client.create_container(container_name) + except: + container_client = blob_service_client.get_container_client(container_name) + + blob_client = container_client.get_blob_client(filename) + buf = io.BytesIO(data) + blob_client.upload_blob(buf) + + def test_create_table_connection_string(cluster): node = cluster.instances["node"] @@ -223,3 +237,134 @@ def test_create_new_files_on_insert(cluster): assert int(result) == 60 azure_query(node, f"drop table test_multiple_inserts") + +def test_overwrite(cluster): + + node = cluster.instances["node"] + + azure_query(node, f"create table test_overwrite(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_parquet_overwrite', format='Parquet')") + azure_query(node, "truncate table test_overwrite") + + azure_query(node, + f"insert into test_overwrite select number, randomString(100) from numbers(50) settings azure_truncate_on_insert=1" + ) + node.query_and_get_error( + f"insert into test_overwrite select number, randomString(100) from numbers(100)" + ) + azure_query(node, + f"insert into test_overwrite select number, randomString(100) from numbers(200) settings azure_truncate_on_insert=1" + ) + + result = azure_query(node, f"select count() from test_overwrite") + assert int(result) == 200 + +def test_insert_with_path_with_globs(cluster): + node = cluster.instances["node"] + azure_query(node, f"create table test_insert_globs(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_insert_with_globs*', format='Parquet')") + node.query_and_get_error( + f"insert into table function test_insert_globs SELECT number, randomString(100) FROM numbers(500)" + ) + +def test_put_get_with_globs(cluster): + # type: (ClickHouseCluster) -> None + unique_prefix = random.randint(1, 10000) + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + max_path = "" + for i in range(10): + for j in range(10): + path = "{}/{}_{}/{}.csv".format( + unique_prefix, i, random.choice(["a", "b", "c", "d"]), j + ) + max_path = max(path, max_path) + values = f"({i},{j},{i + j})" + + azure_query(node, f"CREATE TABLE test_{i}_{j} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSV')") + + query = f"insert into test_{i}_{j} VALUES {values}" + azure_query(node, query) + + + azure_query(node, f"CREATE TABLE test_glob_select ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV')") + query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from test_glob_select" + assert azure_query(node, query).splitlines() == [ + "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( + bucket='cont', max_path=max_path + ) + ] + +def test_azure_glob_scheherazade(cluster): + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + values = "(1, 1, 1)" + nights_per_job = 1001 // 30 + jobs = [] + for night in range(0, 1001, nights_per_job): + + def add_tales(start, end): + for i in range(start, end): + path = "night_{}/tale.csv".format(i) + unique_num = random.randint(1, 10000) + azure_query(node, f"CREATE TABLE test_{i}_{unique_num} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSV')") + query = f"insert into test_{i}_{unique_num} VALUES {values}" + azure_query(node, query) + + jobs.append( + threading.Thread( + target=add_tales, args=(night, min(night + nights_per_job, 1001)) + ) + ) + jobs[-1].start() + + for job in jobs: + job.join() + + + azure_query(node, f"CREATE TABLE test_glob_select_scheherazade ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='night_*/tale.csv', format='CSV')") + query = "select count(), sum(column1), sum(column2), sum(column3) from test_glob_select_scheherazade" + assert azure_query(node, query).splitlines() == ["1001\t1001\t1001\t1001"] + +@pytest.mark.parametrize( + "extension,method", + [pytest.param("bin", "gzip", id="bin"), pytest.param("gz", "auto", id="gz")], +) +def test_storage_azure_get_gzip(cluster, extension, method): + node = cluster.instances["node"] + filename = f"test_get_gzip.{extension}" + name = f"test_get_gzip_{extension}" + data = [ + "Sophia Intrieri,55", + "Jack Taylor,71", + "Christopher Silva,66", + "Clifton Purser,35", + "Richard Aceuedo,43", + "Lisa Hensley,31", + "Alice Wehrley,1", + "Mary Farmer,47", + "Samara Ramirez,19", + "Shirley Lloyd,51", + "Santos Cowger,0", + "Richard Mundt,88", + "Jerry Gonzalez,15", + "Angela James,10", + "Norman Ortega,33", + "", + ] + azure_query(node, f"DROP TABLE IF EXISTS {name}") + + buf = io.BytesIO() + compressed = gzip.GzipFile(fileobj=buf, mode="wb") + compressed.write(("\n".join(data)).encode()) + compressed.close() + put_azure_file_content(filename, buf.getvalue()) + + azure_query( + node, + f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = Azure( + azure_conf2, container='cont', blob_path ='{filename}', + format='CSV', + compression='{method}')""", + ) + + assert azure_query(node, f"SELECT sum(id) FROM {name}").splitlines() == ["565"] + azure_query(node, f"DROP TABLE {name}") From ad2b926248a4fd464f6f278045c9103a75d92ca7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 6 Jun 2023 07:17:30 +0000 Subject: [PATCH 1246/2223] Fix jepsen runs in PRs --- tests/ci/jepsen_check.py | 8 ++------ tests/jepsen.clickhouse/resources/keeper_config.xml | 8 ++++---- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 9d35d2d6e35..c21fafa2605 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -25,6 +25,7 @@ from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results from version_helper import get_version_from_repo +from build_check import get_release_or_pr JEPSEN_GROUP_NAME = "jepsen_group" @@ -210,12 +211,7 @@ if __name__ == "__main__": build_name = get_build_name_for_check(check_name) - if pr_info.number == 0: - version = get_version_from_repo() - release_or_pr = f"{version.major}.{version.minor}" - else: - # PR number for anything else - release_or_pr = str(pr_info.number) + release_or_pr, _ = get_release_or_pr(pr_info, get_version_from_repo()) # This check run separately from other checks because it requires exclusive # run (see .github/workflows/jepsen.yml) So we cannot add explicit diff --git a/tests/jepsen.clickhouse/resources/keeper_config.xml b/tests/jepsen.clickhouse/resources/keeper_config.xml index 52f2a0dbdc2..1972ef6b917 100644 --- a/tests/jepsen.clickhouse/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse/resources/keeper_config.xml @@ -13,19 +13,19 @@ local - /var/lib/clickhouse/coordination/logs/ + /home/robot-clickhouse/db/coordination/logs/ local - /var/lib/clickhouse/coordination/latest_log/ + /home/robot-clickhouse/db/coordination/latest_log/ local - /var/lib/clickhouse/coordination/snapshots/ + /home/robot-clickhouse/db/coordination/snapshots/ local - /var/lib/clickhouse/coordination/latest_snapshot/ + /home/robot-clickhouse/db/coordination/latest_snapshot/
From e054fbccd83dd20a5f748b62e3c40002eb95c551 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 6 Jun 2023 13:09:53 +0000 Subject: [PATCH 1247/2223] Automatic style fix --- .../test_storage_azure_blob_storage/test.py | 52 +++++++++++++------ 1 file changed, 37 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 221005e414b..81560fb0da1 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -62,6 +62,7 @@ def get_azure_file_content(filename): download_stream = blob_client.download_blob() return download_stream.readall().decode("utf-8") + def put_azure_file_content(filename, data): container_name = "cont" connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" @@ -76,7 +77,6 @@ def put_azure_file_content(filename, data): blob_client.upload_blob(buf) - def test_create_table_connection_string(cluster): node = cluster.instances["node"] azure_query( @@ -243,33 +243,43 @@ def test_create_new_files_on_insert(cluster): azure_query(node, f"drop table test_multiple_inserts") -def test_overwrite(cluster): +def test_overwrite(cluster): node = cluster.instances["node"] - azure_query(node, f"create table test_overwrite(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_parquet_overwrite', format='Parquet')") + azure_query( + node, + f"create table test_overwrite(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_parquet_overwrite', format='Parquet')", + ) azure_query(node, "truncate table test_overwrite") - azure_query(node, - f"insert into test_overwrite select number, randomString(100) from numbers(50) settings azure_truncate_on_insert=1" + azure_query( + node, + f"insert into test_overwrite select number, randomString(100) from numbers(50) settings azure_truncate_on_insert=1", ) node.query_and_get_error( f"insert into test_overwrite select number, randomString(100) from numbers(100)" ) - azure_query(node, - f"insert into test_overwrite select number, randomString(100) from numbers(200) settings azure_truncate_on_insert=1" + azure_query( + node, + f"insert into test_overwrite select number, randomString(100) from numbers(200) settings azure_truncate_on_insert=1", ) result = azure_query(node, f"select count() from test_overwrite") assert int(result) == 200 + def test_insert_with_path_with_globs(cluster): node = cluster.instances["node"] - azure_query(node, f"create table test_insert_globs(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_insert_with_globs*', format='Parquet')") + azure_query( + node, + f"create table test_insert_globs(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_insert_with_globs*', format='Parquet')", + ) node.query_and_get_error( f"insert into table function test_insert_globs SELECT number, randomString(100) FROM numbers(500)" ) + def test_put_get_with_globs(cluster): # type: (ClickHouseCluster) -> None unique_prefix = random.randint(1, 10000) @@ -284,20 +294,26 @@ def test_put_get_with_globs(cluster): max_path = max(path, max_path) values = f"({i},{j},{i + j})" - azure_query(node, f"CREATE TABLE test_{i}_{j} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSV')") + azure_query( + node, + f"CREATE TABLE test_{i}_{j} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + ) query = f"insert into test_{i}_{j} VALUES {values}" azure_query(node, query) - - azure_query(node, f"CREATE TABLE test_glob_select ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV')") + azure_query( + node, + f"CREATE TABLE test_glob_select ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV')", + ) query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from test_glob_select" assert azure_query(node, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( - bucket='cont', max_path=max_path + bucket="cont", max_path=max_path ) ] + def test_azure_glob_scheherazade(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -310,7 +326,10 @@ def test_azure_glob_scheherazade(cluster): for i in range(start, end): path = "night_{}/tale.csv".format(i) unique_num = random.randint(1, 10000) - azure_query(node, f"CREATE TABLE test_{i}_{unique_num} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSV')") + azure_query( + node, + f"CREATE TABLE test_{i}_{unique_num} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + ) query = f"insert into test_{i}_{unique_num} VALUES {values}" azure_query(node, query) @@ -324,11 +343,14 @@ def test_azure_glob_scheherazade(cluster): for job in jobs: job.join() - - azure_query(node, f"CREATE TABLE test_glob_select_scheherazade ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='night_*/tale.csv', format='CSV')") + azure_query( + node, + f"CREATE TABLE test_glob_select_scheherazade ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='night_*/tale.csv', format='CSV')", + ) query = "select count(), sum(column1), sum(column2), sum(column3) from test_glob_select_scheherazade" assert azure_query(node, query).splitlines() == ["1001\t1001\t1001\t1001"] + @pytest.mark.parametrize( "extension,method", [pytest.param("bin", "gzip", id="bin"), pytest.param("gz", "auto", id="gz")], From 5ffbe2d9d4de6e47268be38ac84e5de45faded49 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:14:31 -0300 Subject: [PATCH 1248/2223] Update docs/en/sql-reference/data-types/ipv6.md Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- docs/en/sql-reference/data-types/ipv6.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/ipv6.md b/docs/en/sql-reference/data-types/ipv6.md index 284a1f80854..97959308b58 100644 --- a/docs/en/sql-reference/data-types/ipv6.md +++ b/docs/en/sql-reference/data-types/ipv6.md @@ -6,7 +6,7 @@ sidebar_label: IPv6 ## IPv6 -IPv6 addresses. Stored in 16 bytes as UInt128. +IPv6 addresses. Stored in 16 bytes as UInt128 big-endian. ### Basic Usage From a96c1ea86f1fa662e8e775faeaf4b3a8053c6eb2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 6 Jun 2023 15:15:51 +0200 Subject: [PATCH 1249/2223] Fix --- src/Interpreters/Cache/FileSegment.cpp | 2 +- src/Interpreters/Cache/Metadata.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 1b7fe54a769..7b82c58080c 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -381,7 +381,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) const auto file_size = fs::file_size(file_segment_path); chassert(downloaded_size <= file_size); chassert(reserved_size >= file_size); - chassert(file_size <= range().right + 1); + chassert(file_size <= range().size()); if (downloaded_size != file_size) downloaded_size = file_size; } diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 9dff77e2af8..4794d1b3264 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -370,8 +370,8 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm bool exists = fs::exists(path); if (exists) { - LOG_TEST(log, "Removed file segment at path: {}", path); fs::remove(path); + LOG_TEST(log, "Removed file segment at path: {}", path); } else if (file_segment->downloaded_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist"); From 9ae4d929eacac5ae4681cce91ec2f65b198cf3c9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 6 Jun 2023 15:16:18 +0200 Subject: [PATCH 1250/2223] Update src/Interpreters/tests/gtest_convertFieldToType.cpp --- src/Interpreters/tests/gtest_convertFieldToType.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_convertFieldToType.cpp b/src/Interpreters/tests/gtest_convertFieldToType.cpp index 5421c192ac7..f4de36cbecc 100644 --- a/src/Interpreters/tests/gtest_convertFieldToType.cpp +++ b/src/Interpreters/tests/gtest_convertFieldToType.cpp @@ -56,7 +56,7 @@ TEST_P(ConvertFieldToTypeTest, convert) } } -// Basically nuber of seconds in a day, works for UTC here +// Basically, the number of seconds in a day works for UTC here const long long int Day = 24 * 60 * 60; // 123 is arbitrary value here From 4ca902ddddc36dbe54b7a6e0f4a7e8185d0b35ca Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:16:56 -0300 Subject: [PATCH 1251/2223] Update docs/en/sql-reference/aggregate-functions/index.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/index.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index a3808335168..050142fdd99 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -76,9 +76,12 @@ FROM t_null_big Also you can use [Tuple](../data-types/tuple.md) to change NULL skipping behavior. ```sql -select groupArray(b), groupArray(tuple(b)).1 from t_null_big; +SELECT + groupArray(y), + groupArray(tuple(y)).1 +FROM t_null_big; -┌─groupArray(b)─┬─tupleElement(groupArray(tuple(b)), 1)─┐ +┌─groupArray(y)─┬─tupleElement(groupArray(tuple(y)), 1)─┐ │ [2,2,3] │ [2,NULL,2,3,NULL] │ └───────────────┴───────────────────────────────────────┘ ``` From 5d7b8b3d13502a2eb842dfbfe3b1b704b638badb Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:17:04 -0300 Subject: [PATCH 1252/2223] Update docs/en/sql-reference/aggregate-functions/index.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/index.md | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 050142fdd99..25f9d05a76c 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -85,6 +85,3 @@ FROM t_null_big; │ [2,2,3] │ [2,NULL,2,3,NULL] │ └───────────────┴───────────────────────────────────────┘ ``` - - - From 2a9792cebd6e15818a65bd6566cf683bcb623225 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:17:21 -0300 Subject: [PATCH 1253/2223] Update docs/en/sql-reference/aggregate-functions/reference/argmax.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 8a84f361589..5e80131df4c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -90,7 +90,7 @@ select (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB from test; select argMax(a, b), max(b) from test where a is Null and b is Null; ┌─argMax(a, b)─┬─max(b)─┐ -│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- Nulls are not skipped because only Null values are available +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- All aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` └──────────────┴────────┘ select argMax(a, (b,a)) from test; From cda9535554fa28c76d11115fdf276c25e0449f04 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:17:46 -0300 Subject: [PATCH 1254/2223] Update docs/en/sql-reference/aggregate-functions/index.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 25f9d05a76c..ea270e83a3c 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -73,7 +73,7 @@ FROM t_null_big └────────────────────┴─────────────────────┘ ``` -Also you can use [Tuple](../data-types/tuple.md) to change NULL skipping behavior. +Also you can use [Tuple](../data-types/tuple.md) to work around NULL skipping behavior. The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value. ```sql SELECT From 1a517bb332f01da3b5b64dadbb481339b6bdf7ac Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:17:55 -0300 Subject: [PATCH 1255/2223] Update docs/en/sql-reference/aggregate-functions/reference/argmax.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 5e80131df4c..76b9e206abe 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -80,7 +80,7 @@ select argMax(a, b), max(b) from test; select argMax(tuple(a), b) from test; ┌─argMax(tuple(a), b)─┐ -│ (NULL) │ -- Tuple allows to get Null value. +│ (NULL) │ -- The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value └─────────────────────┘ select (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB from test; From 5308abb93a22cf19d10078e533a3b09741a713f7 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:18:13 -0300 Subject: [PATCH 1256/2223] Update docs/en/sql-reference/aggregate-functions/reference/argmin.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 067c81f56cf..304abf512da 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -90,7 +90,7 @@ select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; select argMin(a, b), min(b) from test where a is Null and b is Null; ┌─argMin(a, b)─┬─min(b)─┐ -│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- Nulls are not skipped because only Null values are available +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- ll aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` └──────────────┴────────┘ select argMin(a, (b, a)), min(tuple(b, a)) from test; From f992d10ae7af2cd7d37da65055be4f26fdfa9957 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:18:22 -0300 Subject: [PATCH 1257/2223] Update docs/en/sql-reference/aggregate-functions/reference/argmin.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 304abf512da..816dd9fcac1 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -75,7 +75,7 @@ select * from test; select argMin(a, b), min(b) from test; ┌─argMin(a, b)─┬─min(b)─┐ -│ a │ 0 │ -- argMin = a because it the first not-Null value, min(b) is from another row! +│ a │ 0 │ -- argMin = a because it the first not Null value, min(b) is from another row! └──────────────┴────────┘ select argMin(tuple(a), b) from test; From 41be2fcbc7bd2227df9c506d2cbde82ded06e690 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:18:34 -0300 Subject: [PATCH 1258/2223] Update docs/en/sql-reference/aggregate-functions/reference/argmin.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 816dd9fcac1..df4b28b070c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -95,7 +95,7 @@ select argMin(a, b), min(b) from test where a is Null and b is Null; select argMin(a, (b, a)), min(tuple(b, a)) from test; ┌─argMin(a, tuple(b, a))─┬─min(tuple(b, a))─┐ -│ d │ (NULL,NULL) │ -- 'd' is the first Not null value for the min +│ d │ (NULL,NULL) │ -- 'd' is the first not Null value for the min └────────────────────────┴──────────────────┘ select argMin((a, b), (b, a)), min(tuple(b, a)) from test; From 834918dc1cef6af6eca60d57953a82b982f993aa Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:19:08 -0300 Subject: [PATCH 1259/2223] Update docs/en/sql-reference/aggregate-functions/reference/argmin.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index df4b28b070c..4db656f7e54 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -105,7 +105,7 @@ select argMin((a, b), (b, a)), min(tuple(b, a)) from test; select argMin(a, tuple(b)) from test; ┌─argMax(a, tuple(b))─┐ -│ b │ -- Tuple can be used in `Min` to not skip Nulls in `Min` +│ d │ -- Tuple can be used in `min` to not skip rows with Null values as b. └─────────────────────┘ ``` From 9a1f6ac719b0018aa050c0c8ad099bcd65111721 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:20:42 -0300 Subject: [PATCH 1260/2223] Update argmax.md --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 76b9e206abe..91b85bce2ff 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -6,7 +6,7 @@ sidebar_position: 106 # argMax Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not-Null values if not-Null values are available. +Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not Null values if not Null values are available. **Syntax** From d6ee50577a5d77887612a74bd43f7bd66a10e666 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:25:32 -0300 Subject: [PATCH 1261/2223] Update argmin.md --- .../sql-reference/aggregate-functions/reference/argmin.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 4db656f7e54..a5208f11de6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -6,7 +6,7 @@ sidebar_position: 105 # argMin Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not-Null values if not-Null values are available. +Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not Null values if not Null values are available. **Syntax** @@ -90,7 +90,7 @@ select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; select argMin(a, b), min(b) from test where a is Null and b is Null; ┌─argMin(a, b)─┬─min(b)─┐ -│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- ll aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- All aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` └──────────────┴────────┘ select argMin(a, (b, a)), min(tuple(b, a)) from test; @@ -100,7 +100,7 @@ select argMin(a, (b, a)), min(tuple(b, a)) from test; select argMin((a, b), (b, a)), min(tuple(b, a)) from test; ┌─argMin(tuple(a, b), tuple(b, a))─┬─min(tuple(b, a))─┐ -│ (NULL,NULL) │ (NULL,NULL) │ +│ (NULL,NULL) │ (NULL,NULL) │ -- argMin returns (NULL,NULL) here because Tuple allows to don't skip Nulls and min(tuple(b, a)) in this case is minimal value for this dataset └──────────────────────────────────┴──────────────────┘ select argMin(a, tuple(b)) from test; From 496bc25bff11c023ecc7f05420889538ff59779e Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:26:35 -0300 Subject: [PATCH 1262/2223] Update docs/en/sql-reference/aggregate-functions/reference/argmin.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index a5208f11de6..3787c028564 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -80,7 +80,7 @@ select argMin(a, b), min(b) from test; select argMin(tuple(a), b) from test; ┌─argMin(tuple(a), b)─┐ -│ (NULL) │ -- Tuple allows to get Null value. +│ (NULL) │ -- The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value └─────────────────────┘ select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; From e63fc91e73a5d718f3daf6383114978ce93a3b40 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:30:10 -0300 Subject: [PATCH 1263/2223] Update argmax.md --- .../aggregate-functions/reference/argmax.md | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 91b85bce2ff..93e1fac6d67 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -6,7 +6,7 @@ sidebar_position: 106 # argMax Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not Null values if not Null values are available. +Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. **Syntax** @@ -61,7 +61,7 @@ CREATE TABLE test ) ENGINE = Memory AS SELECT * -FROM values(('a', 1), ('b', 2), ('c', 2), (NULL, 3), (NULL, NULL), ('d', NULL)); +FROM VALUES(('a', 1), ('b', 2), ('c', 2), (NULL, 3), (NULL, NULL), ('d', NULL)); select * from test; ┌─a────┬────b─┐ @@ -73,34 +73,34 @@ select * from test; │ d │ ᴺᵁᴸᴸ │ └──────┴──────┘ -select argMax(a, b), max(b) from test; +SELECT argMax(a, b), max(b) FROM test; ┌─argMax(a, b)─┬─max(b)─┐ -│ b │ 3 │ -- argMax = b because it the first not-Null value, max(b) is from another row! +│ b │ 3 │ -- argMax = 'b' because it the first not Null value, max(b) is from another row! └──────────────┴────────┘ -select argMax(tuple(a), b) from test; +SELECT argMax(tuple(a), b) FROM test; ┌─argMax(tuple(a), b)─┐ │ (NULL) │ -- The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value └─────────────────────┘ -select (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB from test; +SELECT (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB FROM test; ┌─argMaxA─┬─argMaxB─┐ │ ᴺᵁᴸᴸ │ 3 │ -- you can use Tuple and get both (all - tuple(*)) columns for the according max(b) └─────────┴─────────┘ -select argMax(a, b), max(b) from test where a is Null and b is Null; +SELECT argMax(a, b), max(b) FROM test WHERE a IS NULL AND b IS NULL; ┌─argMax(a, b)─┬─max(b)─┐ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- All aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` └──────────────┴────────┘ -select argMax(a, (b,a)) from test; +SELECT argMax(a, (b,a)) FROM test; ┌─argMax(a, tuple(b, a))─┐ -│ c │ -- There are two rows with b=2, Tuple in the `Max` allows to get not the first `arg` +│ c │ -- There are two rows with b=2, `Tuple` in the `Max` allows to get not the first `arg` └────────────────────────┘ -select argMax(a, tuple(b)) from test; +SELECT argMax(a, tuple(b)) FROM test; ┌─argMax(a, tuple(b))─┐ -│ b │ -- Tuple can be used in `Max` to not skip Nulls in `Max` +│ b │ -- `Tuple` can be used in `Max` to not skip Nulls in `Max` └─────────────────────┘ ``` From 67a6623a3896b4deae5d69cdf84742da5b24da3b Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:32:10 -0300 Subject: [PATCH 1264/2223] Update argmin.md --- .../aggregate-functions/reference/argmin.md | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 3787c028564..4e549e5b04c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -6,7 +6,7 @@ sidebar_position: 105 # argMin Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not Null values if not Null values are available. +Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. **Syntax** @@ -61,7 +61,7 @@ CREATE TABLE test ) ENGINE = Memory AS SELECT * -FROM values((NULL, 0), ('a', 1), ('b', 2), ('c', 2), (NULL, NULL), ('d', NULL)); +FROM VALUES((NULL, 0), ('a', 1), ('b', 2), ('c', 2), (NULL, NULL), ('d', NULL)); select * from test; ┌─a────┬────b─┐ @@ -73,39 +73,39 @@ select * from test; │ d │ ᴺᵁᴸᴸ │ └──────┴──────┘ -select argMin(a, b), min(b) from test; +SELECT argMin(a, b), min(b) FROM test; ┌─argMin(a, b)─┬─min(b)─┐ -│ a │ 0 │ -- argMin = a because it the first not Null value, min(b) is from another row! +│ a │ 0 │ -- argMin = a because it the first not `NULL` value, min(b) is from another row! └──────────────┴────────┘ -select argMin(tuple(a), b) from test; +SELECT argMin(tuple(a), b) FROM test; ┌─argMin(tuple(a), b)─┐ │ (NULL) │ -- The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value └─────────────────────┘ -select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; +SELECT (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; ┌─argMinA─┬─argMinB─┐ -│ ᴺᵁᴸᴸ │ 0 │ -- you can use Tuple and get both (all - tuple(*)) columns for the according max(b) +│ ᴺᵁᴸᴸ │ 0 │ -- you can use `Tuple` and get both (all - tuple(*)) columns for the according max(b) └─────────┴─────────┘ -select argMin(a, b), min(b) from test where a is Null and b is Null; +SELECT argMin(a, b), min(b) FROM test WHERE a IS NULL and b IS NULL; ┌─argMin(a, b)─┬─min(b)─┐ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- All aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` └──────────────┴────────┘ -select argMin(a, (b, a)), min(tuple(b, a)) from test; +SELECT argMin(a, (b, a)), min(tuple(b, a)) FROM test; ┌─argMin(a, tuple(b, a))─┬─min(tuple(b, a))─┐ -│ d │ (NULL,NULL) │ -- 'd' is the first not Null value for the min +│ d │ (NULL,NULL) │ -- 'd' is the first not `NULL` value for the min └────────────────────────┴──────────────────┘ -select argMin((a, b), (b, a)), min(tuple(b, a)) from test; +SELECT argMin((a, b), (b, a)), min(tuple(b, a)) FROM test; ┌─argMin(tuple(a, b), tuple(b, a))─┬─min(tuple(b, a))─┐ -│ (NULL,NULL) │ (NULL,NULL) │ -- argMin returns (NULL,NULL) here because Tuple allows to don't skip Nulls and min(tuple(b, a)) in this case is minimal value for this dataset +│ (NULL,NULL) │ (NULL,NULL) │ -- argMin returns (NULL,NULL) here because `Tuple` allows to don't skip `NULL` and min(tuple(b, a)) in this case is minimal value for this dataset └──────────────────────────────────┴──────────────────┘ select argMin(a, tuple(b)) from test; ┌─argMax(a, tuple(b))─┐ -│ d │ -- Tuple can be used in `min` to not skip rows with Null values as b. +│ d │ -- `Tuple` can be used in `min` to not skip rows with `NULL` values as b. └─────────────────────┘ ``` From 68a9ea72dee4ff94ac061544f366ce9a10fc2053 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Jun 2023 16:10:01 +0200 Subject: [PATCH 1265/2223] Revert "Merge pull request #50307 from ZhiguoZh/20230527-toyyyymm" This reverts commit 17261e52da18a739b59895e9cd97dff3e72b66b8, reversing changes made to d302eae85efc7f4c1d29993484c7d8b068e1be62. --- .../OptimizeDateFilterVisitor.cpp | 33 ++------- ..._date_filter_predicate_optimizer.reference | 69 ------------------- .../02764_date_filter_predicate_optimizer.sql | 23 ------- 3 files changed, 5 insertions(+), 120 deletions(-) diff --git a/src/Interpreters/OptimizeDateFilterVisitor.cpp b/src/Interpreters/OptimizeDateFilterVisitor.cpp index aec2dec19c8..58e1b3335f9 100644 --- a/src/Interpreters/OptimizeDateFilterVisitor.cpp +++ b/src/Interpreters/OptimizeDateFilterVisitor.cpp @@ -10,37 +10,14 @@ namespace DB { -ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 compare_to) +ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 year) { const DateLUTImpl & date_lut = DateLUT::instance(); - String start_date; - String end_date; + if (converter != "toYear") return {}; - if (converter == "toYear") - { - UInt64 year = compare_to; - start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); - end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); - } - else if (converter == "toYYYYMM") - { - UInt64 year = compare_to / 100; - UInt64 month = compare_to % 100; - - if (month == 0 || month > 12) return {}; - - static constexpr UInt8 days_of_month[] = {31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; - - bool leap_year = (year & 3) == 0 && (year % 100 || (year % 400 == 0 && year)); - - start_date = date_lut.dateToString(date_lut.makeDayNum(year, month, 1)); - end_date = date_lut.dateToString(date_lut.makeDayNum(year, month, days_of_month[month - 1] + (leap_year && month == 2))); - } - else - { - return {}; - } + String start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); + String end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); if (comparator == "equals") { @@ -105,7 +82,7 @@ bool rewritePredicateInPlace(ASTFunction & function, ASTPtr & ast) { if (const auto * func = function.arguments->children[i]->as(); func) { - if (func->name == "toYear" || func->name == "toYYYYMM") + if (func->name == "toYear") { func_id = i; } diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference index 54704fb3b3e..e5c608ddc1a 100644 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference @@ -37,72 +37,3 @@ WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) A SELECT value1 FROM t WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) -SELECT value1 -FROM t -WHERE ((date1 >= \'1900-02-01\') AND (date1 <= \'1900-02-28\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1992-02-01\') AND (date1 <= \'1992-02-29\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'2000-02-01\') AND (date1 <= \'2000-02-29\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (toYYYYMM(date1) = 199300) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-01-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-02-01\') AND (date1 <= \'1993-02-28\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-03-01\') AND (date1 <= \'1993-03-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-04-01\') AND (date1 <= \'1993-04-30\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-05-01\') AND (date1 <= \'1993-05-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-06-01\') AND (date1 <= \'1993-06-30\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-07-01\') AND (date1 <= \'1993-07-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-08-01\') AND (date1 <= \'1993-08-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-09-01\') AND (date1 <= \'1993-09-30\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-10-01\') AND (date1 <= \'1993-10-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-11-01\') AND (date1 <= \'1993-11-30\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-12-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (toYYYYMM(date1) = 199313) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 < \'1992-03-01\') OR (date1 > \'1992-03-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 < \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 > \'1992-03-31\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 <= \'1992-03-31\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 >= \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1992-03-01\') OR ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\'))) AND ((id >= 1) AND (id <= 3)) diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql index a04273bbb18..563468d4f82 100644 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql @@ -13,28 +13,5 @@ EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM t WHERE year1 = 1993 A EXPLAIN SYNTAX SELECT value1 FROM t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; EXPLAIN SYNTAX SELECT value1 FROM t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; EXPLAIN SYNTAX SELECT value1 FROM t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 190002 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199202 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 200002 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199301 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199302 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199303 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199304 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199305 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199306 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199307 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199308 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199309 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199310 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199311 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3; DROP TABLE t; From 8d2b8683125b3ba1e2ccddff078c3af9820a19af Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Jun 2023 16:10:22 +0200 Subject: [PATCH 1266/2223] Revert "Merge pull request #50062 from ZhiguoZh/20230511-toyear" This reverts commit 55c2dbcc2d1068dae78e7be0929b193edb23d75c, reversing changes made to 23f894b995feb4d0045ba24593bd457e39b7d11d. --- .../OptimizeDateFilterVisitor.cpp | 121 ------------------ src/Interpreters/OptimizeDateFilterVisitor.h | 20 --- src/Interpreters/TreeOptimizer.cpp | 19 --- ..._date_filter_predicate_optimizer.reference | 39 ------ .../02764_date_filter_predicate_optimizer.sql | 17 --- 5 files changed, 216 deletions(-) delete mode 100644 src/Interpreters/OptimizeDateFilterVisitor.cpp delete mode 100644 src/Interpreters/OptimizeDateFilterVisitor.h delete mode 100644 tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference delete mode 100644 tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql diff --git a/src/Interpreters/OptimizeDateFilterVisitor.cpp b/src/Interpreters/OptimizeDateFilterVisitor.cpp deleted file mode 100644 index 58e1b3335f9..00000000000 --- a/src/Interpreters/OptimizeDateFilterVisitor.cpp +++ /dev/null @@ -1,121 +0,0 @@ -#include - -#include -#include -#include -#include -#include - - -namespace DB -{ - -ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 year) -{ - const DateLUTImpl & date_lut = DateLUT::instance(); - - if (converter != "toYear") return {}; - - String start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); - String end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); - - if (comparator == "equals") - { - return makeASTFunction("and", - makeASTFunction("greaterOrEquals", - std::make_shared(column), - std::make_shared(start_date) - ), - makeASTFunction("lessOrEquals", - std::make_shared(column), - std::make_shared(end_date) - ) - ); - } - else if (comparator == "notEquals") - { - return makeASTFunction("or", - makeASTFunction("less", - std::make_shared(column), - std::make_shared(start_date) - ), - makeASTFunction("greater", - std::make_shared(column), - std::make_shared(end_date) - ) - ); - } - else if (comparator == "less" || comparator == "greaterOrEquals") - { - return makeASTFunction(comparator, - std::make_shared(column), - std::make_shared(start_date) - ); - } - else - { - return makeASTFunction(comparator, - std::make_shared(column), - std::make_shared(end_date) - ); - } -} - -bool rewritePredicateInPlace(ASTFunction & function, ASTPtr & ast) -{ - const static std::unordered_map swap_relations = { - {"equals", "equals"}, - {"notEquals", "notEquals"}, - {"less", "greater"}, - {"greater", "less"}, - {"lessOrEquals", "greaterOrEquals"}, - {"greaterOrEquals", "lessOrEquals"}, - }; - - if (!swap_relations.contains(function.name)) return false; - - if (!function.arguments || function.arguments->children.size() != 2) return false; - - size_t func_id = function.arguments->children.size(); - - for (size_t i = 0; i < function.arguments->children.size(); i++) - { - if (const auto * func = function.arguments->children[i]->as(); func) - { - if (func->name == "toYear") - { - func_id = i; - } - } - } - - if (func_id == function.arguments->children.size()) return false; - - size_t literal_id = 1 - func_id; - const auto * literal = function.arguments->children[literal_id]->as(); - - if (!literal || literal->value.getType() != Field::Types::UInt64) return false; - - UInt64 compare_to = literal->value.get(); - String comparator = literal_id > func_id ? function.name : swap_relations.at(function.name); - - const auto * func = function.arguments->children[func_id]->as(); - const auto * column_id = func->arguments->children.at(0)->as(); - - if (!column_id) return false; - - String column = column_id->name(); - - const auto new_ast = generateOptimizedDateFilterAST(comparator, func->name, column, compare_to); - - if (!new_ast) return false; - - ast = new_ast; - return true; -} - -void OptimizeDateFilterInPlaceData::visit(ASTFunction & function, ASTPtr & ast) const -{ - rewritePredicateInPlace(function, ast); -} -} diff --git a/src/Interpreters/OptimizeDateFilterVisitor.h b/src/Interpreters/OptimizeDateFilterVisitor.h deleted file mode 100644 index 84394372901..00000000000 --- a/src/Interpreters/OptimizeDateFilterVisitor.h +++ /dev/null @@ -1,20 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class ASTFunction; - -/// Rewrite the predicates in place -class OptimizeDateFilterInPlaceData -{ -public: - using TypeToVisit = ASTFunction; - void visit(ASTFunction & function, ASTPtr & ast) const; -}; - -using OptimizeDateFilterInPlaceMatcher = OneTypeMatcher; -using OptimizeDateFilterInPlaceVisitor = InDepthNodeVisitor; -} diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 825114b20b7..c38b3c79026 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include @@ -678,21 +677,6 @@ void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, ContextPtr context) RemoveInjectiveFunctionsVisitor(data).visit(query); } -void optimizeDateFilters(ASTSelectQuery * select_query) -{ - /// Predicates in HAVING clause has been moved to WHERE clause. - if (select_query->where()) - { - OptimizeDateFilterInPlaceVisitor::Data data; - OptimizeDateFilterInPlaceVisitor(data).visit(select_query->refWhere()); - } - if (select_query->prewhere()) - { - OptimizeDateFilterInPlaceVisitor::Data data; - OptimizeDateFilterInPlaceVisitor(data).visit(select_query->refPrewhere()); - } -} - void transformIfStringsIntoEnum(ASTPtr & query) { std::unordered_set function_names = {"if", "transform"}; @@ -796,9 +780,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, tables_with_columns, result.storage_snapshot->metadata, result.storage); } - /// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, toISOWeek, etc. - optimizeDateFilters(select_query); - /// GROUP BY injective function elimination. optimizeGroupBy(select_query, context); diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference deleted file mode 100644 index e5c608ddc1a..00000000000 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference +++ /dev/null @@ -1,39 +0,0 @@ -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 < \'1993-01-01\') OR (date1 > \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 > \'1993-12-31\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 <= \'1993-12-31\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 >= \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1997-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) OR ((date1 >= \'1994-01-01\') AND (date1 <= \'1994-12-31\'))) AND ((id >= 1) AND (id <= 3)) -SELECT - value1, - toYear(date1) AS year1 -FROM t -WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -PREWHERE (date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\') -WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql deleted file mode 100644 index 563468d4f82..00000000000 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql +++ /dev/null @@ -1,17 +0,0 @@ -DROP TABLE IF EXISTS t; -CREATE TABLE t (id UInt32, value1 String, date1 Date) ENGINE ReplacingMergeTree() ORDER BY id; - -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM t WHERE year1 = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; - -DROP TABLE t; From 1910d6580e6a6cd7ad985976ed08885b3b091219 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Jun 2023 16:13:51 +0200 Subject: [PATCH 1267/2223] Add test for the reverted broken optimizations --- .../02783_date_predicate_optimizations.reference | 2 ++ .../02783_date_predicate_optimizations.sql | 13 +++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02783_date_predicate_optimizations.reference create mode 100644 tests/queries/0_stateless/02783_date_predicate_optimizations.sql diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.reference b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference new file mode 100644 index 00000000000..cd689b93034 --- /dev/null +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference @@ -0,0 +1,2 @@ +2021-12-31 23:00:00 0 +2021-12-31 23:00:00 0 diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql new file mode 100644 index 00000000000..abb13f1005e --- /dev/null +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql @@ -0,0 +1,13 @@ +CREATE TABLE source +( + `ts` DateTime('UTC'), + `n` Int32 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(ts) +ORDER BY tuple(); + +INSERT INTO source values ('2021-12-31 23:00:00', 0); + +SELECT * FROM source WHERE toYYYYMM(ts) = 202112; +SELECT * FROM source WHERE toYear(ts) = 2021; From 88f2f4f5fc9a0a8b400d74064c44997b2304d27c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 6 Jun 2023 16:16:49 +0200 Subject: [PATCH 1268/2223] Added createAsyncAzureReadBuffer --- .../AzureBlobStorage/AzureObjectStorage.h | 2 + src/Storages/StorageAzure.cpp | 89 ++++++++++++------- src/Storages/StorageAzure.h | 2 + 3 files changed, 63 insertions(+), 30 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index b3cda54e752..f5918f9d598 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -127,6 +127,8 @@ public: bool isRemote() const override { return true; } + MultiVersion getClient() { return client; } + private: const String name; /// client used to access the files in the Blob Storage cloud diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index e0a1d8c514e..95d3ae95a76 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -40,6 +40,9 @@ #include #include +#include +#include + using namespace Azure::Storage::Blobs; @@ -154,11 +157,6 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); - LOG_INFO(&Poco::Logger::get("StorageAzure"), "connection_url = {}", configuration.connection_url); - LOG_INFO(&Poco::Logger::get("StorageAzure"), "container = {}", configuration.container); - LOG_INFO(&Poco::Logger::get("StorageAzure"), "blobpath = {}", configuration.blob_path); - - auto is_format_arg = [] (const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); @@ -230,8 +228,6 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C configuration.blobs_paths = {configuration.blob_path}; - LOG_INFO(&Poco::Logger::get("StorageAzure"), "get_format_from_file = {}", get_format_from_file); - if (configuration.format == "auto" && get_format_from_file) configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); @@ -310,7 +306,6 @@ AzureClientPtr StorageAzure::createClient(StorageAzure::Configuration configurat if (configuration.is_connection_string) { - LOG_INFO(&Poco::Logger::get("StorageAzure"), "createClient is_connection_string "); result = std::make_unique(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container)); result->CreateIfNotExists(); } @@ -416,18 +411,6 @@ StorageAzure::StorageAzure( for (const auto & key : configuration.blobs_paths) objects.emplace_back(key); - for (auto obj : objects) - { - LOG_INFO(&Poco::Logger::get("StorageAzure"), "constructor obj.remote_paths = {}", obj.remote_path); - if (object_storage->exists(obj)) - { - LOG_INFO(&Poco::Logger::get("StorageAzure"), "constructor exists obj.remote_paths = {}", obj.remote_path); -// auto read_buffer = object_storage->readObject(obj); -// LOG_INFO(&Poco::Logger::get("StorageAzure"), "constructor read size obj.remote_paths = {} , size = {}", obj.remote_path, read_buffer->getFileSize()); - } - } - - auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, {"_file", std::make_shared(std::make_shared())}}; @@ -1146,22 +1129,68 @@ std::unique_ptr StorageAzureSource::createAzureReadBuffer(const Stri { auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); read_settings.enable_filesystem_cache = false; - //auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; - //const bool object_too_small = object_size <= 2 * download_buffer_size; + auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; + const bool object_too_small = object_size <= 2 * download_buffer_size; - ///// Create a read buffer that will prefetch the first ~1 MB of the file. - ///// When reading lots of tiny files, this prefetching almost doubles the throughput. - ///// For bigger files, parallel reading is more useful. - //if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - //{ - // LOG_TRACE(log, "Downloading object {} of size {} from S3 with initial prefetch", key, object_size); - // return object_storage->readObjects({StoredObject(key)}, read_settings, {}, object_size); - //} + // Create a read buffer that will prefetch the first ~1 MB of the file. + // When reading lots of tiny files, this prefetching almost doubles the throughput. + // For bigger files, parallel reading is more useful. + if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) + { + LOG_TRACE(log, "Downloading object of size {} from Azure with initial prefetch", object_size); + return createAsyncAzureReadBuffer(key, read_settings, object_size); + } return object_storage->readObject(StoredObject(key), read_settings, {}, object_size); } +std::unique_ptr StorageAzureSource::createAsyncAzureReadBuffer( + const String & key, const ReadSettings & read_settings, size_t object_size) +{ + auto context = getContext(); + + const auto & context_settings = context->getSettingsRef(); + auto max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; + auto max_single_read_retries = context_settings.azure_max_single_read_retries; + + auto read_buffer_creator = + [this, read_settings, max_single_part_upload_size, max_single_read_retries] + (const std::string & path, size_t read_until_position) -> std::unique_ptr + { + return std::make_unique( + object_storage->getClient().get(), + path, + read_settings, + max_single_part_upload_size, + max_single_read_retries, + /* use_external_buffer */true, + read_until_position); + }; + + auto azure_impl = std::make_unique( + std::move(read_buffer_creator), + StoredObjects{StoredObject{key, object_size}}, + read_settings, + /* cache_log */nullptr); + + auto modified_settings{read_settings}; + /// FIXME: Changing this setting to default value breaks something around parquet reading + modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size; + + auto & pool_reader = context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + + auto async_reader = std::make_unique( + std::move(azure_impl), pool_reader, modified_settings, + context->getAsyncReadCounters(), context->getFilesystemReadPrefetchesLog()); + + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); + + return async_reader; +} + } #endif diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index f114184c336..cf3ed5e2596 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -274,6 +274,8 @@ private: std::future createReaderAsync(); std::unique_ptr createAzureReadBuffer(const String & key, size_t object_size); + std::unique_ptr createAsyncAzureReadBuffer( + const String & key, const ReadSettings & read_settings, size_t object_size); }; } From 75d0f9fc97f284a47eb8b4e8b03c3089631c46d4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 6 Jun 2023 16:32:05 +0200 Subject: [PATCH 1269/2223] Updated to use readObjects for async --- src/Storages/StorageAzure.cpp | 34 +--------------------------------- 1 file changed, 1 insertion(+), 33 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 95d3ae95a76..c98430858eb 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -1148,41 +1148,9 @@ std::unique_ptr StorageAzureSource::createAzureReadBuffer(const Stri std::unique_ptr StorageAzureSource::createAsyncAzureReadBuffer( const String & key, const ReadSettings & read_settings, size_t object_size) { - auto context = getContext(); - - const auto & context_settings = context->getSettingsRef(); - auto max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; - auto max_single_read_retries = context_settings.azure_max_single_read_retries; - - auto read_buffer_creator = - [this, read_settings, max_single_part_upload_size, max_single_read_retries] - (const std::string & path, size_t read_until_position) -> std::unique_ptr - { - return std::make_unique( - object_storage->getClient().get(), - path, - read_settings, - max_single_part_upload_size, - max_single_read_retries, - /* use_external_buffer */true, - read_until_position); - }; - - auto azure_impl = std::make_unique( - std::move(read_buffer_creator), - StoredObjects{StoredObject{key, object_size}}, - read_settings, - /* cache_log */nullptr); - auto modified_settings{read_settings}; - /// FIXME: Changing this setting to default value breaks something around parquet reading modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size; - - auto & pool_reader = context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - auto async_reader = std::make_unique( - std::move(azure_impl), pool_reader, modified_settings, - context->getAsyncReadCounters(), context->getFilesystemReadPrefetchesLog()); + auto async_reader = object_storage->readObjects(StoredObjects{StoredObject{key, object_size}}, modified_settings); async_reader->setReadUntilEnd(); if (read_settings.remote_fs_prefetch) From cbe4ea67aec5f285e3cb3fd905917679041ced0a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 6 Jun 2023 16:42:56 +0200 Subject: [PATCH 1270/2223] Removed unwanted code & debug lines --- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 2 -- src/TableFunctions/TableFunctionS3.cpp | 4 ---- 2 files changed, 6 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index f5918f9d598..b3cda54e752 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -127,8 +127,6 @@ public: bool isRemote() const override { return true; } - MultiVersion getClient() { return client; } - private: const String name; /// client used to access the files in the Blob Storage cloud diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 7f283afd6b4..e63f32b1cbc 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -294,8 +294,6 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String & ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) const { - LOG_INFO(&Poco::Logger::get("TableFunctionS3"), "getActualTableStructure configuration.structure = {} ",configuration.structure); - if (configuration.structure == "auto") { context->checkAccess(getSourceAccessType()); @@ -321,8 +319,6 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context else if (!structure_hint.empty()) columns = structure_hint; - LOG_INFO(&Poco::Logger::get("TableFunctionS3"), "executeImpl structre = {} structure_hint = {} ",configuration.structure, structure_hint.getAll().toString()); - StoragePtr storage = std::make_shared( configuration, From ebae79f7d410dd67879e041d2db4395d852bcae3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 16:57:51 +0200 Subject: [PATCH 1271/2223] Schema inference --- src/Core/Settings.h | 1 + src/Storages/StorageAzure.cpp | 145 ++++++++++++++++-- src/Storages/StorageAzure.h | 28 ++-- .../test_storage_azure_blob_storage/test.py | 48 ++++++ 4 files changed, 200 insertions(+), 22 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a484e8e816d..b95dc9a26b3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -710,6 +710,7 @@ class IColumn; \ M(Bool, schema_inference_use_cache_for_file, true, "Use cache in schema inference while using file table function", 0) \ M(Bool, schema_inference_use_cache_for_s3, true, "Use cache in schema inference while using s3 table function", 0) \ + M(Bool, schema_inference_use_cache_for_azure, true, "Use cache in schema inference while using azure table function", 0) \ M(Bool, schema_inference_use_cache_for_hdfs, true, "Use cache in schema inference while using hdfs table function", 0) \ M(Bool, schema_inference_use_cache_for_url, true, "Use cache in schema inference while using url table function", 0) \ M(Bool, schema_inference_cache_require_modification_time_for_url, true, "Use schema from cache for URL with last modification time validation (for urls with Last-Modified header)", 0) \ diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index e0a1d8c514e..4c47fe318c6 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -58,6 +58,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int DATABASE_ACCESS_DENIED; extern const int CANNOT_COMPILE_REGEXP; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } namespace @@ -403,7 +404,8 @@ StorageAzure::StorageAzure( StorageInMemoryMetadata storage_metadata; if (columns_.empty()) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Schema inference is not supported yet"); + auto columns = getTableStructureFromDataImpl(context); + storage_metadata.setColumns(columns); } else storage_metadata.setColumns(columns_); @@ -624,13 +626,13 @@ Pipe StorageAzure::read( /// Iterate through disclosed globs and make a source for each file iterator_wrapper = std::make_shared( object_storage.get(), configuration.container, std::nullopt, - configuration.blob_path, query_info.query, virtual_block, local_context); + configuration.blob_path, query_info.query, virtual_block, local_context, nullptr); } else { iterator_wrapper = std::make_shared( object_storage.get(), configuration.container, configuration.blobs_paths, - std::nullopt, query_info.query, virtual_block, local_context); + std::nullopt, query_info.query, virtual_block, local_context, nullptr); } ColumnsDescription columns_description; @@ -800,7 +802,8 @@ StorageAzureSource::Iterator::Iterator( std::optional blob_path_with_globs_, ASTPtr query_, const Block & virtual_header_, - ContextPtr context_) + ContextPtr context_, + RelativePathsWithMetadata * outer_blobs_) : WithContext(context_) , object_storage(object_storage_) , container(container_) @@ -808,6 +811,7 @@ StorageAzureSource::Iterator::Iterator( , blob_path_with_globs(blob_path_with_globs_) , query(query_) , virtual_header(virtual_header_) + , outer_blobs(outer_blobs_) { if (keys.has_value() && blob_path_with_globs.has_value()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify keys and glob simulatenously it's a bug"); @@ -854,6 +858,8 @@ StorageAzureSource::Iterator::Iterator( ObjectMetadata object_metadata = object_storage->getObjectMetadata(key); total_size += object_metadata.size_bytes; blobs_with_metadata->emplace_back(RelativePathWithMetadata{key, object_metadata}); + if (outer_blobs) + outer_blobs->emplace_back(blobs_with_metadata->back()); } } else @@ -866,6 +872,8 @@ StorageAzureSource::Iterator::Iterator( { ObjectMetadata object_metadata = object_storage->getObjectMetadata(*blob_path_with_globs); blobs_with_metadata->emplace_back(*blob_path_with_globs, object_metadata); + if (outer_blobs) + outer_blobs->emplace_back(blobs_with_metadata->back()); return; } @@ -903,37 +911,29 @@ RelativePathWithMetadata StorageAzureSource::Iterator::next() } else { - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "GLOBS IN NEXt"); if (!blobs_with_metadata || index >= blobs_with_metadata->size()) { - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "INITIALIZING BLOBS BATCH"); RelativePathsWithMetadata new_batch; while (new_batch.empty()) { if (object_storage_iterator->isValid()) { - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "ITERATOR VALID FETCHING BATCH"); new_batch = object_storage_iterator->currentBatch(); - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "BATCH SIZE {}", new_batch.size()); object_storage_iterator->nextBatch(); } else { - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "ITERATOR INVALID"); is_finished = true; return {}; } for (auto it = new_batch.begin(); it != new_batch.end();) { - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "ITERATOR FILTER {} MATCH {}", it->relative_path, re2::RE2::FullMatch(it->relative_path, *matcher)); if (!recursive && !re2::RE2::FullMatch(it->relative_path, *matcher)) it = new_batch.erase(it); else ++it; } - - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "NEW BATCH AFTER FILTEr {}", new_batch.size()); } index.store(0, std::memory_order_relaxed); @@ -958,10 +958,15 @@ RelativePathWithMetadata StorageAzureSource::Iterator::next() { total_size.fetch_add(new_batch[idx].metadata.size_bytes, std::memory_order_relaxed); blobs_with_metadata->emplace_back(std::move(new_batch[idx])); + if (outer_blobs) + outer_blobs->emplace_back(blobs_with_metadata->back()); } } else { + if (outer_blobs) + outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); + blobs_with_metadata = std::move(new_batch); for (const auto & [_, info] : *blobs_with_metadata) total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); @@ -1161,6 +1166,122 @@ std::unique_ptr StorageAzureSource::createAzureReadBuffer(const Stri return object_storage->readObject(StoredObject(key), read_settings, {}, object_size); } +ColumnsDescription StorageAzure::getTableStructureFromDataImpl(ContextPtr ctx) +{ + RelativePathsWithMetadata read_keys; + std::shared_ptr file_iterator; + if (configuration.withGlobs()) + { + file_iterator = std::make_shared( + object_storage.get(), configuration.container, std::nullopt, + configuration.blob_path, nullptr, virtual_block, ctx, &read_keys); + } + else + { + file_iterator = std::make_shared( + object_storage.get(), configuration.container, configuration.blobs_paths, + std::nullopt, nullptr, virtual_block, ctx, &read_keys); + } + + std::optional columns_from_cache; + size_t prev_read_keys_size = read_keys.size(); + if (ctx->getSettingsRef().schema_inference_use_cache_for_azure) + columns_from_cache = tryGetColumnsFromCache(read_keys.begin(), read_keys.end(), ctx); + + ReadBufferIterator read_buffer_iterator = [&, first = true](ColumnsDescription & cached_columns) mutable -> std::unique_ptr + { + auto [key, metadata] = file_iterator->next(); + + if (key.empty()) + { + if (first) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because there are no files with provided path " + "in S3. You must specify table structure manually", configuration.format); + + return nullptr; + } + + /// S3 file iterator could get new keys after new iteration, check them in schema cache. + if (ctx->getSettingsRef().schema_inference_use_cache_for_azure && read_keys.size() > prev_read_keys_size) + { + columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), ctx); + prev_read_keys_size = read_keys.size(); + if (columns_from_cache) + { + cached_columns = *columns_from_cache; + return nullptr; + } + } + + first = false; + int zstd_window_log_max = static_cast(ctx->getSettingsRef().zstd_window_log_max); + return wrapReadBufferWithCompressionMethod( + object_storage->readObject(StoredObject(key), ctx->getReadSettings(), {}, metadata.size_bytes), + chooseCompressionMethod(key, configuration.compression_method), + zstd_window_log_max); + }; + + ColumnsDescription columns; + if (columns_from_cache) + columns = *columns_from_cache; + else + columns = readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx); + + if (ctx->getSettingsRef().schema_inference_use_cache_for_azure) + addColumnsToCache(read_keys, columns, configuration.format, ctx); + + return columns; + +} + +std::optional StorageAzure::tryGetColumnsFromCache( + const RelativePathsWithMetadata::const_iterator & begin, + const RelativePathsWithMetadata::const_iterator & end, + const ContextPtr & ctx) +{ + auto & schema_cache = getSchemaCache(ctx); + for (auto it = begin; it < end; ++it) + { + auto get_last_mod_time = [&] -> time_t + { + return it->metadata.last_modified->epochTime(); + }; + + auto host_and_bucket = configuration.connection_url + '/' + configuration.container; + String source = host_and_bucket + '/' + it->relative_path; + auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, ctx); + auto columns = schema_cache.tryGet(cache_key, get_last_mod_time); + if (columns) + return columns; + } + + return std::nullopt; + +} + +void StorageAzure::addColumnsToCache( + const RelativePathsWithMetadata & keys, + const ColumnsDescription & columns, + const String & format_name, + const ContextPtr & ctx) +{ + auto host_and_bucket = configuration.connection_url + '/' + configuration.container; + Strings sources; + sources.reserve(keys.size()); + std::transform(keys.begin(), keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket + '/' + elem.relative_path; }); + auto cache_keys = getKeysForSchemaCache(sources, format_name, format_settings, ctx); + auto & schema_cache = getSchemaCache(ctx); + schema_cache.addMany(cache_keys, columns); +} + +SchemaCache & StorageAzure::getSchemaCache(const ContextPtr & ctx) +{ + static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_azure", DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; +} + } diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index f114184c336..83bcb874efc 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -79,10 +79,6 @@ public: static StorageAzure::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file = true); static AzureClientPtr createClient(StorageAzure::Configuration configuration); static AzureObjectStorage::SettingsPtr createSettings(StorageAzure::Configuration configuration); - static ColumnsDescription getTableStructureFromData( - const StorageAzure::Configuration & configuration, - const std::optional & format_settings, - ContextPtr ctx); String getName() const override { @@ -127,10 +123,19 @@ private: std::optional format_settings; ASTPtr partition_by; - static ColumnsDescription getTableStructureFromDataImpl( - const Configuration & configuration, - const std::optional & format_settings, - ContextPtr ctx); + ColumnsDescription getTableStructureFromDataImpl(ContextPtr ctx); + + std::optional tryGetColumnsFromCache( + const RelativePathsWithMetadata::const_iterator & begin, + const RelativePathsWithMetadata::const_iterator & end, + const ContextPtr & ctx); + + void addColumnsToCache( + const RelativePathsWithMetadata & keys, + const ColumnsDescription & columns, + const String & format_name, + const ContextPtr & ctx); + }; @@ -147,12 +152,14 @@ public: std::optional blob_path_with_globs_, ASTPtr query_, const Block & virtual_header_, - ContextPtr context_); + ContextPtr context_, + RelativePathsWithMetadata * outer_blobs_); RelativePathWithMetadata next(); size_t getTotalSize() const; ~Iterator() = default; - private: + + private: AzureObjectStorage * object_storage; std::string container; std::optional keys; @@ -165,6 +172,7 @@ public: std::atomic total_size = 0; std::optional blobs_with_metadata; + RelativePathsWithMetadata * outer_blobs; ObjectStorageIteratorPtr object_storage_iterator; bool recursive{false}; diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 221005e414b..83a7bf71181 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -373,3 +373,51 @@ def test_storage_azure_get_gzip(cluster, extension, method): assert azure_query(node, f"SELECT sum(id) FROM {name}").splitlines() == ["565"] azure_query(node, f"DROP TABLE {name}") + + +def test_schema_inference_no_globs(cluster): + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 String, column3 UInt32" + azure_query(node, f"CREATE TABLE test_schema_inference_src ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv', format='CSVWithNames')") + + query = f"insert into test_schema_inference_src SELECT number, toString(number), number * number FROM numbers(1000)" + azure_query(node, query) + + azure_query(node, f"CREATE TABLE test_select_inference Engine = Azure(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv')") + + print(node.query("SHOW CREATE TABLE test_select_inference")) + + query = "select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from test_select_inference" + assert azure_query(node, query).splitlines() == ['499500\t2890\t332833500\ttest_schema_inference_no_globs.csv\tcont/test_schema_inference_no_globs.csv'] + + +def test_schema_inference_from_globs(cluster): + node = cluster.instances["node"] + unique_prefix = random.randint(1, 10000) + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + max_path = "" + for i in range(10): + for j in range(10): + path = "{}/{}_{}/{}.csv".format( + unique_prefix, i, random.choice(["a", "b", "c", "d"]), j + ) + max_path = max(path, max_path) + values = f"({i},{j},{i + j})" + + azure_query(node, f"CREATE TABLE test_schema_{i}_{j} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames')") + + query = f"insert into test_schema_{i}_{j} VALUES {values}" + azure_query(node, query) + + + azure_query(node, f"CREATE TABLE test_glob_select_inference Engine = Azure(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')") + + print(node.query("SHOW CREATE TABLE test_glob_select_inference")) + + query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from test_glob_select_inference" + assert azure_query(node, query).splitlines() == [ + "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( + bucket='cont', max_path=max_path + ) + ] From 3bda231203644989f0be3edcc390271f14d72cb4 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 6 Jun 2023 15:11:21 +0000 Subject: [PATCH 1272/2223] Automatic style fix --- .../test_storage_azure_blob_storage/test.py | 27 ++++++++++++++----- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 9b71ff1a490..621af773160 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -400,17 +400,25 @@ def test_storage_azure_get_gzip(cluster, extension, method): def test_schema_inference_no_globs(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 String, column3 UInt32" - azure_query(node, f"CREATE TABLE test_schema_inference_src ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv', format='CSVWithNames')") + azure_query( + node, + f"CREATE TABLE test_schema_inference_src ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv', format='CSVWithNames')", + ) query = f"insert into test_schema_inference_src SELECT number, toString(number), number * number FROM numbers(1000)" azure_query(node, query) - azure_query(node, f"CREATE TABLE test_select_inference Engine = Azure(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv')") + azure_query( + node, + f"CREATE TABLE test_select_inference Engine = Azure(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv')", + ) print(node.query("SHOW CREATE TABLE test_select_inference")) query = "select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from test_select_inference" - assert azure_query(node, query).splitlines() == ['499500\t2890\t332833500\ttest_schema_inference_no_globs.csv\tcont/test_schema_inference_no_globs.csv'] + assert azure_query(node, query).splitlines() == [ + "499500\t2890\t332833500\ttest_schema_inference_no_globs.csv\tcont/test_schema_inference_no_globs.csv" + ] def test_schema_inference_from_globs(cluster): @@ -427,19 +435,24 @@ def test_schema_inference_from_globs(cluster): max_path = max(path, max_path) values = f"({i},{j},{i + j})" - azure_query(node, f"CREATE TABLE test_schema_{i}_{j} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames')") + azure_query( + node, + f"CREATE TABLE test_schema_{i}_{j} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames')", + ) query = f"insert into test_schema_{i}_{j} VALUES {values}" azure_query(node, query) - - azure_query(node, f"CREATE TABLE test_glob_select_inference Engine = Azure(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')") + azure_query( + node, + f"CREATE TABLE test_glob_select_inference Engine = Azure(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')", + ) print(node.query("SHOW CREATE TABLE test_glob_select_inference")) query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from test_glob_select_inference" assert azure_query(node, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( - bucket='cont', max_path=max_path + bucket="cont", max_path=max_path ) ] From 6c82ee45e2f1886219967dbadcde8e6a59bd84b1 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 6 Jun 2023 18:27:16 +0200 Subject: [PATCH 1273/2223] Fix build --- src/Interpreters/tests/gtest_convertFieldToType.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_convertFieldToType.cpp b/src/Interpreters/tests/gtest_convertFieldToType.cpp index f4de36cbecc..e259418d1c3 100644 --- a/src/Interpreters/tests/gtest_convertFieldToType.cpp +++ b/src/Interpreters/tests/gtest_convertFieldToType.cpp @@ -9,7 +9,6 @@ #include #include "base/Decimal.h" #include "base/types.h" -#include "gtest/gtest.h" using namespace DB; From 38edd6c3e778aad7a5a8a38c294c0c1e340990a1 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 6 Jun 2023 18:28:34 +0200 Subject: [PATCH 1274/2223] Update src/Interpreters/tests/gtest_convertFieldToType.cpp --- src/Interpreters/tests/gtest_convertFieldToType.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_convertFieldToType.cpp b/src/Interpreters/tests/gtest_convertFieldToType.cpp index e259418d1c3..cda9311dcbe 100644 --- a/src/Interpreters/tests/gtest_convertFieldToType.cpp +++ b/src/Interpreters/tests/gtest_convertFieldToType.cpp @@ -56,7 +56,7 @@ TEST_P(ConvertFieldToTypeTest, convert) } // Basically, the number of seconds in a day works for UTC here -const long long int Day = 24 * 60 * 60; +const Int64 Day = 24 * 60 * 60; // 123 is arbitrary value here From 473743b49fa7f37ee869d694e9233b4678efbd98 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Jun 2023 18:38:32 +0200 Subject: [PATCH 1275/2223] Disable pure parallel replicas if trivial count optimization is possible (#50594) --- src/Interpreters/InterpreterSelectQuery.cpp | 116 ++++++++++-------- src/Interpreters/InterpreterSelectQuery.h | 2 + src/Planner/PlannerJoinTree.cpp | 19 ++- ...licas_trivial_count_optimization.reference | 12 ++ ...lel_replicas_trivial_count_optimization.sh | 95 ++++++++++++++ 5 files changed, 191 insertions(+), 53 deletions(-) create mode 100644 tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.reference create mode 100755 tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d2be48dafb3..e84a400a220 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -833,6 +833,19 @@ InterpreterSelectQuery::InterpreterSelectQuery( need_analyze_again = true; } + if (can_analyze_again + && settings.max_parallel_replicas > 1 + && settings.allow_experimental_parallel_reading_from_replicas > 0 + && settings.parallel_replicas_custom_key.value.empty() + && getTrivialCount(0).has_value()) + { + /// The query could use trivial count if it didn't use parallel replicas, so let's disable it and reanalyze + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + context->setSetting("max_parallel_replicas", UInt64{0}); + need_analyze_again = true; + LOG_TRACE(log, "Disabling parallel replicas to be able to use a trivial count optimization"); + } + if (need_analyze_again) { size_t current_query_analyze_count = context->getQueryContext()->kitchen_sink.analyze_counter.load(); @@ -2254,79 +2267,84 @@ void InterpreterSelectQuery::addPrewhereAliasActions() } } -void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan) +/// Based on the query analysis, check if optimizing the count trivial count to use totalRows is possible +std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_parallel_replicas) { - auto & query = getSelectQuery(); const Settings & settings = context->getSettingsRef(); - - /// Optimization for trivial query like SELECT count() FROM table. bool optimize_trivial_count = syntax_analyzer_result->optimize_trivial_count - && (settings.max_parallel_replicas <= 1) + && (max_parallel_replicas <= 1) && !settings.allow_experimental_query_deduplication && !settings.empty_result_for_aggregation_by_empty_set && storage && storage->getName() != "MaterializedMySQL" && !storage->hasLightweightDeletedMask() && query_info.filter_asts.empty() - && processing_stage == QueryProcessingStage::FetchColumns && query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) && typeid_cast(query_analyzer->aggregates()[0].function.get()); - if (optimize_trivial_count) + if (!optimize_trivial_count) + return {}; + + auto & query = getSelectQuery(); + if (!query.prewhere() && !query.where() && !context->getCurrentTransaction()) + { + return storage->totalRows(settings); + } + else + { + // It's possible to optimize count() given only partition predicates + SelectQueryInfo temp_query_info; + temp_query_info.query = query_ptr; + temp_query_info.syntax_analyzer_result = syntax_analyzer_result; + temp_query_info.prepared_sets = query_analyzer->getPreparedSets(); + + return storage->totalRowsByPartitionPredicate(temp_query_info, context); + } +} + +void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan) +{ + auto & query = getSelectQuery(); + const Settings & settings = context->getSettingsRef(); + std::optional num_rows; + + /// Optimization for trivial query like SELECT count() FROM table. + if (processing_stage == QueryProcessingStage::FetchColumns && (num_rows = getTrivialCount(settings.max_parallel_replicas))) { const auto & desc = query_analyzer->aggregates()[0]; const auto & func = desc.function; - std::optional num_rows{}; + const AggregateFunctionCount & agg_count = static_cast(*func); - if (!query.prewhere() && !query.where() && !context->getCurrentTransaction()) - { - num_rows = storage->totalRows(settings); - } - else // It's possible to optimize count() given only partition predicates - { - SelectQueryInfo temp_query_info; - temp_query_info.query = query_ptr; - temp_query_info.syntax_analyzer_result = syntax_analyzer_result; - temp_query_info.prepared_sets = query_analyzer->getPreparedSets(); + /// We will process it up to "WithMergeableState". + std::vector state(agg_count.sizeOfData()); + AggregateDataPtr place = state.data(); - num_rows = storage->totalRowsByPartitionPredicate(temp_query_info, context); - } + agg_count.create(place); + SCOPE_EXIT_MEMORY_SAFE(agg_count.destroy(place)); - if (num_rows) - { - const AggregateFunctionCount & agg_count = static_cast(*func); + agg_count.set(place, *num_rows); - /// We will process it up to "WithMergeableState". - std::vector state(agg_count.sizeOfData()); - AggregateDataPtr place = state.data(); + auto column = ColumnAggregateFunction::create(func); + column->insertFrom(place); - agg_count.create(place); - SCOPE_EXIT_MEMORY_SAFE(agg_count.destroy(place)); + Block header = analysis_result.before_aggregation->getResultColumns(); + size_t arguments_size = desc.argument_names.size(); + DataTypes argument_types(arguments_size); + for (size_t j = 0; j < arguments_size; ++j) + argument_types[j] = header.getByName(desc.argument_names[j]).type; - agg_count.set(place, *num_rows); + Block block_with_count{ + {std::move(column), std::make_shared(func, argument_types, desc.parameters), desc.column_name}}; - auto column = ColumnAggregateFunction::create(func); - column->insertFrom(place); - - Block header = analysis_result.before_aggregation->getResultColumns(); - size_t arguments_size = desc.argument_names.size(); - DataTypes argument_types(arguments_size); - for (size_t j = 0; j < arguments_size; ++j) - argument_types[j] = header.getByName(desc.argument_names[j]).type; - - Block block_with_count{ - {std::move(column), std::make_shared(func, argument_types, desc.parameters), desc.column_name}}; - - auto source = std::make_shared(block_with_count); - auto prepared_count = std::make_unique(Pipe(std::move(source))); - prepared_count->setStepDescription("Optimized trivial count"); - query_plan.addStep(std::move(prepared_count)); - from_stage = QueryProcessingStage::WithMergeableState; - analysis_result.first_stage = false; - return; - } + auto source = std::make_shared(block_with_count); + auto prepared_count = std::make_unique(Pipe(std::move(source))); + prepared_count->setStepDescription("Optimized trivial count"); + query_plan.addStep(std::move(prepared_count)); + from_stage = QueryProcessingStage::WithMergeableState; + analysis_result.first_stage = false; + return; } /// Limitation on the number of columns to read. diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index e39dd675136..0739e818cd6 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -187,6 +188,7 @@ private: void executeExtremes(QueryPlan & query_plan); void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan); bool autoFinalOnQuery(ASTSelectQuery & select_query); + std::optional getTrivialCount(UInt64 max_parallel_replicas); enum class Modificator { diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4f091f73187..9672738ae6b 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -170,7 +170,7 @@ bool applyTrivialCountIfPossible( QueryPlan & query_plan, const TableNode & table_node, const QueryTreeNodePtr & query_tree, - const ContextPtr & query_context, + ContextMutablePtr & query_context, const Names & columns_names) { const auto & settings = query_context->getSettingsRef(); @@ -208,8 +208,7 @@ bool applyTrivialCountIfPossible( if (storage->hasLightweightDeletedMask()) return false; - if (settings.max_parallel_replicas > 1 || - settings.allow_experimental_query_deduplication + if (settings.allow_experimental_query_deduplication || settings.empty_result_for_aggregation_by_empty_set) return false; @@ -228,6 +227,18 @@ bool applyTrivialCountIfPossible( if (!num_rows) return false; + if (settings.max_parallel_replicas > 1) + { + if (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas == 0) + return false; + + /// The query could use trivial count if it didn't use parallel replicas, so let's disable it + query_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + query_context->setSetting("max_parallel_replicas", UInt64{0}); + LOG_TRACE(&Poco::Logger::get("Planner"), "Disabling parallel replicas to be able to use a trivial count optimization"); + + } + /// Set aggregation state const AggregateFunctionCount & agg_count = *count_func; std::vector state(agg_count.sizeOfData()); @@ -619,7 +630,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres is_single_table_expression && table_node && select_query_info.has_aggregates && - applyTrivialCountIfPossible(query_plan, *table_node, select_query_info.query_tree, planner_context->getQueryContext(), table_expression_data.getColumnNames()); + applyTrivialCountIfPossible(query_plan, *table_node, select_query_info.query_tree, planner_context->getMutableQueryContext(), table_expression_data.getColumnNames()); if (is_trivial_count_applied) { diff --git a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.reference b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.reference new file mode 100644 index 00000000000..48795e2cd39 --- /dev/null +++ b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.reference @@ -0,0 +1,12 @@ +100000 +100000 +100000 +100000 +100000 +100000 +02783_count-default_0_disabled Not parallel 1 16 +02783_count-default_0_pure Not parallel 1 16 +02783_count-default_0_pure_analyzer Not parallel 1 16 +02783_count-default_1_disabled Not parallel 1 16 +02783_count-default_1_pure Not parallel 1 16 +02783_count-default_1_pure_analyzer Not parallel 1 16 diff --git a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh new file mode 100755 index 00000000000..4c29e513183 --- /dev/null +++ b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh @@ -0,0 +1,95 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function has_used_parallel_replicas () { + $CLICKHOUSE_CLIENT --query " + SELECT + initial_query_id, + if(count() != 2, 'Used parallel', 'Not parallel'), + sumIf(read_rows, is_initial_query) as read_rows, + sumIf(read_bytes, is_initial_query) as read_bytes + 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 + FORMAT TSV" +} + +function run_query_with_pure_parallel_replicas () { + $CLICKHOUSE_CLIENT \ + --query "$2" \ + --query_id "${1}_disabled" \ + --max_parallel_replicas 0 + + $CLICKHOUSE_CLIENT \ + --query "$2" \ + --query_id "${1}_pure" \ + --max_parallel_replicas 3 \ + --prefer_localhost_replica 1 \ + --use_hedged_requests 0 \ + --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ + --allow_experimental_parallel_reading_from_replicas 1 \ + --allow_experimental_analyzer 0 + + # Not implemented yet + $CLICKHOUSE_CLIENT \ + --query "$2" \ + --query_id "${1}_pure_analyzer" \ + --max_parallel_replicas 3 \ + --prefer_localhost_replica 1 \ + --use_hedged_requests 0 \ + --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ + --allow_experimental_parallel_reading_from_replicas 1 \ + --allow_experimental_analyzer 1 +} + +function run_query_with_custom_key_parallel_replicas () { + $CLICKHOUSE_CLIENT \ + --query "$2" \ + --query_id "${1}_disabled" \ + --max_parallel_replicas 0 + + $CLICKHOUSE_CLIENT \ + --query "$2" \ + --query_id "${1}_custom_key" \ + --max_parallel_replicas 3 \ + --use_hedged_requests 0 \ + --parallel_replicas_custom_key_filter_type 'default' \ + --parallel_replicas_custom_key "$2" \ + --allow_experimental_analyzer 0 + + $CLICKHOUSE_CLIENT \ + --query "$2" \ + --query_id "${1}_custom_key_analyzer" \ + --max_parallel_replicas 3 \ + --use_hedged_requests 0 \ + --parallel_replicas_custom_key_filter_type 'default' \ + --parallel_replicas_custom_key "$2" \ + --allow_experimental_analyzer 1 +} + +$CLICKHOUSE_CLIENT --query " + CREATE TABLE replicated_numbers + ( + number Int64, + ) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/replicated_numbers', 'r1') + ORDER BY (number) + AS SELECT number FROM numbers(100000); +" + +query_id_base="02783_count-$CLICKHOUSE_DATABASE" + +run_query_with_pure_parallel_replicas "${query_id_base}_0" "SELECT count() FROM replicated_numbers" +run_query_with_pure_parallel_replicas "${query_id_base}_1" "SELECT * FROM (SELECT count() FROM replicated_numbers) LIMIT 20" + +# Not implemented yet as the query fails to execute correctly to begin with +#run_query_with_custom_key_parallel_replicas "${query_id_base}_2" "SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), replicated_numbers)" "sipHash64(number)" +#run_query_with_custom_key_parallel_replicas "${query_id_base}_3" "SELECT * FROM (SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), replicated_numbers)) LIMIT 20" "sipHash64(number)" + + +$CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" +has_used_parallel_replicas "${query_id_base}" From c910f0034b20fcdbe3c336d6851fdd1c7bb21138 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 18:48:20 +0200 Subject: [PATCH 1276/2223] Some code for table function --- src/Storages/StorageAzure.cpp | 52 ++++--- src/Storages/StorageAzure.h | 22 ++- src/TableFunctions/TableFunctionAzure.cpp | 175 ++++++++++++++++++++-- src/TableFunctions/TableFunctionAzure.h | 8 +- 4 files changed, 214 insertions(+), 43 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index e725aa17dd6..e3051236118 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -67,12 +67,12 @@ namespace ErrorCodes namespace { -static const std::unordered_set required_configuration_keys = { +const std::unordered_set required_configuration_keys = { "blob_path", "container", }; -static const std::unordered_set optional_configuration_keys = { +const std::unordered_set optional_configuration_keys = { "format", "compression", "compression_method", @@ -87,8 +87,9 @@ bool isConnectionString(const std::string & candidate) return candidate.starts_with("DefaultEndpointsProtocol"); } +} -void processNamedCollectionResult(StorageAzure::Configuration & configuration, const NamedCollection & collection) +void StorageAzure::processNamedCollectionResult(StorageAzure::Configuration & configuration, const NamedCollection & collection) { validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); @@ -113,11 +114,11 @@ void processNamedCollectionResult(StorageAzure::Configuration & configuration, c if (collection.has("account_key")) configuration.account_key = collection.get("account_key"); + configuration.structure = collection.getOrDefault("structure", "auto"); configuration.format = collection.getOrDefault("format", configuration.format); configuration.compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); } -} StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file) { @@ -236,6 +237,17 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C } +AzureObjectStorage::SettingsPtr StorageAzure::createSettings(ContextPtr local_context) +{ + const auto & context_settings = local_context->getSettingsRef(); + auto settings_ptr = std::make_unique(); + settings_ptr->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; + settings_ptr->max_single_read_retries = context_settings.azure_max_single_read_retries; + settings_ptr->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); + + return settings_ptr; +} + void registerStorageAzure(StorageFactory & factory) { factory.registerStorage("Azure", [](const StorageFactory::Arguments & args) @@ -276,11 +288,7 @@ void registerStorageAzure(StorageFactory & factory) if (args.storage_def->partition_by) partition_by = args.storage_def->partition_by->clone(); - const auto & context_settings = args.getContext()->getSettingsRef(); - auto settings = std::make_unique(); - settings->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; - settings->max_single_read_retries = context_settings.azure_max_single_read_retries; - settings->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); + auto settings = StorageAzure::createSettings(args.getContext()); return std::make_shared( std::move(configuration), @@ -399,7 +407,7 @@ StorageAzure::StorageAzure( StorageInMemoryMetadata storage_metadata; if (columns_.empty()) { - auto columns = getTableStructureFromDataImpl(context); + auto columns = getTableStructureFromData(object_storage.get(), configuration, format_settings, context); storage_metadata.setColumns(columns); } else @@ -1149,27 +1157,31 @@ std::unique_ptr StorageAzureSource::createAzureReadBuffer(const Stri return object_storage->readObject(StoredObject(key), read_settings, {}, object_size); } -ColumnsDescription StorageAzure::getTableStructureFromDataImpl(ContextPtr ctx) +ColumnsDescription StorageAzure::getTableStructureFromData( + AzureObjectStorage * object_storage, + const Configuration & configuration, + const std::optional & format_settings, + ContextPtr ctx) { RelativePathsWithMetadata read_keys; std::shared_ptr file_iterator; if (configuration.withGlobs()) { file_iterator = std::make_shared( - object_storage.get(), configuration.container, std::nullopt, - configuration.blob_path, nullptr, virtual_block, ctx, &read_keys); + object_storage, configuration.container, std::nullopt, + configuration.blob_path, nullptr, Block{}, ctx, &read_keys); } else { file_iterator = std::make_shared( - object_storage.get(), configuration.container, configuration.blobs_paths, - std::nullopt, nullptr, virtual_block, ctx, &read_keys); + object_storage, configuration.container, configuration.blobs_paths, + std::nullopt, nullptr, Block{}, ctx, &read_keys); } std::optional columns_from_cache; size_t prev_read_keys_size = read_keys.size(); if (ctx->getSettingsRef().schema_inference_use_cache_for_azure) - columns_from_cache = tryGetColumnsFromCache(read_keys.begin(), read_keys.end(), ctx); + columns_from_cache = tryGetColumnsFromCache(read_keys.begin(), read_keys.end(), configuration, format_settings, ctx); ReadBufferIterator read_buffer_iterator = [&, first = true](ColumnsDescription & cached_columns) mutable -> std::unique_ptr { @@ -1189,7 +1201,7 @@ ColumnsDescription StorageAzure::getTableStructureFromDataImpl(ContextPtr ctx) /// S3 file iterator could get new keys after new iteration, check them in schema cache. if (ctx->getSettingsRef().schema_inference_use_cache_for_azure && read_keys.size() > prev_read_keys_size) { - columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), ctx); + columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), configuration, format_settings, ctx); prev_read_keys_size = read_keys.size(); if (columns_from_cache) { @@ -1213,7 +1225,7 @@ ColumnsDescription StorageAzure::getTableStructureFromDataImpl(ContextPtr ctx) columns = readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx); if (ctx->getSettingsRef().schema_inference_use_cache_for_azure) - addColumnsToCache(read_keys, columns, configuration.format, ctx); + addColumnsToCache(read_keys, columns, configuration, format_settings, configuration.format, ctx); return columns; @@ -1222,6 +1234,8 @@ ColumnsDescription StorageAzure::getTableStructureFromDataImpl(ContextPtr ctx) std::optional StorageAzure::tryGetColumnsFromCache( const RelativePathsWithMetadata::const_iterator & begin, const RelativePathsWithMetadata::const_iterator & end, + const StorageAzure::Configuration & configuration, + const std::optional & format_settings, const ContextPtr & ctx) { auto & schema_cache = getSchemaCache(ctx); @@ -1247,6 +1261,8 @@ std::optional StorageAzure::tryGetColumnsFromCache( void StorageAzure::addColumnsToCache( const RelativePathsWithMetadata & keys, const ColumnsDescription & columns, + const StorageAzure::Configuration & configuration, + const std::optional & format_settings, const String & format_name, const ContextPtr & ctx) { diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index f20da74c2a8..8341026b624 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -31,6 +32,7 @@ using AzureCredentials = std::variant; @@ -78,7 +80,10 @@ public: static StorageAzure::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file = true); static AzureClientPtr createClient(StorageAzure::Configuration configuration); - static AzureObjectStorage::SettingsPtr createSettings(StorageAzure::Configuration configuration); + + static AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); + + static void processNamedCollectionResult(StorageAzure::Configuration & configuration, const NamedCollection & collection); String getName() const override { @@ -112,6 +117,12 @@ public: static SchemaCache & getSchemaCache(const ContextPtr & ctx); + static ColumnsDescription getTableStructureFromData( + AzureObjectStorage * object_storage, + const Configuration & configuration, + const std::optional & format_settings, + ContextPtr ctx); + private: std::string name; Configuration configuration; @@ -123,16 +134,19 @@ private: std::optional format_settings; ASTPtr partition_by; - ColumnsDescription getTableStructureFromDataImpl(ContextPtr ctx); - std::optional tryGetColumnsFromCache( + static std::optional tryGetColumnsFromCache( const RelativePathsWithMetadata::const_iterator & begin, const RelativePathsWithMetadata::const_iterator & end, + const StorageAzure::Configuration & configuration, + const std::optional & format_settings, const ContextPtr & ctx); - void addColumnsToCache( + static void addColumnsToCache( const RelativePathsWithMetadata & keys, const ColumnsDescription & columns, + const Configuration & configuration, + const std::optional & format_settings, const String & format_name, const ContextPtr & ctx); diff --git a/src/TableFunctions/TableFunctionAzure.cpp b/src/TableFunctions/TableFunctionAzure.cpp index f565a365a13..ac3fa1cc8af 100644 --- a/src/TableFunctions/TableFunctionAzure.cpp +++ b/src/TableFunctions/TableFunctionAzure.cpp @@ -36,20 +36,148 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - -void TableFunctionAzure::parseArgumentsImpl(ASTs & args, const ContextPtr & context) +namespace { - if (args.size() != 5) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); - for (auto & arg : args) - arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); +bool isConnectionString(const std::string & candidate) +{ + return candidate.starts_with("DefaultEndpointsProtocol"); +} - configuration.connection_url = checkAndGetLiteralArgument(args[0], "connection_url"); - configuration.container = checkAndGetLiteralArgument(args[1], "container"); - configuration.blob_path = checkAndGetLiteralArgument(args[2], "blob_path"); - configuration.format = checkAndGetLiteralArgument(args[3], "format"); - configuration.structure = checkAndGetLiteralArgument(args[4], "structure"); +} + +StorageAzure::Configuration TableFunctionAzure::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context, bool get_format_from_file) +{ + StorageAzure::Configuration configuration; + + /// Supported signatures: + /// + /// Azure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) + /// + + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + { + StorageAzure::processNamedCollectionResult(configuration, *named_collection); + + configuration.blobs_paths = {configuration.blob_path}; + + if (configuration.format == "auto" && get_format_from_file) + configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); + + return configuration; + } + + if (engine_args.size() < 3 || engine_args.size() > 8) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage Azure requires 3 to 7 arguments: " + "Azure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); + + std::unordered_map engine_args_to_idx; + + configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); + configuration.is_connection_string = isConnectionString(configuration.connection_url); + + configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); + configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); + + auto is_format_arg = [] (const std::string & s) -> bool + { + return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); + }; + + if (engine_args.size() == 4) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name/structure"); + if (is_format_arg(fourth_arg)) + { + configuration.format = fourth_arg; + } + else + { + configuration.structure = fourth_arg; + } + } + else if (engine_args.size() == 5) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + configuration.format = fourth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + } + } + else if (engine_args.size() == 6) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + configuration.format = fourth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); + configuration.structure = checkAndGetLiteralArgument(engine_args[5], "structure"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; + } + } + else if (engine_args.size() == 7) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format, compression and structure must be last arguments"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); + } + } + else if (engine_args.size() == 8) + { + + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); + configuration.structure = checkAndGetLiteralArgument(engine_args[7], "structure"); + } + } + + configuration.blobs_paths = {configuration.blob_path}; + + if (configuration.format == "auto" && get_format_from_file) + configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); + + return configuration; } void TableFunctionAzure::parseArguments(const ASTPtr & ast_function, ContextPtr context) @@ -69,6 +197,16 @@ void TableFunctionAzure::parseArguments(const ASTPtr & ast_function, ContextPtr ColumnsDescription TableFunctionAzure::getActualTableStructure(ContextPtr context) const { + if (configuration.structure == "auto") + { + context->checkAccess(getSourceAccessType()); + auto client = StorageAzure::createClient(configuration); + auto settings = StorageAzure::createSettings(context); + + auto object_storage = std::make_unique("AzureTableFunction", std::move(client), std::move(settings)); + return StorageAzure::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context); + } + return parseColumnsListFromString(configuration.structure, context); } @@ -79,24 +217,29 @@ bool TableFunctionAzure::supportsReadingSubsetOfColumns() StoragePtr TableFunctionAzure::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - ColumnsDescription columns; - columns = parseColumnsListFromString(configuration.structure, context); - configuration.is_connection_string = true; configuration.blobs_paths = {configuration.blob_path}; auto client = StorageAzure::createClient(configuration); + auto settings = StorageAzure::createSettings(context); + + ColumnsDescription columns; + if (configuration.structure != "auto") + columns = parseColumnsListFromString(configuration.structure, context); + else if (!structure_hint.empty()) + columns = structure_hint; StoragePtr storage = std::make_shared( configuration, - std::make_unique(table_name, std::move(client), std::make_unique()), + std::make_unique(table_name, std::move(client), std::move(settings)), context, StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, /// No format_settings for table function Azure - std::nullopt, nullptr); + std::nullopt, + nullptr); storage->startup(); diff --git a/src/TableFunctions/TableFunctionAzure.h b/src/TableFunctions/TableFunctionAzure.h index a6fb5415113..b3508e7f95c 100644 --- a/src/TableFunctions/TableFunctionAzure.h +++ b/src/TableFunctions/TableFunctionAzure.h @@ -19,9 +19,9 @@ class TableFunctionAzure : public ITableFunction { public: static constexpr auto name = "azure_blob"; - static constexpr auto signature = "- connection_url, container, blob, format, structure\n"; + static constexpr auto signature = "- connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]\n"; - static size_t getMaxNumberOfArguments() { return 5; } + static size_t getMaxNumberOfArguments() { return 8; } String getName() const override { @@ -46,9 +46,7 @@ public: return {"_path", "_file"}; } - virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); - - static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context); + static StorageAzure::Configuration parseArgumentsImpl(ASTs & args, const ContextPtr & context, bool get_format_from_file = true); protected: From e76a7022fcc26f02cb98214e72f039b12b04423f Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 18:50:45 +0200 Subject: [PATCH 1277/2223] Add some tests --- .../test_storage_azure_blob_storage/test.py | 43 +++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 9b71ff1a490..d4a5f6e24bb 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -443,3 +443,46 @@ def test_schema_inference_from_globs(cluster): bucket='cont', max_path=max_path ) ] + +def test_simple_write_account_string_table_function(cluster): + node = cluster.instances["node"] + azure_query(node, "INSERT INTO azure_blob('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')") + print(get_azure_file_content("test_simple_write_tf.csv")) + assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' + + +def test_simple_write_connection_string_table_function(cluster): + node = cluster.instances["node"] + azure_query( + node, + "CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = Azure('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_c.csv', 'CSV')", + ) + azure_query(node, "INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") + print(get_azure_file_content("test_simple_write_c.csv")) + assert get_azure_file_content("test_simple_write_c.csv") == '1,"a"\n' + + +def test_simple_write_named_collection_1_table_function(cluster): + node = cluster.instances["node"] + azure_query( + node, + "CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = Azure(azure_conf1)", + ) + azure_query( + node, "INSERT INTO test_simple_write_named_collection_1 VALUES (1, 'a')" + ) + print(get_azure_file_content("test_simple_write_named.csv")) + assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' + + +def test_simple_write_named_collection_2_table_function(cluster): + node = cluster.instances["node"] + azure_query( + node, + "CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')", + ) + azure_query( + node, "INSERT INTO test_simple_write_named_collection_2 VALUES (1, 'a')" + ) + print(get_azure_file_content("test_simple_write_named_2.csv")) + assert get_azure_file_content("test_simple_write_named_2.csv") == '1,"a"\n' From 2a756a7e6c7d564aae6e4895b53e4b2d6f18cfbd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 17:05:23 +0000 Subject: [PATCH 1278/2223] Cosmetics: Make default tree count / distance function constants --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index b15b1bb1a91..e6c11c839fe 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -314,12 +314,14 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) { - uint64_t trees = 100; - String distance_function = "L2Distance"; + static constexpr auto default_trees = 100uz; + static constexpr auto default_distance_function = "L2Distance"; + String distance_function = default_distance_function; if (!index.arguments.empty()) distance_function = index.arguments[0].get(); + uint64_t trees = default_trees; if (index.arguments.size() > 1) trees = index.arguments[1].get(); From a74d3ca3289806eccf6cd66ac29725bc3655bbe8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 17:06:47 +0000 Subject: [PATCH 1279/2223] Fix style check --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index e6c11c839fe..1a28f28f746 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -23,7 +23,6 @@ namespace ErrorCodes extern const int INCORRECT_NUMBER_OF_COLUMNS; extern const int INCORRECT_QUERY; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; } From ae97f45c1c9bc8e3f55c01fc57bab5179964b36b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 6 Jun 2023 17:10:48 +0000 Subject: [PATCH 1280/2223] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 23d1b44daf2..b431837bccb 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -457,9 +457,13 @@ def test_schema_inference_from_globs(cluster): ) ] + def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] - azure_query(node, "INSERT INTO azure_blob('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')") + azure_query( + node, + "INSERT INTO azure_blob('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + ) print(get_azure_file_content("test_simple_write_tf.csv")) assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' From 6a96cf441e24a5994aa38839ea69dad60b2ecd83 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 6 Jun 2023 19:19:17 +0200 Subject: [PATCH 1281/2223] Renamed to azure_blob_storage --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index b431837bccb..b81730fb24a 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -462,7 +462,7 @@ def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] azure_query( node, - "INSERT INTO azure_blob('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + "INSERT INTO azure_blob_storage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", ) print(get_azure_file_content("test_simple_write_tf.csv")) assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' From b2db6b47896c03aa728c4fe66d9271eb1a34c529 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 6 Jun 2023 19:19:56 +0200 Subject: [PATCH 1282/2223] Renamed to azure_blob_storage --- src/TableFunctions/TableFunctionAzure.cpp | 2 +- src/TableFunctions/TableFunctionAzure.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/TableFunctions/TableFunctionAzure.cpp b/src/TableFunctions/TableFunctionAzure.cpp index ac3fa1cc8af..27137bda9ff 100644 --- a/src/TableFunctions/TableFunctionAzure.cpp +++ b/src/TableFunctions/TableFunctionAzure.cpp @@ -251,7 +251,7 @@ void registerTableFunctionAzure(TableFunctionFactory & factory) factory.registerFunction( {.documentation = {.description=R"(The table function can be used to read the data stored on Azure Blob Storage.)", - .examples{{"azure_blob", "SELECT * FROM azure_blob(connection, container, blob_path, format, structure)", ""}}}, + .examples{{"azure_blob_storage", "SELECT * FROM azure_blob_storage(connection, container, blob_path, format, structure)", ""}}}, .allow_readonly = false}); } diff --git a/src/TableFunctions/TableFunctionAzure.h b/src/TableFunctions/TableFunctionAzure.h index b3508e7f95c..e2815973010 100644 --- a/src/TableFunctions/TableFunctionAzure.h +++ b/src/TableFunctions/TableFunctionAzure.h @@ -18,7 +18,7 @@ class Context; class TableFunctionAzure : public ITableFunction { public: - static constexpr auto name = "azure_blob"; + static constexpr auto name = "azure_blob_storage"; static constexpr auto signature = "- connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]\n"; static size_t getMaxNumberOfArguments() { return 8; } From 7100bc51526c5d88cf3aa84b932ede4e9f1ad15f Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 19:31:50 +0200 Subject: [PATCH 1283/2223] Fixes for azure table function --- src/Storages/StorageAzure.cpp | 3 +- src/TableFunctions/TableFunctionAzure.cpp | 7 +- .../configs/named_collections.xml | 1 + .../test_storage_azure_blob_storage/test.py | 107 ++++++++++++++---- 4 files changed, 89 insertions(+), 29 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index e3051236118..c6001f59b6f 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -75,6 +75,7 @@ const std::unordered_set required_configuration_keys = { const std::unordered_set optional_configuration_keys = { "format", "compression", + "structure", "compression_method", "account_name", "account_key", @@ -1193,7 +1194,7 @@ ColumnsDescription StorageAzure::getTableStructureFromData( throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file, because there are no files with provided path " - "in S3. You must specify table structure manually", configuration.format); + "in AzureBlobStorage. You must specify table structure manually", configuration.format); return nullptr; } diff --git a/src/TableFunctions/TableFunctionAzure.cpp b/src/TableFunctions/TableFunctionAzure.cpp index ac3fa1cc8af..8b18ed42d71 100644 --- a/src/TableFunctions/TableFunctionAzure.cpp +++ b/src/TableFunctions/TableFunctionAzure.cpp @@ -78,6 +78,7 @@ StorageAzure::Configuration TableFunctionAzure::parseArgumentsImpl(ASTs & engine std::unordered_map engine_args_to_idx; configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "CONFIGURATION {}", configuration.connection_url); configuration.is_connection_string = isConnectionString(configuration.connection_url); configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); @@ -192,7 +193,8 @@ void TableFunctionAzure::parseArguments(const ASTPtr & ast_function, ContextPtr auto & args = args_func.at(0)->children; - parseArgumentsImpl(args, context); + configuration = parseArgumentsImpl(args, context); + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "CONFIGURATION {}", configuration.connection_url); } ColumnsDescription TableFunctionAzure::getActualTableStructure(ContextPtr context) const @@ -217,9 +219,6 @@ bool TableFunctionAzure::supportsReadingSubsetOfColumns() StoragePtr TableFunctionAzure::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - configuration.is_connection_string = true; - configuration.blobs_paths = {configuration.blob_path}; - auto client = StorageAzure::createClient(configuration); auto settings = StorageAzure::createSettings(context); diff --git a/tests/integration/test_storage_azure_blob_storage/configs/named_collections.xml b/tests/integration/test_storage_azure_blob_storage/configs/named_collections.xml index dc70895bc05..e0c18d11940 100644 --- a/tests/integration/test_storage_azure_blob_storage/configs/named_collections.xml +++ b/tests/integration/test_storage_azure_blob_storage/configs/named_collections.xml @@ -4,6 +4,7 @@ DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1; cont test_simple_write_named.csv + key UInt64, data String CSV diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 23d1b44daf2..ad96d2d304a 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -459,43 +459,102 @@ def test_schema_inference_from_globs(cluster): def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] - azure_query(node, "INSERT INTO azure_blob('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')") + azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')") print(get_azure_file_content("test_simple_write_tf.csv")) assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' def test_simple_write_connection_string_table_function(cluster): node = cluster.instances["node"] - azure_query( - node, - "CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = Azure('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_c.csv', 'CSV')", - ) - azure_query(node, "INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") - print(get_azure_file_content("test_simple_write_c.csv")) - assert get_azure_file_content("test_simple_write_c.csv") == '1,"a"\n' + azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')") + print(get_azure_file_content("test_simple_write_connection_tf.csv")) + assert get_azure_file_content("test_simple_write_connection_tf.csv") == '1,"a"\n' def test_simple_write_named_collection_1_table_function(cluster): node = cluster.instances["node"] - azure_query( - node, - "CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = Azure(azure_conf1)", - ) - azure_query( - node, "INSERT INTO test_simple_write_named_collection_1 VALUES (1, 'a')" - ) + azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob(azure_conf1) VALUES (1, 'a')") print(get_azure_file_content("test_simple_write_named.csv")) assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' def test_simple_write_named_collection_2_table_function(cluster): node = cluster.instances["node"] - azure_query( - node, - "CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')", - ) - azure_query( - node, "INSERT INTO test_simple_write_named_collection_2 VALUES (1, 'a')" - ) - print(get_azure_file_content("test_simple_write_named_2.csv")) - assert get_azure_file_content("test_simple_write_named_2.csv") == '1,"a"\n' + + azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob(azure_conf2, container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')") + print(get_azure_file_content("test_simple_write_named_2_tf.csv")) + assert get_azure_file_content("test_simple_write_named_2_tf.csv") == '1,"a"\n' + +def test_put_get_with_globs_tf(cluster): + # type: (ClickHouseCluster) -> None + unique_prefix = random.randint(1, 10000) + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + max_path = "" + for i in range(10): + for j in range(10): + path = "{}/{}_{}/{}.csv".format( + unique_prefix, i, random.choice(["a", "b", "c", "d"]), j + ) + max_path = max(path, max_path) + values = f"({i},{j},{i + j})" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azure_blob(azure_conf2, container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + ) + query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azure_blob(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" + assert azure_query(node, query).splitlines() == [ + "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( + bucket="cont", max_path=max_path + ) + ] + +def test_schema_inference_no_globs_tf(cluster): + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 String, column3 UInt32" + + query = f"insert into table function azure_blob(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') SELECT number, toString(number), number * number FROM numbers(1000)" + azure_query(node, query) + + query = "select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from azure_blob(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv')" + assert azure_query(node, query).splitlines() == [ + "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" + ] + +def test_schema_inference_from_globs_tf(cluster): + node = cluster.instances["node"] + unique_prefix = random.randint(1, 10000) + node = cluster.instances["node"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + max_path = "" + for i in range(10): + for j in range(10): + path = "{}/{}_{}/{}.csv".format( + unique_prefix, i, random.choice(["a", "b", "c", "d"]), j + ) + max_path = max(path, max_path) + values = f"({i},{j},{i + j})" + + query = f"insert into table function azure_blob(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" + azure_query(node, query) + + query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azure_blob(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" + assert azure_query(node, query).splitlines() == [ + "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( + bucket="cont", max_path=max_path + ) + ] + +def test_partition_by_tf(cluster): + node = cluster.instances["node"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + partition_by = "column3" + values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" + filename = "test_tf_{_partition_id}.csv" + + azure_query(node, f"INSERT INTO TABLE FUNCTION azure_blob('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}") + + assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv") + assert "3,2,1\n" == get_azure_file_content("test_tf_1.csv") + assert "78,43,45\n" == get_azure_file_content("test_tf_45.csv") From df50833b709ea862ea49f24763c74cf898b10907 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Jun 2023 17:33:05 +0000 Subject: [PATCH 1284/2223] Allow to skip trailing empty lines in CSV/TSV/CustomeSeparated formats --- docs/en/interfaces/formats.md | 6 +++++- .../en/operations/settings/settings-formats.md | 18 ++++++++++++++++++ src/Core/Settings.h | 3 +++ src/Formats/FormatFactory.cpp | 3 +++ src/Formats/FormatSettings.h | 3 +++ .../Formats/Impl/CSVRowInputFormat.cpp | 14 ++++++++++++++ .../Formats/Impl/CSVRowInputFormat.h | 1 + .../Impl/CustomSeparatedRowInputFormat.cpp | 4 ++++ .../Impl/TabSeparatedRowInputFormat.cpp | 14 ++++++++++++++ .../Formats/Impl/TabSeparatedRowInputFormat.h | 2 ++ ..._custom_skip_trailing_empty_lines.reference | 3 +++ ...sv_csv_custom_skip_trailing_empty_lines.sql | 12 ++++++++++++ 12 files changed, 82 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02771_tsv_csv_custom_skip_trailing_empty_lines.reference create mode 100644 tests/queries/0_stateless/02771_tsv_csv_custom_skip_trailing_empty_lines.sql diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 2ab9e8caec4..70479b8ac71 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -193,6 +193,7 @@ SELECT * FROM nestedt FORMAT TSV - [output_format_tsv_crlf_end_of_line](/docs/en/operations/settings/settings-formats.md/#output_format_tsv_crlf_end_of_line) - if it is set true, end of line in TSV output format will be `\r\n` instead of `\n`. Default value - `false`. - [input_format_tsv_skip_first_lines](/docs/en/operations/settings/settings-formats.md/#input_format_tsv_skip_first_lines) - skip specified number of lines at the beginning of data. Default value - `0`. - [input_format_tsv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_tsv_detect_header) - automatically detect header with names and types in TSV format. Default value - `true`. +- [input_format_tsv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_tsv_skip_trailing_empty_lines) - skip trailing empty lines at the end of data. Default value - `false`. ## TabSeparatedRaw {#tabseparatedraw} @@ -467,6 +468,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [output_format_csv_crlf_end_of_line](/docs/en/operations/settings/settings-formats.md/#output_format_csv_crlf_end_of_line) - if it is set to true, end of line in CSV output format will be `\r\n` instead of `\n`. Default value - `false`. - [input_format_csv_skip_first_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_first_lines) - skip the specified number of lines at the beginning of data. Default value - `0`. - [input_format_csv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_csv_detect_header) - automatically detect header with names and types in CSV format. Default value - `true`. +- [input_format_csv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_trailing_empty_lines) - skip trailing empty lines at the end of data. Default value - `false`. ## CSVWithNames {#csvwithnames} @@ -494,7 +496,9 @@ the types from input data will be compared with the types of the corresponding c Similar to [Template](#format-template), but it prints or reads all names and types of columns and uses escaping rule from [format_custom_escaping_rule](/docs/en/operations/settings/settings-formats.md/#format_custom_escaping_rule) setting and delimiters from [format_custom_field_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_field_delimiter), [format_custom_row_before_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_row_before_delimiter), [format_custom_row_after_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_row_after_delimiter), [format_custom_row_between_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_row_between_delimiter), [format_custom_result_before_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_result_before_delimiter) and [format_custom_result_after_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_result_after_delimiter) settings, not from format strings. -If setting [input_format_custom_detect_header](/docs/en/operations/settings/settings.md/#input_format_custom_detect_header) is enabled, ClickHouse will automatically detect header with names and types if any. +If setting [input_format_custom_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_custom_detect_header) is enabled, ClickHouse will automatically detect header with names and types if any. + +If setting [input_format_tsv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_custom_detect_header) is enabled, trailing empty lines at the end of file will be skipped. There is also `CustomSeparatedIgnoreSpaces` format, which is similar to [TemplateIgnoreSpaces](#templateignorespaces). diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 65038d3a256..a1a75446c37 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -728,6 +728,12 @@ My NULL My NULL ``` +### input_format_tsv_skip_trailing_empty_lines {input_format_tsv_skip_trailing_empty_lines} + +When enabled, trailing empty lines at the end of TSV file will be skipped. + +Disabled by default. + ## CSV format settings {#csv-format-settings} ### format_csv_delimiter {#format_csv_delimiter} @@ -882,6 +888,12 @@ My NULL My NULL ``` +### input_format_csv_skip_trailing_empty_lines {input_format_csv_skip_trailing_empty_lines} + +When enabled, trailing empty lines at the end of CSV file will be skipped. + +Disabled by default. + ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} @@ -1443,6 +1455,12 @@ Sets the character that is interpreted as a suffix after the result set for [Cus Default value: `''`. +### input_format_custom_skip_trailing_empty_lines {input_format_custom_skip_trailing_empty_lines} + +When enabled, trailing empty lines at the end of file in CustomSeparated format will be skipped. + +Disabled by default. + ## Regexp format settings {#regexp-format-settings} ### format_regexp_escaping_rule {#format_regexp_escaping_rule} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 67c92a0be8b..f688811028e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -873,6 +873,9 @@ class IColumn; M(Bool, output_format_protobuf_nullables_with_google_wrappers, false, "When serializing Nullable columns with Google wrappers, serialize default values as empty wrappers. If turned off, default and null values are not serialized", 0) \ M(UInt64, input_format_csv_skip_first_lines, 0, "Skip specified number of lines at the beginning of data in CSV format", 0) \ M(UInt64, input_format_tsv_skip_first_lines, 0, "Skip specified number of lines at the beginning of data in TSV format", 0) \ + M(Bool, input_format_csv_skip_trailing_empty_lines, false, "Skip trailing empty lines in CSV format", 0) \ + M(Bool, input_format_tsv_skip_trailing_empty_lines, false, "Skip trailing empty lines in TSV format", 0) \ + M(Bool, input_format_custom_skip_trailing_empty_lines, false, "Skip trailing empty lines in CustomSeparated format", 0) \ \ M(Bool, input_format_native_allow_types_conversion, true, "Allow data types conversion in Native input format", 0) \ \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 586e1bb7251..021ccd35602 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -69,6 +69,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.use_best_effort_in_schema_inference = settings.input_format_csv_use_best_effort_in_schema_inference; format_settings.csv.skip_first_lines = settings.input_format_csv_skip_first_lines; format_settings.csv.try_detect_header = settings.input_format_csv_detect_header; + format_settings.csv.skip_trailing_empty_lines = settings.input_format_csv_skip_trailing_empty_lines; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; @@ -80,6 +81,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; format_settings.custom.try_detect_header = settings.input_format_custom_detect_header; + format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; format_settings.input_format_ipv4_default_on_conversion_error = settings.input_format_ipv4_default_on_conversion_error; @@ -149,6 +151,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.tsv.use_best_effort_in_schema_inference = settings.input_format_tsv_use_best_effort_in_schema_inference; format_settings.tsv.skip_first_lines = settings.input_format_tsv_skip_first_lines; format_settings.tsv.try_detect_header = settings.input_format_tsv_detect_header; + format_settings.tsv.skip_trailing_empty_lines = settings.input_format_tsv_skip_trailing_empty_lines; format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals; format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index e332bd749a1..9d6cd384b68 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -136,6 +136,7 @@ struct FormatSettings UInt64 skip_first_lines = 0; String custom_delimiter; bool try_detect_header = true; + bool skip_trailing_empty_lines = false; } csv; struct HiveText @@ -156,6 +157,7 @@ struct FormatSettings std::string field_delimiter; EscapingRule escaping_rule = EscapingRule::Escaped; bool try_detect_header = true; + bool skip_trailing_empty_lines = false; } custom; struct @@ -291,6 +293,7 @@ struct FormatSettings bool use_best_effort_in_schema_inference = true; UInt64 skip_first_lines = 0; bool try_detect_header = true; + bool skip_trailing_empty_lines = false; } tsv; struct diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index de955d81651..f01f20a0a3c 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -322,6 +322,20 @@ void CSVFormatReader::setReadBuffer(ReadBuffer & in_) FormatWithNamesAndTypesReader::setReadBuffer(*buf); } +bool CSVFormatReader::checkForSuffix() +{ + if (!format_settings.csv.skip_trailing_empty_lines) + return buf->eof(); + + PeekableReadBufferCheckpoint checkpoint(*buf); + while (checkChar('\n', *buf) || checkChar('\r', *buf)); + if (buf->eof()) + return true; + + buf->rollbackToCheckpoint(); + return false; +} + CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesSchemaReader( buf, diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index f51f674e4af..0c8099a216c 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -75,6 +75,7 @@ public: std::vector readRow() { return readRowImpl(); } std::vector readRowForHeaderDetection() override { return readHeaderRow(); } + bool checkForSuffix() override; template std::vector readRowImpl(); diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 1c2efe3a41d..1e67db79a2c 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -283,6 +283,8 @@ bool CustomSeparatedFormatReader::checkForSuffixImpl(bool check_eof) /// Allow optional \n before eof. checkChar('\n', *buf); + if (format_settings.custom.skip_trailing_empty_lines) + while (checkChar('\n', *buf) || checkChar('\r', *buf)); return buf->eof(); } @@ -294,6 +296,8 @@ bool CustomSeparatedFormatReader::checkForSuffixImpl(bool check_eof) /// Allow optional \n before eof. checkChar('\n', *buf); + if (format_settings.custom.skip_trailing_empty_lines) + while (checkChar('\n', *buf) || checkChar('\r', *buf)); if (buf->eof()) return true; } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index af5f1f90732..2239c8539e3 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -286,6 +286,20 @@ void TabSeparatedFormatReader::setReadBuffer(ReadBuffer & in_) FormatWithNamesAndTypesReader::setReadBuffer(*buf); } +bool TabSeparatedFormatReader::checkForSuffix() +{ + if (!format_settings.tsv.skip_trailing_empty_lines) + return buf->eof(); + + PeekableReadBufferCheckpoint checkpoint(*buf); + while (checkChar('\n', *buf) || checkChar('\r', *buf)); + if (buf->eof()) + return true; + + buf->rollbackToCheckpoint(); + return false; +} + TabSeparatedSchemaReader::TabSeparatedSchemaReader( ReadBuffer & in_, bool with_names_, bool with_types_, bool is_raw_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesSchemaReader( diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index 0f4bff8d7d0..8df57675cf5 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -75,6 +75,8 @@ public: void setReadBuffer(ReadBuffer & in_) override; + bool checkForSuffix() override; + private: template std::vector readRowImpl(); diff --git a/tests/queries/0_stateless/02771_tsv_csv_custom_skip_trailing_empty_lines.reference b/tests/queries/0_stateless/02771_tsv_csv_custom_skip_trailing_empty_lines.reference new file mode 100644 index 00000000000..37e32ce62ee --- /dev/null +++ b/tests/queries/0_stateless/02771_tsv_csv_custom_skip_trailing_empty_lines.reference @@ -0,0 +1,3 @@ +1 2 +1 2 +1 2 diff --git a/tests/queries/0_stateless/02771_tsv_csv_custom_skip_trailing_empty_lines.sql b/tests/queries/0_stateless/02771_tsv_csv_custom_skip_trailing_empty_lines.sql new file mode 100644 index 00000000000..917a434cd58 --- /dev/null +++ b/tests/queries/0_stateless/02771_tsv_csv_custom_skip_trailing_empty_lines.sql @@ -0,0 +1,12 @@ +select * from format(TSV, 'x UInt32, y UInt32', '1\t2\n\n') settings input_format_tsv_skip_trailing_empty_lines=0; -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} +select * from format(TSV, 'x UInt32, y UInt32', '1\t2\n\n') settings input_format_tsv_skip_trailing_empty_lines=1; +select * from format(TSV, 'x UInt32, y UInt32', '1\t2\n\n1\t2\n') settings input_format_tsv_skip_trailing_empty_lines=1; -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} + +select * from format(CSV, 'x UInt32, y UInt32', '1,2\n\n') settings input_format_csv_skip_trailing_empty_lines=0; -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} +select * from format(CSV, 'x UInt32, y UInt32', '1,2\n\n') settings input_format_csv_skip_trailing_empty_lines=1; +select * from format(CSV, 'x UInt32, y UInt32', '1,2\n\n1,2\n') settings input_format_csv_skip_trailing_empty_lines=1; -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} + +select * from format(CustomSeparated, 'x UInt32, y UInt32', '1\t2\n\n\n') settings input_format_custom_skip_trailing_empty_lines=0; -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} +select * from format(CustomSeparated, 'x UInt32, y UInt32', '1\t2\n\n\n') settings input_format_custom_skip_trailing_empty_lines=1; +select * from format(CustomSeparated, 'x UInt32, y UInt32', '1\t2\n\n\n1\t2\n\n\n') settings input_format_custom_skip_trailing_empty_lines=1; -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} + From b6c8ce30ec5d8eeff846b273ed8aa2e6b00241f0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 6 Jun 2023 19:38:11 +0200 Subject: [PATCH 1285/2223] Disable 01676_clickhouse_client_autocomplete under UBSan --- .../queries/0_stateless/01676_clickhouse_client_autocomplete.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh index 42ae5e84f44..db62dedb5b4 100755 --- a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-ubsan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From d902592703757c161c8502dd07793c7da62f9b17 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 19:38:15 +0200 Subject: [PATCH 1286/2223] Fix new tests --- .../test_storage_azure_blob_storage/test.py | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 17817ca4e8e..e4e459428d0 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -467,14 +467,14 @@ def test_simple_write_account_string_table_function(cluster): def test_simple_write_connection_string_table_function(cluster): node = cluster.instances["node"] - azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')") + azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob_storage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')") print(get_azure_file_content("test_simple_write_connection_tf.csv")) assert get_azure_file_content("test_simple_write_connection_tf.csv") == '1,"a"\n' def test_simple_write_named_collection_1_table_function(cluster): node = cluster.instances["node"] - azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob(azure_conf1) VALUES (1, 'a')") + azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob_storage(azure_conf1) VALUES (1, 'a')") print(get_azure_file_content("test_simple_write_named.csv")) assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' @@ -482,7 +482,7 @@ def test_simple_write_named_collection_1_table_function(cluster): def test_simple_write_named_collection_2_table_function(cluster): node = cluster.instances["node"] - azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob(azure_conf2, container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')") + azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob_storage(azure_conf2, container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')") print(get_azure_file_content("test_simple_write_named_2_tf.csv")) assert get_azure_file_content("test_simple_write_named_2_tf.csv") == '1,"a"\n' @@ -502,9 +502,9 @@ def test_put_get_with_globs_tf(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azure_blob(azure_conf2, container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + f"INSERT INTO TABLE FUNCTION azure_blob_storage(azure_conf2, container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", ) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azure_blob(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" + query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azure_blob_storage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" assert azure_query(node, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path @@ -515,10 +515,10 @@ def test_schema_inference_no_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 String, column3 UInt32" - query = f"insert into table function azure_blob(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') SELECT number, toString(number), number * number FROM numbers(1000)" + query = f"insert into table function azure_blob_storage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') SELECT number, toString(number), number * number FROM numbers(1000)" azure_query(node, query) - query = "select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from azure_blob(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv')" + query = "select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from azure_blob_storage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv')" assert azure_query(node, query).splitlines() == [ "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] @@ -537,10 +537,10 @@ def test_schema_inference_from_globs_tf(cluster): max_path = max(path, max_path) values = f"({i},{j},{i + j})" - query = f"insert into table function azure_blob(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" + query = f"insert into table function azure_blob_storage(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" azure_query(node, query) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azure_blob(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" + query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azure_blob_storage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" assert azure_query(node, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path @@ -554,7 +554,7 @@ def test_partition_by_tf(cluster): values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" filename = "test_tf_{_partition_id}.csv" - azure_query(node, f"INSERT INTO TABLE FUNCTION azure_blob('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}") + azure_query(node, f"INSERT INTO TABLE FUNCTION azure_blob_storage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}") assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv") assert "3,2,1\n" == get_azure_file_content("test_tf_1.csv") From 934df5e5bb3714cf031104deff914cde9987eb31 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 19:44:41 +0200 Subject: [PATCH 1287/2223] Rename to AzureBlobStorage --- src/Storages/StorageAzure.cpp | 2 +- .../test_storage_azure_blob_storage/test.py | 46 +++++++++---------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index c6001f59b6f..46446940f75 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -251,7 +251,7 @@ AzureObjectStorage::SettingsPtr StorageAzure::createSettings(ContextPtr local_co void registerStorageAzure(StorageFactory & factory) { - factory.registerStorage("Azure", [](const StorageFactory::Arguments & args) + factory.registerStorage("AzureBlobStorage", [](const StorageFactory::Arguments & args) { auto & engine_args = args.engine_args; if (engine_args.empty()) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index e4e459428d0..9dea5d24686 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -81,7 +81,7 @@ def test_create_table_connection_string(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = Azure('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', 'cont', 'test_create_connection_string', 'CSV')", + "CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', 'cont', 'test_create_connection_string', 'CSV')", ) @@ -89,7 +89,7 @@ def test_create_table_account_string(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", + "CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", ) @@ -97,7 +97,7 @@ def test_simple_write_account_string(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_simple_write (key UInt64, data String) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", + "CREATE TABLE test_simple_write (key UInt64, data String) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", ) azure_query(node, "INSERT INTO test_simple_write VALUES (1, 'a')") print(get_azure_file_content("test_simple_write.csv")) @@ -108,7 +108,7 @@ def test_simple_write_connection_string(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = Azure('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_c.csv', 'CSV')", + "CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_c.csv', 'CSV')", ) azure_query(node, "INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") print(get_azure_file_content("test_simple_write_c.csv")) @@ -119,7 +119,7 @@ def test_simple_write_named_collection_1(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = Azure(azure_conf1)", + "CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = AzureBlobStorage(azure_conf1)", ) azure_query( node, "INSERT INTO test_simple_write_named_collection_1 VALUES (1, 'a')" @@ -132,7 +132,7 @@ def test_simple_write_named_collection_2(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')", + "CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')", ) azure_query( node, "INSERT INTO test_simple_write_named_collection_2 VALUES (1, 'a')" @@ -150,7 +150,7 @@ def test_partition_by(cluster): azure_query( node, - f"CREATE TABLE test_partitioned_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + f"CREATE TABLE test_partitioned_write ({table_format}) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", ) azure_query(node, f"INSERT INTO test_partitioned_write VALUES {values}") @@ -167,7 +167,7 @@ def test_partition_by_string_column(cluster): filename = "test_{_partition_id}.csv" azure_query( node, - f"CREATE TABLE test_partitioned_string_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + f"CREATE TABLE test_partitioned_string_write ({table_format}) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", ) azure_query(node, f"INSERT INTO test_partitioned_string_write VALUES {values}") @@ -185,7 +185,7 @@ def test_partition_by_const_column(cluster): filename = "test_{_partition_id}.csv" azure_query( node, - f"CREATE TABLE test_partitioned_const_write ({table_format}) Engine = Azure('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + f"CREATE TABLE test_partitioned_const_write ({table_format}) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", ) azure_query(node, f"INSERT INTO test_partitioned_const_write VALUES {values}") assert values_csv == get_azure_file_content("test_88.csv") @@ -195,7 +195,7 @@ def test_truncate(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_truncate (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_truncate.csv', format='CSV')", + "CREATE TABLE test_truncate (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_truncate.csv', format='CSV')", ) azure_query(node, "INSERT INTO test_truncate VALUES (1, 'a')") assert get_azure_file_content("test_truncate.csv") == '1,"a"\n' @@ -208,7 +208,7 @@ def test_simple_read_write(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_simple_read_write (key UInt64, data String) Engine = Azure(azure_conf2, container='cont', blob_path='test_simple_read_write.csv', format='CSV')", + "CREATE TABLE test_simple_read_write (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_simple_read_write.csv', format='CSV')", ) azure_query(node, "INSERT INTO test_simple_read_write VALUES (1, 'a')") @@ -222,7 +222,7 @@ def test_create_new_files_on_insert(cluster): azure_query( node, - f"create table test_multiple_inserts(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_parquet', format='Parquet')", + f"create table test_multiple_inserts(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_parquet', format='Parquet')", ) azure_query(node, "truncate table test_multiple_inserts") azure_query( @@ -249,7 +249,7 @@ def test_overwrite(cluster): azure_query( node, - f"create table test_overwrite(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_parquet_overwrite', format='Parquet')", + f"create table test_overwrite(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_parquet_overwrite', format='Parquet')", ) azure_query(node, "truncate table test_overwrite") @@ -273,7 +273,7 @@ def test_insert_with_path_with_globs(cluster): node = cluster.instances["node"] azure_query( node, - f"create table test_insert_globs(a Int32, b String) ENGINE = Azure(azure_conf2, container='cont', blob_path='test_insert_with_globs*', format='Parquet')", + f"create table test_insert_globs(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_insert_with_globs*', format='Parquet')", ) node.query_and_get_error( f"insert into table function test_insert_globs SELECT number, randomString(100) FROM numbers(500)" @@ -296,7 +296,7 @@ def test_put_get_with_globs(cluster): azure_query( node, - f"CREATE TABLE test_{i}_{j} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + f"CREATE TABLE test_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", ) query = f"insert into test_{i}_{j} VALUES {values}" @@ -304,7 +304,7 @@ def test_put_get_with_globs(cluster): azure_query( node, - f"CREATE TABLE test_glob_select ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV')", + f"CREATE TABLE test_glob_select ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV')", ) query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from test_glob_select" assert azure_query(node, query).splitlines() == [ @@ -328,7 +328,7 @@ def test_azure_glob_scheherazade(cluster): unique_num = random.randint(1, 10000) azure_query( node, - f"CREATE TABLE test_{i}_{unique_num} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + f"CREATE TABLE test_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", ) query = f"insert into test_{i}_{unique_num} VALUES {values}" azure_query(node, query) @@ -345,7 +345,7 @@ def test_azure_glob_scheherazade(cluster): azure_query( node, - f"CREATE TABLE test_glob_select_scheherazade ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='night_*/tale.csv', format='CSV')", + f"CREATE TABLE test_glob_select_scheherazade ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='night_*/tale.csv', format='CSV')", ) query = "select count(), sum(column1), sum(column2), sum(column3) from test_glob_select_scheherazade" assert azure_query(node, query).splitlines() == ["1001\t1001\t1001\t1001"] @@ -387,7 +387,7 @@ def test_storage_azure_get_gzip(cluster, extension, method): azure_query( node, - f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = Azure( + f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = AzureBlobStorage( azure_conf2, container='cont', blob_path ='{filename}', format='CSV', compression='{method}')""", @@ -402,7 +402,7 @@ def test_schema_inference_no_globs(cluster): table_format = "column1 UInt32, column2 String, column3 UInt32" azure_query( node, - f"CREATE TABLE test_schema_inference_src ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv', format='CSVWithNames')", + f"CREATE TABLE test_schema_inference_src ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv', format='CSVWithNames')", ) query = f"insert into test_schema_inference_src SELECT number, toString(number), number * number FROM numbers(1000)" @@ -410,7 +410,7 @@ def test_schema_inference_no_globs(cluster): azure_query( node, - f"CREATE TABLE test_select_inference Engine = Azure(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv')", + f"CREATE TABLE test_select_inference Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv')", ) print(node.query("SHOW CREATE TABLE test_select_inference")) @@ -437,7 +437,7 @@ def test_schema_inference_from_globs(cluster): azure_query( node, - f"CREATE TABLE test_schema_{i}_{j} ({table_format}) Engine = Azure(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames')", + f"CREATE TABLE test_schema_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames')", ) query = f"insert into test_schema_{i}_{j} VALUES {values}" @@ -445,7 +445,7 @@ def test_schema_inference_from_globs(cluster): azure_query( node, - f"CREATE TABLE test_glob_select_inference Engine = Azure(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')", + f"CREATE TABLE test_glob_select_inference Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')", ) print(node.query("SHOW CREATE TABLE test_glob_select_inference")) From e9c267ed696f30d440b35ed3bb215d550eb8aec7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Jun 2023 17:51:53 +0000 Subject: [PATCH 1288/2223] Fix converting Null to LowCardinality(Nullable) in values table function --- src/Interpreters/convertFieldToType.cpp | 2 +- .../0_stateless/02782_values_null_to_lc_nullable.reference | 1 + tests/queries/0_stateless/02782_values_null_to_lc_nullable.sql | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02782_values_null_to_lc_nullable.reference create mode 100644 tests/queries/0_stateless/02782_values_null_to_lc_nullable.sql diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index dc61e748db6..3e8fab80aaf 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -534,7 +534,7 @@ Field convertFieldToType(const Field & from_value, const IDataType & to_type, co Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_type, const IDataType * from_type_hint) { bool is_null = from_value.isNull(); - if (is_null && !to_type.isNullable()) + if (is_null && !to_type.isNullable() && !to_type.isLowCardinalityNullable()) throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert NULL to {}", to_type.getName()); Field converted = convertFieldToType(from_value, to_type, from_type_hint); diff --git a/tests/queries/0_stateless/02782_values_null_to_lc_nullable.reference b/tests/queries/0_stateless/02782_values_null_to_lc_nullable.reference new file mode 100644 index 00000000000..dec7d2fabd2 --- /dev/null +++ b/tests/queries/0_stateless/02782_values_null_to_lc_nullable.reference @@ -0,0 +1 @@ +\N diff --git a/tests/queries/0_stateless/02782_values_null_to_lc_nullable.sql b/tests/queries/0_stateless/02782_values_null_to_lc_nullable.sql new file mode 100644 index 00000000000..250fe6b7551 --- /dev/null +++ b/tests/queries/0_stateless/02782_values_null_to_lc_nullable.sql @@ -0,0 +1,2 @@ +select * from values('s LowCardinality(Nullable(String))', (NULL)); + From 95b054b42523f3e8271a095bbb570c1d02181ccd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 6 Jun 2023 18:01:22 +0000 Subject: [PATCH 1289/2223] Automatic style fix --- .../test_storage_azure_blob_storage/test.py | 29 +++++++++++++++---- 1 file changed, 24 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 9dea5d24686..8a0a68f5200 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -460,21 +460,30 @@ def test_schema_inference_from_globs(cluster): def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] - azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob_storage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')") + azure_query( + node, + "INSERT INTO TABLE FUNCTION azure_blob_storage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + ) print(get_azure_file_content("test_simple_write_tf.csv")) assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' def test_simple_write_connection_string_table_function(cluster): node = cluster.instances["node"] - azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob_storage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')") + azure_query( + node, + "INSERT INTO TABLE FUNCTION azure_blob_storage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + ) print(get_azure_file_content("test_simple_write_connection_tf.csv")) assert get_azure_file_content("test_simple_write_connection_tf.csv") == '1,"a"\n' def test_simple_write_named_collection_1_table_function(cluster): node = cluster.instances["node"] - azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob_storage(azure_conf1) VALUES (1, 'a')") + azure_query( + node, + "INSERT INTO TABLE FUNCTION azure_blob_storage(azure_conf1) VALUES (1, 'a')", + ) print(get_azure_file_content("test_simple_write_named.csv")) assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' @@ -482,10 +491,14 @@ def test_simple_write_named_collection_1_table_function(cluster): def test_simple_write_named_collection_2_table_function(cluster): node = cluster.instances["node"] - azure_query(node, "INSERT INTO TABLE FUNCTION azure_blob_storage(azure_conf2, container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')") + azure_query( + node, + "INSERT INTO TABLE FUNCTION azure_blob_storage(azure_conf2, container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", + ) print(get_azure_file_content("test_simple_write_named_2_tf.csv")) assert get_azure_file_content("test_simple_write_named_2_tf.csv") == '1,"a"\n' + def test_put_get_with_globs_tf(cluster): # type: (ClickHouseCluster) -> None unique_prefix = random.randint(1, 10000) @@ -511,6 +524,7 @@ def test_put_get_with_globs_tf(cluster): ) ] + def test_schema_inference_no_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 String, column3 UInt32" @@ -523,6 +537,7 @@ def test_schema_inference_no_globs_tf(cluster): "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] + def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] unique_prefix = random.randint(1, 10000) @@ -547,6 +562,7 @@ def test_schema_inference_from_globs_tf(cluster): ) ] + def test_partition_by_tf(cluster): node = cluster.instances["node"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -554,7 +570,10 @@ def test_partition_by_tf(cluster): values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" filename = "test_tf_{_partition_id}.csv" - azure_query(node, f"INSERT INTO TABLE FUNCTION azure_blob_storage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}") + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azure_blob_storage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + ) assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv") assert "3,2,1\n" == get_azure_file_content("test_tf_1.csv") From 49b019b26dfb674d31fa752335711c135377df4c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 6 Jun 2023 20:23:20 +0200 Subject: [PATCH 1290/2223] Refactored TableFunction name to TableFunctionAzureBlobStorage --- src/TableFunctions/ITableFunctionCluster.h | 10 ++++----- ....cpp => TableFunctionAzureBlobStorage.cpp} | 22 +++++++++---------- ...zure.h => TableFunctionAzureBlobStorage.h} | 2 +- src/TableFunctions/registerTableFunctions.cpp | 2 +- src/TableFunctions/registerTableFunctions.h | 2 +- 5 files changed, 19 insertions(+), 19 deletions(-) rename src/TableFunctions/{TableFunctionAzure.cpp => TableFunctionAzureBlobStorage.cpp} (88%) rename src/TableFunctions/{TableFunctionAzure.h => TableFunctionAzureBlobStorage.h} (97%) diff --git a/src/TableFunctions/ITableFunctionCluster.h b/src/TableFunctions/ITableFunctionCluster.h index f68558596ca..a8329684ee6 100644 --- a/src/TableFunctions/ITableFunctionCluster.h +++ b/src/TableFunctions/ITableFunctionCluster.h @@ -2,13 +2,13 @@ #include "config.h" -#include -#include -#include -#include -#include #include +#include +#include #include +#include +#include +#include namespace DB diff --git a/src/TableFunctions/TableFunctionAzure.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp similarity index 88% rename from src/TableFunctions/TableFunctionAzure.cpp rename to src/TableFunctions/TableFunctionAzureBlobStorage.cpp index e2d88a85eec..07bdb0f8393 100644 --- a/src/TableFunctions/TableFunctionAzure.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -46,13 +46,13 @@ bool isConnectionString(const std::string & candidate) } -StorageAzure::Configuration TableFunctionAzure::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context, bool get_format_from_file) +StorageAzure::Configuration TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context, bool get_format_from_file) { StorageAzure::Configuration configuration; /// Supported signatures: /// - /// Azure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) + /// AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) /// if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) @@ -70,7 +70,7 @@ StorageAzure::Configuration TableFunctionAzure::parseArgumentsImpl(ASTs & engine if (engine_args.size() < 3 || engine_args.size() > 8) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage Azure requires 3 to 7 arguments: " - "Azure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); + "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); @@ -181,7 +181,7 @@ StorageAzure::Configuration TableFunctionAzure::parseArgumentsImpl(ASTs & engine return configuration; } -void TableFunctionAzure::parseArguments(const ASTPtr & ast_function, ContextPtr context) +void TableFunctionAzureBlobStorage::parseArguments(const ASTPtr & ast_function, ContextPtr context) { /// Clone ast function, because we can modify its arguments like removing headers. auto ast_copy = ast_function->clone(); @@ -197,7 +197,7 @@ void TableFunctionAzure::parseArguments(const ASTPtr & ast_function, ContextPtr LOG_DEBUG(&Poco::Logger::get("DEBUG"), "CONFIGURATION {}", configuration.connection_url); } -ColumnsDescription TableFunctionAzure::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(ContextPtr context) const { if (configuration.structure == "auto") { @@ -205,19 +205,19 @@ ColumnsDescription TableFunctionAzure::getActualTableStructure(ContextPtr contex auto client = StorageAzure::createClient(configuration); auto settings = StorageAzure::createSettings(context); - auto object_storage = std::make_unique("AzureTableFunction", std::move(client), std::move(settings)); + auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings)); return StorageAzure::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context); } return parseColumnsListFromString(configuration.structure, context); } -bool TableFunctionAzure::supportsReadingSubsetOfColumns() +bool TableFunctionAzureBlobStorage::supportsReadingSubsetOfColumns() { return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); } -StoragePtr TableFunctionAzure::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { auto client = StorageAzure::createClient(configuration); auto settings = StorageAzure::createSettings(context); @@ -245,9 +245,9 @@ StoragePtr TableFunctionAzure::executeImpl(const ASTPtr & /*ast_function*/, Cont return storage; } -void registerTableFunctionAzure(TableFunctionFactory & factory) +void registerTableFunctionAzureBlobStorage(TableFunctionFactory & factory) { - factory.registerFunction( + factory.registerFunction( {.documentation = {.description=R"(The table function can be used to read the data stored on Azure Blob Storage.)", .examples{{"azure_blob_storage", "SELECT * FROM azure_blob_storage(connection, container, blob_path, format, structure)", ""}}}, diff --git a/src/TableFunctions/TableFunctionAzure.h b/src/TableFunctions/TableFunctionAzureBlobStorage.h similarity index 97% rename from src/TableFunctions/TableFunctionAzure.h rename to src/TableFunctions/TableFunctionAzureBlobStorage.h index e2815973010..6f3a3422958 100644 --- a/src/TableFunctions/TableFunctionAzure.h +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.h @@ -15,7 +15,7 @@ class Context; /* AzureBlob(source, [access_key_id, secret_access_key,] [format, structure, compression]) - creates a temporary storage for a file in AzureBlob. */ -class TableFunctionAzure : public ITableFunction +class TableFunctionAzureBlobStorage : public ITableFunction { public: static constexpr auto name = "azure_blob_storage"; diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index e0114368e44..0499524a912 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -73,7 +73,7 @@ void registerTableFunctions() registerTableFunctionExplain(factory); #if USE_AZURE_BLOB_STORAGE - registerTableFunctionAzure(factory); + registerTableFunctionAzureBlobStorage(factory); #endif diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index fa4fec2b03a..393bc080a3d 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -70,7 +70,7 @@ void registerTableFunctionFormat(TableFunctionFactory & factory); void registerTableFunctionExplain(TableFunctionFactory & factory); #if USE_AZURE_BLOB_STORAGE -void registerTableFunctionAzure(TableFunctionFactory & factory); +void registerTableFunctionAzureBlobStorage(TableFunctionFactory & factory); #endif void registerTableFunctions(); From ceab5117a923be20e8caf46d6330cdb76e0f9a6b Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 20:39:54 +0200 Subject: [PATCH 1291/2223] Fxi style --- src/Storages/StorageAzure.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index 46446940f75..f683a62e8e1 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -12,7 +12,6 @@ #include #include -#include #include #include #include @@ -29,7 +28,6 @@ #include #include #include -#include #include #include #include @@ -806,7 +804,7 @@ StorageAzureSource::Iterator::Iterator( , outer_blobs(outer_blobs_) { if (keys.has_value() && blob_path_with_globs.has_value()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify keys and glob simulatenously it's a bug"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify keys and glob simultaneously it's a bug"); if (!keys.has_value() && !blob_path_with_globs.has_value()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Both keys and glob mask are not specified"); From 5637858182e98070a9435e858c83ccaca316dd0a Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 21:06:45 +0200 Subject: [PATCH 1292/2223] Fix the most important check in the world --- src/Storages/StorageAzure.cpp | 3 +++ src/Storages/StorageAzure.h | 5 ----- src/Storages/registerStorages.cpp | 4 ---- src/TableFunctions/TableFunctionAzureBlobStorage.cpp | 2 +- 4 files changed, 4 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzure.cpp index f683a62e8e1..ac7ae8a4b36 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzure.cpp @@ -60,6 +60,9 @@ namespace ErrorCodes extern const int DATABASE_ACCESS_DENIED; extern const int CANNOT_COMPILE_REGEXP; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; + } namespace diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzure.h index 8341026b624..826582ccaa5 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzure.h @@ -20,11 +20,6 @@ struct AzureSimpleAccountConfiguration std::string storage_account_url; }; -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - using AzureConnectionString = std::string; using AzureCredentials = std::variant; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 03bd4dbb310..c66cf85a29b 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -94,14 +94,10 @@ void registerStorageFileLog(StorageFactory & factory); void registerStorageSQLite(StorageFactory & factory); #endif - - void registerStorageKeeperMap(StorageFactory & factory); #if USE_AZURE_BLOB_STORAGE - void registerStorageAzure(StorageFactory & factory); - #endif void registerStorages() diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 07bdb0f8393..96131bd884a 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -33,7 +33,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } namespace From 42c054789561920adf7ce4770968ba303a70f244 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 19:25:43 +0000 Subject: [PATCH 1293/2223] Remove clang-tidy exclude --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 1a28f28f746..ffed9e01df0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -123,7 +123,6 @@ MergeTreeIndexAggregatorAnnoy::MergeTreeIndexAggregatorAnnoy( template MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy::getGranuleAndReset() { - // NOLINTNEXTLINE(*) index->build(static_cast(trees), /*number_of_threads=*/1); auto granule = std::make_shared>(index_name, index_sample_block, index); index = nullptr; From 99f0be8ef507b197f12f285548d94bbf1f4dc3c2 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 6 Jun 2023 21:58:54 +0200 Subject: [PATCH 1294/2223] Refactored to StorageAzureBlob --- ...{StorageAzure.cpp => StorageAzureBlob.cpp} | 124 +++++++++--------- .../{StorageAzure.h => StorageAzureBlob.h} | 18 +-- .../TableFunctionAzureBlobStorage.cpp | 20 +-- .../TableFunctionAzureBlobStorage.h | 6 +- 4 files changed, 84 insertions(+), 84 deletions(-) rename src/Storages/{StorageAzure.cpp => StorageAzureBlob.cpp} (90%) rename src/Storages/{StorageAzure.h => StorageAzureBlob.h} (93%) diff --git a/src/Storages/StorageAzure.cpp b/src/Storages/StorageAzureBlob.cpp similarity index 90% rename from src/Storages/StorageAzure.cpp rename to src/Storages/StorageAzureBlob.cpp index ac7ae8a4b36..17374ba2d92 100644 --- a/src/Storages/StorageAzure.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_AZURE_BLOB_STORAGE @@ -91,7 +91,7 @@ bool isConnectionString(const std::string & candidate) } -void StorageAzure::processNamedCollectionResult(StorageAzure::Configuration & configuration, const NamedCollection & collection) +void StorageAzureBlob::processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection) { validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); @@ -122,15 +122,15 @@ void StorageAzure::processNamedCollectionResult(StorageAzure::Configuration & co } -StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file) +StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file) { - LOG_INFO(&Poco::Logger::get("StorageAzure"), "get_format_from_file = {}", get_format_from_file); + LOG_INFO(&Poco::Logger::get("StorageAzureBlob"), "get_format_from_file = {}", get_format_from_file); - StorageAzure::Configuration configuration; + StorageAzureBlob::Configuration configuration; /// Supported signatures: /// - /// Azure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) + /// AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) /// if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) @@ -147,8 +147,8 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C if (engine_args.size() < 3 || engine_args.size() > 7) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage Azure requires 3 to 7 arguments: " - "Azure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression])"); + "Storage AzureBlobStorage requires 3 to 7 arguments: " + "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression])"); for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); @@ -239,7 +239,7 @@ StorageAzure::Configuration StorageAzure::getConfiguration(ASTs & engine_args, C } -AzureObjectStorage::SettingsPtr StorageAzure::createSettings(ContextPtr local_context) +AzureObjectStorage::SettingsPtr StorageAzureBlob::createSettings(ContextPtr local_context) { const auto & context_settings = local_context->getSettingsRef(); auto settings_ptr = std::make_unique(); @@ -250,7 +250,7 @@ AzureObjectStorage::SettingsPtr StorageAzure::createSettings(ContextPtr local_co return settings_ptr; } -void registerStorageAzure(StorageFactory & factory) +void registerStorageAzureBlob(StorageFactory & factory) { factory.registerStorage("AzureBlobStorage", [](const StorageFactory::Arguments & args) { @@ -258,8 +258,8 @@ void registerStorageAzure(StorageFactory & factory) if (engine_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - auto configuration = StorageAzure::getConfiguration(engine_args, args.getLocalContext()); - auto client = StorageAzure::createClient(configuration); + auto configuration = StorageAzureBlob::getConfiguration(engine_args, args.getLocalContext()); + auto client = StorageAzureBlob::createClient(configuration); // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current // session and user are ignored. @@ -290,11 +290,11 @@ void registerStorageAzure(StorageFactory & factory) if (args.storage_def->partition_by) partition_by = args.storage_def->partition_by->clone(); - auto settings = StorageAzure::createSettings(args.getContext()); + auto settings = StorageAzureBlob::createSettings(args.getContext()); - return std::make_shared( + return std::make_shared( std::move(configuration), - std::make_unique("AzureStorage", std::move(client), std::move(settings)), + std::make_unique("AzureBlobStorage", std::move(client), std::move(settings)), args.getContext(), args.table_id, args.columns, @@ -311,7 +311,7 @@ void registerStorageAzure(StorageFactory & factory) }); } -AzureClientPtr StorageAzure::createClient(StorageAzure::Configuration configuration) +AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration configuration) { AzureClientPtr result; @@ -375,7 +375,7 @@ AzureClientPtr StorageAzure::createClient(StorageAzure::Configuration configurat return result; } -Poco::URI StorageAzure::Configuration::getConnectionURL() const +Poco::URI StorageAzureBlob::Configuration::getConnectionURL() const { if (!is_connection_string) return Poco::URI(connection_url); @@ -385,7 +385,7 @@ Poco::URI StorageAzure::Configuration::getConnectionURL() const } -StorageAzure::StorageAzure( +StorageAzureBlob::StorageAzureBlob( const Configuration & configuration_, std::unique_ptr && object_storage_, ContextPtr context, @@ -434,7 +434,7 @@ StorageAzure::StorageAzure( virtual_block.insert({column.type->createColumn(), column.type, column.name}); } -void StorageAzure::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) +void StorageAzureBlob::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { if (configuration.withGlobs()) { @@ -454,10 +454,10 @@ void StorageAzure::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextP namespace { -class StorageAzureSink : public SinkToStorage +class StorageAzureBlobSink : public SinkToStorage { public: - StorageAzureSink( + StorageAzureBlobSink( const String & format, const Block & sample_block_, ContextPtr context, @@ -474,7 +474,7 @@ public: writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings); } - String getName() const override { return "StorageS3Sink"; } + String getName() const override { return "StorageAzureBlobSink"; } void consume(Chunk chunk) override { @@ -532,10 +532,10 @@ private: std::mutex cancel_mutex; }; -class PartitionedStorageAzureSink : public PartitionedSink +class PartitionedStorageAzureBlobSink : public PartitionedSink { public: - PartitionedStorageAzureSink( + PartitionedStorageAzureBlobSink( const ASTPtr & partition_by, const String & format_, const Block & sample_block_, @@ -560,7 +560,7 @@ public: auto partition_key = replaceWildcards(blob, partition_id); validateKey(partition_key); - return std::make_shared( + return std::make_shared( format, sample_block, context, @@ -590,7 +590,7 @@ private: } -Pipe StorageAzure::read( +Pipe StorageAzureBlob::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, @@ -613,17 +613,17 @@ Pipe StorageAzure::read( requested_virtual_columns.push_back(virtual_column); } - std::shared_ptr iterator_wrapper; + std::shared_ptr iterator_wrapper; if (configuration.withGlobs()) { /// Iterate through disclosed globs and make a source for each file - iterator_wrapper = std::make_shared( + iterator_wrapper = std::make_shared( object_storage.get(), configuration.container, std::nullopt, configuration.blob_path, query_info.query, virtual_block, local_context, nullptr); } else { - iterator_wrapper = std::make_shared( + iterator_wrapper = std::make_shared( object_storage.get(), configuration.container, configuration.blobs_paths, std::nullopt, query_info.query, virtual_block, local_context, nullptr); } @@ -653,7 +653,7 @@ Pipe StorageAzure::read( for (size_t i = 0; i < num_streams; ++i) { - pipes.emplace_back(std::make_shared( + pipes.emplace_back(std::make_shared( requested_virtual_columns, configuration.format, getName(), @@ -671,7 +671,7 @@ Pipe StorageAzure::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageAzure::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { auto sample_block = metadata_snapshot->getSampleBlock(); auto chosen_compression_method = chooseCompressionMethod(configuration.blobs_paths.back(), configuration.compression_method); @@ -682,7 +682,7 @@ SinkToStoragePtr StorageAzure::write(const ASTPtr & query, const StorageMetadata if (is_partitioned_implementation) { - return std::make_shared( + return std::make_shared( partition_by_ast, configuration.format, sample_block, @@ -696,7 +696,7 @@ SinkToStoragePtr StorageAzure::write(const ASTPtr & query, const StorageMetadata { if (configuration.withGlobs()) throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "Azure key '{}' contains globs, so the table is in readonly mode", configuration.blob_path); + "AzureBlobStorage key '{}' contains globs, so the table is in readonly mode", configuration.blob_path); bool truncate_in_insert = local_context->getSettingsRef().azure_truncate_on_insert; @@ -730,7 +730,7 @@ SinkToStoragePtr StorageAzure::write(const ASTPtr & query, const StorageMetadata } } - return std::make_shared( + return std::make_shared( configuration.format, sample_block, local_context, @@ -741,32 +741,32 @@ SinkToStoragePtr StorageAzure::write(const ASTPtr & query, const StorageMetadata } } -NamesAndTypesList StorageAzure::getVirtuals() const +NamesAndTypesList StorageAzureBlob::getVirtuals() const { return virtual_columns; } -bool StorageAzure::supportsPartitionBy() const +bool StorageAzureBlob::supportsPartitionBy() const { return true; } -bool StorageAzure::supportsSubcolumns() const +bool StorageAzureBlob::supportsSubcolumns() const { return FormatFactory::instance().checkIfFormatSupportsSubcolumns(configuration.format); } -bool StorageAzure::supportsSubsetOfColumns() const +bool StorageAzureBlob::supportsSubsetOfColumns() const { return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); } -bool StorageAzure::prefersLargeBlocks() const +bool StorageAzureBlob::prefersLargeBlocks() const { return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration.format); } -bool StorageAzure::parallelizeOutputAfterReading(ContextPtr context) const +bool StorageAzureBlob::parallelizeOutputAfterReading(ContextPtr context) const { return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration.format, context); } @@ -788,7 +788,7 @@ static void addPathToVirtualColumns(Block & block, const String & path, size_t i block.getByName("_idx").column->assumeMutableRef().insert(idx); } -StorageAzureSource::Iterator::Iterator( +StorageAzureBlobSource::Iterator::Iterator( AzureObjectStorage * object_storage_, const std::string & container_, std::optional keys_, @@ -886,7 +886,7 @@ StorageAzureSource::Iterator::Iterator( } -RelativePathWithMetadata StorageAzureSource::Iterator::next() +RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { if (is_finished) return {}; @@ -971,13 +971,13 @@ RelativePathWithMetadata StorageAzureSource::Iterator::next() } } -size_t StorageAzureSource::Iterator::getTotalSize() const +size_t StorageAzureBlobSource::Iterator::getTotalSize() const { return total_size.load(std::memory_order_relaxed); } -void StorageAzureSource::Iterator::createFilterAST(const String & any_key) +void StorageAzureBlobSource::Iterator::createFilterAST(const String & any_key) { if (!query || !virtual_header) return; @@ -992,7 +992,7 @@ void StorageAzureSource::Iterator::createFilterAST(const String & any_key) } -Chunk StorageAzureSource::generate() +Chunk StorageAzureBlobSource::generate() { while (true) { @@ -1049,7 +1049,7 @@ Chunk StorageAzureSource::generate() return {}; } -Block StorageAzureSource::getHeader(Block sample_block, const std::vector & requested_virtual_columns) +Block StorageAzureBlobSource::getHeader(Block sample_block, const std::vector & requested_virtual_columns) { for (const auto & virtual_column : requested_virtual_columns) sample_block.insert({virtual_column.type->createColumn(), virtual_column.type, virtual_column.name}); @@ -1057,7 +1057,7 @@ Block StorageAzureSource::getHeader(Block sample_block, const std::vector & requested_virtual_columns_, const String & format_, String name_, @@ -1092,17 +1092,17 @@ StorageAzureSource::StorageAzureSource( } -StorageAzureSource::~StorageAzureSource() +StorageAzureBlobSource::~StorageAzureBlobSource() { create_reader_pool.wait(); } -String StorageAzureSource::getName() const +String StorageAzureBlobSource::getName() const { return name; } -StorageAzureSource::ReaderHolder StorageAzureSource::createReader() +StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() { auto [current_key, info] = file_iterator->next(); LOG_DEBUG(log, "KEY {} SIZE {}", current_key, info.size_bytes); @@ -1135,12 +1135,12 @@ StorageAzureSource::ReaderHolder StorageAzureSource::createReader() return ReaderHolder{fs::path(container) / current_key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; } -std::future StorageAzureSource::createReaderAsync() +std::future StorageAzureBlobSource::createReaderAsync() { return create_reader_scheduler([this] { return createReader(); }, Priority{}); } -std::unique_ptr StorageAzureSource::createAzureReadBuffer(const String & key, size_t object_size) +std::unique_ptr StorageAzureBlobSource::createAzureReadBuffer(const String & key, size_t object_size) { auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); read_settings.enable_filesystem_cache = false; @@ -1159,23 +1159,23 @@ std::unique_ptr StorageAzureSource::createAzureReadBuffer(const Stri return object_storage->readObject(StoredObject(key), read_settings, {}, object_size); } -ColumnsDescription StorageAzure::getTableStructureFromData( +ColumnsDescription StorageAzureBlob::getTableStructureFromData( AzureObjectStorage * object_storage, const Configuration & configuration, const std::optional & format_settings, ContextPtr ctx) { RelativePathsWithMetadata read_keys; - std::shared_ptr file_iterator; + std::shared_ptr file_iterator; if (configuration.withGlobs()) { - file_iterator = std::make_shared( + file_iterator = std::make_shared( object_storage, configuration.container, std::nullopt, configuration.blob_path, nullptr, Block{}, ctx, &read_keys); } else { - file_iterator = std::make_shared( + file_iterator = std::make_shared( object_storage, configuration.container, configuration.blobs_paths, std::nullopt, nullptr, Block{}, ctx, &read_keys); } @@ -1233,10 +1233,10 @@ ColumnsDescription StorageAzure::getTableStructureFromData( } -std::optional StorageAzure::tryGetColumnsFromCache( +std::optional StorageAzureBlob::tryGetColumnsFromCache( const RelativePathsWithMetadata::const_iterator & begin, const RelativePathsWithMetadata::const_iterator & end, - const StorageAzure::Configuration & configuration, + const StorageAzureBlob::Configuration & configuration, const std::optional & format_settings, const ContextPtr & ctx) { @@ -1260,10 +1260,10 @@ std::optional StorageAzure::tryGetColumnsFromCache( } -void StorageAzure::addColumnsToCache( +void StorageAzureBlob::addColumnsToCache( const RelativePathsWithMetadata & keys, const ColumnsDescription & columns, - const StorageAzure::Configuration & configuration, + const StorageAzureBlob::Configuration & configuration, const std::optional & format_settings, const String & format_name, const ContextPtr & ctx) @@ -1277,14 +1277,14 @@ void StorageAzure::addColumnsToCache( schema_cache.addMany(cache_keys, columns); } -SchemaCache & StorageAzure::getSchemaCache(const ContextPtr & ctx) +SchemaCache & StorageAzureBlob::getSchemaCache(const ContextPtr & ctx) { static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_azure", DEFAULT_SCHEMA_CACHE_ELEMENTS)); return schema_cache; } -std::unique_ptr StorageAzureSource::createAsyncAzureReadBuffer( +std::unique_ptr StorageAzureBlobSource::createAsyncAzureReadBuffer( const String & key, const ReadSettings & read_settings, size_t object_size) { auto modified_settings{read_settings}; diff --git a/src/Storages/StorageAzure.h b/src/Storages/StorageAzureBlob.h similarity index 93% rename from src/Storages/StorageAzure.h rename to src/Storages/StorageAzureBlob.h index 826582ccaa5..6e4dfaf19eb 100644 --- a/src/Storages/StorageAzure.h +++ b/src/Storages/StorageAzureBlob.h @@ -24,7 +24,7 @@ using AzureConnectionString = std::string; using AzureCredentials = std::variant; -class StorageAzure : public IStorage +class StorageAzureBlob : public IStorage { public: @@ -62,7 +62,7 @@ public: std::vector blobs_paths; }; - StorageAzure( + StorageAzureBlob( const Configuration & configuration_, std::unique_ptr && object_storage_, ContextPtr context_, @@ -73,12 +73,12 @@ public: std::optional format_settings_, ASTPtr partition_by_); - static StorageAzure::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file = true); - static AzureClientPtr createClient(StorageAzure::Configuration configuration); + static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file = true); + static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration); static AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); - static void processNamedCollectionResult(StorageAzure::Configuration & configuration, const NamedCollection & collection); + static void processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection); String getName() const override { @@ -133,7 +133,7 @@ private: static std::optional tryGetColumnsFromCache( const RelativePathsWithMetadata::const_iterator & begin, const RelativePathsWithMetadata::const_iterator & end, - const StorageAzure::Configuration & configuration, + const StorageAzureBlob::Configuration & configuration, const std::optional & format_settings, const ContextPtr & ctx); @@ -148,7 +148,7 @@ private: }; -class StorageAzureSource : public ISource, WithContext +class StorageAzureBlobSource : public ISource, WithContext { public: class Iterator : WithContext @@ -192,7 +192,7 @@ public: bool is_initialized = false; }; - StorageAzureSource( + StorageAzureBlobSource( const std::vector & requested_virtual_columns_, const String & format_, String name_, @@ -206,7 +206,7 @@ public: const String & container_, std::shared_ptr file_iterator_); - ~StorageAzureSource() override; + ~StorageAzureBlobSource() override; Chunk generate() override; diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 96131bd884a..986ad07fdde 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include @@ -46,9 +46,9 @@ bool isConnectionString(const std::string & candidate) } -StorageAzure::Configuration TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context, bool get_format_from_file) +StorageAzureBlob::Configuration TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context, bool get_format_from_file) { - StorageAzure::Configuration configuration; + StorageAzureBlob::Configuration configuration; /// Supported signatures: /// @@ -57,7 +57,7 @@ StorageAzure::Configuration TableFunctionAzureBlobStorage::parseArgumentsImpl(AS if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) { - StorageAzure::processNamedCollectionResult(configuration, *named_collection); + StorageAzureBlob::processNamedCollectionResult(configuration, *named_collection); configuration.blobs_paths = {configuration.blob_path}; @@ -202,11 +202,11 @@ ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(Contex if (configuration.structure == "auto") { context->checkAccess(getSourceAccessType()); - auto client = StorageAzure::createClient(configuration); - auto settings = StorageAzure::createSettings(context); + auto client = StorageAzureBlob::createClient(configuration); + auto settings = StorageAzureBlob::createSettings(context); auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings)); - return StorageAzure::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context); + return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context); } return parseColumnsListFromString(configuration.structure, context); @@ -219,8 +219,8 @@ bool TableFunctionAzureBlobStorage::supportsReadingSubsetOfColumns() StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - auto client = StorageAzure::createClient(configuration); - auto settings = StorageAzure::createSettings(context); + auto client = StorageAzureBlob::createClient(configuration); + auto settings = StorageAzureBlob::createSettings(context); ColumnsDescription columns; if (configuration.structure != "auto") @@ -228,7 +228,7 @@ StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_funct else if (!structure_hint.empty()) columns = structure_hint; - StoragePtr storage = std::make_shared( + StoragePtr storage = std::make_shared( configuration, std::make_unique(table_name, std::move(client), std::move(settings)), context, diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.h b/src/TableFunctions/TableFunctionAzureBlobStorage.h index 6f3a3422958..0bb872de3f3 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.h +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.h @@ -5,7 +5,7 @@ #if USE_AZURE_BLOB_STORAGE #include -#include +#include namespace DB @@ -46,7 +46,7 @@ public: return {"_path", "_file"}; } - static StorageAzure::Configuration parseArgumentsImpl(ASTs & args, const ContextPtr & context, bool get_format_from_file = true); + static StorageAzureBlob::Configuration parseArgumentsImpl(ASTs & args, const ContextPtr & context, bool get_format_from_file = true); protected: @@ -61,7 +61,7 @@ protected: ColumnsDescription getActualTableStructure(ContextPtr context) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - mutable StorageAzure::Configuration configuration; + mutable StorageAzureBlob::Configuration configuration; ColumnsDescription structure_hint; }; From 6ab2a50c393696cf0444fda405ffce6453ecbd09 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 6 Jun 2023 22:48:53 +0200 Subject: [PATCH 1295/2223] Fix two tests and build --- src/Storages/registerStorages.cpp | 4 ++-- tests/queries/0_stateless/01271_show_privileges.reference | 1 + .../0_stateless/02117_show_create_table_system.reference | 6 +++--- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index c66cf85a29b..5606e6728d4 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -97,7 +97,7 @@ void registerStorageSQLite(StorageFactory & factory); void registerStorageKeeperMap(StorageFactory & factory); #if USE_AZURE_BLOB_STORAGE -void registerStorageAzure(StorageFactory & factory); +void registerStorageAzureBlob(StorageFactory & factory); #endif void registerStorages() @@ -197,7 +197,7 @@ void registerStorages() registerStorageKeeperMap(factory); #if USE_AZURE_BLOB_STORAGE - registerStorageAzure(factory); + registerStorageAzureBlob(factory); #endif } diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index ec245d8b9e0..5d30da5d2ea 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -158,6 +158,7 @@ JDBC [] GLOBAL SOURCES HDFS [] GLOBAL SOURCES S3 [] GLOBAL SOURCES HIVE [] GLOBAL SOURCES +AZURE [] GLOBAL SOURCES SOURCES [] \N ALL CLUSTER [] GLOBAL ALL ALL ['ALL PRIVILEGES'] \N \N diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 09cc62dac00..e864ba85018 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -297,7 +297,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'AZURE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -581,10 +581,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'AZURE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'AZURE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From be8e0487994d70c3c6350cb675475b12dffda8e5 Mon Sep 17 00:00:00 2001 From: johanngan Date: Tue, 6 Jun 2023 16:28:44 -0500 Subject: [PATCH 1296/2223] Revert invalid RegExpTreeDictionary optimization This reverts the following commits: - e77dd810369ad5fcf957393e4fc71a8a6220b04e - e8527e720b2ab12b3327f1e3886aace402a292c6 Additionally, functional tests are added. When scanning complex regexp nodes sequentially with RE2, the old code has an optimization to break out of the loop early upon finding a leaf node that matches. This is an invalid optimization because there's no guarantee that it's actually a VALID match, because its parents might NOT have matched. Semantically, a user would expect this match to be discarded and for the search to continue. Instead, since we skipped matching after the first false positive, subsequent nodes that would have matched are missing from the output value. This affects both dictGet and dictGetAll. It's difficult to distinguish a true positive from a false positive while looping through complex_regexp_nodes because we would have to scan all the parents of a matching node to confirm a true positive. Trying to do this might actually end up being slower than just scanning every complex regexp node, because complex_regexp_nodes is only a subset of all the tree nodes; we may end up duplicating work with scanning that Vectorscan has already done, depending on whether the parent nodes are "simple" or "complex". So instead of trying to fix this optimization, just remove it entirely. --- src/Dictionaries/RegExpTreeDictionary.cpp | 14 ---- ...04_regexp_dictionary_yaml_source.reference | 16 +++++ .../02504_regexp_dictionary_yaml_source.sh | 64 +++++++++++++++++++ 3 files changed, 80 insertions(+), 14 deletions(-) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 8d0af9b0abf..3852cca6928 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -129,17 +129,6 @@ struct RegExpTreeDictionary::RegexTreeNode return searcher.Match(haystack, 0, size, re2_st::RE2::Anchor::UNANCHORED, nullptr, 0); } - /// check if this node can cover all the attributes from the query. - bool containsAll(const std::unordered_map & matching_attributes) const - { - for (const auto & [key, value] : matching_attributes) - { - if (!attributes.contains(key)) - return false; - } - return true; - } - struct AttributeValue { Field field; @@ -691,9 +680,6 @@ std::unordered_map RegExpTreeDictionary::match( if (node_ptr->match(reinterpret_cast(keys_data.data()) + offset, length)) { match_result.insertNodeID(node_ptr->id); - /// When this node is leaf and contains all the required attributes, it means a match. - if (node_ptr->containsAll(attributes) && node_ptr->children.empty()) - break; } } diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference index 437012dd516..79871e3716c 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference +++ b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference @@ -11,3 +11,19 @@ (['ClickHouse Documentation','ClickHouse'],[0,1],['/en'],['ClickHouse']) (['Documentation','GitHub'],[2,3],[NULL],[]) (['Documentation','GitHub'],[2,3],[NULL],[]) +ClickHouse +['ClickHouse'] +ClickHouse Documentation +['ClickHouse Documentation','ClickHouse','Documentation'] +GitHub Documentation +['GitHub Documentation','GitHub'] +Documentation +['Documentation'] +ClickHouse +['ClickHouse'] +ClickHouse Documentation +['ClickHouse Documentation','ClickHouse','Documentation'] +GitHub Documentation +['GitHub Documentation','GitHub'] +Documentation +['Documentation'] diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh index ac0793460a9..5e8985406ae 100755 --- a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh +++ b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh @@ -175,6 +175,70 @@ select dictGetAll('regexp_dict3', ('tag', 'topological_index', 'captured', 'pare select dictGetAll('regexp_dict3', ('tag', 'topological_index', 'captured', 'parent'), 'github.com/clickhouse/tree/master/docs', 2); " +# Test that things work the same for "simple" regexps that go through Hyperscan and "complex" regexps that go through RE2. +# An easy way to force the use of RE2 is to disable Hyperscan. +# This tree is constructed purposely so that text might (falsely) match leaf nodes without matching their corresponding parent nodes +cat > "$yaml" < #include #include +#include #include #include #include @@ -948,25 +949,52 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd std::list useful_indices; std::map, MergedDataSkippingIndexAndCondition> merged_indices; + std::unordered_set ignored_index_names; + + if (use_skip_indexes && settings.ignore_data_skipping_indices.changed) + { + const auto & indices = settings.ignore_data_skipping_indices.toString(); + Tokens tokens(indices.data(), &indices[indices.size()], settings.max_query_size); + IParser::Pos pos(tokens, static_cast(settings.max_parser_depth)); + Expected expected; + + /// Use an unordered list rather than string vector + auto parse_single_id_or_literal = [&] + { + String str; + if (!parseIdentifierOrStringLiteral(pos, expected, str)) + return false; + + ignored_index_names.insert(std::move(str)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_single_id_or_literal, false)) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse ignore_data_skipping_indices ('{}')", indices); + } if (use_skip_indexes) { for (const auto & index : metadata_snapshot->getSecondaryIndices()) { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) - { - auto [it, inserted] = merged_indices.try_emplace({index_helper->index.type, index_helper->getGranularity()}); - if (inserted) - it->second.condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); - it->second.addIndex(index_helper); - } - else + auto index_helper = MergeTreeIndexFactory::instance().get(index); + if(!ignored_index_names.contains(index.name)) { - auto condition = index_helper->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - useful_indices.emplace_back(index_helper, condition); + if (index_helper->isMergeable()) + { + auto [it, inserted] = merged_indices.try_emplace({index_helper->index.type, index_helper->getGranularity()}); + if (inserted) + it->second.condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + + it->second.addIndex(index_helper); + } + else + { + auto condition = index_helper->createIndexCondition(query_info, context); + if (!condition->alwaysUnknownOrTrue()) + useful_indices.emplace_back(index_helper, condition); + } } } } diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference new file mode 100644 index 00000000000..af1fce5ba13 --- /dev/null +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference @@ -0,0 +1,44 @@ +1 2 3 +1 2 3 +1 2 3 +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + Skip + Name: xy_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql new file mode 100644 index 00000000000..ab314ae5ea2 --- /dev/null +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS data_02771; +CREATE TABLE data_02771 +( + key Int, + x Int, + y Int, + INDEX x_idx x TYPE minmax GRANULARITY 1, + INDEX y_idx y TYPE minmax GRANULARITY 1, + INDEX xy_idx (x,y) TYPE minmax GRANULARITY 1 +) +Engine=MergeTree() +ORDER BY key; + +INSERT INTO data_02771 VALUES (1, 2, 3); + +SELECT * FROM data_02771; +SELECT * FROM data_02771 SETTINGS ignore_data_skipping_indices=''; -- { serverError 6 } +SELECT * FROM data_02771 SETTINGS ignore_data_skipping_indices='x_idx'; +SELECT * FROM data_02771 SETTINGS ignore_data_skipping_indices='na_idx'; + +SELECT * FROM data_02771 WHERE x = 1 AND y = 1 SETTINGS ignore_data_skipping_indices='xy_idx',force_data_skipping_indices='xy_idx' ; -- { serverError 277 } +SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; +EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2; +EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; + +DROP TABLE data_02771; From f552b96451bd4c826a9e7d1bff669301c3c4bccc Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Mon, 29 May 2023 14:00:00 -0700 Subject: [PATCH 1301/2223] Add docs for ignore index --- docs/en/operations/settings/settings.md | 83 +++++++++++++++++++++++++ 1 file changed, 83 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5730503a670..5b0c6b3c8c2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -227,6 +227,89 @@ SELECT * FROM data_01515 WHERE d1 = 0 SETTINGS force_data_skipping_indices='`d1_ SELECT * FROM data_01515 WHERE d1 = 0 AND assumeNotNull(d1_null) = 0 SETTINGS force_data_skipping_indices='`d1_idx`, d1_null_idx'; -- Ok. ``` +## ignore_data_skipping_indices {#settings-ignore_data_skipping_indices} + +Ignores the skipping indexes specified if used by the query. + +Consider the following example: + +```sql +CREATE TABLE data +( + key Int, + x Int, + y Int, + INDEX x_idx x TYPE minmax GRANULARITY 1, + INDEX y_idx y TYPE minmax GRANULARITY 1, + INDEX xy_idx (x,y) TYPE minmax GRANULARITY 1 +) +Engine=MergeTree() +ORDER BY key; + +INSERT INTO data VALUES (1, 2, 3); + +SELECT * FROM data; +SELECT * FROM data SETTINGS ignore_data_skipping_indices=''; -- query will produce CANNOT_PARSE_TEXT error. +SELECT * FROM data SETTINGS ignore_data_skipping_indices='x_idx'; -- Ok. +SELECT * FROM data SETTINGS ignore_data_skipping_indices='na_idx'; -- Ok. + +SELECT * FROM data WHERE x = 1 AND y = 1 SETTINGS ignore_data_skipping_indices='xy_idx',force_data_skipping_indices='xy_idx' ; -- query will produce INDEX_NOT_USED error, since xy_idx is explictly ignored. +SELECT * FROM data WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; +``` + +The query without ignoring any indexes: +```sql +EXPLAIN indexes = 1 SELECT * FROM data WHERE x = 1 AND y = 2; + +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + ReadFromMergeTree (default.data) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + Skip + Name: xy_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 +``` + +Ignoring the `xy_idx` index: +```sql +EXPLAIN indexes = 1 SELECT * FROM data WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; + +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + ReadFromMergeTree (default.data) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 +``` + Works with tables in the MergeTree family. ## convert_query_to_cnf {#convert_query_to_cnf} From 689e0cabe0ca3cfc4b710a7426dbb1d363437984 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Tue, 30 May 2023 05:31:54 -0700 Subject: [PATCH 1302/2223] Add space to if --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ca6ab931499..c07d887588b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -979,7 +979,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd { auto index_helper = MergeTreeIndexFactory::instance().get(index); - if(!ignored_index_names.contains(index.name)) + if (!ignored_index_names.contains(index.name)) { if (index_helper->isMergeable()) { From 7c2b88a00eb1972fbd27b534ad578c2e69486605 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Thu, 1 Jun 2023 07:14:39 -0700 Subject: [PATCH 1303/2223] Make test invariant --- .../02771_ignore_data_skipping_indices.reference | 4 ---- .../0_stateless/02771_ignore_data_skipping_indices.sql | 6 +++--- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference index af1fce5ba13..786360783fd 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference @@ -1,8 +1,6 @@ 1 2 3 1 2 3 1 2 3 -Expression ((Projection + Before ORDER BY)) - Filter (WHERE) ReadFromMergeTree (default.data_02771) Indexes: PrimaryKey @@ -24,8 +22,6 @@ Expression ((Projection + Before ORDER BY)) Description: minmax GRANULARITY 1 Parts: 0/0 Granules: 0/0 -Expression ((Projection + Before ORDER BY)) - Filter (WHERE) ReadFromMergeTree (default.data_02771) Indexes: PrimaryKey diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql index ab314ae5ea2..baa2d497863 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -20,7 +20,7 @@ SELECT * FROM data_02771 SETTINGS ignore_data_skipping_indices='na_idx'; SELECT * FROM data_02771 WHERE x = 1 AND y = 1 SETTINGS ignore_data_skipping_indices='xy_idx',force_data_skipping_indices='xy_idx' ; -- { serverError 277 } SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; -EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2; -EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; +SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; +SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx' ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; -DROP TABLE data_02771; +DROP TABLE data_02771; \ No newline at end of file From 1fa1215d1549e5887695cfd0f6bf4aaa61101fec Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Fri, 2 Jun 2023 07:21:22 -0700 Subject: [PATCH 1304/2223] Avoid UB --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../queries/0_stateless/02771_ignore_data_skipping_indices.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c07d887588b..4967de8424b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -954,7 +954,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (use_skip_indexes && settings.ignore_data_skipping_indices.changed) { const auto & indices = settings.ignore_data_skipping_indices.toString(); - Tokens tokens(indices.data(), &indices[indices.size()], settings.max_query_size); + Tokens tokens(indices.data(), indices.data() + indices.size(), settings.max_query_size); IParser::Pos pos(tokens, static_cast(settings.max_parser_depth)); Expected expected; diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql index baa2d497863..289d5240b57 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -23,4 +23,4 @@ SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_ind SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx' ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; -DROP TABLE data_02771; \ No newline at end of file +DROP TABLE data_02771; From 45d000b71780cb7a022c6c0694d978596ac8fb96 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Mon, 5 Jun 2023 05:27:44 -0700 Subject: [PATCH 1305/2223] Turn off analyzer for test --- .../0_stateless/02771_ignore_data_skipping_indices.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql index 289d5240b57..a49239e9de2 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -1,4 +1,8 @@ +SET allow_experimental_analyzer = 0; + DROP TABLE IF EXISTS data_02771; + + CREATE TABLE data_02771 ( key Int, From f3959aa9e16fd50ad5e7081c12a9a9948113e898 Mon Sep 17 00:00:00 2001 From: Derek Chia Date: Wed, 7 Jun 2023 11:07:16 +0800 Subject: [PATCH 1306/2223] Update settings.md `max_final_threads` is now set to the number of cores by default. See https://github.com/ClickHouse/ClickHouse/pull/47915 --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5730503a670..8e2cd8d6027 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3155,7 +3155,7 @@ Possible values: - Positive integer. - 0 or 1 — Disabled. `SELECT` queries are executed in a single thread. -Default value: `16`. +Default value: the number of physical CPU cores. ## opentelemetry_start_trace_probability {#opentelemetry-start-trace-probability} From 23a30268369c3166965d34815fd963db33740a64 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 7 Jun 2023 03:16:29 +0000 Subject: [PATCH 1307/2223] Implemented connection string --- docs/en/interfaces/cli.md | 110 +++++++++ docs/ru/interfaces/cli.md | 110 +++++++++ programs/client/Client.cpp | 13 +- src/Client/ConnectionString.cpp | 219 ++++++++++++++++++ src/Client/ConnectionString.h | 22 ++ .../02784_connection_string.reference | 125 ++++++++++ .../0_stateless/02784_connection_string.sh | 156 +++++++++++++ 7 files changed, 753 insertions(+), 2 deletions(-) create mode 100644 src/Client/ConnectionString.cpp create mode 100644 src/Client/ConnectionString.h create mode 100644 tests/queries/0_stateless/02784_connection_string.reference create mode 100755 tests/queries/0_stateless/02784_connection_string.sh diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index f670d464006..5255657ddfd 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -158,6 +158,116 @@ $ clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="number" --query "SELECT {col:Identifier} FROM {db:Identifier}.{tbl:Identifier} LIMIT 10" ``` +## Connection string {#connection_string} + +The connection string for clickhouse-client is presented in URI format: + +```text +clickhouse://[user_info@][hosts_and_ports][/dbname][?query_parameters] +``` + +where user_info is: ```user[:password]``` +and hosts_and_ports is a list of values: ```[host][:port],[host][:port]``` Port is not mandatory. +and query_parameters is a list of parameter[=value]: ```param_name[=value]¶m_name[=value]...``` value may not be required for some of parameters. + +Allowed query_parameters keys: + +- **secure** or shorthanded **s** - no value. If specified, client will connect to the server over a secure connection (TLS). See **secure** in [command-line-options](#command-line-options) + +These examples illustrate valid connection strings for clickhouse-client: + +```text +clickhouse: +clickhouse://localhost +clickhouse://localhost:9000 +clickhouse://localhost/default +clickhouse://default@localhost +clickhouse://user:password@localhost +clickhouse://user_name@localhost/some_database?secure +clickhouse://host1:9000,host2:5000/some_database +``` + +The host component can either be an IP address or a host name. Put an IPv6 address in square brackets to specify it: + +```text +clickhouse://[2001:db8::1234] +``` + +If user or/and password are not specified, default values will be used. +If host is not specified, the default host will be used (localhost). +If port is not specified, the default port will be used (9000). +If database is not specified, the default database will be used. + +User, password, and database can be specified in the connection string either in --user command line option. + +The connection string must be specified in the first argument of clickhouse-client. The connection string can be combined with other [command-line-options](#command-line-options) except **--host(h)** and **--port**. + +### Multiple hosts {#connection_string_multiple_hosts} + +URI allows multiple hosts to be connected to, and the client will try to connect to those hosts using the order from URI and command line options. The hosts and ports in the URI accept comma-separated lists of values. + +If more than one host is supplied, or if a single host name is translated to more than one address, each host and address will be attempted one at a time until one is successful. The remaining hosts after successful connection in the list are not tried. + +### Percent encoding {#connection_string_uri_percent_encoding} + +Hosts, user name, password, database and query parameters should be [Percent-Encoded](https://en.wikipedia.org/wiki/URL_encoding) if values contain URI invalid characters. + +### Examples {#connection_string_examples} + +Connect to localhost using port 9000 and executes the query "SELECT 1". + +``` bash +clickhouse-client "clickhouse://localhost:9000" --query "SELECT 1" +``` + +Connect to localhost using port 9000 in interactive, multiline mode. + +``` bash +clickhouse-client "clickhouse://localhost:9000" -m +``` + +Connect to localhost using port 9000 in interactive mode with the user specified in --user option. + +``` bash +clickhouse-client "clickhouse://localhost:9000" --user default +``` + +Connect to localhost using port 9000 in interactive mode with database 'my_database' specified in command line option + +``` bash +clickhouse-client "clickhouse://localhost:9000" --database my_database +``` + +Connect to localhost using port 9000 in interactive mode with the database specified in the connection string. + +``` bash +clickhouse-client "clickhouse://localhost:9000/my_database" +``` + +Connect to localhost using port 9000 in interactive mode with a database specified in the connection string and a secure connection using shorthanded 's' URI parameter. + +```bash +clickhouse-client "clickhouse://localhost/my_database?s" +``` + +Connect to default host using the default port, default user, and default database. + +``` bash +clickhouse-client "clickhouse:" +``` + +Connect to the default host using the default port, using user user_name and no password. + +``` bash +clickhouse-client "clickhouse://user_name@" +``` + +Connect to localhost using email user name. Symbol '@' is percent encoded to '%40'. + +``` bash +clickhouse-client "clickhouse://some_user%40some_mail.com@localhost:9000" +``` + ## Configuring {#interfaces_cli_configuration} You can pass parameters to `clickhouse-client` (all parameters have a default value) using: diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 4c22eae0207..06642800cc6 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -110,6 +110,116 @@ $ clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="number" --query "SELECT {col:Identifier} FROM {db:Identifier}.{tbl:Identifier} LIMIT 10" ``` +## Строка подключения {#connection_string} + +Строка подключения для clickhouse-client представлена в формате URI: + +```text +clickhouse://[user_info@][hosts_and_ports][/dbname][?query_parameters] +``` + +где user_info - это: ```user[:password]``` +hosts_and_ports - это список значений: ```[host][:port],[host][:port]```. Port может быть не задан. +query_parameters - это список пар ключ[=значение]: ```param_name[=value]¶m_name[=value]...```. Значение может быть пустым + +Допустимые ключи query_parameters: + +- **secure** или сокращенно **s** - без значение. Если параметр указан, то соединение с сервером будет осуществляться по защищенному каналу (TLS). См. **secure** в [command-line-options](#command-line-options). + +Эти примеры иллюстрируют допустимые строки подключения для clickhouse-client: + +```text +clickhouse: +clickhouse://localhost +clickhouse://localhost:9000 +clickhouse://localhost/default +clickhouse://default@localhost +clickhouse://user:password@localhost +clickhouse://имя_пользователя@localhost/some_database?secure +clickhouse://host1:9000,host2:5000/some_database +``` + +Параметр host может быть либо IP-адресом, либо именем хоста. Для указания IPv6-адреса поместите его в квадратные скобки: + +```text +clickhouse://[2001:db8::1234] +``` + +Если пользователь или/и пароль не указаны, будут использоваться значения по умолчанию. +Если host не указан, будет использован хост по умолчанию (localhost). +Если port не указан, будет использоваться порт по умолчанию (9000). +Если база данных не указана, будет использоваться база данных по умолчанию (default). + +Пользователь, пароль и база данных могут быть указаны в строке подключения либо в опциях командной строки --user, --password, --database. + +Строка подключения должна быть указана в первом аргументе clickhouse-client. Строка подключения может комбинироваться с другими [параметрами командной строки] (#command-line-options) кроме **--host(h)** и **--port**. + +### Несколько хостов {#connection_string_multiple_hosts} + +URI позволяет подключаться к нескольким хостам, и клиент будет пытаться подключиться к этим хостам, используя порядок из URI и опций командной строки. Хосты и порты в URI принимают списки значений, разделенные запятыми. + +Если указано более одного хоста или если одно имя хоста транслируется в несколько адресов, Клиент будет будет пытаться подключится к каждому хосту и адресу в порядке в котором они встречаются в URI И опциях клиента, пока не будет установлено соединение. Соединение разрывается, если соединение установлено и аутентификация прошла успешно, остальные хосты в списке игнорируются. + +### Кодирование URI {#connection_string_uri_percent_encoding} + +Хосты, имя пользователя, пароль, имя базы данных, и параметры запроса должны быть [закодированы](https://ru.wikipedia.org/wiki/URL#%D0%9A%D0%BE%D0%B4%D0%B8%D1%80%D0%BE%D0%B2%D0%B0%D0%BD%D0%B8%D0%B5_URL), если значения содержат невалидные символы URI. + +### Примеры {#connection_string_examples} + +Подключиться к localhost через порт 9000 и выполнить запрос "SELECT 1" + +``` bash +clickhouse-client "clickhouse://localhost:9000" --query "SELECT 1" +``` + +Подключиться к localhost через порт 9000 в интерактивном, многострочном режиме. + +``` bash +clickhouse-client "clickhouse://localhost:9000" -m +``` + +Подключиться к localhost через порт 9000 в интерактивном режиме с пользователем default, указанным в опции --user. + +``` bash +clickhouse-client "clickhouse://localhost:9000" --user default +``` + +Подключиться к localhost, используя порт 9000 в интерактивном режиме с базой данных 'my_database', указанной в опции командной строки. + +``` bash +clickhouse-client "clickhouse://localhost:9000" --database my_database +``` + +Подключиться к localhost через порт 9000 в интерактивном режиме с базой данных my_database, указанной в строке подключения. + +``` bash +clickhouse-client "clickhouse://localhost:9000/my_database" +``` + +Подключиться к localhost через порт 9000 в интерактивном режиме с базой данных, указанной в строке подключения, и безопасным соединением с использованием сокращенного параметра URI 's'. + +``` bash +clickhouse-client "clickhouse://localhost/my_database?s" +``` + +Подключиться к хосту по умолчанию с использованием порта по умолчанию, пользователя по умолчанию, и базы данных по умолчанию. + +``` bash +clickhouse-client "clickhouse:" +``` + +Подключиться к хосту по умолчанию через порт по умолчанию, используя имя пользователя user_name без пароля. + +``` bash +clickhouse-client "clickhouse://user_name@" +``` + +Подключиться к localhost, используя электронную почту, как имя пользователя. Символ '@' закодирован как '%40'. + +``` bash +clickhouse-client "clickhouse://some_user%40some_mail.com@localhost:9000" +``` + ## Конфигурирование {#interfaces_cli_configuration} В `clickhouse-client` можно передавать различные параметры (все параметры имеют значения по умолчанию) с помощью: diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 32a07284d26..e513314387f 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -5,13 +5,13 @@ #include #include #include -#include #include #include #include #include #include #include "Client.h" +#include "Client/ConnectionString.h" #include "Core/Protocol.h" #include "Parsers/formatAST.h" @@ -1248,6 +1248,9 @@ void Client::readArguments( std::vector & external_tables_arguments, std::vector & hosts_and_ports_arguments) { + bool has_connection_string = argc >= 2 && tryParseConnectionString(std::string_view(argv[1]), common_arguments, hosts_and_ports_arguments); + int start_argument_index = has_connection_string ? 2 : 1; + /** We allow different groups of arguments: * - common arguments; * - arguments for any number of external tables each in form "--external args...", @@ -1260,7 +1263,7 @@ void Client::readArguments( std::string prev_host_arg; std::string prev_port_arg; - for (int arg_num = 1; arg_num < argc; ++arg_num) + for (int arg_num = start_argument_index; arg_num < argc; ++arg_num) { std::string_view arg = argv[arg_num]; @@ -1322,6 +1325,9 @@ void Client::readArguments( } else if (arg.starts_with("--host") || arg.starts_with("-h")) { + if (has_connection_string) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mixing connection string and --host/--port client arguments is prohibited"); + std::string host_arg; /// --host host if (arg == "--host" || arg == "-h") @@ -1353,6 +1359,9 @@ void Client::readArguments( } else if (arg.starts_with("--port")) { + if (has_connection_string) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mixing connection string and --host/--port client arguments is prohibited"); + auto port_arg = String{arg}; /// --port port if (arg == "--port") diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp new file mode 100644 index 00000000000..a8b87726a65 --- /dev/null +++ b/src/Client/ConnectionString.cpp @@ -0,0 +1,219 @@ +#include "ConnectionString.h" + +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +} + +namespace +{ + +using namespace std::string_literals; +using namespace std::literals::string_view_literals; + +constexpr auto CONNECTION_URI_SCHEME = "clickhouse:"sv; + +void uriDecode(std::string & uri_encoded_string, bool plus_as_space) +{ + std::string temp; + Poco::URI::decode(uri_encoded_string, temp, plus_as_space); + std::swap(temp, uri_encoded_string); +} + +void getHostAndPort(const Poco::URI & uri, std::vector> & hosts_and_ports_arguments) +{ + auto host = uri.getHost(); + std::vector host_and_port; + if (!host.empty()) + { + uriDecode(host, false); + host_and_port.push_back("--host="s + host); + } + + // Port can be written without host (":9000"). Empty host name equals to default host. + auto port = uri.getPort(); + if (port != 0) + host_and_port.push_back("--port="s + std::to_string(port)); + + if (!host_and_port.empty()) + hosts_and_ports_arguments.push_back(std::move(host_and_port)); +} + +void getHostAndPort( + Poco::URI & uri, + std::vector> & hosts_and_ports_arguments, + const char * host_begin, + const char * host_end, + const char * right_part_start, + const char * connection_string_end) +{ + // User info does not matter in sub URI + std::string uri_string = {CONNECTION_URI_SCHEME.begin(), CONNECTION_URI_SCHEME.end()}; + if (host_begin != nullptr && host_begin != host_end) + { + uri_string.append("//"); + uri_string.append(host_begin, host_end); + } + + // Right part from string includes '/database?[params]' + uri_string.append(right_part_start, connection_string_end); + try + { + uri = Poco::URI(uri_string); + } + catch (const Poco::URISyntaxException & invalid_uri_exception) + { + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, + "Invalid connection string syntax {}: {}", uri_string, invalid_uri_exception.what()); + } + + getHostAndPort(uri, hosts_and_ports_arguments); +} + +std::string makeArgument(const std::string & connection_string_parameter_name) +{ + return (connection_string_parameter_name.size() == 1 ? "-"s : "--"s) + connection_string_parameter_name; +} + +} + +namespace DB +{ + +bool tryParseConnectionString( + std::string_view connection_string, + std::vector & common_arguments, + std::vector> & hosts_and_ports_arguments) +{ + if (!connection_string.starts_with(CONNECTION_URI_SCHEME)) + return false; + + if (connection_string.size() == CONNECTION_URI_SCHEME.size()) + return true; + + auto offset = CONNECTION_URI_SCHEME.size(); + if ((connection_string.substr(offset).starts_with("//"))) + offset += 2; + + auto hosts_end_pos = std::string_view::npos; + auto hosts_or_user_info_end_pos = connection_string.find_first_of("?/@", offset); + + auto has_user_info = hosts_or_user_info_end_pos != std::string_view::npos && connection_string[hosts_or_user_info_end_pos] == '@'; + if (has_user_info) + { + // Move offset right after user info + offset = hosts_or_user_info_end_pos + 1; + hosts_end_pos = connection_string.find_first_of("?/@", offset); + // Several '@' symbols in connection string is prohibited. + // If user name contains '@' then it should be percent-encoded. + // several users: 'usr1@host1,@usr2@host2' is invalid. + if (hosts_end_pos != std::string_view::npos && connection_string[hosts_end_pos] == '@') + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Symbols '@' in URI in password or user name should be percent-encoded. Individual user names for different hosts also prohibited. {}", + connection_string); + } + } + else + hosts_end_pos = hosts_or_user_info_end_pos; + + auto hosts_end = hosts_end_pos != std::string_view::npos ? connection_string.begin() + hosts_end_pos + : connection_string.end(); + + try + { + // Poco::URI doesn't support several hosts in URI. + // Split string clickhouse:[user_info]host1:port1, ... , hostN:portN[database]?[query_parameters] + // into multiple string for each host: + // clickhouse:[user_info]host1:port1[database]?[query_parameters] + // ... + // clickhouse:[user_info]hostN:portN[database]?[query_parameters] + Poco::URI uri; + auto last_host_begin = connection_string.begin() + offset; + for (auto it = last_host_begin; it != hosts_end; ++it) + { + if (*it == ',') + { + getHostAndPort(uri, hosts_and_ports_arguments, last_host_begin, it, hosts_end, connection_string.end()); + last_host_begin = it + 1; + } + } + + if (uri.empty()) + { + // URI has no host specified + uri = std::string{connection_string.begin(), connection_string.end()}; + getHostAndPort(uri, hosts_and_ports_arguments); + } + else + getHostAndPort(uri, hosts_and_ports_arguments, last_host_begin, hosts_end, hosts_end, connection_string.end()); + + Poco::URI::QueryParameters params = uri.getQueryParameters(); + for (const auto & param : params) + { + if (param.first == "secure" || param.first == "s") + { + if (!param.second.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "secure URI argument does not require value"); + + common_arguments.push_back(makeArgument(param.first)); + } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "URI argument {} is unknown", param.first); + } + + auto user_info = uri.getUserInfo(); + if (!user_info.empty()) + { + // Poco::URI doesn't decode user name/password by default. + // But ClickHouse allows to have users with email user name like: 'john@some_mail.com' + // john@some_mail.com should be percent-encoded: 'john%40some_mail.com' + uriDecode(user_info, true); + std::string::size_type pos = user_info.find(':'); + if (pos != std::string::npos) + { + common_arguments.push_back("--user"); + common_arguments.push_back(user_info.substr(0, pos)); + + ++pos; // Skip ':' + common_arguments.push_back("--password"); + common_arguments.push_back(user_info.substr(pos)); + } + else + { + common_arguments.push_back("--user"); + common_arguments.push_back(user_info); + } + } + + const auto & database_name = uri.getPath(); + size_t start_symbol = database_name.size() > 0u && database_name[0] == '/' ? 1u : 0u; + if (database_name.size() > start_symbol) + { + common_arguments.push_back("--database"); + common_arguments.push_back(start_symbol == 0u ? database_name : database_name.substr(start_symbol)); + } + } + catch (const Poco::URISyntaxException & invalid_uri_exception) + { + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, + "Invalid connection string {}: {}", connection_string, invalid_uri_exception.what()); + } + + return true; +} + +} diff --git a/src/Client/ConnectionString.h b/src/Client/ConnectionString.h new file mode 100644 index 00000000000..aafb1139b00 --- /dev/null +++ b/src/Client/ConnectionString.h @@ -0,0 +1,22 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ +/** Tries to parse ClickHouse connection string. + * if @connection_string starts with 'clickhouse:' then connection string will be parsed + * and converted into a set of arguments for the client. + * Connection string format is similar to URI "clickhouse:[//[user_info@][hosts_and_ports]][/dbname][?query_parameters]" + * with the difference that hosts_and_ports can contain multiple hosts separated by ','. + * example: clickhouse://user@host1:port1,host2:port2 + * @return returns true if there is a URI, false otherwise. + * @exception throws DB::Exception if URI has valid scheme (clickhouse:), but invalid internals. +*/ +bool tryParseConnectionString( + std::string_view connection_string, + std::vector & common_arguments, + std::vector> & hosts_and_ports_arguments); +} diff --git a/tests/queries/0_stateless/02784_connection_string.reference b/tests/queries/0_stateless/02784_connection_string.reference new file mode 100644 index 00000000000..6a36abae8e0 --- /dev/null +++ b/tests/queries/0_stateless/02784_connection_string.reference @@ -0,0 +1,125 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +30 +31 +32 +33 +34 +35 +36 +37 +38 +39 +40 +41 +42 +43 +44 +45 +46 +47 +48 +49 +50 +51 +52 +53 +54 +55 +56 +57 +58 +59 +60 +61 +62 +63 +64 +65 +66 +67 +68 +500 +501 +502 +1000 +1001 +1002 +1003 +Bad arguments +Bad arguments +Bad arguments +Bad arguments +Bad arguments +Bad arguments +Bad arguments +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +Authentication failed +Authentication failed diff --git a/tests/queries/0_stateless/02784_connection_string.sh b/tests/queries/0_stateless/02784_connection_string.sh new file mode 100755 index 00000000000..fce93fdad74 --- /dev/null +++ b/tests/queries/0_stateless/02784_connection_string.sh @@ -0,0 +1,156 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +USER_INFOS=('default' '') +HOSTS_PORTS=("$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP" "$CLICKHOUSE_HOST" "$CLICKHOUSE_HOST:" ":$CLICKHOUSE_PORT_TCP" "127.0.0.1" "127.0.0.1:$CLICKHOUSE_PORT_TCP" "$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP,invalid_host:9000" "[0000:0000:0000:0000:0000:0000:0000:0001]" "[::1]" "[::1]:$CLICKHOUSE_PORT_TCP" "" ) +DATABASES=("$CLICKHOUSE_DATABASE" "") + +TEST_INDEX=0 + +function runClient() +{ + $CLICKHOUSE_CLIENT_BINARY "$@" -q "SELECT $TEST_INDEX" --log_comment 02766_connection_string.sh --send_logs_level=warning + ((++TEST_INDEX)) +} + +function testConnectionString() +{ + if [ "$database" == "" ]; then + runClient "clickhouse:$1" + runClient "clickhouse:$1/" + else + runClient "clickhouse:$1/$database" + fi +} + +function testConnectionWithUserName() +{ +if [ "$user_info" == "" ] && [ "$host_port" == "" ]; then + testConnectionString "//" + testConnectionString "" + else + testConnectionString "//$user_info@$host_port" + fi +} + +for user_info in "${USER_INFOS[@]}" +do + for host_port in "${HOSTS_PORTS[@]}" + do + for database in "${DATABASES[@]}" + do + testConnectionWithUserName + done + done +done + +# Specific user and password +TEST_INDEX=500 +TEST_USER_NAME="test_user_02771_$$" +TEST_USER_EMAIL_NAME="test_user_02771_$$@some_mail.com" +TEST_USER_EMAIL_NAME_ENCODED="test_user_02771_$$%40some_mail.com" + +TEST_USER_PASSWORD="zyx%$&abc" +# %, $, & percent encoded +TEST_USER_PASSWORD_ENCODED="zyx%25%24%26abc" + +$CLICKHOUSE_CLIENT -q "CREATE USER '$TEST_USER_NAME'" +$CLICKHOUSE_CLIENT -q "CREATE USER '$TEST_USER_EMAIL_NAME' IDENTIFIED WITH plaintext_password BY '$TEST_USER_PASSWORD'" + +runClient "clickhouse://$TEST_USER_NAME@$CLICKHOUSE_HOST/$CLICKHOUSE_DATABASE" +runClient "clickhouse://$TEST_USER_EMAIL_NAME_ENCODED:$TEST_USER_PASSWORD_ENCODED@$CLICKHOUSE_HOST/$CLICKHOUSE_DATABASE" + +$CLICKHOUSE_CLIENT -q "DROP USER '$TEST_USER_NAME'" +$CLICKHOUSE_CLIENT -q "DROP USER '$TEST_USER_EMAIL_NAME'" + +# Percent-encoded database in non-ascii symbols +UTF8_DATABASE="БазаДанных_$$" +UTF8_DATABASE_PERCENT_ENCODED="%D0%91%D0%B0%D0%B7%D0%B0%D0%94%D0%B0%D0%BD%D0%BD%D1%8B%D1%85_$$" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE IF NOT EXISTS \`$UTF8_DATABASE\`" +runClient "clickhouse://default@$CLICKHOUSE_HOST/$UTF8_DATABASE_PERCENT_ENCODED" +$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS \`$UTF8_DATABASE\`" + +# clickhouse-client extra options cases +TEST_INDEX=1000 + +runClient "clickhouse://$CLICKHOUSE_HOST/" --user 'default' +runClient "clickhouse://$CLICKHOUSE_HOST/default" --user 'default' +runClient "clickhouse:" --database "$CLICKHOUSE_DATABASE" + +# User 'default' and default host +runClient "clickhouse://default@" + +# Invalid URI cases +TEST_INDEX=10000 +runClient "clickhouse://default:@$CLICKHOUSE_HOST/" --user 'default' 2>&1 | grep -o 'Bad arguments' +runClient "clickhouse://default:pswrd@$CLICKHOUSE_HOST/" --user 'default' 2>&1 | grep -o 'Bad arguments' +runClient "clickhouse://default:pswrd@$CLICKHOUSE_HOST/" --password 'pswrd' 2>&1 | grep -o 'Bad arguments' +runClient "clickhouse:///$CLICKHOUSE_DATABASE" --database "$CLICKHOUSE_DATABASE" 2>&1 | grep -o 'Bad arguments' +runClient "clickhouse://$CLICKHOUSE_HOST/$CLICKHOUSE_DATABASE" --database "$CLICKHOUSE_DATABASE" 2>&1 | grep -o 'Bad arguments' +runClient "clickhouse://$CLICKHOUSE_HOST/$CLICKHOUSE_DATABASE?s" --database "$CLICKHOUSE_DATABASE" 2>&1 | grep -o 'Bad arguments' +runClient "clickhouse:/$CLICKHOUSE_DATABASE?s" --database "$CLICKHOUSE_DATABASE" 2>&1 | grep -o 'Bad arguments' + +runClient "http://" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "click_house:" 2>&1 | grep -o 'BAD_ARGUMENTS' + +TEST_INDEX=1000087 +# Using connection string prohibits to use --host and --port options +runClient "clickhouse://default:@$CLICKHOUSE_HOST/" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://default:@$CLICKHOUSE_HOST/" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://default:@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://default:@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" --port "$CLICKHOUSE_PORT_TCP" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://default:@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://default:@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" --host "$CLICKHOUSE_HOST" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://default:@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" --port "$CLICKHOUSE_PORT_TCP" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://default:@$CLICKHOUSE_HOST/" --port "$CLICKHOUSE_PORT_TCP" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://$CLICKHOUSE_HOST/" --port "$CLICKHOUSE_PORT_TCP" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://:@$CLICKHOUSE_HOST/" --port "$CLICKHOUSE_PORT_TCP" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://$CLICKHOUSE_HOST/" --port "$CLICKHOUSE_PORT_TCP" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse:" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse:///" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse:///?" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://:/?" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse:" --database "$CLICKHOUSE_DATABASE" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' + +# Space is used in connection string (This is prohibited). +runClient " clickhouse:" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse: " 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://host1 /" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://host1, host2/" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://host1 ,host2/" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://host1 host2/" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://host1/ database:" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://user :password@host1" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://user: password@host1" 2>&1 | grep -o 'BAD_ARGUMENTS' + +# Query is not first argument +runClient --multiline "clickhouse://default:@$CLICKHOUSE_HOST/" 2>&1 | grep -o 'BAD_ARGUMENTS' +# Query used as the first and the second argument of client +runClient "clickhouse://default:@$CLICKHOUSE_HOST/" "clickhouse://default:@$CLICKHOUSE_HOST/" 2>&1 | grep -o 'BAD_ARGUMENTS' + +# Invalid hosts +runClient "clickhouse://host1,,," 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://," 2>&1 | grep -o 'BAD_ARGUMENTS' + +# Invalid parameters +runClient "clickhouse:?invalid_parameter" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse:?invalid_parameter&secure" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse:?s&invalid_parameter" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse:?s&invalid_parameter=val" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse:?invalid_parameter=arg" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse:?invalid_parameter=arg&s" 2>&1 | grep -o 'BAD_ARGUMENTS' +# Several users prohibited +runClient "clickhouse://user1@localhost,default@localhost/" 2>&1 | grep -o 'BAD_ARGUMENTS' +# Using '@' in user name is prohibited. User name should be percent-encoded. +runClient "clickhouse://my_mail@email.com@host/" 2>&1 | grep -o 'BAD_ARGUMENTS' + +# Wrong input cases +TEST_INDEX=100000 +# Invalid user name +runClient "clickhouse://non_exist_user@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" 2>&1 | grep -o 'Authentication failed' +# Invalid password +runClient "clickhouse://default:invalid_password@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" 2>&1 | grep -o 'Authentication failed' From 17754bf6941aa0754db2bb2de5c7098f890c2898 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 7 Jun 2023 05:59:13 +0000 Subject: [PATCH 1308/2223] minor changes in documentation --- docs/en/interfaces/cli.md | 4 ++-- docs/ru/interfaces/cli.md | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 5255657ddfd..94f1fbf9e41 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -168,7 +168,7 @@ clickhouse://[user_info@][hosts_and_ports][/dbname][?query_parameters] where user_info is: ```user[:password]``` and hosts_and_ports is a list of values: ```[host][:port],[host][:port]``` Port is not mandatory. -and query_parameters is a list of parameter[=value]: ```param_name[=value]¶m_name[=value]...``` value may not be required for some of parameters. +and query_parameters is a list of parameter[=value]: ```param_name[=value]¶m_name[=value]...``` value may not be required for some of parameters. Parameter names are case sensitive. Allowed query_parameters keys: @@ -198,7 +198,7 @@ If host is not specified, the default host will be used (localhost). If port is not specified, the default port will be used (9000). If database is not specified, the default database will be used. -User, password, and database can be specified in the connection string either in --user command line option. +User, password, and database can be specified in the connection string either in --user, --password, --database command line options. The connection string must be specified in the first argument of clickhouse-client. The connection string can be combined with other [command-line-options](#command-line-options) except **--host(h)** and **--port**. diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 06642800cc6..30cd9757ebb 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -120,7 +120,8 @@ clickhouse://[user_info@][hosts_and_ports][/dbname][?query_parameters] где user_info - это: ```user[:password]``` hosts_and_ports - это список значений: ```[host][:port],[host][:port]```. Port может быть не задан. -query_parameters - это список пар ключ[=значение]: ```param_name[=value]¶m_name[=value]...```. Значение может быть пустым +query_parameters - это список пар ключ[=значение]: ```param_name[=value]¶m_name[=value]...```. Значение может быть пустым. +Имена параметров чувствительны к регистру. Допустимые ключи query_parameters: From 4a0ccc25d21e9f333057a421bd8009d648df17ae Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 7 Jun 2023 06:15:10 +0000 Subject: [PATCH 1309/2223] Minor improvement --- src/Client/ConnectionString.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp index a8b87726a65..7d76deb6238 100644 --- a/src/Client/ConnectionString.cpp +++ b/src/Client/ConnectionString.cpp @@ -61,7 +61,7 @@ void getHostAndPort( const char * connection_string_end) { // User info does not matter in sub URI - std::string uri_string = {CONNECTION_URI_SCHEME.begin(), CONNECTION_URI_SCHEME.end()}; + auto uri_string = std::string(CONNECTION_URI_SCHEME); if (host_begin != nullptr && host_begin != host_end) { uri_string.append("//"); From aaa4d0367e9d51cd5308a7d5a02fd8333e9e7bb1 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 7 Jun 2023 06:29:14 +0000 Subject: [PATCH 1310/2223] Minor improvement for connection string --- src/Client/ConnectionString.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp index 7d76deb6238..aeb1c1dca02 100644 --- a/src/Client/ConnectionString.cpp +++ b/src/Client/ConnectionString.cpp @@ -55,21 +55,19 @@ void getHostAndPort(const Poco::URI & uri, std::vector> void getHostAndPort( Poco::URI & uri, std::vector> & hosts_and_ports_arguments, - const char * host_begin, - const char * host_end, - const char * right_part_start, - const char * connection_string_end) + std::string_view host_and_port, + std::string_view right_part) { // User info does not matter in sub URI auto uri_string = std::string(CONNECTION_URI_SCHEME); - if (host_begin != nullptr && host_begin != host_end) + if (!host_and_port.empty()) { uri_string.append("//"); - uri_string.append(host_begin, host_end); + uri_string.append(host_and_port); } // Right part from string includes '/database?[params]' - uri_string.append(right_part_start, connection_string_end); + uri_string.append(right_part); try { uri = Poco::URI(uri_string); @@ -147,7 +145,7 @@ bool tryParseConnectionString( { if (*it == ',') { - getHostAndPort(uri, hosts_and_ports_arguments, last_host_begin, it, hosts_end, connection_string.end()); + getHostAndPort(uri, hosts_and_ports_arguments, {last_host_begin, it}, {hosts_end, connection_string.end()}); last_host_begin = it + 1; } } @@ -159,7 +157,7 @@ bool tryParseConnectionString( getHostAndPort(uri, hosts_and_ports_arguments); } else - getHostAndPort(uri, hosts_and_ports_arguments, last_host_begin, hosts_end, hosts_end, connection_string.end()); + getHostAndPort(uri, hosts_and_ports_arguments, {last_host_begin, hosts_end}, {hosts_end, connection_string.end()}); Poco::URI::QueryParameters params = uri.getQueryParameters(); for (const auto & param : params) From bea4e8e81f6a782edc3dc2a672ae3011843de305 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Jun 2023 06:45:14 +0000 Subject: [PATCH 1311/2223] Add new files --- programs/keeper/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index f775e8a5a22..90f4f870df6 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -131,6 +131,8 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/ObjectStorageIterator.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/StoredObject.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/registerDiskS3.cpp From 4050b637f16554421423d92c501d9790deb42394 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Jun 2023 09:01:20 +0000 Subject: [PATCH 1312/2223] ALTER TABLE ADD INDEX: Add default GRANULARITY argument for secondary indexes - Related to #45451, which provides a default GRANULARITY when the skipping index is created in CREATE TABLE. --- docs/en/sql-reference/statements/alter/skipping-index.md | 2 +- src/Parsers/ParserCreateIndexQuery.cpp | 9 ++++++--- src/Parsers/ParserCreateQuery.cpp | 2 +- .../0_stateless/02534_default_granularity.reference | 1 + tests/queries/0_stateless/02534_default_granularity.sql | 7 ++++++- 5 files changed, 15 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/skipping-index.md b/docs/en/sql-reference/statements/alter/skipping-index.md index 67af76986da..4194731d33a 100644 --- a/docs/en/sql-reference/statements/alter/skipping-index.md +++ b/docs/en/sql-reference/statements/alter/skipping-index.md @@ -10,7 +10,7 @@ sidebar_label: INDEX The following operations are available: -- `ALTER TABLE [db].table_name [ON CLUSTER cluster] ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name]` - Adds index description to tables metadata. +- `ALTER TABLE [db].table_name [ON CLUSTER cluster] ADD INDEX name expression TYPE type [GRANULARITY value] [FIRST|AFTER name]` - Adds index description to tables metadata. - `ALTER TABLE [db].table_name [ON CLUSTER cluster] DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index ab31d3f9b7a..e878b347e62 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -36,17 +36,20 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (!data_type_p.parse(pos, type, expected)) return false; - if (!s_granularity.ignore(pos, expected)) - return false; + if (s_granularity.ignore(pos, expected)) + { + if (!granularity_p.parse(pos, granularity, expected)) + return false; + } if (!granularity_p.parse(pos, granularity, expected)) return false; auto index = std::make_shared(); index->part_of_create_index_query = true; - index->granularity = granularity->as().value.safeGet(); index->set(index->expr, expr); index->set(index->type, type); + index->granularity = granularity ? granularity->as().value.safeGet() : 1; node = index; return true; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 8cbfac91465..f975e8ba3c8 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -139,9 +139,9 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe auto index = std::make_shared(); index->name = name->as().name(); - index->granularity = granularity ? granularity->as().value.safeGet() : 1; index->set(index->expr, expr); index->set(index->type, type); + index->granularity = granularity ? granularity->as().value.safeGet() : 1; node = index; return true; diff --git a/tests/queries/0_stateless/02534_default_granularity.reference b/tests/queries/0_stateless/02534_default_granularity.reference index e60036653c9..0fe7fe0a1b3 100644 --- a/tests/queries/0_stateless/02534_default_granularity.reference +++ b/tests/queries/0_stateless/02534_default_granularity.reference @@ -1 +1,2 @@ CREATE TABLE default.users_02534\n(\n `id` Int16,\n `name` String,\n INDEX bf_idx name TYPE minmax GRANULARITY 1\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.users_02534\n(\n `id` Int16,\n `name` String,\n INDEX bf_idx name TYPE minmax GRANULARITY 1\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02534_default_granularity.sql b/tests/queries/0_stateless/02534_default_granularity.sql index 781df3ce934..e3de5fce7c8 100644 --- a/tests/queries/0_stateless/02534_default_granularity.sql +++ b/tests/queries/0_stateless/02534_default_granularity.sql @@ -1,4 +1,9 @@ DROP TABLE IF EXISTS users_02534; CREATE TABLE users_02534 (id Int16, name String, INDEX bf_idx(name) TYPE minmax) ENGINE=MergeTree ORDER BY id; SHOW CREATE TABLE users_02534; -DROP TABLE users_02534; \ No newline at end of file +DROP TABLE users_02534; + +CREATE TABLE users_02534 (id Int16, name String) ENGINE=MergeTree ORDER BY id; +ALTER TABLE users_02534 ADD INDEX bf_idx(name) TYPE minmax; +SHOW CREATE TABLE users_02534; +DROP TABLE users_02534; From c795eb03299f751dd4a0c69facb2d5a6bec101da Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Jun 2023 09:46:10 +0000 Subject: [PATCH 1313/2223] Temporarily disable a test --- tests/queries/0_stateless/02354_annoy_index.sql | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 170c048d420..0168fa04c6f 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -20,11 +20,12 @@ FROM tab ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) LIMIT 3; -SELECT 'Reference ARRAYs with non-matching dimension are rejected'; -SELECT * -FROM tab -ORDER BY L2Distance(embedding, [0.0, 0.0]) -LIMIT 3; -- { serverError INCORRECT_QUERY } +-- Produces different error code with analyzer, TODO: check +-- SELECT 'Reference ARRAYs with non-matching dimension are rejected'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(embedding, [0.0, 0.0]) +-- LIMIT 3; -- { serverError INCORRECT_QUERY } SELECT 'WHERE type, L2Distance, check that index is used'; EXPLAIN indexes=1 From 7c800468349e4aea2c125459f44b581d14391a10 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Jun 2023 09:47:54 +0000 Subject: [PATCH 1314/2223] Revert "Remove clang-tidy exclude" This reverts commit 42c054789561920adf7ce4770968ba303a70f244. --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index ffed9e01df0..1a28f28f746 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -123,6 +123,7 @@ MergeTreeIndexAggregatorAnnoy::MergeTreeIndexAggregatorAnnoy( template MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy::getGranuleAndReset() { + // NOLINTNEXTLINE(*) index->build(static_cast(trees), /*number_of_threads=*/1); auto granule = std::make_shared>(index_name, index_sample_block, index); index = nullptr; From e0bc695e2d95085e1927b44fc2ad5a9d3384c1d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Jun 2023 10:07:35 +0000 Subject: [PATCH 1315/2223] Use correct link format --- docs/en/sql-reference/aggregate-functions/index.md | 4 ++-- .../sql-reference/aggregate-functions/reference/argmax.md | 6 +++--- .../sql-reference/aggregate-functions/reference/argmin.md | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index ea270e83a3c..5d2229fbcce 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -4,7 +4,7 @@ sidebar_label: Aggregate Functions sidebar_position: 33 --- -# Aggregate Functions +# Aggregate Functions Aggregate functions work in the [normal](http://www.sql-tutorial.com/sql-aggregate-functions-sql-tutorial) way as expected by database experts. @@ -73,7 +73,7 @@ FROM t_null_big └────────────────────┴─────────────────────┘ ``` -Also you can use [Tuple](../data-types/tuple.md) to work around NULL skipping behavior. The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value. +Also you can use [Tuple](/docs/en/sql-reference/data-types/tuple.md) to work around NULL skipping behavior. The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value. ```sql SELECT diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 93e1fac6d67..8f10318838b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -5,8 +5,8 @@ sidebar_position: 106 # argMax -Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. +Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. +Both parts the `arg` and the `max` behave as [aggregate functions](/docs/en/sql-reference/aggregate-functions/index.md), they both [skip `Null`](/docs/en/sql-reference/aggregate-functions/index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. **Syntax** @@ -106,4 +106,4 @@ SELECT argMax(a, tuple(b)) FROM test; **See also** -- [Tuple](../../data-types/tuple.md) +- [Tuple](/docs/en/sql-reference/data-types/tuple.md) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 4e549e5b04c..47d4ab398de 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -6,7 +6,7 @@ sidebar_position: 105 # argMin Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. +Both parts the `arg` and the `min` behave as [aggregate functions](/docs/en/sql-reference/aggregate-functions/index.md), they both [skip `Null`](/docs/en/sql-reference/aggregate-functions/index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. **Syntax** @@ -111,4 +111,4 @@ select argMin(a, tuple(b)) from test; **See also** -- [Tuple](../../data-types/tuple.md) +- [Tuple](/docs/en/sql-reference/data-types/tuple.md) From 35ef14482d785226a660c62fea558fdb91a1d26f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Jun 2023 10:11:49 +0000 Subject: [PATCH 1316/2223] Fix keyword capitalization --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 47d4ab398de..fdfce0833e0 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -103,9 +103,9 @@ SELECT argMin((a, b), (b, a)), min(tuple(b, a)) FROM test; │ (NULL,NULL) │ (NULL,NULL) │ -- argMin returns (NULL,NULL) here because `Tuple` allows to don't skip `NULL` and min(tuple(b, a)) in this case is minimal value for this dataset └──────────────────────────────────┴──────────────────┘ -select argMin(a, tuple(b)) from test; +SELECT argMin(a, tuple(b)) FROM test; ┌─argMax(a, tuple(b))─┐ -│ d │ -- `Tuple` can be used in `min` to not skip rows with `NULL` values as b. +│ d │ -- `Tuple` can be used in `min` to not skip rows with `NULL` values as b. └─────────────────────┘ ``` From 71ae54f089f51c396616842743e8ba0a7f38bc59 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 7 Jun 2023 13:34:01 +0200 Subject: [PATCH 1317/2223] Fix args --- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 07173e65448..20c60cfe8f5 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -52,7 +52,6 @@ public: options.Prefix = path_prefix; options.PageSizeHint = static_cast(max_list_size); - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "ITER PREFIX {}", path_prefix); } private: @@ -63,7 +62,6 @@ private: auto blob_list_response = client->ListBlobs(options); auto blobs_list = blob_list_response.Blobs; - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "BLOB LIST SIZE {}", blobs_list.size()); for (const auto & blob : blobs_list) { batch.emplace_back( @@ -77,13 +75,9 @@ private: } if (!blob_list_response.NextPageToken.HasValue() || blob_list_response.NextPageToken.Value().empty()) - { - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "RETURN FALSE {}", blobs_list.size()); return false; - } options.ContinuationToken = blob_list_response.NextPageToken; - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "RETURN TRUE {}", blobs_list.size()); return true; } @@ -222,6 +216,7 @@ std::unique_ptr AzureObjectStorage::readObjects( /// NOL settings_ptr->max_single_read_retries, settings_ptr->max_single_download_retries, /* use_external_buffer */true, + /* restricted_seek */true, read_until_position); }; From 036ddcd47baf88ab0c360efe647e01060d1ce636 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 7 Jun 2023 13:48:08 +0200 Subject: [PATCH 1318/2223] Fix excessive memory usage for FINAL (due to too much streams usage) (#50429) Previously it could create MergeTreeInOrder for each mark, however this could be very suboptimal, due to each MergeTreeInOrder has some memory overhead. Now, by collapsing all marks for one part together it is more memory effiecient. I've tried the query from the altinity wiki [1] and it decreases memory usage twice: SELECT * FROM repl_tbl FINAL WHERE key IN (SELECT toUInt32(number) FROM numbers(1000000) WHERE number % 50000 = 0) FORMAT Null - upstream: MemoryTracker: Peak memory usage (for query): 520.27 MiB. - patched: MemoryTracker: Peak memory usage (for query): 260.95 MiB. [1]: https://kb.altinity.com/engines/mergetree-table-engine-family/replacingmergetree/#multiple-keys And it could be not 2x and even more or less, it depends on the gaps in marks for reading (for example in my setup the memory usage increased a lot, from ~16GiB of RAM to >64GiB due to lots of marks and gaps). Signed-off-by: Azat Khuzhin --- src/Processors/QueryPlan/PartsSplitter.cpp | 35 ++++++++++----- ...inal_streams_data_skipping_index.reference | 43 +++++++++++++++++++ ...2780_final_streams_data_skipping_index.sql | 28 ++++++++++++ 3 files changed, 95 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02780_final_streams_data_skipping_index.reference create mode 100644 tests/queries/0_stateless/02780_final_streams_data_skipping_index.sql diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 936182f8c00..9796e696f6c 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -126,7 +126,9 @@ std::pair, std::vector> split(RangesInDat return marks_in_current_layer < intersected_parts * 2; }; - result_layers.emplace_back(); + auto & current_layer = result_layers.emplace_back(); + /// Map part_idx into index inside layer, used to merge marks from the same part into one reader + std::unordered_map part_idx_in_layer; while (rows_in_current_layer < rows_per_layer || layers_intersection_is_too_big() || result_layers.size() == max_layers) { @@ -140,11 +142,16 @@ std::pair, std::vector> split(RangesInDat if (current.event == PartsRangesIterator::EventType::RangeEnd) { - result_layers.back().emplace_back( - parts[part_idx].data_part, - parts[part_idx].alter_conversions, - parts[part_idx].part_index_in_query, - MarkRanges{{current_part_range_begin[part_idx], current.range.end}}); + const auto & mark = MarkRange{current_part_range_begin[part_idx], current.range.end}; + auto it = part_idx_in_layer.emplace(std::make_pair(part_idx, current_layer.size())); + if (it.second) + current_layer.emplace_back( + parts[part_idx].data_part, + parts[part_idx].alter_conversions, + parts[part_idx].part_index_in_query, + MarkRanges{mark}); + else + current_layer[it.first->second].ranges.push_back(mark); current_part_range_begin.erase(part_idx); current_part_range_end.erase(part_idx); @@ -170,11 +177,17 @@ std::pair, std::vector> split(RangesInDat } for (const auto & [part_idx, last_mark] : current_part_range_end) { - result_layers.back().emplace_back( - parts[part_idx].data_part, - parts[part_idx].alter_conversions, - parts[part_idx].part_index_in_query, - MarkRanges{{current_part_range_begin[part_idx], last_mark + 1}}); + const auto & mark = MarkRange{current_part_range_begin[part_idx], last_mark + 1}; + auto it = part_idx_in_layer.emplace(std::make_pair(part_idx, current_layer.size())); + + if (it.second) + result_layers.back().emplace_back( + parts[part_idx].data_part, + parts[part_idx].alter_conversions, + parts[part_idx].part_index_in_query, + MarkRanges{mark}); + else + current_layer[it.first->second].ranges.push_back(mark); current_part_range_begin[part_idx] = current_part_range_end[part_idx]; } diff --git a/tests/queries/0_stateless/02780_final_streams_data_skipping_index.reference b/tests/queries/0_stateless/02780_final_streams_data_skipping_index.reference new file mode 100644 index 00000000000..d7a540ae479 --- /dev/null +++ b/tests/queries/0_stateless/02780_final_streams_data_skipping_index.reference @@ -0,0 +1,43 @@ +-- { echoOn } +EXPLAIN PIPELINE SELECT * FROM data FINAL WHERE v1 >= now() - INTERVAL 180 DAY +SETTINGS max_threads=2, max_final_threads=2, force_data_skipping_indices='v1_index', use_skip_indexes_if_final=1 +FORMAT LineAsString; +(Expression) +ExpressionTransform × 2 + (Filter) + FilterTransform × 2 + (ReadFromMergeTree) + ExpressionTransform × 2 + AggregatingSortedTransform 2 → 1 + ExpressionTransform × 2 + FilterSortedStreamByRange × 2 + Description: filter values in [(999424), +inf) + ExpressionTransform × 2 + MergeTreeInOrder × 2 0 → 1 + AggregatingSortedTransform + ExpressionTransform + FilterSortedStreamByRange + Description: filter values in [-inf, (999424)) + ExpressionTransform + MergeTreeInOrder 0 → 1 +EXPLAIN PIPELINE SELECT * FROM data FINAL WHERE v1 >= now() - INTERVAL 180 DAY +SETTINGS max_threads=2, max_final_threads=2, force_data_skipping_indices='v1_index', use_skip_indexes_if_final=0 +FORMAT LineAsString; +(Expression) +ExpressionTransform × 2 + (Filter) + FilterTransform × 2 + (ReadFromMergeTree) + ExpressionTransform × 2 + AggregatingSortedTransform 2 → 1 + ExpressionTransform × 2 + FilterSortedStreamByRange × 2 + Description: filter values in [(999424), +inf) + ExpressionTransform × 2 + MergeTreeInOrder × 2 0 → 1 + AggregatingSortedTransform + ExpressionTransform + FilterSortedStreamByRange + Description: filter values in [-inf, (999424)) + ExpressionTransform + MergeTreeInOrder 0 → 1 diff --git a/tests/queries/0_stateless/02780_final_streams_data_skipping_index.sql b/tests/queries/0_stateless/02780_final_streams_data_skipping_index.sql new file mode 100644 index 00000000000..7de7a58e2e1 --- /dev/null +++ b/tests/queries/0_stateless/02780_final_streams_data_skipping_index.sql @@ -0,0 +1,28 @@ +-- Tags: no-random-merge-tree-settings, no-random-settings + +DROP TABLE IF EXISTS data; + +CREATE TABLE data +( + key Int, + v1 DateTime, + INDEX v1_index v1 TYPE minmax GRANULARITY 1 +) ENGINE=AggregatingMergeTree() +ORDER BY key +SETTINGS index_granularity=8192, min_bytes_for_wide_part=0, min_rows_for_wide_part=0; + +SYSTEM STOP MERGES data; + +-- generate 50% of marks that cannot be skipped with v1_index +-- this will create a gap in marks +INSERT INTO data SELECT number, if(number/8192 % 2 == 0, now(), now() - INTERVAL 200 DAY) FROM numbers(1e6); +INSERT INTO data SELECT number+1e6, if(number/8192 % 2 == 0, now(), now() - INTERVAL 200 DAY) FROM numbers(1e6); + +-- { echoOn } +EXPLAIN PIPELINE SELECT * FROM data FINAL WHERE v1 >= now() - INTERVAL 180 DAY +SETTINGS max_threads=2, max_final_threads=2, force_data_skipping_indices='v1_index', use_skip_indexes_if_final=1 +FORMAT LineAsString; + +EXPLAIN PIPELINE SELECT * FROM data FINAL WHERE v1 >= now() - INTERVAL 180 DAY +SETTINGS max_threads=2, max_final_threads=2, force_data_skipping_indices='v1_index', use_skip_indexes_if_final=0 +FORMAT LineAsString; From 81cd3defd79331fc0af016c4c40a957be15a227a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Jun 2023 12:29:09 +0000 Subject: [PATCH 1319/2223] Fix expected results --- tests/queries/0_stateless/02354_annoy_index.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 5bd1377d6f4..45515bc7733 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -9,7 +9,6 @@ ORDER BY type, L2Distance 1 [0,0,10] 5 [0,0,10.2] 4 [0,0,9.7] -Reference ARRAYs with non-matching dimension are rejected WHERE type, L2Distance, check that index is used Expression ((Projection + Before ORDER BY)) Limit (preliminary LIMIT (without OFFSET)) From 87ac6b8b637e9931c32fffb8a273101295c161ba Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 7 Jun 2023 12:49:28 +0000 Subject: [PATCH 1320/2223] Fix reading negative decimals in avro format --- .../Formats/Impl/AvroRowInputFormat.cpp | 10 ++++++++-- .../0_stateless/02782_avro_decimals.reference | 12 ++++++++++++ .../queries/0_stateless/data_avro/decimals.avro | Bin 295 -> 353 bytes 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 60e541a0109..c8e4a499b81 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -184,8 +184,14 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No field_type_size, tmp.size()); else if (tmp.size() != field_type_size) - /// Add padding with 0-bytes. - tmp = std::string(field_type_size - tmp.size(), '\0') + tmp; + { + /// Extent value to required size by adding paddinf. + /// Check if value is negative or positive. + if (tmp[0] & 128) + tmp = std::string(field_type_size - tmp.size(), 0xff) + tmp; + else + tmp = std::string(field_type_size - tmp.size(), 0) + tmp; + } typename DecimalType::FieldType field; ReadBufferFromString buf(tmp); diff --git a/tests/queries/0_stateless/02782_avro_decimals.reference b/tests/queries/0_stateless/02782_avro_decimals.reference index ed46f1c3758..a16e5e4ac56 100644 --- a/tests/queries/0_stateless/02782_avro_decimals.reference +++ b/tests/queries/0_stateless/02782_avro_decimals.reference @@ -11,3 +11,15 @@ d Decimal(14, 4) 12345678.1234 123456789.1234 1234567890.1234 +0 +-1 +-1.1 +-12.12 +-123.123 +-1234.1234 +-12345.1234 +-123456.1234 +-1234567.1234 +-12345678.1234 +-123456789.1234 +-1234567890.1234 diff --git a/tests/queries/0_stateless/data_avro/decimals.avro b/tests/queries/0_stateless/data_avro/decimals.avro index 5c29ac235d59003696735c8c8092eed5bcce41b2..630d6864047e9638da9dc1437780c41bd820ca12 100644 GIT binary patch delta 103 zcmZ3^^pI)78DSNbwU@s*w@gW!@aK?AzgoA!6UK@6EIpYRSZ;h^xyr%zPi+F*TeBvP w|IQouIDRbgn8&d#OI(lZuYwBWKCZ`(=`4Oc{~q+Qe7nc<$6Jnr7T_unL4xsVKrqKWq{1<*wS0^Syk From cf947e6e01036d51ef0337378b12e868a07ecce2 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 7 Jun 2023 12:50:16 +0000 Subject: [PATCH 1321/2223] Fix typo --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index c8e4a499b81..1ec7491658e 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -185,7 +185,7 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No tmp.size()); else if (tmp.size() != field_type_size) { - /// Extent value to required size by adding paddinf. + /// Extent value to required size by adding padding. /// Check if value is negative or positive. if (tmp[0] & 128) tmp = std::string(field_type_size - tmp.size(), 0xff) + tmp; From db1c03d6db270a4a6b059d4b7f09c5d264f13081 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Jun 2023 15:54:39 +0000 Subject: [PATCH 1322/2223] Cleanup moving parts --- src/Storages/MergeTree/MergeTreeData.cpp | 26 +++++++++----- src/Storages/MergeTree/MergeTreeData.h | 1 + .../MergeTree/MergeTreePartsMover.cpp | 36 ++++++++++++++----- src/Storages/MergeTree/MergeTreePartsMover.h | 10 +++++- 4 files changed, 55 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e806e1bb93f..047f063cb7c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1998,8 +1998,18 @@ static bool isOldPartDirectory(const DiskPtr & disk, const String & directory_pa return true; } - size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes) +{ + size_t cleared_count = 0; + + cleared_count += clearOldTemporaryDirectories(relative_data_path, custom_directories_lifetime_seconds, valid_prefixes); + + /// Clear _all_ parts from the `moving` directory + cleared_count += clearOldTemporaryDirectories(fs::path(relative_data_path) / "moving", custom_directories_lifetime_seconds, {""}); + return cleared_count; +} + +size_t MergeTreeData::clearOldTemporaryDirectories(const String & root_path, size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes) { /// If the method is already called from another thread, then we don't need to do anything. std::unique_lock lock(clear_old_temporary_directories_mutex, std::defer_lock); @@ -2018,7 +2028,7 @@ size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lif if (disk->isBroken()) continue; - for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next()) + for (auto it = disk->iterateDirectory(root_path); it->isValid(); it->next()) { const std::string & basename = it->name(); bool start_with_valid_prefix = false; @@ -7802,7 +7812,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & for (const auto & moving_part : moving_tagger->parts_to_move) { Stopwatch stopwatch; - MutableDataPartPtr cloned_part; + MergeTreePartsMover::TemporaryClonedPart cloned_part; ProfileEventsScope profile_events_scope; auto write_part_log = [&](const ExecutionStatus & execution_status) @@ -7812,7 +7822,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & execution_status, stopwatch.elapsed(), moving_part.part->name, - cloned_part, + cloned_part.part, {moving_part.part}, nullptr, profile_events_scope.getSnapshot()); @@ -7854,7 +7864,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & if (lock->isLocked()) { cloned_part = parts_mover.clonePart(moving_part); - parts_mover.swapClonedPart(cloned_part); + parts_mover.swapClonedPart(cloned_part.part); break; } else if (wait_for_move_if_zero_copy) @@ -7881,15 +7891,15 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & else /// Ordinary move as it should be { cloned_part = parts_mover.clonePart(moving_part); - parts_mover.swapClonedPart(cloned_part); + parts_mover.swapClonedPart(cloned_part.part); } write_part_log({}); } catch (...) { write_part_log(ExecutionStatus::fromCurrentException("", true)); - if (cloned_part) - cloned_part->remove(); + if (cloned_part.part) + cloned_part.part->remove(); throw; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2f254f9a787..444bd8f47ac 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -676,6 +676,7 @@ public: /// Delete all directories which names begin with "tmp" /// Must be called with locked lockForShare() because it's using relative_data_path. size_t clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes = {"tmp_", "tmp-fetch_"}); + size_t clearOldTemporaryDirectories(const String & root_path, size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes); size_t clearEmptyParts(); diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index e1da57744b3..08815fa1f0c 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -11,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int ABORTED; + extern const int DIRECTORY_ALREADY_EXISTS; } namespace @@ -203,7 +204,7 @@ bool MergeTreePartsMover::selectPartsForMove( return false; } -MergeTreeMutableDataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEntry & moving_part) const +MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const MergeTreeMoveEntry & moving_part) const { if (moves_blocker.isCancelled()) throw Exception(ErrorCodes::ABORTED, "Cancelled moving parts."); @@ -222,8 +223,10 @@ MergeTreeMutableDataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEn String relative_path = part->getDataPartStorage().getPartDirectory(); if (disk->exists(path_to_clone + relative_path)) { - LOG_WARNING(log, "Path {} already exists. Will remove it and clone again.", fullPath(disk, path_to_clone + relative_path)); - disk->removeRecursive(fs::path(path_to_clone) / relative_path / ""); + throw Exception(ErrorCodes::DIRECTORY_ALREADY_EXISTS, + "Cannot clone part {} from '{}' to '{}': path '{}' already exists", + part->name, part->getDataPartStorage().getDiskName(), disk->getName(), + fullPath(disk, path_to_clone + relative_path)); } disk->createDirectories(path_to_clone); @@ -240,14 +243,22 @@ MergeTreeMutableDataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEn { cloned_part_storage = part->makeCloneOnDisk(disk, MergeTreeData::MOVING_DIR_NAME); } + String data_part_directory = cloned_part_storage->getFullPath(); + + TemporaryClonedPart cloned_part; + cloned_part.temporary_directory_lock = data->getTemporaryPartDirectoryHolder(data_part_directory); MergeTreeDataPartBuilder builder(*data, part->name, cloned_part_storage); - auto cloned_part = std::move(builder).withPartFormatFromDisk().build(); - LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part->getDataPartStorage().getFullPath()); + cloned_part.part = std::move(builder).withPartFormatFromDisk().build(); - cloned_part->loadColumnsChecksumsIndexes(true, true); - cloned_part->loadVersionMetadata(); - cloned_part->modification_time = cloned_part->getDataPartStorage().getLastModified().epochTime(); + String part_directory = cloned_part.part->getDataPartStorage().getFullPath(); + + LOG_TRACE(log, "Part {} was cloned to {}", part->name, data_part_directory); + + cloned_part.part->loadColumnsChecksumsIndexes(true, true); + cloned_part.part->loadVersionMetadata(); + cloned_part.part->modification_time = cloned_part.part->getDataPartStorage().getLastModified().epochTime(); + cloned_part.part->is_temp = true; return cloned_part; } @@ -262,10 +273,17 @@ void MergeTreePartsMover::swapClonedPart(const MergeTreeMutableDataPartPtr & clo /// It's ok, because we don't block moving parts for merges or mutations if (!active_part || active_part->name != cloned_part->name) { - LOG_INFO(log, "Failed to swap {}. Active part doesn't exist. Possible it was merged or mutated. Will remove copy on path '{}'.", cloned_part->name, cloned_part->getDataPartStorage().getFullPath()); + LOG_INFO(log, + "Failed to swap {}. Active part doesn't exist (containing part {}). " + "Possible it was merged or mutated. Will remove copy on path '{}'", + cloned_part->name, + active_part ? active_part->name : "doesn't exist", + cloned_part->getDataPartStorage().getFullPath()); return; } + cloned_part->is_temp = false; + /// Don't remove new directory but throw an error because it may contain part which is currently in use. cloned_part->renameTo(active_part->name, false); diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index 1cee98bcba9..dde2ff1a630 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -43,12 +44,19 @@ private: using AllowedMovingPredicate = std::function &, String * reason)>; public: + explicit MergeTreePartsMover(MergeTreeData * data_) : data(data_) , log(&Poco::Logger::get("MergeTreePartsMover")) { } + struct TemporaryClonedPart + { + MergeTreeMutableDataPartPtr part; + scope_guard temporary_directory_lock; + }; + /// Select parts for background moves according to storage_policy configuration. /// Returns true if at least one part was selected for move. bool selectPartsForMove( @@ -57,7 +65,7 @@ public: const std::lock_guard & moving_parts_lock); /// Copies part to selected reservation in detached folder. Throws exception if part already exists. - MergeTreeMutableDataPartPtr clonePart(const MergeTreeMoveEntry & moving_part) const; + TemporaryClonedPart clonePart(const MergeTreeMoveEntry & moving_part) const; /// Replaces cloned part from detached directory into active data parts set. /// Replacing part changes state to DeleteOnDestroy and will be removed from disk after destructor of From 19bb802b04cb9c31585b24f6ebe4765f0dc46c87 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Jun 2023 17:41:16 +0000 Subject: [PATCH 1323/2223] Set temporary_directories_lifetime to integration tests with MOVE --- .../test_consistant_parts_after_move_partition/test.py | 2 +- tests/integration/test_encrypted_disk/test.py | 2 +- tests/integration/test_merge_tree_azure_blob_storage/test.py | 1 + tests/integration/test_merge_tree_hdfs/test.py | 3 ++- tests/integration/test_merge_tree_s3/test.py | 1 + .../test_move_partition_to_disk_on_cluster/test.py | 2 +- .../configs/config.d/storage_configuration.xml | 4 ++++ .../configs/config.d/storage_configuration.xml | 3 ++- .../test_replicated_merge_tree_hdfs_zero_copy/test.py | 2 +- tests/integration/test_s3_zero_copy_replication/test.py | 2 +- tests/integration/test_ttl_move/test.py | 2 +- tests/integration/test_zero_copy_fetch/test.py | 2 +- 12 files changed, 17 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_consistant_parts_after_move_partition/test.py b/tests/integration/test_consistant_parts_after_move_partition/test.py index 63a51472773..0b19e194e0e 100644 --- a/tests/integration/test_consistant_parts_after_move_partition/test.py +++ b/tests/integration/test_consistant_parts_after_move_partition/test.py @@ -18,7 +18,7 @@ def initialize_database(nodes, shard): CREATE TABLE `{database}`.dest (p UInt64, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/tables/test_consistent_shard2{shard}/replicated', '{replica}') ORDER BY d PARTITION BY p - SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0; + SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0, temporary_directories_lifetime=1; """.format( shard=shard, replica=node.name, database=CLICKHOUSE_DATABASE ) diff --git a/tests/integration/test_encrypted_disk/test.py b/tests/integration/test_encrypted_disk/test.py index 9f5415f4bea..fbf2b59785b 100644 --- a/tests/integration/test_encrypted_disk/test.py +++ b/tests/integration/test_encrypted_disk/test.py @@ -96,7 +96,7 @@ def test_part_move(policy, destination_disks): data String ) ENGINE=MergeTree() ORDER BY id - SETTINGS storage_policy='{}' + SETTINGS storage_policy='{}', temporary_directories_lifetime=1 """.format( policy ) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 8bf4df17c39..761b5257a34 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -66,6 +66,7 @@ def create_table(node, table_name, **additional_settings): "storage_policy": "blob_storage_policy", "old_parts_lifetime": 1, "index_granularity": 512, + "temporary_directories_lifetime": 1, } settings.update(additional_settings) diff --git a/tests/integration/test_merge_tree_hdfs/test.py b/tests/integration/test_merge_tree_hdfs/test.py index c79986c34f0..d1a145c00c1 100644 --- a/tests/integration/test_merge_tree_hdfs/test.py +++ b/tests/integration/test_merge_tree_hdfs/test.py @@ -29,7 +29,8 @@ def create_table(cluster, table_name, additional_settings=None): SETTINGS storage_policy='hdfs', old_parts_lifetime=0, - index_granularity=512 + index_granularity=512, + temporary_directories_lifetime=1 """.format( table_name ) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 2ccd517923a..3ab31f4728b 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -75,6 +75,7 @@ def create_table(node, table_name, **additional_settings): "storage_policy": "s3", "old_parts_lifetime": 0, "index_granularity": 512, + "temporary_directories_lifetime": 1, } settings.update(additional_settings) diff --git a/tests/integration/test_move_partition_to_disk_on_cluster/test.py b/tests/integration/test_move_partition_to_disk_on_cluster/test.py index 90753fc8ce3..c639e080cdf 100644 --- a/tests/integration/test_move_partition_to_disk_on_cluster/test.py +++ b/tests/integration/test_move_partition_to_disk_on_cluster/test.py @@ -46,7 +46,7 @@ def test_move_partition_to_disk_on_cluster(start_cluster): "(x UInt64) " "ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_local_table', '{replica}') " "ORDER BY tuple()" - "SETTINGS storage_policy = 'jbod_with_external';", + "SETTINGS storage_policy = 'jbod_with_external', temporary_directories_lifetime=1;", ) node1.query("INSERT INTO test_local_table VALUES (0)") diff --git a/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml b/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml index ef40bfb0a0e..e7a87fb77b1 100644 --- a/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml @@ -123,4 +123,8 @@ + + 1 + + diff --git a/tests/integration/test_rename_column/configs/config.d/storage_configuration.xml b/tests/integration/test_rename_column/configs/config.d/storage_configuration.xml index da297e40037..12a598c64b5 100644 --- a/tests/integration/test_rename_column/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_rename_column/configs/config.d/storage_configuration.xml @@ -24,9 +24,10 @@ - + 0 + 1 diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py index bd1c890950a..eb3d62eb718 100644 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py @@ -128,7 +128,7 @@ def test_hdfs_zero_copy_replication_single_move(cluster, storage_policy, init_ob CREATE TABLE single_node_move_test (dt DateTime, id Int64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/single_node_move_test', '{replica}') ORDER BY (dt, id) - SETTINGS storage_policy='$policy' + SETTINGS storage_policy='$policy',temporary_directories_lifetime=1 """ ).substitute(policy=storage_policy) ) diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 100f062de2f..bc13c127610 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -163,7 +163,7 @@ def test_s3_zero_copy_on_hybrid_storage(started_cluster): CREATE TABLE hybrid_test ON CLUSTER test_cluster (id UInt32, value String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/hybrid_test', '{}') ORDER BY id - SETTINGS storage_policy='hybrid' + SETTINGS storage_policy='hybrid',temporary_directories_lifetime=1 """.format( "{replica}" ) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 7635d784fef..a2f28e21666 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1549,7 +1549,7 @@ def test_double_move_while_select(started_cluster, name, positive): ) ENGINE = MergeTree ORDER BY tuple() PARTITION BY n - SETTINGS storage_policy='small_jbod_with_external' + SETTINGS storage_policy='small_jbod_with_external',temporary_directories_lifetime=1 """.format( name=name ) diff --git a/tests/integration/test_zero_copy_fetch/test.py b/tests/integration/test_zero_copy_fetch/test.py index 9b9aa5e0da7..4f3d42096c3 100644 --- a/tests/integration/test_zero_copy_fetch/test.py +++ b/tests/integration/test_zero_copy_fetch/test.py @@ -45,7 +45,7 @@ CREATE TABLE test1 (EventDate Date, CounterID UInt32) ENGINE = ReplicatedMergeTree('/clickhouse-tables/test1', 'r1') PARTITION BY toMonday(EventDate) ORDER BY (CounterID, EventDate) -SETTINGS index_granularity = 8192, storage_policy = 's3'""" +SETTINGS index_granularity = 8192, storage_policy = 's3', temporary_directories_lifetime=1""" ) node1.query( From 09fecace434aaeb1c54049f94a855a2843766145 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Jun 2023 17:43:08 +0000 Subject: [PATCH 1324/2223] upd --- src/Storages/MergeTree/MergeTreeData.cpp | 3 --- src/Storages/MergeTree/MergeTreePartsMover.cpp | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 047f063cb7c..0e542aa3407 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7898,9 +7898,6 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & catch (...) { write_part_log(ExecutionStatus::fromCurrentException("", true)); - if (cloned_part.part) - cloned_part.part->remove(); - throw; } } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 08815fa1f0c..2c3b3d1a621 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -255,10 +255,10 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me LOG_TRACE(log, "Part {} was cloned to {}", part->name, data_part_directory); + cloned_part.part->is_temp = true; cloned_part.part->loadColumnsChecksumsIndexes(true, true); cloned_part.part->loadVersionMetadata(); cloned_part.part->modification_time = cloned_part.part->getDataPartStorage().getLastModified().epochTime(); - cloned_part.part->is_temp = true; return cloned_part; } From 80f918d4b77cbad22aeb0371ac2f7881fe603550 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 5 Jun 2023 18:22:41 +0000 Subject: [PATCH 1325/2223] Fixes for cleanup moving parts --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 13 ++++++-- src/Storages/MergeTree/MergeTreeData.cpp | 4 +-- .../MergeTree/MergeTreePartsMover.cpp | 31 +++++++++---------- src/Storages/MergeTree/MergeTreePartsMover.h | 2 +- 4 files changed, 28 insertions(+), 22 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index d27b03fff44..9084b5790af 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -492,13 +492,17 @@ void IMergeTreeDataPart::removeIfNeeded() if (is_temp) { - String file_name = fileName(getDataPartStorage().getPartDirectory()); + const auto & part_directory = getDataPartStorage().getPartDirectory(); + + String file_name = fileName(part_directory); if (file_name.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "relative_path {} of part {} is invalid or not set", getDataPartStorage().getPartDirectory(), name); - if (!startsWith(file_name, "tmp") && !endsWith(file_name, ".tmp_proj")) + const auto part_parent_directory = directoryPath(part_directory); + bool is_moving_part = part_parent_directory.ends_with("moving/"); + if (!startsWith(file_name, "tmp") && !endsWith(file_name, ".tmp_proj") && !is_moving_part) { LOG_ERROR( storage.log, @@ -507,6 +511,11 @@ void IMergeTreeDataPart::removeIfNeeded() path); return; } + + if (is_moving_part) + { + LOG_TRACE(storage.log, "Removing unneeded moved part from {}", path); + } } remove(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0e542aa3407..7fe3efaf6d5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7864,7 +7864,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & if (lock->isLocked()) { cloned_part = parts_mover.clonePart(moving_part); - parts_mover.swapClonedPart(cloned_part.part); + parts_mover.swapClonedPart(cloned_part); break; } else if (wait_for_move_if_zero_copy) @@ -7891,7 +7891,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & else /// Ordinary move as it should be { cloned_part = parts_mover.clonePart(moving_part); - parts_mover.swapClonedPart(cloned_part.part); + parts_mover.swapClonedPart(cloned_part); } write_part_log({}); } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 2c3b3d1a621..656167de986 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -213,6 +213,8 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me auto part = moving_part.part; auto disk = moving_part.reserved_space->getDisk(); LOG_DEBUG(log, "Cloning part {} from '{}' to '{}'", part->name, part->getDataPartStorage().getDiskName(), disk->getName()); + TemporaryClonedPart cloned_part; + cloned_part.temporary_directory_lock = data->getTemporaryPartDirectoryHolder(part->name); MutableDataPartStoragePtr cloned_part_storage; if (disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication) @@ -243,17 +245,10 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me { cloned_part_storage = part->makeCloneOnDisk(disk, MergeTreeData::MOVING_DIR_NAME); } - String data_part_directory = cloned_part_storage->getFullPath(); - - TemporaryClonedPart cloned_part; - cloned_part.temporary_directory_lock = data->getTemporaryPartDirectoryHolder(data_part_directory); MergeTreeDataPartBuilder builder(*data, part->name, cloned_part_storage); cloned_part.part = std::move(builder).withPartFormatFromDisk().build(); - - String part_directory = cloned_part.part->getDataPartStorage().getFullPath(); - - LOG_TRACE(log, "Part {} was cloned to {}", part->name, data_part_directory); + LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part.part->getDataPartStorage().getFullPath()); cloned_part.part->is_temp = true; cloned_part.part->loadColumnsChecksumsIndexes(true, true); @@ -263,34 +258,36 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me } -void MergeTreePartsMover::swapClonedPart(const MergeTreeMutableDataPartPtr & cloned_part) const +void MergeTreePartsMover::swapClonedPart(TemporaryClonedPart & cloned_part) const { if (moves_blocker.isCancelled()) throw Exception(ErrorCodes::ABORTED, "Cancelled moving parts."); - auto active_part = data->getActiveContainingPart(cloned_part->name); + auto active_part = data->getActiveContainingPart(cloned_part.part->name); /// It's ok, because we don't block moving parts for merges or mutations - if (!active_part || active_part->name != cloned_part->name) + if (!active_part || active_part->name != cloned_part.part->name) { LOG_INFO(log, "Failed to swap {}. Active part doesn't exist (containing part {}). " "Possible it was merged or mutated. Will remove copy on path '{}'", - cloned_part->name, + cloned_part.part->name, active_part ? active_part->name : "doesn't exist", - cloned_part->getDataPartStorage().getFullPath()); + cloned_part.part->getDataPartStorage().getFullPath()); return; } - cloned_part->is_temp = false; + cloned_part.part->is_temp = false; /// Don't remove new directory but throw an error because it may contain part which is currently in use. - cloned_part->renameTo(active_part->name, false); + cloned_part.part->renameTo(active_part->name, false); /// TODO what happen if server goes down here? - data->swapActivePart(cloned_part); + data->swapActivePart(cloned_part.part); - LOG_TRACE(log, "Part {} was moved to {}", cloned_part->name, cloned_part->getDataPartStorage().getFullPath()); + LOG_TRACE(log, "Part {} was moved to {}", cloned_part.part->name, cloned_part.part->getDataPartStorage().getFullPath()); + + cloned_part.temporary_directory_lock = {}; } } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index dde2ff1a630..82fd271ee5f 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -72,7 +72,7 @@ public: /// IMergeTreeDataPart called. If replacing part doesn't exists or not active (committed) than /// cloned part will be removed and log message will be reported. It may happen in case of concurrent /// merge or mutation. - void swapClonedPart(const MergeTreeMutableDataPartPtr & cloned_parts) const; + void swapClonedPart(TemporaryClonedPart & cloned_part) const; /// Can stop background moves and moves from queries ActionBlocker moves_blocker; From b410a4d44ce6f1ac1048efb565f87bae0e97c183 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 5 Jun 2023 18:23:24 +0000 Subject: [PATCH 1326/2223] Add test test_alter_moving_garbage --- .../test_alter_moving_garbage/__init__.py | 0 .../configs/config.d/storage_conf.xml | 26 ++++++ .../configs/config.xml | 7 ++ .../test_alter_moving_garbage/test.py | 90 +++++++++++++++++++ 4 files changed, 123 insertions(+) create mode 100644 tests/integration/test_alter_moving_garbage/__init__.py create mode 100644 tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml create mode 100644 tests/integration/test_alter_moving_garbage/configs/config.xml create mode 100644 tests/integration/test_alter_moving_garbage/test.py diff --git a/tests/integration/test_alter_moving_garbage/__init__.py b/tests/integration/test_alter_moving_garbage/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml b/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml new file mode 100644 index 00000000000..659f59a41b2 --- /dev/null +++ b/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml @@ -0,0 +1,26 @@ + + + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + + + default + + + s3 + + + + + + diff --git a/tests/integration/test_alter_moving_garbage/configs/config.xml b/tests/integration/test_alter_moving_garbage/configs/config.xml new file mode 100644 index 00000000000..f4be5ab6b7c --- /dev/null +++ b/tests/integration/test_alter_moving_garbage/configs/config.xml @@ -0,0 +1,7 @@ + + 9000 + 127.0.0.1 + 500 + ./clickhouse/ + users.xml + diff --git a/tests/integration/test_alter_moving_garbage/test.py b/tests/integration/test_alter_moving_garbage/test.py new file mode 100644 index 00000000000..b369c9ad377 --- /dev/null +++ b/tests/integration/test_alter_moving_garbage/test.py @@ -0,0 +1,90 @@ +import logging +import time + +import pytest +import threading + +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node1", + main_configs=[ + "configs/config.d/storage_conf.xml", + ], + with_minio=True, + ) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def create_table(node, table_name, **additional_settings): + settings = { + "storage_policy": "two_disks", + "old_parts_lifetime": 1, + "index_granularity": 512, + "temporary_directories_lifetime": 0, + "merge_tree_clear_old_temporary_directories_interval_seconds": 1, + } + settings.update(additional_settings) + + create_table_statement = f""" + CREATE TABLE {table_name} ( + dt Date, + id Int64, + data String, + INDEX min_max (id) TYPE minmax GRANULARITY 3 + ) ENGINE=MergeTree() + PARTITION BY dt + ORDER BY (dt, id) + SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))}""" + + node.query(create_table_statement) + + +def test_create_table(cluster): + node = cluster.instances["node1"] + create_table(node, "test_table") + node.query( + "INSERT INTO test_table SELECT toDate('2021-01-01') + INTERVAL number % 10 DAY, number, toString(sipHash64(number)) FROM numbers(100_000)" + ) + + stop_alter = False + + def alter(): + d = 0 + node.query(f"ALTER TABLE test_table ADD COLUMN col0 String") + while not stop_alter: + d = d + 1 + node.query(f"DELETE FROM test_table WHERE id < {d}") + time.sleep(0.1) + + alter_thread = threading.Thread(target=alter) + alter_thread.start() + + for i in range(1, 10): + partition = f"2021-01-{i:02d}" + try: + node.query( + f"ALTER TABLE test_table MOVE PARTITION '{partition}' TO DISK 's3'", + ) + except QueryRuntimeException as e: + # PART_IS_TEMPORARILY_LOCKED + assert 384 == e.returncode + continue + + # clear old temporary directories wakes up every 1 second + time.sleep(0.5) + + stop_alter = True + alter_thread.join() From ea1aa4bd9e312a36b578fff3ec3573ff0844d9d5 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 7 Jun 2023 16:02:16 +0200 Subject: [PATCH 1327/2223] update comment --- src/Client/Suggest.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index e249aa1bb04..4e38add0ef5 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -101,7 +101,7 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti add_column("name", "columns", true, suggestion_limit); } - /// FIXME: Forbid this query using new analyzer because of bug https://github.com/ClickHouse/ClickHouse/pull/50430#issuecomment-1576860893 + /// FIXME: Forbid this query using new analyzer because of bug https://github.com/ClickHouse/ClickHouse/issues/50669 /// We should remove this restriction after resolving this bug. query = "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" + query + ") WHERE notEmpty(res) SETTINGS allow_experimental_analyzer=0"; return query; From 4fd64a28b20966246743ec6408a3018e8724249e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 7 Jun 2023 16:07:18 +0200 Subject: [PATCH 1328/2223] and add more tests --- .../01763_filter_push_down_bugs.reference | 23 +++++++++++++++---- .../01763_filter_push_down_bugs.sql | 6 +++++ 2 files changed, 25 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 7df35e2948d..db9cd7a2d16 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -9,13 +9,11 @@ String1_0 String2_0 String3_0 String4_0 1 Expression ((Projection + Before ORDER BY)) Filter (WHERE) Join (JOIN FillRightFirst) - Filter (( + Before JOIN)) + Expression (Before JOIN) ReadFromMergeTree (default.t1) Indexes: PrimaryKey - Keys: - id - Condition: (id in [101, 101]) + Condition: true Parts: 1/1 Granules: 1/1 Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) @@ -25,3 +23,20 @@ Expression ((Projection + Before ORDER BY)) Condition: true Parts: 1/1 Granules: 1/1 +Expression ((Project names + Projection)) + Filter ((WHERE + DROP unused columns after JOIN)) + Join (JOIN FillRightFirst) + Expression (Change column names to column identifiers) + ReadFromMergeTree (default.t1) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Expression ((Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) + ReadFromMergeTree (default.t2) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 5f7f4379714..9a5ef4727c5 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -58,5 +58,11 @@ EXPLAIN indexes=1 SELECT id, delete_time FROM t1 FROM t2 ) AS d WHERE create_time < delete_time AND id = 101 SETTINGS allow_experimental_analyzer=0; +EXPLAIN indexes=1 SELECT id, delete_time FROM t1 + CROSS JOIN ( + SELECT delete_time + FROM t2 +) AS d WHERE create_time < delete_time AND id = 101 SETTINGS allow_experimental_analyzer=1; + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From 0dd75d7648d3ba12b9593b312ce428e1b12799f8 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 7 Jun 2023 17:50:20 +0300 Subject: [PATCH 1329/2223] Add 02783_parseDateTimeBestEffort_syslog test --- ...3_parseDateTimeBestEffort_syslog.reference | 20 +++++ .../02783_parseDateTimeBestEffort_syslog.sql | 83 +++++++++++++++++++ 2 files changed, 103 insertions(+) create mode 100644 tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference create mode 100644 tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference new file mode 100644 index 00000000000..7409b413260 --- /dev/null +++ b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference @@ -0,0 +1,20 @@ +parseDateTimeBestEffort + dt_ref res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc + + Jun 7 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 + Jun 7 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 +parseDateTimeBestEffortUS + dt_ref res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc + + Jun 7 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 + Jun 7 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 +parseDateTime64BestEffort + dt_ref res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc + + Jun 7 04:55:00 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 + Jun 7 04:56:00 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 +parseDateTime64BestEffortUS + dt_ref res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc + + Jun 7 04:55:00 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 + Jun 7 04:56:00 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql new file mode 100644 index 00000000000..91ae230205b --- /dev/null +++ b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql @@ -0,0 +1,83 @@ +SELECT 'parseDateTimeBestEffort'; + +WITH + now() AS ts_now, + '2023-06-07 04:55:30' AS ref_point, + dateDiff('second', toDateTime(ref_point), ts_now) AS impedimenta, + formatDateTime(ts_around, '%b %e %T') AS dt_curr +SELECT + formatDateTime(ts_around - impedimenta, '%b %e %H:%i:%s') AS dt_ref, + parseDateTimeBestEffort(dt_curr) - impedimenta AS res, + parseDateTimeBestEffort(dt_curr, 'US/Samoa') - impedimenta AS res_sam, + parseDateTimeBestEffort(dt_curr, 'Pacific/Auckland') - impedimenta AS res_auc, + parseDateTimeBestEffortOrNull(dt_curr) - impedimenta AS res_null, + parseDateTimeBestEffortOrNull(dt_curr, 'US/Samoa') - impedimenta AS res_null_sam, + parseDateTimeBestEffortOrNull(dt_curr, 'Pacific/Auckland') - impedimenta AS res_null_auc, + parseDateTimeBestEffortOrZero(dt_curr) - impedimenta AS res_zero, + parseDateTimeBestEffortOrZero(dt_curr, 'US/Samoa') - impedimenta AS res_zero_sam, + parseDateTimeBestEffortOrZero(dt_curr, 'Pacific/Auckland') - impedimenta AS res_zero_auc +FROM (SELECT arrayJoin([ts_now - 30, ts_now + 30]) AS ts_around) +FORMAT PrettySpaceNoEscapes; + +SELECT 'parseDateTimeBestEffortUS'; + +WITH + now() AS ts_now, + '2023-06-07 04:55:30' AS ref_point, + dateDiff('second', toDateTime(ref_point), ts_now) AS impedimenta, + formatDateTime(ts_around, '%b %e %T') AS dt_curr +SELECT + formatDateTime(ts_around - impedimenta, '%b %e %H:%i:%s') AS dt_ref, + parseDateTimeBestEffortUS(dt_curr) - impedimenta AS res, + parseDateTimeBestEffortUS(dt_curr, 'US/Samoa') - impedimenta AS res_sam, + parseDateTimeBestEffortUS(dt_curr, 'Pacific/Auckland') - impedimenta AS res_auc, + parseDateTimeBestEffortUSOrNull(dt_curr) - impedimenta AS res_null, + parseDateTimeBestEffortUSOrNull(dt_curr, 'US/Samoa') - impedimenta AS res_null_sam, + parseDateTimeBestEffortUSOrNull(dt_curr, 'Pacific/Auckland') - impedimenta AS res_null_auc, + parseDateTimeBestEffortUSOrZero(dt_curr) - impedimenta AS res_zero, + parseDateTimeBestEffortUSOrZero(dt_curr, 'US/Samoa') - impedimenta AS res_zero_sam, + parseDateTimeBestEffortUSOrZero(dt_curr, 'Pacific/Auckland') - impedimenta AS res_zero_auc +FROM (SELECT arrayJoin([ts_now - 30, ts_now + 30]) AS ts_around) +FORMAT PrettySpaceNoEscapes; + +SELECT 'parseDateTime64BestEffort'; + +WITH + now() AS ts_now, + '2023-06-07 04:55:30' AS ref_point, + dateDiff('second', toDateTime(ref_point), ts_now) AS impedimenta, + formatDateTime(ts_around, '%b %e %T') AS dt_curr +SELECT + formatDateTime(ts_around - impedimenta, '%b %e %H:%i:%s') AS dt_ref, + parseDateTime64BestEffort(dt_curr) - impedimenta AS res, + parseDateTime64BestEffort(dt_curr, 3, 'US/Samoa') - impedimenta AS res_sam, + parseDateTime64BestEffort(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_auc, + parseDateTime64BestEffortOrNull(dt_curr) - impedimenta AS res_null, + parseDateTime64BestEffortOrNull(dt_curr, 3, 'US/Samoa') - impedimenta AS res_null_sam, + parseDateTime64BestEffortOrNull(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_null_auc, + parseDateTime64BestEffortOrZero(dt_curr) - impedimenta AS res_zero, + parseDateTime64BestEffortOrZero(dt_curr, 3, 'US/Samoa') - impedimenta AS res_zero_sam, + parseDateTime64BestEffortOrZero(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_zero_auc +FROM (SELECT arrayJoin([ts_now - 30, ts_now + 30]) AS ts_around) +FORMAT PrettySpaceNoEscapes; + +SELECT 'parseDateTime64BestEffortUS'; + +WITH + now() AS ts_now, + '2023-06-07 04:55:30' AS ref_point, + dateDiff('second', toDateTime(ref_point), ts_now) AS impedimenta, + formatDateTime(ts_around, '%b %e %T') AS dt_curr +SELECT + formatDateTime(ts_around - impedimenta, '%b %e %H:%i:%s') AS dt_ref, + parseDateTime64BestEffortUS(dt_curr) - impedimenta AS res, + parseDateTime64BestEffortUS(dt_curr, 3, 'US/Samoa') - impedimenta AS res_sam, + parseDateTime64BestEffortUS(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_auc, + parseDateTime64BestEffortUSOrNull(dt_curr) - impedimenta AS res_null, + parseDateTime64BestEffortUSOrNull(dt_curr, 3, 'US/Samoa') - impedimenta AS res_null_sam, + parseDateTime64BestEffortUSOrNull(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_null_auc, + parseDateTime64BestEffortUSOrZero(dt_curr) - impedimenta AS res_zero, + parseDateTime64BestEffortUSOrZero(dt_curr, 3, 'US/Samoa') - impedimenta AS res_zero_sam, + parseDateTime64BestEffortUSOrZero(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_zero_auc +FROM (SELECT arrayJoin([ts_now - 30, ts_now + 30]) AS ts_around) +FORMAT PrettySpaceNoEscapes; From bf6900f64ca7614a686dfaa56f87c84c43408506 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 7 Jun 2023 17:08:18 +0200 Subject: [PATCH 1330/2223] Write 1 part and do not use OPTIMIZE FINAL --- ...e_row_level_policy_lightweight_delete.sql.j2 | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02461_prewhere_row_level_policy_lightweight_delete.sql.j2 b/tests/queries/0_stateless/02461_prewhere_row_level_policy_lightweight_delete.sql.j2 index 1e4258cef7e..0ec6b2ed144 100644 --- a/tests/queries/0_stateless/02461_prewhere_row_level_policy_lightweight_delete.sql.j2 +++ b/tests/queries/0_stateless/02461_prewhere_row_level_policy_lightweight_delete.sql.j2 @@ -8,25 +8,10 @@ ORDER BY (SiteId, DateVisit) SETTINGS index_granularity = {{ index_granularity }}, min_bytes_for_wide_part = 0; -- Insert some data to have 110K rows in the range 2022-08-10 .. 2022-08-20 and some more rows before and after that range -insert into url_na_log select 209, '2022-08-09' from numbers(10000); -insert into url_na_log select 209, '2022-08-10' from numbers(10000); -insert into url_na_log select 209, '2022-08-11' from numbers(10000); -insert into url_na_log select 209, '2022-08-12' from numbers(10000); -insert into url_na_log select 209, '2022-08-13' from numbers(10000); -insert into url_na_log select 209, '2022-08-14' from numbers(10000); -insert into url_na_log select 209, '2022-08-15' from numbers(10000); -insert into url_na_log select 209, '2022-08-16' from numbers(10000); -insert into url_na_log select 209, '2022-08-17' from numbers(10000); -insert into url_na_log select 209, '2022-08-18' from numbers(10000); -insert into url_na_log select 209, '2022-08-19' from numbers(10000); -insert into url_na_log select 209, '2022-08-20' from numbers(10000); -insert into url_na_log select 209, '2022-08-21' from numbers(10000); - +insert into url_na_log select 209, ('2022-08-09'::Date + INTERVAL intDiv(number,10000) DAY) from numbers(130000) SETTINGS max_insert_block_size=200000; SET mutations_sync=2; -OPTIMIZE TABLE url_na_log FINAL; - -- { echoOn } SELECT count() FROM url_na_log; From a67dd6e47947295bd853ee1d26ad66a94861dabe Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 7 Jun 2023 17:25:48 +0200 Subject: [PATCH 1331/2223] Readuntilend --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 16 ++++++++++++++++ src/Disks/IO/ReadBufferFromAzureBlobStorage.h | 1 + 2 files changed, 17 insertions(+) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index a086eb0a6df..129bb97be09 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -56,6 +56,22 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( } } + +void ReadBufferFromAzureBlobStorage::setReadUntilEnd() +{ + if (read_until_position) + { + read_until_position = 0; + if (initialized) + { + offset = getPosition(); + resetWorkingBuffer(); + initialized = false; + } + } + +} + void ReadBufferFromAzureBlobStorage::setReadUntilPosition(size_t position) { read_until_position = position; diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h index 599ecba1dd1..4e21f543653 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h @@ -38,6 +38,7 @@ public: String getFileName() const override { return path; } void setReadUntilPosition(size_t position) override; + void setReadUntilEnd() override; bool supportsRightBoundedReads() const override { return true; } From cf65ac499c8f8cc8e49fa9654bc45723a99d3a6d Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 7 Jun 2023 17:34:26 +0200 Subject: [PATCH 1332/2223] Fix iterator --- src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index fd6452b7c2a..f91c19f2fb9 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -25,7 +25,6 @@ void IObjectStorageIteratorAsync::nextBatch() current_batch = std::move(next_batch.batch); accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); current_batch_iterator = current_batch.begin(); - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "HAS NEXT {}", next_batch.has_next); if (next_batch.has_next) outcome_future = scheduleBatch(); else @@ -78,6 +77,7 @@ bool IObjectStorageIteratorAsync::isValid() if (!is_initialized) nextBatch(); + std::lock_guard lock(mutex); return current_batch_iterator != current_batch.end(); } @@ -86,16 +86,17 @@ RelativePathWithMetadata IObjectStorageIteratorAsync::current() if (!isValid()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); + std::lock_guard lock(mutex); return *current_batch_iterator; } RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch() { - std::lock_guard lock(mutex); if (!isValid()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); + std::lock_guard lock(mutex); return current_batch; } From b567dc2a1dce59d3ced34463601e63326d56aa50 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 7 Jun 2023 17:48:06 +0200 Subject: [PATCH 1333/2223] fix test --- .../0_stateless/01763_filter_push_down_bugs.reference | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index db9cd7a2d16..c8045dd26f5 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -9,11 +9,13 @@ String1_0 String2_0 String3_0 String4_0 1 Expression ((Projection + Before ORDER BY)) Filter (WHERE) Join (JOIN FillRightFirst) - Expression (Before JOIN) + Filter (( + Before JOIN)) ReadFromMergeTree (default.t1) Indexes: PrimaryKey - Condition: true + Keys: + id + Condition: (id in [101, 101]) Parts: 1/1 Granules: 1/1 Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) @@ -30,7 +32,9 @@ Expression ((Project names + Projection)) ReadFromMergeTree (default.t1) Indexes: PrimaryKey - Condition: true + Keys: + id + Condition: (id in [101, 101]) Parts: 1/1 Granules: 1/1 Expression ((Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) From b78e330129575d9b77f9da5cb6511eee56d5deaf Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 7 Jun 2023 18:09:55 +0200 Subject: [PATCH 1334/2223] Better test --- .../integration/test_storage_azure_blob_storage/test.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 8a0a68f5200..319500e6226 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -126,6 +126,7 @@ def test_simple_write_named_collection_1(cluster): ) print(get_azure_file_content("test_simple_write_named.csv")) assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' + azure_query(node, "TRUNCATE TABLE test_simple_write_named_collection_1") def test_simple_write_named_collection_2(cluster): @@ -487,6 +488,14 @@ def test_simple_write_named_collection_1_table_function(cluster): print(get_azure_file_content("test_simple_write_named.csv")) assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' + azure_query( + node, + "CREATE TABLE drop_table (key UInt64, data String) Engine = AzureBlobStorage(azure_conf1)", + ) + + azure_query( + node, "TRUNCATE TABLE drop_table", + ) def test_simple_write_named_collection_2_table_function(cluster): node = cluster.instances["node"] From 8d67296e3aee48f92ec82fd55637dfe16122c02d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 7 Jun 2023 16:22:42 +0000 Subject: [PATCH 1335/2223] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 319500e6226..2c052c005ca 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -494,9 +494,11 @@ def test_simple_write_named_collection_1_table_function(cluster): ) azure_query( - node, "TRUNCATE TABLE drop_table", + node, + "TRUNCATE TABLE drop_table", ) + def test_simple_write_named_collection_2_table_function(cluster): node = cluster.instances["node"] From c12c62a7fb7072772f423c907c467c39762ec5a7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Jun 2023 18:22:53 +0200 Subject: [PATCH 1336/2223] Update src/Parsers/ParserCreateIndexQuery.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Parsers/ParserCreateIndexQuery.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index e878b347e62..7323c5da141 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -42,9 +42,6 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected return false; } - if (!granularity_p.parse(pos, granularity, expected)) - return false; - auto index = std::make_shared(); index->part_of_create_index_query = true; index->set(index->expr, expr); From 2df0e0c66962ce91e47eb4c98bae4fabe1ce9cc1 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 7 Jun 2023 16:25:52 +0000 Subject: [PATCH 1337/2223] Unify priorities for connections --- src/Client/ConnectionPool.cpp | 4 ++-- src/Client/ConnectionPool.h | 13 +++++++------ src/Client/ConnectionPoolWithFailover.cpp | 2 +- src/Client/ConnectionPoolWithFailover.h | 2 +- src/Common/GetPriorityForLoadBalancing.cpp | 15 ++++++++------- src/Common/GetPriorityForLoadBalancing.h | 2 +- src/Common/PoolWithFailoverBase.h | 9 +++++---- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Databases/DatabaseReplicated.cpp | 2 +- src/Functions/hasColumnInTable.cpp | 2 +- .../tests/gtest_resource_manager_static.cpp | 4 ++-- src/Interpreters/Cluster.cpp | 2 +- src/Interpreters/Cluster.h | 5 +++-- src/Interpreters/ClusterDiscovery.cpp | 2 +- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 16 files changed, 37 insertions(+), 33 deletions(-) diff --git a/src/Client/ConnectionPool.cpp b/src/Client/ConnectionPool.cpp index 8433b0833fa..5cabb1465d1 100644 --- a/src/Client/ConnectionPool.cpp +++ b/src/Client/ConnectionPool.cpp @@ -18,7 +18,7 @@ ConnectionPoolPtr ConnectionPoolFactory::get( String client_name, Protocol::Compression compression, Protocol::Secure secure, - Int64 priority) + Priority priority) { Key key{ max_connections, host, port, default_database, user, password, quota_key, cluster, cluster_secret, client_name, compression, secure, priority}; @@ -74,7 +74,7 @@ size_t ConnectionPoolFactory::KeyHash::operator()(const ConnectionPoolFactory::K hash_combine(seed, hash_value(k.client_name)); hash_combine(seed, hash_value(k.compression)); hash_combine(seed, hash_value(k.secure)); - hash_combine(seed, hash_value(k.priority)); + hash_combine(seed, hash_value(k.priority.value)); return seed; } diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index aacd0a063c7..b6d03daacfb 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -34,7 +35,7 @@ public: const Settings * settings = nullptr, bool force_connected = true) = 0; - virtual Int64 getPriority() const { return 1; } + virtual Priority getPriority() const { return Priority{1}; } }; using ConnectionPoolPtr = std::shared_ptr; @@ -60,7 +61,7 @@ public: const String & client_name_, Protocol::Compression compression_, Protocol::Secure secure_, - Int64 priority_ = 1) + Priority priority_ = Priority{1}) : Base(max_connections_, &Poco::Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")), host(host_), @@ -103,7 +104,7 @@ public: return host + ":" + toString(port); } - Int64 getPriority() const override + Priority getPriority() const override { return priority; } @@ -134,7 +135,7 @@ private: String client_name; Protocol::Compression compression; /// Whether to compress data when interacting with the server. Protocol::Secure secure; /// Whether to encrypt data when interacting with the server. - Int64 priority; /// priority from + Priority priority; /// priority from }; /** @@ -157,7 +158,7 @@ public: String client_name; Protocol::Compression compression; Protocol::Secure secure; - Int64 priority; + Priority priority; }; struct KeyHash @@ -180,7 +181,7 @@ public: String client_name, Protocol::Compression compression, Protocol::Secure secure, - Int64 priority); + Priority priority); private: mutable std::mutex mutex; using ConnectionPoolWeakPtr = std::weak_ptr; diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 129bc10bc27..feb4c01c374 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -71,7 +71,7 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts return Base::get(max_ignored_errors, fallback_to_stale_replicas, try_get_entry, get_priority); } -Int64 ConnectionPoolWithFailover::getPriority() const +Priority ConnectionPoolWithFailover::getPriority() const { return (*std::max_element(nested_pools.begin(), nested_pools.end(), [](const auto & a, const auto & b) { diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 0273ce41589..75a0dafd977 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -48,7 +48,7 @@ public: const Settings * settings, bool force_connected) override; /// From IConnectionPool - Int64 getPriority() const override; /// From IConnectionPool + Priority getPriority() const override; /// From IConnectionPool /** Allocates up to the specified number of connections to work. * Connections provide access to different replicas of one shard. diff --git a/src/Common/GetPriorityForLoadBalancing.cpp b/src/Common/GetPriorityForLoadBalancing.cpp index 5da60fb1bae..c4d36acc70c 100644 --- a/src/Common/GetPriorityForLoadBalancing.cpp +++ b/src/Common/GetPriorityForLoadBalancing.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -8,23 +9,23 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -std::function GetPriorityForLoadBalancing::getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const +std::function GetPriorityForLoadBalancing::getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const { - std::function get_priority; + std::function get_priority; switch (load_balance) { case LoadBalancing::NEAREST_HOSTNAME: if (hostname_differences.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: hostname_differences is not initialized"); - get_priority = [this](size_t i) { return hostname_differences[i]; }; + get_priority = [this](size_t i) { return Priority{static_cast(hostname_differences[i])}; }; break; case LoadBalancing::IN_ORDER: - get_priority = [](size_t i) { return i; }; + get_priority = [](size_t i) { return Priority{static_cast(i)}; }; break; case LoadBalancing::RANDOM: break; case LoadBalancing::FIRST_OR_RANDOM: - get_priority = [offset](size_t i) -> size_t { return i != offset; }; + get_priority = [offset](size_t i) { return i != offset ? Priority{1} : Priority{0}; }; break; case LoadBalancing::ROUND_ROBIN: if (last_used >= pool_size) @@ -38,8 +39,8 @@ std::function GetPriorityForLoadBalancing::getPriorityFunc * */ get_priority = [this, pool_size](size_t i) { - ++i; - return i < last_used ? pool_size - i : i - last_used; + ++i; // To make `i` indexing start with 1 instead of 0 as `last_used` does + return Priority{static_cast(i < last_used ? pool_size - i : i - last_used)}; }; break; } diff --git a/src/Common/GetPriorityForLoadBalancing.h b/src/Common/GetPriorityForLoadBalancing.h index e57b02b5e90..8052185ac13 100644 --- a/src/Common/GetPriorityForLoadBalancing.h +++ b/src/Common/GetPriorityForLoadBalancing.h @@ -21,7 +21,7 @@ public: return !(*this == other); } - std::function getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const; + std::function getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const; std::vector hostname_differences; /// Distances from name of this host to the names of hosts of pools. diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 646e10d6443..c6f44a7701a 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -34,7 +35,7 @@ namespace ProfileEvents /// This class provides a pool with fault tolerance. It is used for pooling of connections to replicated DB. /// Initialized by several PoolBase objects. /// When a connection is requested, tries to create or choose an alive connection from one of the nested pools. -/// Pools are tried in the order consistent with lexicographical order of (error count, priority, random number) tuples. +/// Pools are tried in the order consistent with lexicographical order of (error count, slowdown count, config priority, priority, random number) tuples. /// Number of tries for a single pool is limited by max_tries parameter. /// The client can set nested pool priority by passing a GetPriority functor. /// @@ -113,7 +114,7 @@ public: /// The client can provide this functor to affect load balancing - the index of a pool is passed to /// this functor. The pools with lower result value will be tried first. - using GetPriorityFunc = std::function; + using GetPriorityFunc = std::function; /// Returns at least min_entries and at most max_entries connections (at most one connection per nested pool). /// The method will throw if it is unable to get min_entries alive connections or @@ -336,9 +337,9 @@ struct PoolWithFailoverBase::PoolState /// The number of slowdowns that led to changing replica in HedgedRequestsFactory UInt64 slowdown_count = 0; /// Priority from the configuration. - Int64 config_priority = 1; + Priority config_priority{1}; /// Priority from the GetPriorityFunc. - Int64 priority = 0; + Priority priority{0}; UInt64 random = 0; void randomize() diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index a587ad6caf4..62807fe2433 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -179,7 +179,7 @@ ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std std::vector ZooKeeper::shuffleHosts() const { - std::function get_priority = args.get_priority_load_balancing.getPriorityFunc(args.get_priority_load_balancing.load_balancing, 0, args.hosts.size()); + std::function get_priority = args.get_priority_load_balancing.getPriorityFunc(args.get_priority_load_balancing.load_balancing, 0, args.hosts.size()); std::vector shuffle_hosts; for (size_t i = 0; i < args.hosts.size(); ++i) { diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 96f9914b597..d48ca0a4ef5 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -49,7 +49,7 @@ constexpr size_t MULTI_BATCH_SIZE = 100; struct ShuffleHost { String host; - Int64 priority = 0; + Priority priority; UInt64 random = 0; void randomize() diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 9bbf5b9565d..583607bda1d 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -252,7 +252,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const treat_local_as_remote, treat_local_port_as_remote, cluster_auth_info.cluster_secure_connection, - /*priority=*/ 1, + Priority{1}, TSA_SUPPRESS_WARNING_FOR_READ(database_name), /// FIXME cluster_auth_info.cluster_secret}; diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index 4676b4083b7..66ed515e490 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -137,7 +137,7 @@ ColumnPtr FunctionHasColumnInTable::executeImpl(const ColumnsWithTypeAndName & a treat_local_as_remote, treat_local_port_as_remote, /* secure= */ false, - /* priority= */ 1, + /* priority= */ Priority{1}, /* cluster_name= */ "", /* password= */ "" }; diff --git a/src/IO/Resource/tests/gtest_resource_manager_static.cpp b/src/IO/Resource/tests/gtest_resource_manager_static.cpp index 091f6923714..976eac41a49 100644 --- a/src/IO/Resource/tests/gtest_resource_manager_static.cpp +++ b/src/IO/Resource/tests/gtest_resource_manager_static.cpp @@ -44,8 +44,8 @@ TEST(IOResourceStaticResourceManager, Smoke) TEST(IOResourceStaticResourceManager, Prioritization) { - std::optional last_priority; - auto check = [&] (Int64 priority) + std::optional last_priority; + auto check = [&] (Priority priority) { // Lock is not required here because this is called during request execution and we have max_requests = 1 if (last_priority) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 8c30dbe230f..edbef77ef02 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -108,7 +108,7 @@ Cluster::Address::Address( password = config.getString(config_prefix + ".password", ""); default_database = config.getString(config_prefix + ".default_database", ""); secure = ConfigHelper::getBool(config, config_prefix + ".secure", false, /* empty_as */true) ? Protocol::Secure::Enable : Protocol::Secure::Disable; - priority = config.getInt(config_prefix + ".priority", 1); + priority = Priority{config.getInt(config_prefix + ".priority", 1)}; const char * port_type = secure == Protocol::Secure::Enable ? "tcp_port_secure" : "tcp_port"; auto default_port = config.getInt(port_type, 0); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index 4798384f29c..de10a445d01 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -44,7 +45,7 @@ struct ClusterConnectionParameters bool treat_local_as_remote; bool treat_local_port_as_remote; bool secure = false; - Int64 priority = 1; + Priority priority{1}; String cluster_name; String cluster_secret; }; @@ -131,7 +132,7 @@ public: Protocol::Compression compression = Protocol::Compression::Enable; Protocol::Secure secure = Protocol::Secure::Disable; - Int64 priority = 1; + Priority priority{1}; Address() = default; diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 884e3b87343..553488edf50 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -246,7 +246,7 @@ ClusterPtr ClusterDiscovery::makeCluster(const ClusterInfo & cluster_info) /* treat_local_as_remote= */ false, /* treat_local_port_as_remote= */ false, /// should be set only for clickhouse-local, but cluster discovery is not used there /* secure= */ secure, - /* priority= */ 1, + /* priority= */ Priority{1}, /* cluster_name= */ "", /* password= */ ""}; auto cluster = std::make_shared( diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index b2f09adf773..4143014a7b3 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -262,7 +262,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr treat_local_as_remote, treat_local_port_as_remote, secure, - /* priority= */ 1, + /* priority= */ Priority{1}, /* cluster_name= */ "", /* password= */ "" }; From 32372967e9814e629cbad2ce2ff57f82aba86e97 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 7 Jun 2023 16:55:14 +0200 Subject: [PATCH 1338/2223] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++------- src/Storages/StorageReplicatedMergeTree.h | 2 +- .../0_stateless/02432_s3_parallel_parts_cleanup.sql | 6 +----- .../0_stateless/02448_clone_replica_lost_part.sql | 9 ++++++--- 4 files changed, 11 insertions(+), 16 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6edd7531ec1..36bc3476e91 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6743,14 +6743,12 @@ size_t StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() if (parts.empty()) return total_parts_to_remove; - size_t res = 0; - NOEXCEPT_SCOPE({ res = clearOldPartsAndRemoveFromZKImpl(zookeeper, std::move(parts)); }); - return res; + NOEXCEPT_SCOPE({ clearOldPartsAndRemoveFromZKImpl(zookeeper, std::move(parts)); }); + return total_parts_to_remove; } -size_t StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZKImpl(zkutil::ZooKeeperPtr zookeeper, DataPartsVector && parts) +void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZKImpl(zkutil::ZooKeeperPtr zookeeper, DataPartsVector && parts) { - DataPartsVector parts_to_delete_only_from_filesystem; // Only duplicates DataPartsVector parts_to_delete_completely; // All parts except duplicates DataPartsVector parts_to_retry_deletion; // Parts that should be retried due to network problems @@ -6861,8 +6859,6 @@ size_t StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZKImpl(zkutil::ZooK /// Otherwise nobody will try to remove them again (see grabOldParts). delete_parts_from_fs_and_rollback_in_case_of_error(parts_to_remove_from_filesystem, "old"); } - - return total_parts_to_remove; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 368d7d1b948..290266ca00c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -344,7 +344,7 @@ private: /// Delete old parts from disk and from ZooKeeper. Returns the number of removed parts size_t clearOldPartsAndRemoveFromZK(); - size_t clearOldPartsAndRemoveFromZKImpl(zkutil::ZooKeeperPtr zookeeper, DataPartsVector && parts); + void clearOldPartsAndRemoveFromZKImpl(zkutil::ZooKeeperPtr zookeeper, DataPartsVector && parts); template friend class ReplicatedMergeTreeSinkImpl; diff --git a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql index 3f8aa545298..948ec9e9e8a 100644 --- a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql +++ b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql @@ -38,11 +38,7 @@ select count(), sum(n), sum(m) from rmt; -- New table can assign merges/mutations and can remove old parts create table rmt2 (n int, m int, k String) engine=ReplicatedMergeTree('/test/02432/{database}', '2') order by tuple() settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1, -<<<<<<< HEAD - max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0, -======= - concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, ->>>>>>> master + concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0, min_bytes_for_wide_part=0, min_rows_for_wide_part=0, max_replicated_merges_in_queue=1, old_parts_lifetime=0; diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index 1e99e1869cc..eb4d0f255a7 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -7,11 +7,11 @@ drop table if exists rmt2; create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '1') order by tuple() settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime=0, max_parts_to_merge_at_once=4, - merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=500; + merge_selecting_sleep_ms=1000, max_merge_selecting_sleep_ms=2000; create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '2') order by tuple() settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0, old_parts_lifetime=0, max_parts_to_merge_at_once=4, - merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=500; + merge_selecting_sleep_ms=1000, max_merge_selecting_sleep_ms=2000; -- insert part only on one replica system stop replicated sends rmt1; @@ -141,7 +141,10 @@ system sync replica rmt2; -- merge through gap optimize table rmt2; -- give it a chance to cleanup log -select sleep(2) format Null; -- increases probability of reproducing the issue + +select sleepEachRow(2) from url('http://localhost:8123/?param_tries={1..10}&query=' || encodeURLComponent( + 'select value from system.zookeeper where path=''//test/02448/' || currentDatabase() || '/rmt/replicas/1/is_lost'' and value=''1''' + ), 'LineAsString', 's String') settings max_threads=1 format Null; -- rmt1 will mimic rmt2, but will not be able to fetch parts for a while system stop replicated sends rmt2; From 3cdbccd9ea2eefa230d70fb23506fcbe5c01148b Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 7 Jun 2023 16:50:04 +0000 Subject: [PATCH 1339/2223] Unify priorities: `IExecutableTask`s --- src/Storages/MergeTree/IExecutableTask.h | 5 +++-- src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergeFromLogEntryTask.h | 4 ++-- src/Storages/MergeTree/MergePlainMergeTreeTask.h | 6 +++--- src/Storages/MergeTree/MergeTreeBackgroundExecutor.h | 2 +- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MutateFromLogEntryTask.h | 4 ++-- src/Storages/MergeTree/MutatePlainMergeTreeTask.h | 6 +++--- src/Storages/MergeTree/MutateTask.cpp | 7 ++++--- src/Storages/MergeTree/tests/gtest_executor.cpp | 10 +++++----- 10 files changed, 25 insertions(+), 23 deletions(-) diff --git a/src/Storages/MergeTree/IExecutableTask.h b/src/Storages/MergeTree/IExecutableTask.h index 9617960c182..d0c2d4a840e 100644 --- a/src/Storages/MergeTree/IExecutableTask.h +++ b/src/Storages/MergeTree/IExecutableTask.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB { @@ -32,7 +33,7 @@ public: virtual bool executeStep() = 0; virtual void onCompleted() = 0; virtual StorageID getStorageID() = 0; - virtual UInt64 getPriority() = 0; + virtual Priority getPriority() = 0; virtual ~IExecutableTask() = default; }; @@ -63,7 +64,7 @@ public: void onCompleted() override { job_result_callback(!res); } StorageID getStorageID() override { return id; } - UInt64 getPriority() override + Priority getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "getPriority() method is not supported by LambdaAdapter"); } diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index a5b4a29cb18..5cee402f88c 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -291,7 +291,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() /// Adjust priority for (auto & item : future_merged_part->parts) - priority += item->getBytesOnDisk(); + priority.value += item->getBytesOnDisk(); return {true, true, [this, stopwatch = *stopwatch_ptr] (const ExecutionStatus & execution_status) { diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.h b/src/Storages/MergeTree/MergeFromLogEntryTask.h index 9d870cd9a4b..2c559c06d7e 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.h +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.h @@ -22,7 +22,7 @@ public: StorageReplicatedMergeTree & storage_, IExecutableTask::TaskResultCallback & task_result_callback_); - UInt64 getPriority() override { return priority; } + Priority getPriority() override { return priority; } protected: /// Both return false if we can't execute merge. @@ -44,7 +44,7 @@ private: StopwatchUniquePtr stopwatch_ptr{nullptr}; MergeTreeData::MutableDataPartPtr part; - UInt64 priority{0}; + Priority priority; MergeTaskPtr merge_task; }; diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.h b/src/Storages/MergeTree/MergePlainMergeTreeTask.h index 369b4390da7..95df8c90c9b 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.h @@ -34,13 +34,13 @@ public: , task_result_callback(task_result_callback_) { for (auto & item : merge_mutate_entry->future_part->parts) - priority += item->getBytesOnDisk(); + priority.value += item->getBytesOnDisk(); } bool executeStep() override; void onCompleted() override; StorageID getStorageID() override; - UInt64 getPriority() override { return priority; } + Priority getPriority() override { return priority; } void setCurrentTransaction(MergeTreeTransactionHolder && txn_holder_, MergeTreeTransactionPtr && txn_) { @@ -77,7 +77,7 @@ private: using MergeListEntryPtr = std::unique_ptr; MergeListEntryPtr merge_list_entry; - UInt64 priority{0}; + Priority priority; std::function write_part_log; std::function transfer_profile_counters_to_initial_query; diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index 8142e383d0c..552ed32e2d2 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -63,7 +63,7 @@ struct TaskRuntimeData /// This scenario in not possible in reality. Poco::Event is_done{/*autoreset=*/false}; /// This is equal to task->getPriority() not to do useless virtual calls in comparator - UInt64 priority{0}; + Priority priority; /// By default priority queue will have max element at top static bool comparePtrByPriority(const TaskRuntimeDataPtr & lhs, const TaskRuntimeDataPtr & rhs) diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index d0b85ee65b8..42dccef7e6f 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -184,7 +184,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() /// Adjust priority for (auto & item : future_mutated_part->parts) - priority += item->getBytesOnDisk(); + priority.value += item->getBytesOnDisk(); return {true, true, [this] (const ExecutionStatus & execution_status) { diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.h b/src/Storages/MergeTree/MutateFromLogEntryTask.h index 2a2cc308f85..42b4debcbf2 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.h +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.h @@ -27,7 +27,7 @@ public: {} - UInt64 getPriority() override { return priority; } + Priority getPriority() override { return priority; } private: @@ -40,7 +40,7 @@ private: return mutate_task->execute(); } - UInt64 priority{0}; + Priority priority; TableLockHolder table_lock_holder{nullptr}; ReservationSharedPtr reserved_space{nullptr}; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h index 823ea6d7a0f..bd03c276256 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h @@ -36,13 +36,13 @@ public: , task_result_callback(task_result_callback_) { for (auto & part : merge_mutate_entry->future_part->parts) - priority += part->getBytesOnDisk(); + priority.value += part->getBytesOnDisk(); } bool executeStep() override; void onCompleted() override; StorageID getStorageID() override; - UInt64 getPriority() override { return priority; } + Priority getPriority() override { return priority; } private: @@ -66,7 +66,7 @@ private: std::unique_ptr stopwatch; MergeTreeData::MutableDataPartPtr new_part; - UInt64 priority{0}; + Priority priority; using MergeListEntryPtr = std::unique_ptr; MergeListEntryPtr merge_list_entry; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 76096d00641..31b3c249177 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1,5 +1,6 @@ #include +#include "Common/Priority.h" #include #include #include @@ -961,7 +962,7 @@ public: void onCompleted() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + Priority getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } bool executeStep() override { @@ -1283,7 +1284,7 @@ public: void onCompleted() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + Priority getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } bool executeStep() override { @@ -1412,7 +1413,7 @@ public: void onCompleted() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + Priority getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } bool executeStep() override { diff --git a/src/Storages/MergeTree/tests/gtest_executor.cpp b/src/Storages/MergeTree/tests/gtest_executor.cpp index 3a4f147b456..5815b74284a 100644 --- a/src/Storages/MergeTree/tests/gtest_executor.cpp +++ b/src/Storages/MergeTree/tests/gtest_executor.cpp @@ -51,7 +51,7 @@ public: throw std::runtime_error("Unlucky..."); } - UInt64 getPriority() override { return 0; } + Priority getPriority() override { return {}; } private: std::mt19937 generator; @@ -65,11 +65,11 @@ using StepFunc = std::function; class LambdaExecutableTask : public IExecutableTask { public: - explicit LambdaExecutableTask(const String & name_, size_t step_count_, StepFunc step_func_ = {}, UInt64 priority_ = 0) + explicit LambdaExecutableTask(const String & name_, size_t step_count_, StepFunc step_func_ = {}, Int64 priority_value = 0) : name(name_) , step_count(step_count_) , step_func(step_func_) - , priority(priority_) + , priority{priority_value} {} bool executeStep() override @@ -86,13 +86,13 @@ public: void onCompleted() override {} - UInt64 getPriority() override { return priority; } + Priority getPriority() override { return priority; } private: String name; size_t step_count; StepFunc step_func; - UInt64 priority; + Priority priority; }; From 6f6d806f927ba1e2c539a92e78906334bee20ce4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 6 Jun 2023 15:17:06 +0000 Subject: [PATCH 1340/2223] Upd test test_alter_moving_garbage --- .../configs/config.d/storage_conf.xml | 2 -- .../test_alter_moving_garbage/test.py | 31 +++++++++++++------ 2 files changed, 21 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml b/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml index 659f59a41b2..f6898ed1d7e 100644 --- a/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml +++ b/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml @@ -1,6 +1,4 @@ - - diff --git a/tests/integration/test_alter_moving_garbage/test.py b/tests/integration/test_alter_moving_garbage/test.py index b369c9ad377..4a42b73b8d4 100644 --- a/tests/integration/test_alter_moving_garbage/test.py +++ b/tests/integration/test_alter_moving_garbage/test.py @@ -28,7 +28,7 @@ def cluster(): cluster.shutdown() -def create_table(node, table_name, **additional_settings): +def create_table(node, table_name, additional_settings): settings = { "storage_policy": "two_disks", "old_parts_lifetime": 1, @@ -52,21 +52,32 @@ def create_table(node, table_name, **additional_settings): node.query(create_table_statement) -def test_create_table(cluster): +@pytest.mark.parametrize("allow_remote_fs_zero_copy_replication", [False, True]) +def test_create_table(cluster, allow_remote_fs_zero_copy_replication): node = cluster.instances["node1"] - create_table(node, "test_table") + + additional_settings = {} + table_name = "test_table" + + if allow_remote_fs_zero_copy_replication: + # different names for logs readability + table_name = "test_table_zero_copy" + additional_settings["allow_remote_fs_zero_copy_replication"] = 1 + + create_table(node, table_name, additional_settings) + node.query( - "INSERT INTO test_table SELECT toDate('2021-01-01') + INTERVAL number % 10 DAY, number, toString(sipHash64(number)) FROM numbers(100_000)" + f"INSERT INTO {table_name} SELECT toDate('2021-01-01') + INTERVAL number % 10 DAY, number, toString(sipHash64(number)) FROM numbers(100_000)" ) stop_alter = False def alter(): d = 0 - node.query(f"ALTER TABLE test_table ADD COLUMN col0 String") + node.query(f"ALTER TABLE {table_name} ADD COLUMN col0 String") while not stop_alter: d = d + 1 - node.query(f"DELETE FROM test_table WHERE id < {d}") + node.query(f"DELETE FROM {table_name} WHERE id < {d}") time.sleep(0.1) alter_thread = threading.Thread(target=alter) @@ -76,12 +87,12 @@ def test_create_table(cluster): partition = f"2021-01-{i:02d}" try: node.query( - f"ALTER TABLE test_table MOVE PARTITION '{partition}' TO DISK 's3'", + f"ALTER TABLE {table_name} MOVE PARTITION '{partition}' TO DISK 's3'", ) except QueryRuntimeException as e: - # PART_IS_TEMPORARILY_LOCKED - assert 384 == e.returncode - continue + if "PART_IS_TEMPORARILY_LOCKED" in str(e): + continue + raise e # clear old temporary directories wakes up every 1 second time.sleep(0.5) From 989540e5b1ae4c10605e7609a7906e789ad755a4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 7 Jun 2023 17:37:32 +0000 Subject: [PATCH 1341/2223] Upd test_alter_moving_garbage: use replicated engine --- .../configs/config.d/remote_servers.xml | 16 +++ .../test_alter_moving_garbage/test.py | 99 ++++++++++++++----- 2 files changed, 88 insertions(+), 27 deletions(-) create mode 100644 tests/integration/test_alter_moving_garbage/configs/config.d/remote_servers.xml diff --git a/tests/integration/test_alter_moving_garbage/configs/config.d/remote_servers.xml b/tests/integration/test_alter_moving_garbage/configs/config.d/remote_servers.xml new file mode 100644 index 00000000000..45713eaed59 --- /dev/null +++ b/tests/integration/test_alter_moving_garbage/configs/config.d/remote_servers.xml @@ -0,0 +1,16 @@ + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_alter_moving_garbage/test.py b/tests/integration/test_alter_moving_garbage/test.py index 4a42b73b8d4..dc3f6c35ead 100644 --- a/tests/integration/test_alter_moving_garbage/test.py +++ b/tests/integration/test_alter_moving_garbage/test.py @@ -3,22 +3,29 @@ import time import pytest import threading +import random from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +# two replicas in remote_servers.xml +REPLICA_COUNT = 2 @pytest.fixture(scope="module") def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "node1", - main_configs=[ - "configs/config.d/storage_conf.xml", - ], - with_minio=True, - ) + for i in range(1, REPLICA_COUNT + 1): + cluster.add_instance( + f"node{i}", + main_configs=[ + "configs/config.d/storage_conf.xml", + "configs/config.d/remote_servers.xml", + ], + with_minio=True, + with_zookeeper=True, + ) + logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -28,7 +35,7 @@ def cluster(): cluster.shutdown() -def create_table(node, table_name, additional_settings): +def create_table(node, table_name, replicated, additional_settings): settings = { "storage_policy": "two_disks", "old_parts_lifetime": 1, @@ -38,55 +45,91 @@ def create_table(node, table_name, additional_settings): } settings.update(additional_settings) + table_engine = ( + f"ReplicatedMergeTree('/clickhouse/tables/0/{table_name}', '{node.name}')" + if replicated + else "MergeTree()" + ) + create_table_statement = f""" CREATE TABLE {table_name} ( dt Date, id Int64, data String, INDEX min_max (id) TYPE minmax GRANULARITY 3 - ) ENGINE=MergeTree() + ) ENGINE = {table_engine} PARTITION BY dt ORDER BY (dt, id) SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))}""" - node.query(create_table_statement) + if replicated: + node.query_with_retry(create_table_statement) + else: + node.query(create_table_statement) -@pytest.mark.parametrize("allow_remote_fs_zero_copy_replication", [False, True]) -def test_create_table(cluster, allow_remote_fs_zero_copy_replication): - node = cluster.instances["node1"] +@pytest.mark.parametrize( + "allow_remote_fs_zero_copy_replication,replicated_engine", + [(False, False), (False, True), (True, True)], +) +def test_create_table( + cluster, allow_remote_fs_zero_copy_replication, replicated_engine +): + if replicated_engine: + nodes = list(cluster.instances.values()) + else: + nodes = [cluster.instances["node1"]] additional_settings = {} - table_name = "test_table" + # different names for logs readability + table_name = "test_table" if allow_remote_fs_zero_copy_replication: - # different names for logs readability table_name = "test_table_zero_copy" additional_settings["allow_remote_fs_zero_copy_replication"] = 1 + if replicated_engine: + table_name = table_name + "_replicated" - create_table(node, table_name, additional_settings) + for node in nodes: + create_table(node, table_name, replicated_engine, additional_settings) - node.query( - f"INSERT INTO {table_name} SELECT toDate('2021-01-01') + INTERVAL number % 10 DAY, number, toString(sipHash64(number)) FROM numbers(100_000)" - ) + for i in range(1, 11): + partition = f"2021-01-{i:02d}" + random.choice(nodes).query( + f"INSERT INTO {table_name} SELECT toDate('{partition}'), number as id, toString(sipHash64(number, {i})) FROM numbers(10_000)" + ) + + def check_count(): + if replicated_engine: + return random.choice(nodes).query_with_retry( + f"SELECT countDistinct(dt, data) FROM clusterAllReplicas(test_cluster, default.{table_name}) WHERE id % 100 = 0" + ) + else: + return random.choice(nodes).query( + f"SELECT countDistinct(dt, data) FROM {table_name} WHERE id % 100 = 0" + ) + + assert check_count() == "1000\n" stop_alter = False def alter(): - d = 0 - node.query(f"ALTER TABLE {table_name} ADD COLUMN col0 String") - while not stop_alter: - d = d + 1 - node.query(f"DELETE FROM {table_name} WHERE id < {d}") + random.choice(nodes).query(f"ALTER TABLE {table_name} ADD COLUMN col0 String") + for d in range(1, 100): + if stop_alter: + break + # I managed to reproduce issue with DELETE, but it can be any other lightweight mutation + # Do not delete rows with id % 100 = 0, because they are used in check_count to check that data is not corrupted + random.choice(nodes).query(f"DELETE FROM {table_name} WHERE id % 100 = {d}") time.sleep(0.1) alter_thread = threading.Thread(target=alter) alter_thread.start() - for i in range(1, 10): + for i in range(1, 11): partition = f"2021-01-{i:02d}" try: - node.query( + random.choice(nodes).query( f"ALTER TABLE {table_name} MOVE PARTITION '{partition}' TO DISK 's3'", ) except QueryRuntimeException as e: @@ -94,8 +137,10 @@ def test_create_table(cluster, allow_remote_fs_zero_copy_replication): continue raise e - # clear old temporary directories wakes up every 1 second + # Function to clear old temporary directories wakes up every 1 second, sleep to make sure it is called time.sleep(0.5) stop_alter = True alter_thread.join() + + assert check_count() == "1000\n" From e1ceb01ad0b8988ec70818472740fdeff5acf112 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 7 Jun 2023 19:45:27 +0200 Subject: [PATCH 1342/2223] Parallel replicas for short queries (#50639) --- src/Core/Settings.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 39 +++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 7 ++++ tests/broken_tests.txt | 2 +- ...lel_replicas_automatic_disabling.reference | 2 + ..._parallel_replicas_automatic_disabling.sql | 15 +++++++ 6 files changed, 65 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference create mode 100644 tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0037acedede..a87e321bed2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -160,6 +160,7 @@ class IColumn; M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ + M(UInt64, parallel_replicas_min_number_of_granules_to_enable, 0, "If the number of marks to read is less than the value of this setting - parallel replicas will be disabled", 0) \ \ M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e806e1bb93f..2f6870f8b41 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7154,6 +7154,9 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( /// Parallel replicas if (query_context->canUseParallelReplicasOnInitiator() && to_stage >= QueryProcessingStage::WithMergeableState) { + if (!canUseParallelReplicasBasedOnPKAnalysis(query_context, storage_snapshot, query_info)) + return QueryProcessingStage::Enum::FetchColumns; + /// ReplicatedMergeTree if (supportsReplication()) return QueryProcessingStage::Enum::WithMergeableState; @@ -7179,6 +7182,42 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( } +bool MergeTreeData::canUseParallelReplicasBasedOnPKAnalysis( + ContextPtr query_context, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info) const +{ + const auto & snapshot_data = assert_cast(*storage_snapshot->data); + const auto & parts = snapshot_data.parts; + + MergeTreeDataSelectExecutor reader(*this); + auto result_ptr = reader.estimateNumMarksToRead( + parts, + query_info.prewhere_info, + storage_snapshot->getMetadataForQuery()->getColumns().getAll().getNames(), + storage_snapshot->metadata, + storage_snapshot->metadata, + query_info, + /*added_filter_nodes*/ActionDAGNodes{}, + query_context, + query_context->getSettingsRef().max_threads); + + if (result_ptr->error()) + std::rethrow_exception(std::get(result_ptr->result)); + + LOG_TRACE(log, "Estimated number of granules to read is {}", result_ptr->marks()); + + bool decision = result_ptr->marks() >= query_context->getSettingsRef().parallel_replicas_min_number_of_granules_to_enable; + + if (!decision) + LOG_DEBUG(log, "Parallel replicas will be disabled, because the estimated number of granules to read {} is less than the threshold which is {}", + result_ptr->marks(), + query_context->getSettingsRef().parallel_replicas_min_number_of_granules_to_enable); + + return decision; +} + + MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const { MergeTreeData * src_data = dynamic_cast(&source_table); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2f254f9a787..b1e1e43bd0b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1536,6 +1536,13 @@ private: static MutableDataPartPtr asMutableDeletingPart(const DataPartPtr & part); mutable TemporaryParts temporary_parts; + + /// Estimate the number of marks to read to make a decision whether to enable parallel replicas (distributed processing) or not + /// Note: it could be very rough. + bool canUseParallelReplicasBasedOnPKAnalysis( + ContextPtr query_context, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info) const; }; /// RAII struct to record big parts that are submerging or emerging. diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index 02935712325..7ee497973b8 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -137,4 +137,4 @@ 02534_s3_cluster_insert_select_schema_inference 02764_parallel_replicas_plain_merge_tree 02765_parallel_replicas_final_modifier - +02784_parallel_replicas_automatic_disabling diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference new file mode 100644 index 00000000000..af81158ecae --- /dev/null +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference @@ -0,0 +1,2 @@ +10 +1 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql b/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql new file mode 100644 index 00000000000..b2f674ddb64 --- /dev/null +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test_parallel_replicas_automatic_disabling; +CREATE TABLE test_parallel_replicas_automatic_disabling (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO test_parallel_replicas_automatic_disabling SELECT * FROM numbers(10); + +SYSTEM FLUSH LOGS; + +SET skip_unavailable_shards=1, allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1, parallel_replicas_min_number_of_granules_to_enable=10000; +SET send_logs_level='error'; +SELECT count() FROM test_parallel_replicas_automatic_disabling WHERE NOT ignore(*); + +SYSTEM FLUSH LOGS; + +SELECT count() > 0 FROM system.text_log WHERE event_time >= now() - INTERVAL 2 MINUTE AND message LIKE '%Parallel replicas will be disabled, because the estimated number of granules to read%'; + +DROP TABLE test_parallel_replicas_automatic_disabling; From c0f2141bd0432c7ebcab5ee0ef033141194fd59d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 7 Jun 2023 19:51:41 +0200 Subject: [PATCH 1343/2223] Revert "date_trunc function to always return DateTime type" (#50670) --- src/Functions/date_trunc.cpp | 98 ++++++++----------- .../00189_time_zones_long.reference | 56 +++++------ ...21_datetime64_compatibility_long.reference | 8 +- 3 files changed, 71 insertions(+), 91 deletions(-) diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index 87fff0b7f3c..016b8f4da5e 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -1,6 +1,6 @@ #include -#include #include +#include #include #include #include @@ -25,7 +25,7 @@ class FunctionDateTrunc : public IFunction public: static constexpr auto name = "dateTrunc"; - explicit FunctionDateTrunc(ContextPtr context_) : context(context_) { } + explicit FunctionDateTrunc(ContextPtr context_) : context(context_) {} static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } @@ -39,58 +39,51 @@ public: { /// The first argument is a constant string with the name of datepart. - intermediate_type_is_date = false; + auto result_type_is_date = false; String datepart_param; - auto check_first_argument = [&] - { + auto check_first_argument = [&] { const ColumnConst * datepart_column = checkAndGetColumnConst(arguments[0].column.get()); if (!datepart_column) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "First argument for function {} must be constant string: " - "name of datepart", - getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be constant string: " + "name of datepart", getName()); datepart_param = datepart_column->getValue(); if (datepart_param.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty", getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty", + getName()); if (!IntervalKind::tryParseString(datepart_param, datepart_kind)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName()); - intermediate_type_is_date = (datepart_kind == IntervalKind::Year) || (datepart_kind == IntervalKind::Quarter) - || (datepart_kind == IntervalKind::Month) || (datepart_kind == IntervalKind::Week); + result_type_is_date = (datepart_kind == IntervalKind::Year) + || (datepart_kind == IntervalKind::Quarter) || (datepart_kind == IntervalKind::Month) + || (datepart_kind == IntervalKind::Week); }; bool second_argument_is_date = false; - auto check_second_argument = [&] - { + auto check_second_argument = [&] { if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 2nd argument of function {}. " - "Should be a date or a date with time", - arguments[1].type->getName(), - getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. " + "Should be a date or a date with time", arguments[1].type->getName(), getName()); second_argument_is_date = isDate(arguments[1].type); - if (second_argument_is_date - && ((datepart_kind == IntervalKind::Hour) || (datepart_kind == IntervalKind::Minute) - || (datepart_kind == IntervalKind::Second))) + if (second_argument_is_date && ((datepart_kind == IntervalKind::Hour) + || (datepart_kind == IntervalKind::Minute) || (datepart_kind == IntervalKind::Second))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName()); }; - auto check_timezone_argument = [&] - { + auto check_timezone_argument = [&] { if (!WhichDataType(arguments[2].type).isString()) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of argument of function {}. " + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " "This argument is optional and must be a constant string with timezone name", - arguments[2].type->getName(), - getName()); + arguments[2].type->getName(), getName()); + + if (second_argument_is_date && result_type_is_date) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The timezone argument of function {} with datepart '{}' " + "is allowed only when the 2nd argument has the type DateTime", + getName(), datepart_param); }; if (arguments.size() == 2) @@ -106,14 +99,15 @@ public: } else { - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", - getName(), - arguments.size()); + getName(), arguments.size()); } - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1)); + if (result_type_is_date) + return std::make_shared(); + else + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1)); } bool useDefaultImplementationForConstants() const override { return true; } @@ -130,40 +124,26 @@ public: auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context); - ColumnPtr truncated_column; - auto date_type = std::make_shared(); - if (arguments.size() == 2) - truncated_column = to_start_of_interval->build(temp_columns) - ->execute(temp_columns, intermediate_type_is_date ? date_type : result_type, input_rows_count); - else - { - temp_columns[2] = arguments[2]; - truncated_column = to_start_of_interval->build(temp_columns) - ->execute(temp_columns, intermediate_type_is_date ? date_type : result_type, input_rows_count); - } + return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count); - if (!intermediate_type_is_date) - return truncated_column; - - ColumnsWithTypeAndName temp_truncated_column(1); - temp_truncated_column[0] = {truncated_column, date_type, ""}; - - auto to_date_time_or_default = FunctionFactory::instance().get("toDateTime", context); - return to_date_time_or_default->build(temp_truncated_column)->execute(temp_truncated_column, result_type, input_rows_count); + temp_columns[2] = arguments[2]; + return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count); } - bool hasInformationAboutMonotonicity() const override { return true; } + bool hasInformationAboutMonotonicity() const override + { + return true; + } Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override { - return {.is_monotonic = true, .is_always_monotonic = true}; + return { .is_monotonic = true, .is_always_monotonic = true }; } private: ContextPtr context; mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second; - mutable bool intermediate_type_is_date = false; }; } diff --git a/tests/queries/0_stateless/00189_time_zones_long.reference b/tests/queries/0_stateless/00189_time_zones_long.reference index a4287217a19..7d2ad3c8a01 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.reference +++ b/tests/queries/0_stateless/00189_time_zones_long.reference @@ -258,18 +258,18 @@ toUnixTimestamp 1426415400 1426415400 date_trunc -2019-01-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-10-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-12-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-12-30 00:00:00 -2019-12-30 00:00:00 -2019-12-30 00:00:00 +2019-01-01 +2020-01-01 +2020-01-01 +2019-10-01 +2020-01-01 +2020-01-01 +2019-12-01 +2020-01-01 +2020-01-01 +2019-12-30 +2019-12-30 +2019-12-30 2019-12-31 00:00:00 2020-01-01 00:00:00 2020-01-02 00:00:00 @@ -282,18 +282,18 @@ date_trunc 2019-12-31 20:11:22 2020-01-01 12:11:22 2020-01-02 05:11:22 -2019-01-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-10-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-12-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-12-30 00:00:00 -2019-12-30 00:00:00 -2019-12-30 00:00:00 +2019-01-01 +2020-01-01 +2020-01-01 +2019-10-01 +2020-01-01 +2020-01-01 +2019-12-01 +2020-01-01 +2020-01-01 +2019-12-30 +2019-12-30 +2019-12-30 2019-12-31 00:00:00 2020-01-01 00:00:00 2020-01-02 00:00:00 @@ -306,8 +306,8 @@ date_trunc 2019-12-31 20:11:22 2020-01-01 12:11:22 2020-01-02 05:11:22 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-12-30 00:00:00 +2020-01-01 +2020-01-01 +2020-01-01 +2019-12-30 2020-01-01 00:00:00 diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference index 2f56230db37..a946a114bf4 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference +++ b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference @@ -135,13 +135,13 @@ Code: 43 ------------------------------------------ SELECT date_trunc(\'year\', N, \'Asia/Istanbul\') Code: 43 -"DateTime('Asia/Istanbul')","2019-01-01 00:00:00" -"DateTime('Asia/Istanbul')","2019-01-01 00:00:00" +"Date","2019-01-01" +"Date","2019-01-01" ------------------------------------------ SELECT date_trunc(\'month\', N, \'Asia/Istanbul\') Code: 43 -"DateTime('Asia/Istanbul')","2019-09-01 00:00:00" -"DateTime('Asia/Istanbul')","2019-09-01 00:00:00" +"Date","2019-09-01" +"Date","2019-09-01" ------------------------------------------ SELECT date_trunc(\'day\', N, \'Asia/Istanbul\') "DateTime('Asia/Istanbul')","2019-09-16 00:00:00" From 7d5b98f0288fdebedf1984fce168260023448676 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 7 Jun 2023 20:03:17 +0200 Subject: [PATCH 1344/2223] Remove logging add sleeps --- src/Storages/StorageAzureBlob.cpp | 8 -------- src/TableFunctions/TableFunctionAzureBlobStorage.cpp | 2 -- tests/integration/test_storage_azure_blob_storage/test.py | 1 + 3 files changed, 1 insertion(+), 10 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 17374ba2d92..4901f6701fb 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -124,8 +124,6 @@ void StorageAzureBlob::processNamedCollectionResult(StorageAzureBlob::Configurat StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file) { - LOG_INFO(&Poco::Logger::get("StorageAzureBlob"), "get_format_from_file = {}", get_format_from_file); - StorageAzureBlob::Configuration configuration; /// Supported signatures: @@ -857,7 +855,6 @@ StorageAzureBlobSource::Iterator::Iterator( } else { - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "GLOBS BRANCH"); const String key_prefix = blob_path_with_globs->substr(0, blob_path_with_globs->find_first_of("*?{")); /// We don't have to list bucket, because there is no asterisks. @@ -870,11 +867,8 @@ StorageAzureBlobSource::Iterator::Iterator( return; } - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "KEY PREFIX {}", key_prefix); object_storage_iterator = object_storage->iterate(key_prefix); - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "BLOBS BLOBS{}", *blob_path_with_globs); - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "REGEXP PATTERN {}", makeRegexpPatternFromGlobs(*blob_path_with_globs)); matcher = std::make_unique(makeRegexpPatternFromGlobs(*blob_path_with_globs)); if (!matcher->ok()) @@ -1105,7 +1099,6 @@ String StorageAzureBlobSource::getName() const StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() { auto [current_key, info] = file_iterator->next(); - LOG_DEBUG(log, "KEY {} SIZE {}", current_key, info.size_bytes); if (current_key.empty()) return {}; @@ -1118,7 +1111,6 @@ StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() format_settings, std::nullopt, std::nullopt, /* is_remote_fs */ true, compression_method); - LOG_DEBUG(log, "FORMAT {}", format); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 986ad07fdde..265092ddefa 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -78,7 +78,6 @@ StorageAzureBlob::Configuration TableFunctionAzureBlobStorage::parseArgumentsImp std::unordered_map engine_args_to_idx; configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "CONFIGURATION {}", configuration.connection_url); configuration.is_connection_string = isConnectionString(configuration.connection_url); configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); @@ -194,7 +193,6 @@ void TableFunctionAzureBlobStorage::parseArguments(const ASTPtr & ast_function, auto & args = args_func.at(0)->children; configuration = parseArgumentsImpl(args, context); - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "CONFIGURATION {}", configuration.connection_url); } ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(ContextPtr context) const diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 319500e6226..a7ca049e809 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -47,6 +47,7 @@ def azure_query(node, query, try_num=3, settings={}): if error in str(ex): retry = True print(f"Try num: {i}. Having retriable error: {ex}") + time.sleep(i) break if not retry or i == try_num - 1: raise Exception(ex) From f3b5a87a66baa6ffed9cba8caa52e4c4a63cfc3d Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 6 Jun 2023 19:03:23 +0000 Subject: [PATCH 1345/2223] Fixing crash in anti/semi join --- src/Interpreters/HashJoin.cpp | 12 ++-- src/Interpreters/TableJoin.h | 15 ++++- src/Planner/PlannerJoinTree.cpp | 7 ++- .../02771_semi_join_use_nulls.reference | 0 .../02771_semi_join_use_nulls.sql.j2 | 60 +++++++++++++++++++ 5 files changed, 85 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02771_semi_join_use_nulls.reference create mode 100644 tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 9306c9b99eb..191dd74928d 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -679,7 +679,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) || - isRightOrFull(kind) || + isRightOrFull(kind) || strictness == JoinStrictness::Semi || strictness == JoinStrictness::Anti || multiple_disjuncts; if (save_key_columns) { @@ -707,8 +707,9 @@ Block HashJoin::prepareRightBlock(const Block & block, const Block & saved_block for (const auto & sample_column : saved_block_sample_.getColumnsWithTypeAndName()) { ColumnWithTypeAndName column = block.getByName(sample_column.name); - if (sample_column.column->isNullable()) - JoinCommon::convertColumnToNullable(column); + + /// There's no optimization for right side const columns. Remove constness if any. + column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst()); if (column.column->lowCardinality() && !sample_column.column->lowCardinality()) { @@ -716,8 +717,9 @@ Block HashJoin::prepareRightBlock(const Block & block, const Block & saved_block column.type = removeLowCardinality(column.type); } - /// There's no optimization for right side const columns. Remove constness if any. - column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst()); + if (sample_column.column->isNullable()) + JoinCommon::convertColumnToNullable(column); + structured_block.insert(std::move(column)); } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 0e0c905e30c..4a020684793 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -232,8 +232,19 @@ public: bool allowParallelHashJoin() const; bool joinUseNulls() const { return join_use_nulls; } - bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(kind()); } - bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(kind()); } + + /// Join use nulls doen't make sense for semi and anti joins + /// Only columns from corresponding table should be used, values in other table are undefined. + bool forceNullableRight() const + { + return join_use_nulls && isLeftOrFull(kind()) && strictness() != JoinStrictness::Semi && strictness() != JoinStrictness::Anti; + } + + bool forceNullableLeft() const + { + return join_use_nulls && isRightOrFull(kind()) && strictness() != JoinStrictness::Semi && strictness() != JoinStrictness::Anti; + } + 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/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 9672738ae6b..c0c6d301d88 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -873,10 +873,11 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ JoinClausesAndActions join_clauses_and_actions; JoinKind join_kind = join_node.getKind(); + JoinStrictness join_strictness = join_node.getStrictness(); std::optional join_constant; - if (join_node.getStrictness() == JoinStrictness::All) + if (join_strictness == JoinStrictness::All) join_constant = tryExtractConstantFromJoinNode(join_table_expression); if (join_constant) @@ -996,7 +997,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ plan_to_add_cast.addStep(std::move(cast_join_columns_step)); }; - if (join_use_nulls) + /// Join use nulls doen't make sense for semi and anti joins + /// Only columns from corresponding table should be used, values in other table are undefined. + if (join_use_nulls && join_strictness != JoinStrictness::Semi && join_strictness != JoinStrictness::Anti) { if (isFull(join_kind)) { diff --git a/tests/queries/0_stateless/02771_semi_join_use_nulls.reference b/tests/queries/0_stateless/02771_semi_join_use_nulls.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 b/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 new file mode 100644 index 00000000000..308df9a6094 --- /dev/null +++ b/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 @@ -0,0 +1,60 @@ + +{% for allow_experimental_analyzer in [0, 1] -%} +{% for join_use_nulls in [0, 1] -%} +{% for kind in ['LEFT', 'RIGHT'] -%} +{% for strictness in ['SEMI', 'ANTI'] -%} +{% for maybe_materialize in ['', 'materialize'] -%} + +SET allow_experimental_analyzer = {{ allow_experimental_analyzer }}; + +SET join_use_nulls = {{ join_use_nulls }}; + +-- FORMAT Null is used because one side is undefined (e.g. for SEMI LEFT only columns from the left side contain values) + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d +USING (id) +FORMAT Null +; + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d +USING (id) +FORMAT Null +; + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id) AS d +USING (id) +FORMAT Null +; + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id) AS d +USING (id) +FORMAT Null +; + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(1)) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(2)) AS id) AS d +USING (id) +FORMAT Null +; + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d +ON a.id = d.id +FORMAT Null +; + +SELECT id > 1, d.idd FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS idd) AS d +ON a.id = d.idd +FORMAT Null +; + +{% endfor -%} +{% endfor -%} +{% endfor -%} +{% endfor -%} +{% endfor -%} From a268250aff8fad35828641d6242c9d6d0e9ee3a3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 7 Jun 2023 13:25:06 +0000 Subject: [PATCH 1346/2223] Fixes for LowCardinality Nullable in HashJoin --- src/Columns/ColumnLowCardinality.cpp | 5 + src/Columns/tests/gtest_low_cardinality.cpp | 13 ++ src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/JoinUtils.cpp | 18 +-- src/Interpreters/TableJoin.h | 6 +- src/Planner/PlannerJoinTree.cpp | 4 +- .../02771_semi_join_use_nulls.reference | 112 ++++++++++++++++++ .../02771_semi_join_use_nulls.sql.j2 | 9 -- 8 files changed, 140 insertions(+), 29 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 4f9ab8215be..9269ea4ee4d 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -313,6 +313,11 @@ MutableColumnPtr ColumnLowCardinality::cloneResized(size_t size) const MutableColumnPtr ColumnLowCardinality::cloneNullable() const { auto res = cloneFinalized(); + /* Compact required not to share dictionary. + * If `shared` flag is not set `cloneFinalized` will return shallow copy + * and `nestedToNullable` will mutate source column. + */ + assert_cast(*res).compactInplace(); assert_cast(*res).nestedToNullable(); return res; } diff --git a/src/Columns/tests/gtest_low_cardinality.cpp b/src/Columns/tests/gtest_low_cardinality.cpp index 3ffc88f6a7d..5e01279b7df 100644 --- a/src/Columns/tests/gtest_low_cardinality.cpp +++ b/src/Columns/tests/gtest_low_cardinality.cpp @@ -48,3 +48,16 @@ TEST(ColumnLowCardinality, Insert) testLowCardinalityNumberInsert(std::make_shared()); testLowCardinalityNumberInsert(std::make_shared()); } + +TEST(ColumnLowCardinality, Clone) +{ + auto data_type = std::make_shared(); + auto low_cardinality_type = std::make_shared(data_type); + auto column = low_cardinality_type->createColumn(); + ASSERT_FALSE(assert_cast(*column).nestedIsNullable()); + + auto nullable_column = assert_cast(*column).cloneNullable(); + + ASSERT_TRUE(assert_cast(*nullable_column).nestedIsNullable()); + ASSERT_FALSE(assert_cast(*column).nestedIsNullable()); +} diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 191dd74928d..c4d1615a119 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -679,7 +679,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) || - isRightOrFull(kind) || strictness == JoinStrictness::Semi || strictness == JoinStrictness::Anti || + isRightOrFull(kind) || multiple_disjuncts; if (save_key_columns) { diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index a05b58e14a1..67ee2a64264 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -160,16 +160,14 @@ static ColumnPtr tryConvertColumnToNullable(ColumnPtr col) if (col->lowCardinality()) { - auto mut_col = IColumn::mutate(std::move(col)); - ColumnLowCardinality * col_lc = assert_cast(mut_col.get()); - if (col_lc->nestedIsNullable()) + const ColumnLowCardinality & col_lc = assert_cast(*col); + if (col_lc.nestedIsNullable()) { - return mut_col; + return col; } - else if (col_lc->nestedCanBeInsideNullable()) + else if (col_lc.nestedCanBeInsideNullable()) { - col_lc->nestedToNullable(); - return mut_col; + return col_lc.cloneNullable(); } } else if (const ColumnConst * col_const = checkAndGetColumn(*col)) @@ -232,11 +230,7 @@ void removeColumnNullability(ColumnWithTypeAndName & column) if (column.column && column.column->lowCardinality()) { - auto mut_col = IColumn::mutate(std::move(column.column)); - ColumnLowCardinality * col_as_lc = typeid_cast(mut_col.get()); - if (col_as_lc && col_as_lc->nestedIsNullable()) - col_as_lc->nestedRemoveNullable(); - column.column = std::move(mut_col); + column.column = assert_cast(column.column.get())->cloneWithDefaultOnNull(); } } else diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 4a020684793..6737cd8f13a 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -233,16 +233,14 @@ public: bool joinUseNulls() const { return join_use_nulls; } - /// Join use nulls doen't make sense for semi and anti joins - /// Only columns from corresponding table should be used, values in other table are undefined. bool forceNullableRight() const { - return join_use_nulls && isLeftOrFull(kind()) && strictness() != JoinStrictness::Semi && strictness() != JoinStrictness::Anti; + return join_use_nulls && isLeftOrFull(kind()); } bool forceNullableLeft() const { - return join_use_nulls && isRightOrFull(kind()) && strictness() != JoinStrictness::Semi && strictness() != JoinStrictness::Anti; + return join_use_nulls && isRightOrFull(kind()); } size_t defaultMaxBytes() const { return default_max_bytes; } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c0c6d301d88..d875cc38bce 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -997,9 +997,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ plan_to_add_cast.addStep(std::move(cast_join_columns_step)); }; - /// Join use nulls doen't make sense for semi and anti joins - /// Only columns from corresponding table should be used, values in other table are undefined. - if (join_use_nulls && join_strictness != JoinStrictness::Semi && join_strictness != JoinStrictness::Anti) + if (join_use_nulls) { if (isFull(join_kind)) { diff --git a/tests/queries/0_stateless/02771_semi_join_use_nulls.reference b/tests/queries/0_stateless/02771_semi_join_use_nulls.reference index e69de29bb2d..8d4b1a3a75e 100644 --- a/tests/queries/0_stateless/02771_semi_join_use_nulls.reference +++ b/tests/queries/0_stateless/02771_semi_join_use_nulls.reference @@ -0,0 +1,112 @@ +0 0 +0 0 +0 \N +0 0 +0 0 +0 0 +0 0 +0 0 +0 \N +0 0 +0 0 +0 0 +0 1 +0 1 +0 0 +0 0 +0 \N +0 0 +0 0 +0 0 +0 0 +0 0 +0 \N +0 0 +0 0 +0 0 +0 2 +0 2 +0 \N +0 0 +0 \N +0 0 +0 \N +0 \N +0 \N +0 0 +0 \N +0 0 +0 \N +0 \N +0 1 +0 1 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +\N 2 +\N 2 +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 1 +0 1 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +1 2 +1 2 +0 \N +0 0 +0 0 +0 0 +0 \N +0 \N +0 \N +0 0 +0 0 +0 0 +0 \N +0 \N +0 1 +0 1 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +1 2 +1 2 diff --git a/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 b/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 index 308df9a6094..37b2e63761b 100644 --- a/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 +++ b/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 @@ -9,48 +9,39 @@ SET allow_experimental_analyzer = {{ allow_experimental_analyzer }}; SET join_use_nulls = {{ join_use_nulls }}; --- FORMAT Null is used because one side is undefined (e.g. for SEMI LEFT only columns from the left side contain values) - SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d USING (id) -FORMAT Null ; SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d USING (id) -FORMAT Null ; SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id) AS d USING (id) -FORMAT Null ; SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id) AS d USING (id) -FORMAT Null ; SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(1)) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(2)) AS id) AS d USING (id) -FORMAT Null ; SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d ON a.id = d.id -FORMAT Null ; SELECT id > 1, d.idd FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS idd) AS d ON a.id = d.idd -FORMAT Null ; {% endfor -%} From b11f744252b486ca0ba25deeb07181b4025e0edf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Jun 2023 20:33:08 +0200 Subject: [PATCH 1347/2223] Correctly disable async insert with deduplication when it's not needed (#50663) * Correctly disable async insert when it's not used * Better * Add comment * Better * Fix tests --------- Co-authored-by: Nikita Mikhaylov --- src/Core/ExternalTable.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 4 +-- .../Transforms/CreatingSetsTransform.cpp | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 9 +++--- .../Transforms/buildPushingToViewsChain.h | 2 ++ src/Server/GRPCServer.cpp | 2 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/HDFS/StorageHDFS.h | 2 +- src/Storages/Hive/StorageHive.cpp | 2 +- src/Storages/Hive/StorageHive.h | 2 +- src/Storages/IStorage.h | 5 +++- src/Storages/Kafka/StorageKafka.cpp | 2 +- src/Storages/Kafka/StorageKafka.h | 3 +- .../MeiliSearch/StorageMeiliSearch.cpp | 2 +- src/Storages/MeiliSearch/StorageMeiliSearch.h | 2 +- src/Storages/NATS/StorageNATS.cpp | 2 +- src/Storages/NATS/StorageNATS.h | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageDistributed.cpp | 4 +-- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageFile.cpp | 3 +- src/Storages/StorageFile.h | 3 +- src/Storages/StorageJoin.cpp | 4 +-- src/Storages/StorageJoin.h | 2 +- src/Storages/StorageKeeperMap.cpp | 2 +- src/Storages/StorageKeeperMap.h | 2 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMaterializedMySQL.h | 2 +- src/Storages/StorageMaterializedView.cpp | 4 +-- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMemory.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageMongoDB.cpp | 2 +- src/Storages/StorageMongoDB.h | 3 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StorageMySQL.h | 2 +- src/Storages/StorageNull.h | 2 +- src/Storages/StoragePostgreSQL.cpp | 2 +- src/Storages/StoragePostgreSQL.h | 2 +- src/Storages/StorageProxy.h | 4 +-- src/Storages/StorageReplicatedMergeTree.cpp | 4 +-- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3.h | 2 +- src/Storages/StorageSQLite.cpp | 2 +- src/Storages/StorageSQLite.h | 2 +- src/Storages/StorageSet.cpp | 2 +- src/Storages/StorageSet.h | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTableFunction.h | 5 ++-- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageURL.h | 2 +- src/Storages/StorageXDBC.cpp | 2 +- src/Storages/StorageXDBC.h | 2 +- .../System/StorageSystemZooKeeper.cpp | 2 +- src/Storages/System/StorageSystemZooKeeper.h | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 2 +- src/Storages/tests/gtest_storage_log.cpp | 2 +- ...sable_async_with_dedup_correctly.reference | 17 +++++++++++ ...2784_disable_async_with_dedup_correctly.sh | 29 +++++++++++++++++++ 71 files changed, 136 insertions(+), 78 deletions(-) create mode 100644 tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference create mode 100755 tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.sh diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 0f880ed967f..676af280cad 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -167,7 +167,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, auto temporary_table = TemporaryTableHolder(getContext(), ColumnsDescription{columns}, {}); auto storage = temporary_table.getTable(); getContext()->addExternalTable(data->table_name, std::move(temporary_table)); - auto sink = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), getContext()); + auto sink = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), getContext(), /*async_insert=*/false); /// Write data auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*data->pipe)); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 08862032007..2901f2e23d0 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -170,7 +170,7 @@ public: else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) { auto external_table = external_storage_holder->getTable(); - auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); + auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext(), /*async_insert=*/false); auto io = interpreter->execute(); io.pipeline.complete(std::move(table_out)); CompletedPipelineExecutor executor(io.pipeline); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index e87b16f0e9d..078499fb013 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -282,7 +282,7 @@ Chain InterpreterInsertQuery::buildSink( /// Otherwise we'll get duplicates when MV reads same rows again from Kafka. if (table->noPushingToViews() && !no_destination) { - auto sink = table->write(query_ptr, metadata_snapshot, context_ptr); + auto sink = table->write(query_ptr, metadata_snapshot, context_ptr, async_insert); sink->setRuntimeData(thread_status, elapsed_counter_ms); out.addSource(std::move(sink)); } @@ -290,7 +290,7 @@ Chain InterpreterInsertQuery::buildSink( { out = buildPushingToViewsChain(table, metadata_snapshot, context_ptr, query_ptr, no_destination, - thread_status_holder, running_group, elapsed_counter_ms); + thread_status_holder, running_group, elapsed_counter_ms, async_insert); } return out; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index e3ae2d4fd4e..c6ac8bea5ba 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -91,7 +91,7 @@ void CreatingSetsTransform::startSubquery() if (subquery.table) /// TODO: make via port - table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); + table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext(), /*async_insert=*/false)); done_with_set = !subquery.set_in_progress; done_with_table = !subquery.table; diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 31bab46b868..0d33f250b5d 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -196,6 +196,7 @@ Chain buildPushingToViewsChain( ThreadStatusesHolderPtr thread_status_holder, ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms, + bool async_insert, const Block & live_view_header) { checkStackSize(); @@ -347,7 +348,7 @@ Chain buildPushingToViewsChain( out = buildPushingToViewsChain( view, view_metadata_snapshot, insert_context, ASTPtr(), /* no_destination= */ true, - thread_status_holder, running_group, view_counter_ms, storage_header); + thread_status_holder, running_group, view_counter_ms, async_insert, storage_header); } else if (auto * window_view = dynamic_cast(view.get())) { @@ -356,13 +357,13 @@ Chain buildPushingToViewsChain( out = buildPushingToViewsChain( view, view_metadata_snapshot, insert_context, ASTPtr(), /* no_destination= */ true, - thread_status_holder, running_group, view_counter_ms); + thread_status_holder, running_group, view_counter_ms, async_insert); } else out = buildPushingToViewsChain( view, view_metadata_snapshot, insert_context, ASTPtr(), /* no_destination= */ false, - thread_status_holder, running_group, view_counter_ms); + thread_status_holder, running_group, view_counter_ms, async_insert); views_data->views.emplace_back(ViewRuntimeData{ std::move(query), @@ -444,7 +445,7 @@ Chain buildPushingToViewsChain( /// Do not push to destination table if the flag is set else if (!no_destination) { - auto sink = storage->write(query_ptr, metadata_snapshot, context); + auto sink = storage->write(query_ptr, metadata_snapshot, context, async_insert); metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); sink->setRuntimeData(thread_status, elapsed_counter_ms); result_chain.addSource(std::move(sink)); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 0f413bee5c6..53aceeda1cc 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -69,6 +69,8 @@ Chain buildPushingToViewsChain( ThreadGroupPtr running_group, /// Counter to measure time spent separately per view. Should be improved. std::atomic_uint64_t * elapsed_counter_ms, + /// True if it's part of async insert flush + bool async_insert, /// LiveView executes query itself, it needs source block structure. const Block & live_view_header = {}); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 7b8eaa21947..e335d247a82 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1101,7 +1101,7 @@ namespace { /// The data will be written directly to the table. auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto sink = storage->write(ASTPtr(), metadata_snapshot, query_context); + auto sink = storage->write(ASTPtr(), metadata_snapshot, query_context, /*async_insert=*/false); std::unique_ptr buf = std::make_unique(external_table.data().data(), external_table.data().size()); buf = wrapReadBufferWithCompressionMethod(std::move(buf), chooseCompressionMethod("", external_table.compression_type())); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 0522b6d8a48..1ded7d97248 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1692,7 +1692,7 @@ bool TCPHandler::receiveData(bool scalar) } auto metadata_snapshot = storage->getInMemoryMetadataPtr(); /// The data will be written directly to the table. - QueryPipeline temporary_table_out(storage->write(ASTPtr(), metadata_snapshot, query_context)); + QueryPipeline temporary_table_out(storage->write(ASTPtr(), metadata_snapshot, query_context, /*async_insert=*/false)); PushingPipelineExecutor executor(temporary_table_out); executor.start(); executor.push(block); diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 98c2579e355..194a8f982d8 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -624,7 +624,7 @@ Pipe StorageHDFS::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_) +SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_, bool /*async_insert*/) { String current_uri = uris.back(); diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index b123834e981..c6226c2618d 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -41,7 +41,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; void truncate( const ASTPtr & query, diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index f554a14ec75..00c942fd56b 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -905,7 +905,7 @@ HiveFiles StorageHive::collectHiveFiles( return hive_files; } -SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot*/, ContextPtr /*context*/) +SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot*/, ContextPtr /*context*/, bool /*async_insert*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is not implemented for StorageHive"); } diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index b4afb2421b1..604df70f4d0 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -61,7 +61,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) override; + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool async_insert) override; NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index c163e8107ac..d44772850fd 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -402,11 +402,14 @@ public: * passed in all parts of the returned streams. Storage metadata can be * changed during lifetime of the returned streams, but the snapshot is * guaranteed to be immutable. + * + * async_insert - set to true if the write is part of async insert flushing */ virtual SinkToStoragePtr write( const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr /*context*/) + ContextPtr /*context*/, + bool /*async_insert*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is not supported by storage {}", getName()); } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 7d504833a0a..2aba76c1a3f 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -374,7 +374,7 @@ Pipe StorageKafka::read( } -SinkToStoragePtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto modified_context = Context::createCopy(local_context); modified_context->applySettingsChanges(settings_adjustments); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 3559129cf74..09aa091ef18 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -60,7 +60,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context) override; + ContextPtr context, + bool async_insert) override; /// We want to control the number of rows in a chunk inserted into Kafka bool prefersLargeBlocks() const override { return false; } diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp index e7350d38a20..5d77fc080a4 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp @@ -137,7 +137,7 @@ Pipe StorageMeiliSearch::read( return Pipe(std::make_shared(config, sample_block, max_block_size, route, kv_pairs_params)); } -SinkToStoragePtr StorageMeiliSearch::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageMeiliSearch::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { LOG_TRACE(log, "Trying update index: {}", config.index); return std::make_shared(config, metadata_snapshot->getSampleBlock(), local_context); diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.h b/src/Storages/MeiliSearch/StorageMeiliSearch.h index 41c1db53437..77cd2afb80a 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.h +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.h @@ -26,7 +26,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert) override; static MeiliSearchConfiguration getConfiguration(ASTs engine_args, ContextPtr context); diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index aa4ec77b0d8..a3478069356 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -353,7 +353,7 @@ void StorageNATS::read( } -SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto modified_context = addSettings(local_context); std::string subject = modified_context->getSettingsRef().stream_like_engine_insert_queue.changed diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 518d81fb145..efe54243ee9 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -51,7 +51,7 @@ public: size_t /* max_block_size */, size_t /* num_streams */) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; /// We want to control the number of rows in a chunk inserted into NATS bool prefersLargeBlocks() const override { return false; } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index e84f5c963a8..651c63e1b91 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -764,7 +764,7 @@ void StorageRabbitMQ::read( } -SinkToStoragePtr StorageRabbitMQ::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageRabbitMQ::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto producer = std::make_unique( configuration, routing_keys, exchange_name, exchange_type, producer_id.fetch_add(1), persistent, shutdown_called, log); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index c531026d83a..dc410c4f298 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -57,7 +57,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, - ContextPtr context) override; + ContextPtr context, + bool async_insert) override; /// We want to control the number of rows in a chunk inserted into RabbitMQ bool prefersLargeBlocks() const override { return false; } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index d1195a9132e..27e8de78b0f 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -461,7 +461,7 @@ Pipe StorageEmbeddedRocksDB::read( } SinkToStoragePtr StorageEmbeddedRocksDB::write( - const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) + const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool /*async_insert*/) { return std::make_shared(*this, metadata_snapshot); } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 32d7740009e..97fd07626a8 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -48,7 +48,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a4cb15d5711..d021667f771 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -656,7 +656,7 @@ private: }; -SinkToStoragePtr StorageBuffer::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) +SinkToStoragePtr StorageBuffer::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool /*async_insert*/) { return std::make_shared(*this, metadata_snapshot); } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 83d2376216b..8f089a4d580 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -88,7 +88,7 @@ public: bool supportsSubcolumns() const override { return true; } - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool /*async_insert*/) override; void startup() override; /// Flush all buffers into the subordinate table and stop background thread. diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index bcf6f68d00d..d86e735f4b4 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -897,7 +897,7 @@ QueryTreeNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, auto temporary_table_expression_node = std::make_shared(external_storage, mutable_context); temporary_table_expression_node->setTemporaryTableName(temporary_table_name); - auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context); + auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context, /*async_insert=*/false); auto io = interpreter.execute(); io.pipeline.complete(std::move(table_out)); CompletedPipelineExecutor executor(io.pipeline); @@ -1132,7 +1132,7 @@ void StorageDistributed::read( } -SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto cluster = getCluster(); const auto & settings = local_context->getSettingsRef(); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 417fb6447bf..f45286341cf 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -118,7 +118,7 @@ public: bool supportsParallelInsert() const override { return true; } std::optional totalBytes(const Settings &) const override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool /*async_insert*/) override; std::optional distributedWrite(const ASTInsertQuery & query, ContextPtr context) override; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 052775aefca..72347789790 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1049,7 +1049,8 @@ private: SinkToStoragePtr StorageFile::write( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, - ContextPtr context) + ContextPtr context, + bool /*async_insert*/) { if (format_name == "Distributed") throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is not implemented for Distributed format"); diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 53ce7eeaaf6..9eb0d4b4383 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -50,7 +50,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context) override; + ContextPtr context, + bool async_insert) override; void truncate( const ASTPtr & /*query*/, diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 5113320548d..a238e9ef26c 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -89,10 +89,10 @@ RWLockImpl::LockHolder StorageJoin::tryLockForCurrentQueryTimedWithContext(const return lock->getLock(type, query_id, acquire_timeout, false); } -SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool /*async_insert*/) { std::lock_guard mutate_lock(mutate_mutex); - return StorageSetOrJoinBase::write(query, metadata_snapshot, context); + return StorageSetOrJoinBase::write(query, metadata_snapshot, context, /*async_insert=*/false); } void StorageJoin::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context, TableExclusiveLockHolder &) diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index a5e85d8788a..5559b5d1ec8 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -59,7 +59,7 @@ public: /// (but not during processing whole query, it's safe for joinGet that doesn't involve `used_flags` from HashJoin) ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add, ContextPtr context) const; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; Pipe read( const Names & column_names, diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index deebb9e0096..f2b1b907832 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -524,7 +524,7 @@ Pipe StorageKeeperMap::read( return process_keys(std::move(filtered_keys)); } -SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { checkTable(); return std::make_shared(*this, metadata_snapshot->getSampleBlock(), local_context); diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 552e6b35fe8..ad7b719e972 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -42,7 +42,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; void drop() override; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 02dc4843660..ac68de43332 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -855,7 +855,7 @@ Pipe StorageLog::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { WriteLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index a2b1356f240..f1d05ed39ac 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -55,7 +55,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; diff --git a/src/Storages/StorageMaterializedMySQL.h b/src/Storages/StorageMaterializedMySQL.h index 9896265b576..08fbb61960f 100644 --- a/src/Storages/StorageMaterializedMySQL.h +++ b/src/Storages/StorageMaterializedMySQL.h @@ -32,7 +32,7 @@ public: QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr) override { throwNotAllowed(); } + SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, bool) override { throwNotAllowed(); } NamesAndTypesList getVirtuals() const override; ColumnSizeByName getColumnSizes() const override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index b96c132d601..16d724d54d8 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -192,13 +192,13 @@ void StorageMaterializedView::read( } } -SinkToStoragePtr StorageMaterializedView::write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr local_context) +SinkToStoragePtr StorageMaterializedView::write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr local_context, bool async_insert) { auto storage = getTargetTable(); auto lock = storage->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto sink = storage->write(query, metadata_snapshot, local_context); + auto sink = storage->write(query, metadata_snapshot, local_context, async_insert); sink->addTableLock(lock); return sink; diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index f7876005c49..3ec789aa7e3 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -39,7 +39,7 @@ public: return target_table->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; void drop() override; void dropInnerTableIfAny(bool sync, ContextPtr local_context) override; diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index caeefa5d96d..1b45b9ae3f4 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -159,7 +159,7 @@ void StorageMemory::read( } -SinkToStoragePtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool /*async_insert*/) { return std::make_shared(*this, metadata_snapshot, context); } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index ce8a59b8bcd..c4f4331ca64 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -64,7 +64,7 @@ public: bool hasEvenlyDistributedRead() const override { return true; } - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; void drop() override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 3da4724471d..a2a46229660 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -274,7 +274,7 @@ std::optional StorageMergeTree::totalBytes(const Settings &) const } SinkToStoragePtr -StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { const auto & settings = local_context->getSettingsRef(); return std::make_shared( diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index be9f5426bbd..8099f9c16aa 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -71,7 +71,7 @@ public: std::optional totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const override; std::optional totalBytes(const Settings &) const override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; /** Perform the next step in combining the parts. */ diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 63b8c2d00a1..ab52fde3e3d 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -165,7 +165,7 @@ Pipe StorageMongoDB::read( return Pipe(std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size)); } -SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */) +SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */, bool /*async_insert*/) { connectIfNotConnected(); return std::make_shared(collection_name, database_name, metadata_snapshot, connection); diff --git a/src/Storages/StorageMongoDB.h b/src/Storages/StorageMongoDB.h index 2b77f076e7e..36090d4584e 100644 --- a/src/Storages/StorageMongoDB.h +++ b/src/Storages/StorageMongoDB.h @@ -41,7 +41,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context) override; + ContextPtr context, + bool async_insert) override; struct Configuration { diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 2a96581d591..3e928c3a811 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -252,7 +252,7 @@ private: }; -SinkToStoragePtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { return std::make_shared( *this, diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index 9f47f9925d5..a98afc7ac4d 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -49,7 +49,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; struct Configuration { diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index d35c6a0b8b5..f6dd7064a22 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -46,7 +46,7 @@ public: bool supportsParallelInsert() const override { return true; } - SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr) override + SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, bool) override { return std::make_shared(metadata_snapshot->getSampleBlock()); } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 5d7dc285c5d..3551ee36819 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -451,7 +451,7 @@ private: SinkToStoragePtr StoragePostgreSQL::write( - const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */) + const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */, bool /*async_insert*/) { return std::make_shared(metadata_snapshot, pool->get(), remote_table_name, remote_table_schema, on_conflict); } diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index be6bbc5ec63..fb8b5a22df2 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -46,7 +46,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; struct Configuration { diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index e8a664a6382..14b7fc15af2 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -68,9 +68,9 @@ public: return getNested()->read(query_plan, column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override { - return getNested()->write(query, metadata_snapshot, context); + return getNested()->write(query, metadata_snapshot, context, async_insert); } void drop() override { getNested()->drop(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index db9c209a5fd..f9c9e958258 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4822,7 +4822,7 @@ void StorageReplicatedMergeTree::assertNotReadonly() const } -SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert) { if (!initialization_done) throw Exception(ErrorCodes::NOT_INITIALIZED, "Table is not initialized yet"); @@ -4839,7 +4839,7 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con const auto storage_settings_ptr = getSettings(); const Settings & query_settings = local_context->getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; - bool async_deduplicate = query_settings.async_insert && query_settings.async_insert_deduplicate && storage_settings_ptr->replicated_deduplication_window_for_async_inserts != 0 && query_settings.insert_deduplicate; + bool async_deduplicate = async_insert && query_settings.async_insert_deduplicate && storage_settings_ptr->replicated_deduplication_window_for_async_inserts != 0 && query_settings.insert_deduplicate; if (async_deduplicate) return std::make_shared( *this, metadata_snapshot, query_settings.insert_quorum.valueOr(0), diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index dd7ea84f76b..c0d9e36a8a7 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -138,7 +138,7 @@ public: std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context) const override; std::optional totalBytes(const Settings & settings) const override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; std::optional distributedWrite(const ASTInsertQuery & /*query*/, ContextPtr /*context*/) override; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index f1a7bcb71a2..c3ed0f1af16 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1086,7 +1086,7 @@ Pipe StorageS3::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto query_configuration = updateConfigurationAndGetCopy(local_context); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index a4c120b99a6..9c2728c785d 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -293,7 +293,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override; diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index bd445217979..d5ae6f2383f 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -169,7 +169,7 @@ private: }; -SinkToStoragePtr StorageSQLite::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr) +SinkToStoragePtr StorageSQLite::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool /*async_insert*/) { if (!sqlite_db) sqlite_db = openSQLiteDB(database_path, getContext(), /* throw_on_error */true); diff --git a/src/Storages/StorageSQLite.h b/src/Storages/StorageSQLite.h index 323c29ac8bb..9da040cbd5c 100644 --- a/src/Storages/StorageSQLite.h +++ b/src/Storages/StorageSQLite.h @@ -40,7 +40,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; static ColumnsDescription getTableStructureFromData( const SQLitePtr & sqlite_db_, diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 00b5dbfc5e3..f90539689e6 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -106,7 +106,7 @@ void SetOrJoinSink::onFinish() } -SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool /*async_insert*/) { UInt64 id = ++increment; return std::make_shared( diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index ccd1eb9912b..b310f817eb9 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -24,7 +24,7 @@ class StorageSetOrJoinBase : public IStorage public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {path}; } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index b2e7c202800..5c704d877d1 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -394,7 +394,7 @@ Pipe StorageStripeLog::read( } -SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { WriteLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 3f1b4ed0ad5..f889a1de71b 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -49,7 +49,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index ccec087a8d9..26cbe1f0233 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -130,7 +130,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, - ContextPtr context) override + ContextPtr context, + bool async_insert) override { auto storage = getNested(); auto cached_structure = metadata_snapshot->getSampleBlock(); @@ -139,7 +140,7 @@ public: { throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Source storage and table function have different structure"); } - return storage->write(query, metadata_snapshot, context); + return storage->write(query, metadata_snapshot, context, async_insert); } void renameInMemory(const StorageID & new_table_id) override diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index efc44a069dd..520576d3961 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -803,7 +803,7 @@ Pipe StorageURLWithFailover::read( } -SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool /*async_insert*/) { if (http_method.empty()) http_method = Poco::Net::HTTPRequest::HTTP_POST; diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 316b142aec0..e80e19621e8 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -41,7 +41,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; bool supportsPartitionBy() const override { return true; } diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index 9b3e203e337..b532d1c91f0 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -116,7 +116,7 @@ Pipe StorageXDBC::read( return IStorageURLBase::read(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); } -SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { bridge_helper->startBridgeSync(); diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index aa313e024ca..d7a1138c710 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -38,7 +38,7 @@ public: ContextPtr context_, BridgeHelperPtr bridge_helper_); - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; std::string getName() const override; private: diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 6ca74406b17..cef2feedcc5 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -215,7 +215,7 @@ void StorageSystemZooKeeper::read( query_plan.addStep(std::move(read_step)); } -SinkToStoragePtr StorageSystemZooKeeper::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context) +SinkToStoragePtr StorageSystemZooKeeper::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context, bool /*async_insert*/) { if (!context->getConfigRef().getBool("allow_zookeeper_write", false)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Prohibit writing to system.zookeeper, unless config `allow_zookeeper_write` as true"); diff --git a/src/Storages/System/StorageSystemZooKeeper.h b/src/Storages/System/StorageSystemZooKeeper.h index c8988d787a0..a016d3ad74c 100644 --- a/src/Storages/System/StorageSystemZooKeeper.h +++ b/src/Storages/System/StorageSystemZooKeeper.h @@ -20,7 +20,7 @@ public: static NamesAndTypesList getNamesAndTypes(); - SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override; + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/, bool /*async_insert*/) override; void read( QueryPlan & query_plan, diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 94d5db170a8..7fca9b5f078 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1549,7 +1549,7 @@ void StorageWindowView::writeIntoWindowView( auto lock = inner_table->lockForShare( local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = inner_table->getInMemoryMetadataPtr(); - auto output = inner_table->write(window_view.getMergeableQuery(), metadata_snapshot, local_context); + auto output = inner_table->write(window_view.getMergeableQuery(), metadata_snapshot, local_context, /*async_insert=*/false); output->addTableLock(lock); if (!blocksHaveEqualStructure(builder.getHeader(), output->getHeader())) diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index b63de6a66ef..352c44554bd 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -95,7 +95,7 @@ std::string writeData(int rows, DB::StoragePtr & table, const DB::ContextPtr con block.insert(column); } - QueryPipeline pipeline(table->write({}, metadata_snapshot, context)); + QueryPipeline pipeline(table->write({}, metadata_snapshot, context, /*async_insert=*/false)); PushingPipelineExecutor executor(pipeline); executor.push(block); diff --git a/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference b/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference new file mode 100644 index 00000000000..014be4ce1a9 --- /dev/null +++ b/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference @@ -0,0 +1,17 @@ +0 +1 +1 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 diff --git a/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.sh b/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.sh new file mode 100755 index 00000000000..40e7c9feabf --- /dev/null +++ b/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02784_async_table_with_dedup" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE 02784_async_table_with_dedup (a Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/02784_async_table_with_dedup', 'r1') ORDER BY a" + +CLICKHOUSE_CLIENT_WITH_LOG=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') + +function insert_with_log_check() { + $CLICKHOUSE_CLIENT_WITH_LOG --async-insert=1 --async_insert_deduplicate=1 --wait_for_async_insert=1 -q "$1" 2>&1 | grep -Fc "Setting async_insert=1, but INSERT query will be executed synchronously" +} + +insert_with_log_check "INSERT INTO 02784_async_table_with_dedup VALUES (1), (2)" +insert_with_log_check "INSERT INTO 02784_async_table_with_dedup SELECT number as a FROM system.numbers LIMIT 10 OFFSET 3" + +DATA_FILE=test_02784_async_$CLICKHOUSE_TEST_UNIQUE_NAME.csv +echo -e '13\n14' > $DATA_FILE + +insert_with_log_check "INSERT INTO 02784_async_table_with_dedup FROM INFILE '$DATA_FILE' FORMAT CSV" + +$CLICKHOUSE_CLIENT -q "SELECT a FROM 02784_async_table_with_dedup ORDER BY a" + +$CLICKHOUSE_CLIENT -q "DROP TABLE 02784_async_table_with_dedup" + +rm $DATA_FILE \ No newline at end of file From 129473ae744b7d8516e35ada5293e4b54be6f094 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 7 Jun 2023 20:48:03 +0200 Subject: [PATCH 1348/2223] Fix --- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 30 ++++++++++--------- .../IO/CachedOnDiskWriteBufferFromFile.h | 4 +-- src/Interpreters/Cache/FileSegment.h | 6 ---- 3 files changed, 18 insertions(+), 22 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 9153af90312..b7727555480 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -52,18 +52,20 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset FileSegment * file_segment; - if (file_segments.empty() || file_segments.back().isDownloaded()) + if (!file_segments || file_segments->empty() || file_segments->front().isDownloaded()) { file_segment = &allocateFileSegment(expected_write_offset, segment_kind); } else { - file_segment = &file_segments.back(); + file_segment = &file_segments->front(); } SCOPE_EXIT({ - if (file_segments.back().isDownloader()) - file_segments.back().completePartAndResetDownloader(); + if (!file_segments || file_segments->empty()) + return; + if (file_segments->front().isDownloader()) + file_segments->front().completePartAndResetDownloader(); }); while (size > 0) @@ -71,7 +73,7 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset size_t available_size = file_segment->range().size() - file_segment->getDownloadedSize(false); if (available_size == 0) { - completeFileSegment(*file_segment); + completeFileSegment(); file_segment = &allocateFileSegment(expected_write_offset, segment_kind); continue; } @@ -114,10 +116,7 @@ void FileSegmentRangeWriter::finalize() if (finalized) return; - if (file_segments.empty()) - return; - - completeFileSegment(file_segments.back()); + completeFileSegment(); finalized = true; } @@ -145,10 +144,9 @@ FileSegment & FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSeg /// We set max_file_segment_size to be downloaded, /// if we have less size to write, file segment will be resized in complete() method. - auto holder = cache->set(key, offset, cache->getMaxFileSegmentSize(), create_settings); - chassert(holder->size() == 1); - holder->moveTo(file_segments); - return file_segments.back(); + file_segments = cache->set(key, offset, cache->getMaxFileSegmentSize(), create_settings); + chassert(file_segments->size() == 1); + return file_segments->front(); } void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_segment) @@ -176,8 +174,12 @@ void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_s cache_log->add(elem); } -void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment) +void FileSegmentRangeWriter::completeFileSegment() { + if (!file_segments || file_segments->empty()) + return; + + auto & file_segment = file_segments->front(); /// File segment can be detached if space reservation failed. if (file_segment.isDetached() || file_segment.isCompleted()) return; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index 194afe88d88..8642886d6de 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -43,7 +43,7 @@ private: void appendFilesystemCacheLog(const FileSegment & file_segment); - void completeFileSegment(FileSegment & file_segment); + void completeFileSegment(); FileCache * cache; FileSegment::Key key; @@ -53,7 +53,7 @@ private: String query_id; String source_path; - FileSegmentsHolder file_segments{}; + FileSegmentsHolderPtr file_segments; size_t expected_write_offset = 0; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 163a15fcfda..186c65f12d9 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -360,12 +360,6 @@ struct FileSegmentsHolder : private boost::noncopyable FileSegments::const_iterator begin() const { return file_segments.begin(); } FileSegments::const_iterator end() const { return file_segments.end(); } - void moveTo(FileSegmentsHolder & holder) - { - holder.file_segments.insert(holder.file_segments.end(), file_segments.begin(), file_segments.end()); - file_segments.clear(); - } - private: FileSegments file_segments{}; const bool complete_on_dtor = true; From c2fc0713f2882e3ac09cc1f1c290180a29de180f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 7 Jun 2023 21:08:48 +0200 Subject: [PATCH 1349/2223] Update FileCache_fwd.h --- src/Interpreters/Cache/FileCache_fwd.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index dbb85fa0e7a..01f518d0c4e 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -5,7 +5,7 @@ namespace DB { static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 8 * 1024 * 1024; -static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 1024 * 1024; +static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; static constexpr size_t FILECACHE_DELAYED_CLEANUP_INTERVAL_MS = 1000 * 60; /// 1 min From 1e4bcd0d5d6ed04a02ad27ebb89ebd46f8b46e4c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 7 Jun 2023 21:44:57 +0200 Subject: [PATCH 1350/2223] Test to repro asan issue --- ...conditions_to_prewhere_analyzer_asan.reference | 4 ++++ ...e_all_conditions_to_prewhere_analyzer_asan.sql | 15 +++++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.reference create mode 100644 tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.sql diff --git a/tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.reference b/tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.reference new file mode 100644 index 00000000000..a08e8a1d440 --- /dev/null +++ b/tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.reference @@ -0,0 +1,4 @@ +0 0 +0 0 +0 0 +0 0 diff --git a/tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.sql b/tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.sql new file mode 100644 index 00000000000..44b9ce4fdc1 --- /dev/null +++ b/tests/queries/0_stateless/02784_move_all_conditions_to_prewhere_analyzer_asan.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS t_02784; + +CREATE TABLE t_02784 (c1 UInt64, c2 UInt64) ENGINE=MergeTree() ORDER BY c1 SETTINGS min_bytes_for_wide_part=1; + +INSERT INTO t_02784 SELECT number, number FROM numbers(1); + +SET allow_experimental_analyzer=1; +SET move_all_conditions_to_prewhere=1; + +SELECT c1, c2 FROM t_02784 WHERE c1 = 0 AND c2 = 0; +SELECT c1, c2 FROM t_02784 WHERE c2 = 0 AND c1 = 0; +SELECT c2, c1 FROM t_02784 WHERE c1 = 0 AND c2 = 0; +SELECT c2, c1 FROM t_02784 WHERE c2 = 0 AND c1 = 0; + +DROP TABLE t_02784; From 03b031eb401cbb46a5c25c0d66b9a8fae169b447 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 7 Jun 2023 21:48:54 +0200 Subject: [PATCH 1351/2223] Fix for using frame reference after stack was updated by previous optimizaton --- .../QueryPlan/Optimizations/optimizeTree.cpp | 45 ++++++++++--------- 1 file changed, 24 insertions(+), 21 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index dd17c1b5a30..091eecf99e5 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -114,32 +114,35 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s while (!stack.empty()) { - auto & frame = stack.back(); - - if (frame.next_child == 0) { - has_reading_from_mt |= typeid_cast(frame.node->step.get()) != nullptr; + /// NOTE: frame cannot be safely used after adding new elements to stack + auto & frame = stack.back(); - if (optimization_settings.read_in_order) - optimizeReadInOrder(*frame.node, nodes); + if (frame.next_child == 0) + { + has_reading_from_mt |= typeid_cast(frame.node->step.get()) != nullptr; - if (optimization_settings.optimize_projection) - num_applied_projection += optimizeUseAggregateProjections(*frame.node, nodes); + if (optimization_settings.read_in_order) + optimizeReadInOrder(*frame.node, nodes); - if (optimization_settings.aggregation_in_order) - optimizeAggregationInOrder(*frame.node, nodes); + if (optimization_settings.optimize_projection) + num_applied_projection += optimizeUseAggregateProjections(*frame.node, nodes); - if (optimization_settings.distinct_in_order) - tryDistinctReadInOrder(frame.node); - } + if (optimization_settings.aggregation_in_order) + optimizeAggregationInOrder(*frame.node, nodes); - /// Traverse all children first. - if (frame.next_child < frame.node->children.size()) - { - auto next_frame = Frame{.node = frame.node->children[frame.next_child]}; - ++frame.next_child; - stack.push_back(next_frame); - continue; + if (optimization_settings.distinct_in_order) + tryDistinctReadInOrder(frame.node); + } + + /// Traverse all children first. + if (frame.next_child < frame.node->children.size()) + { + auto next_frame = Frame{.node = frame.node->children[frame.next_child]}; + ++frame.next_child; + stack.push_back(next_frame); + continue; + } } if (optimization_settings.optimize_projection) @@ -162,7 +165,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizePrewhere(stack, nodes); optimizePrimaryKeyCondition(stack); - enableMemoryBoundMerging(*frame.node, nodes); + enableMemoryBoundMerging(*stack.back().node, nodes); stack.pop_back(); } From 92c87dedad043a17a1612f24b9fb43f175214b3f Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 8 Jun 2023 06:41:32 +0800 Subject: [PATCH 1352/2223] Add parallel state merge for some other combinator except If (#50413) * Add parallel state merge for some other combinator except If * add test * update test --- src/AggregateFunctions/AggregateFunctionArray.h | 7 +++++++ src/AggregateFunctions/AggregateFunctionMerge.h | 7 +++++++ src/AggregateFunctions/AggregateFunctionNull.h | 7 +++++++ src/AggregateFunctions/AggregateFunctionState.h | 7 +++++++ tests/performance/uniqExactIf.xml | 6 +++++- 5 files changed, 33 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionArray.h b/src/AggregateFunctions/AggregateFunctionArray.h index 21394e3ce05..7f38453f86b 100644 --- a/src/AggregateFunctions/AggregateFunctionArray.h +++ b/src/AggregateFunctions/AggregateFunctionArray.h @@ -141,6 +141,13 @@ public: nested_func->merge(place, rhs, arena); } + bool isAbleToParallelizeMerge() const override { return nested_func->isAbleToParallelizeMerge(); } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override + { + nested_func->merge(place, rhs, thread_pool, arena); + } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { nested_func->serialize(place, buf, version); diff --git a/src/AggregateFunctions/AggregateFunctionMerge.h b/src/AggregateFunctions/AggregateFunctionMerge.h index 0cb44259816..5b9e8e606af 100644 --- a/src/AggregateFunctions/AggregateFunctionMerge.h +++ b/src/AggregateFunctions/AggregateFunctionMerge.h @@ -110,6 +110,13 @@ public: nested_func->merge(place, rhs, arena); } + bool isAbleToParallelizeMerge() const override { return nested_func->isAbleToParallelizeMerge(); } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override + { + nested_func->merge(place, rhs, thread_pool, arena); + } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { nested_func->serialize(place, buf, version); diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index b817bad82fa..de7b190c949 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -148,6 +148,13 @@ public: nested_function->merge(nestedPlace(place), nestedPlace(rhs), arena); } + bool isAbleToParallelizeMerge() const override { return nested_function->isAbleToParallelizeMerge(); } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override + { + nested_function->merge(nestedPlace(place), nestedPlace(rhs), thread_pool, arena); + } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { bool flag = getFlag(place); diff --git a/src/AggregateFunctions/AggregateFunctionState.h b/src/AggregateFunctions/AggregateFunctionState.h index 625fe1f36bc..8335d21cb1e 100644 --- a/src/AggregateFunctions/AggregateFunctionState.h +++ b/src/AggregateFunctions/AggregateFunctionState.h @@ -91,6 +91,13 @@ public: nested_func->merge(place, rhs, arena); } + bool isAbleToParallelizeMerge() const override { return nested_func->isAbleToParallelizeMerge(); } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override + { + nested_func->merge(place, rhs, thread_pool, arena); + } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { nested_func->serialize(place, buf, version); diff --git a/tests/performance/uniqExactIf.xml b/tests/performance/uniqExactIf.xml index 409be257737..263fa75382c 100644 --- a/tests/performance/uniqExactIf.xml +++ b/tests/performance/uniqExactIf.xml @@ -1,3 +1,7 @@ - SELECT uniqExactIf(number, 1) FROM numbers_mt(1e6) + SELECT uniqExactIf(number, 1) FROM numbers_mt(1e7) + SELECT uniqExactState(number) FROM numbers_mt(1e7) Format Null + SELECT uniqExactArray([number]) FROM numbers_mt(1e7) Format Null + with (SELECT uniqExactState(number) FROM numbers_mt(1e7)) as a select uniqExactMerge(a) + SELECT uniqExactOrNull(number) FROM numbers_mt(1e7) From d3eb0805d44564fbbf093c5c1d39fbad532b0f59 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 7 Jun 2023 23:28:19 +0000 Subject: [PATCH 1353/2223] clang-tidy run + changes in docs --- docs/en/interfaces/cli.md | 12 ++++++------ docs/ru/interfaces/cli.md | 10 +++++----- src/Client/ConnectionString.cpp | 23 ++++++++++++----------- 3 files changed, 23 insertions(+), 22 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 94f1fbf9e41..ba54694faa9 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -198,9 +198,9 @@ If host is not specified, the default host will be used (localhost). If port is not specified, the default port will be used (9000). If database is not specified, the default database will be used. -User, password, and database can be specified in the connection string either in --user, --password, --database command line options. +User, password, and database can be specified in the connection string either in `--user`, `--password`, `--database` command line options. -The connection string must be specified in the first argument of clickhouse-client. The connection string can be combined with other [command-line-options](#command-line-options) except **--host(h)** and **--port**. +The connection string must be specified in the first argument of clickhouse-client. The connection string can be combined with other [command-line-options](#command-line-options) except `--host(h)` and `--port`. ### Multiple hosts {#connection_string_multiple_hosts} @@ -226,25 +226,25 @@ Connect to localhost using port 9000 in interactive, multiline mode. clickhouse-client "clickhouse://localhost:9000" -m ``` -Connect to localhost using port 9000 in interactive mode with the user specified in --user option. +Connect to localhost using port 9000 in interactive mode with the user specified in `--user` option. ``` bash clickhouse-client "clickhouse://localhost:9000" --user default ``` -Connect to localhost using port 9000 in interactive mode with database 'my_database' specified in command line option +Connect to localhost using port 9000 in interactive mode to `my_database` database specified in command line option ``` bash clickhouse-client "clickhouse://localhost:9000" --database my_database ``` -Connect to localhost using port 9000 in interactive mode with the database specified in the connection string. +Connect to localhost using port 9000 in interactive mode to `my_database` database specified in the connection string. ``` bash clickhouse-client "clickhouse://localhost:9000/my_database" ``` -Connect to localhost using port 9000 in interactive mode with a database specified in the connection string and a secure connection using shorthanded 's' URI parameter. +Connect to localhost using port 9000 in interactive mode to `my_database` database specified in the connection string and a secure connection using shorthanded 's' URI parameter. ```bash clickhouse-client "clickhouse://localhost/my_database?s" diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 30cd9757ebb..5f119ad9544 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -151,9 +151,9 @@ clickhouse://[2001:db8::1234] Если port не указан, будет использоваться порт по умолчанию (9000). Если база данных не указана, будет использоваться база данных по умолчанию (default). -Пользователь, пароль и база данных могут быть указаны в строке подключения либо в опциях командной строки --user, --password, --database. +Пользователь, пароль и база данных могут быть указаны в строке подключения либо в опциях командной строки `--user`, `--password`, `--database`. -Строка подключения должна быть указана в первом аргументе clickhouse-client. Строка подключения может комбинироваться с другими [параметрами командной строки] (#command-line-options) кроме **--host(h)** и **--port**. +Строка подключения должна быть указана в первом аргументе clickhouse-client. Строка подключения может комбинироваться с другими [параметрами командной строки] (#command-line-options) кроме `--host (h)` и `--port`. ### Несколько хостов {#connection_string_multiple_hosts} @@ -185,19 +185,19 @@ clickhouse-client "clickhouse://localhost:9000" -m clickhouse-client "clickhouse://localhost:9000" --user default ``` -Подключиться к localhost, используя порт 9000 в интерактивном режиме с базой данных 'my_database', указанной в опции командной строки. +Подключиться к localhost, используя порт 9000 в интерактивном режиме с базой данных `my_database`, указанной в опции командной строки. ``` bash clickhouse-client "clickhouse://localhost:9000" --database my_database ``` -Подключиться к localhost через порт 9000 в интерактивном режиме с базой данных my_database, указанной в строке подключения. +Подключиться к localhost через порт 9000 в интерактивном режиме с базой данных `my_database`, указанной в строке подключения. ``` bash clickhouse-client "clickhouse://localhost:9000/my_database" ``` -Подключиться к localhost через порт 9000 в интерактивном режиме с базой данных, указанной в строке подключения, и безопасным соединением с использованием сокращенного параметра URI 's'. +Подключиться к localhost через порт 9000 в интерактивном режиме с базой данных `my_database`, указанной в строке подключения, и безопасным соединением, используя короткий вариант команды URI 's'. ``` bash clickhouse-client "clickhouse://localhost/my_database?s" diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp index aeb1c1dca02..95fec5b52ee 100644 --- a/src/Client/ConnectionString.cpp +++ b/src/Client/ConnectionString.cpp @@ -128,20 +128,21 @@ bool tryParseConnectionString( else hosts_end_pos = hosts_or_user_info_end_pos; - auto hosts_end = hosts_end_pos != std::string_view::npos ? connection_string.begin() + hosts_end_pos - : connection_string.end(); + const auto * hosts_end = hosts_end_pos != std::string_view::npos ? connection_string.begin() + hosts_end_pos + : connection_string.end(); try { - // Poco::URI doesn't support several hosts in URI. - // Split string clickhouse:[user_info]host1:port1, ... , hostN:portN[database]?[query_parameters] - // into multiple string for each host: - // clickhouse:[user_info]host1:port1[database]?[query_parameters] - // ... - // clickhouse:[user_info]hostN:portN[database]?[query_parameters] + /** Poco::URI doesn't support several hosts in URI. + * Split string clickhouse:[user_info]host1:port1, ... , hostN:portN[database]?[query_parameters] + * into multiple string for each host: + * clickhouse:[user_info]host1:port1[database]?[query_parameters] + * ... + * clickhouse:[user_info]hostN:portN[database]?[query_parameters] + */ Poco::URI uri; - auto last_host_begin = connection_string.begin() + offset; - for (auto it = last_host_begin; it != hosts_end; ++it) + const auto * last_host_begin = connection_string.begin() + offset; + for (const auto * it = last_host_begin; it != hosts_end; ++it) { if (*it == ',') { @@ -198,7 +199,7 @@ bool tryParseConnectionString( } const auto & database_name = uri.getPath(); - size_t start_symbol = database_name.size() > 0u && database_name[0] == '/' ? 1u : 0u; + size_t start_symbol = !database_name.empty() && database_name[0] == '/' ? 1u : 0u; if (database_name.size() > start_symbol) { common_arguments.push_back("--database"); From cf24d70bfd54dd45606ad1c53a136495c317bb9f Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 8 Jun 2023 02:20:50 +0000 Subject: [PATCH 1354/2223] minor documentation changes --- docs/en/interfaces/cli.md | 18 +++++++++--------- docs/ru/interfaces/cli.md | 10 +++++----- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index ba54694faa9..c36887672c7 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -166,13 +166,13 @@ The connection string for clickhouse-client is presented in URI format: clickhouse://[user_info@][hosts_and_ports][/dbname][?query_parameters] ``` -where user_info is: ```user[:password]``` -and hosts_and_ports is a list of values: ```[host][:port],[host][:port]``` Port is not mandatory. -and query_parameters is a list of parameter[=value]: ```param_name[=value]¶m_name[=value]...``` value may not be required for some of parameters. Parameter names are case sensitive. +where user_info is: `user[:password]` +and hosts_and_ports is a list of values: `[host][:port],[host][:port]` Port is not mandatory. +and query_parameters is a list of parameter[=value]: `param_name[=value]¶m_name[=value]...` value may not be required for some of the parameters. Parameter names are case sensitive. Allowed query_parameters keys: -- **secure** or shorthanded **s** - no value. If specified, client will connect to the server over a secure connection (TLS). See **secure** in [command-line-options](#command-line-options) +- `secure` or shorthanded `s` - no value. If specified, client will connect to the server over a secure connection (TLS). See `secure` in [command-line-options](#command-line-options) These examples illustrate valid connection strings for clickhouse-client: @@ -210,11 +210,11 @@ If more than one host is supplied, or if a single host name is translated to mor ### Percent encoding {#connection_string_uri_percent_encoding} -Hosts, user name, password, database and query parameters should be [Percent-Encoded](https://en.wikipedia.org/wiki/URL_encoding) if values contain URI invalid characters. +Hosts, user name, password, database, and query parameters should be [Percent-Encoded](https://en.wikipedia.org/wiki/URL_encoding) if values contain invalid URI characters. ### Examples {#connection_string_examples} -Connect to localhost using port 9000 and executes the query "SELECT 1". +Connect to localhost using port 9000 and execute the query "SELECT 1". ``` bash clickhouse-client "clickhouse://localhost:9000" --query "SELECT 1" @@ -232,7 +232,7 @@ Connect to localhost using port 9000 in interactive mode with the user specified clickhouse-client "clickhouse://localhost:9000" --user default ``` -Connect to localhost using port 9000 in interactive mode to `my_database` database specified in command line option +Connect to localhost using port 9000 in interactive mode to `my_database` database specified in the command line option. ``` bash clickhouse-client "clickhouse://localhost:9000" --database my_database @@ -250,7 +250,7 @@ Connect to localhost using port 9000 in interactive mode to `my_database` databa clickhouse-client "clickhouse://localhost/my_database?s" ``` -Connect to default host using the default port, default user, and default database. +Connect to default host using default port, default user, and default database. ``` bash clickhouse-client "clickhouse:" @@ -262,7 +262,7 @@ Connect to the default host using the default port, using user user_name and no clickhouse-client "clickhouse://user_name@" ``` -Connect to localhost using email user name. Symbol '@' is percent encoded to '%40'. +Connect to localhost using email as the user name. `@` symbol is percent encoded to `%40`. ``` bash clickhouse-client "clickhouse://some_user%40some_mail.com@localhost:9000" diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 5f119ad9544..801a72e48ec 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -118,14 +118,14 @@ $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="numbe clickhouse://[user_info@][hosts_and_ports][/dbname][?query_parameters] ``` -где user_info - это: ```user[:password]``` -hosts_and_ports - это список значений: ```[host][:port],[host][:port]```. Port может быть не задан. -query_parameters - это список пар ключ[=значение]: ```param_name[=value]¶m_name[=value]...```. Значение может быть пустым. +где user_info - это: `user[:password]` +hosts_and_ports - это список значений: `[host][:port],[host][:port]`. Port может быть не задан. +query_parameters - это список пар ключ[=значение]: `param_name[=value]¶m_name[=value]...`. Значение может быть пустым. Имена параметров чувствительны к регистру. Допустимые ключи query_parameters: -- **secure** или сокращенно **s** - без значение. Если параметр указан, то соединение с сервером будет осуществляться по защищенному каналу (TLS). См. **secure** в [command-line-options](#command-line-options). +- `secure` или сокращенно `s` - без значение. Если параметр указан, то соединение с сервером будет осуществляться по защищенному каналу (TLS). См. `secure` в [command-line-options](#command-line-options). Эти примеры иллюстрируют допустимые строки подключения для clickhouse-client: @@ -215,7 +215,7 @@ clickhouse-client "clickhouse:" clickhouse-client "clickhouse://user_name@" ``` -Подключиться к localhost, используя электронную почту, как имя пользователя. Символ '@' закодирован как '%40'. +Подключиться к localhost, используя электронную почту, как имя пользователя. Символ `@` закодирован как `%40`. ``` bash clickhouse-client "clickhouse://some_user%40some_mail.com@localhost:9000" From 286f3b247b33b08b17bd76320604d9b2b1c282b4 Mon Sep 17 00:00:00 2001 From: Derek Chia Date: Thu, 8 Jun 2023 11:28:44 +0800 Subject: [PATCH 1355/2223] Update settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8e2cd8d6027..3968751d5ee 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3155,7 +3155,7 @@ Possible values: - Positive integer. - 0 or 1 — Disabled. `SELECT` queries are executed in a single thread. -Default value: the number of physical CPU cores. +Default value: `max_threads`. ## opentelemetry_start_trace_probability {#opentelemetry-start-trace-probability} From 7263769d20a30747f7a80a45a6def3abf41cccfa Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 8 Jun 2023 06:12:54 +0000 Subject: [PATCH 1356/2223] Add constexpr / fix date check --- src/Functions/DateTimeTransforms.h | 8 ++------ .../0_stateless/01746_convert_type_with_default.reference | 1 - .../0_stateless/01746_convert_type_with_default.sql | 1 - 3 files changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 823272e0324..09b0d71daf8 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1449,18 +1449,14 @@ struct Transformer { bool check_range_result = true; - if constexpr (std::is_same_v) - { - check_range_result = vec_from[i] >= 0 && vec_from[i] <= DATE_LUT_MAX_DAY_NUM; - } - else if constexpr (std::is_same_v) + if constexpr (std::is_same_v || std::is_same_v) { check_range_result = vec_from[i] >= 0 && vec_from[i] <= 0xFFFFFFFFL; } if (!check_range_result) { - if (std::is_same_v) + if constexpr (std::is_same_v) { vec_to[i] = 0; if (vec_null_map_to) diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.reference b/tests/queries/0_stateless/01746_convert_type_with_default.reference index 85bf2064fdc..959ee29b5e7 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.reference +++ b/tests/queries/0_stateless/01746_convert_type_with_default.reference @@ -45,7 +45,6 @@ 2023-05-30 14:38:20 2023-05-30 14:38:20 2023-05-30 14:38:20 -2023-05-30 14:38:20 1970-01-01 00:00:19 1970-01-01 00:00:19 1970-01-01 00:00:19 diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index 1065eefa94d..099652a8a39 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -56,7 +56,6 @@ select toDateOrDefault(cast(19 as UInt256)); select toDateOrDefault(19507, '2000-01-01'::Date); select toDateOrDefault(-1, '2023-05-30'::Date); -select toDateTimeOrDefault('2023-05-30 14:38:20'); select toDateTimeOrDefault('2023-05-30 14:38:20', 'UTC'); select toDateTimeOrDefault('1xxx', 'UTC', '2023-05-30 14:38:20'::DateTime('UTC')); select toDateTimeOrDefault(1685457500, 'UTC'); From f5816c27aa3c675c3cd02ce292675d2214e5e56f Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 8 Jun 2023 06:20:14 +0000 Subject: [PATCH 1357/2223] Use hex value in tests --- tests/queries/0_stateless/01601_accurate_cast.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01601_accurate_cast.sql b/tests/queries/0_stateless/01601_accurate_cast.sql index f7f4d588ccc..5555129f0ad 100644 --- a/tests/queries/0_stateless/01601_accurate_cast.sql +++ b/tests/queries/0_stateless/01601_accurate_cast.sql @@ -24,13 +24,13 @@ SELECT accurateCast('123', 'FixedString(2)'); -- { serverError 131 } SELECT accurateCast('12', 'FixedString(2)'); SELECT accurateCast(-1, 'DateTime'); -- { serverError 70 } -SELECT accurateCast(5000000000, 'DateTime'); -- { serverError 70 } +SELECT accurateCast(0xFFFFFFFF + 1, 'DateTime'); -- { serverError 70 } SELECT accurateCast('1xxx', 'DateTime'); -- { serverError 41 } SELECT accurateCast('2023-05-30 14:38:20', 'DateTime'); SELECT toString(accurateCast(19, 'DateTime'), 'UTC'); SELECT accurateCast(-1, 'Date'); -- { serverError 70 } -SELECT accurateCast(999999, 'Date'); -- { serverError 70 } +SELECT accurateCast(0xFFFFFFFF + 1, 'Date'); -- { serverError 70 } SELECT accurateCast('1xxx', 'Date'); -- { serverError 38 } SELECT accurateCast('2023-05-30', 'Date'); SELECT accurateCast(19, 'Date'); From 2297995c1a46240145df4b04ba7fff727944f5e7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 8 Jun 2023 07:15:11 +0000 Subject: [PATCH 1358/2223] Disable grace_hash join in stress tests Until https://github.com/ClickHouse/ClickHouse/issues/50220 is fixed --- tests/ci/stress.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index b9044874071..e370ddbdd21 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -37,9 +37,6 @@ def get_options(i, upgrade_check): client_options.append("join_algorithm='partial_merge'") if join_alg_num % 5 == 2: client_options.append("join_algorithm='full_sorting_merge'") - if join_alg_num % 5 == 3 and not upgrade_check: - # Some crashes are not fixed in 23.2 yet, so ignore the setting in Upgrade check - client_options.append("join_algorithm='grace_hash'") if join_alg_num % 5 == 4: client_options.append("join_algorithm='auto'") client_options.append("max_rows_in_join=1000") From b878ddb35fce2247d657e3d2c2156023b4c67bdd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 8 Jun 2023 11:02:09 +0200 Subject: [PATCH 1359/2223] Fix test --- tests/queries/0_stateless/02344_describe_cache.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index a803ca1fab1..d8a2ffab1fa 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1,2 @@ -134217728 1048576 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 -134217728 1048576 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 +134217728 10000000 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 +134217728 10000000 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 From 17a560cca7cd5bb71bf673fafea76695855b3c6a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 8 Jun 2023 09:11:02 +0000 Subject: [PATCH 1360/2223] Add datetime test --- .../0_stateless/01746_convert_type_with_default.reference | 2 ++ tests/queries/0_stateless/01746_convert_type_with_default.sql | 3 +++ 2 files changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.reference b/tests/queries/0_stateless/01746_convert_type_with_default.reference index 959ee29b5e7..541580d67f5 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.reference +++ b/tests/queries/0_stateless/01746_convert_type_with_default.reference @@ -39,6 +39,8 @@ 1970-01-20 1970-01-20 1970-01-20 +2149-06-06 +1970-01-01 2023-05-30 2023-05-30 2023-05-30 14:38:20 diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index 099652a8a39..2620780cfb9 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -53,6 +53,9 @@ select toDateOrDefault(cast(19 as UInt128)); select toDateOrDefault(cast(19 as Int256)); select toDateOrDefault(cast(19 as UInt256)); +select toDateOrDefault(65535); +select toDateOrDefault(65536); + select toDateOrDefault(19507, '2000-01-01'::Date); select toDateOrDefault(-1, '2023-05-30'::Date); From 15b6651df6f10e7f623ec74dd3aaa42b82df90a5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 8 Jun 2023 09:40:11 +0000 Subject: [PATCH 1361/2223] Revert "Merge pull request #49816 from bigo-sg/grace_hash_reserve_hash_table" This reverts commit 3f892ceb12b868b1b9deb0607a7df2dde5f1a139, reversing changes made to 32ffa2ae0b781af5c7ae3e5cbf975d3e9b74d86f. --- src/Interpreters/GraceHashJoin.cpp | 17 +++++------------ src/Interpreters/GraceHashJoin.h | 3 +-- src/Interpreters/HashJoin.cpp | 8 +++----- src/Interpreters/HashJoin.h | 30 ++---------------------------- 4 files changed, 11 insertions(+), 47 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index f54ee9d85c7..4a4c69ff473 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -571,13 +571,7 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; - size_t prev_keys_num = 0; - // If there is only one bucket, don't take this check. - if (hash_join && buckets.size() > 1) - { - // Use previous hash_join's keys number to estimate next hash_join's size is reasonable. - prev_keys_num = hash_join->getTotalRowCount(); - } + hash_join = makeInMemoryJoin(); for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) { @@ -591,7 +585,6 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() continue; } - hash_join = makeInMemoryJoin(prev_keys_num); auto right_reader = current_bucket->startJoining(); size_t num_rows = 0; /// count rows that were written and rehashed while (Block block = right_reader.read()) @@ -611,9 +604,9 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() return nullptr; } -GraceHashJoin::InMemoryJoinPtr GraceHashJoin::makeInMemoryJoin(size_t reserve_num) +GraceHashJoin::InMemoryJoinPtr GraceHashJoin::makeInMemoryJoin() { - return std::make_unique(table_join, right_sample_block, any_take_last_row, reserve_num); + return std::make_unique(table_join, right_sample_block, any_take_last_row); } Block GraceHashJoin::prepareRightBlock(const Block & block) @@ -653,7 +646,6 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) if (!current_block.rows()) return; } - auto prev_keys_num = hash_join->getTotalRowCount(); hash_join->addJoinedBlock(current_block, /* check_limits = */ false); if (!hasMemoryOverflow(hash_join)) @@ -662,6 +654,7 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) current_block = {}; auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); + hash_join = nullptr; buckets_snapshot = rehashBuckets(buckets_snapshot.size() * 2); @@ -681,7 +674,7 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) current_block = concatenateBlocks(current_blocks); } - hash_join = makeInMemoryJoin(prev_keys_num); + hash_join = makeInMemoryJoin(); if (current_block.rows() > 0) hash_join->addJoinedBlock(current_block, /* check_limits = */ false); diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index ec611f373ed..eb39ee09208 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -90,8 +90,7 @@ public: private: void initBuckets(); /// Create empty join for in-memory processing. - /// reserve_num for reserving space in hash table. - InMemoryJoinPtr makeInMemoryJoin(size_t reserve_num = 0); + InMemoryJoinPtr makeInMemoryJoin(); /// Add right table block to the @join. Calls @rehash on overflow. void addJoinedBlockImpl(Block block); diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 146b57049a6..0af33a8bd20 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -217,7 +217,7 @@ static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nulla JoinCommon::removeColumnNullability(column); } -HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_sample_block_, bool any_take_last_row_, size_t reserve_num) +HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_sample_block_, bool any_take_last_row_) : table_join(table_join_) , kind(table_join->kind()) , strictness(table_join->strictness()) @@ -302,7 +302,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s } for (auto & maps : data->maps) - dataMapInit(maps, reserve_num); + dataMapInit(maps); } HashJoin::Type HashJoin::chooseMethod(JoinKind kind, const ColumnRawPtrs & key_columns, Sizes & key_sizes) @@ -454,15 +454,13 @@ struct KeyGetterForType using Type = typename KeyGetterForTypeImpl::Type; }; -void HashJoin::dataMapInit(MapsVariant & map, size_t reserve_num) +void HashJoin::dataMapInit(MapsVariant & map) { if (kind == JoinKind::Cross) return; joinDispatchInit(kind, strictness, map); joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.create(data->type); }); - if (reserve_num) - joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); }); } bool HashJoin::empty() const diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 58e47432d41..50eda4482bd 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -146,7 +146,7 @@ public: class HashJoin : public IJoin { public: - HashJoin(std::shared_ptr table_join_, const Block & right_sample_block, bool any_take_last_row_ = false, size_t reserve_num = 0); + HashJoin(std::shared_ptr table_join_, const Block & right_sample_block, bool any_take_last_row_ = false); ~HashJoin() override; @@ -217,16 +217,6 @@ public: M(keys256) \ M(hashed) - /// Only for maps using hash table. - #define APPLY_FOR_HASH_JOIN_VARIANTS(M) \ - M(key32) \ - M(key64) \ - M(key_string) \ - M(key_fixed_string) \ - M(keys128) \ - M(keys256) \ - M(hashed) - /// Used for reading from StorageJoin and applying joinGet function #define APPLY_FOR_JOIN_VARIANTS_LIMITED(M) \ @@ -276,22 +266,6 @@ public: } } - void reserve(Type which, size_t num) - { - switch (which) - { - case Type::EMPTY: break; - case Type::CROSS: break; - case Type::key8: break; - case Type::key16: break; - - #define M(NAME) \ - case Type::NAME: NAME->reserve(num); break; - APPLY_FOR_HASH_JOIN_VARIANTS(M) - #undef M - } - } - size_t getTotalRowCount(Type which) const { switch (which) @@ -435,7 +409,7 @@ private: /// If set HashJoin instance is not available for modification (addJoinedBlock) TableLockHolder storage_join_lock = nullptr; - void dataMapInit(MapsVariant &, size_t); + void dataMapInit(MapsVariant &); void initRightBlockStructure(Block & saved_block_sample); From f0a1c8afa208cbc0ec7a0b056a7d2036d62e10e0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 8 Jun 2023 09:40:41 +0000 Subject: [PATCH 1362/2223] Revert "Merge pull request #49483 from bigo-sg/grace_hash_full_join" This reverts commit fa93c388b1d17f3561c63e9265c8aefde1b2d5d0, reversing changes made to a2c0a65344f4197c6c2849f800423309fd4b22b4. --- docs/en/operations/settings/settings.md | 2 - src/Interpreters/GraceHashJoin.cpp | 55 +++++--------- src/Interpreters/GraceHashJoin.h | 3 +- .../Transforms/JoiningTransform.cpp | 73 ++----------------- src/Processors/Transforms/JoiningTransform.h | 25 +------ src/QueryPipeline/QueryPipelineBuilder.cpp | 2 +- ...01721_join_implicit_cast_long.reference.j2 | 40 ++++++++++ .../01721_join_implicit_cast_long.sql.j2 | 1 + .../02273_full_sort_join.reference.j2 | 18 ++++- .../0_stateless/02273_full_sort_join.sql.j2 | 4 +- ...274_full_sort_join_nodistinct.reference.j2 | 34 ++++++++- .../02274_full_sort_join_nodistinct.sql.j2 | 6 +- .../02275_full_sort_join_long.reference | 24 +----- .../02275_full_sort_join_long.sql.j2 | 9 ++- 14 files changed, 138 insertions(+), 158 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4f3b4e43358..f674fe1781e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -452,8 +452,6 @@ Possible values: The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which don’t belong to the current bucket are flushed and reassigned. - Supports `INNER/LEFT/RIGHT/FULL ALL/ANY JOIN`. - - hash [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 4a4c69ff473..7795061072c 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -304,10 +304,8 @@ void GraceHashJoin::initBuckets() bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) { - bool is_asof = (table_join->strictness() == JoinStrictness::Asof); - auto kind = table_join->kind(); - return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct(); + return !is_asof && isInnerOrLeft(table_join->kind()) && table_join->oneDisjunct(); } GraceHashJoin::~GraceHashJoin() = default; @@ -327,6 +325,7 @@ bool GraceHashJoin::hasMemoryOverflow(size_t total_rows, size_t total_bytes) con /// One row can't be split, avoid loop if (total_rows < 2) return false; + bool has_overflow = !table_join->sizeLimits().softCheck(total_rows, total_bytes); if (has_overflow) @@ -471,30 +470,18 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -/// Each bucket are handled by the following steps -/// 1. build hash_join by the right side blocks. -/// 2. join left side with the hash_join, -/// 3. read right non-joined blocks from hash_join. -/// buckets are handled one by one, each hash_join will not be release before the right non-joined blocks are emitted. -/// -/// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform, -/// only one processor could take the non-joined blocks from right stream, and ensure all rows from -/// left stream have been emitted before this. -IBlocksStreamPtr -GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const + +IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block &, const Block &, UInt64) const { - return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); + /// We do no support returning non joined blocks here. + /// TODO: They _should_ be reported by getDelayedBlocks instead + return nullptr; } class GraceHashJoin::DelayedBlocks : public IBlocksStream { public: - explicit DelayedBlocks( - size_t current_bucket_, - Buckets buckets_, - InMemoryJoinPtr hash_join_, - const Names & left_key_names_, - const Names & right_key_names_) + explicit DelayedBlocks(size_t current_bucket_, Buckets buckets_, InMemoryJoinPtr hash_join_, const Names & left_key_names_, const Names & right_key_names_) : current_bucket(current_bucket_) , buckets(std::move(buckets_)) , hash_join(std::move(hash_join_)) @@ -512,15 +499,12 @@ public: do { - // One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransform. - // There is a lock inside left_reader.read(). block = left_reader.read(); if (!block) { return {}; } - // block comes from left_reader, need to join with right table to get the result. Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets); block = std::move(blocks[current_idx]); @@ -571,6 +555,16 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() size_t bucket_idx = current_bucket->idx; + if (hash_join) + { + auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); + for (auto & block : right_blocks) + { + Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, block, buckets.size()); + flushBlocksToBuckets(blocks, buckets, bucket_idx); + } + } + hash_join = makeInMemoryJoin(); for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) @@ -595,6 +589,7 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() LOG_TRACE(log, "Loaded bucket {} with {}(/{}) rows", bucket_idx, hash_join->getTotalRowCount(), num_rows); + return std::make_unique(current_bucket->idx, buckets, hash_join, left_key_names, right_key_names); } @@ -634,18 +629,6 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) if (!hash_join) hash_join = makeInMemoryJoin(); - // buckets size has been changed in other threads. Need to scatter current_block again. - // rehash could only happen under hash_join_mutex's scope. - auto current_buckets = getCurrentBuckets(); - if (buckets_snapshot.size() != current_buckets.size()) - { - LOG_TRACE(log, "mismatch buckets size. previous:{}, current:{}", buckets_snapshot.size(), getCurrentBuckets().size()); - Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, current_block, current_buckets.size()); - flushBlocksToBuckets(blocks, current_buckets, bucket_index); - current_block = std::move(blocks[bucket_index]); - if (!current_block.rows()) - return; - } hash_join->addJoinedBlock(current_block, /* check_limits = */ false); if (!hasMemoryOverflow(hash_join)) diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index eb39ee09208..b8d83f4cad0 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -13,6 +13,7 @@ namespace DB { + class TableJoin; class HashJoin; @@ -78,7 +79,7 @@ public: bool supportTotals() const override { return false; } IBlocksStreamPtr - getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size) const override; + getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; /// Open iterator over joined blocks. /// Must be called after all @joinBlock calls. diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 256ef66a27d..bba8ec6fa16 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -189,6 +189,7 @@ void JoiningTransform::transform(Chunk & chunk) } else block = readExecute(chunk); + auto num_rows = block.rows(); chunk.setColumns(block.getColumns(), num_rows); } @@ -310,16 +311,8 @@ void FillingRightJoinSideTransform::work() } -DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform( - Block left_header_, - Block output_header_, - size_t max_block_size_, - JoinPtr join_) - : IProcessor(InputPorts{Block()}, OutputPorts{output_header_}) - , left_header(left_header_) - , output_header(output_header_) - , max_block_size(max_block_size_) - , join(join_) +DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform(Block output_header) + : IProcessor(InputPorts{Block()}, OutputPorts{output_header}) { } @@ -373,7 +366,6 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (!data.chunk.hasChunkInfo()) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); - } else { @@ -394,24 +386,12 @@ void DelayedJoinedBlocksWorkerTransform::work() { if (!task) return; - Block block; - if (!left_delayed_stream_finished) - { - block = task->delayed_blocks->next(); - if (!block) - { - left_delayed_stream_finished = true; - block = nextNonJoinedBlock(); - } - } - else - { - block = nextNonJoinedBlock(); - } + Block block = task->delayed_blocks->next(); + if (!block) { - resetTask(); + task.reset(); return; } @@ -420,38 +400,6 @@ void DelayedJoinedBlocksWorkerTransform::work() output_chunk.setColumns(block.getColumns(), rows); } -void DelayedJoinedBlocksWorkerTransform::resetTask() -{ - task.reset(); - left_delayed_stream_finished = false; - setup_non_joined_stream = false; - non_joined_delayed_stream = nullptr; -} - -Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() -{ - if (!setup_non_joined_stream) - { - setup_non_joined_stream = true; - // Before read from non-joined stream, all blocks in left file reader must have been joined. - // For example, in HashJoin, it may return invalid mismatch rows from non-joined stream before - // the all blocks in left file reader have been finished, since the used flags are incomplete. - // To make only one processor could read from non-joined stream seems be a easy way. - if (task && task->left_delayed_stream_finish_counter->isLast()) - { - if (!non_joined_delayed_stream) - { - non_joined_delayed_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); - } - } - } - if (non_joined_delayed_stream) - { - return non_joined_delayed_stream->next(); - } - return {}; -} - DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, JoinPtr join_) : IProcessor(InputPorts{}, OutputPorts(num_streams, Block())) , join(std::move(join_)) @@ -485,9 +433,6 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (finished) { - // Since have memory limit, cannot handle all buckets parallelly by different - // DelayedJoinedBlocksWorkerTransform. So send the same task to all outputs. - // Wait for all DelayedJoinedBlocksWorkerTransform be idle before getting next bucket. for (auto & output : outputs) { if (output.isFinished()) @@ -503,14 +448,10 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (delayed_blocks) { - // This counter is used to ensure that only the last DelayedJoinedBlocksWorkerTransform - // could read right non-joined blocks from the join. - auto left_delayed_stream_finished_counter = std::make_shared(outputs.size()); for (auto & output : outputs) { Chunk chunk; - auto task = std::make_shared(delayed_blocks, left_delayed_stream_finished_counter); - chunk.setChunkInfo(task); + chunk.setChunkInfo(std::make_shared(delayed_blocks)); output.push(std::move(chunk)); } delayed_blocks = nullptr; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 3577906b26a..e7edff40c56 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -116,14 +116,9 @@ class DelayedBlocksTask : public ChunkInfo public: explicit DelayedBlocksTask() : finished(true) {} - explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) - : delayed_blocks(std::move(delayed_blocks_)) - , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) - { - } + explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_) : delayed_blocks(std::move(delayed_blocks_)) {} IBlocksStreamPtr delayed_blocks = nullptr; - JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter = nullptr; bool finished = false; }; @@ -152,11 +147,7 @@ private: class DelayedJoinedBlocksWorkerTransform : public IProcessor { public: - explicit DelayedJoinedBlocksWorkerTransform( - Block left_header_, - Block output_header_, - size_t max_block_size_, - JoinPtr join_); + explicit DelayedJoinedBlocksWorkerTransform(Block output_header); String getName() const override { return "DelayedJoinedBlocksWorkerTransform"; } @@ -164,20 +155,10 @@ public: void work() override; private: - Block left_header; - Block output_header; - size_t max_block_size; - JoinPtr join; DelayedBlocksTaskPtr task; Chunk output_chunk; - /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left - bool left_delayed_stream_finished = false; - bool setup_non_joined_stream = false; - IBlocksStreamPtr non_joined_delayed_stream = nullptr; - - void resetTask(); - Block nextNonJoinedBlock(); + bool finished = false; }; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 764997e7b7e..a4edf107b2f 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -491,7 +491,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (delayed_root) { // Process delayed joined blocks when all JoiningTransform are finished. - auto delayed = std::make_shared(left_header, joined_header, max_block_size, join); + auto delayed = std::make_shared(joined_header); if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output"); diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 index ae43aa7195c..e9f32087439 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 @@ -1,6 +1,7 @@ {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} === {{ join_algorithm }} === = full = +{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -16,6 +17,7 @@ 8 108 \N 9 109 \N 10 110 \N +{% endif -%} = left = 1 101 201 2 102 202 @@ -28,6 +30,7 @@ 9 109 \N 10 110 \N = right = +{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -38,6 +41,7 @@ 3 103 203 4 104 204 5 105 205 +{% endif -%} = inner = 1 101 201 2 102 202 @@ -45,6 +49,7 @@ 4 104 204 5 105 205 = full = +{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -60,6 +65,7 @@ 8 8 0 9 9 0 10 10 0 +{% endif -%} = left = 1 1 1 2 2 2 @@ -72,6 +78,7 @@ 9 9 0 10 10 0 = right = +{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -82,6 +89,7 @@ 3 3 3 4 4 4 5 5 5 +{% endif -%} = inner = 1 1 1 2 2 2 @@ -90,6 +98,7 @@ 5 5 5 = join on = = full = +{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -105,6 +114,7 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N +{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -117,6 +127,7 @@ 9 109 0 \N 10 110 0 \N = right = +{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -127,6 +138,7 @@ 3 103 3 203 4 104 4 204 5 105 5 205 +{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -134,6 +146,7 @@ 4 104 4 204 5 105 5 205 = full = +{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -149,6 +162,7 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N +{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -161,6 +175,7 @@ 9 109 0 \N 10 110 0 \N = right = +{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -171,6 +186,7 @@ 3 103 3 203 4 104 4 204 5 105 5 205 +{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -180,6 +196,7 @@ = agg = 1 1 +{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -188,11 +205,13 @@ 1 55 1055 0 0 -10 0 990 1 55 15 1055 1015 +{% endif -%} = types = 1 1 1 1 +{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -200,9 +219,11 @@ 1 1 1 +{% endif -%} {% if join_algorithm not in ['full_sorting_merge'] -%} === join use nulls === = full = +{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -218,6 +239,7 @@ 8 108 \N 9 109 \N 10 110 \N +{% endif -%} = left = 1 101 201 2 102 202 @@ -230,6 +252,7 @@ 9 109 \N 10 110 \N = right = +{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -240,6 +263,7 @@ 3 103 203 4 104 204 5 105 205 +{% endif -%} = inner = 1 101 201 2 102 202 @@ -247,6 +271,7 @@ 4 104 204 5 105 205 = full = +{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -262,6 +287,7 @@ \N \N -2 \N \N -1 \N \N 0 +{% endif -%} = left = 1 1 1 2 2 2 @@ -274,6 +300,7 @@ 9 9 \N 10 10 \N = right = +{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -284,6 +311,7 @@ \N \N -2 \N \N -1 \N \N 0 +{% endif -%} = inner = 1 1 1 2 2 2 @@ -292,6 +320,7 @@ 5 5 5 = join on = = full = +{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -307,6 +336,7 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 +{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -319,6 +349,7 @@ 9 109 \N \N 10 110 \N \N = right = +{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -329,6 +360,7 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 +{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -336,6 +368,7 @@ 4 104 4 204 5 105 5 205 = full = +{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -351,6 +384,7 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 +{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -363,6 +397,7 @@ 9 109 \N \N 10 110 \N \N = right = +{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -373,6 +408,7 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 +{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -382,6 +418,7 @@ = agg = 1 1 +{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -390,11 +427,13 @@ 1 55 1055 1 55 15 1055 1015 \N \N -10 \N 990 +{% endif -%} = types = 1 1 1 1 +{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -403,4 +442,5 @@ 1 1 {% endif -%} +{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 index 38f71f4c5ec..f5321939f28 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 @@ -10,6 +10,7 @@ INSERT INTO t1 SELECT number as a, 100 + number as b FROM system.numbers LIMIT 1 INSERT INTO t2 SELECT number - 5 as a, 200 + number - 5 as b FROM system.numbers LIMIT 1, 10; {% macro is_implemented(join_algorithm) -%} +{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 index 0af4158e971..98bfd9d9b2b 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 @@ -1,7 +1,7 @@ {% set table_size = 15 -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} -- {{ join_algorithm }} -- -{% for block_size in range(1, table_size + 1, 4) -%} +{% for block_size in range(1, table_size + 1) -%} ALL INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 @@ -50,6 +50,7 @@ ALL LEFT | bs = {{ block_size }} 14 14 val9 0 14 14 val9 0 ALL RIGHT | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -63,6 +64,7 @@ ALL RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 +{% endif -%} ALL INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -83,6 +85,7 @@ ALL LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | bs = {{ block_size }} | copmosite key +{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -96,6 +99,7 @@ ALL RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 +{% endif -%} ANY INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 @@ -133,6 +137,7 @@ ANY LEFT | bs = {{ block_size }} 13 13 val13 0 14 14 val9 0 ANY RIGHT | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -145,6 +150,7 @@ ANY RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 +{% endif -%} ANY INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | bs = {{ block_size }} | copmosite key @@ -164,6 +170,7 @@ ANY LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | bs = {{ block_size }} | copmosite key +{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -176,6 +183,7 @@ ANY RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 1 \N 1 val3 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 +{% endif -%} {% endfor -%} ALL INNER | join_use_nulls = 1 4 4 0 0 @@ -211,6 +219,7 @@ ALL LEFT | join_use_nulls = 1 14 14 val9 0 14 14 val9 0 ALL RIGHT | join_use_nulls = 1 +{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -224,6 +233,7 @@ ALL RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 +{% endif -%} ALL INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -244,6 +254,7 @@ ALL LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | join_use_nulls = 1 | copmosite key +{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 @@ -257,6 +268,7 @@ ALL RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 +{% endif -%} ANY INNER | join_use_nulls = 1 4 4 0 0 5 5 0 0 @@ -284,6 +296,7 @@ ANY LEFT | join_use_nulls = 1 13 13 val13 0 14 14 val9 0 ANY RIGHT | join_use_nulls = 1 +{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -296,6 +309,7 @@ ANY RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 +{% endif -%} ANY INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | join_use_nulls = 1 | copmosite key @@ -315,6 +329,7 @@ ANY LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | join_use_nulls = 1 | copmosite key +{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 \N \N \N 1 1 1 \N val7 @@ -327,4 +342,5 @@ ANY RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 +{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 6500306356c..8b739330364 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -26,7 +26,9 @@ INSERT INTO t2 'val' || toString(number) as s FROM numbers_mt({{ table_size - 3 }}); + {% macro is_implemented(join_algorithm) -%} +{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} @@ -36,7 +38,7 @@ SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}10K{% else %}0{% SELECT '-- {{ join_algorithm }} --'; SET join_algorithm = '{{ join_algorithm }}'; -{% for block_size in range(1, table_size + 1, 4) -%} +{% for block_size in range(1, table_size + 1) -%} {% for kind in ['ALL', 'ANY'] -%} SET max_block_size = {{ block_size }}; diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index df968e86e8d..2cc6c6e85d6 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -1,6 +1,6 @@ {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} --- {{ join_algorithm }} --- -{% for block_size in range(1, 11, 4) -%} +{% for block_size in range(1, 11) -%} t1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -108,6 +108,7 @@ t1 ALL LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -160,6 +161,7 @@ t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val28 2 2 5 val28 3 3 4 val3 +{% endif -%} t1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 2 2 5 5 @@ -175,6 +177,7 @@ t1 ANY LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -185,7 +188,9 @@ t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val27 2 2 5 val28 3 3 4 val3 +{% endif -%} t1 ALL FULL JOIN t2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 2 2 5 5 @@ -238,7 +243,9 @@ t1 ALL FULL JOIN t2 | bs = {{ block_size }} 2 2 5 5 2 2 5 5 3 3 4 4 +{% endif -%} t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 5 5 @@ -291,6 +298,7 @@ t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} 2 5 5 2 5 5 3 4 4 +{% endif -%} t1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -307,6 +315,7 @@ t1 ALL LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -317,6 +326,7 @@ t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 +{% endif -%} t1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -331,6 +341,7 @@ t1 ANY LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -341,7 +352,9 @@ t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 +{% endif -%} t1 ALL FULL JOIN tn2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 0 \N 0 5 0 \N 0 5 0 \N 0 5 @@ -359,8 +372,9 @@ t1 ALL FULL JOIN tn2 | bs = {{ block_size }} 2 \N 5 0 2 \N 5 0 3 3 4 4 -{% if join_algorithm != 'grace_hash' -%} +{% endif -%} t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 5 0 @@ -395,6 +409,7 @@ tn1 ALL LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -405,6 +420,7 @@ tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 +{% endif -%} tn1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -419,6 +435,7 @@ tn1 ANY LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -429,7 +446,9 @@ tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 +{% endif -%} tn1 ALL FULL JOIN t2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -447,7 +466,9 @@ tn1 ALL FULL JOIN t2 | bs = {{ block_size }} \N 2 0 5 \N 2 0 5 \N 2 0 5 +{% endif -%} tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 0 5 @@ -465,6 +486,7 @@ tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} \N 5 0 \N 5 0 \N 5 0 +{% endif -%} tn1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -481,6 +503,7 @@ tn1 ALL LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -491,6 +514,7 @@ tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 +{% endif -%} tn1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -505,6 +529,7 @@ tn1 ANY LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -515,7 +540,9 @@ tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 +{% endif -%} tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -533,8 +560,9 @@ tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} \N \N 5 0 \N \N 5 0 \N \N 5 0 -{% if join_algorithm != 'grace_hash' -%} +{% endif -%} tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} +{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 3 4 4 diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index f8eb4b1a53e..613da65421e 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -16,6 +16,7 @@ INSERT INTO t2 VALUES (1, 'val11'), (1, 'val12'), (2, 'val22'), (2, 'val23'), (2 INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (NULL, 'val28'), (3, 'val3'); {% macro is_implemented(join_algorithm) -%} +{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} @@ -26,7 +27,7 @@ SET join_algorithm = '{{ join_algorithm }}'; SELECT '--- {{ join_algorithm }} ---'; -{% for block_size in range(1, 11, 4) -%} +{% for block_size in range(1, 11) -%} SET max_block_size = {{ block_size }}; {% for t1, t2 in [('t1', 't2'), ('t1', 'tn2'), ('tn1', 't2'), ('tn1', 'tn2')] -%} @@ -46,10 +47,9 @@ SELECT t1.key, t2.key, length(t1.s), t2.s FROM {{ t1 }} AS t1 {{ kind }} RIGHT J SELECT '{{ t1 }} ALL FULL JOIN {{ t2 }} | bs = {{ block_size }}'; SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} FULL JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} -{% if join_algorithm == 'full_sorting_merge' or t2 != 'tn2' -%} SELECT '{{ t1 }} ALL FULL JOIN USING {{ t2 }} | bs = {{ block_size }}'; SELECT key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 ALL FULL JOIN {{ t2 }} AS t2 USING (key) ORDER BY key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} -{% endif -%} + {% endfor -%} {% endfor -%} SET max_bytes_in_join = 0; diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.reference b/tests/queries/0_stateless/02275_full_sort_join_long.reference index 73482358d12..9ec06aea3e6 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.reference +++ b/tests/queries/0_stateless/02275_full_sort_join_long.reference @@ -41,34 +41,16 @@ ALL INNER ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -500353531835 684008812186 1367170 1000342 1367170 +skipped ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -500353531835 684008812186 1367170 1000342 1367170 +skipped ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -500353531835 684008812186 1367170 1000342 1367170 -ANY INNER -199622811843 199622811843 399458 399458 399458 -ANY LEFT -50010619420459 315220291655 10000000 10000000 630753 -ANY RIGHT -316611844056 500267124407 1000000 633172 1000000 -ANY INNER -199622811843 199622811843 399458 399458 399458 -ANY LEFT -50010619420459 315220291655 10000000 10000000 630753 -ANY RIGHT -316611844056 500267124407 1000000 633172 1000000 -ANY INNER -199622811843 199622811843 399458 399458 399458 -ANY LEFT -50010619420459 315220291655 10000000 10000000 630753 -ANY RIGHT -316611844056 500267124407 1000000 633172 1000000 +skipped diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 0b28fd67050..98cc46c9cb4 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -22,6 +22,11 @@ INSERT INTO t2 FROM numbers_mt({{ rtable_size }}) ; +{% macro is_implemented(join_algorithm) -%} +{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } +SELECT 'skipped'; +{% endif -%} +{% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} @@ -35,6 +40,7 @@ SET join_algorithm = '{{ join_algorithm }}'; SET max_block_size = {{ block_size }}; +{% if not (kind == 'ANY' and join_algorithm == 'grace_hash') -%} SELECT '{{ kind }} INNER'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 @@ -52,8 +58,9 @@ SELECT '{{ kind }} RIGHT'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 {{ kind }} RIGHT JOIN t2 ON t1.key == t2.key -; +; {{ is_implemented(join_algorithm) }} +{% endif -%} {% endfor -%} {% endfor -%} From 44c68ffdab34642192f16fb1f9ecb9bf96bdd73b Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 8 Jun 2023 10:17:02 +0000 Subject: [PATCH 1363/2223] Add config param allow_remove_stale_moving_parts --- src/Storages/MergeTree/MergeTreeData.cpp | 15 +++++++++++++-- src/Storages/MergeTree/MergeTreeData.h | 3 +++ src/Storages/MergeTree/MergeTreePartsMover.cpp | 7 ++++--- .../configs/config.d/storage_conf.xml | 2 ++ .../configs/remote_servers.xml | 1 + .../test_encrypted_disk/configs/storage.xml | 1 + .../configs/config.xml | 1 + .../configs/config.d/storage_conf.xml | 1 + .../test_merge_tree_s3/configs/config.xml | 1 + .../configs/config.d/storage_configuration.xml | 1 + .../configs/config.d/storage_configuration.xml | 1 + .../configs/config.d/storage_configuration.xml | 1 + .../configs/config.d/storage_conf.xml | 1 + .../configs/config.d/s3.xml | 1 + .../configs/config.d/storage_configuration.xml | 1 + .../test_zero_copy_fetch/configs/storage_conf.xml | 1 + 16 files changed, 34 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7fe3efaf6d5..9ce4e55e341 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -1998,14 +1999,19 @@ static bool isOldPartDirectory(const DiskPtr & disk, const String & directory_pa return true; } + size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes) { size_t cleared_count = 0; cleared_count += clearOldTemporaryDirectories(relative_data_path, custom_directories_lifetime_seconds, valid_prefixes); - /// Clear _all_ parts from the `moving` directory - cleared_count += clearOldTemporaryDirectories(fs::path(relative_data_path) / "moving", custom_directories_lifetime_seconds, {""}); + if (allowRemoveStaleMovingParts()) + { + /// Clear _all_ parts from the `moving` directory + cleared_count += clearOldTemporaryDirectories(fs::path(relative_data_path) / "moving", custom_directories_lifetime_seconds, {""}); + } + return cleared_count; } @@ -8412,6 +8418,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createEmptyPart( return new_data_part; } +bool MergeTreeData::allowRemoveStaleMovingParts() const +{ + return ConfigHelper::getBool(getContext()->getConfigRef(), "allow_remove_stale_moving_parts"); +} + CurrentlySubmergingEmergingTagger::~CurrentlySubmergingEmergingTagger() { std::lock_guard lock(storage.currently_submerging_emerging_mutex); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 444bd8f47ac..8755ffebd7e 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1060,6 +1060,9 @@ public: void waitForOutdatedPartsToBeLoaded() const; bool canUsePolymorphicParts() const; + /// TODO: make enabled by default in the next release if no problems found. + bool allowRemoveStaleMovingParts() const; + protected: friend class IMergeTreeDataPart; friend class MergeTreeDataMergerMutator; diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 656167de986..8fa4ac6c78a 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -250,7 +250,7 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me cloned_part.part = std::move(builder).withPartFormatFromDisk().build(); LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part.part->getDataPartStorage().getFullPath()); - cloned_part.part->is_temp = true; + cloned_part.part->is_temp = data->allowRemoveStaleMovingParts(); cloned_part.part->loadColumnsChecksumsIndexes(true, true); cloned_part.part->loadVersionMetadata(); cloned_part.part->modification_time = cloned_part.part->getDataPartStorage().getLastModified().epochTime(); @@ -270,10 +270,11 @@ void MergeTreePartsMover::swapClonedPart(TemporaryClonedPart & cloned_part) cons { LOG_INFO(log, "Failed to swap {}. Active part doesn't exist (containing part {}). " - "Possible it was merged or mutated. Will remove copy on path '{}'", + "Possible it was merged or mutated. Part on path '{}' {}", cloned_part.part->name, active_part ? active_part->name : "doesn't exist", - cloned_part.part->getDataPartStorage().getFullPath()); + cloned_part.part->getDataPartStorage().getFullPath(), + data->allowRemoveStaleMovingParts() ? "will be removed" : "will remain intact (set in config.xml, exercise caution when using)"); return; } diff --git a/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml b/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml index f6898ed1d7e..1450a459257 100644 --- a/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml +++ b/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml @@ -21,4 +21,6 @@ + + true diff --git a/tests/integration/test_consistant_parts_after_move_partition/configs/remote_servers.xml b/tests/integration/test_consistant_parts_after_move_partition/configs/remote_servers.xml index 3d4ccd584b1..f22b7dc4032 100644 --- a/tests/integration/test_consistant_parts_after_move_partition/configs/remote_servers.xml +++ b/tests/integration/test_consistant_parts_after_move_partition/configs/remote_servers.xml @@ -16,4 +16,5 @@ + true diff --git a/tests/integration/test_encrypted_disk/configs/storage.xml b/tests/integration/test_encrypted_disk/configs/storage.xml index 1e48c80d50f..2b84e0d6daa 100644 --- a/tests/integration/test_encrypted_disk/configs/storage.xml +++ b/tests/integration/test_encrypted_disk/configs/storage.xml @@ -105,4 +105,5 @@ + true diff --git a/tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml index feb537ebbce..a6e0d26f695 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml +++ b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml @@ -15,4 +15,5 @@ 500 ./clickhouse/ users.xml + true diff --git a/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml index 890c396ed95..858d77e9ea0 100644 --- a/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml @@ -29,4 +29,5 @@ 0 + true diff --git a/tests/integration/test_merge_tree_s3/configs/config.xml b/tests/integration/test_merge_tree_s3/configs/config.xml index 314f23f5788..a25da96215e 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.xml @@ -8,4 +8,5 @@ true + true diff --git a/tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/storage_configuration.xml b/tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/storage_configuration.xml index 3289186c175..cd2f0867c61 100644 --- a/tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/storage_configuration.xml @@ -24,5 +24,6 @@ +true diff --git a/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml b/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml index e7a87fb77b1..033699f4634 100644 --- a/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml @@ -122,6 +122,7 @@ +true 1 diff --git a/tests/integration/test_rename_column/configs/config.d/storage_configuration.xml b/tests/integration/test_rename_column/configs/config.d/storage_configuration.xml index 12a598c64b5..65cac905e9a 100644 --- a/tests/integration/test_rename_column/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_rename_column/configs/config.d/storage_configuration.xml @@ -29,5 +29,6 @@ 0 1 +true diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml index cb444c728c9..bb4aba94e0b 100644 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml @@ -89,4 +89,5 @@ test_cluster 1
+ true diff --git a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml index f7d9efc2cae..63162c3c19b 100644 --- a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml +++ b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml @@ -93,4 +93,5 @@ test_cluster
+ true diff --git a/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml b/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml index ae1dc9dd038..09e6fc99411 100644 --- a/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_ttl_move/configs/config.d/storage_configuration.xml @@ -107,4 +107,5 @@ +true diff --git a/tests/integration/test_zero_copy_fetch/configs/storage_conf.xml b/tests/integration/test_zero_copy_fetch/configs/storage_conf.xml index b3ce0735a3c..9e9dab6a972 100644 --- a/tests/integration/test_zero_copy_fetch/configs/storage_conf.xml +++ b/tests/integration/test_zero_copy_fetch/configs/storage_conf.xml @@ -38,4 +38,5 @@ true + true From ac638615aeace304b6a196e68561d73a03e86344 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 8 Jun 2023 10:29:01 +0000 Subject: [PATCH 1364/2223] Upd test_alter_moving_garbage --- .../test_alter_moving_garbage/test.py | 39 ++++++++++++------- 1 file changed, 24 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_alter_moving_garbage/test.py b/tests/integration/test_alter_moving_garbage/test.py index dc3f6c35ead..330df3ac490 100644 --- a/tests/integration/test_alter_moving_garbage/test.py +++ b/tests/integration/test_alter_moving_garbage/test.py @@ -11,6 +11,7 @@ from helpers.cluster import ClickHouseCluster # two replicas in remote_servers.xml REPLICA_COUNT = 2 + @pytest.fixture(scope="module") def cluster(): try: @@ -82,7 +83,7 @@ def test_create_table( additional_settings = {} - # different names for logs readability + # Different names for logs readability table_name = "test_table" if allow_remote_fs_zero_copy_replication: table_name = "test_table_zero_copy" @@ -99,17 +100,7 @@ def test_create_table( f"INSERT INTO {table_name} SELECT toDate('{partition}'), number as id, toString(sipHash64(number, {i})) FROM numbers(10_000)" ) - def check_count(): - if replicated_engine: - return random.choice(nodes).query_with_retry( - f"SELECT countDistinct(dt, data) FROM clusterAllReplicas(test_cluster, default.{table_name}) WHERE id % 100 = 0" - ) - else: - return random.choice(nodes).query( - f"SELECT countDistinct(dt, data) FROM {table_name} WHERE id % 100 = 0" - ) - - assert check_count() == "1000\n" + # Run ALTER in parallel with moving parts stop_alter = False @@ -118,9 +109,14 @@ def test_create_table( for d in range(1, 100): if stop_alter: break - # I managed to reproduce issue with DELETE, but it can be any other lightweight mutation - # Do not delete rows with id % 100 = 0, because they are used in check_count to check that data is not corrupted + + # Some lightweight mutation should change moving part before it is swapped, then we will have to cleanup it. + # Messages `Failed to swap {}. Active part doesn't exist` should appear in logs. + # + # I managed to reproduce issue with DELETE (`ALTER TABLE {table_name} ADD/DROP COLUMN` also works on real s3 instead of minio) + # Note: do not delete rows with id % 100 = 0, because they are used in `check_count` to use them in check that data is not corrupted random.choice(nodes).query(f"DELETE FROM {table_name} WHERE id % 100 = {d}") + time.sleep(0.1) alter_thread = threading.Thread(target=alter) @@ -143,4 +139,17 @@ def test_create_table( stop_alter = True alter_thread.join() - assert check_count() == "1000\n" + # Check that no data was lost + + data_digest = None + if replicated_engine: + # We don't know what data was replicated, so we need to check all replicas and take unique values + data_digest = random.choice(nodes).query_with_retry( + f"SELECT countDistinct(dt, data) FROM clusterAllReplicas(test_cluster, default.{table_name}) WHERE id % 100 == 0" + ) + else: + data_digest = random.choice(nodes).query( + f"SELECT countDistinct(dt, data) FROM {table_name} WHERE id % 100 == 0" + ) + + assert data_digest == "1000\n" From 32d1acb3a57b0c1e942c64b7c434701f37fa1910 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Jun 2023 12:29:26 +0200 Subject: [PATCH 1365/2223] Add changelog for 23.5 --- CHANGELOG.md | 253 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 253 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1ccd4f9846d..f2ffdad9a7c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v23.5, 2023-06-08](#235)**
**[ClickHouse release v23.4, 2023-04-26](#234)**
**[ClickHouse release v23.3 LTS, 2023-03-30](#233)**
**[ClickHouse release v23.2, 2023-02-23](#232)**
@@ -7,6 +8,258 @@ # 2023 Changelog +### ClickHouse release 23.5, 2023-06-08 + +#### Upgrade Notes +* Compress marks and primary key by default. It significantly reduces the cold query time. Upgrade notes: the support for compressed marks and primary key has been added in version 22.9. If you turned on compressed marks or primary key or installed version 23.5 or newer, which has compressed marks or primary key on by default, you will not be able to downgrade to version 22.8 or earlier. You can also explicitly disable compressed marks or primary keys by specifying the `compress_marks` and `compress_primary_key` settings in the `` section of the server configuration file. **Upgrade notes:** If you upgrade from versions prior to 22.9, you should either upgrade all replicas at once or disable the compression before upgrade, or upgrade through an intermediate version, where the compressed marks are supported but not enabled by default, such as 23.3. [#42587](https://github.com/ClickHouse/ClickHouse/pull/42587) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make local object storage work consistently with s3 object storage, fix problem with append (closes [#48465](https://github.com/ClickHouse/ClickHouse/issues/48465)), make it configurable as independent storage. The change is backward incompatible because the cache on top of local object storage is not incompatible to previous versions. [#48791](https://github.com/ClickHouse/ClickHouse/pull/48791) ([Kseniia Sumarokova](https://github.com/kssenii)). +* The experimental feature "in-memory data parts" is removed. The data format is still supported, but the settings are no-op, and compact or wide parts will be used instead. This closes [#45409](https://github.com/ClickHouse/ClickHouse/issues/45409). [#49429](https://github.com/ClickHouse/ClickHouse/pull/49429) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Changed default values of settings `parallelize_output_from_storages` and `input_format_parquet_preserve_order`. This allows ClickHouse to reorder rows when reading from files (e.g. CSV or Parquet), greatly improving performance in many cases. To restore the old behavior of preserving order, use `parallelize_output_from_storages = 0`, `input_format_parquet_preserve_order = 1`. [#49479](https://github.com/ClickHouse/ClickHouse/pull/49479) ([Michael Kolupaev](https://github.com/al13n321)). +* Make projections production-ready. Add the `optimize_use_projections` setting to control whether the projections will be selected for SELECT queries. The setting `allow_experimental_projection_optimization` is obsolete and does nothing. [#49719](https://github.com/ClickHouse/ClickHouse/pull/49719) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Mark `joinGet` as non deterministic (so as `dictGet`). It allows using them in mutations without an extra setting. [#49843](https://github.com/ClickHouse/ClickHouse/pull/49843) ([Azat Khuzhin](https://github.com/azat)). +* Revert the "`groupArray` returns cannot be nullable" change (due to binary compatibility breakage for `groupArray`/`groupArrayLast`/`groupArraySample` over `Nullable` types, which likely will lead to `TOO_LARGE_ARRAY_SIZE` or `CANNOT_READ_ALL_DATA`). [#49971](https://github.com/ClickHouse/ClickHouse/pull/49971) ([Azat Khuzhin](https://github.com/azat)). +* Setting `enable_memory_bound_merging_of_aggregation_results` is enabled by default. If you update from version prior to 22.12, we recommend to set this flag to `false` until update is finished. [#50319](https://github.com/ClickHouse/ClickHouse/pull/50319) ([Nikita Taranov](https://github.com/nickitat)). + +#### New Feature +* Added native ClickHouse Keeper CLI Client, it is available as `clickhouse keeper-client` [#47414](https://github.com/ClickHouse/ClickHouse/pull/47414) ([pufit](https://github.com/pufit)). +* Add `urlCluster` table function. Refactor all *Cluster table functions to reduce code duplication. Make schema inference work for all possible *Cluster function signatures and for named collections. Closes [#38499](https://github.com/ClickHouse/ClickHouse/issues/38499). [#45427](https://github.com/ClickHouse/ClickHouse/pull/45427) ([attack204](https://github.com/attack204)), Pavel Kruglov. +* The query cache can now be used for production workloads. [#47977](https://github.com/ClickHouse/ClickHouse/pull/47977) ([Robert Schulze](https://github.com/rschu1ze)). The query cache can now support queries with totals and extremes modifier. [#48853](https://github.com/ClickHouse/ClickHouse/pull/48853) ([Robert Schulze](https://github.com/rschu1ze)). Make `allow_experimental_query_cache` setting as obsolete for backward-compatibility. It was removed in https://github.com/ClickHouse/ClickHouse/pull/47977. [#49934](https://github.com/ClickHouse/ClickHouse/pull/49934) ([Timur Solodovnikov](https://github.com/tsolodov)). +* Geographical data types (`Point`, `Ring`, `Polygon`, and `MultiPolygon`) are production-ready. [#50022](https://github.com/ClickHouse/ClickHouse/pull/50022) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add schema inference to PostgreSQL, MySQL, MeiliSearch, and SQLite table engines. Closes [#49972](https://github.com/ClickHouse/ClickHouse/issues/49972). [#50000](https://github.com/ClickHouse/ClickHouse/pull/50000) ([Nikolay Degterinsky](https://github.com/evillique)). +* Password type in queries like `CREATE USER u IDENTIFIED BY 'p'` will be automatically set according to the setting `default_password_type` in the `config.xml` on the server. Closes [#42915](https://github.com/ClickHouse/ClickHouse/issues/42915). [#44674](https://github.com/ClickHouse/ClickHouse/pull/44674) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add bcrypt password authentication type. Closes [#34599](https://github.com/ClickHouse/ClickHouse/issues/34599). [#44905](https://github.com/ClickHouse/ClickHouse/pull/44905) ([Nikolay Degterinsky](https://github.com/evillique)). +* Introduces new keyword `INTO OUTFILE 'file.txt' APPEND`. [#48880](https://github.com/ClickHouse/ClickHouse/pull/48880) ([alekar](https://github.com/alekar)). +* Added `system.zookeeper_connection` table that shows information about Keeper connections. [#45245](https://github.com/ClickHouse/ClickHouse/pull/45245) ([mateng915](https://github.com/mateng0915)). +* Add new function `generateRandomStructure` that generates random table structure. It can be used in combination with table function `generateRandom`. [#47409](https://github.com/ClickHouse/ClickHouse/pull/47409) ([Kruglov Pavel](https://github.com/Avogar)). +* Allow the use of `CASE` without an `ELSE` branch and extended `transform` to deal with more types. Also fix some issues that made transform() return incorrect results when decimal types were mixed with other numeric types. [#48300](https://github.com/ClickHouse/ClickHouse/pull/48300) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Added [server-side encryption using KMS keys](https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingKMSEncryption.html) with S3 tables, and the `header` setting with S3 disks. Closes [#48723](https://github.com/ClickHouse/ClickHouse/issues/48723). [#48724](https://github.com/ClickHouse/ClickHouse/pull/48724) ([Johann Gan](https://github.com/johanngan)). +* Add MemoryTracker for the background tasks (merges and mutation). Introduces `merges_mutations_memory_usage_soft_limit` and `merges_mutations_memory_usage_to_ram_ratio` settings that represent the soft memory limit for merges and mutations. If this limit is reached ClickHouse won't schedule new merge or mutation tasks. Also `MergesMutationsMemoryTracking` metric is introduced to allow observing current memory usage of background tasks. Resubmit [#46089](https://github.com/ClickHouse/ClickHouse/issues/46089). Closes [#48774](https://github.com/ClickHouse/ClickHouse/issues/48774). [#48787](https://github.com/ClickHouse/ClickHouse/pull/48787) ([Dmitry Novik](https://github.com/novikd)). +* Function `dotProduct` work for array. [#49050](https://github.com/ClickHouse/ClickHouse/pull/49050) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Support statement `SHOW INDEX` to improve compatibility with MySQL. [#49158](https://github.com/ClickHouse/ClickHouse/pull/49158) ([Robert Schulze](https://github.com/rschu1ze)). +* Add virtual column `_file` and `_path` support to table function `url`. - Impove error message for table function `url`. - resolves [#49231](https://github.com/ClickHouse/ClickHouse/issues/49231) - resolves [#49232](https://github.com/ClickHouse/ClickHouse/issues/49232). [#49356](https://github.com/ClickHouse/ClickHouse/pull/49356) ([Ziyi Tan](https://github.com/Ziy1-Tan)). +* Adding the `grants` field in the users.xml file, which allows specifying grants for users. [#49381](https://github.com/ClickHouse/ClickHouse/pull/49381) ([pufit](https://github.com/pufit)). +* Support full/right join by using grace hash join algorithm. [#49483](https://github.com/ClickHouse/ClickHouse/pull/49483) ([lgbo](https://github.com/lgbo-ustc)). +* `WITH FILL` modifier groups filling by sorting prefix. Controlled by `use_with_fill_by_sorting_prefix` setting (enabled by default). Related to [#33203](https://github.com/ClickHouse/ClickHouse/issues/33203)#issuecomment-1418736794. [#49503](https://github.com/ClickHouse/ClickHouse/pull/49503) ([Igor Nikonov](https://github.com/devcrafter)). +* Clickhouse-client now accepts queries after "--multiquery" when "--query" (or "-q") is absent. example: clickhouse-client --multiquery "select 1; select 2;". [#49870](https://github.com/ClickHouse/ClickHouse/pull/49870) ([Alexey Gerasimchuk](https://github.com/Demilivor)). +* Add separate `handshake_timeout` for receiving Hello packet from replica. Closes [#48854](https://github.com/ClickHouse/ClickHouse/issues/48854). [#49948](https://github.com/ClickHouse/ClickHouse/pull/49948) ([Kruglov Pavel](https://github.com/Avogar)). +* Added a function "space" which repeats a space as many times as specified. [#50103](https://github.com/ClickHouse/ClickHouse/pull/50103) ([Robert Schulze](https://github.com/rschu1ze)). +* Added --input_format_csv_trim_whitespaces option. [#50215](https://github.com/ClickHouse/ClickHouse/pull/50215) ([Alexey Gerasimchuk](https://github.com/Demilivor)). +* Allow the `dictGetAll` function for regexp tree dictionaries to return values from multiple matches as arrays. Closes [#50254](https://github.com/ClickHouse/ClickHouse/issues/50254). [#50255](https://github.com/ClickHouse/ClickHouse/pull/50255) ([Johann Gan](https://github.com/johanngan)). +* Added `toLastDayOfWeek` function to round a date or a date with time up to the nearest Saturday or Sunday. [#50315](https://github.com/ClickHouse/ClickHouse/pull/50315) ([Victor Krasnov](https://github.com/sirvickr)). +* Ability to ignore a skip index by specifying `ignore_data_skipping_indices`. [#50329](https://github.com/ClickHouse/ClickHouse/pull/50329) ([Boris Kuschel](https://github.com/bkuschel)). +* Add `system.user_processes` table and `SHOW USER PROCESSES` query to show memory info and ProfileEvents on user level. [#50492](https://github.com/ClickHouse/ClickHouse/pull/50492) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Add server and format settings `display_secrets_in_show_and_select` for displaying secrets of tables, databases, table functions, and dictionaries. Add privilege `displaySecretsInShowAndSelect` controlling which users can view secrets. [#46528](https://github.com/ClickHouse/ClickHouse/pull/46528) ([Mike Kot](https://github.com/myrrc)). +* Allow to set up a ROW POLICY for all tables that belong to a DATABASE. [#47640](https://github.com/ClickHouse/ClickHouse/pull/47640) ([Ilya Golshtein](https://github.com/ilejn)). + +#### Performance Improvement +* Compress marks and primary key by default. It significantly reduces the cold query time. Upgrade notes: the support for compressed marks and primary key has been added in version 22.9. If you turned on compressed marks or primary key or installed version 23.5 or newer, which has compressed marks or primary key on by default, you will not be able to downgrade to version 22.8 or earlier. You can also explicitly disable compressed marks or primary keys by specifying the `compress_marks` and `compress_primary_key` settings in the `` section of the server configuration file. [#42587](https://github.com/ClickHouse/ClickHouse/pull/42587) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* New setting s3_max_inflight_parts_for_one_file sets the limit of concurrently loaded parts with multipart upload request in scope of one file. [#49961](https://github.com/ClickHouse/ClickHouse/pull/49961) ([Sema Checherinda](https://github.com/CheSema)). +* When reading from multiple files reduce parallel parsing threads for each file. Resolves [#42192](https://github.com/ClickHouse/ClickHouse/issues/42192). [#46661](https://github.com/ClickHouse/ClickHouse/pull/46661) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Use aggregate projection only if it reads fewer granules than normal reading. It should help in case if query hits the PK of the table, but not the projection. Fixes [#49150](https://github.com/ClickHouse/ClickHouse/issues/49150). [#49417](https://github.com/ClickHouse/ClickHouse/pull/49417) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Do not store blocks in `ANY` hash join if nothing is inserted. [#48633](https://github.com/ClickHouse/ClickHouse/pull/48633) ([vdimir](https://github.com/vdimir)). +* Fixes aggregate combinator `-If` when JIT compiled, and enable JIT compilation for aggregate functions. Closes [#48120](https://github.com/ClickHouse/ClickHouse/issues/48120). [#49083](https://github.com/ClickHouse/ClickHouse/pull/49083) ([Igor Nikonov](https://github.com/devcrafter)). +* For reading from remote tables we use smaller tasks (instead of reading the whole part) to make tasks stealing work * task size is determined by size of columns to read * always use 1mb buffers for reading from s3 * boundaries of cache segments aligned to 1mb so they have decent size even with small tasks. it also should prevent fragmentation. [#49287](https://github.com/ClickHouse/ClickHouse/pull/49287) ([Nikita Taranov](https://github.com/nickitat)). +* Introduced settings: - `merge_max_block_size_bytes` to limit the amount of memory used for background operations. - `vertical_merge_algorithm_min_bytes_to_activate` to add another condition to activate vertical merges. [#49313](https://github.com/ClickHouse/ClickHouse/pull/49313) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Default size of a read buffer for reading from local filesystem changed to a slightly better value. Also two new settings are introduced: `max_read_buffer_size_local_fs` and `max_read_buffer_size_remote_fs`. [#49321](https://github.com/ClickHouse/ClickHouse/pull/49321) ([Nikita Taranov](https://github.com/nickitat)). +* Improve memory usage and speed of `SPARSE_HASHED`/`HASHED` dictionaries (e.g. `SPARSE_HASHED` now eats 2.6x less memory, and is ~2x faster). [#49380](https://github.com/ClickHouse/ClickHouse/pull/49380) ([Azat Khuzhin](https://github.com/azat)). +* Optimize the `system.query_log` and `system.query_thread_log` tables by applying `LowCardinality` when appropriate. The queries over these tables will be faster. [#49530](https://github.com/ClickHouse/ClickHouse/pull/49530) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better performance when reading local `Parquet` files (through parallel reading). [#49539](https://github.com/ClickHouse/ClickHouse/pull/49539) ([Michael Kolupaev](https://github.com/al13n321)). +* Improve the performance of `RIGHT/FULL JOIN` by up to 2 times in certain scenarios, especially when joining a small left table with a large right table. [#49585](https://github.com/ClickHouse/ClickHouse/pull/49585) ([lgbo](https://github.com/lgbo-ustc)). +* Improve performance of BLAKE3 by 11% by enabling LTO for Rust. [#49600](https://github.com/ClickHouse/ClickHouse/pull/49600) ([Azat Khuzhin](https://github.com/azat)). Now it is on par with C++. +* Optimize the structure of the `system.opentelemetry_span_log`. Use `LowCardinality` where appropriate. Although this table is generally stupid (it is using the Map data type even for common attributes), it will be slightly better. [#49647](https://github.com/ClickHouse/ClickHouse/pull/49647) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Try to reserve hash table's size in `grace_hash` join. [#49816](https://github.com/ClickHouse/ClickHouse/pull/49816) ([lgbo](https://github.com/lgbo-ustc)). +* As is addresed in issue [#49748](https://github.com/ClickHouse/ClickHouse/issues/49748), the predicates with date converters, such as `toYear`, `toYYYYMM`, could be rewritten with the equivalent date (YYYY-MM-DD) comparisons at the AST level. And this transformation could bring performance improvement as it is free from the expensive date converter and the comparison between dates (or integers in the low level representation) is quite low-cost. The [prototype](https://github.com/ZhiguoZh/ClickHouse/commit/c7f1753f0c9363a19d95fa46f1cfed1d9f505ee0) shows that, with all identified date converters optimized, the overall QPS of the 13 queries is enhanced by **~11%** on the ICX server (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads). [#50062](https://github.com/ClickHouse/ClickHouse/pull/50062) [#50307](https://github.com/ClickHouse/ClickHouse/pull/50307) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Parallel merge of `uniqExactIf` states. Closes [#49885](https://github.com/ClickHouse/ClickHouse/issues/49885). [#50285](https://github.com/ClickHouse/ClickHouse/pull/50285) ([flynn](https://github.com/ucasfl)). +* Keeper improvement: add `CheckNotExists` request to Keeper, which allows to improve the performance of Replicated tables. [#48897](https://github.com/ClickHouse/ClickHouse/pull/48897) ([Antonio Andelic](https://github.com/antonio2368)). +* Keeper performance improvements: avoid serializing same request twice while processing. Cache deserialization results of large requests. Controlled by new coordination setting `min_request_size_for_cache`. [#49004](https://github.com/ClickHouse/ClickHouse/pull/49004) ([Antonio Andelic](https://github.com/antonio2368)). +* Reduced number of `List` ZooKeeper requests when selecting parts to merge and a lot of partitions do not have anything to merge. [#49637](https://github.com/ClickHouse/ClickHouse/pull/49637) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Rework locking in the FS cache [#44985](https://github.com/ClickHouse/ClickHouse/pull/44985) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Disable pure parallel replicas if trivial count optimization is possible. [#50594](https://github.com/ClickHouse/ClickHouse/pull/50594) ([Raúl Marín](https://github.com/Algunenano)). +* Don't send head request for all keys in Iceberg schema inference, only for keys that are used for reaing data. [#50203](https://github.com/ClickHouse/ClickHouse/pull/50203) ([Kruglov Pavel](https://github.com/Avogar)). +* Setting `enable_memory_bound_merging_of_aggregation_results` is enabled by default. [#50319](https://github.com/ClickHouse/ClickHouse/pull/50319) ([Nikita Taranov](https://github.com/nickitat)). + +#### Experimental Feature +* `DEFLATE_QPL` codec lower the minimum simd version to SSE 4.2. [doc change in qpl](https://github.com/intel/qpl/commit/3f8f5cea27739f5261e8fd577dc233ffe88bf679) - Intel® QPL relies on a run-time kernels dispatcher and cpuid check to choose the best available implementation(sse/avx2/avx512) - restructured cmakefile for qpl build in clickhouse to align with latest upstream qpl. [#49811](https://github.com/ClickHouse/ClickHouse/pull/49811) ([jasperzhu](https://github.com/jinjunzh)). +* Add initial support to do JOINs with pure parallel replicas. [#49544](https://github.com/ClickHouse/ClickHouse/pull/49544) ([Raúl Marín](https://github.com/Algunenano)). +* More parallelism on `Outdated` parts removal with "zero-copy replication". [#49630](https://github.com/ClickHouse/ClickHouse/pull/49630) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Parallel Replicas: 1) Fixed an error `NOT_FOUND_COLUMN_IN_BLOCK` in case of using parallel replicas with non-replicated storage with disabled setting `parallel_replicas_for_non_replicated_merge_tree` 2) Now `allow_experimental_parallel_reading_from_replicas` have 3 possible values - 0, 1 and 2. 0 - disabled, 1 - enabled, silently disable them in case of failure (in case of FINAL or JOIN), 2 - enabled, throw an expection in case of failure. 3) If FINAL modifier is used in SELECT query and parallel replicas are enabled, ClickHouse will try to disable them if `allow_experimental_parallel_reading_from_replicas` is set to 1 and throw an exception otherwise. [#50195](https://github.com/ClickHouse/ClickHouse/pull/50195) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* When parallel replicas are enabled they will always skip unavailable servers (the behavior is controlled by the setting `skip_unavailable_shards`, enabled by default and can be only disabled). This closes: [#48565](https://github.com/ClickHouse/ClickHouse/issues/48565). [#50293](https://github.com/ClickHouse/ClickHouse/pull/50293) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Improvement +* The `BACKUP` command will not decrypt data from encrypted disks while making a backup. Instead the data will be stored in a backup in encrypted form. Such backups can be restored only to an encrypted disk with the same (or extended) list of encryption keys. [#48896](https://github.com/ClickHouse/ClickHouse/pull/48896) ([Vitaly Baranov](https://github.com/vitlibar)). +* Added possibility to use temporary tables in FROM part of ATTACH PARTITION FROM and REPLACE PARTITION FROM. [#49436](https://github.com/ClickHouse/ClickHouse/pull/49436) ([Roman Vasin](https://github.com/rvasin)). +* Added setting `async_insert` for `MergeTree` tables. It has the same meaning as query-level setting `async_insert` and enables asynchronous inserts for specific table. Note: it doesn't take effect for insert queries from `clickhouse-client`, use query-level setting in that case. [#49122](https://github.com/ClickHouse/ClickHouse/pull/49122) ([Anton Popov](https://github.com/CurtizJ)). +* Add support for size suffixes in quota creation statement parameters. [#49087](https://github.com/ClickHouse/ClickHouse/pull/49087) ([Eridanus](https://github.com/Eridanus117)). +* Extend `first_value` and `last_value` to accept NULL. [#46467](https://github.com/ClickHouse/ClickHouse/pull/46467) ([lgbo](https://github.com/lgbo-ustc)). +* Add alias `str_to_map` and `mapFromString` for `extractKeyValuePairs`. closes https://github.com/clickhouse/clickhouse/issues/47185. [#49466](https://github.com/ClickHouse/ClickHouse/pull/49466) ([flynn](https://github.com/ucasfl)). +* Add support for CGroup version 2 for asynchronous metrics about the memory usage and availability. This closes [#37983](https://github.com/ClickHouse/ClickHouse/issues/37983). [#45999](https://github.com/ClickHouse/ClickHouse/pull/45999) ([sichenzhao](https://github.com/sichenzhao)). +* Cluster table functions should always skip unavailable shards. close [#46314](https://github.com/ClickHouse/ClickHouse/issues/46314). [#46765](https://github.com/ClickHouse/ClickHouse/pull/46765) ([zk_kiger](https://github.com/zk-kiger)). +* Allow CSV file to contain empty columns in its header. [#47496](https://github.com/ClickHouse/ClickHouse/pull/47496) ([你不要过来啊](https://github.com/iiiuwioajdks)). +* Add Google Cloud Storage S3 compatible table function `gcs`. Like the `oss` and `cosn` functions, it is just an alias over the `s3` table function, and it does not bring any new features. [#47815](https://github.com/ClickHouse/ClickHouse/pull/47815) ([Kuba Kaflik](https://github.com/jkaflik)). +* Add ability to use strict parts size for S3 (compatibility with CloudFlare R2 S3 Storage). [#48492](https://github.com/ClickHouse/ClickHouse/pull/48492) ([Azat Khuzhin](https://github.com/azat)). +* Added new columns with info about `Replicated` database replicas to `system.clusters`: `database_shard_name`, `database_replica_name`, `is_active`. Added an optional `FROM SHARD` clause to `SYSTEM DROP DATABASE REPLICA` query. [#48548](https://github.com/ClickHouse/ClickHouse/pull/48548) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add a new column `zookeeper_name` in system.replicas, to indicate on which (auxiliary) zookeeper cluster the replicated table's metadata is stored. [#48549](https://github.com/ClickHouse/ClickHouse/pull/48549) ([cangyin](https://github.com/cangyin)). +* `IN` operator support the comparison of `Date` and `Date32`. Closes [#48736](https://github.com/ClickHouse/ClickHouse/issues/48736). [#48806](https://github.com/ClickHouse/ClickHouse/pull/48806) ([flynn](https://github.com/ucasfl)). +* Support for erasure codes in `HDFS`, author: @M1eyu2018, @tomscut. [#48833](https://github.com/ClickHouse/ClickHouse/pull/48833) ([M1eyu](https://github.com/M1eyu2018)). +* Implement SYSTEM DROP REPLICA from auxillary ZooKeeper clusters, may be close [#48931](https://github.com/ClickHouse/ClickHouse/issues/48931). [#48932](https://github.com/ClickHouse/ClickHouse/pull/48932) ([wangxiaobo](https://github.com/wzb5212)). +* Add Array data type to MongoDB. Closes [#48598](https://github.com/ClickHouse/ClickHouse/issues/48598). [#48983](https://github.com/ClickHouse/ClickHouse/pull/48983) ([Nikolay Degterinsky](https://github.com/evillique)). +* Support storing `Interval` data types in tables. [#49085](https://github.com/ClickHouse/ClickHouse/pull/49085) ([larryluogit](https://github.com/larryluogit)). +* Allow using `ntile` window function without explicit window frame definition: `ntile(3) OVER (ORDER BY a)`, close [#46763](https://github.com/ClickHouse/ClickHouse/issues/46763). [#49093](https://github.com/ClickHouse/ClickHouse/pull/49093) ([vdimir](https://github.com/vdimir)). +* Added settings (`number_of_mutations_to_delay`, `number_of_mutations_to_throw`) to delay or throw `ALTER` queries that create mutations (`ALTER UPDATE`, `ALTER DELETE`, `ALTER MODIFY COLUMN`, ...) in case when table already has a lot of unfinished mutations. [#49117](https://github.com/ClickHouse/ClickHouse/pull/49117) ([Anton Popov](https://github.com/CurtizJ)). +* Catch exception from `create_directories` in filesystem cache. [#49203](https://github.com/ClickHouse/ClickHouse/pull/49203) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Copies embedded examples to a new field `example` in `system.functions` to supplement the field `description`. [#49222](https://github.com/ClickHouse/ClickHouse/pull/49222) ([Dan Roscigno](https://github.com/DanRoscigno)). +* Enable connection options for the MongoDB dictionary. Example: ``` xml localhost 27017 test dictionary_source ssl=true ``` ### Documentation entry for user-facing changes. [#49225](https://github.com/ClickHouse/ClickHouse/pull/49225) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Added an alias `asymptotic` for `asymp` computational method for `kolmogorovSmirnovTest`. Improved documentation. [#49286](https://github.com/ClickHouse/ClickHouse/pull/49286) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Aggregation function groupBitAnd/Or/Xor now work on signed integer data. This makes them consistent with the behavior of scalar functions bitAnd/Or/Xor. [#49292](https://github.com/ClickHouse/ClickHouse/pull/49292) ([exmy](https://github.com/exmy)). +* Split function-documentation into more fine-granular fields. [#49300](https://github.com/ClickHouse/ClickHouse/pull/49300) ([Robert Schulze](https://github.com/rschu1ze)). +* Use multiple threads shared between all tables within a server to load outdated data parts. The the size of the pool and its queue is controlled by `max_outdated_parts_loading_thread_pool_size` and `outdated_part_loading_thread_pool_queue_size` settings. [#49317](https://github.com/ClickHouse/ClickHouse/pull/49317) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Don't overestimate the size of processed data for `LowCardinality` columns when they share dictionaries between blocks. This closes [#49322](https://github.com/ClickHouse/ClickHouse/issues/49322). See also [#48745](https://github.com/ClickHouse/ClickHouse/issues/48745). [#49323](https://github.com/ClickHouse/ClickHouse/pull/49323) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Parquet writer now uses reasonable row group size when invoked through `OUTFILE`. [#49325](https://github.com/ClickHouse/ClickHouse/pull/49325) ([Michael Kolupaev](https://github.com/al13n321)). +* Allow restricted keywords like `ARRAY` as an alias if the alias is quoted. Closes [#49324](https://github.com/ClickHouse/ClickHouse/issues/49324). [#49360](https://github.com/ClickHouse/ClickHouse/pull/49360) ([Nikolay Degterinsky](https://github.com/evillique)). +* Data parts loading and deletion jobs were moved to shared server-wide pools instead of per-table pools. Pools sizes are controlled via settings `max_active_parts_loading_thread_pool_size`, `max_outdated_parts_loading_thread_pool_size` and `max_parts_cleaning_thread_pool_size` in top-level config. Table-level settings `max_part_loading_threads` and `max_part_removal_threads` became obsolete. [#49474](https://github.com/ClickHouse/ClickHouse/pull/49474) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Allow `?password=pass` in URL of the Play UI. Password is replaced in browser history. [#49505](https://github.com/ClickHouse/ClickHouse/pull/49505) ([Mike Kot](https://github.com/myrrc)). +* Allow reading zero-size objects from remote filesystems. (because empty files are not backup'd, so we might end up with zero blobs in metadata file). Closes [#49480](https://github.com/ClickHouse/ClickHouse/issues/49480). [#49519](https://github.com/ClickHouse/ClickHouse/pull/49519) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Attach thread MemoryTracker to `total_memory_tracker` after `ThreadGroup` detached. [#49527](https://github.com/ClickHouse/ClickHouse/pull/49527) ([Dmitry Novik](https://github.com/novikd)). +* Fix parameterized views when a query parameter is used multiple times in the query. [#49556](https://github.com/ClickHouse/ClickHouse/pull/49556) ([Azat Khuzhin](https://github.com/azat)). +* Release memory allocated for the last sent ProfileEvents snapshot in the context of a query. Followup [#47564](https://github.com/ClickHouse/ClickHouse/issues/47564). [#49561](https://github.com/ClickHouse/ClickHouse/pull/49561) ([Dmitry Novik](https://github.com/novikd)). +* Function "makeDate" now provides a MySQL-compatible overload (year & day of the year argument). [#49603](https://github.com/ClickHouse/ClickHouse/pull/49603) ([Robert Schulze](https://github.com/rschu1ze)). +* Support `dictionary` table function for `RegExpTreeDictionary`. [#49666](https://github.com/ClickHouse/ClickHouse/pull/49666) ([Han Fei](https://github.com/hanfei1991)). +* Added weighted fair IO scheduling policy. Added dynamic resource manager, which allows IO scheduling hierarchy to be updated in runtime w/o server restarts. [#49671](https://github.com/ClickHouse/ClickHouse/pull/49671) ([Sergei Trifonov](https://github.com/serxa)). +* Add compose request after multipart upload to GCS. This enables the usage of copy operation on objects uploaded with the multipart upload. It's recommended to set `s3_strict_upload_part_size` to some value because compose request can fail on objects created with parts of different sizes. [#49693](https://github.com/ClickHouse/ClickHouse/pull/49693) ([Antonio Andelic](https://github.com/antonio2368)). +* For the `extractKeyValuePairs` function: improve the "best-effort" parsing logic to accept `key_value_delimiter` as a valid part of the value. This also simplifies branching and might even speed up things a bit. [#49760](https://github.com/ClickHouse/ClickHouse/pull/49760) ([Arthur Passos](https://github.com/arthurpassos)). +* Add `initial_query_id` field for system.processors_profile_log [#49777](https://github.com/ClickHouse/ClickHouse/pull/49777) ([helifu](https://github.com/helifu)). +* System log tables can now have custom sorting keys. [#49778](https://github.com/ClickHouse/ClickHouse/pull/49778) ([helifu](https://github.com/helifu)). +* A new field `partitions` to `system.query_log` is used to indicate which partitions are participating in the calculation. [#49779](https://github.com/ClickHouse/ClickHouse/pull/49779) ([helifu](https://github.com/helifu)). +* Added `enable_the_endpoint_id_with_zookeeper_name_prefix` setting for `ReplicatedMergeTree` (disabled by default). When enabled, it adds ZooKeeper cluster name to table's interserver communication endpoint. It avoids `Duplicate interserver IO endpoint` errors when having replicated tables with the same path, but different auxiliary ZooKeepers. [#49780](https://github.com/ClickHouse/ClickHouse/pull/49780) ([helifu](https://github.com/helifu)). +* Add query parameters to `clickhouse-local`. Closes [#46561](https://github.com/ClickHouse/ClickHouse/issues/46561). [#49785](https://github.com/ClickHouse/ClickHouse/pull/49785) ([Nikolay Degterinsky](https://github.com/evillique)). +* Allow loading dictionaries and functions from YAML by default. In previous versions, it required editing the `dictionaries_config` or `user_defined_executable_functions_config` in the configuration file, as they expected `*.xml` files. [#49812](https://github.com/ClickHouse/ClickHouse/pull/49812) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The Kafka table engine now allows to use alias columns. [#49824](https://github.com/ClickHouse/ClickHouse/pull/49824) ([Aleksandr Musorin](https://github.com/AVMusorin)). +* Add setting to limit the max number of pairs produced by `extractKeyValuePairs`, a safeguard to avoid using way too much memory. [#49836](https://github.com/ClickHouse/ClickHouse/pull/49836) ([Arthur Passos](https://github.com/arthurpassos)). +* Add support for (an unusual) case where the arguments in the `IN` operator are single-element tuples. [#49844](https://github.com/ClickHouse/ClickHouse/pull/49844) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* `bitHammingDistance` function support `String` and `FixedString` data type. Closes [#48827](https://github.com/ClickHouse/ClickHouse/issues/48827). [#49858](https://github.com/ClickHouse/ClickHouse/pull/49858) ([flynn](https://github.com/ucasfl)). +* Fix timeout resetting errors in the client on OS X. [#49863](https://github.com/ClickHouse/ClickHouse/pull/49863) ([alekar](https://github.com/alekar)). +* Add support for big integers, such as UInt128, Int128, UInt256, and Int256 in the function `bitCount`. This enables Hamming distance over large bit masks for AI applications. [#49867](https://github.com/ClickHouse/ClickHouse/pull/49867) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fingerprints to be used instead of key IDs in encrypted disks. This simplifies the configuration of encrypted disks. [#49882](https://github.com/ClickHouse/ClickHouse/pull/49882) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add UUID data type to PostgreSQL. Closes [#49739](https://github.com/ClickHouse/ClickHouse/issues/49739). [#49894](https://github.com/ClickHouse/ClickHouse/pull/49894) ([Nikolay Degterinsky](https://github.com/evillique)). +* Function `toUnixTimestamp` now accepts `Date` and `Date32` arguments. [#49989](https://github.com/ClickHouse/ClickHouse/pull/49989) ([Victor Krasnov](https://github.com/sirvickr)). +* Charge only server memory for dictionaries. [#49995](https://github.com/ClickHouse/ClickHouse/pull/49995) ([Azat Khuzhin](https://github.com/azat)). +* The server will allow using the `SQL_*` settings such as `SQL_AUTO_IS_NULL` as no-ops for MySQL compatibility. This closes [#49927](https://github.com/ClickHouse/ClickHouse/issues/49927). [#50013](https://github.com/ClickHouse/ClickHouse/pull/50013) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Preserve initial_query_id for ON CLUSTER queries, which is useful for introspection (under `distributed_ddl_entry_format_version=5`). [#50015](https://github.com/ClickHouse/ClickHouse/pull/50015) ([Azat Khuzhin](https://github.com/azat)). +* Preserve backward incompatibility for renamed settings by using aliases (`allow_experimental_projection_optimization` for `optimize_use_projections`, `allow_experimental_lightweight_delete` for `enable_lightweight_delete`). [#50044](https://github.com/ClickHouse/ClickHouse/pull/50044) ([Azat Khuzhin](https://github.com/azat)). +* Support passing FQDN through setting my_hostname to register cluster node in keeper. Add setting of invisible to support multi compute groups. A compute group as a cluster, is invisible to other compute groups. [#50186](https://github.com/ClickHouse/ClickHouse/pull/50186) ([Yangkuan Liu](https://github.com/LiuYangkuan)). +* Fix PostgreSQL reading all the data even though `LIMIT n` could be specified. [#50187](https://github.com/ClickHouse/ClickHouse/pull/50187) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add new profile events for queries with subqueries (`QueriesWithSubqueries`/`SelectQueriesWithSubqueries`/`InsertQueriesWithSubqueries`). [#50204](https://github.com/ClickHouse/ClickHouse/pull/50204) ([Azat Khuzhin](https://github.com/azat)). +* Adding the roles field in the users.xml file, which allows specifying roles with grants via a config file. [#50278](https://github.com/ClickHouse/ClickHouse/pull/50278) ([pufit](https://github.com/pufit)). +* Report `CGroupCpuCfsPeriod` and `CGroupCpuCfsQuota` in AsynchronousMetrics. - Respect cgroup v2 memory limits during server startup. [#50379](https://github.com/ClickHouse/ClickHouse/pull/50379) ([alekar](https://github.com/alekar)). +* Add a signal handler for SIGQUIT to work the same way as SIGINT. Closes [#50298](https://github.com/ClickHouse/ClickHouse/issues/50298). [#50435](https://github.com/ClickHouse/ClickHouse/pull/50435) ([Nikolay Degterinsky](https://github.com/evillique)). +* In case JSON parse fails due to the large size of the object output the last position to allow debugging. [#50474](https://github.com/ClickHouse/ClickHouse/pull/50474) ([Valentin Alexeev](https://github.com/valentinalexeev)). +* Support decimals with not fixed size. Closes [#49130](https://github.com/ClickHouse/ClickHouse/issues/49130). [#50586](https://github.com/ClickHouse/ClickHouse/pull/50586) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Build/Testing/Packaging Improvement +* New and improved `keeper-bench`. Everything can be customized from YAML/XML file: - request generator - each type of request generator can have a specific set of fields - multi requests can be generated just by doing the same under `multi` key - for each request or subrequest in multi a `weight` field can be defined to control distribution - define trees that need to be setup for a test run - hosts can be defined with all timeouts customizable and it's possible to control how many sessions to generate for each host - integers defined with `min_value` and `max_value` fields are random number generators. [#48547](https://github.com/ClickHouse/ClickHouse/pull/48547) ([Antonio Andelic](https://github.com/antonio2368)). +* Io_uring is not supported on macos, don't choose it when running tests on local to avoid occassional failures. [#49250](https://github.com/ClickHouse/ClickHouse/pull/49250) ([Frank Chen](https://github.com/FrankChen021)). +* Support named fault injection for testing. [#49361](https://github.com/ClickHouse/ClickHouse/pull/49361) ([Han Fei](https://github.com/hanfei1991)). +* Allow running ClickHouse in the OS where the `prctl` (process control) syscall is not available, such as AWS Lambda. [#49538](https://github.com/ClickHouse/ClickHouse/pull/49538) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixed the issue of build conflict between contrib/isa-l and isa-l in qpl [49296](https://github.com/ClickHouse/ClickHouse/issues/49296). [#49584](https://github.com/ClickHouse/ClickHouse/pull/49584) ([jasperzhu](https://github.com/jinjunzh)). +* Utilities are now only build if explicitly requested ("-DENABLE_UTILS=1") instead of by default, this reduces link times in typical development builds. [#49620](https://github.com/ClickHouse/ClickHouse/pull/49620) ([Robert Schulze](https://github.com/rschu1ze)). +* Pull build description of idxd-config into a separate CMake file to avoid accidental removal in future. [#49651](https://github.com/ClickHouse/ClickHouse/pull/49651) ([jasperzhu](https://github.com/jinjunzh)). +* Add CI check with an enabled analyzer in the master. Follow-up [#49562](https://github.com/ClickHouse/ClickHouse/issues/49562). [#49668](https://github.com/ClickHouse/ClickHouse/pull/49668) ([Dmitry Novik](https://github.com/novikd)). +* Switch to LLVM/clang 16. [#49678](https://github.com/ClickHouse/ClickHouse/pull/49678) ([Azat Khuzhin](https://github.com/azat)). +* Allow building ClickHouse with clang-17. [#49851](https://github.com/ClickHouse/ClickHouse/pull/49851) ([Alexey Milovidov](https://github.com/alexey-milovidov)). [#50410](https://github.com/ClickHouse/ClickHouse/pull/50410) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ClickHouse is now easier to be integrated into other cmake projects. [#49991](https://github.com/ClickHouse/ClickHouse/pull/49991) ([Amos Bird](https://github.com/amosbird)). (Which is strongly discouraged - Alexey Milovidov). +* Fix strange additional QEMU logging after [#47151](https://github.com/ClickHouse/ClickHouse/issues/47151), see https://s3.amazonaws.com/clickhouse-test-reports/50078/a4743996ee4f3583884d07bcd6501df0cfdaa346/stateless_tests__release__databasereplicated__[3_4].html. [#50442](https://github.com/ClickHouse/ClickHouse/pull/50442) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* ClickHouse can work on Linux RISC-V 6.1.22. This closes [#50456](https://github.com/ClickHouse/ClickHouse/issues/50456). [#50457](https://github.com/ClickHouse/ClickHouse/pull/50457) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Bump internal protobuf to v3.18 (fixes bogus CVE-2022-1941). [#50400](https://github.com/ClickHouse/ClickHouse/pull/50400) ([Robert Schulze](https://github.com/rschu1ze)). +* Bump internal libxml2 to v2.10.4 (fixes bogus CVE-2023-28484 and bogus CVE-2023-29469). [#50402](https://github.com/ClickHouse/ClickHouse/pull/50402) ([Robert Schulze](https://github.com/rschu1ze)). +* Bump c-ares to v1.19.1 (bogus CVE-2023-32067, bogus CVE-2023-31130, bogus CVE-2023-31147). [#50403](https://github.com/ClickHouse/ClickHouse/pull/50403) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bogus CVE-2022-2469 in libgsasl. [#50404](https://github.com/ClickHouse/ClickHouse/pull/50404) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* ActionsDAG: fix wrong optimization [#47584](https://github.com/ClickHouse/ClickHouse/pull/47584) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Correctly handle concurrent snapshots in Keeper [#48466](https://github.com/ClickHouse/ClickHouse/pull/48466) ([Antonio Andelic](https://github.com/antonio2368)). +* MergeTreeMarksLoader holds DataPart instead of DataPartStorage [#48515](https://github.com/ClickHouse/ClickHouse/pull/48515) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Sequence state fix [#48603](https://github.com/ClickHouse/ClickHouse/pull/48603) ([Ilya Golshtein](https://github.com/ilejn)). +* Back/Restore concurrency check on previous fails [#48726](https://github.com/ClickHouse/ClickHouse/pull/48726) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix Attaching a table with non-existent ZK path does not increase the ReadonlyReplica metric [#48954](https://github.com/ClickHouse/ClickHouse/pull/48954) ([wangxiaobo](https://github.com/wzb5212)). +* Fix possible terminate called for uncaught exception in some places [#49112](https://github.com/ClickHouse/ClickHouse/pull/49112) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix key not found error for queries with multiple StorageJoin [#49137](https://github.com/ClickHouse/ClickHouse/pull/49137) ([vdimir](https://github.com/vdimir)). +* Fix wrong query result when using nullable primary key [#49172](https://github.com/ClickHouse/ClickHouse/pull/49172) ([Duc Canh Le](https://github.com/canhld94)). +* Fix reinterpretAs*() on big endian machines [#49198](https://github.com/ClickHouse/ClickHouse/pull/49198) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* (Experimental zero-copy replication) Lock zero copy parts more atomically [#49211](https://github.com/ClickHouse/ClickHouse/pull/49211) ([alesapin](https://github.com/alesapin)). +* Fix race on Outdated parts loading [#49223](https://github.com/ClickHouse/ClickHouse/pull/49223) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix all key value is null and group use rollup return wrong answer [#49282](https://github.com/ClickHouse/ClickHouse/pull/49282) ([Shuai li](https://github.com/loneylee)). +* Fix calculating load_factor for HASHED dictionaries with SHARDS [#49319](https://github.com/ClickHouse/ClickHouse/pull/49319) ([Azat Khuzhin](https://github.com/azat)). +* Disallow configuring compression CODECs for alias columns [#49363](https://github.com/ClickHouse/ClickHouse/pull/49363) ([Timur Solodovnikov](https://github.com/tsolodov)). +* Fix bug in removal of existing part directory [#49365](https://github.com/ClickHouse/ClickHouse/pull/49365) ([alesapin](https://github.com/alesapin)). +* Properly fix GCS when HMAC is used [#49390](https://github.com/ClickHouse/ClickHouse/pull/49390) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix fuzz bug when subquery set is not built when reading from remote() [#49425](https://github.com/ClickHouse/ClickHouse/pull/49425) ([Alexander Gololobov](https://github.com/davenger)). +* Invert `shutdown_wait_unfinished_queries` [#49427](https://github.com/ClickHouse/ClickHouse/pull/49427) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* (Experimental zero-copy replication) Fix another zero copy bug [#49473](https://github.com/ClickHouse/ClickHouse/pull/49473) ([alesapin](https://github.com/alesapin)). +* Fix postgres database setting [#49481](https://github.com/ClickHouse/ClickHouse/pull/49481) ([Mal Curtis](https://github.com/snikch)). +* Correctly handle `s3Cluster` arguments [#49490](https://github.com/ClickHouse/ClickHouse/pull/49490) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix bug in TraceCollector destructor. [#49508](https://github.com/ClickHouse/ClickHouse/pull/49508) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix AsynchronousReadIndirectBufferFromRemoteFS breaking on short seeks [#49525](https://github.com/ClickHouse/ClickHouse/pull/49525) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix dictionaries loading order [#49560](https://github.com/ClickHouse/ClickHouse/pull/49560) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Forbid the change of data type of Object('json') column [#49563](https://github.com/ClickHouse/ClickHouse/pull/49563) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix stress test (Logical error: Expected 7134 >= 11030) [#49623](https://github.com/ClickHouse/ClickHouse/pull/49623) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix bug in DISTINCT [#49628](https://github.com/ClickHouse/ClickHouse/pull/49628) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix: DISTINCT in order with zero values in non-sorted columns [#49636](https://github.com/ClickHouse/ClickHouse/pull/49636) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix one-off error in big integers found by UBSan with fuzzer [#49645](https://github.com/ClickHouse/ClickHouse/pull/49645) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix reading from sparse columns after restart [#49660](https://github.com/ClickHouse/ClickHouse/pull/49660) ([Anton Popov](https://github.com/CurtizJ)). +* Fix assert in SpanHolder::finish() with fibers [#49673](https://github.com/ClickHouse/ClickHouse/pull/49673) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix short circuit functions and mutations with sparse arguments [#49716](https://github.com/ClickHouse/ClickHouse/pull/49716) ([Anton Popov](https://github.com/CurtizJ)). +* Fix writing appended files to incremental backups [#49725](https://github.com/ClickHouse/ClickHouse/pull/49725) ([Vitaly Baranov](https://github.com/vitlibar)). +* Ignore LWD column in checkPartDynamicColumns [#49737](https://github.com/ClickHouse/ClickHouse/pull/49737) ([Alexander Gololobov](https://github.com/davenger)). +* Fix msan issue in randomStringUTF8(uneven number) [#49750](https://github.com/ClickHouse/ClickHouse/pull/49750) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix aggregate function kolmogorovSmirnovTest [#49768](https://github.com/ClickHouse/ClickHouse/pull/49768) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Fix settings aliases in native protocol [#49776](https://github.com/ClickHouse/ClickHouse/pull/49776) ([Azat Khuzhin](https://github.com/azat)). +* Fix `arrayMap` with array of tuples with single argument [#49789](https://github.com/ClickHouse/ClickHouse/pull/49789) ([Anton Popov](https://github.com/CurtizJ)). +* Fix per-query IO/BACKUPs throttling settings [#49797](https://github.com/ClickHouse/ClickHouse/pull/49797) ([Azat Khuzhin](https://github.com/azat)). +* Fix setting NULL in profile definition [#49831](https://github.com/ClickHouse/ClickHouse/pull/49831) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix a bug with projections and the aggregate_functions_null_for_empty setting (for query_plan_optimize_projection) [#49873](https://github.com/ClickHouse/ClickHouse/pull/49873) ([Amos Bird](https://github.com/amosbird)). +* Fix processing pending batch for Distributed async INSERT after restart [#49884](https://github.com/ClickHouse/ClickHouse/pull/49884) ([Azat Khuzhin](https://github.com/azat)). +* Fix assertion in CacheMetadata::doCleanup [#49914](https://github.com/ClickHouse/ClickHouse/pull/49914) ([Kseniia Sumarokova](https://github.com/kssenii)). +* fix `is_prefix` in OptimizeRegularExpression [#49919](https://github.com/ClickHouse/ClickHouse/pull/49919) ([Han Fei](https://github.com/hanfei1991)). +* Fix metrics `WriteBufferFromS3Bytes`, `WriteBufferFromS3Microseconds` and `WriteBufferFromS3RequestsErrors` [#49930](https://github.com/ClickHouse/ClickHouse/pull/49930) ([Aleksandr Musorin](https://github.com/AVMusorin)). +* Fix IPv6 encoding in protobuf [#49933](https://github.com/ClickHouse/ClickHouse/pull/49933) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix possible Logical error on bad Nullable parsing for text formats [#49960](https://github.com/ClickHouse/ClickHouse/pull/49960) ([Kruglov Pavel](https://github.com/Avogar)). +* Add setting output_format_parquet_compliant_nested_types to produce more compatible Parquet files [#50001](https://github.com/ClickHouse/ClickHouse/pull/50001) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix logical error in stress test "Not enough space to add ..." [#50021](https://github.com/ClickHouse/ClickHouse/pull/50021) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Avoid deadlock when starting table in attach thread of `ReplicatedMergeTree` [#50026](https://github.com/ClickHouse/ClickHouse/pull/50026) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix assert in SpanHolder::finish() with fibers attempt 2 [#50034](https://github.com/ClickHouse/ClickHouse/pull/50034) ([Kruglov Pavel](https://github.com/Avogar)). +* Add proper escaping for DDL OpenTelemetry context serialization [#50045](https://github.com/ClickHouse/ClickHouse/pull/50045) ([Azat Khuzhin](https://github.com/azat)). +* Fix reporting broken projection parts [#50052](https://github.com/ClickHouse/ClickHouse/pull/50052) ([Amos Bird](https://github.com/amosbird)). +* JIT compilation not equals NaN fix [#50056](https://github.com/ClickHouse/ClickHouse/pull/50056) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix crashing in case of Replicated database without arguments [#50058](https://github.com/ClickHouse/ClickHouse/pull/50058) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash with `multiIf` and constant condition and nullable arguments [#50123](https://github.com/ClickHouse/ClickHouse/pull/50123) ([Anton Popov](https://github.com/CurtizJ)). +* Fix invalid index analysis for date related keys [#50153](https://github.com/ClickHouse/ClickHouse/pull/50153) ([Amos Bird](https://github.com/amosbird)). +* do not allow modify order by when there are no order by cols [#50154](https://github.com/ClickHouse/ClickHouse/pull/50154) ([Han Fei](https://github.com/hanfei1991)). +* Fix broken index analysis when binary operator contains a null constant argument [#50177](https://github.com/ClickHouse/ClickHouse/pull/50177) ([Amos Bird](https://github.com/amosbird)). +* clickhouse-client: disallow usage of `--query` and `--queries-file` at the same time [#50210](https://github.com/ClickHouse/ClickHouse/pull/50210) ([Alexey Gerasimchuk](https://github.com/Demilivor)). +* Fix UB for INTO OUTFILE extensions (APPEND / AND STDOUT) and WATCH EVENTS [#50216](https://github.com/ClickHouse/ClickHouse/pull/50216) ([Azat Khuzhin](https://github.com/azat)). +* Fix skipping spaces at end of row in CustomSeparatedIgnoreSpaces format [#50224](https://github.com/ClickHouse/ClickHouse/pull/50224) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix iceberg metadata parsing [#50232](https://github.com/ClickHouse/ClickHouse/pull/50232) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix nested distributed SELECT in WITH clause [#50234](https://github.com/ClickHouse/ClickHouse/pull/50234) ([Azat Khuzhin](https://github.com/azat)). +* Fix msan issue in keyed siphash [#50245](https://github.com/ClickHouse/ClickHouse/pull/50245) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bugs in Poco sockets in non-blocking mode, use true non-blocking sockets [#50252](https://github.com/ClickHouse/ClickHouse/pull/50252) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix checksum calculation for backup entries [#50264](https://github.com/ClickHouse/ClickHouse/pull/50264) ([Vitaly Baranov](https://github.com/vitlibar)). +* Comparison functions NaN fix [#50287](https://github.com/ClickHouse/ClickHouse/pull/50287) ([Maksim Kita](https://github.com/kitaisreal)). +* JIT aggregation nullable key fix [#50291](https://github.com/ClickHouse/ClickHouse/pull/50291) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix clickhouse-local crashing when writing empty Arrow or Parquet output [#50328](https://github.com/ClickHouse/ClickHouse/pull/50328) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix crash when Pool::Entry::disconnect() is called [#50334](https://github.com/ClickHouse/ClickHouse/pull/50334) ([Val Doroshchuk](https://github.com/valbok)). +* Improved fetch part by holding directory lock longer [#50339](https://github.com/ClickHouse/ClickHouse/pull/50339) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix bitShift* functions with both constant arguments [#50343](https://github.com/ClickHouse/ClickHouse/pull/50343) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix Keeper deadlock on exception when preprocessing requests. [#50387](https://github.com/ClickHouse/ClickHouse/pull/50387) ([frinkr](https://github.com/frinkr)). +* Fix hashing of const integer values [#50421](https://github.com/ClickHouse/ClickHouse/pull/50421) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix merge_tree_min_rows_for_seek/merge_tree_min_bytes_for_seek for data skipping indexes [#50432](https://github.com/ClickHouse/ClickHouse/pull/50432) ([Azat Khuzhin](https://github.com/azat)). +* Limit the number of in-flight tasks for loading outdated parts [#50450](https://github.com/ClickHouse/ClickHouse/pull/50450) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Keeper fix: apply uncommitted state after snapshot install [#50483](https://github.com/ClickHouse/ClickHouse/pull/50483) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix incorrect constant folding [#50536](https://github.com/ClickHouse/ClickHouse/pull/50536) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix logical error in stress test (Not enough space to add ...) [#50583](https://github.com/ClickHouse/ClickHouse/pull/50583) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix converting Null to LowCardinality(Nullable) in values table function [#50637](https://github.com/ClickHouse/ClickHouse/pull/50637) ([Kruglov Pavel](https://github.com/Avogar)). +* Revert invalid RegExpTreeDictionary optimization [#50642](https://github.com/ClickHouse/ClickHouse/pull/50642) ([Johann Gan](https://github.com/johanngan)). + ### ClickHouse release 23.4, 2023-04-26 #### Backward Incompatible Change From e66531affac094567d95d038cfbf8c9ca90027ae Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 8 Jun 2023 12:50:26 +0200 Subject: [PATCH 1366/2223] Update test --- tests/queries/0_stateless/02782_bitmap_overflow.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02782_bitmap_overflow.sql b/tests/queries/0_stateless/02782_bitmap_overflow.sql index 656a3e7c144..71ddce5c3b9 100644 --- a/tests/queries/0_stateless/02782_bitmap_overflow.sql +++ b/tests/queries/0_stateless/02782_bitmap_overflow.sql @@ -1,2 +1,4 @@ +-- Tags: no-msan, no-asan + select unhex('0181808080908380808000')::AggregateFunction(groupBitmap, UInt64); -- {serverError TOO_LARGE_ARRAY_SIZE} From 44ee530b4d50ed09c39133fb81e827e6c3402a31 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 8 Jun 2023 11:04:01 +0000 Subject: [PATCH 1367/2223] Kill gdb in clickhouse-test before getting stacktraces --- tests/clickhouse-test | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index d8fad77b95c..a7ec7b15e16 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -340,9 +340,22 @@ def get_transactions_list(args): return f"Cannot get list of transactions: {e}" +def kill_gdb(): + for i in range(5): + code = subprocess.call("kill -TERM $(pidof gdb)", shell=True, stderr=subprocess.STDOUT, timeout=30) + if code != 0: + time.sleep(i) + else: + break + # collect server stacktraces using gdb def get_stacktraces_from_gdb(server_pid): try: + # We could attach gdb to clickhouse-server before running some tests + # to print stacktraces of all crashes even if clickhouse cannot print it for some reason. + # We should kill existing gdb if any before starting new one. + kill_gdb() + cmd = f"gdb -batch -ex 'thread apply all backtrace' -p {server_pid}" return subprocess.check_output(cmd, shell=True).decode("utf-8") except Exception as e: From 54414be47b33f352bea49ee51cc2502f1f41b21d Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 8 Jun 2023 11:14:43 +0000 Subject: [PATCH 1368/2223] Better --- tests/clickhouse-test | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a7ec7b15e16..56cf2f0ce0f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -340,9 +340,14 @@ def get_transactions_list(args): return f"Cannot get list of transactions: {e}" -def kill_gdb(): +def kill_gdb_if_any(): + # Check if we have running gdb. + code = subprocess.call("pidof gdb", shell=True) + if code != 0: + return + for i in range(5): - code = subprocess.call("kill -TERM $(pidof gdb)", shell=True, stderr=subprocess.STDOUT, timeout=30) + code = subprocess.call("kill -TERM $(pidof gdb)", shell=True, timeout=30) if code != 0: time.sleep(i) else: @@ -354,7 +359,7 @@ def get_stacktraces_from_gdb(server_pid): # We could attach gdb to clickhouse-server before running some tests # to print stacktraces of all crashes even if clickhouse cannot print it for some reason. # We should kill existing gdb if any before starting new one. - kill_gdb() + kill_gdb_if_any() cmd = f"gdb -batch -ex 'thread apply all backtrace' -p {server_pid}" return subprocess.check_output(cmd, shell=True).decode("utf-8") From c37b80593c0db79b064bf5e54a817ef90238a343 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Thu, 8 Jun 2023 13:22:55 +0200 Subject: [PATCH 1369/2223] MaterializedMySQL: Add support of `TRUNCATE db.table` (#50624) Additional to `TRUNCATE TABLE db.table`. Co-authored-by: Alexander Tokmakov --- src/Parsers/MySQL/ASTDropQuery.cpp | 3 +- .../materialize_with_ddl.py | 40 +++++++++++++++++++ 2 files changed, 42 insertions(+), 1 deletion(-) diff --git a/src/Parsers/MySQL/ASTDropQuery.cpp b/src/Parsers/MySQL/ASTDropQuery.cpp index fb76d93363a..890451e3e55 100644 --- a/src/Parsers/MySQL/ASTDropQuery.cpp +++ b/src/Parsers/MySQL/ASTDropQuery.cpp @@ -44,8 +44,9 @@ bool ParserDropQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & ex bool if_exists = false; bool is_truncate = false; - if (s_truncate.ignore(pos, expected) && s_table.ignore(pos, expected)) + if (s_truncate.ignore(pos, expected)) { + s_table.ignore(pos, expected); is_truncate = true; query->kind = ASTDropQuery::Kind::Table; ASTDropQuery::QualifiedName name; diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 97e2de49ceb..2bbbe9a3f13 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -379,6 +379,46 @@ def drop_table_with_materialized_mysql_database( "", ) + mysql_node.query( + "CREATE TABLE test_database_drop.test_table_3 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB" + ) + mysql_node.query("INSERT INTO test_database_drop.test_table_3 VALUES(1), (2)") + check_query( + clickhouse_node, + "SHOW TABLES FROM test_database_drop FORMAT TSV", + "test_table_2\ntest_table_3\n", + ) + check_query( + clickhouse_node, + "SELECT * FROM test_database_drop.test_table_3 ORDER BY id FORMAT TSV", + "1\n2\n", + ) + mysql_node.query("TRUNCATE test_database_drop.test_table_3") + check_query( + clickhouse_node, + "SELECT * FROM test_database_drop.test_table_3 ORDER BY id FORMAT TSV", + "", + ) + + mysql_node.query( + "CREATE TABLE test_database_drop.test_table_4 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB" + ) + mysql_node.query("INSERT INTO test_database_drop.test_table_4 VALUES(1), (2)") + check_query( + clickhouse_node, + "SELECT * FROM test_database_drop.test_table_4 ORDER BY id FORMAT TSV", + "1\n2\n", + ) + with mysql_node.alloc_connection() as mysql: + mysql.query("USE test_database_drop") + mysql.query("TRUNCATE test_table_4") + + check_query( + clickhouse_node, + "SELECT * FROM test_database_drop.test_table_4 ORDER BY id FORMAT TSV", + "", + ) + clickhouse_node.query("DROP DATABASE test_database_drop") mysql_node.query("DROP DATABASE test_database_drop") From 6b49816079016e362110da78e4e8b660894e9b6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Jun 2023 14:25:49 +0300 Subject: [PATCH 1370/2223] Update CHANGELOG.md Co-authored-by: Alexander Gololobov <440544+davenger@users.noreply.github.com> --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f2ffdad9a7c..959a7bad5dd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -210,7 +210,7 @@ * Fix assert in SpanHolder::finish() with fibers [#49673](https://github.com/ClickHouse/ClickHouse/pull/49673) ([Kruglov Pavel](https://github.com/Avogar)). * Fix short circuit functions and mutations with sparse arguments [#49716](https://github.com/ClickHouse/ClickHouse/pull/49716) ([Anton Popov](https://github.com/CurtizJ)). * Fix writing appended files to incremental backups [#49725](https://github.com/ClickHouse/ClickHouse/pull/49725) ([Vitaly Baranov](https://github.com/vitlibar)). -* Ignore LWD column in checkPartDynamicColumns [#49737](https://github.com/ClickHouse/ClickHouse/pull/49737) ([Alexander Gololobov](https://github.com/davenger)). +* Fix "There is no physical column _row_exists in table" error occurring during lightweight delete mutation on a table with Object column. [#49737](https://github.com/ClickHouse/ClickHouse/pull/49737) ([Alexander Gololobov](https://github.com/davenger)). * Fix msan issue in randomStringUTF8(uneven number) [#49750](https://github.com/ClickHouse/ClickHouse/pull/49750) ([Robert Schulze](https://github.com/rschu1ze)). * Fix aggregate function kolmogorovSmirnovTest [#49768](https://github.com/ClickHouse/ClickHouse/pull/49768) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). * Fix settings aliases in native protocol [#49776](https://github.com/ClickHouse/ClickHouse/pull/49776) ([Azat Khuzhin](https://github.com/azat)). From c6b2ed234dcde2802020f730dc84101d7ad84f65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Jun 2023 13:27:13 +0200 Subject: [PATCH 1371/2223] Fix typos --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f2ffdad9a7c..c674af5852a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,7 +16,7 @@ * The experimental feature "in-memory data parts" is removed. The data format is still supported, but the settings are no-op, and compact or wide parts will be used instead. This closes [#45409](https://github.com/ClickHouse/ClickHouse/issues/45409). [#49429](https://github.com/ClickHouse/ClickHouse/pull/49429) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Changed default values of settings `parallelize_output_from_storages` and `input_format_parquet_preserve_order`. This allows ClickHouse to reorder rows when reading from files (e.g. CSV or Parquet), greatly improving performance in many cases. To restore the old behavior of preserving order, use `parallelize_output_from_storages = 0`, `input_format_parquet_preserve_order = 1`. [#49479](https://github.com/ClickHouse/ClickHouse/pull/49479) ([Michael Kolupaev](https://github.com/al13n321)). * Make projections production-ready. Add the `optimize_use_projections` setting to control whether the projections will be selected for SELECT queries. The setting `allow_experimental_projection_optimization` is obsolete and does nothing. [#49719](https://github.com/ClickHouse/ClickHouse/pull/49719) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Mark `joinGet` as non deterministic (so as `dictGet`). It allows using them in mutations without an extra setting. [#49843](https://github.com/ClickHouse/ClickHouse/pull/49843) ([Azat Khuzhin](https://github.com/azat)). +* Mark `joinGet` as non-deterministic (so as `dictGet`). It allows using them in mutations without an extra setting. [#49843](https://github.com/ClickHouse/ClickHouse/pull/49843) ([Azat Khuzhin](https://github.com/azat)). * Revert the "`groupArray` returns cannot be nullable" change (due to binary compatibility breakage for `groupArray`/`groupArrayLast`/`groupArraySample` over `Nullable` types, which likely will lead to `TOO_LARGE_ARRAY_SIZE` or `CANNOT_READ_ALL_DATA`). [#49971](https://github.com/ClickHouse/ClickHouse/pull/49971) ([Azat Khuzhin](https://github.com/azat)). * Setting `enable_memory_bound_merging_of_aggregation_results` is enabled by default. If you update from version prior to 22.12, we recommend to set this flag to `false` until update is finished. [#50319](https://github.com/ClickHouse/ClickHouse/pull/50319) ([Nikita Taranov](https://github.com/nickitat)). @@ -68,7 +68,7 @@ * Improve performance of BLAKE3 by 11% by enabling LTO for Rust. [#49600](https://github.com/ClickHouse/ClickHouse/pull/49600) ([Azat Khuzhin](https://github.com/azat)). Now it is on par with C++. * Optimize the structure of the `system.opentelemetry_span_log`. Use `LowCardinality` where appropriate. Although this table is generally stupid (it is using the Map data type even for common attributes), it will be slightly better. [#49647](https://github.com/ClickHouse/ClickHouse/pull/49647) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Try to reserve hash table's size in `grace_hash` join. [#49816](https://github.com/ClickHouse/ClickHouse/pull/49816) ([lgbo](https://github.com/lgbo-ustc)). -* As is addresed in issue [#49748](https://github.com/ClickHouse/ClickHouse/issues/49748), the predicates with date converters, such as `toYear`, `toYYYYMM`, could be rewritten with the equivalent date (YYYY-MM-DD) comparisons at the AST level. And this transformation could bring performance improvement as it is free from the expensive date converter and the comparison between dates (or integers in the low level representation) is quite low-cost. The [prototype](https://github.com/ZhiguoZh/ClickHouse/commit/c7f1753f0c9363a19d95fa46f1cfed1d9f505ee0) shows that, with all identified date converters optimized, the overall QPS of the 13 queries is enhanced by **~11%** on the ICX server (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads). [#50062](https://github.com/ClickHouse/ClickHouse/pull/50062) [#50307](https://github.com/ClickHouse/ClickHouse/pull/50307) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* As is addressed in issue [#49748](https://github.com/ClickHouse/ClickHouse/issues/49748), the predicates with date converters, such as `toYear`, `toYYYYMM`, could be rewritten with the equivalent date (YYYY-MM-DD) comparisons at the AST level. And this transformation could bring performance improvement as it is free from the expensive date converter and the comparison between dates (or integers in the low level representation) is quite low-cost. The [prototype](https://github.com/ZhiguoZh/ClickHouse/commit/c7f1753f0c9363a19d95fa46f1cfed1d9f505ee0) shows that, with all identified date converters optimized, the overall QPS of the 13 queries is enhanced by **~11%** on the ICX server (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads). [#50062](https://github.com/ClickHouse/ClickHouse/pull/50062) [#50307](https://github.com/ClickHouse/ClickHouse/pull/50307) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). * Parallel merge of `uniqExactIf` states. Closes [#49885](https://github.com/ClickHouse/ClickHouse/issues/49885). [#50285](https://github.com/ClickHouse/ClickHouse/pull/50285) ([flynn](https://github.com/ucasfl)). * Keeper improvement: add `CheckNotExists` request to Keeper, which allows to improve the performance of Replicated tables. [#48897](https://github.com/ClickHouse/ClickHouse/pull/48897) ([Antonio Andelic](https://github.com/antonio2368)). * Keeper performance improvements: avoid serializing same request twice while processing. Cache deserialization results of large requests. Controlled by new coordination setting `min_request_size_for_cache`. [#49004](https://github.com/ClickHouse/ClickHouse/pull/49004) ([Antonio Andelic](https://github.com/antonio2368)). From 2aaaec2abebe4860120bd5db1cf6a1b38c28a12b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 8 Jun 2023 11:51:57 +0200 Subject: [PATCH 1372/2223] Catch issues with dockerd during the build --- tests/ci/build_check.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 4bc61c79fc0..35b98a7c3bb 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -343,6 +343,15 @@ def main(): f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True ) logging.info("Build finished with %s, log path %s", success, log_path) + if not success: + # We check if docker works, because if it's down, it's infrastructure + try: + subprocess.check_call("docker info", shell=True) + except subprocess.CalledProcessError: + logging.error( + "The dockerd looks down, won't upload anything and generate report" + ) + sys.exit(1) # FIXME performance performance_urls = [] From 7079b4c885656fba788400002f012d3ff43e01de Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 8 Jun 2023 12:40:16 +0300 Subject: [PATCH 1373/2223] ReverseTransform small improvement --- src/Processors/Transforms/ReverseTransform.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/ReverseTransform.cpp b/src/Processors/Transforms/ReverseTransform.cpp index 98f2bf54aa5..66b774ab50d 100644 --- a/src/Processors/Transforms/ReverseTransform.cpp +++ b/src/Processors/Transforms/ReverseTransform.cpp @@ -6,11 +6,11 @@ namespace DB void ReverseTransform::transform(Chunk & chunk) { - IColumn::Permutation permutation; - size_t num_rows = chunk.getNumRows(); + IColumn::Permutation permutation(num_rows); + for (size_t i = 0; i < num_rows; ++i) - permutation.emplace_back(num_rows - 1 - i); + permutation[i] = num_rows - 1 - i; auto columns = chunk.detachColumns(); From 64783194138fe4f823da38deb749bef44e15bf68 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 8 Jun 2023 09:24:11 -0300 Subject: [PATCH 1374/2223] Update CHANGELOG.md --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a5b6dd85941..4cf1023cac2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -68,7 +68,6 @@ * Improve performance of BLAKE3 by 11% by enabling LTO for Rust. [#49600](https://github.com/ClickHouse/ClickHouse/pull/49600) ([Azat Khuzhin](https://github.com/azat)). Now it is on par with C++. * Optimize the structure of the `system.opentelemetry_span_log`. Use `LowCardinality` where appropriate. Although this table is generally stupid (it is using the Map data type even for common attributes), it will be slightly better. [#49647](https://github.com/ClickHouse/ClickHouse/pull/49647) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Try to reserve hash table's size in `grace_hash` join. [#49816](https://github.com/ClickHouse/ClickHouse/pull/49816) ([lgbo](https://github.com/lgbo-ustc)). -* As is addressed in issue [#49748](https://github.com/ClickHouse/ClickHouse/issues/49748), the predicates with date converters, such as `toYear`, `toYYYYMM`, could be rewritten with the equivalent date (YYYY-MM-DD) comparisons at the AST level. And this transformation could bring performance improvement as it is free from the expensive date converter and the comparison between dates (or integers in the low level representation) is quite low-cost. The [prototype](https://github.com/ZhiguoZh/ClickHouse/commit/c7f1753f0c9363a19d95fa46f1cfed1d9f505ee0) shows that, with all identified date converters optimized, the overall QPS of the 13 queries is enhanced by **~11%** on the ICX server (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads). [#50062](https://github.com/ClickHouse/ClickHouse/pull/50062) [#50307](https://github.com/ClickHouse/ClickHouse/pull/50307) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). * Parallel merge of `uniqExactIf` states. Closes [#49885](https://github.com/ClickHouse/ClickHouse/issues/49885). [#50285](https://github.com/ClickHouse/ClickHouse/pull/50285) ([flynn](https://github.com/ucasfl)). * Keeper improvement: add `CheckNotExists` request to Keeper, which allows to improve the performance of Replicated tables. [#48897](https://github.com/ClickHouse/ClickHouse/pull/48897) ([Antonio Andelic](https://github.com/antonio2368)). * Keeper performance improvements: avoid serializing same request twice while processing. Cache deserialization results of large requests. Controlled by new coordination setting `min_request_size_for_cache`. [#49004](https://github.com/ClickHouse/ClickHouse/pull/49004) ([Antonio Andelic](https://github.com/antonio2368)). From 1cbcd2f2ef4032cfb718433befbd8742c1e4b9cf Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 8 Jun 2023 13:39:05 +0200 Subject: [PATCH 1375/2223] Refactor reading from object storages --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 59 ++++++--- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 10 +- src/Disks/IO/ReadBufferFromWebServer.h | 2 - .../IO/ReadIndirectBufferFromRemoteFS.cpp | 118 ------------------ src/Disks/IO/ReadIndirectBufferFromRemoteFS.h | 46 ------- .../AzureBlobStorage/AzureObjectStorage.cpp | 44 ++++--- .../AzureBlobStorage/AzureObjectStorage.h | 1 - .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 8 +- .../Local/LocalObjectStorage.cpp | 39 +++--- .../ObjectStorages/S3/S3ObjectStorage.cpp | 45 ++++--- .../ObjectStorages/Web/WebObjectStorage.cpp | 43 ++++--- src/IO/SeekAvoidingReadBuffer.cpp | 35 ------ src/IO/SeekAvoidingReadBuffer.h | 26 ---- src/Storages/StorageS3.cpp | 2 +- 14 files changed, 149 insertions(+), 329 deletions(-) delete mode 100644 src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp delete mode 100644 src/Disks/IO/ReadIndirectBufferFromRemoteFS.h delete mode 100644 src/IO/SeekAvoidingReadBuffer.cpp delete mode 100644 src/IO/SeekAvoidingReadBuffer.h diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 04030fe5f8f..eb9c509e459 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -22,13 +22,15 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( ReadBufferCreator && read_buffer_creator_, const StoredObjects & blobs_to_read_, const ReadSettings & settings_, - std::shared_ptr cache_log_) - : ReadBufferFromFileBase(0, nullptr, 0) + std::shared_ptr cache_log_, + bool use_external_buffer_) + : ReadBufferFromFileBase(use_external_buffer_ ? 0 : settings_.remote_fs_buffer_size, nullptr, 0) , settings(settings_) , blobs_to_read(blobs_to_read_) , read_buffer_creator(std::move(read_buffer_creator_)) , cache_log(settings.enable_filesystem_cache_log ? cache_log_ : nullptr) , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") + , use_external_buffer(use_external_buffer_) , log(&Poco::Logger::get("ReadBufferFromRemoteFSGather")) { if (!blobs_to_read.empty()) @@ -36,7 +38,9 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( with_cache = settings.remote_fs_cache && settings.enable_filesystem_cache - && (!query_id.empty() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache || !settings.avoid_readthrough_cache_outside_query_context); + && (!query_id.empty() + || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache + || !settings.avoid_readthrough_cache_outside_query_context); } SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object) @@ -235,22 +239,49 @@ void ReadBufferFromRemoteFSGather::reset() off_t ReadBufferFromRemoteFSGather::seek(off_t offset, int whence) { - if (whence != SEEK_SET) - throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only seeking with SEEK_SET is allowed"); + if (offset == getPosition() && whence == SEEK_SET) + return offset; + + if (whence != SEEK_SET) + throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET mode is allowed."); + + if (use_external_buffer) + { + /// In case use_external_buffer == true, the buffer manages seeks itself. + reset(); + } + else + { + if (!working_buffer.empty() + && static_cast(offset) >= file_offset_of_buffer_end - working_buffer.size() + && static_cast(offset) < file_offset_of_buffer_end) + { + pos = working_buffer.end() - (file_offset_of_buffer_end - offset); + assert(pos >= working_buffer.begin()); + assert(pos < working_buffer.end()); + + return getPosition(); + } + + off_t position = getPosition(); + if (current_buf && offset > position) + { + size_t diff = offset - position; + if (diff < settings.remote_read_min_bytes_for_seek) + { + ignore(diff); + return offset; + } + } + + resetWorkingBuffer(); + reset(); + } - reset(); file_offset_of_buffer_end = offset; return file_offset_of_buffer_end; } -size_t ReadBufferFromRemoteFSGather::getImplementationBufferOffset() const -{ - if (!current_buf) - return file_offset_of_buffer_end; - - return current_buf->getFileOffsetOfBufferEnd(); -} - ReadBufferFromRemoteFSGather::~ReadBufferFromRemoteFSGather() { if (!with_cache) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 39b81d6f9ac..cb98ac6d9f3 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -27,7 +27,8 @@ public: ReadBufferCreator && read_buffer_creator_, const StoredObjects & blobs_to_read_, const ReadSettings & settings_, - std::shared_ptr cache_log_); + std::shared_ptr cache_log_, + bool use_external_buffer_); ~ReadBufferFromRemoteFSGather() override; @@ -37,16 +38,14 @@ public: void setReadUntilPosition(size_t position) override; + void setReadUntilEnd() override { return setReadUntilPosition(getFileSize()); } + IAsynchronousReader::Result readInto(char * data, size_t size, size_t offset, size_t ignore) override; size_t getFileSize() override { return getTotalSize(blobs_to_read); } size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; } - bool initialized() const { return current_buf != nullptr; } - - size_t getImplementationBufferOffset() const; - off_t seek(off_t offset, int whence) override; off_t getPosition() override { return file_offset_of_buffer_end - available() + bytes_to_ignore; } @@ -71,6 +70,7 @@ private: const ReadBufferCreator read_buffer_creator; const std::shared_ptr cache_log; const String query_id; + const bool use_external_buffer; bool with_cache; size_t read_until_position = 0; diff --git a/src/Disks/IO/ReadBufferFromWebServer.h b/src/Disks/IO/ReadBufferFromWebServer.h index dd9cf63224f..fa899cf2c5e 100644 --- a/src/Disks/IO/ReadBufferFromWebServer.h +++ b/src/Disks/IO/ReadBufferFromWebServer.h @@ -12,8 +12,6 @@ namespace DB /* Read buffer, which reads via http, but is used as ReadBufferFromFileBase. * Used to read files, hosted on a web server with static files. - * - * Usage: ReadIndirectBufferFromRemoteFS -> SeekAvoidingReadBuffer -> ReadBufferFromWebServer -> ReadWriteBufferFromHTTP. */ class ReadBufferFromWebServer : public ReadBufferFromFileBase { diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp deleted file mode 100644 index a559b47f2cc..00000000000 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp +++ /dev/null @@ -1,118 +0,0 @@ -#include "ReadIndirectBufferFromRemoteFS.h" - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_SEEK_THROUGH_FILE; -} - - -ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( - std::shared_ptr impl_, const ReadSettings & settings) - : ReadBufferFromFileBase(settings.remote_fs_buffer_size, nullptr, 0) - , impl(impl_) - , read_settings(settings) -{ -} - -size_t ReadIndirectBufferFromRemoteFS::getFileSize() -{ - return impl->getFileSize(); -} - -off_t ReadIndirectBufferFromRemoteFS::getPosition() -{ - return impl->file_offset_of_buffer_end - available(); -} - - -String ReadIndirectBufferFromRemoteFS::getFileName() const -{ - return impl->getFileName(); -} - - -void ReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t position) -{ - impl->setReadUntilPosition(position); -} - - -void ReadIndirectBufferFromRemoteFS::setReadUntilEnd() -{ - impl->setReadUntilPosition(impl->getFileSize()); -} - - -off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) -{ - if (whence == SEEK_CUR) - { - /// If position within current working buffer - shift pos. - if (!working_buffer.empty() && size_t(getPosition() + offset_) < impl->file_offset_of_buffer_end) - { - pos += offset_; - return getPosition(); - } - else - { - impl->file_offset_of_buffer_end += offset_; - } - } - else if (whence == SEEK_SET) - { - /// If position within current working buffer - shift pos. - if (!working_buffer.empty() - && size_t(offset_) >= impl->file_offset_of_buffer_end - working_buffer.size() - && size_t(offset_) < impl->file_offset_of_buffer_end) - { - pos = working_buffer.end() - (impl->file_offset_of_buffer_end - offset_); - return getPosition(); - } - else - { - impl->file_offset_of_buffer_end = offset_; - } - } - else - throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET or SEEK_CUR modes are allowed."); - - impl->seek(impl->file_offset_of_buffer_end, SEEK_SET); - resetWorkingBuffer(); - - file_offset_of_buffer_end = impl->file_offset_of_buffer_end; - return impl->file_offset_of_buffer_end; -} - - -bool ReadIndirectBufferFromRemoteFS::nextImpl() -{ - chassert(internal_buffer.size() == read_settings.remote_fs_buffer_size); - chassert(file_offset_of_buffer_end <= impl->getFileSize()); - - auto [size, offset, _] = impl->readInto(internal_buffer.begin(), internal_buffer.size(), file_offset_of_buffer_end, /* ignore */0); - - chassert(offset <= size); - chassert(size <= internal_buffer.size()); - - size_t bytes_read = size - offset; - if (bytes_read) - working_buffer = Buffer(internal_buffer.begin() + offset, internal_buffer.begin() + size); - - file_offset_of_buffer_end = impl->getFileOffsetOfBufferEnd(); - - /// In case of multiple files for the same file in clickhouse (i.e. log family) - /// file_offset_of_buffer_end will not match getImplementationBufferOffset() - /// so we use [impl->getImplementationBufferOffset(), impl->getFileSize()] - chassert(file_offset_of_buffer_end >= impl->getImplementationBufferOffset()); - chassert(file_offset_of_buffer_end <= impl->getFileSize()); - - return bytes_read; -} - -} diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h deleted file mode 100644 index 19647b1fa39..00000000000 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h +++ /dev/null @@ -1,46 +0,0 @@ -#pragma once - -#include "config.h" -#include -#include -#include - - -namespace DB -{ - -class ReadBufferFromRemoteFSGather; - -/** -* Reads data from S3/HDFS/Web using stored paths in metadata. -* There is asynchronous version of this class -- AsynchronousReadIndirectBufferFromRemoteFS. -*/ -class ReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase -{ - -public: - explicit ReadIndirectBufferFromRemoteFS(std::shared_ptr impl_, const ReadSettings & settings); - - off_t seek(off_t offset_, int whence) override; - - off_t getPosition() override; - - String getFileName() const override; - - void setReadUntilPosition(size_t position) override; - - void setReadUntilEnd() override; - - size_t getFileSize() override; - -private: - bool nextImpl() override; - - std::shared_ptr impl; - - ReadSettings read_settings; - - size_t file_offset_of_buffer_end = 0; -}; - -} diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 23a0da39dd3..3636c5780fb 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include @@ -218,24 +217,33 @@ std::unique_ptr AzureObjectStorage::readObjects( /// NOL read_until_position); }; - auto reader_impl = std::make_unique( - std::move(read_buffer_creator), - objects, - disk_read_settings, - global_context->getFilesystemCacheLog()); + switch (read_settings.remote_fs_method) + { + case RemoteFSReadMethod::read: + { + return std::make_unique( + std::move(read_buffer_creator), + objects, + disk_read_settings, + global_context->getFilesystemCacheLog(), + /* use_external_buffer */false); - if (disk_read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - { - auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - std::move(reader_impl), reader, disk_read_settings, - global_context->getAsyncReadCounters(), - global_context->getFilesystemReadPrefetchesLog()); - } - else - { - auto buf = std::make_unique(std::move(reader_impl), disk_read_settings); - return std::make_unique(std::move(buf), settings_ptr->min_bytes_for_seek); + } + case RemoteFSReadMethod::threadpool: + { + auto impl = std::make_unique( + std::move(read_buffer_creator), + objects, + disk_read_settings, + global_context->getFilesystemCacheLog(), + /* use_external_buffer */true); + + auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + return std::make_unique( + std::move(impl), reader, disk_read_settings, + global_context->getAsyncReadCounters(), + global_context->getFilesystemReadPrefetchesLog()); + } } } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 5b08ceb80e3..2fbd1514abd 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -5,7 +5,6 @@ #include #include -#include #include #include diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index a3092bc6f12..60230ce2fb0 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -1,13 +1,10 @@ #include -#include #include - #include #include #include -#include #include #include @@ -72,9 +69,8 @@ std::unique_ptr HDFSObjectStorage::readObjects( /// NOLI hdfs_uri, hdfs_path, config, disk_read_settings, /* read_until_position */0, /* use_external_buffer */true); }; - auto hdfs_impl = std::make_unique(std::move(read_buffer_creator), objects, disk_read_settings, nullptr); - auto buf = std::make_unique(std::move(hdfs_impl), read_settings); - return std::make_unique(std::move(buf), settings->min_bytes_for_seek); + return std::make_unique( + std::move(read_buffer_creator), objects, disk_read_settings, nullptr, /* use_external_buffer */false); } std::unique_ptr HDFSObjectStorage::writeObject( /// NOLINT diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 05c0c8f3961..69ccf309096 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -4,11 +4,9 @@ #include #include #include -#include #include #include #include -#include #include #include #include @@ -59,25 +57,26 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL return createReadBufferFromFileBase(file_path, modified_settings, read_hint, file_size); }; - auto impl = std::make_unique( - std::move(read_buffer_creator), objects, modified_settings, - global_context->getFilesystemCacheLog()); + switch (read_settings.remote_fs_method) + { + case RemoteFSReadMethod::read: + { + return std::make_unique( + std::move(read_buffer_creator), objects, modified_settings, + global_context->getFilesystemCacheLog(), /* use_external_buffer */false); + } + case RemoteFSReadMethod::threadpool: + { + auto impl = std::make_unique( + std::move(read_buffer_creator), objects, modified_settings, + global_context->getFilesystemCacheLog(), /* use_external_buffer */true); - /// We use `remove_fs_method` (not `local_fs_method`) because we are about to use - /// AsynchronousBoundedReadBuffer which works by the remote_fs_* settings. - if (modified_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - { - auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - std::move(impl), reader, modified_settings, - global_context->getAsyncReadCounters(), - global_context->getFilesystemReadPrefetchesLog()); - } - else - { - auto buf = std::make_unique(std::move(impl), modified_settings); - return std::make_unique( - std::move(buf), modified_settings.remote_read_min_bytes_for_seek); + auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + return std::make_unique( + std::move(impl), reader, read_settings, + global_context->getAsyncReadCounters(), + global_context->getFilesystemReadPrefetchesLog()); + } } } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index d19be20f920..e48924326e1 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -8,11 +8,9 @@ #include #include #include -#include #include #include #include -#include #include #include #include @@ -182,24 +180,33 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT /* restricted_seek */true); }; - auto s3_impl = std::make_unique( - std::move(read_buffer_creator), - objects, - disk_read_settings, - global_context->getFilesystemCacheLog()); + switch (read_settings.remote_fs_method) + { + case RemoteFSReadMethod::read: + { + return std::make_unique( + std::move(read_buffer_creator), + objects, + disk_read_settings, + global_context->getFilesystemCacheLog(), + /* use_external_buffer */false); - if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - { - auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - std::move(s3_impl), reader, disk_read_settings, - global_context->getAsyncReadCounters(), - global_context->getFilesystemReadPrefetchesLog()); - } - else - { - auto buf = std::make_unique(std::move(s3_impl), disk_read_settings); - return std::make_unique(std::move(buf), settings_ptr->min_bytes_for_seek); + } + case RemoteFSReadMethod::threadpool: + { + auto impl = std::make_unique( + std::move(read_buffer_creator), + objects, + disk_read_settings, + global_context->getFilesystemCacheLog(), + /* use_external_buffer */true); + + auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + return std::make_unique( + std::move(impl), reader, disk_read_settings, + global_context->getAsyncReadCounters(), + global_context->getFilesystemReadPrefetchesLog()); + } } } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 4f34f3eed9c..690a0d3372c 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -4,11 +4,9 @@ #include #include -#include #include #include -#include #include #include #include @@ -181,24 +179,33 @@ std::unique_ptr WebObjectStorage::readObject( /// NOLINT }; auto global_context = Context::getGlobalContextInstance(); - auto web_impl = std::make_unique( - std::move(read_buffer_creator), - StoredObjects{object}, - read_settings, - global_context->getFilesystemCacheLog()); - if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) + switch (read_settings.remote_fs_method) { - auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - return std::make_unique( - std::move(web_impl), reader, read_settings, - global_context->getAsyncReadCounters(), - global_context->getFilesystemReadPrefetchesLog()); - } - else - { - auto buf = std::make_unique(std::move(web_impl), read_settings); - return std::make_unique(std::move(buf), min_bytes_for_seek); + case RemoteFSReadMethod::read: + { + return std::make_unique( + std::move(read_buffer_creator), + StoredObjects{object}, + read_settings, + global_context->getFilesystemCacheLog(), + /* use_external_buffer */false); + } + case RemoteFSReadMethod::threadpool: + { + auto impl = std::make_unique( + std::move(read_buffer_creator), + StoredObjects{object}, + read_settings, + global_context->getFilesystemCacheLog(), + /* use_external_buffer */true); + + auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + return std::make_unique( + std::move(impl), reader, read_settings, + global_context->getAsyncReadCounters(), + global_context->getFilesystemReadPrefetchesLog()); + } } } diff --git a/src/IO/SeekAvoidingReadBuffer.cpp b/src/IO/SeekAvoidingReadBuffer.cpp deleted file mode 100644 index 4d6406d8ddf..00000000000 --- a/src/IO/SeekAvoidingReadBuffer.cpp +++ /dev/null @@ -1,35 +0,0 @@ -#include - - -namespace DB -{ - -SeekAvoidingReadBuffer::SeekAvoidingReadBuffer(std::unique_ptr impl_, UInt64 min_bytes_for_seek_) - : ReadBufferFromFileDecorator(std::move(impl_)) - , min_bytes_for_seek(min_bytes_for_seek_) -{ -} - - -off_t SeekAvoidingReadBuffer::seek(off_t off, int whence) -{ - off_t position = getPosition(); - - if (whence == SEEK_CUR) - { - off += position; - whence = SEEK_SET; - } - - if (whence == SEEK_SET && off >= position && off < position + static_cast(min_bytes_for_seek)) - { - swap(*impl); - impl->ignore(off - position); - swap(*impl); - return off; - } - - return ReadBufferFromFileDecorator::seek(off, whence); -} - -} diff --git a/src/IO/SeekAvoidingReadBuffer.h b/src/IO/SeekAvoidingReadBuffer.h deleted file mode 100644 index 716d7c5046c..00000000000 --- a/src/IO/SeekAvoidingReadBuffer.h +++ /dev/null @@ -1,26 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/// `SeekAvoidingReadBuffer` prefers sequential reads over seeks within specified window. -/// It is useful in network and spinning disk storage media when seek is relatively expensive -/// operation. -/// See also: `merge_tree_min_rows_for_seek`. -class SeekAvoidingReadBuffer : public ReadBufferFromFileDecorator -{ -public: - SeekAvoidingReadBuffer(std::unique_ptr impl_, UInt64 min_bytes_for_seek_); - - off_t seek(off_t off, int whence) override; - - void prefetch(Priority priority) override { impl->prefetch(priority); } - -private: - UInt64 min_bytes_for_seek; /// Minimum positive seek offset which shall be executed using seek operation. -}; - -} diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index c3ed0f1af16..8bab596901c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -657,7 +657,7 @@ std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( std::move(read_buffer_creator), StoredObjects{StoredObject{key, object_size}}, read_settings, - /* cache_log */nullptr); + /* cache_log */nullptr, /* use_external_buffer */true); auto modified_settings{read_settings}; /// FIXME: Changing this setting to default value breaks something around parquet reading From 48e1b21aabd8ef642c1bcab7ba863f8c61123723 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 8 Jun 2023 20:34:30 +0800 Subject: [PATCH 1376/2223] Add feature to support read csv by space & tab delimiter --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/CSVRowInputFormat.cpp | 36 ++++++++++--------- ...h_whitespace_tab_field_delimiter.reference | 2 ++ ...ext_with_whitespace_tab_field_delimiter.sh | 18 ++++++++++ .../data_csv/csv_with_space_delimiter.csv | 1 + .../data_csv/csv_with_tab_delimiter.csv | 1 + 8 files changed, 45 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.reference create mode 100755 tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh create mode 100644 tests/queries/0_stateless/data_csv/csv_with_space_delimiter.csv create mode 100644 tests/queries/0_stateless/data_csv/csv_with_tab_delimiter.csv diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a87e321bed2..45641e76689 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -850,6 +850,7 @@ class IColumn; M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ + M(Bool, input_format_csv_skip_whitespaces_tabs, true, "Skips spaces and tabs(\\t) characters in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 39b28e025a6..73a7d4f73f2 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -70,6 +70,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.skip_first_lines = settings.input_format_csv_skip_first_lines; format_settings.csv.try_detect_header = settings.input_format_csv_detect_header; format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; + format_settings.csv.skip_whitespaces_tabs = settings.input_format_csv_skip_whitespaces_tabs; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 6b4caffbf43..434389e31a1 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -137,6 +137,7 @@ struct FormatSettings String custom_delimiter; bool try_detect_header = true; bool trim_whitespaces = true; + bool skip_whitespaces_tabs = true; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 8b4dbbffe1d..4094285e1ad 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -25,10 +25,10 @@ namespace ErrorCodes namespace { - void checkBadDelimiter(char delimiter) + void checkBadDelimiter(char delimiter, bool skip_whitespaces_tabs) { constexpr std::string_view bad_delimiters = " \t\"'.UL"; - if (bad_delimiters.find(delimiter) != std::string_view::npos) + if (bad_delimiters.find(delimiter) != std::string_view::npos && skip_whitespaces_tabs) throw Exception( ErrorCodes::BAD_ARGUMENTS, "CSV format may not work correctly with delimiter '{}'. Try use CustomSeparated format instead", @@ -68,7 +68,7 @@ CSVRowInputFormat::CSVRowInputFormat( format_settings_.csv.try_detect_header), buf(std::move(in_)) { - checkBadDelimiter(format_settings_.csv.delimiter); + checkBadDelimiter(format_settings_.csv.delimiter, format_settings_.csv.skip_whitespaces_tabs); } CSVRowInputFormat::CSVRowInputFormat( @@ -90,7 +90,7 @@ CSVRowInputFormat::CSVRowInputFormat( format_settings_.csv.try_detect_header), buf(std::move(in_)) { - checkBadDelimiter(format_settings_.csv.delimiter); + checkBadDelimiter(format_settings_.csv.delimiter, format_settings_.csv.skip_whitespaces_tabs); } void CSVRowInputFormat::syncAfterError() @@ -134,8 +134,12 @@ static void skipEndOfLine(ReadBuffer & in) } /// Skip `whitespace` symbols allowed in CSV. -static inline void skipWhitespacesAndTabs(ReadBuffer & in) +static inline void skipWhitespacesAndTabs(ReadBuffer & in, const bool & skip_whitespaces_tabs) { + if (!skip_whitespaces_tabs) + { + return; + } while (!in.eof() && (*in.position() == ' ' || *in.position() == '\t')) ++in.position(); } @@ -146,7 +150,7 @@ CSVFormatReader::CSVFormatReader(PeekableReadBuffer & buf_, const FormatSettings void CSVFormatReader::skipFieldDelimiter() { - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); assertChar(format_settings.csv.delimiter, *buf); } @@ -154,7 +158,7 @@ template String CSVFormatReader::readCSVFieldIntoString() { if (format_settings.csv.trim_whitespaces) [[likely]] - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); String field; if constexpr (read_string) @@ -166,14 +170,14 @@ String CSVFormatReader::readCSVFieldIntoString() void CSVFormatReader::skipField() { - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); NullOutput out; readCSVStringInto(out, *buf, format_settings.csv); } void CSVFormatReader::skipRowEndDelimiter() { - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); if (buf->eof()) return; @@ -182,7 +186,7 @@ void CSVFormatReader::skipRowEndDelimiter() if (*buf->position() == format_settings.csv.delimiter) ++buf->position(); - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); if (buf->eof()) return; @@ -194,7 +198,7 @@ void CSVFormatReader::skipHeaderRow() do { skipField(); - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); } while (checkChar(format_settings.csv.delimiter, *buf)); skipRowEndDelimiter(); @@ -207,7 +211,7 @@ std::vector CSVFormatReader::readRowImpl() do { fields.push_back(readCSVFieldIntoString()); - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); } while (checkChar(format_settings.csv.delimiter, *buf)); skipRowEndDelimiter(); @@ -220,7 +224,7 @@ bool CSVFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) try { - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); assertChar(delimiter, *buf); } catch (const DB::Exception &) @@ -246,7 +250,7 @@ bool CSVFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) { - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); if (buf->eof()) return true; @@ -255,7 +259,7 @@ bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) if (*buf->position() == format_settings.csv.delimiter) { ++buf->position(); - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); if (buf->eof()) return true; } @@ -283,7 +287,7 @@ bool CSVFormatReader::readField( const String & /*column_name*/) { if (format_settings.csv.trim_whitespaces || !isStringOrFixedString(removeNullable(type))) [[likely]] - skipWhitespacesAndTabs(*buf); + skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; const bool at_last_column_line_end = is_last_file_column && (buf->eof() || *buf->position() == '\n' || *buf->position() == '\r'); diff --git a/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.reference b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.reference new file mode 100644 index 00000000000..531391394a7 --- /dev/null +++ b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.reference @@ -0,0 +1,2 @@ +1 a b +2 c d diff --git a/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh new file mode 100755 index 00000000000..19d343c352f --- /dev/null +++ b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +# NOTE: this sh wrapper is required because of shell_config + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_whitespace" +$CLICKHOUSE_CLIENT -q "drop table if exists test_tab" +$CLICKHOUSE_CLIENT -q "create table test_whitespace (x UInt32, y String, z String) engine=MergeTree order by x" +$CLICKHOUSE_CLIENT -q "create table test_tab (x UInt32, y String, z String) engine=MergeTree order by x" +$CUR_DIR/data_csv/csv_with_space_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_whitespace SETTINGS format_csv_delimiter=' ', input_format_csv_skip_whitespaces_tabs=false FORMAT CSV" +$CUR_DIR/data_csv/csv_with_tab_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tab SETTINGS format_csv_delimiter='\t', input_format_csv_skip_whitespaces_tabs=false FORMAT CSV" +$CLICKHOUSE_CLIENT -q "select * from test_whitespace" +$CLICKHOUSE_CLIENT -q "select * from test_tab" +$CLICKHOUSE_CLIENT -q "drop table test_whitespace" +$CLICKHOUSE_CLIENT -q "drop table test_tab"; diff --git a/tests/queries/0_stateless/data_csv/csv_with_space_delimiter.csv b/tests/queries/0_stateless/data_csv/csv_with_space_delimiter.csv new file mode 100644 index 00000000000..967f8ae450e --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_space_delimiter.csv @@ -0,0 +1 @@ +1 a b diff --git a/tests/queries/0_stateless/data_csv/csv_with_tab_delimiter.csv b/tests/queries/0_stateless/data_csv/csv_with_tab_delimiter.csv new file mode 100644 index 00000000000..f3b63950ea8 --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_tab_delimiter.csv @@ -0,0 +1 @@ +2 c d From d35573a2063f2c6f3091001c238f8ba7ac96a44c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 8 Jun 2023 14:40:58 +0200 Subject: [PATCH 1377/2223] Removed logs and small name fixes --- src/Common/ProfileEvents.cpp | 4 ++-- src/Core/Settings.h | 2 +- src/Storages/StorageS3.cpp | 3 --- src/TableFunctions/TableFunctionAzureBlobStorage.cpp | 2 -- src/TableFunctions/TableFunctionS3.cpp | 1 - 5 files changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 3cc41c1972d..f66f7bc6465 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -348,8 +348,8 @@ The server successfully detected this situation and will download merged part fr M(S3PutObject, "Number of S3 API PutObject calls.") \ M(S3GetObject, "Number of S3 API GetObject calls.") \ \ - M(AzureDeleteObjects, "Number of S3 API DeleteObject(s) calls.") \ - M(AzureListObjects, "Number of S3 API ListObjects calls.") \ + M(AzureDeleteObjects, "Number of Azure blob storage API DeleteObject(s) calls.") \ + M(AzureListObjects, "Number of Azure blob storage API ListObjects calls.") \ \ M(DiskS3DeleteObjects, "Number of DiskS3 API DeleteObject(s) calls.") \ M(DiskS3CopyObject, "Number of DiskS3 API CopyObject calls.") \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6abc2e1f697..25ad58e22fa 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -83,7 +83,7 @@ class IColumn; M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ - M(UInt64, azure_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ + M(UInt64, azure_max_single_read_retries, 4, "The maximum number of retries during single Azure blob storage read.", 0) \ M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 1c9efc31898..f1a7bcb71a2 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -957,9 +957,6 @@ StorageS3::StorageS3( {"_file", std::make_shared(std::make_shared())}}; auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList(); - - LOG_INFO(&Poco::Logger::get("StorageS3"), "constructor columns = {}", columns.toString()); - virtual_columns = getVirtualsForStorage(columns, default_virtuals); for (const auto & column : virtual_columns) virtual_block.insert({column.type->createColumn(), column.type, column.name}); diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 265092ddefa..38d9362894a 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -2,7 +2,6 @@ #if USE_AZURE_BLOB_STORAGE -//#include #include #include #include @@ -15,7 +14,6 @@ #include #include #include -#include #include #include #include diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index e63f32b1cbc..c8cc0cddd30 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -319,7 +319,6 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context else if (!structure_hint.empty()) columns = structure_hint; - StoragePtr storage = std::make_shared( configuration, context, From 6fcc4158d2ed912e0dcf34ca6e8f00060ec47dc0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Jun 2023 12:46:49 +0000 Subject: [PATCH 1378/2223] Add SHOW INDICES as alias for statement SHOW INDEX/INDEXES/KEYS --- docs/en/sql-reference/statements/show.md | 2 +- src/Parsers/ParserShowIndexesQuery.cpp | 2 +- tests/queries/0_stateless/02724_show_indexes.reference | 5 +++++ tests/queries/0_stateless/02724_show_indexes.sql | 1 + 4 files changed, 8 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 21c0010498a..f96eb55aa45 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -273,7 +273,7 @@ SHOW DICTIONARIES FROM db LIKE '%reg%' LIMIT 2 Displays a list of primary and data skipping indexes of a table. ```sql -SHOW [EXTENDED] {INDEX | INDEXES | KEYS } {FROM | IN} [{FROM | IN} ] [WHERE ] [INTO OUTFILE ] [FORMAT ] +SHOW [EXTENDED] {INDEX | INDEXES | INDICES | KEYS } {FROM | IN}
[{FROM | IN} ] [WHERE ] [INTO OUTFILE ] [FORMAT ] ``` The database and table name can be specified in abbreviated form as `.
`, i.e. `FROM tab FROM db` and `FROM db.tab` are diff --git a/src/Parsers/ParserShowIndexesQuery.cpp b/src/Parsers/ParserShowIndexesQuery.cpp index fe08b463069..7852ed9806d 100644 --- a/src/Parsers/ParserShowIndexesQuery.cpp +++ b/src/Parsers/ParserShowIndexesQuery.cpp @@ -28,7 +28,7 @@ bool ParserShowIndexesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (ParserKeyword("EXTENDED").ignore(pos, expected)) query->extended = true; - if (!(ParserKeyword("INDEX").ignore(pos, expected) || ParserKeyword("INDEXES").ignore(pos, expected) || ParserKeyword("KEYS").ignore(pos, expected))) + if (!(ParserKeyword("INDEX").ignore(pos, expected) || ParserKeyword("INDEXES").ignore(pos, expected) || ParserKeyword("INDICES").ignore(pos, expected) || ParserKeyword("KEYS").ignore(pos, expected))) return false; if (ParserKeyword("FROM").ignore(pos, expected) || ParserKeyword("IN").ignore(pos, expected)) diff --git a/tests/queries/0_stateless/02724_show_indexes.reference b/tests/queries/0_stateless/02724_show_indexes.reference index 20af3954fa5..8365ade3231 100644 --- a/tests/queries/0_stateless/02724_show_indexes.reference +++ b/tests/queries/0_stateless/02724_show_indexes.reference @@ -14,6 +14,11 @@ tbl 0 mm1_idx \N \N \N \N \N \N \N minmax \N \N YES a, c, d tbl 0 mm2_idx \N \N \N \N \N \N \N minmax \N \N YES c, d, e tbl 0 PRIMARY \N \N A \N \N \N \N primary \N \N YES c, a tbl 0 set_idx \N \N \N \N \N \N \N set \N \N YES e +tbl 0 blf_idx \N \N \N \N \N \N \N bloom_filter \N \N YES d, b +tbl 0 mm1_idx \N \N \N \N \N \N \N minmax \N \N YES a, c, d +tbl 0 mm2_idx \N \N \N \N \N \N \N minmax \N \N YES c, d, e +tbl 0 PRIMARY \N \N A \N \N \N \N primary \N \N YES c, a +tbl 0 set_idx \N \N \N \N \N \N \N set \N \N YES e --- EXTENDED tbl 0 blf_idx \N \N \N \N \N \N \N bloom_filter \N \N YES d, b tbl 0 mm1_idx \N \N \N \N \N \N \N minmax \N \N YES a, c, d diff --git a/tests/queries/0_stateless/02724_show_indexes.sql b/tests/queries/0_stateless/02724_show_indexes.sql index ce8ed67c524..04a481fea4e 100644 --- a/tests/queries/0_stateless/02724_show_indexes.sql +++ b/tests/queries/0_stateless/02724_show_indexes.sql @@ -22,6 +22,7 @@ PRIMARY KEY (c, a); SELECT '--- Aliases of SHOW INDEX'; SHOW INDEX FROM tbl; SHOW INDEXES FROM tbl; +SHOW INDICES FROM tbl; SHOW KEYS FROM tbl; SELECT '--- EXTENDED'; From f4202963ad04f52da3aa0ada96c6151cfadd4a69 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 8 Jun 2023 21:06:38 +0800 Subject: [PATCH 1379/2223] test modify --- .../02785_text_with_whitespace_tab_field_delimiter.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh index 19d343c352f..e3f61262674 100755 --- a/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh +++ b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh @@ -10,8 +10,8 @@ $CLICKHOUSE_CLIENT -q "drop table if exists test_whitespace" $CLICKHOUSE_CLIENT -q "drop table if exists test_tab" $CLICKHOUSE_CLIENT -q "create table test_whitespace (x UInt32, y String, z String) engine=MergeTree order by x" $CLICKHOUSE_CLIENT -q "create table test_tab (x UInt32, y String, z String) engine=MergeTree order by x" -$CUR_DIR/data_csv/csv_with_space_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_whitespace SETTINGS format_csv_delimiter=' ', input_format_csv_skip_whitespaces_tabs=false FORMAT CSV" -$CUR_DIR/data_csv/csv_with_tab_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tab SETTINGS format_csv_delimiter='\t', input_format_csv_skip_whitespaces_tabs=false FORMAT CSV" +cat $CURDIR/data_csv/csv_with_space_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_whitespace SETTINGS format_csv_delimiter=' ', input_format_csv_skip_whitespaces_tabs=false FORMAT CSV" +cat $CURDIR/data_csv/csv_with_tab_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tab SETTINGS format_csv_delimiter='\t', input_format_csv_skip_whitespaces_tabs=false FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_whitespace" $CLICKHOUSE_CLIENT -q "select * from test_tab" $CLICKHOUSE_CLIENT -q "drop table test_whitespace" From 5344ff2516d8f135c2a65f3979bf18109d76a10b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Jun 2023 15:13:12 +0200 Subject: [PATCH 1380/2223] Temporarily disable annoy index tests (flaky for analyzer) (#50714) --- tests/queries/0_stateless/02354_annoy_index.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 0168fa04c6f..abee5e8a6e4 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check +-- Tags: disabled, no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check SET allow_experimental_annoy_index = 1; From fa877f456185b5de5b5f2e36e775acdb8dec7f31 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 8 Jun 2023 16:05:14 +0200 Subject: [PATCH 1381/2223] cosmetic changes --- src/Common/DateLUT.h | 2 +- src/Common/LocalDate.h | 13 +++++-------- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 23698331afe..833b2291254 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -39,7 +39,7 @@ public: /// Timezone is passed in query_context, but on CH-Client we have no query context, /// and each time we modify client's global context - const auto global_context = DB::CurrentThread::get().getGlobalContext(); + const DB::ContextPtr global_context = DB::CurrentThread::get().getGlobalContext(); if (global_context) { context_timezone = extractTimezoneFromContext(global_context); diff --git a/src/Common/LocalDate.h b/src/Common/LocalDate.h index 4a383129ae4..2331a40fd12 100644 --- a/src/Common/LocalDate.h +++ b/src/Common/LocalDate.h @@ -24,9 +24,8 @@ private: unsigned char m_month; unsigned char m_day; - void init(time_t time) + void init(time_t time, const DateLUTImpl & date_lut) { - const auto & date_lut = DateLUT::instance(); const auto & values = date_lut.getValues(time); m_year = values.year; @@ -56,9 +55,9 @@ private: } public: - explicit LocalDate(time_t time) + explicit LocalDate(time_t time, const DateLUTImpl & time_zone = DateLUT::instance()) { - init(time); + init(time, time_zone); } LocalDate(DayNum day_num, const DateLUTImpl & time_zone = DateLUT::instance()) /// NOLINT @@ -99,15 +98,13 @@ public: LocalDate(const LocalDate &) noexcept = default; LocalDate & operator= (const LocalDate &) noexcept = default; - DayNum getDayNum() const + DayNum getDayNum(const DateLUTImpl & lut = DateLUT::instance()) const { - const auto & lut = DateLUT::instance(); return DayNum(lut.makeDayNum(m_year, m_month, m_day).toUnderType()); } - ExtendedDayNum getExtenedDayNum() const + ExtendedDayNum getExtenedDayNum(const DateLUTImpl & lut = DateLUT::instance()) const { - const auto & lut = DateLUT::instance(); return ExtendedDayNum (lut.makeDayNum(m_year, m_month, m_day).toUnderType()); } From a918f88c744aa1fe171b500e3b56bfe6a3e13a2c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 8 Jun 2023 16:11:27 +0200 Subject: [PATCH 1382/2223] Fixes --- src/Storages/StorageAzureBlob.cpp | 18 ++++++++++++++---- src/Storages/StorageAzureBlob.h | 12 ++---------- .../configs/disable_profilers.xml | 9 +++++++++ .../test_storage_azure_blob_storage/test.py | 6 ++++-- 4 files changed, 29 insertions(+), 16 deletions(-) create mode 100644 tests/integration/test_storage_azure_blob_storage/configs/disable_profilers.xml diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 4901f6701fb..ce6c864f7ea 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -122,7 +122,7 @@ void StorageAzureBlob::processNamedCollectionResult(StorageAzureBlob::Configurat } -StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file) +StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine_args, ContextPtr local_context) { StorageAzureBlob::Configuration configuration; @@ -137,7 +137,7 @@ StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine configuration.blobs_paths = {configuration.blob_path}; - if (configuration.format == "auto" && get_format_from_file) + if (configuration.format == "auto") configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); return configuration; @@ -230,7 +230,7 @@ StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine configuration.blobs_paths = {configuration.blob_path}; - if (configuration.format == "auto" && get_format_from_file) + if (configuration.format == "auto") configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); return configuration; @@ -898,7 +898,13 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() } else { - if (!blobs_with_metadata || index >= blobs_with_metadata->size()) + bool need_new_batch = false; + { + std::lock_guard lock(next_mutex); + need_new_batch = !blobs_with_metadata || index >= blobs_with_metadata->size(); + } + + if (need_new_batch) { RelativePathsWithMetadata new_batch; while (new_batch.empty()) @@ -939,6 +945,7 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); const auto & idxs = typeid_cast(*block.getByName("_idx").column); + std::lock_guard lock(next_mutex); blob_path_with_globs.reset(); blob_path_with_globs.emplace(); for (UInt64 idx : idxs.getData()) @@ -954,6 +961,7 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() if (outer_blobs) outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); + std::lock_guard lock(next_mutex); blobs_with_metadata = std::move(new_batch); for (const auto & [_, info] : *blobs_with_metadata) total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); @@ -961,6 +969,8 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() } size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + + std::lock_guard lock(next_mutex); return (*blobs_with_metadata)[current_index]; } } diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 6e4dfaf19eb..c0380d7c065 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -15,15 +15,6 @@ namespace DB { -struct AzureSimpleAccountConfiguration -{ - std::string storage_account_url; -}; - -using AzureConnectionString = std::string; - -using AzureCredentials = std::variant; - class StorageAzureBlob : public IStorage { public: @@ -73,7 +64,7 @@ public: std::optional format_settings_, ASTPtr partition_by_); - static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file = true); + static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration); static AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); @@ -190,6 +181,7 @@ public: void createFilterAST(const String & any_key); bool is_finished = false; bool is_initialized = false; + std::mutex next_mutex; }; StorageAzureBlobSource( diff --git a/tests/integration/test_storage_azure_blob_storage/configs/disable_profilers.xml b/tests/integration/test_storage_azure_blob_storage/configs/disable_profilers.xml new file mode 100644 index 00000000000..a39badbf8ec --- /dev/null +++ b/tests/integration/test_storage_azure_blob_storage/configs/disable_profilers.xml @@ -0,0 +1,9 @@ + + + + + 0 + 0 + + + diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index de7c662127c..f0934d3aa80 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -25,6 +25,7 @@ def cluster(): cluster.add_instance( "node", main_configs=["configs/named_collections.xml"], + user_configs=["configs/disable_profilers.xml"], with_azurite=True, ) cluster.start() @@ -34,13 +35,14 @@ def cluster(): cluster.shutdown() -def azure_query(node, query, try_num=3, settings={}): +def azure_query(node, query, try_num=10, settings={}): for i in range(try_num): try: return node.query(query, settings=settings) except Exception as ex: retriable_errors = [ - "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response" + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", ] retry = False for error in retriable_errors: From 964e0d042ca4fb10629da2437eae5efa9559955f Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 8 Jun 2023 16:21:45 +0200 Subject: [PATCH 1383/2223] Fix --- src/Interpreters/Cache/FileCache.cpp | 9 +++++---- src/Interpreters/Cache/Metadata.cpp | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 8d6146b4eea..cd0f96d8e0e 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -988,17 +988,18 @@ void FileCache::cleanup() void FileCache::cleanupThreadFunc() { -#ifndef NDEBUG - assertCacheCorrectness(); -#endif - try { +#ifdef ABORT_ON_LOGICAL_ERROR + assertCacheCorrectness(); +#endif + cleanup(); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); } cleanup_task->scheduleAfter(delayed_cleanup_interval_ms); diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 5b6561a665e..cd3b4e4f09f 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -278,7 +278,7 @@ void CacheMetadata::doCleanup() try { if (fs::exists(key_prefix_directory) && fs::is_empty(key_prefix_directory)) - fs::remove_all(key_prefix_directory); + fs::remove(key_prefix_directory); } catch (const fs::filesystem_error & e) { From 9f61c786ed67ae571384f90ef934bb839ab20e0a Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 8 Jun 2023 17:25:53 +0300 Subject: [PATCH 1384/2223] Amend the tests --- ...3_parseDateTimeBestEffort_syslog.reference | 21 +--- .../02783_parseDateTimeBestEffort_syslog.sql | 97 ++++++------------- 2 files changed, 34 insertions(+), 84 deletions(-) diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference index 7409b413260..63e7e367941 100644 --- a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference +++ b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference @@ -1,20 +1,5 @@ parseDateTimeBestEffort - dt_ref res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc + dt_ref res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc res_us res_us_sam res_us_auc res_us_null res_us_null_sam res_us_null_auc res_us_zero res_us_zero_sam res_us_zero_auc res64 res64_sam res64_auc res64_null res64_null_sam res64_null_auc res64_zero res64_zero_sam res64_zero_auc res64_us res64_us_sam res64_us_auc res64_us_null res64_us_null_sam res64_us_null_auc res64_us_zero res64_us_zero_sam res64_us_zero_auc - Jun 7 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 - Jun 7 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 -parseDateTimeBestEffortUS - dt_ref res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc - - Jun 7 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 2023-06-07 04:55:00 2022-06-07 04:55:00 2023-06-07 04:55:00 - Jun 7 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 2022-06-07 04:56:00 2022-06-07 04:56:00 2023-06-07 04:56:00 -parseDateTime64BestEffort - dt_ref res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc - - Jun 7 04:55:00 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 - Jun 7 04:56:00 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 -parseDateTime64BestEffortUS - dt_ref res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc - - Jun 7 04:55:00 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 2023-06-07 04:55:00.000 2022-06-07 04:55:00.000 2023-06-07 04:55:00.000 - Jun 7 04:56:00 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 2022-06-07 04:56:00.000 2022-06-07 04:56:00.000 2023-06-07 04:56:00.000 + Jun 6 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 + Jun 8 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql index 91ae230205b..59211d3e6a0 100644 --- a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql +++ b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql @@ -1,8 +1,9 @@ SELECT 'parseDateTimeBestEffort'; WITH + 86400 AS secs_in_day, now() AS ts_now, - '2023-06-07 04:55:30' AS ref_point, + '2023-06-07' AS ref_point, dateDiff('second', toDateTime(ref_point), ts_now) AS impedimenta, formatDateTime(ts_around, '%b %e %T') AS dt_curr SELECT @@ -15,69 +16,33 @@ SELECT parseDateTimeBestEffortOrNull(dt_curr, 'Pacific/Auckland') - impedimenta AS res_null_auc, parseDateTimeBestEffortOrZero(dt_curr) - impedimenta AS res_zero, parseDateTimeBestEffortOrZero(dt_curr, 'US/Samoa') - impedimenta AS res_zero_sam, - parseDateTimeBestEffortOrZero(dt_curr, 'Pacific/Auckland') - impedimenta AS res_zero_auc -FROM (SELECT arrayJoin([ts_now - 30, ts_now + 30]) AS ts_around) -FORMAT PrettySpaceNoEscapes; - -SELECT 'parseDateTimeBestEffortUS'; - -WITH - now() AS ts_now, - '2023-06-07 04:55:30' AS ref_point, - dateDiff('second', toDateTime(ref_point), ts_now) AS impedimenta, - formatDateTime(ts_around, '%b %e %T') AS dt_curr -SELECT - formatDateTime(ts_around - impedimenta, '%b %e %H:%i:%s') AS dt_ref, - parseDateTimeBestEffortUS(dt_curr) - impedimenta AS res, - parseDateTimeBestEffortUS(dt_curr, 'US/Samoa') - impedimenta AS res_sam, - parseDateTimeBestEffortUS(dt_curr, 'Pacific/Auckland') - impedimenta AS res_auc, - parseDateTimeBestEffortUSOrNull(dt_curr) - impedimenta AS res_null, - parseDateTimeBestEffortUSOrNull(dt_curr, 'US/Samoa') - impedimenta AS res_null_sam, - parseDateTimeBestEffortUSOrNull(dt_curr, 'Pacific/Auckland') - impedimenta AS res_null_auc, - parseDateTimeBestEffortUSOrZero(dt_curr) - impedimenta AS res_zero, - parseDateTimeBestEffortUSOrZero(dt_curr, 'US/Samoa') - impedimenta AS res_zero_sam, - parseDateTimeBestEffortUSOrZero(dt_curr, 'Pacific/Auckland') - impedimenta AS res_zero_auc -FROM (SELECT arrayJoin([ts_now - 30, ts_now + 30]) AS ts_around) -FORMAT PrettySpaceNoEscapes; - -SELECT 'parseDateTime64BestEffort'; - -WITH - now() AS ts_now, - '2023-06-07 04:55:30' AS ref_point, - dateDiff('second', toDateTime(ref_point), ts_now) AS impedimenta, - formatDateTime(ts_around, '%b %e %T') AS dt_curr -SELECT - formatDateTime(ts_around - impedimenta, '%b %e %H:%i:%s') AS dt_ref, - parseDateTime64BestEffort(dt_curr) - impedimenta AS res, - parseDateTime64BestEffort(dt_curr, 3, 'US/Samoa') - impedimenta AS res_sam, - parseDateTime64BestEffort(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_auc, - parseDateTime64BestEffortOrNull(dt_curr) - impedimenta AS res_null, - parseDateTime64BestEffortOrNull(dt_curr, 3, 'US/Samoa') - impedimenta AS res_null_sam, - parseDateTime64BestEffortOrNull(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_null_auc, - parseDateTime64BestEffortOrZero(dt_curr) - impedimenta AS res_zero, - parseDateTime64BestEffortOrZero(dt_curr, 3, 'US/Samoa') - impedimenta AS res_zero_sam, - parseDateTime64BestEffortOrZero(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_zero_auc -FROM (SELECT arrayJoin([ts_now - 30, ts_now + 30]) AS ts_around) -FORMAT PrettySpaceNoEscapes; - -SELECT 'parseDateTime64BestEffortUS'; - -WITH - now() AS ts_now, - '2023-06-07 04:55:30' AS ref_point, - dateDiff('second', toDateTime(ref_point), ts_now) AS impedimenta, - formatDateTime(ts_around, '%b %e %T') AS dt_curr -SELECT - formatDateTime(ts_around - impedimenta, '%b %e %H:%i:%s') AS dt_ref, - parseDateTime64BestEffortUS(dt_curr) - impedimenta AS res, - parseDateTime64BestEffortUS(dt_curr, 3, 'US/Samoa') - impedimenta AS res_sam, - parseDateTime64BestEffortUS(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_auc, - parseDateTime64BestEffortUSOrNull(dt_curr) - impedimenta AS res_null, - parseDateTime64BestEffortUSOrNull(dt_curr, 3, 'US/Samoa') - impedimenta AS res_null_sam, - parseDateTime64BestEffortUSOrNull(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_null_auc, - parseDateTime64BestEffortUSOrZero(dt_curr) - impedimenta AS res_zero, - parseDateTime64BestEffortUSOrZero(dt_curr, 3, 'US/Samoa') - impedimenta AS res_zero_sam, - parseDateTime64BestEffortUSOrZero(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res_zero_auc -FROM (SELECT arrayJoin([ts_now - 30, ts_now + 30]) AS ts_around) + parseDateTimeBestEffortOrZero(dt_curr, 'Pacific/Auckland') - impedimenta AS res_zero_auc, + parseDateTimeBestEffortUS(dt_curr) - impedimenta AS res_us, + parseDateTimeBestEffortUS(dt_curr, 'US/Samoa') - impedimenta AS res_us_sam, + parseDateTimeBestEffortUS(dt_curr, 'Pacific/Auckland') - impedimenta AS res_us_auc, + parseDateTimeBestEffortUSOrNull(dt_curr) - impedimenta AS res_us_null, + parseDateTimeBestEffortUSOrNull(dt_curr, 'US/Samoa') - impedimenta AS res_us_null_sam, + parseDateTimeBestEffortUSOrNull(dt_curr, 'Pacific/Auckland') - impedimenta AS res_us_null_auc, + parseDateTimeBestEffortUSOrZero(dt_curr) - impedimenta AS res_us_zero, + parseDateTimeBestEffortUSOrZero(dt_curr, 'US/Samoa') - impedimenta AS res_us_zero_sam, + parseDateTimeBestEffortUSOrZero(dt_curr, 'Pacific/Auckland') - impedimenta AS res_us_zero_auc, + parseDateTime64BestEffort(dt_curr) - impedimenta AS res64, + parseDateTime64BestEffort(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_sam, + parseDateTime64BestEffort(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_auc, + parseDateTime64BestEffortOrNull(dt_curr) - impedimenta AS res64_null, + parseDateTime64BestEffortOrNull(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_null_sam, + parseDateTime64BestEffortOrNull(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_null_auc, + parseDateTime64BestEffortOrZero(dt_curr) - impedimenta AS res64_zero, + parseDateTime64BestEffortOrZero(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_zero_sam, + parseDateTime64BestEffortOrZero(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_zero_auc, + parseDateTime64BestEffortUS(dt_curr) - impedimenta AS res64_us, + parseDateTime64BestEffortUS(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_us_sam, + parseDateTime64BestEffortUS(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_us_auc, + parseDateTime64BestEffortUSOrNull(dt_curr) - impedimenta AS res64_us_null, + parseDateTime64BestEffortUSOrNull(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_us_null_sam, + parseDateTime64BestEffortUSOrNull(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_us_null_auc, + parseDateTime64BestEffortUSOrZero(dt_curr) - impedimenta AS res64_us_zero, + parseDateTime64BestEffortUSOrZero(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_us_zero_sam, + parseDateTime64BestEffortUSOrZero(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_us_zero_auc +FROM (SELECT arrayJoin([ts_now - secs_in_day, ts_now + secs_in_day]) AS ts_around) FORMAT PrettySpaceNoEscapes; From caabbfd5b16249652fc247e37fe1b2318c9c9994 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 8 Jun 2023 16:25:55 +0200 Subject: [PATCH 1385/2223] Fix one more race --- src/Storages/StorageAzureBlob.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index c0380d7c065..1f91e47ddbe 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -179,8 +179,8 @@ public: std::unique_ptr matcher; void createFilterAST(const String & any_key); - bool is_finished = false; - bool is_initialized = false; + std::atomic is_finished = false; + std::atomic is_initialized = false; std::mutex next_mutex; }; From 9b04e8513700996d3c9e558c910913ab96016af5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 8 Jun 2023 16:43:27 +0200 Subject: [PATCH 1386/2223] Fix --- src/Interpreters/tests/gtest_lru_file_cache.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index fe9e3a18024..8c8e715ce92 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -467,6 +467,7 @@ TEST_F(FileCacheTest, get) cv.notify_one(); file_segment2.wait(file_segment2.range().left); + file_segment2.complete(); ASSERT_TRUE(file_segment2.state() == State::DOWNLOADED); }); From 116df09b5df6dd7362b36c9eea6f08b16a3e1b2b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 8 Jun 2023 17:06:15 +0200 Subject: [PATCH 1387/2223] Fix build --- src/Storages/StorageAzureBlob.cpp | 2 +- src/Storages/StorageAzureBlob.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index ce6c864f7ea..3ee176a68b7 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -669,7 +669,7 @@ Pipe StorageAzureBlob::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto sample_block = metadata_snapshot->getSampleBlock(); auto chosen_compression_method = chooseCompressionMethod(configuration.blobs_paths.back(), configuration.compression_method); diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 1f91e47ddbe..e2001fa24ae 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -85,7 +85,7 @@ public: size_t, size_t) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr context, bool /*async_insert*/) override; void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override; From b8178088d020d956cdf27d390f3d3b7a72a813e8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Jun 2023 08:10:40 +0000 Subject: [PATCH 1388/2223] Misc Annoy fixes --- .../mergetree-family/annindexes.md | 169 +++++--- .../mergetree-family/mergetree.md | 2 +- src/Parsers/ParserCreateIndexQuery.cpp | 11 +- src/Parsers/ParserCreateQuery.cpp | 12 +- ...pproximateNearestNeighborIndexesCommon.cpp | 2 +- .../ApproximateNearestNeighborIndexesCommon.h | 4 +- .../MergeTree/MergeTreeIndexAnnoy.cpp | 43 +- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 16 +- .../0_stateless/02354_annoy_index.reference | 260 ++++++------ .../queries/0_stateless/02354_annoy_index.sql | 371 +++++++++++------- .../aspell-ignore/en/aspell-dict.txt | 3 + 11 files changed, 548 insertions(+), 345 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index f600f9a015c..2b0b77a0735 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -1,104 +1,142 @@ # Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex} -Nearest neighborhood search refers to the problem of finding the point(s) with the smallest distance to a given point in an n-dimensional -space. Since exact search is in practice usually typically too slow, the task is often solved with approximate algorithms. A popular use -case of of neighbor search is finding similar pictures (texts) for a given picture (text). Pictures (texts) can be decomposed into -[embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning), and instead of -comparing pictures (texts) pixel-by-pixel (character-by-character), only the embeddings are compared. +Nearest neighborhood search is the problem of finding the M closest points for a given point in an N-dimensional vector space. The most +straightforward approach to solve this problem is a brute force search where the distance between all points in the vector space and the +reference point is computed. This method guarantees perfect accuracy but it is usually too slow for practical applications. Thus, nearest +neighborhood search problems are often solved with [approximative algorithms](https://github.com/erikbern/ann-benchmarks). Approximative +nearest neighborhood search techniques, in conjunction with [embedding +methods](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning) allow to search huge +amounts of media (pictures, songs, articles, etc.) in milliseconds. -In terms of SQL, the problem can be expressed as follows: +Blogs: +- [Vector Search with ClickHouse - Part 1](https://clickhouse.com/blog/vector-search-clickhouse-p1) +- [Vector Search with ClickHouse - Part 2](https://clickhouse.com/blog/vector-search-clickhouse-p2) + + +In terms of SQL, the nearest neighborhood problem can be expressed as follows: ``` sql SELECT * FROM table -WHERE L2Distance(column, Point) < MaxDistance +ORDER BY Distance(vectors, Point) LIMIT N ``` +`vectors` contains N-dimensional values of type [Array](../../../sql-reference/data-types/array.md) or +[Tuple](../../../sql-reference/data-types/tuple.md), for example embeddings. Function `Distance` computes the distance between two vectors. +Often, the the Euclidean (L2) distance is chosen as distance function but [other +distance functions](/docs/en/sql-reference/functions/distance-functions.md) are also possible. `Point` is the reference point, e.g. `(0.17, +0.33, ...)`, and `N` limits the number of search results. + +An alternative formulation of the nearest neighborhood search problem looks as follows: + ``` sql SELECT * FROM table -ORDER BY L2Distance(column, Point) +WHERE Distance(vectors, Point) < MaxDistance LIMIT N ``` -The queries are expensive because the L2 (Euclidean) distance between `Point` and all points in `column` and must be computed. To speed this process up, Approximate Nearest Neighbor Search Indexes (ANN indexes) store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer quickly. +While the first query returns the top-`N` closest points to the reference point, the second query returns all points closer to the reference +point than a maximally allowed radius `MaxDistance`. Parameter `N` limits the number of returned values which is useful for situations where +`MaxDistance` is difficult to determine in advance. -# Creating ANN Indexes +With brute force search, both queries are expensive (linear in the number of points) because the distance between all points in `vectors` and +`Point` must be computed. To speed this process up, Approximate Nearest Neighbor Search Indexes (ANN indexes) store a compact representation +of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer much quicker (in sub-linear time). -As long as ANN indexes are experimental, you first need to `SET allow_experimental_annoy_index = 1`. +# Creating and Using ANN Indexes -Syntax to create an ANN index over an `Array` column: +Syntax to create an ANN index over an [Array](../../../sql-reference/data-types/array.md) column: ```sql CREATE TABLE table ( `id` Int64, - `embedding` Array(Float32), - INDEX embedding TYPE () GRANULARITY + `vectors` Array(Float32), + INDEX vectors TYPE () [GRANULARITY ] ) ENGINE = MergeTree ORDER BY id; ``` -Syntax to create an ANN index over a `Tuple` column: +Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column: ```sql CREATE TABLE table ( `id` Int64, - `embedding` Tuple(Float32[, Float32[, ...]]), - INDEX embedding TYPE () GRANULARITY + `vectors` Tuple(Float32[, Float32[, ...]]), + INDEX vectors TYPE () [GRANULARITY ] ) ENGINE = MergeTree ORDER BY id; ``` -ANN indexes are built during column insertion and merge and `INSERT` and `OPTIMIZE` statements will be slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively there are much more read requests than write requests. - -Similar to regular skip indexes, ANN indexes are constructed over granules and each indexed block consists of `GRANULARITY = `-many -granules. For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`, -then each indexed block will consist of 16384 rows. However, unlike skip indexes, ANN indexes are not only able to skip the entire indexed -block, they are able to skip individual granules in indexed blocks. As a result, the `GRANULARITY` parameter has a different meaning in ANN -indexes than in normal skip indexes. Basically, the bigger `GRANULARITY` is chosen, the more data is provided to a single ANN index, and the -higher the chance that with the right hyper parameters, the index will remember the data structure better. - -# Using ANN Indexes +ANN indexes are built during column insertion and merge. As a result, `INSERT` and `OPTIMIZE` statements will be slower than for ordinary +tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively when are far more read requests than write +requests. ANN indexes support two types of queries: -- WHERE queries: - - ``` sql - SELECT * - FROM table - WHERE DistanceFunction(column, Point) < MaxDistance - LIMIT N - ``` - - ORDER BY queries: ``` sql SELECT * FROM table [WHERE ...] - ORDER BY DistanceFunction(column, Point) + ORDER BY Distance(vectors, Point) LIMIT N ``` -`DistanceFunction` is a [distance function](/docs/en/sql-reference/functions/distance-functions.md), `Point` is a reference vector (e.g. `(0.17, 0.33, ...)`) and `MaxDistance` is a floating point value which restricts the size of the neighbourhood. +- WHERE queries: + + ``` sql + SELECT * + FROM table + WHERE Distance(vectors, Point) < MaxDistance + LIMIT N + ``` :::tip -To avoid writing out large vectors, you can use [query parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. +To avoid writing out large vectors, you can use [query +parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. ```bash -clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(embedding, {vec: Array(Float32)}) < 1.0" +clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(vectors, {vec: Array(Float32)}) < 1.0" ``` ::: -ANN indexes cannot speed up queries that contain both a `WHERE DistanceFunction(column, Point) < MaxDistance` and an `ORDER BY DistanceFunction(column, Point)` clause. Also, the approximate algorithms used to determine the nearest neighbors require a limit, hence queries that use an ANN index must have a `LIMIT` clause. +**Restrictions**: Queries that contain both a `WHERE Distance(vectors, Point) < MaxDistance` and an `ORDER BY Distance(vectors, Point)` +clause cannot use ANN indexes. Also, the approximate algorithms used to determine the nearest neighbors require a limit, hence queries +without `LIMIT` clause cannot utilize ANN indexes. Also ANN indexes are only used if the query has a `LIMIT` value smaller than setting +`max_limit_for_ann_queries` (default: 1 million rows). This is a safeguard to prevent large memory allocations by external libraries for +approximate neighbor search. + +**Differences to Skip Indexes** Similar to regular [skip indexes](https://clickhouse.com/docs/en/optimize/skipping-indexes), ANN indexes are +constructed over granules and each indexed block consists of `GRANULARITY = `-many granules (`` = 1 by default for normal skip +indexes). For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`, +then each indexed block will contain 16384 rows. However, data structures and algorithms for approximate neighborhood search (usually +provided by external libraries) are inherently row-oriented. They store a compact representation of a set of rows and also return rows for +ANN queries. This causes some rather unintuitive differences in the way ANN indexes behave compared to normal skip indexes. + +When a user defines a ANN index on a column, ClickHouse internally creates a ANN "sub-index" for each index block. The sub-index is "local" +in the sense that it only knows about the rows of its containing index block. In the previous example and assuming that a column has 65536 +rows, we obtain four index blocks (spanning eight granules) and a ANN sub-index for each index block. A sub-index is theoretically able to +return the rows with the N closest points within its index block directly. However, since ClickHouse loads data from disk to memory at the +granularity of granules, sub-indexes extrapolate matching rows to granule granularity. This is different from regular skip indexes which +skip data at the granularity of index blocks. + +The `GRANULARITY` parameter determines how many ANN sub-indexes are created. Bigger `GRANULARITY` values mean fewer but larger ANN +sub-indexes, up to the point where a column (or a column part) has only a single sub-index. In that case, the sub-index has a "global" view of +all column rows and can directly return all granules of the column (part) with relevant rows (there are at at most `LIMIT `-many +such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a brute-force distance +calculation over all rows of the granules. With a small `GRANULARITY` value, each of the sub-indexes returns up to `LIMIT N`-many granules. +As a result, more granules need to be loaded and post-filtered. Note that the search accuracy is with both cases equally good, only the +processing performance differs. It is generally recommended to use a large `GRANULARITY` for ANN indexes and fall back to a smaller +`GRANULARITY` values only in case of problems like excessive memory consumption of the ANN structures. If no `GRANULARITY` was specified for +ANN indexes, the default value is 100 million. -An ANN index is only used if the query has a `LIMIT` value smaller than setting `max_limit_for_ann_queries` (default: 1 million rows). This is a safety measure which helps to avoid large memory consumption by external libraries for approximate neighbor search. # Available ANN Indexes @@ -106,51 +144,68 @@ An ANN index is only used if the query has a `LIMIT` value smaller than setting ## Annoy {#annoy} -(currently disabled on ARM due to memory safety problems with the algorithm) +Annoy indexes are currently experimental, to use them you first need to `SET allow_experimental_annoy_index = 1`. They are also currently +disabled on ARM due to memory safety problems with the algorithm. -This type of ANN index implements [the Annoy algorithm](https://github.com/spotify/annoy) which uses a recursive division of the space in random linear surfaces (lines in 2D, planes in 3D etc.). +This type of ANN index implements [the Annoy algorithm](https://github.com/spotify/annoy) which is based on a recursive division of the +space in random linear surfaces (lines in 2D, planes in 3D etc.). -Syntax to create a Annoy index over a `Array` column: +
+ +
+ +Syntax to create an Annoy index over an [Array](../../../sql-reference/data-types/array.md) column: ```sql CREATE TABLE table ( id Int64, - embedding Array(Float32), - INDEX embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N + vectors Array(Float32), + INDEX vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N] ) ENGINE = MergeTree ORDER BY id; ``` -Syntax to create a Annoy index over a `Tuple` column: +Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column: ```sql CREATE TABLE table ( id Int64, - embedding Tuple(Float32[, Float32[, ...]]), - INDEX embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N + vectors Tuple(Float32[, Float32[, ...]]), + INDEX vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N] ) ENGINE = MergeTree ORDER BY id; ``` -Parameter `DistanceName` is name of a distance function (default `L2Distance`). Annoy currently supports `L2Distance` and `cosineDistance` as distance functions. Parameter `NumTrees` (default: 100) is the number of trees which the algorithm will create. Higher values of `NumTree` mean slower `CREATE` and `SELECT` statements (approximately linearly), but increase the accuracy of search results. +Annoy currently supports `L2Distance` and `cosineDistance` as distance function `Distance`. If no distance function was specified during +index creation, `L2Distance` is used as default. Parameter `NumTrees` is the number of trees which the algorithm creates (default if not +specified: 100). Higher values of `NumTree` mean more accurate search results but slower index creation / query times (approximately +linearly) as well as larger index sizes. :::note -Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(embedding) = 256`. +Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use +[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 +CHECK length(vectors) = 256`. ::: -Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. It can be used to -balance runtime and accuracy at runtime. - -Example: +Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. Larger +values mean more accurate results at the cost of longer query runtime: ``` sql SELECT * FROM table_name [WHERE ...] -ORDER BY L2Distance(column, Point) +ORDER BY L2Distance(vectors, Point) LIMIT N SETTINGS annoy_index_search_k_nodes=100 ``` diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 07f706af91d..61276110138 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -491,7 +491,7 @@ Syntax: `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, ran #### Special-purpose -- An experimental index to support approximate nearest neighbor (ANN) search. See [here](annindexes.md) for details. +- Experimental indexes to support approximate nearest neighbor (ANN) search. See [here](annindexes.md) for details. - An experimental inverted index to support full-text search. See [here](invertedindexes.md) for details. ### Functions Support {#functions-support} diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 7323c5da141..57afd3fb99e 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -46,7 +46,16 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected index->part_of_create_index_query = true; index->set(index->expr, expr); index->set(index->type, type); - index->granularity = granularity ? granularity->as().value.safeGet() : 1; + + if (granularity) + index->granularity = granularity->as().value.safeGet(); + else + { + if (index->type->name == "annoy") + index->granularity = 100'000'000; + else + index->granularity = 1; + } node = index; return true; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index f975e8ba3c8..c6273f369b1 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -141,7 +141,17 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe index->name = name->as().name(); index->set(index->expr, expr); index->set(index->type, type); - index->granularity = granularity ? granularity->as().value.safeGet() : 1; + + if (granularity) + index->granularity = granularity->as().value.safeGet(); + else + { + if (index->type->name == "annoy") + index->granularity = 100'000'000; + else + index->granularity = 1; + } + node = index; return true; diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index bf277c55863..69e54dd5f0c 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -88,7 +88,7 @@ std::vector ApproximateNearestNeighborCondition::getReferenceVector() con throw Exception(ErrorCodes::LOGICAL_ERROR, "Reference vector was requested for useless or uninitialized index."); } -size_t ApproximateNearestNeighborCondition::getNumOfDimensions() const +size_t ApproximateNearestNeighborCondition::getDimensions() const { if (index_is_useful && query_information.has_value()) return query_information->reference_vector.size(); diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h index 4fb95c3f492..310890eba1e 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h @@ -90,8 +90,8 @@ public: /// Distance should be calculated regarding to referenceVector std::vector getReferenceVector() const; - /// Reference vector's dimension size - size_t getNumOfDimensions() const; + /// Reference vector's dimension count + size_t getDimensions() const; String getColumnName() const; diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 1a28f28f746..f77cfe4fed0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -27,13 +27,13 @@ namespace ErrorCodes template -AnnoyIndexWithSerialization::AnnoyIndexWithSerialization(uint64_t dim) - : Base::AnnoyIndex(dim) +AnnoyIndexWithSerialization::AnnoyIndexWithSerialization(size_t dimensions) + : Base::AnnoyIndex(dimensions) { } template -void AnnoyIndexWithSerialization::serialize(WriteBuffer& ostr) const +void AnnoyIndexWithSerialization::serialize(WriteBuffer & ostr) const { chassert(Base::_built); writeIntBinary(Base::_s, ostr); @@ -43,11 +43,11 @@ void AnnoyIndexWithSerialization::serialize(WriteBuffer& ostr) const writeIntBinary(Base::_K, ostr); writeIntBinary(Base::_seed, ostr); writeVectorBinary(Base::_roots, ostr); - ostr.write(reinterpret_cast(Base::_nodes), Base::_s * Base::_n_nodes); + ostr.write(reinterpret_cast(Base::_nodes), Base::_s * Base::_n_nodes); } template -void AnnoyIndexWithSerialization::deserialize(ReadBuffer& istr) +void AnnoyIndexWithSerialization::deserialize(ReadBuffer & istr) { chassert(!Base::_built); readIntBinary(Base::_s, istr); @@ -69,7 +69,7 @@ void AnnoyIndexWithSerialization::deserialize(ReadBuffer& istr) } template -uint64_t AnnoyIndexWithSerialization::getNumOfDimensions() const +size_t AnnoyIndexWithSerialization::getDimensions() const { return Base::get_f(); } @@ -97,14 +97,14 @@ void MergeTreeIndexGranuleAnnoy::serializeBinary(WriteBuffer & ostr) c { /// Number of dimensions is required in the index constructor, /// so it must be written and read separately from the other part - writeIntBinary(index->getNumOfDimensions(), ostr); // write dimension + writeIntBinary(static_cast(index->getDimensions()), ostr); // write dimension index->serialize(ostr); } template void MergeTreeIndexGranuleAnnoy::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/) { - uint64_t dimension; + UInt64 dimension; readIntBinary(dimension, istr); index = std::make_shared>(dimension); index->deserialize(istr); @@ -114,7 +114,7 @@ template MergeTreeIndexAggregatorAnnoy::MergeTreeIndexAggregatorAnnoy( const String & index_name_, const Block & index_sample_block_, - uint64_t trees_) + UInt64 trees_) : index_name(index_name_) , index_sample_block(index_sample_block_) , trees(trees_) @@ -251,10 +251,10 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI const AnnoyIndexWithSerializationPtr annoy = granule->index; - if (ann_condition.getNumOfDimensions() != annoy->getNumOfDimensions()) + if (ann_condition.getDimensions() != annoy->getDimensions()) throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) " "does not match the dimension in the index ({})", - ann_condition.getNumOfDimensions(), annoy->getNumOfDimensions()); + ann_condition.getDimensions(), annoy->getDimensions()); std::vector neighbors; /// indexes of dots which were closest to the reference vector std::vector distances; @@ -281,7 +281,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI return granule_numbers; } -MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String & distance_function_) +MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, UInt64 trees_, const String & distance_function_) : IMergeTreeIndex(index_) , trees(trees_) , distance_function(distance_function_) @@ -320,9 +320,9 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) if (!index.arguments.empty()) distance_function = index.arguments[0].get(); - uint64_t trees = default_trees; + UInt64 trees = default_trees; if (index.arguments.size() > 1) - trees = index.arguments[1].get(); + trees = index.arguments[1].get(); return std::make_shared(index, trees, distance_function); } @@ -338,7 +338,7 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance function argument of Annoy index must be of type String"); if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::UInt64) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Number of trees argument of Annoy index must be UInt64"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Number of trees argument of Annoy index must be of type UInt64"); /// Check that the index is created on a single column @@ -351,17 +351,16 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { String distance_name = index.arguments[0].get(); if (distance_name != "L2Distance" && distance_name != "cosineDistance") - throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index supports only distance functions 'L2Distance' and 'cosineDistance'. Given distance function: {}", distance_name); + throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index only supports distance functions 'L2Distance' and 'cosineDistance'"); } /// Check data type of indexed column: - auto throw_unsupported_underlying_column_exception = [](DataTypePtr data_type) + auto throw_unsupported_underlying_column_exception = []() { throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32). Given type: {}", - data_type->getName()); + "Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32)"); }; DataTypePtr data_type = index.sample_block.getDataTypes()[0]; @@ -370,7 +369,7 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) - throw_unsupported_underlying_column_exception(data_type); + throw_unsupported_underlying_column_exception(); } else if (const auto * data_type_tuple = typeid_cast(data_type.get())) { @@ -379,11 +378,11 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { TypeIndex nested_type_index = inner_type->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) - throw_unsupported_underlying_column_exception(data_type); + throw_unsupported_underlying_column_exception(); } } else - throw_unsupported_underlying_column_exception(data_type); + throw_unsupported_underlying_column_exception(); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 5204ff07b27..cfc3b7519b8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -16,10 +16,10 @@ class AnnoyIndexWithSerialization : public Annoy::AnnoyIndex; public: - explicit AnnoyIndexWithSerialization(uint64_t dim); - void serialize(WriteBuffer& ostr) const; - void deserialize(ReadBuffer& istr); - uint64_t getNumOfDimensions() const; + explicit AnnoyIndexWithSerialization(size_t dimensions); + void serialize(WriteBuffer & ostr) const; + void deserialize(ReadBuffer & istr); + size_t getDimensions() const; }; template @@ -46,7 +46,7 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule template struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator { - MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, uint64_t trees); + MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, UInt64 trees); ~MergeTreeIndexAggregatorAnnoy() override = default; bool empty() const override { return !index || index->get_n_items() == 0; } @@ -55,7 +55,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator const String index_name; const Block index_sample_block; - const uint64_t trees; + const UInt64 trees; AnnoyIndexWithSerializationPtr index; }; @@ -89,7 +89,7 @@ class MergeTreeIndexAnnoy : public IMergeTreeIndex { public: - MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String & distance_function_); + MergeTreeIndexAnnoy(const IndexDescription & index_, UInt64 trees_, const String & distance_function_); ~MergeTreeIndexAnnoy() override = default; @@ -100,7 +100,7 @@ public: bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; } private: - const uint64_t trees; + const UInt64 trees; const String distance_function; }; diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 45515bc7733..5e01a6e566e 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -1,118 +1,144 @@ ---- Test with Array --- -WHERE type, L2Distance -1 [0,0,10] -2 [0,0,10.5] -3 [0,0,9.5] -4 [0,0,9.7] -5 [0,0,10.2] -ORDER BY type, L2Distance -1 [0,0,10] -5 [0,0,10.2] -4 [0,0,9.7] -WHERE type, L2Distance, check that index is used -Expression ((Projection + Before ORDER BY)) - Limit (preliminary LIMIT (without OFFSET)) - ReadFromMergeTree (default.tab) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 3/3 - Skip - Name: annoy_index - Description: annoy GRANULARITY 1 - Parts: 1/1 - Granules: 1/3 -ORDER BY type, L2Distance, check that index is used -Expression (Projection) - Limit (preliminary LIMIT (without OFFSET)) - Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) - ReadFromMergeTree (default.tab) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 3/3 - Skip - Name: annoy_index - Description: annoy GRANULARITY 1 - Parts: 1/1 - Granules: 3/3 -parameter annoy_index_search_k_nodes -parameter max_limit_for_ann_queries -Expression (Projection) - Limit (preliminary LIMIT (without OFFSET)) - Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) - ReadFromMergeTree (default.tab) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 3/3 ---- Test with Tuple --- -WHERE type, L2Distance -1 (0,0,10) -2 (0,0,10.5) -3 (0,0,9.5) -4 (0,0,9.7) -5 (0,0,10.2) -ORDER BY type, L2Distance -1 (0,0,10) -5 (0,0,10.2) -4 (0,0,9.7) -WHERE type, L2Distance, check that index is used -Expression ((Projection + Before ORDER BY)) - Limit (preliminary LIMIT (without OFFSET)) - ReadFromMergeTree (default.tab) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 3/3 - Skip - Name: annoy_index - Description: annoy GRANULARITY 1 - Parts: 1/1 - Granules: 1/3 -ORDER BY type, L2Distance, check that index is used -Expression (Projection) - Limit (preliminary LIMIT (without OFFSET)) - Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) - ReadFromMergeTree (default.tab) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 3/3 - Skip - Name: annoy_index - Description: annoy GRANULARITY 1 - Parts: 1/1 - Granules: 3/3 -parameter annoy_index_search_k_nodes -parameter max_limit_for_ann_queries -Expression (Projection) - Limit (preliminary LIMIT (without OFFSET)) - Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) - ReadFromMergeTree (default.tab) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 3/3 ---- Test alternative metric (cosine distance) and non-default NumTrees --- -WHERE type, L2Distance -1 [0,0,10] -2 [0,0,10.5] -3 [0,0,9.5] -4 [0,0,9.7] -5 [0,0,10.2] -ORDER BY type, L2Distance -1 [0,0,10] -5 [0,0,10.2] -4 [0,0,9.7] --- Negative tests --- +--- Test default GRANULARITY (should be 100 mio. for annoy)--- +CREATE TABLE default.tab\n(\n `id` Int32,\n `vector` Array(Float32),\n INDEX annoy_index vector TYPE annoy GRANULARITY 100000000\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `id` Int32,\n `vector` Array(Float32),\n INDEX annoy_index vector TYPE annoy GRANULARITY 100000000\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +--- Test with Array, GRANULARITY = 1, index_granularity = 5 --- +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 1/3 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 3/3 +Reference ARRAYs with non-matching dimension are rejected +Special case: MaximumDistance is negative +WHERE type, L2Distance +Special case: setting annoy_index_search_k_nodes +Special case: setting max_limit_for_ann_queries +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 +--- Test with Tuple, GRANULARITY = 1, index_granularity = 5 --- +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 1/3 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 3/3 +--- Test non-default metric (cosine distance) + non-default NumTrees (200) --- +--- Test with Array, GRANULARITY = 2, index_granularity = 4 --- +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: annoy_index + Description: annoy GRANULARITY 2 + Parts: 0/1 + Granules: 2/4 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: annoy_index + Description: annoy GRANULARITY 2 + Parts: 1/1 + Granules: 4/4 +--- Test with Array, GRANULARITY = 4, index_granularity = 4 --- +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: annoy_index + Description: annoy GRANULARITY 4 + Parts: 0/1 + Granules: 3/4 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: annoy_index + Description: annoy GRANULARITY 4 + Parts: 1/1 + Granules: 4/4 diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index abee5e8a6e4..fefb51dfcc9 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -1,150 +1,251 @@ --- Tags: disabled, no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check +-- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check SET allow_experimental_annoy_index = 1; - -SELECT '--- Test with Array ---'; - -DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; -INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); - -SELECT 'WHERE type, L2Distance'; -SELECT * -FROM tab -WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 5; - -SELECT 'ORDER BY type, L2Distance'; -SELECT * -FROM tab -ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; - --- Produces different error code with analyzer, TODO: check --- SELECT 'Reference ARRAYs with non-matching dimension are rejected'; --- SELECT * --- FROM tab --- ORDER BY L2Distance(embedding, [0.0, 0.0]) --- LIMIT 3; -- { serverError INCORRECT_QUERY } - -SELECT 'WHERE type, L2Distance, check that index is used'; -EXPLAIN indexes=1 -SELECT * -FROM tab -WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 5; - -SELECT 'ORDER BY type, L2Distance, check that index is used'; -EXPLAIN indexes=1 -SELECT * -FROM tab -ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; - -SELECT 'parameter annoy_index_search_k_nodes'; -SELECT * -FROM tab -ORDER BY L2Distance(embedding, [5.3, 7.3, 2.1]) -LIMIT 5 -SETTINGS annoy_index_search_k_nodes=0; -- searches zero nodes --> no results - -SELECT 'parameter max_limit_for_ann_queries'; -EXPLAIN indexes=1 -SELECT * -FROM tab -ORDER BY L2Distance(embedding, [5.3, 7.3, 2.1]) -LIMIT 5 -SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index - -DROP TABLE tab; - -SELECT '--- Test with Tuple ---'; - -CREATE TABLE tab(id Int32, embedding Tuple(Float32, Float32, Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; -INSERT INTO tab VALUES (1, (0.0, 0.0, 10.0)), (2, (0.0, 0.0, 10.5)), (3, (0.0, 0.0, 9.5)), (4, (0.0, 0.0, 9.7)), (5, (0.0, 0.0, 10.2)), (6, (10.0, 0.0, 0.0)), (7, (9.5, 0.0, 0.0)), (8, (9.7, 0.0, 0.0)), (9, (10.2, 0.0, 0.0)), (10, (10.5, 0.0, 0.0)), (11, (0.0, 10.0, 0.0)), (12, (0.0, 9.5, 0.0)), (13, (0.0, 9.7, 0.0)), (14, (0.0, 10.2, 0.0)), (15, (0.0, 10.5, 0.0)); - -SELECT 'WHERE type, L2Distance'; -SELECT * -FROM tab -WHERE L2Distance(embedding, (0.0, 0.0, 10.0)) < 1.0 -LIMIT 5; - -SELECT 'ORDER BY type, L2Distance'; -SELECT * -FROM tab -ORDER BY L2Distance(embedding, (0.0, 0.0, 10.0)) -LIMIT 3; - -SELECT 'WHERE type, L2Distance, check that index is used'; -EXPLAIN indexes=1 -SELECT * -FROM tab -WHERE L2Distance(embedding, (0.0, 0.0, 10.0)) < 1.0 -LIMIT 5; - -SELECT 'ORDER BY type, L2Distance, check that index is used'; -EXPLAIN indexes=1 -SELECT * -FROM tab -ORDER BY L2Distance(embedding, (0.0, 0.0, 10.0)) -LIMIT 3; - -SELECT 'parameter annoy_index_search_k_nodes'; -SELECT * -FROM tab -ORDER BY L2Distance(embedding, (5.3, 7.3, 2.1)) -LIMIT 5 -SETTINGS annoy_index_search_k_nodes=0; -- searches zero nodes --> no results - -SELECT 'parameter max_limit_for_ann_queries'; -EXPLAIN indexes=1 -SELECT * -FROM tab -ORDER BY L2Distance(embedding, (5.3, 7.3, 2.1)) -LIMIT 5 -SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index - -DROP TABLE tab; - -SELECT '--- Test alternative metric (cosine distance) and non-default NumTrees ---'; - -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('cosineDistance', 200)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; -INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); - -SELECT 'WHERE type, L2Distance'; -SELECT * -FROM tab -WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 5; - -SELECT 'ORDER BY type, L2Distance'; -SELECT * -FROM tab -ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; - -DROP TABLE tab; +SET allow_experimental_analyzer = 0; SELECT '--- Negative tests ---'; +DROP TABLE IF EXISTS tab; + -- must have at most 2 arguments -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } -- first argument (distance_function) must be String -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } -- 2nd argument (number of trees) must be UInt64 -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('L2Distance', 'not an UInt64')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } - --- reject unsupported distance functions -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy('L2Distance', 'not an UInt64')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } -- must be created on single column -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index (embedding, id) TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index (vector, id) TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } + +-- reject unsupported distance functions +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } -- must be created on Array/Tuple(Float32) columns SET allow_suspicious_low_cardinality_types = 1; -CREATE TABLE tab(id Int32, embedding Float32, INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, embedding Array(Float64), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, embedding LowCardinality(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } -CREATE TABLE tab(id Int32, embedding Nullable(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vector Float32, INDEX annoy_index vector TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vector Array(Float64), INDEX annoy_index vector TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vector Tuple(Float64), INDEX annoy_index vector TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vector LowCardinality(Float32), INDEX annoy_index vector TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vector Nullable(Float32), INDEX annoy_index vector TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } + +SELECT '--- Test default GRANULARITY (should be 100 mio. for annoy)---'; + +CREATE TABLE tab (id Int32, vector Array(Float32), INDEX annoy_index(vector) TYPE annoy) ENGINE=MergeTree ORDER BY id; +SHOW CREATE TABLE tab; +DROP TABLE tab; + +CREATE TABLE tab (id Int32, vector Array(Float32)) ENGINE=MergeTree ORDER BY id; +ALTER TABLE tab ADD INDEX annoy_index(vector) TYPE annoy; +SHOW CREATE TABLE tab; + +DROP TABLE tab; + +SELECT '--- Test with Array, GRANULARITY = 1, index_granularity = 5 ---'; + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy() GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 5; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); + +-- rows = 15, index_granularity = 5, GRANULARITY = 1 gives 3 annoy-indexed blocks (each comprising a single granule) +-- condition 'L2Distance(vector, reference_vector) < 1.0' ensures that only one annoy-indexed block produces results --> "Granules: 1/3" + +-- See (*) why commented out +-- SELECT 'WHERE type, L2Distance'; +-- SELECT * +-- FROM tab +-- WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0 +-- LIMIT 3; + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0 +LIMIT 3; + +-- See (*) why commented out +-- SELECT 'ORDER BY type, L2Distance'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(vector, [0.0, 0.0, 10.0]) +-- LIMIT 3; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(vector, [0.0, 0.0, 10.0]) +LIMIT 3; + +-- Test special cases. Corresponding special case tests are omitted from later tests. + +SELECT 'Reference ARRAYs with non-matching dimension are rejected'; +SELECT * +FROM tab +ORDER BY L2Distance(vector, [0.0, 0.0]) +LIMIT 3; -- { serverError INCORRECT_QUERY } + +SELECT 'Special case: MaximumDistance is negative'; +SELECT 'WHERE type, L2Distance'; +SELECT * +FROM tab +WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < -1.0 +LIMIT 3; -- { serverError INCORRECT_QUERY } + +SELECT 'Special case: setting annoy_index_search_k_nodes'; +SELECT * +FROM tab +ORDER BY L2Distance(vector, [5.3, 7.3, 2.1]) +LIMIT 3 +SETTINGS annoy_index_search_k_nodes=0; -- searches zero nodes --> no results + +SELECT 'Special case: setting max_limit_for_ann_queries'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(vector, [5.3, 7.3, 2.1]) +LIMIT 3 +SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index + +DROP TABLE tab; + +-- Test Tuple embeddings. Triggers different logic than Array inside MergeTreeIndexAnnoy but the same logic as Array above MergeTreeIndexAnnoy. +-- Therefore test Tuple case just once. + +SELECT '--- Test with Tuple, GRANULARITY = 1, index_granularity = 5 ---'; + +CREATE TABLE tab(id Int32, vector Tuple(Float32, Float32, Float32), INDEX annoy_index vector TYPE annoy() GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 5; +INSERT INTO tab VALUES (1, (0.0, 0.0, 10.0)), (2, (0.0, 0.0, 10.5)), (3, (0.0, 0.0, 9.5)), (4, (0.0, 0.0, 9.7)), (5, (0.0, 0.0, 10.2)), (6, (10.0, 0.0, 0.0)), (7, (9.5, 0.0, 0.0)), (8, (9.7, 0.0, 0.0)), (9, (10.2, 0.0, 0.0)), (10, (10.5, 0.0, 0.0)), (11, (0.0, 10.0, 0.0)), (12, (0.0, 9.5, 0.0)), (13, (0.0, 9.7, 0.0)), (14, (0.0, 10.2, 0.0)), (15, (0.0, 10.5, 0.0)); + +-- See (*) why commented out +-- SELECT 'WHERE type, L2Distance'; +-- SELECT * +-- FROM tab +-- WHERE L2Distance(vector, (0.0, 0.0, 10.0)) < 1.0 +-- LIMIT 3; + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(vector, (0.0, 0.0, 10.0)) < 1.0 +LIMIT 3; + +-- See (*) why commented out +-- SELECT 'ORDER BY type, L2Distance'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(vector, (0.0, 0.0, 10.0)) +-- LIMIT 3; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(vector, (0.0, 0.0, 10.0)) +LIMIT 3; + +DROP TABLE tab; + +-- Not a systematic test, just to make sure no bad things happen +SELECT '--- Test non-default metric (cosine distance) + non-default NumTrees (200) ---'; + +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy('cosineDistance', 200) GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 5; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); + +-- See (*) why commented out +-- SELECT 'WHERE type, L2Distance'; +-- SELECT * +-- FROM tab +-- WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0 +-- LIMIT 3; + +-- See (*) why commented out +-- SELECT 'ORDER BY type, L2Distance'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(vector, [0.0, 0.0, 10.0]) +-- LIMIT 3; + +DROP TABLE tab; + +SELECT '--- Test with Array, GRANULARITY = 2, index_granularity = 4 ---'; + +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 4; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0, 0.0]), (2, [0.0, 0.0, 10.5, 0.0]), (3, [0.0, 0.0, 9.5, 0.0]), (4, [0.0, 0.0, 9.7, 0.0]), (5, [10.0, 0.0, 0.0, 0.0]), (6, [9.5, 0.0, 0.0, 0.0]), (7, [9.7, 0.0, 0.0, 0.0]), (8, [10.2, 0.0, 0.0, 0.0]), (9, [0.0, 10.0, 0.0, 0.0]), (10, [0.0, 9.5, 0.0, 0.0]), (11, [0.0, 9.7, 0.0, 0.0]), (12, [0.0, 9.7, 0.0, 0.0]), (13, [0.0, 0.0, 0.0, 10.3]), (14, [0.0, 0.0, 0.0, 9.5]), (15, [0.0, 0.0, 0.0, 10.0]), (16, [0.0, 0.0, 0.0, 10.5]); + +-- rows = 16, index_granularity = 4, GRANULARITY = 2 gives 2 annoy-indexed blocks (each comprising two granules) +-- condition 'L2Distance(vector, reference_vector) < 1.0' ensures that only one annoy-indexed block produces results --> "Granules: 2/4" + +-- See (*) why commented out +-- SELECT 'WHERE type, L2Distance'; +-- SELECT * +-- FROM tab +-- WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0 +-- LIMIT 3; + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0 +LIMIT 3; + +-- See (*) why commented out +-- SELECT 'ORDER BY type, L2Distance'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) +-- LIMIT 3; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) +LIMIT 3; + +DROP TABLE tab; + +SELECT '--- Test with Array, GRANULARITY = 4, index_granularity = 4 ---'; + +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX annoy_index vector TYPE annoy() GRANULARITY 4) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 4; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0, 0.0]), (2, [0.0, 0.0, 10.5, 0.0]), (3, [0.0, 0.0, 9.5, 0.0]), (4, [0.0, 0.0, 9.7, 0.0]), (5, [10.0, 0.0, 0.0, 0.0]), (6, [9.5, 0.0, 0.0, 0.0]), (7, [9.7, 0.0, 0.0, 0.0]), (8, [10.2, 0.0, 0.0, 0.0]), (9, [0.0, 10.0, 0.0, 0.0]), (10, [0.0, 9.5, 0.0, 0.0]), (11, [0.0, 9.7, 0.0, 0.0]), (12, [0.0, 9.7, 0.0, 0.0]), (13, [0.0, 0.0, 0.0, 10.3]), (14, [0.0, 0.0, 0.0, 9.5]), (15, [0.0, 0.0, 0.0, 10.0]), (16, [0.0, 0.0, 0.0, 10.5]); + +-- rows = 16, index_granularity = 4, GRANULARITY = 4 gives a single annoy-indexed block (comprising all granules) +-- no two matches happen to be located in the same granule, so with LIMIT = 3, we'll get "Granules: 2/4" + +-- See (*) why commented out +-- SELECT 'WHERE type, L2Distance'; +-- SELECT * +-- FROM tab +-- WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0 +-- LIMIT 3; + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0 +LIMIT 3; + +-- See (*) why commented out +-- SELECT 'ORDER BY type, L2Distance'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) +-- LIMIT 3; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) +LIMIT 3; + +DROP TABLE tab; + +-- (*) Storage and search in Annoy indexes is inherently random. Tests which check for exact row matches would be unstable. Therefore, +-- comment them out. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d6cef1883f4..021855e399f 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -26,6 +26,7 @@ AlertManager Alexey AnyEvent AppleClang +Approximative ArrayJoin ArrowStream AsyncInsertCacheSize @@ -1005,6 +1006,7 @@ anyLast anyheavy anylast appendTrailingCharIfAbsent +approximative argMax argMin argmax @@ -2419,6 +2421,7 @@ unescaping unhex unicode unidimensional +unintuitive uniq uniqCombined uniqExact From 4d4e5c690e446db23f8a1ef7fc1e577df93e9373 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 8 Jun 2023 17:10:51 +0200 Subject: [PATCH 1389/2223] update docs spelling check failed --- docs/en/operations/settings/settings.md | 2 +- docs/en/sql-reference/functions/date-time-functions.md | 4 +--- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 77c9238e4c7..6c9c8349519 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4116,7 +4116,7 @@ SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS sessi This happens due to different parsing pipelines: - `toDateTime('2000-01-01 00:00:00')` creates a new DateTime in a usual way, and thus `session_timezone` setting from query context is applied. - - `2000-01-01 00:00:00` is parsed to a DateTime inheriting type of `d` column, including DateTime's time zone, and `session_timezone` has no impact on this value. + - `2000-01-01 00:00:00` is parsed to a DateTime inheriting type of `d` column, including its time zone, and `session_timezone` has no impact on this value. Possible values: diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 1a5b0dcabf9..89ac6d438ff 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -163,7 +163,7 @@ Type: [String](../../sql-reference/data-types/string.md). ## serverTimeZone Returns the default timezone of the server, i.e. the value of setting [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise, it produces a constant value. **Syntax** @@ -171,8 +171,6 @@ If it is executed in the context of a distributed table, then it generates a nor serverTimeZone() ``` -Alias: `ServerTimezone`, `servertimezone`. - **Returned value** - Timezone. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ded7a4643a9..8301579b6a8 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -484,6 +484,7 @@ russian rw sasl schemas +servertimezone simdjson skippingerrors sparsehash From b6e32cd5e3ba632805905c2c1ea8493471053f9f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Jun 2023 15:30:23 +0000 Subject: [PATCH 1390/2223] Do not read all the columns from right GLOBAL JOIN table. --- src/Interpreters/ExpressionAnalyzer.cpp | 11 ++------ src/Interpreters/GlobalSubqueriesVisitor.h | 28 +++++++++++++++---- src/Interpreters/TableJoin.cpp | 9 ++++++ src/Interpreters/TableJoin.h | 2 ++ ...785_global_join_too_many_columns.reference | 1 + .../02785_global_join_too_many_columns.sql | 14 ++++++++++ 6 files changed, 51 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/02785_global_join_too_many_columns.reference create mode 100644 tests/queries/0_stateless/02785_global_join_too_many_columns.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index c7c66f6f414..307b46b3a0b 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -444,7 +444,7 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global, b if (do_global) { GlobalSubqueriesVisitor::Data subqueries_data( - getContext(), subquery_depth, isRemoteStorage(), is_explain, external_tables, prepared_sets, has_global_subqueries); + getContext(), subquery_depth, isRemoteStorage(), is_explain, external_tables, prepared_sets, has_global_subqueries, syntax->analyzed_join.get()); GlobalSubqueriesVisitor(subqueries_data).visit(query); } } @@ -1056,13 +1056,6 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin( return join; } -static ActionsDAGPtr createJoinedBlockActions(ContextPtr context, const TableJoin & analyzed_join) -{ - ASTPtr expression_list = analyzed_join.rightKeysList(); - auto syntax_result = TreeRewriter(context).analyze(expression_list, analyzed_join.columnsFromJoinedTable()); - return ExpressionAnalyzer(expression_list, syntax_result, context).getActionsDAG(true, false); -} - std::shared_ptr tryKeyValueJoin(std::shared_ptr analyzed_join, const Block & right_sample_block); @@ -1144,7 +1137,7 @@ static std::unique_ptr buildJoinedPlan( SelectQueryOptions query_options) { /// Actions which need to be calculated on joined block. - auto joined_block_actions = createJoinedBlockActions(context, analyzed_join); + auto joined_block_actions = analyzed_join.createJoinedBlockActions(context); NamesWithAliases required_columns_with_aliases = analyzed_join.getRequiredColumns( Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 2901f2e23d0..f5b837fc7f7 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -43,6 +44,7 @@ public: TemporaryTablesMapping & external_tables; PreparedSetsPtr prepared_sets; bool & has_global_subqueries; + TableJoin * table_join; Data( ContextPtr context_, @@ -51,7 +53,8 @@ public: bool is_explain_, TemporaryTablesMapping & tables, PreparedSetsPtr prepared_sets_, - bool & has_global_subqueries_) + bool & has_global_subqueries_, + TableJoin * table_join_) : WithContext(context_) , subquery_depth(subquery_depth_) , is_remote(is_remote_) @@ -59,10 +62,11 @@ public: , external_tables(tables) , prepared_sets(prepared_sets_) , has_global_subqueries(has_global_subqueries_) + , table_join(table_join_) { } - void addExternalStorage(ASTPtr & ast, bool set_alias = false) + void addExternalStorage(ASTPtr & ast, const Names & required_columns, bool set_alias = false) { /// With nondistributed queries, creating temporary tables does not make sense. if (!is_remote) @@ -145,7 +149,7 @@ public: if (external_tables.contains(external_table_name)) return; - auto interpreter = interpretSubquery(subquery_or_table_name, getContext(), subquery_depth, {}); + auto interpreter = interpretSubquery(subquery_or_table_name, getContext(), subquery_depth, required_columns); Block sample = interpreter->getSampleBlock(); NamesAndTypesList columns = sample.getNamesAndTypesList(); @@ -238,7 +242,7 @@ private: return; } - data.addExternalStorage(ast); + data.addExternalStorage(ast, {}); data.has_global_subqueries = true; } } @@ -249,7 +253,21 @@ private: if (table_elem.table_join && (table_elem.table_join->as().locality == JoinLocality::Global || shouldBeExecutedGlobally(data))) { - data.addExternalStorage(table_elem.table_expression, true); + Names required_columns; + + /// Fill required columns for GLOBAL JOIN. + /// This code is partial copy-paste from ExpressionAnalyzer. + if (data.table_join) + { + auto joined_block_actions = data.table_join->createJoinedBlockActions(data.getContext()); + NamesWithAliases required_columns_with_aliases = data.table_join->getRequiredColumns( + Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); + + for (auto & pr : required_columns_with_aliases) + required_columns.push_back(pr.first); + } + + data.addExternalStorage(table_elem.table_expression, required_columns, true); data.has_global_subqueries = true; } } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 5a23fbd00ff..c56d1e6039e 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -14,6 +14,8 @@ #include #include +#include +#include #include #include @@ -760,4 +762,11 @@ bool TableJoin::allowParallelHashJoin() const return true; } +ActionsDAGPtr TableJoin::createJoinedBlockActions(ContextPtr context) +{ + ASTPtr expression_list = rightKeysList(); + auto syntax_result = TreeRewriter(context).analyze(expression_list, columnsFromJoinedTable()); + return ExpressionAnalyzer(expression_list, syntax_result, context).getActionsDAG(true, false); +} + } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 6737cd8f13a..096e58d1292 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -217,6 +217,8 @@ public: const SizeLimits & sizeLimits() const { return size_limits; } VolumePtr getGlobalTemporaryVolume() { return tmp_volume; } + ActionsDAGPtr createJoinedBlockActions(ContextPtr context); + bool isEnabledAlgorithm(JoinAlgorithm val) const { /// When join_algorithm = 'default' (not specified by user) we use hash or direct algorithm. diff --git a/tests/queries/0_stateless/02785_global_join_too_many_columns.reference b/tests/queries/0_stateless/02785_global_join_too_many_columns.reference new file mode 100644 index 00000000000..425151f3a41 --- /dev/null +++ b/tests/queries/0_stateless/02785_global_join_too_many_columns.reference @@ -0,0 +1 @@ +40 diff --git a/tests/queries/0_stateless/02785_global_join_too_many_columns.sql b/tests/queries/0_stateless/02785_global_join_too_many_columns.sql new file mode 100644 index 00000000000..a49aae25f1c --- /dev/null +++ b/tests/queries/0_stateless/02785_global_join_too_many_columns.sql @@ -0,0 +1,14 @@ +drop table if exists local; +drop table if exists distr; + +create table local (a UInt64, b UInt64, c UInt64, d UInt64, e UInt64, f UInt64, g UInt64, h UInt64) engine = Log; +create table distr as local engine = Distributed('test_cluster_two_shards', currentDatabase(), local); + +insert into local (a) select number from numbers(10); + +set max_columns_to_read=1; +select count() from distr as l global all left join distr as r on l.a = r.a; + +drop table if exists local; +drop table if exists distr; + From 01873e9e6d9aeaaa8ae8dd2e78bd9f75ea5c70e6 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Thu, 8 Jun 2023 14:09:01 +0200 Subject: [PATCH 1391/2223] Add async insert system tables documentation --- .../settings.md | 26 ++++++++ .../system-tables/asynchronous_insert_log.md | 63 +++++++++++++++++++ .../system-tables/asynchronous_inserts.md | 45 +++++++++++++ 3 files changed, 134 insertions(+) create mode 100644 docs/en/operations/system-tables/asynchronous_insert_log.md create mode 100644 docs/en/operations/system-tables/asynchronous_inserts.md diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index f93ab264511..3398c7afb67 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1881,6 +1881,32 @@ The default server configuration file `config.xml` contains the following settin ``` +## asynchronous_insert_log {#server_configuration_parameters-asynchronous_insert_log} + +Settings for the [asynchronous_insert_log](../../operations/system-tables/asynchronous_insert_log.md#system_tables-asynchronous_insert_log) system table for logging async inserts. + +Parameters: + +- `database` — Database name. +- `table` — Table name. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. +- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `storage_policy` – Name of storage policy to use for the table (optional) + +**Example** +```xml + + + system +
asynchronous_insert_log
+ 7500 + toYYYYMM(event_date) + + + +``` + ## query_masking_rules {#query-masking-rules} Regexp-based rules, which will be applied to queries as well as all log messages before storing them in server logs, diff --git a/docs/en/operations/system-tables/asynchronous_insert_log.md b/docs/en/operations/system-tables/asynchronous_insert_log.md new file mode 100644 index 00000000000..8b0509d7000 --- /dev/null +++ b/docs/en/operations/system-tables/asynchronous_insert_log.md @@ -0,0 +1,63 @@ +--- +slug: /en/operations/system-tables/asynchronous_insert_log +--- +# asynchronous_insert_log + +Contains information about async inserts. Each entry represents an insert query buffered into an async insert query. + +To start logging configure parameters in the [asynchronous_insert_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-asynchronous_insert_log) section. + +The flushing period of data is set in `flush_interval_milliseconds` parameter of the [asynchronous_insert_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-asynchronous_insert_log) server settings section. To force flushing, use the [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) query. + +ClickHouse does not delete data from the table automatically. See [Introduction](../../operations/system-tables/index.md#system-tables-introduction) for more details. + +Columns: + +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the async insert happened. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the async insert finished execution. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time when the async insert finished execution with microseconds precision. +- `query` ([String](../../sql-reference/data-types/string.md)) — Query string. +- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the table is in. +- `table` ([String](../../sql-reference/data-types/string.md)) — Table name. +- `format` ([String](/docs/en/sql-reference/data-types/string.md)) — Format name. +- `query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the initial query. +- `bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of inserted bytes. +- `exception` ([String](../../sql-reference/data-types/string.md)) — Exception message. +- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — Status of the view. Values: + - `'Ok' = 1` — Successful insert. + - `'ParsingError' = 2` — Exception when parsing the data. + - `'FlushError' = 3` — Exception when flushing the data. +- `flush_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the flush happened. +- `flush_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time when the flush happened with microseconds precision. +- `flush_query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the flush query. + +**Example** + +Query: + +``` sql +SELECT * FROM system.asynchronous_insert_log LIMIT 1 \G; +``` + +Result: + +``` text +event_date: 2023-06-08 +event_time: 2023-06-08 10:08:53 +event_time_microseconds: 2023-06-08 10:08:53.199516 +query: INSERT INTO public.data_guess (user_id, datasource_id, timestamp, path, type, num, str) FORMAT CSV +database: public +table: data_guess +format: CSV +query_id: b46cd4c4-0269-4d0b-99f5-d27668c6102e +bytes: 133223 +exception: +status: Ok +flush_time: 2023-06-08 10:08:55 +flush_time_microseconds: 2023-06-08 10:08:55.139676 +flush_query_id: cd2c1e43-83f5-49dc-92e4-2fbc7f8d3716 + +**See Also** + +- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — Description of the `query_log` system table which contains common information about queries execution. +- [system.asynchronous_inserts](../../operations/system-tables/asynchronous_inserts.md#system_tables-asynchronous_inserts) — This table contains information about pending asynchronous inserts in queue. diff --git a/docs/en/operations/system-tables/asynchronous_inserts.md b/docs/en/operations/system-tables/asynchronous_inserts.md new file mode 100644 index 00000000000..8fd5f2bb520 --- /dev/null +++ b/docs/en/operations/system-tables/asynchronous_inserts.md @@ -0,0 +1,45 @@ +--- +slug: /en/operations/system-tables/asynchronous_inserts +--- +# asynchronous_inserts + +Contains information about pending asynchronous inserts in queue. + +Columns: + +- `query` ([String](../../sql-reference/data-types/string.md)) — Query string. +- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the table is in. +- `table` ([String](../../sql-reference/data-types/string.md)) — Table name. +- `format` ([String](/docs/en/sql-reference/data-types/string.md)) — Format name. +- `first_update` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — First insert time with microseconds resolution. +- `total_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number of bytes waiting in the queue. +- `entries.query_id` ([Array(String)](../../sql-reference/data-types/array.md)) - Array of query ids of the inserts waiting in the queue. +- `entries.bytes` ([Array(UInt64)](../../sql-reference/data-types/array.md)) - Array of bytes of each insert query waiting in the queue. + +**Example** + +Query: + +``` sql +SELECT * FROM system.asynchronous_inserts LIMIT 1 \G; +``` + +Result: + +``` text +Row 1: +────── +query: INSERT INTO public.data_guess (user_id, datasource_id, timestamp, path, type, num, str) FORMAT CSV +database: public +table: data_guess +format: CSV +first_update: 2023-06-08 10:08:54.199606 +total_bytes: 133223 +entries.query_id: ['b46cd4c4-0269-4d0b-99f5-d27668c6102e'] +entries.bytes: [133223] +``` + +**See Also** + +- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — Description of the `query_log` system table which contains common information about queries execution. +- [system.asynchronous_insert_log](../../operations/system-tables/asynchronous_insert_log.md#system_tables-asynchronous_insert_log) — This table contains information about async inserts performed. From ea9d0f6c3c94ae6fef1bb552958455c7ab9b25e7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 8 Jun 2023 18:18:12 +0200 Subject: [PATCH 1392/2223] Fix --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 55 +++++++++---------- 1 file changed, 27 insertions(+), 28 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 6bf72434580..6317aba20e9 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -1038,34 +1038,6 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() current_file_segment_counters.increment(ProfileEvents::FileSegmentUsedBytes, available()); - // No necessary because of the SCOPE_EXIT above, but useful for logging below. - if (download_current_segment) - file_segment.completePartAndResetDownloader(); - - chassert(!file_segment.isDownloader()); - - LOG_TEST( - log, - "Key: {}. Returning with {} bytes, buffer position: {} (offset: {}, predownloaded: {}), " - "buffer available: {}, current range: {}, file offset of buffer end: {}, impl offset: {}, file segment state: {}, " - "current write offset: {}, read_type: {}, reading until position: {}, started with offset: {}, " - "remaining ranges: {}", - cache_key.toString(), - working_buffer.size(), - getPosition(), - offset(), - needed_to_predownload, - available(), - current_read_range.toString(), - file_offset_of_buffer_end, - implementation_buffer->getFileOffsetOfBufferEnd(), - FileSegment::stateToString(file_segment.state()), - file_segment.getCurrentWriteOffset(false), - toString(read_type), - read_until_position, - first_offset, - file_segments->toString()); - if (size == 0 && file_offset_of_buffer_end < read_until_position) { size_t cache_file_size = getFileSizeFromReadBuffer(*implementation_buffer); @@ -1086,6 +1058,33 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() file_segment.getInfoForLog()); } + // No necessary because of the SCOPE_EXIT above, but useful for logging below. + if (download_current_segment) + file_segment.completePartAndResetDownloader(); + + chassert(!file_segment.isDownloader()); + + LOG_TEST( + log, + "Key: {}. Returning with {} bytes, buffer position: {} (offset: {}, predownloaded: {}), " + "buffer available: {}, current range: {}, file offset of buffer end: {}, file segment state: {}, " + "current write offset: {}, read_type: {}, reading until position: {}, started with offset: {}, " + "remaining ranges: {}", + cache_key.toString(), + working_buffer.size(), + getPosition(), + offset(), + needed_to_predownload, + available(), + current_read_range.toString(), + file_offset_of_buffer_end, + FileSegment::stateToString(file_segment.state()), + file_segment.getCurrentWriteOffset(false), + toString(read_type), + read_until_position, + first_offset, + file_segments->toString()); + return result; } From 314df9a6da2ca03cfea6952b5aa47654d974666e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Jun 2023 16:25:06 +0000 Subject: [PATCH 1393/2223] Fixing test. --- .../0_stateless/02731_parallel_replicas_join_subquery.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 df606679523..9b08e69b9d3 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -39,6 +39,6 @@ U c 10 UlI+1 10 bX?}ix [ Ny]2 G 10 t toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `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_11888098645495698704_17868075224240210014` 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; From a1b1e12e5bb5d6a1937f9081eb43374afef60f9b Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 8 Jun 2023 18:38:51 +0200 Subject: [PATCH 1394/2223] upd spell --- docs/en/sql-reference/functions/date-time-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 89ac6d438ff..62bbb84053a 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -158,9 +158,9 @@ Type: [String](../../sql-reference/data-types/string.md). **See also** -- [serverTimeZone](#serverTimeZone) +- [serverTimezone](#serverTimeZone) -## serverTimeZone +## serverTimezone Returns the default timezone of the server, i.e. the value of setting [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise, it produces a constant value. From 43cb2024a733fd5cbd7e0593ce317fad5e223130 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 8 Jun 2023 19:20:32 +0200 Subject: [PATCH 1395/2223] Better comments Co-authored-by: Dmitry Novik --- src/Processors/QueryPlan/Optimizations/optimizeTree.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 091eecf99e5..de4916797e9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -163,6 +163,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } + /// NOTE: optimizePrewhere can modify the stack. optimizePrewhere(stack, nodes); optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*stack.back().node, nodes); From dc08583ba492b7ec4b38c0e1fa4c38fc3310f630 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 8 Jun 2023 19:20:40 +0200 Subject: [PATCH 1396/2223] Better comments Co-authored-by: Dmitry Novik --- src/Processors/QueryPlan/Optimizations/optimizeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index de4916797e9..73632d34671 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -115,7 +115,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s while (!stack.empty()) { { - /// NOTE: frame cannot be safely used after adding new elements to stack + /// NOTE: frame cannot be safely used after stack was modified. auto & frame = stack.back(); if (frame.next_child == 0) From f4bf42cc42a6a2d8966d711874286427956d51d7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 8 Jun 2023 17:29:51 +0000 Subject: [PATCH 1397/2223] more fixes --- .../AggregateFunctionGroupArray.cpp | 1 + .../AggregateFunctionGroupUniqArray.cpp | 11 ++ .../AggregateFunctionTopK.cpp | 19 ++ .../AggregateFunctionUniq.h | 31 +++- .../AggregateFunctionUniqCombined.cpp | 5 + .../AggregateFunctionUniqCombined.h | 16 +- ...es_aggregate_functions_states.reference.j2 | 172 ++++++++++++++++++ ...ip_types_aggregate_functions_states.sql.j2 | 123 +++++++++++++ 8 files changed, 366 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 create mode 100644 tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 15f500b8bb6..bb1368b9ff8 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -25,6 +25,7 @@ IAggregateFunction * createWithNumericOrTimeType(const IDataType & argument_type WhichDataType which(argument_type); if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate(std::forward(args)...); if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate(std::forward(args)...); + if (which.idx == TypeIndex::IPv4) return new AggregateFunctionTemplate(std::forward(args)...); return createWithNumericType(argument_type, std::forward(args)...); } diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp index 16f2feb71bf..9e8060d44cc 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -39,12 +40,22 @@ public: static DataTypePtr createResultType() { return std::make_shared(std::make_shared()); } }; +template +class AggregateFunctionGroupUniqArrayIPv4 : public AggregateFunctionGroupUniqArray +{ +public: + explicit AggregateFunctionGroupUniqArrayIPv4(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits::max()) + : AggregateFunctionGroupUniqArray(argument_type, parameters_, createResultType(), max_elems_) {} + static DataTypePtr createResultType() { return std::make_shared(std::make_shared()); } +}; + template IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, TArgs && ... args) { WhichDataType which(argument_type); if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate(argument_type, std::forward(args)...); else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupUniqArrayDateTime(argument_type, std::forward(args)...); + else if (which.idx == TypeIndex::IPv4) return new AggregateFunctionGroupUniqArrayIPv4(argument_type, std::forward(args)...); else { /// Check that we can use plain version of AggregateFunctionGroupUniqArrayGeneric diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index e568694df02..8f6652223cc 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -5,6 +5,7 @@ #include #include #include +#include static inline constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF; @@ -60,6 +61,22 @@ public: {} }; +template +class AggregateFunctionTopKIPv4 : public AggregateFunctionTopK +{ +public: + using AggregateFunctionTopK::AggregateFunctionTopK; + + AggregateFunctionTopKIPv4(UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params) + : AggregateFunctionTopK( + threshold_, + load_factor, + argument_types_, + params, + std::make_shared(std::make_shared())) + {} +}; + template IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt64 threshold, UInt64 load_factor, const Array & params) @@ -72,6 +89,8 @@ IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt return new AggregateFunctionTopKDate(threshold, load_factor, argument_types, params); if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTopKDateTime(threshold, load_factor, argument_types, params); + if (which.idx == TypeIndex::IPv4) + return new AggregateFunctionTopKIPv4(threshold, load_factor, argument_types, params); /// Check that we can use plain version of AggregateFunctionTopKGeneric if (argument_types[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index 03d999b47e2..de68e9076a0 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -167,6 +167,25 @@ struct AggregateFunctionUniqExactData static String getName() { return "uniqExact"; } }; +/// For historical reasons IPv6 is treated as FixedString(16) +template +struct AggregateFunctionUniqExactData +{ + using Key = UInt128; + + /// When creating, the hash table must be small. + using SingleLevelSet = HashSet, HashTableAllocatorWithStackMemory>; + using TwoLevelSet = TwoLevelHashSet; + using Set = UniqExactSet; + + Set set; + + constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_variadic = false; + + static String getName() { return "uniqExact"; } +}; + template struct AggregateFunctionUniqExactDataForVariadic : AggregateFunctionUniqExactData { @@ -275,12 +294,7 @@ struct Adder else if constexpr (std::is_same_v>) { const auto & column = *columns[0]; - if constexpr (!std::is_same_v) - { - data.set.template insert( - assert_cast &>(column).getData()[row_num]); - } - else + if constexpr (std::is_same_v || std::is_same_v) { StringRef value = column.getDataAt(row_num); @@ -291,6 +305,11 @@ struct Adder data.set.template insert(key); } + else + { + data.set.template insert( + assert_cast &>(column).getData()[row_num]); + } } #if USE_DATASKETCHES else if constexpr (std::is_same_v) diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index 1c59da59e83..8c2cb6ea0de 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -60,6 +61,10 @@ namespace return std::make_shared::template AggregateFunction>(argument_types, params); else if (which.isUUID()) return std::make_shared::template AggregateFunction>(argument_types, params); + else if (which.isIPv4()) + return std::make_shared::template AggregateFunction>(argument_types, params); + else if (which.isIPv6()) + return std::make_shared::template AggregateFunction>(argument_types, params); else if (which.isTuple()) { if (use_exact_hash_function) diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/src/AggregateFunctions/AggregateFunctionUniqCombined.h index d879e3b3dde..5e8fa69f9de 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -119,6 +119,10 @@ struct AggregateFunctionUniqCombinedData : public Aggr { }; +template +struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +{ +}; template class AggregateFunctionUniqCombined final @@ -141,16 +145,16 @@ public: void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { - if constexpr (!std::is_same_v) - { - const auto & value = assert_cast &>(*columns[0]).getElement(row_num); - this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits::hash(value)); - } - else + if constexpr (std::is_same_v || std::is_same_v) { StringRef value = columns[0]->getDataAt(row_num); this->data(place).set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); } + else + { + const auto & value = assert_cast &>(*columns[0]).getElement(row_num); + this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits::hash(value)); + } } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 new file mode 100644 index 00000000000..481dd723b66 --- /dev/null +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 @@ -0,0 +1,172 @@ +----- hash / State / ip4 ----- +Row 1: +────── +minState: 12535288824949910799 +maxState: 18210943739258811465 +first_valueState: 12535288824949910799 +last_valueState: 18210943739258811465 +topKState: 1594227852744382511 +groupArrayState: 8025417272361615478 +groupUniqArrayState: 919082878249747568 +uniqState: 14828781561416784358 +uniqExactState: 11391659146320471795 +uniqCombinedState: 9631896280254268221 +uniqCombined64State: 5156097536649078816 +uniqHLL12State: 9696624347265201099 +uniqThetaState: 10464560810701154023 +----- hash / State / ip6 ----- +Row 1: +────── +minState: 9428555662807296659 +maxState: 18253481702148601156 +first_valueState: 9428555662807296659 +last_valueState: 18253481702148601156 +topKState: 8045294331733869941 +groupArrayState: 10451014709837753966 +groupUniqArrayState: 1954028114836070615 +uniqState: 14986562136250471284 +uniqExactState: 10032843621916709112 +uniqCombinedState: 6379274083567016598 +uniqCombined64State: 6379274083567016598 +uniqHLL12State: 9181286681186915812 +uniqThetaState: 2415188383468008881 +----- finalizeAggregation / State / ip4 ----- +Row 1: +────── +min: 59.154.201.255 +max: 59.154.202.48 +first_value: 59.154.201.255 +last_value: 59.154.202.48 +topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] +groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 49 +uniqTheta: 50 +----- finalizeAggregation / State / ip6 ----- +Row 1: +────── +min: 8c:333c::8c:333c:0:0 +max: ff8b:333c::ff8b:333c:0:0 +first_value: 8c:333c::8c:333c:0:0 +last_value: ff8b:333c::ff8b:333c:0:0 +topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 50 +uniqTheta: 50 +----- hash / IfState / ip4 ----- +Row 1: +────── +minIfState: 12535288824949910799 +maxIfState: 18210943739258811465 +first_valueIfState: 12535288824949910799 +last_valueIfState: 18210943739258811465 +topKIfState: 1594227852744382511 +groupArrayIfState: 8025417272361615478 +groupUniqArrayIfState: 919082878249747568 +uniqIfState: 14828781561416784358 +uniqExactIfState: 11391659146320471795 +uniqCombinedIfState: 9631896280254268221 +uniqCombined64IfState: 5156097536649078816 +uniqHLL12IfState: 9696624347265201099 +uniqThetaIfState: 10464560810701154023 +----- hash / IfState / ip6 ----- +Row 1: +────── +minIfState: 9428555662807296659 +maxIfState: 18253481702148601156 +first_valueIfState: 9428555662807296659 +last_valueIfState: 18253481702148601156 +topKIfState: 8045294331733869941 +groupArrayIfState: 10451014709837753966 +groupUniqArrayIfState: 1954028114836070615 +uniqIfState: 14986562136250471284 +uniqExactIfState: 10032843621916709112 +uniqCombinedIfState: 6379274083567016598 +uniqCombined64IfState: 6379274083567016598 +uniqHLL12IfState: 9181286681186915812 +uniqThetaIfState: 2415188383468008881 +----- finalizeAggregation / IfState / ip4 ----- +Row 1: +────── +min: 59.154.201.255 +max: 59.154.202.48 +first_value: 59.154.201.255 +last_value: 59.154.202.48 +topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] +groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 49 +uniqTheta: 50 +----- finalizeAggregation / IfState / ip6 ----- +Row 1: +────── +min: 8c:333c::8c:333c:0:0 +max: ff8b:333c::ff8b:333c:0:0 +first_value: 8c:333c::8c:333c:0:0 +last_value: ff8b:333c::ff8b:333c:0:0 +topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 50 +uniqTheta: 50 +----- Arg / hash / State / ip4 ----- +Row 1: +────── +argMinState: 13774589216353164344 +argMaxState: 9177365218111013695 +----- Arg / hash / State / ip6 ----- +Row 1: +────── +argMinState: 7320668278649617037 +argMaxState: 16598449636475438091 +----- Arg / finalizeAggregation / State / ip4 ----- +Row 1: +────── +argMinState: 59.154.201.255 +argMaxState: 59.154.202.48 +----- Arg / finalizeAggregation / State / ip6 ----- +Row 1: +────── +argMinState: 8c:333c::8c:333c:0:0 +argMaxState: ff8b:333c::ff8b:333c:0:0 +----- hash / State / ip4 ----- +Row 1: +────── +anyState: 12535288824949910799 +anyHeavyState: 9327034461443333306 +anyLastState: 12535288824949910799 +----- hash / State / ip6 ----- +Row 1: +────── +anyState: 1383994153676807399 +anyHeavyState: 15555709096566410627 +anyLastState: 1383994153676807399 +----- finalizeAggregation / State / ip4 ----- +Row 1: +────── +any: 59.154.201.255 +anyHeavy: 59.154.201.255 +anyLast: 59.154.201.255 +----- finalizeAggregation / State / ip6 ----- +Row 1: +────── +any: dd8b:333c::dd8b:333c:0:0 +anyHeavy: dd8b:333c::dd8b:333c:0:0 +anyLast: dd8b:333c::dd8b:333c:0:0 diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 new file mode 100644 index 00000000000..133d5287fdb --- /dev/null +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -0,0 +1,123 @@ +{# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} + +{% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,50) order by ip" %} +{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 50) order by ip" %} +{% set ip_generators = {'ip4': ip4_generator, 'ip6': ip6_generator} %} + + +{% set agg_func_list = [ "min", "max", "first_value", "last_value", "topK", "groupArray", "groupUniqArray", "uniq", "uniqExact", "uniqCombined", "uniqCombined64", "uniqHLL12", "uniqTheta" ] %} + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(ip) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(ip) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- hash / IfState / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}IfState(ip, 1) )) AS {{ func }}IfState{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- finalizeAggregation / IfState / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}IfState(ip, 1) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + + +{% set agg_func_list = [ "argMin", "argMax" ] %} + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Arg / hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(ip, ip) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Arg / finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(ip, ip) ) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + +{# let's test functions with not deterministic result against 1 row, to make it deterministic #} +{% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,1) order by ip" %} +{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 1) order by ip" %} + +{% set ip_generators = {'ip4': ip4_generator, 'ip6': ip6_generator} %} + +{% set agg_func_list = [ "any", "anyHeavy", "anyLast" ] %} + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(ip) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(ip) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + From 350becba5dcabe9c2cea48dc5ec6ac07033c04e2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Jun 2023 17:56:35 +0000 Subject: [PATCH 1398/2223] Fixing build. --- src/Interpreters/TableJoin.cpp | 2 +- src/Interpreters/TableJoin.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index c56d1e6039e..cabd0be1aa3 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -762,7 +762,7 @@ bool TableJoin::allowParallelHashJoin() const return true; } -ActionsDAGPtr TableJoin::createJoinedBlockActions(ContextPtr context) +ActionsDAGPtr TableJoin::createJoinedBlockActions(ContextPtr context) const { ASTPtr expression_list = rightKeysList(); auto syntax_result = TreeRewriter(context).analyze(expression_list, columnsFromJoinedTable()); diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 096e58d1292..ba3befab59b 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -217,7 +217,7 @@ public: const SizeLimits & sizeLimits() const { return size_limits; } VolumePtr getGlobalTemporaryVolume() { return tmp_volume; } - ActionsDAGPtr createJoinedBlockActions(ContextPtr context); + ActionsDAGPtr createJoinedBlockActions(ContextPtr context) const; bool isEnabledAlgorithm(JoinAlgorithm val) const { From f34937687e6316b11fb2d61b95f818fd4828a9ce Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Jun 2023 18:00:54 +0000 Subject: [PATCH 1399/2223] enable settings for mutation throttling by default --- src/Storages/MergeTree/MergeTreeSettings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 33aea358078..a3d475b74b2 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -67,8 +67,8 @@ struct Settings; M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ M(CleanDeletedRows, clean_deleted_rows, CleanDeletedRows::Never, "Is the Replicated Merge cleanup has to be done automatically at each merge or manually (possible values are 'Always'/'Never' (default))", 0) \ M(UInt64, replicated_max_mutations_in_one_entry, 10000, "Max number of mutation commands that can be merged together and executed in one MUTATE_PART entry (0 means unlimited)", 0) \ - M(UInt64, number_of_mutations_to_delay, 0, "If table has at least that many unfinished mutations, artificially slow down mutations of table. Disabled if set to 0", 0) \ - M(UInt64, number_of_mutations_to_throw, 0, "If table has at least that many unfinished mutations, throw 'Too many mutations' exception. Disabled if set to 0", 0) \ + M(UInt64, number_of_mutations_to_delay, 500, "If table has at least that many unfinished mutations, artificially slow down mutations of table. Disabled if set to 0", 0) \ + M(UInt64, number_of_mutations_to_throw, 1000, "If table has at least that many unfinished mutations, throw 'Too many mutations' exception. Disabled if set to 0", 0) \ M(UInt64, min_delay_to_mutate_ms, 10, "Min delay of mutating MergeTree table in milliseconds, if there are a lot of unfinished mutations", 0) \ M(UInt64, max_delay_to_mutate_ms, 1000, "Max delay of mutating MergeTree table in milliseconds, if there are a lot of unfinished mutations", 0) \ \ From c96989ca14415707e5dc37958ca36093e8292f46 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 8 Jun 2023 19:37:52 +0000 Subject: [PATCH 1400/2223] no-fasttest because uniqTheta --- .../02751_ip_types_aggregate_functions_states.sql.j2 | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 index 133d5287fdb..708eeab7724 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest because uniqTheta + {# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} {% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,50) order by ip" %} From 172dd4af730e5f592353728eed24535b1a5ea672 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Jun 2023 21:04:00 +0000 Subject: [PATCH 1401/2223] Show correct staleness value in system.query_cache --- src/Storages/System/StorageSystemQueryCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 245f4b7fd26..3dfc5cf298a 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -47,7 +47,7 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr res_columns[0]->insert(key.queryStringFromAst()); /// approximates the original query string res_columns[1]->insert(QueryCache::QueryCacheEntryWeight()(*query_result)); res_columns[2]->insert(key.expires_at < std::chrono::system_clock::now()); - res_columns[3]->insert(!key.is_shared); + res_columns[3]->insert(key.is_shared); res_columns[4]->insert(key.is_compressed); res_columns[5]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); res_columns[6]->insert(key.ast->getTreeHash().first); From a2355673d8349f4a689f46a354c335a7647a5c59 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Jun 2023 21:18:29 +0000 Subject: [PATCH 1402/2223] fix tests --- tests/queries/0_stateless/02125_many_mutations.sh | 2 +- tests/queries/0_stateless/02125_many_mutations_2.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02125_many_mutations.sh b/tests/queries/0_stateless/02125_many_mutations.sh index 7a89e5f7c4f..c3108df5ae3 100755 --- a/tests/queries/0_stateless/02125_many_mutations.sh +++ b/tests/queries/0_stateless/02125_many_mutations.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x" +$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x settings number_of_mutations_to_delay = 0, number_of_mutations_to_throw = 0" $CLICKHOUSE_CLIENT -q "insert into many_mutations values (0, 0), (1, 1)" $CLICKHOUSE_CLIENT -q "system stop merges many_mutations" diff --git a/tests/queries/0_stateless/02125_many_mutations_2.sh b/tests/queries/0_stateless/02125_many_mutations_2.sh index df170a402c6..52866a54974 100755 --- a/tests/queries/0_stateless/02125_many_mutations_2.sh +++ b/tests/queries/0_stateless/02125_many_mutations_2.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x" +$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x settings number_of_mutations_to_delay = 0, number_of_mutations_to_throw = 0" $CLICKHOUSE_CLIENT -q "insert into many_mutations select number, number + 1 from numbers(2000)" $CLICKHOUSE_CLIENT -q "system stop merges many_mutations" From 7578203b46b1657373fc27a5f05e56531581916b Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 9 Jun 2023 03:44:43 +0000 Subject: [PATCH 1403/2223] Changes after review --- docs/en/interfaces/cli.md | 240 +++++++++++++++++--------------- docs/ru/interfaces/cli.md | 235 ++++++++++++++++--------------- programs/client/Client.cpp | 4 +- src/Client/ConnectionString.cpp | 49 ++++--- 4 files changed, 281 insertions(+), 247 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index c36887672c7..2126c538c5d 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -158,116 +158,6 @@ $ clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="number" --query "SELECT {col:Identifier} FROM {db:Identifier}.{tbl:Identifier} LIMIT 10" ``` -## Connection string {#connection_string} - -The connection string for clickhouse-client is presented in URI format: - -```text -clickhouse://[user_info@][hosts_and_ports][/dbname][?query_parameters] -``` - -where user_info is: `user[:password]` -and hosts_and_ports is a list of values: `[host][:port],[host][:port]` Port is not mandatory. -and query_parameters is a list of parameter[=value]: `param_name[=value]¶m_name[=value]...` value may not be required for some of the parameters. Parameter names are case sensitive. - -Allowed query_parameters keys: - -- `secure` or shorthanded `s` - no value. If specified, client will connect to the server over a secure connection (TLS). See `secure` in [command-line-options](#command-line-options) - -These examples illustrate valid connection strings for clickhouse-client: - -```text -clickhouse: -clickhouse://localhost -clickhouse://localhost:9000 -clickhouse://localhost/default -clickhouse://default@localhost -clickhouse://user:password@localhost -clickhouse://user_name@localhost/some_database?secure -clickhouse://host1:9000,host2:5000/some_database -``` - -The host component can either be an IP address or a host name. Put an IPv6 address in square brackets to specify it: - -```text -clickhouse://[2001:db8::1234] -``` - -If user or/and password are not specified, default values will be used. -If host is not specified, the default host will be used (localhost). -If port is not specified, the default port will be used (9000). -If database is not specified, the default database will be used. - -User, password, and database can be specified in the connection string either in `--user`, `--password`, `--database` command line options. - -The connection string must be specified in the first argument of clickhouse-client. The connection string can be combined with other [command-line-options](#command-line-options) except `--host(h)` and `--port`. - -### Multiple hosts {#connection_string_multiple_hosts} - -URI allows multiple hosts to be connected to, and the client will try to connect to those hosts using the order from URI and command line options. The hosts and ports in the URI accept comma-separated lists of values. - -If more than one host is supplied, or if a single host name is translated to more than one address, each host and address will be attempted one at a time until one is successful. The remaining hosts after successful connection in the list are not tried. - -### Percent encoding {#connection_string_uri_percent_encoding} - -Hosts, user name, password, database, and query parameters should be [Percent-Encoded](https://en.wikipedia.org/wiki/URL_encoding) if values contain invalid URI characters. - -### Examples {#connection_string_examples} - -Connect to localhost using port 9000 and execute the query "SELECT 1". - -``` bash -clickhouse-client "clickhouse://localhost:9000" --query "SELECT 1" -``` - -Connect to localhost using port 9000 in interactive, multiline mode. - -``` bash -clickhouse-client "clickhouse://localhost:9000" -m -``` - -Connect to localhost using port 9000 in interactive mode with the user specified in `--user` option. - -``` bash -clickhouse-client "clickhouse://localhost:9000" --user default -``` - -Connect to localhost using port 9000 in interactive mode to `my_database` database specified in the command line option. - -``` bash -clickhouse-client "clickhouse://localhost:9000" --database my_database -``` - -Connect to localhost using port 9000 in interactive mode to `my_database` database specified in the connection string. - -``` bash -clickhouse-client "clickhouse://localhost:9000/my_database" -``` - -Connect to localhost using port 9000 in interactive mode to `my_database` database specified in the connection string and a secure connection using shorthanded 's' URI parameter. - -```bash -clickhouse-client "clickhouse://localhost/my_database?s" -``` - -Connect to default host using default port, default user, and default database. - -``` bash -clickhouse-client "clickhouse:" -``` - -Connect to the default host using the default port, using user user_name and no password. - -``` bash -clickhouse-client "clickhouse://user_name@" -``` - -Connect to localhost using email as the user name. `@` symbol is percent encoded to `%40`. - -``` bash -clickhouse-client "clickhouse://some_user%40some_mail.com@localhost:9000" -``` - ## Configuring {#interfaces_cli_configuration} You can pass parameters to `clickhouse-client` (all parameters have a default value) using: @@ -304,7 +194,135 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--print-profile-events` – Print `ProfileEvents` packets. - `--profile-events-delay-ms` – Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet). -Since version 20.5, `clickhouse-client` has automatic syntax highlighting (always enabled). +Instead of --host, --port, --user and --password options, ClickHouse client also supports connection strings. + + +## Connection string {#connection_string} + +clickhouse-client alternatively supports connecting to clickhouse server using a connection string similar to [MongoDB](https://www.mongodb.com/docs/manual/reference/connection-string/), [PostreSQL](https://www.postgresql.org/docs/current/libpq-connect.html#LIBPQ-CONNSTRING), [MySQL](https://dev.mysql.com/doc/refman/8.0/en/connecting-using-uri-or-key-value-pairs.html#connecting-using-uri). It has the following syntax: + +```text +clickhouse:[//[user_info@][hosts_and_ports]][/database][?query_parameters] +``` + +Where + +- `user_spec` - (optional) is a user and an optional password, +- `hostspec` - (optional) is a list of hosts and optional ports `host[:port] [, host:[port]], ...`, +- `database` - (optional) is the database name, +- `paramspec` - (optional) is a list of key-value pairs `param1=value1[,¶m2=value2], ...`. For some parameters, no value is required. Parameter names and values are case-sensitive. + + + +The host component can either be an IP address or a host name. Put an IPv6 address in square brackets to specify it: + +```text +clickhouse://[2001:db8::1234] +``` + +If user is not specified, `default` user without password will be used. +If host is not specified, the `localhost` will be used (localhost). +If port is not specified, `9000` will be used as port. +If database is not specified, the `default` database will be used. + +If the user name, password or database was specified in the connection string, it cannot be specified using `--user`, `--password` or `--database` (and vice versa). + +The connection string must be specified in the first argument of clickhouse-client. The connection string can be combined with other [command-line-options](#command-line-options) except `--host(h)` and `--port`. + +### Multiple hosts {#connection_string_multiple_hosts} + +URI allows multiple hosts to be connected to. Connection strings can contain multiple hosts. ClickHouse-client will try to connect to these hosts in order (i.e. from left to right). After the connection is established, no attempt to connect to the remaining hosts is made. + +### Allowed query_parameters keys {#connection_string_query_parameters} + +- `secure` or shorthanded `s` - no value. If specified, client will connect to the server over a secure connection (TLS). See `secure` in [command-line-options](#command-line-options) + +### Percent encoding {#connection_string_uri_percent_encoding} + +Non-US ASCII characters in the user name, password, hosts, database or query parameters must be [percent-encoded](https://en.wikipedia.org/wiki/URL_encoding). + +### Examples {#connection_string_examples} + +Connect to localhost using port 9000 and execute the query "SELECT 1". + +``` bash +clickhouse-client clickhouse://localhost:9000 --query "SELECT 1" +``` + +Connect to localhost using user `john` with password `secret`, host `127.0.0.1` and port `9000` + +``` bash +clickhouse-client clickhouse://john:secret@127.0.0.1:9000 +``` + +Connect to localhost using default user, host with IPV6 address `[::1]` and port `9000`. + +``` bash +clickhouse-client clickhouse://[::1]:9000 +``` + +Connect to localhost using default user, host with IPV6 address `[2001:db8:3333:4444:5555:6666:7777:8888]` and port `9000`. + +``` bash +clickhouse-client clickhouse://[2001:db8:3333:4444:5555:6666:7777:8888]:9000 +``` + +Connect to localhost using port 9000 in multiline mode. + +``` bash +clickhouse-client clickhouse://localhost:9000 '-m' +``` + +Connect to localhost using port 9000 with the user `default`. + +``` bash +clickhouse-client clickhouse://default@localhost:9000 --user default + +# equivalent to: +clickhouse-client clickhouse://localhost:9000 --user default +``` + +Connect to localhost using port 9000 to `my_database` database. + +``` bash +clickhouse-client clickhouse://localhost:9000/my_database + +# equivalent to: +clickhouse-client clickhouse://localhost:9000 --database my_database +``` + +Connect to localhost using port 9000 to `my_database` database specified in the connection string and a secure connection using shorthanded 's' URI parameter. + +```bash +clickhouse-client clickhouse://localhost/my_database?s + +# equivalent to: +clickhouse-client clickhouse://localhost/my_database -s +``` + +Connect to default host using default port, default user, and default database. + +``` bash +clickhouse-client clickhouse: +``` + +Connect to the default host using the default port, using user user_name and no password. + +``` bash +clickhouse-client clickhouse://user_name@ +``` + +Connect to localhost using email as the user name. `@` symbol is percent encoded to `%40`. + +``` bash +clickhouse-client clickhouse://some_user%40some_mail.com@localhost:9000 +``` + +Connect to one of provides hosts: `192.168.1.15`, `192.168.1.25`. + +``` bash +clickhouse-client clickhouse://192.168.1.15,192.168.1.25 +``` ### Configuration Files {#configuration_files} diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 801a72e48ec..f86ccb42356 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -110,117 +110,6 @@ $ clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="number" --query "SELECT {col:Identifier} FROM {db:Identifier}.{tbl:Identifier} LIMIT 10" ``` -## Строка подключения {#connection_string} - -Строка подключения для clickhouse-client представлена в формате URI: - -```text -clickhouse://[user_info@][hosts_and_ports][/dbname][?query_parameters] -``` - -где user_info - это: `user[:password]` -hosts_and_ports - это список значений: `[host][:port],[host][:port]`. Port может быть не задан. -query_parameters - это список пар ключ[=значение]: `param_name[=value]¶m_name[=value]...`. Значение может быть пустым. -Имена параметров чувствительны к регистру. - -Допустимые ключи query_parameters: - -- `secure` или сокращенно `s` - без значение. Если параметр указан, то соединение с сервером будет осуществляться по защищенному каналу (TLS). См. `secure` в [command-line-options](#command-line-options). - -Эти примеры иллюстрируют допустимые строки подключения для clickhouse-client: - -```text -clickhouse: -clickhouse://localhost -clickhouse://localhost:9000 -clickhouse://localhost/default -clickhouse://default@localhost -clickhouse://user:password@localhost -clickhouse://имя_пользователя@localhost/some_database?secure -clickhouse://host1:9000,host2:5000/some_database -``` - -Параметр host может быть либо IP-адресом, либо именем хоста. Для указания IPv6-адреса поместите его в квадратные скобки: - -```text -clickhouse://[2001:db8::1234] -``` - -Если пользователь или/и пароль не указаны, будут использоваться значения по умолчанию. -Если host не указан, будет использован хост по умолчанию (localhost). -Если port не указан, будет использоваться порт по умолчанию (9000). -Если база данных не указана, будет использоваться база данных по умолчанию (default). - -Пользователь, пароль и база данных могут быть указаны в строке подключения либо в опциях командной строки `--user`, `--password`, `--database`. - -Строка подключения должна быть указана в первом аргументе clickhouse-client. Строка подключения может комбинироваться с другими [параметрами командной строки] (#command-line-options) кроме `--host (h)` и `--port`. - -### Несколько хостов {#connection_string_multiple_hosts} - -URI позволяет подключаться к нескольким хостам, и клиент будет пытаться подключиться к этим хостам, используя порядок из URI и опций командной строки. Хосты и порты в URI принимают списки значений, разделенные запятыми. - -Если указано более одного хоста или если одно имя хоста транслируется в несколько адресов, Клиент будет будет пытаться подключится к каждому хосту и адресу в порядке в котором они встречаются в URI И опциях клиента, пока не будет установлено соединение. Соединение разрывается, если соединение установлено и аутентификация прошла успешно, остальные хосты в списке игнорируются. - -### Кодирование URI {#connection_string_uri_percent_encoding} - -Хосты, имя пользователя, пароль, имя базы данных, и параметры запроса должны быть [закодированы](https://ru.wikipedia.org/wiki/URL#%D0%9A%D0%BE%D0%B4%D0%B8%D1%80%D0%BE%D0%B2%D0%B0%D0%BD%D0%B8%D0%B5_URL), если значения содержат невалидные символы URI. - -### Примеры {#connection_string_examples} - -Подключиться к localhost через порт 9000 и выполнить запрос "SELECT 1" - -``` bash -clickhouse-client "clickhouse://localhost:9000" --query "SELECT 1" -``` - -Подключиться к localhost через порт 9000 в интерактивном, многострочном режиме. - -``` bash -clickhouse-client "clickhouse://localhost:9000" -m -``` - -Подключиться к localhost через порт 9000 в интерактивном режиме с пользователем default, указанным в опции --user. - -``` bash -clickhouse-client "clickhouse://localhost:9000" --user default -``` - -Подключиться к localhost, используя порт 9000 в интерактивном режиме с базой данных `my_database`, указанной в опции командной строки. - -``` bash -clickhouse-client "clickhouse://localhost:9000" --database my_database -``` - -Подключиться к localhost через порт 9000 в интерактивном режиме с базой данных `my_database`, указанной в строке подключения. - -``` bash -clickhouse-client "clickhouse://localhost:9000/my_database" -``` - -Подключиться к localhost через порт 9000 в интерактивном режиме с базой данных `my_database`, указанной в строке подключения, и безопасным соединением, используя короткий вариант команды URI 's'. - -``` bash -clickhouse-client "clickhouse://localhost/my_database?s" -``` - -Подключиться к хосту по умолчанию с использованием порта по умолчанию, пользователя по умолчанию, и базы данных по умолчанию. - -``` bash -clickhouse-client "clickhouse:" -``` - -Подключиться к хосту по умолчанию через порт по умолчанию, используя имя пользователя user_name без пароля. - -``` bash -clickhouse-client "clickhouse://user_name@" -``` - -Подключиться к localhost, используя электронную почту, как имя пользователя. Символ `@` закодирован как `%40`. - -``` bash -clickhouse-client "clickhouse://some_user%40some_mail.com@localhost:9000" -``` - ## Конфигурирование {#interfaces_cli_configuration} В `clickhouse-client` можно передавать различные параметры (все параметры имеют значения по умолчанию) с помощью: @@ -253,7 +142,129 @@ clickhouse-client "clickhouse://some_user%40some_mail.com@localhost:9000" - `--history_file` - путь к файлу с историей команд. - `--param_` — значение параметра для [запроса с параметрами](#cli-queries-with-parameters). -Начиная с версии 20.5, в `clickhouse-client` есть автоматическая подсветка синтаксиса (включена всегда). +## Строка подключения {#connection_string} + +clickhouse-client также поддерживает подключение к серверу clickhouse с помощью строки подключения, аналогичной [MongoDB](https://www.mongodb.com/docs/manual/reference/connection-string/), [PostreSQL](https://www.postgresql.org/docs/current/libpq-connect.html#LIBPQ-CONNSTRING), [MySQL](https://dev.mysql.com/doc/refman/8.0/en/connecting-using-uri-or-key-value-pairs.html#connecting-using-uri). Она имеет следующий синтаксис: + +```text +clickhouse:[//[user_info@][hosts_and_ports]][/database][?query_parameters] +``` + +Где + +- `user_spec` - (необязательно) - это пользователь и необязательный пароль, +- `hostspec` - (необязательно) - список хостов и необязательных портов. `host[:port] [, host:[port]], ...`, +- `database` - (необязательно) - это имя базы данных, +- `paramspec` - (опционально) список пар ключ-значение `param1=value1[,¶m2=value2], ...`. Для некоторых параметров значение не требуется. Имена и значения параметров чувствительны к регистру. + +Параметр host может быть либо IP-адресом, либо именем хоста. Для указания IPv6-адреса поместите его в квадратные скобки: + +```text +clickhouse://[2001:db8::1234] +``` + +Если user не указан, будут использоваться имя пользователя `default`. +Если host не указан, будет использован хост `localhost`. +Если port не указан, будет использоваться порт `9000`. +Если база данных не указана, будет использоваться база данных `default`. + +Если имя пользователя, пароль или база данных были указаны в строке подключения, их нельзя указать с помощью `--user`, `--password` или `--database` (и наоборот). + +Строка подключения должна быть указана в первом аргументе clickhouse-client. Строка подключения может комбинироваться с другими [параметрами командной строки] (#command-line-options) кроме `--host (h)` и `--port`. + +### Несколько хостов {#connection_string_multiple_hosts} + +URI позволяет подключаться к нескольким хостам. Строки подключения могут содержать несколько хостов. ClickHouse-client будет пытаться подключиться к этим хостам по порядку (т.е. слева направо). После установления соединения попытки подключения к оставшимся хостам не предпринимаются. + +### Допустимые ключи query_parameters {#connection_string_query_parameters} + +- `secure` или сокращенно `s` - без значение. Если параметр указан, то соединение с сервером будет осуществляться по защищенному каналу (TLS). См. `secure` в [command-line-options](#command-line-options). + +### Кодирование URI {#connection_string_uri_percent_encoding} + +Не US ASCII символы в имени пользователя, пароле, хостах, базе данных или параметрах запроса должны быть [закодированы](https://ru.wikipedia.org/wiki/URL#%D0%9A%D0%BE%D0%B4%D0%B8%D1%80%D0%BE%D0%B2%D0%B0%D0%BD%D0%B8%D0%B5_URL). + +### Примеры {#connection_string_examples} + +Подключиться к localhost через порт 9000 и выполнить запрос "SELECT 1" + +``` bash +clickhouse-client clickhouse://localhost:9000 --query "SELECT 1" +``` +Подключиться к localhost, используя пользователя `john` с паролем `secret`, хост `127.0.0.1` и порт `9000` + +``bash +clickhouse-client clickhouse://john:secret@127.0.0.1:9000 +``` + +Подключиться к localhost, используя пользователя по умолчанию, хост с IPV6 адресом `[::1]` и порт `9000`. + +``` bash +clickhouse-client clickhouse://[::1]:9000 +``` + +Подключиться к localhost, используя пользователя по умолчанию, хост с IPV6 адресом `[2001:db8:3333:4444:5555:6666:7777:8888]` и портом `9000`. + +`` bash +clickhouse-client clickhouse://[2001:db8:3333:4444:5555:6666:7777:8888]:9000 +``` + +Подключиться к localhost через порт 9000 многострочном режиме. + +``` bash +clickhouse-client clickhouse://localhost:9000 '-m' +``` + +Подключиться к localhost через порт 9000 с пользователем default. + +``` bash +clickhouse-client clickhouse://default@localhost:9000 + +# Эквивалетно: +clickhouse-client clickhouse://localhost:9000 --user default +``` + +Подключиться к localhost через порт 9000 с базой данных `my_database` + +``` bash +clickhouse-client clickhouse://localhost:9000/my_database + +# Эквивалетно: +clickhouse-client clickhouse://localhost:9000 --database my_database +``` + +Подключиться к localhost через порт 9000 с базой данных `my_database`, указанной в строке подключения, используя безопасным соединением при помощи короткого варианта параметра URI 's'. + +``` bash +clickhouse-client clickhouse://localhost/my_database?s + +# Эквивалетно: +clickhouse-client clickhouse://localhost/my_database -s +``` + +Подключиться к хосту по умолчанию с использованием порта по умолчанию, пользователя по умолчанию, и базы данных по умолчанию. + +``` bash +clickhouse-client clickhouse: +``` + +Подключиться к хосту по умолчанию через порт по умолчанию, используя имя пользователя user_name без пароля. + +``` bash +clickhouse-client clickhouse://user_name@ +``` + +Подключиться к localhost, используя электронную почту, как имя пользователя. Символ `@` закодирован как `%40`. + +``` bash +clickhouse-client clickhouse://some_user%40some_mail.com@localhost:9000 +``` + +Подключится к одному из хостов: `192.168.1.15`, `192.168.1.25`. + +``` bash +clickhouse-client clickhouse://192.168.1.15,192.168.1.25 +``` ### Конфигурационные файлы {#configuration_files} diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e513314387f..1429853e333 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1326,7 +1326,7 @@ void Client::readArguments( else if (arg.starts_with("--host") || arg.starts_with("-h")) { if (has_connection_string) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mixing connection string and --host/--port client arguments is prohibited"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mixing a connection string and --host argument is prohibited"); std::string host_arg; /// --host host @@ -1360,7 +1360,7 @@ void Client::readArguments( else if (arg.starts_with("--port")) { if (has_connection_string) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mixing connection string and --host/--port client arguments is prohibited"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mixing a connection string and --port argument is prohibited"); auto port_arg = String{arg}; /// --port port diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp index 95fec5b52ee..2e475a1f49d 100644 --- a/src/Client/ConnectionString.cpp +++ b/src/Client/ConnectionString.cpp @@ -26,21 +26,20 @@ using namespace std::literals::string_view_literals; constexpr auto CONNECTION_URI_SCHEME = "clickhouse:"sv; -void uriDecode(std::string & uri_encoded_string, bool plus_as_space) +std::string uriDecode(const std::string & uri_encoded_string, bool plus_as_space) { - std::string temp; - Poco::URI::decode(uri_encoded_string, temp, plus_as_space); - std::swap(temp, uri_encoded_string); + std::string decoded_string; + Poco::URI::decode(uri_encoded_string, decoded_string, plus_as_space); + return decoded_string; } void getHostAndPort(const Poco::URI & uri, std::vector> & hosts_and_ports_arguments) { - auto host = uri.getHost(); std::vector host_and_port; + auto host = uri.getHost(); if (!host.empty()) { - uriDecode(host, false); - host_and_port.push_back("--host="s + host); + host_and_port.push_back("--host="s + uriDecode(host, false)); } // Port can be written without host (":9000"). Empty host name equals to default host. @@ -52,7 +51,7 @@ void getHostAndPort(const Poco::URI & uri, std::vector> hosts_and_ports_arguments.push_back(std::move(host_and_port)); } -void getHostAndPort( +void buildConnectionString( Poco::URI & uri, std::vector> & hosts_and_ports_arguments, std::string_view host_and_port, @@ -96,13 +95,13 @@ bool tryParseConnectionString( std::vector & common_arguments, std::vector> & hosts_and_ports_arguments) { + if (connection_string == CONNECTION_URI_SCHEME) + return true; + if (!connection_string.starts_with(CONNECTION_URI_SCHEME)) return false; - if (connection_string.size() == CONNECTION_URI_SCHEME.size()) - return true; - - auto offset = CONNECTION_URI_SCHEME.size(); + size_t offset = CONNECTION_URI_SCHEME.size(); if ((connection_string.substr(offset).starts_with("//"))) offset += 2; @@ -146,7 +145,7 @@ bool tryParseConnectionString( { if (*it == ',') { - getHostAndPort(uri, hosts_and_ports_arguments, {last_host_begin, it}, {hosts_end, connection_string.end()}); + buildConnectionString(uri, hosts_and_ports_arguments, {last_host_begin, it}, {hosts_end, connection_string.end()}); last_host_begin = it + 1; } } @@ -154,11 +153,11 @@ bool tryParseConnectionString( if (uri.empty()) { // URI has no host specified - uri = std::string{connection_string.begin(), connection_string.end()}; + uri = std::string(connection_string); getHostAndPort(uri, hosts_and_ports_arguments); } else - getHostAndPort(uri, hosts_and_ports_arguments, {last_host_begin, hosts_end}, {hosts_end, connection_string.end()}); + buildConnectionString(uri, hosts_and_ports_arguments, {last_host_begin, hosts_end}, {hosts_end, connection_string.end()}); Poco::URI::QueryParameters params = uri.getQueryParameters(); for (const auto & param : params) @@ -166,12 +165,12 @@ bool tryParseConnectionString( if (param.first == "secure" || param.first == "s") { if (!param.second.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "secure URI argument does not require value"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "secure URI query parameter does not require value"); common_arguments.push_back(makeArgument(param.first)); } else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "URI argument {} is unknown", param.first); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "URI query parameter {} is unknown", param.first); } auto user_info = uri.getUserInfo(); @@ -180,21 +179,27 @@ bool tryParseConnectionString( // Poco::URI doesn't decode user name/password by default. // But ClickHouse allows to have users with email user name like: 'john@some_mail.com' // john@some_mail.com should be percent-encoded: 'john%40some_mail.com' - uriDecode(user_info, true); std::string::size_type pos = user_info.find(':'); if (pos != std::string::npos) { common_arguments.push_back("--user"); - common_arguments.push_back(user_info.substr(0, pos)); + common_arguments.push_back(uriDecode(user_info.substr(0, pos), true)); ++pos; // Skip ':' common_arguments.push_back("--password"); - common_arguments.push_back(user_info.substr(pos)); + if (user_info.size() > pos + 1) + common_arguments.push_back(uriDecode(user_info.substr(pos), true)); + else + { + // in case of user_info == 'user:', ':' is specified, but password is empty + // then add password argument "\n" which means: Ask user for a password. + common_arguments.push_back("\n"); + } } else { common_arguments.push_back("--user"); - common_arguments.push_back(user_info); + common_arguments.push_back(uriDecode(user_info, true)); } } @@ -209,7 +214,7 @@ bool tryParseConnectionString( catch (const Poco::URISyntaxException & invalid_uri_exception) { throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, - "Invalid connection string {}: {}", connection_string, invalid_uri_exception.what()); + "Invalid connection string '{}': {}", connection_string, invalid_uri_exception.what()); } return true; From d15b7372942040863e6241f9d299b846c2513495 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 9 Jun 2023 03:54:29 +0000 Subject: [PATCH 1404/2223] Minor renaming --- docs/en/interfaces/cli.md | 2 +- docs/ru/interfaces/cli.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 2126c538c5d..fc24bdcad68 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -199,7 +199,7 @@ Instead of --host, --port, --user and --password options, ClickHouse client also ## Connection string {#connection_string} -clickhouse-client alternatively supports connecting to clickhouse server using a connection string similar to [MongoDB](https://www.mongodb.com/docs/manual/reference/connection-string/), [PostreSQL](https://www.postgresql.org/docs/current/libpq-connect.html#LIBPQ-CONNSTRING), [MySQL](https://dev.mysql.com/doc/refman/8.0/en/connecting-using-uri-or-key-value-pairs.html#connecting-using-uri). It has the following syntax: +clickhouse-client alternatively supports connecting to clickhouse server using a connection string similar to [MongoDB](https://www.mongodb.com/docs/manual/reference/connection-string/), [PostgreSQL](https://www.postgresql.org/docs/current/libpq-connect.html#LIBPQ-CONNSTRING), [MySQL](https://dev.mysql.com/doc/refman/8.0/en/connecting-using-uri-or-key-value-pairs.html#connecting-using-uri). It has the following syntax: ```text clickhouse:[//[user_info@][hosts_and_ports]][/database][?query_parameters] diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index f86ccb42356..ee29b122afb 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -144,7 +144,7 @@ $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="numbe ## Строка подключения {#connection_string} -clickhouse-client также поддерживает подключение к серверу clickhouse с помощью строки подключения, аналогичной [MongoDB](https://www.mongodb.com/docs/manual/reference/connection-string/), [PostreSQL](https://www.postgresql.org/docs/current/libpq-connect.html#LIBPQ-CONNSTRING), [MySQL](https://dev.mysql.com/doc/refman/8.0/en/connecting-using-uri-or-key-value-pairs.html#connecting-using-uri). Она имеет следующий синтаксис: +clickhouse-client также поддерживает подключение к серверу clickhouse с помощью строки подключения, аналогичной [MongoDB](https://www.mongodb.com/docs/manual/reference/connection-string/), [PostgreSQL](https://www.postgresql.org/docs/current/libpq-connect.html#LIBPQ-CONNSTRING), [MySQL](https://dev.mysql.com/doc/refman/8.0/en/connecting-using-uri-or-key-value-pairs.html#connecting-using-uri). Она имеет следующий синтаксис: ```text clickhouse:[//[user_info@][hosts_and_ports]][/database][?query_parameters] From 094d661701cc83d84ebcf4bbaffeff47e0f7c547 Mon Sep 17 00:00:00 2001 From: YalalovSM <39567223+YalalovSM@users.noreply.github.com> Date: Fri, 9 Jun 2023 09:26:00 +0500 Subject: [PATCH 1405/2223] Update projection.md Document using keywords IF EXISTS/IF NOT EXISTS with projections --- docs/en/sql-reference/statements/alter/projection.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 030e9352a00..b7399442d41 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -142,11 +142,11 @@ The following operations with [projections](/docs/en/engines/table-engines/merge ## ADD PROJECTION -`ALTER TABLE [db].name ADD PROJECTION name ( SELECT [GROUP BY] [ORDER BY] )` - Adds projection description to tables metadata. +`ALTER TABLE [db].name ADD PROJECTION [IF NOT EXISTS] name ( SELECT [GROUP BY] [ORDER BY] )` - Adds projection description to tables metadata. ## DROP PROJECTION -`ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). +`ALTER TABLE [db].name DROP PROJECTION [IF EXISTS] name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). ## MATERIALIZE PROJECTION @@ -154,7 +154,7 @@ The following operations with [projections](/docs/en/engines/table-engines/merge ## CLEAR PROJECTION -`ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). +`ALTER TABLE [db.]table CLEAR PROJECTION [IF EXISTS] name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only change metadata or remove files. From 95c0b942c141493f8997fb807b4ff72c34b8292b Mon Sep 17 00:00:00 2001 From: YalalovSM <39567223+YalalovSM@users.noreply.github.com> Date: Fri, 9 Jun 2023 10:37:20 +0500 Subject: [PATCH 1406/2223] Update projection.md --- docs/ru/sql-reference/statements/alter/projection.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index 63b068611ec..33e52b93add 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -8,13 +8,13 @@ sidebar_label: PROJECTION Доступны следующие операции с [проекциями](../../../engines/table-engines/mergetree-family/mergetree.md#projections): -- `ALTER TABLE [db].name ADD PROJECTION name ( SELECT [GROUP BY] [ORDER BY] )` — добавляет описание проекции в метаданные. +- `ALTER TABLE [db].name ADD PROJECTION [IF NOT EXISTS] name ( SELECT [GROUP BY] [ORDER BY] )` — добавляет описание проекции в метаданные. -- `ALTER TABLE [db].name DROP PROJECTION name` — удаляет описание проекции из метаданных и удаляет файлы проекции с диска. +- `ALTER TABLE [db].name DROP PROJECTION [IF EXISTS] name` — удаляет описание проекции из метаданных и удаляет файлы проекции с диска. - `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` — перестраивает проекцию в указанной партиции. Реализовано как [мутация](../../../sql-reference/statements/alter/index.md#mutations). -- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` — удаляет файлы проекции с диска без удаления описания. +- `ALTER TABLE [db.]table CLEAR PROJECTION [IF EXISTS] name IN PARTITION partition_name` — удаляет файлы проекции с диска без удаления описания. Команды `ADD`, `DROP` и `CLEAR` — легковесны, поскольку они только меняют метаданные или удаляют файлы. @@ -22,4 +22,4 @@ sidebar_label: PROJECTION :::note Манипуляции с проекциями поддерживаются только для таблиц с движком [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) (включая [replicated](../../../engines/table-engines/mergetree-family/replication.md) варианты). - ::: \ No newline at end of file + ::: From d0938e95e68551c51f54407abe30bbd35534bf2e Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 9 Jun 2023 06:40:57 +0000 Subject: [PATCH 1407/2223] prohibited to use --connection --- programs/client/Client.cpp | 9 +++------ src/Client/ConnectionString.cpp | 19 +++++++++++++++++++ src/Client/ConnectionString.h | 5 +++++ 3 files changed, 27 insertions(+), 6 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 1429853e333..a49447dff69 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1267,6 +1267,9 @@ void Client::readArguments( { std::string_view arg = argv[arg_num]; + if (has_connection_string) + validateConnectionStringClientOption(arg); + if (arg == "--external") { in_external_group = true; @@ -1325,9 +1328,6 @@ void Client::readArguments( } else if (arg.starts_with("--host") || arg.starts_with("-h")) { - if (has_connection_string) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mixing a connection string and --host argument is prohibited"); - std::string host_arg; /// --host host if (arg == "--host" || arg == "-h") @@ -1359,9 +1359,6 @@ void Client::readArguments( } else if (arg.starts_with("--port")) { - if (has_connection_string) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mixing a connection string and --port argument is prohibited"); - auto port_arg = String{arg}; /// --port port if (arg == "--port") diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp index 2e475a1f49d..b9658772e2e 100644 --- a/src/Client/ConnectionString.cpp +++ b/src/Client/ConnectionString.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -26,6 +27,15 @@ using namespace std::literals::string_view_literals; constexpr auto CONNECTION_URI_SCHEME = "clickhouse:"sv; +const std::unordered_map PROHIBITED_CLIENT_OPTIONS = +{ + /// Client option, client option long name + {"-h", "--host"}, + {"--host", "--host"}, + {"--port", "--port"}, + {"--connection", "--connection"}, +}; + std::string uriDecode(const std::string & uri_encoded_string, bool plus_as_space) { std::string decoded_string; @@ -220,4 +230,13 @@ bool tryParseConnectionString( return true; } +void validateConnectionStringClientOption(std::string_view command_line_option) +{ + const auto prohibited_option_iter = PROHIBITED_CLIENT_OPTIONS.find(command_line_option); + if (prohibited_option_iter != PROHIBITED_CLIENT_OPTIONS.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Mixing a connection string and {} option is prohibited", + prohibited_option_iter->second); +} + } diff --git a/src/Client/ConnectionString.h b/src/Client/ConnectionString.h index aafb1139b00..ce72de9edf6 100644 --- a/src/Client/ConnectionString.h +++ b/src/Client/ConnectionString.h @@ -19,4 +19,9 @@ bool tryParseConnectionString( std::string_view connection_string, std::vector & common_arguments, std::vector> & hosts_and_ports_arguments); + +// throws DB::Exception with BAD_ARGUMENTS if the given command line argument is allowed +// to be used with the connection string +void validateConnectionStringClientOption(std::string_view command_line_option); + } From b8fc25ab239fbf8c82f589b175e386ceed737c26 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 9 Jun 2023 06:51:34 +0000 Subject: [PATCH 1408/2223] minor update --- src/Client/ConnectionString.cpp | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp index b9658772e2e..e1f39369b2a 100644 --- a/src/Client/ConnectionString.cpp +++ b/src/Client/ConnectionString.cpp @@ -27,13 +27,12 @@ using namespace std::literals::string_view_literals; constexpr auto CONNECTION_URI_SCHEME = "clickhouse:"sv; -const std::unordered_map PROHIBITED_CLIENT_OPTIONS = -{ - /// Client option, client option long name - {"-h", "--host"}, - {"--host", "--host"}, - {"--port", "--port"}, - {"--connection", "--connection"}, +const std::unordered_map PROHIBITED_CLIENT_OPTIONS = { + /// Client option, client option long name + {"-h", "--host"}, + {"--host", "--host"}, + {"--port", "--port"}, + {"--connection", "--connection"}, }; std::string uriDecode(const std::string & uri_encoded_string, bool plus_as_space) @@ -234,9 +233,8 @@ void validateConnectionStringClientOption(std::string_view command_line_option) { const auto prohibited_option_iter = PROHIBITED_CLIENT_OPTIONS.find(command_line_option); if (prohibited_option_iter != PROHIBITED_CLIENT_OPTIONS.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Mixing a connection string and {} option is prohibited", - prohibited_option_iter->second); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Mixing a connection string and {} option is prohibited", prohibited_option_iter->second); } } From 96d9b88a201a3ee1b79874e60becc0fc62a10ba3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Jun 2023 10:48:13 +0200 Subject: [PATCH 1409/2223] Fix build --- contrib/azure-cmake/CMakeLists.txt | 2 +- src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 3 --- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 1e2a4c97824..9c361db47ca 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD) +if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD OR ARCH_PPC64LE) message(STATUS "Not using Azure blob storage") return() endif() diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index b3cda54e752..5a34adb384a 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -8,10 +8,7 @@ #include #include #include - -#if USE_AZURE_BLOB_STORAGE #include -#endif namespace Poco { From 5c76a8882ef644ca924c5653cdad14e7cfa94270 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 9 Jun 2023 09:12:07 +0000 Subject: [PATCH 1410/2223] Fix docs by pull request comments --- .../functions/type-conversion-functions.md | 177 +++++++++++----- .../functions/type-conversion-functions.md | 199 ++++++++++++++---- 2 files changed, 288 insertions(+), 88 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index a021fed195d..c634a3da27e 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -33,7 +33,7 @@ SELECT toTypeName(toNullable('') AS val) AS source_type, toTypeName(toString(val)) AS to_type_result_type, toTypeName(CAST(val, 'String')) AS cast_result_type - + ┌─source_type──────┬─to_type_result_type─┬─cast_result_type─┐ │ Nullable(String) │ Nullable(String) │ String │ └──────────────────┴─────────────────────┴──────────────────┘ @@ -203,7 +203,7 @@ Result: ## toDate -Converts the argument to [Date](/docs/en/sql-reference/data-types/date.md) data type. +Converts the argument to [Date](/docs/en/sql-reference/data-types/date.md) data type. If the argument is [DateTime](/docs/en/sql-reference/data-types/datetime.md) or [DateTime64](/docs/en/sql-reference/data-types/datetime64.md), it truncates it and leaves the date component of the DateTime: @@ -232,7 +232,7 @@ SELECT │ 2022-12-30 │ Date │ └────────────┴──────────────────────────────────┘ -1 row in set. Elapsed: 0.001 sec. +1 row in set. Elapsed: 0.001 sec. ``` ```sql @@ -314,14 +314,52 @@ SELECT └─────────────────────┴───────────────┴─────────────┴─────────────────────┘ ``` + ## toDateOrZero +The same as [toDate](#todate) but returns lower boundery of [Date](/docs/en/sql-reference/data-types/date.md) if an invalid argument is received. Only [String](/docs/en/sql-reference/data-types/string.md) argument is supported. + +**Example** + +Query: + +``` sql +SELECT toDateOrZero('2022-12-30'), toDateOrZero(''); +``` + +Result: + +```response +┌─toDateOrZero('2022-12-30')─┬─toDateOrZero('')─┐ +│ 2022-12-30 │ 1970-01-01 │ +└────────────────────────────┴──────────────────┘ +``` + + ## toDateOrNull +The same as [toDate](#todate) but returns `NULL` if an invalid argument is received. Only [String](/docs/en/sql-reference/data-types/string.md) argument is supported. + +**Example** + +Query: + +``` sql +SELECT toDateOrNull('2022-12-30'), toDateOrNull(''); +``` + +Result: + +```response +┌─toDateOrNull('2022-12-30')─┬─toDateOrNull('')─┐ +│ 2022-12-30 │ ᴺᵁᴸᴸ │ +└────────────────────────────┴──────────────────┘ +``` + + ## toDateOrDefault -Converts an input value to [Date](/docs/en/sql-reference/data-types/date.md) data type. -If unsuccessful, returns the lower border value supported by [Date](/docs/en/sql-reference/data-types/date.md). The default value can be specified as a second argument. -Similar to [toDate](#todate). + +Like [toDate](#todate) but if unsuccessful, returns a default value which is either the second argument (if specified), or otherwise the lower boundery of [Date](/docs/en/sql-reference/data-types/date.md). **Syntax** @@ -329,62 +367,37 @@ Similar to [toDate](#todate). toDateOrDefault(expr [, default_value]) ``` -**Arguments** - -- `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [Int](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md). -- `default_value` — The default value. [Date](/docs/en/sql-reference/data-types/date.md) - -If `expr` is a number and looks like a UNIX timestamp (is greater than 65535), it is interpreted as a DateTime, then truncated to Date in the current timezone. If `expr` is a number and it is smaller than 65536, it is interpreted as the number of days since 1970-01-01. - -**Returned value** - -- A calendar date. [Date](/docs/en/sql-reference/data-types/date.md) - **Example** Query: ``` sql -SELECT - toDateOrDefault('2021-01-01', '2023-01-01'::Date), - toDateOrDefault('xx2021-01-01', '2023-01-01'::Date); +SELECT toDateOrDefault('2022-12-30'), toDateOrDefault('', '2023-01-01'::Date); ``` Result: ```response -┌─toDateOrDefault('2021-01-01', CAST('2023-01-01', 'Date'))─┬─toDateOrDefault('xx2021-01-01', CAST('2023-01-01', 'Date'))─┐ -│ 2021-01-01 │ 2023-01-01 │ -└───────────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────┘ +┌─toDateOrDefault('2022-12-30')─┬─toDateOrDefault('', CAST('2023-01-01', 'Date'))─┐ +│ 2022-12-30 │ 2023-01-01 │ +└───────────────────────────────┴─────────────────────────────────────────────────┘ ``` -**See Also** -- [toDate](#todate) -- [toDate32OrDefault](#todate32ordefault) - ## toDateTime -## toDateTimeOrZero - -## toDateTimeOrNull - -## toDateTimeOrDefault -Converts an input value to [DateTime](/docs/en/sql-reference/data-types/datetime.md) data type. -If unsuccessful, returns the lower border value supported by [DateTime](/docs/en/sql-reference/data-types/datetime.md). The default value can be specified as a third argument. -Similar to [toDateTime](#todatetime). +Converts an input value to [DateTime](/docs/en/sql-reference/data-types/datetime.md). **Syntax** ``` sql -toDateTimeOrDefault(expr, [, time_zone [, default_value]]) +toDateTime(expr[, time_zone ]) ``` **Arguments** -- `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [Int](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md). +- `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [Int](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md). - `time_zone` — Time zone. [String](/docs/en/sql-reference/data-types/string.md). -- `default_value` — The default value. [DateTime](/docs/en/sql-reference/data-types/datetime.md) If `expr` is a number, it is interpreted as the number of seconds since the beginning of the Unix Epoch (as Unix timestamp). @@ -397,21 +410,86 @@ If `expr` is a number, it is interpreted as the number of seconds since the begi Query: ``` sql -SELECT - toDateTimeOrDefault('2021-01-01', 'UTC', '2023-01-01'::DateTime('UTC')), - toDateTimeOrDefault('xx2021-01-01', 'UTC', '2023-01-01'::DateTime('UTC')); +SELECT toDateTime('2022-12-30 13:44:17'), toDateTime(1685457500, 'UTC'); ``` Result: ```response -┌─toDateTimeOrDefault('2021-01-01', 'UTC', CAST('2023-01-01', 'DateTime(\'UTC\')'))─┬─toDateTimeOrDefault('xx2021-01-01', 'UTC', CAST('2023-01-01', 'DateTime(\'UTC\')'))─┐ -│ 2021-01-01 00:00:00 │ 2023-01-01 00:00:00 │ -└───────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────┘ +┌─toDateTime('2022-12-30 13:44:17')─┬─toDateTime(1685457500, 'UTC')─┐ +│ 2022-12-30 13:44:17 │ 2023-05-30 14:38:20 │ +└───────────────────────────────────┴───────────────────────────────┘ +``` + + +## toDateTimeOrZero + +The same as [toDateTime](#todate) but returns lower boundery of [Date](/docs/en/sql-reference/data-types/date.md) if an invalid argument is received. Only [String](/docs/en/sql-reference/data-types/string.md) argument is supported. + +**Example** + +Query: + +``` sql +SELECT toDateTimeOrZero('2022-12-30 13:44:17'), toDateTimeOrZero(''); +``` + +Result: + +```response +┌─toDateTimeOrZero('2022-12-30 13:44:17')─┬─toDateTimeOrZero('')─┐ +│ 2022-12-30 13:44:17 │ 1970-01-01 00:00:00 │ +└─────────────────────────────────────────┴──────────────────────┘ +``` + + +## toDateTimeOrNull + +The same as [toDateTime](#todatetime) but returns `NULL` if an invalid argument is received. Only [String](/docs/en/sql-reference/data-types/string.md) argument is supported. + +**Example** + +Query: + +``` sql +SELECT toDateTimeOrNull('2022-12-30 13:44:17'), toDateTimeOrNull(''); +``` + +Result: + +```response +┌─toDateTimeOrNull('2022-12-30 13:44:17')─┬─toDateTimeOrNull('')─┐ +│ 2022-12-30 13:44:17 │ ᴺᵁᴸᴸ │ +└─────────────────────────────────────────┴──────────────────────┘ +``` + + +## toDateTimeOrDefault + +Like [toDateTime](#todatetime) but if unsuccessful, returns a default value which is either the third argument (if specified), or otherwise the lower boundery of [DateTime](/docs/en/sql-reference/data-types/datetime.md). + +**Syntax** + +``` sql +toDateTimeOrDefault(expr [, time_zone [, default_value]]) +``` + +**Example** + +Query: + +``` sql +SELECT toDateTimeOrDefault('2022-12-30 13:44:17'), toDateTimeOrDefault('', 'UTC', '2023-01-01'::DateTime('UTC')); +``` + +Result: + +```response +┌─toDateTimeOrDefault('2022-12-30 13:44:17')─┬─toDateTimeOrDefault('', 'UTC', CAST('2023-01-01', 'DateTime(\'UTC\')'))─┐ +│ 2022-12-30 13:44:17 │ 2023-01-01 00:00:00 │ +└────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────┘ ``` -**See Also** -- [toDateTime](#todatetime) ## toDate32 @@ -604,6 +682,11 @@ SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Asia/Istanbul') AS value, toTypeN └─────────────────────────┴─────────────────────────────────────────────────────────────────────┘ ``` +## toDateTime64OrZero + +## toDateTime64OrNull + +## toDateTime64OrDefault ## toDecimal(32\|64\|128\|256) @@ -1332,7 +1415,7 @@ Returns DateTime values parsed from input string according to a MySQL style form **Supported format specifiers** All format specifiers listed in [formatDateTime](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime) except: -- %Q: Quarter (1-4) +- %Q: Quarter (1-4) **Example** diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 67d1732d34e..d43b5415114 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -165,28 +165,17 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); ## toDate {#todate} -Cиноним: `DATE`. - -## toDateOrZero {#todateorzero} - -## toDateOrNull {#todateornull} - -## toDateOrDefault {#todateordefault} - -Конвертирует аргумент в значение [Date](/docs/ru/sql-reference/data-types/date.md) data type. -Если получен недопустимый аргумент, то возвращает значение по умолчанию (нижняя граница [Date](/docs/ru/sql-reference/data-types/date.md). Значение по умолчанию может быть указано вторым аргументом. -Похожа на [toDate](#todate). +Конвертирует аргумент в значение [Date](/docs/ru/sql-reference/data-types/date.md). **Синтаксис** ``` sql -toDateOrDefault(expr [, default_value]) +toDate(expr) ``` **Аргументы** -- `expr` — Значение для преобразования. [String](/docs/ru/sql-reference/data-types/string.md), [Int](/docs/ru/sql-reference/data-types/int-uint.md), [Date](/docs/ru/sql-reference/data-types/date.md) или [DateTime](/docs/ru/sql-reference/data-types/datetime.md). -- `default_value` — Значение по умолчанию. [Date](/docs/ru/sql-reference/data-types/date.md) +- `expr` — Значение для преобразования. [String](/docs/ru/sql-reference/data-types/string.md), [Int](/docs/ru/sql-reference/data-types/int-uint.md), [Date](/docs/ru/sql-reference/data-types/date.md) или [DateTime](/docs/ru/sql-reference/data-types/datetime.md). Если `expr` является числом выглядит как UNIX timestamp (больше чем 65535), оно интерпретируется как DateTime, затем обрезается до Date учитывавая текущую часовой пояс. Если `expr` является числом и меньше чем 65536, оно интерпретируется как количество дней с 1970-01-01. @@ -199,46 +188,101 @@ toDateOrDefault(expr [, default_value]) Запрос: ``` sql -SELECT - toDateOrDefault('2021-01-01', '2023-01-01'::Date), - toDateOrDefault('xx2021-01-01', '2023-01-01'::Date); +SELECT toDate('2022-12-30'), toDate(1685457500); ``` Результат: ```response -┌─toDateOrDefault('2021-01-01', CAST('2023-01-01', 'Date'))─┬─toDateOrDefault('xx2021-01-01', CAST('2023-01-01', 'Date'))─┐ -│ 2021-01-01 │ 2023-01-01 │ -└───────────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────┘ +┌─toDate('2022-12-30')─┬─toDate(1685457500)─┐ +│ 2022-12-30 │ 2023-05-30 │ +└──────────────────────┴────────────────────┘ ``` -**Смотрите также** -- [toDate](#todate) -- [toDate32OrDefault](#todate32ordefault) -## toDateTime {#todatetime} +## toDateOrZero {#todateorzero} -## toDateTimeOrZero {#todatetimeorzero} +Как [toDate](#todate), но в случае неудачи возвращает нижнюю границу [Date](/docs/ru/sql-reference/data-types/date.md)). Поддерживается только аргумент типа [String](/docs/ru/sql-reference/data-types/string.md). -## toDateTimeOrNull {#todatetimeornull} +**Пример** -## toDateTimeOrDefault {#todatetimeordefault} +Запрос: -Конвертирует аргумент в значение [DateTime](/docs/ru/sql-reference/data-types/datetime.md). -Если получен недопустимый аргумент, то возвращает значение по умолчанию (нижняя граница [DateTime](/docs/ru/sql-reference/data-types/datetime.md)). Значение по умолчанию может быть указано третьим аргументом. -Похожа на [toDateTime](#todatetime). +``` sql +SELECT toDateOrZero('2022-12-30'), toDateOrZero(''); +``` + +Результат: + +```response +┌─toDateOrZero('2022-12-30')─┬─toDateOrZero('')─┐ +│ 2022-12-30 │ 1970-01-01 │ +└────────────────────────────┴──────────────────┘ +``` + + +## toDateOrNull {#todateornull} + +Как [toDate](#todate), но в случае неудачи возвращает `NULL`. Поддерживается только аргумент типа [String](/docs/ru/sql-reference/data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT toDateOrNull('2022-12-30'), toDateOrNull(''); +``` + +Результат: + +```response +┌─toDateOrNull('2022-12-30')─┬─toDateOrNull('')─┐ +│ 2022-12-30 │ ᴺᵁᴸᴸ │ +└────────────────────────────┴──────────────────┘ +``` + + +## toDateOrDefault {#todateordefault} + +Как [toDate](#todate), но в случае неудачи возвращает значение по умолчанию (или второй аргумент (если указан), или нижняя граница [Date](/docs/ru/sql-reference/data-types/date.md)). **Синтаксис** ``` sql -toDateTimeOrDefault(expr, [, time_zone [, default_value]]) +toDateOrDefault(expr [, default_value]) +``` + +**Пример** + +Запрос: + +``` sql +SELECT toDateOrDefault('2022-12-30'), toDateOrDefault('', '2023-01-01'::Date); +``` + +Результат: + +```response +┌─toDateOrDefault('2022-12-30')─┬─toDateOrDefault('', CAST('2023-01-01', 'Date'))─┐ +│ 2022-12-30 │ 2023-01-01 │ +└───────────────────────────────┴─────────────────────────────────────────────────┘ +``` + + +## toDateTime {#todatetime} + +Конвертирует аргумент в значение [DateTime](/docs/ru/sql-reference/data-types/datetime.md). + +**Синтаксис** + +``` sql +toDateTime(expr[, time_zone ]) ``` **Аргументы** -- `expr` — Значение для преобразования. [String](/docs/ru/sql-reference/data-types/string.md), [Int](/docs/ru/sql-reference/data-types/int-uint.md), [Date](/docs/ru/sql-reference/data-types/date.md) или [DateTime](/docs/ru/sql-reference/data-types/datetime.md). +- `expr` — Значение для преобразования. [String](/docs/ru/sql-reference/data-types/string.md), [Int](/docs/ru/sql-reference/data-types/int-uint.md), [Date](/docs/ru/sql-reference/data-types/date.md) или [DateTime](/docs/ru/sql-reference/data-types/datetime.md). - `time_zone` — Часовой пояс. [String](/docs/ru/sql-reference/data-types/string.md). -- `default_value` — Значение по умолчанию. [DateTime](/docs/ru/sql-reference/data-types/datetime.md) Если `expr` является числом, оно интерпретируется как количество секунд от начала unix эпохи. @@ -251,21 +295,86 @@ toDateTimeOrDefault(expr, [, time_zone [, default_value]]) Запрос: ``` sql -SELECT - toDateTimeOrDefault('2021-01-01', 'UTC', '2023-01-01'::DateTime('UTC')), - toDateTimeOrDefault('xx2021-01-01', 'UTC', '2023-01-01'::DateTime('UTC')); +SELECT toDateTime('2022-12-30 13:44:17'), toDateTime(1685457500, 'UTC'); ``` Результат: ```response -┌─toDateTimeOrDefault('2021-01-01', 'UTC', CAST('2023-01-01', 'DateTime(\'UTC\')'))─┬─toDateTimeOrDefault('xx2021-01-01', 'UTC', CAST('2023-01-01', 'DateTime(\'UTC\')'))─┐ -│ 2021-01-01 00:00:00 │ 2023-01-01 00:00:00 │ -└───────────────────────────────────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────┘ +┌─toDateTime('2022-12-30 13:44:17')─┬─toDateTime(1685457500, 'UTC')─┐ +│ 2022-12-30 13:44:17 │ 2023-05-30 14:38:20 │ +└───────────────────────────────────┴───────────────────────────────┘ +``` + + +## toDateTimeOrZero {#todatetimeorzero} + +Как [toDateTime](#todatetime), но в случае неудачи возвращает нижнюю границу [DateTime](/docs/ru/sql-reference/data-types/datetime.md)). Поддерживается только аргумент типа [String](/docs/ru/sql-reference/data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT toDateTimeOrZero('2022-12-30 13:44:17'), toDateTimeOrZero(''); +``` + +Результат: + +```response +┌─toDateTimeOrZero('2022-12-30 13:44:17')─┬─toDateTimeOrZero('')─┐ +│ 2022-12-30 13:44:17 │ 1970-01-01 00:00:00 │ +└─────────────────────────────────────────┴──────────────────────┘ +``` + + +## toDateTimeOrNull {#todatetimeornull} + +Как [toDateTime](#todatetime), но в случае неудачи возвращает `NULL`. Поддерживается только аргумент типа [String](/docs/ru/sql-reference/data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT toDateTimeOrNull('2022-12-30 13:44:17'), toDateTimeOrNull(''); +``` + +Result: + +```response +┌─toDateTimeOrNull('2022-12-30 13:44:17')─┬─toDateTimeOrNull('')─┐ +│ 2022-12-30 13:44:17 │ ᴺᵁᴸᴸ │ +└─────────────────────────────────────────┴──────────────────────┘ +``` + + +## toDateTimeOrDefault {#todatetimeordefault} + +Как [toDateTime](#todatetime), но в случае неудачи возвращает значение по умолчанию (или третий аргумент (если указан), или нижняя граница [DateTime](/docs/ru/sql-reference/data-types/datetime.md)). + +**Синтаксис** + +``` sql +toDateTimeOrDefault(expr, [, time_zone [, default_value]]) +``` + +**Пример** + +Запрос: + +``` sql +SELECT toDateTimeOrDefault('2022-12-30 13:44:17'), toDateTimeOrDefault('', 'UTC', '2023-01-01'::DateTime('UTC')); +``` + +Результат: + +```response +┌─toDateTimeOrDefault('2022-12-30 13:44:17')─┬─toDateTimeOrDefault('', 'UTC', CAST('2023-01-01', 'DateTime(\'UTC\')'))─┐ +│ 2022-12-30 13:44:17 │ 2023-01-01 00:00:00 │ +└────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────┘ ``` -**Смотрите также** -- [toDateTime](#todatetime) ## toDate32 {#todate32} @@ -387,6 +496,14 @@ SELECT └─────────────────────────────────────────────────────────┴───────────────────────────────────────────────────────────┘ ``` +## toDateTime64 + +## toDateTime64OrZero + +## toDateTime64OrNull + +## toDateTime64OrDefault + ## toDecimal(32\|64\|128\|256) {#todecimal3264128} Преобразует `value` к типу данных [Decimal](../../sql-reference/functions/type-conversion-functions.md) с точностью `S`. `value` может быть числом или строкой. Параметр `S` (scale) задаёт число десятичных знаков. From 7ac4349d969819b59555a47d42e92ae070c2faa7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Jun 2023 11:21:10 +0200 Subject: [PATCH 1411/2223] Hacking azure function --- .../table-functions/azure_blob_storage.md | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 docs/en/sql-reference/table-functions/azure_blob_storage.md diff --git a/docs/en/sql-reference/table-functions/azure_blob_storage.md b/docs/en/sql-reference/table-functions/azure_blob_storage.md new file mode 100644 index 00000000000..f86307b3b85 --- /dev/null +++ b/docs/en/sql-reference/table-functions/azure_blob_storage.md @@ -0,0 +1,11 @@ +--- +slug: /en/sql-reference/table-functions/azure_blob_storage +sidebar_position: 45 +sidebar_label: azure_blob_storage +keywords: [azure blob storage] +--- + +# azure\_blob\_storage Table Function + +Provides a table-like interface to select/insert files in [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs). This table function is similar to the [s3 function](../../sql-reference/table-functions/s3.md). + From ab8365630b3ee50120e67664ca4ecbab1afcc4c3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Jun 2023 09:19:21 +0000 Subject: [PATCH 1412/2223] Reject logs containing unknown operation --- contrib/NuRaft | 2 +- src/Coordination/Changelog.cpp | 3 +- src/Coordination/Changelog.h | 2 +- src/Coordination/KeeperServer.cpp | 39 +++++++++++++++++++++---- src/Coordination/KeeperStateMachine.cpp | 5 ++-- 5 files changed, 39 insertions(+), 12 deletions(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index b56784be1ae..f43d10dbc97 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit b56784be1aec568fb72aff47f281097c017623cb +Subproject commit f43d10dbc977a63f11dfb3afdd010fcf7ad89950 diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 894fd93cfa7..c0dfbc2cbc3 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -479,7 +480,7 @@ public: continue; /// Create log entry for read data - auto log_entry = nuraft::cs_new(record.header.term, record.blob, record.header.value_type); + auto log_entry = nuraft::cs_new(record.header.term, record.blob, static_cast(record.header.value_type)); if (result.first_read_index == 0) result.first_read_index = record.header.index; diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 56b0475ba8b..3c09370182d 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -39,7 +39,7 @@ struct ChangelogRecordHeader ChangelogVersion version = CURRENT_CHANGELOG_VERSION; uint64_t index = 0; /// entry log number uint64_t term = 0; - nuraft::log_val_type value_type{}; + int32_t value_type{}; uint64_t blob_size = 0; }; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 45db9e85fa5..6e47412cd3a 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -607,12 +608,30 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ } } + const auto follower_preappend = [&](const auto & entry) + { + if (entry->get_val_type() != nuraft::app_log) + return nuraft::cb_func::ReturnCode::Ok; + + try + { + state_machine->parseRequest(entry->get_buf(), /*final=*/false); + } + catch (...) + { + tryLogCurrentException(log, "Failed to parse request from log entry"); + throw; + } + return nuraft::cb_func::ReturnCode::Ok; + + }; + if (initialized_flag) { switch (type) { // This event is called before a single log is appended to the entry on the leader node - case nuraft::cb_func::PreAppendLog: + case nuraft::cb_func::PreAppendLogLeader: { // we are relying on the fact that request are being processed under a mutex // and not a RW lock @@ -665,7 +684,12 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ if (request_for_session->digest->version != KeeperStorage::NO_DIGEST) writeIntBinary(request_for_session->digest->value, write_buf); - break; + return nuraft::cb_func::ReturnCode::Ok; + } + case nuraft::cb_func::PreAppendLogFollower: + { + const auto & entry = *static_cast(param->ctx); + return follower_preappend(entry); } case nuraft::cb_func::AppendLogFailed: { @@ -678,13 +702,11 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ auto & entry_buf = entry->get_buf(); auto request_for_session = state_machine->parseRequest(entry_buf, true); state_machine->rollbackRequest(*request_for_session, true); - break; + return nuraft::cb_func::ReturnCode::Ok; } default: - break; + return nuraft::cb_func::ReturnCode::Ok; } - - return nuraft::cb_func::ReturnCode::Ok; } size_t last_commited = state_machine->last_commit_index(); @@ -737,6 +759,11 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ initial_batch_committed = true; return nuraft::cb_func::ReturnCode::Ok; } + case nuraft::cb_func::PreAppendLogFollower: + { + const auto & entry = *static_cast(param->ctx); + return follower_preappend(entry); + } default: /// ignore other events return nuraft::cb_func::ReturnCode::Ok; } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 65abee44050..7d251ad48b9 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -272,9 +272,8 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); - rollbackRequestNoLock(request_for_session, true); - throw; + tryLogCurrentException(__PRETTY_FUNCTION__, "Failed to preprocess stored log, aborting to avoid inconsistent state"); + std::abort(); } if (keeper_context->digest_enabled && request_for_session.digest) From de70e322cf93d5f10a01bbc7d7aa8f4798755214 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 9 Jun 2023 10:29:44 +0000 Subject: [PATCH 1413/2223] Fix by pull request comments --- src/Functions/DateTimeTransforms.h | 35 ++++++++--------- src/Functions/FunctionsConversion.h | 2 +- .../01556_accurate_cast_or_null.reference | 2 + .../01556_accurate_cast_or_null.sql | 2 + .../0_stateless/01601_accurate_cast.sql | 38 +++++++++---------- 5 files changed, 40 insertions(+), 39 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 09b0d71daf8..1d3ec1bd368 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1436,7 +1436,7 @@ struct Transformer { template static void vector(const FromTypeVector & vec_from, ToTypeVector & vec_to, const DateLUTImpl & time_zone, const Transform & transform, - ColumnUInt8::Container * vec_null_map_to [[maybe_unused]]) + [[maybe_unused]] ColumnUInt8::Container * vec_null_map_to) { using ValueType = typename ToTypeVector::value_type; size_t size = vec_from.size(); @@ -1444,29 +1444,26 @@ struct Transformer for (size_t i = 0; i < size; ++i) { - if constexpr (std::is_same_v - || std::is_same_v) + if constexpr (std::is_same_v || std::is_same_v) { - bool check_range_result = true; - - if constexpr (std::is_same_v || std::is_same_v) + if constexpr (std::is_same_v + || std::is_same_v) { - check_range_result = vec_from[i] >= 0 && vec_from[i] <= 0xFFFFFFFFL; - } + bool is_valid_input = vec_from[i] >= 0 && vec_from[i] <= 0xFFFFFFFFL; - if (!check_range_result) - { - if constexpr (std::is_same_v) + if (!is_valid_input) { - vec_to[i] = 0; - if (vec_null_map_to) + if constexpr (std::is_same_v) + { + vec_to[i] = 0; (*vec_null_map_to)[i] = true; - continue; - } - else - { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Value in column {} cannot be safely converted into type {}", - TypeName, TypeName); + continue; + } + else + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Value in column {} cannot be safely converted into type {}", + TypeName, TypeName); + } } } } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 3a8ddcc9094..ea8efada21d 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2885,7 +2885,7 @@ private: if constexpr (IsDataTypeNumber && (std::is_same_v || std::is_same_v)) { - if (wrapper_cast_type == CastType::accurate) + if (wrapper_cast_type == CastType::accurate) { result_column = ConvertImpl::template execute( arguments, result_type, input_rows_count); diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference index 21faa830636..3bff125068a 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference @@ -35,6 +35,8 @@ \N 2023-05-30 14:38:20 1970-01-01 00:00:19 +1970-01-01 19:26:40 +\N \N \N 2023-05-30 diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql index 3f57358576e..3266198d930 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql @@ -41,8 +41,10 @@ SELECT accurateCastOrNull(5000000000, 'DateTime'); SELECT accurateCastOrNull('1xxx', 'DateTime'); select toString(accurateCastOrNull('2023-05-30 14:38:20', 'DateTime'), timezone()); SELECT toString(accurateCastOrNull(19, 'DateTime'), 'UTC'); +SELECT toString(accurateCastOrNull(70000, 'DateTime'), 'UTC'); SELECT accurateCastOrNull(-1, 'Date'); +SELECT accurateCastOrNull(5000000000, 'Date'); SELECT accurateCastOrNull('1xxx', 'Date'); SELECT accurateCastOrNull('2023-05-30', 'Date'); SELECT accurateCastOrNull(19, 'Date'); diff --git a/tests/queries/0_stateless/01601_accurate_cast.sql b/tests/queries/0_stateless/01601_accurate_cast.sql index 5555129f0ad..2108e42df05 100644 --- a/tests/queries/0_stateless/01601_accurate_cast.sql +++ b/tests/queries/0_stateless/01601_accurate_cast.sql @@ -1,36 +1,36 @@ -SELECT accurateCast(-1, 'UInt8'); -- { serverError 70 } +SELECT accurateCast(-1, 'UInt8'); -- { serverError CANNOT_CONVERT_TYPE } SELECT accurateCast(5, 'UInt8'); -SELECT accurateCast(257, 'UInt8'); -- { serverError 70 } -SELECT accurateCast(-1, 'UInt16'); -- { serverError 70 } +SELECT accurateCast(257, 'UInt8'); -- { serverError CANNOT_CONVERT_TYPE } +SELECT accurateCast(-1, 'UInt16'); -- { serverError CANNOT_CONVERT_TYPE } SELECT accurateCast(5, 'UInt16'); -SELECT accurateCast(65536, 'UInt16'); -- { serverError 70 } -SELECT accurateCast(-1, 'UInt32'); -- { serverError 70 } +SELECT accurateCast(65536, 'UInt16'); -- { serverError CANNOT_CONVERT_TYPE } +SELECT accurateCast(-1, 'UInt32'); -- { serverError CANNOT_CONVERT_TYPE } SELECT accurateCast(5, 'UInt32'); -SELECT accurateCast(4294967296, 'UInt32'); -- { serverError 70 } -SELECT accurateCast(-1, 'UInt64'); -- { serverError 70 } +SELECT accurateCast(4294967296, 'UInt32'); -- { serverError CANNOT_CONVERT_TYPE } +SELECT accurateCast(-1, 'UInt64'); -- { serverError CANNOT_CONVERT_TYPE } SELECT accurateCast(5, 'UInt64'); -SELECT accurateCast(-1, 'UInt256'); -- { serverError 70 } +SELECT accurateCast(-1, 'UInt256'); -- { serverError CANNOT_CONVERT_TYPE } SELECT accurateCast(5, 'UInt256'); -SELECT accurateCast(-129, 'Int8'); -- { serverError 70 } +SELECT accurateCast(-129, 'Int8'); -- { serverError CANNOT_CONVERT_TYPE } SELECT accurateCast(5, 'Int8'); -SELECT accurateCast(128, 'Int8'); -- { serverError 70 } +SELECT accurateCast(128, 'Int8'); -- { serverError CANNOT_CONVERT_TYPE } -SELECT accurateCast(10, 'Decimal32(9)'); -- { serverError 407 } +SELECT accurateCast(10, 'Decimal32(9)'); -- { serverError DECIMAL_OVERFLOW } SELECT accurateCast(1, 'Decimal32(9)'); -SELECT accurateCast(-10, 'Decimal32(9)'); -- { serverError 407 } +SELECT accurateCast(-10, 'Decimal32(9)'); -- { serverError DECIMAL_OVERFLOW } -SELECT accurateCast('123', 'FixedString(2)'); -- { serverError 131 } +SELECT accurateCast('123', 'FixedString(2)'); -- { serverError TOO_LARGE_STRING_SIZE } SELECT accurateCast('12', 'FixedString(2)'); -SELECT accurateCast(-1, 'DateTime'); -- { serverError 70 } -SELECT accurateCast(0xFFFFFFFF + 1, 'DateTime'); -- { serverError 70 } -SELECT accurateCast('1xxx', 'DateTime'); -- { serverError 41 } +SELECT accurateCast(-1, 'DateTime'); -- { serverError CANNOT_CONVERT_TYPE } +SELECT accurateCast(0xFFFFFFFF + 1, 'DateTime'); -- { serverError CANNOT_CONVERT_TYPE } +SELECT accurateCast('1xxx', 'DateTime'); -- { serverError CANNOT_PARSE_DATETIME } SELECT accurateCast('2023-05-30 14:38:20', 'DateTime'); SELECT toString(accurateCast(19, 'DateTime'), 'UTC'); -SELECT accurateCast(-1, 'Date'); -- { serverError 70 } -SELECT accurateCast(0xFFFFFFFF + 1, 'Date'); -- { serverError 70 } -SELECT accurateCast('1xxx', 'Date'); -- { serverError 38 } +SELECT accurateCast(-1, 'Date'); -- { serverError CANNOT_CONVERT_TYPE } +SELECT accurateCast(0xFFFFFFFF + 1, 'Date'); -- { serverError CANNOT_CONVERT_TYPE } +SELECT accurateCast('1xxx', 'Date'); -- { serverError CANNOT_PARSE_DATE } SELECT accurateCast('2023-05-30', 'Date'); SELECT accurateCast(19, 'Date'); From 7a02a70ad4239e920c6e23b9bc2bcc0a5c5db58b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 9 Jun 2023 11:11:49 +0000 Subject: [PATCH 1414/2223] Add value to exceptions text --- src/Functions/DateTimeTransforms.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 1d3ec1bd368..019e0c42cde 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1461,8 +1461,8 @@ struct Transformer } else { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Value in column {} cannot be safely converted into type {}", - TypeName, TypeName); + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Value {} cannot be safely converted into type {}", + vec_from[i], TypeName); } } } From 16a6190446e7cfc35bfc13f1077c6f32ba430184 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Jun 2023 13:25:50 +0200 Subject: [PATCH 1415/2223] More strict build Aarch64 --- contrib/azure-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 9c361db47ca..968882dbfcd 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD OR ARCH_PPC64LE) +if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD OR NOT ARCH_AMD64) message(STATUS "Not using Azure blob storage") return() endif() From 430a19bb80e497d8c4f5d02af604d78a3d8016c5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Jun 2023 13:26:53 +0200 Subject: [PATCH 1416/2223] Better --- contrib/azure-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 968882dbfcd..887122e7653 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD OR NOT ARCH_AMD64) +if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD OR (NOT ARCH_AMD64)) message(STATUS "Not using Azure blob storage") return() endif() From f437d5d8b51b69347eecb79a0af878bfd2707d9d Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 9 Jun 2023 11:27:27 +0000 Subject: [PATCH 1417/2223] Fix toDateTimeOrZero description --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index c634a3da27e..28db7e6e677 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -424,7 +424,7 @@ Result: ## toDateTimeOrZero -The same as [toDateTime](#todate) but returns lower boundery of [Date](/docs/en/sql-reference/data-types/date.md) if an invalid argument is received. Only [String](/docs/en/sql-reference/data-types/string.md) argument is supported. +The same as [toDateTime](#todatetime) but returns lower boundery of [DateTime](/docs/en/sql-reference/data-types/datetime.md) if an invalid argument is received. Only [String](/docs/en/sql-reference/data-types/string.md) argument is supported. **Example** From 8f9c74debb002b74180ad534e828714b49bba44a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 9 Jun 2023 14:31:49 +0300 Subject: [PATCH 1418/2223] Support OPTIMIZE for temporary tables (#50710) * Update InterpreterOptimizeQuery.cpp * Update 02525_different_engines_in_temporary_tables.sql * Update MergeTreeData.cpp --- src/Interpreters/InterpreterOptimizeQuery.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../02525_different_engines_in_temporary_tables.sql | 4 +++- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index a4507391c4e..ae456e8b31d 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -34,7 +34,7 @@ BlockIO InterpreterOptimizeQuery::execute() getContext()->checkAccess(getRequiredAccess()); - auto table_id = getContext()->resolveStorageID(ast, Context::ResolveOrdinary); + auto table_id = getContext()->resolveStorageID(ast); StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); checkStorageSupportsTransactionsIfNeeded(table, getContext()); auto metadata_snapshot = table->getInMemoryMetadataPtr(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2f6870f8b41..9cca471fddb 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4789,7 +4789,7 @@ void MergeTreeData::checkAlterPartitionIsPossible( if (partition_ast && partition_ast->all) { if (command.type != PartitionCommand::DROP_PARTITION) - throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DETACH PARTITION ALL currently"); + throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH PARTITION ALL currently"); } else getPartitionIDFromQuery(command.partition, getContext()); diff --git a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql index 7ebc05dfece..58e9ecab30c 100644 --- a/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql +++ b/tests/queries/0_stateless/02525_different_engines_in_temporary_tables.sql @@ -7,7 +7,9 @@ CREATE TEMPORARY TABLE table_merge_tree_02525 ENGINE = MergeTree ORDER BY id PRIMARY KEY id; -INSERT INTO table_merge_tree_02525 VALUES (1, 'a'), (2, 'b'), (3, 'c'); +INSERT INTO table_merge_tree_02525 VALUES (1, 'a'), (2, 'b'); +INSERT INTO table_merge_tree_02525 VALUES (3, 'c'); +OPTIMIZE TABLE table_merge_tree_02525 FINAL; SELECT * FROM table_merge_tree_02525; -- Check that temporary table with MergeTree is not sent to remote servers -- The query with remote() should not fail From ad74189bc2ed4039b0cf129928141e13f6db435b Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 9 Jun 2023 19:32:45 +0800 Subject: [PATCH 1419/2223] Fix Log family table return wrong rows count after truncate (#50585) * Fix Log family table return wrong rows count after truncate * fix test * update test * update test --- src/Storages/StorageLog.cpp | 2 ++ src/Storages/StorageStripeLog.cpp | 2 ++ .../02771_log_faminy_truncate_count.reference | 2 ++ .../02771_log_faminy_truncate_count.sql | 26 +++++++++++++++++++ 4 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/02771_log_faminy_truncate_count.reference create mode 100644 tests/queries/0_stateless/02771_log_faminy_truncate_count.sql diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index ac68de43332..d8065b8bb3c 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -775,6 +775,8 @@ void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr marks_loaded = true; num_marks_saved = 0; + total_rows = 0; + total_bytes = 0; getContext()->dropMMappedFileCache(); } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 5c704d877d1..d8bbd523cbf 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -424,6 +424,8 @@ void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont indices_loaded = true; num_indices_saved = 0; + total_rows = 0; + total_bytes = 0; getContext()->dropMMappedFileCache(); } diff --git a/tests/queries/0_stateless/02771_log_faminy_truncate_count.reference b/tests/queries/0_stateless/02771_log_faminy_truncate_count.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/02771_log_faminy_truncate_count.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/02771_log_faminy_truncate_count.sql b/tests/queries/0_stateless/02771_log_faminy_truncate_count.sql new file mode 100644 index 00000000000..3fb22837f5b --- /dev/null +++ b/tests/queries/0_stateless/02771_log_faminy_truncate_count.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS test_log; +CREATE TABLE test_log +( + `crypto_name` String, + `trade_date` Date +) +ENGINE = Log; + +INSERT INTO test_log (crypto_name, trade_date) VALUES ('abc', '2021-01-01'), ('def', '2022-02-02'); + +TRUNCATE TABLE test_log; +SELECT count() FROM test_log; + +DROP TABLE IF EXISTS test_log; +CREATE TABLE test_log +( + `crypto_name` String, + `trade_date` Date +) +ENGINE = StripeLog; + +INSERT INTO test_log (crypto_name, trade_date) VALUES ('abc', '2021-01-01'), ('def', '2022-02-02'); + +TRUNCATE TABLE test_log; +SELECT count() FROM test_log; +DROP TABLE test_log; From bc2af59278bdd0addeda2bdaedff411117f23f04 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Jun 2023 13:38:30 +0200 Subject: [PATCH 1420/2223] Update autogenerated version to 23.6.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 ++-- .../StorageSystemContributors.generated.cpp | 49 +++++++++++++++++++ 2 files changed, 54 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 462529fbc13..015037b2de6 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54474) +SET(VERSION_REVISION 54475) SET(VERSION_MAJOR 23) -SET(VERSION_MINOR 5) +SET(VERSION_MINOR 6) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 3920eb987f7ed837ada5de8907284adf123f0583) -SET(VERSION_DESCRIBE v23.5.1.1-testing) -SET(VERSION_STRING 23.5.1.1) +SET(VERSION_GITHASH 2fec796e73efda10a538a03af3205ce8ffa1b2de) +SET(VERSION_DESCRIBE v23.6.1.1-testing) +SET(VERSION_STRING 23.6.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 0f307650c9c..f83ee3197fe 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -83,6 +83,8 @@ const char * auto_contributors[] { "Alexey Boykov", "Alexey Dushechkin", "Alexey Elymanov", + "Alexey Gerasimchuck", + "Alexey Gerasimchuk", "Alexey Gusev", "Alexey Ilyukhov", "Alexey Ivanov", @@ -143,6 +145,7 @@ const char * auto_contributors[] { "Anmol Arora", "Anna", "Anna Shakhova", + "AnneClickHouse", "Anselmo D. Adams", "Anthony N. Simon", "Anton Ivashkin", @@ -305,6 +308,7 @@ const char * auto_contributors[] { "Dr. Strange Looker", "Duc Canh Le", "DuckSoft", + "Duyet Le", "Egor O'Sten", "Egor Savin", "Eirik", @@ -321,6 +325,7 @@ const char * auto_contributors[] { "Eric", "Eric Daniel", "Eric Thomas", + "Eridanus", "Erixonich", "Ernest Poletaev", "Eugene Galkin", @@ -347,6 +352,7 @@ const char * auto_contributors[] { "Federico Ceratto", "Federico Rodriguez", "FeehanG", + "Feng Kaiyu", "FgoDt", "Filatenkov Artur", "Filipe Caixeta", @@ -444,6 +450,7 @@ const char * auto_contributors[] { "Ivan Milov", "Ivan Remen", "Ivan Starkov", + "Ivan Takarlikov", "Ivan Zhukov", "Jachen Duschletta", "Jack Song", @@ -477,6 +484,7 @@ const char * auto_contributors[] { "John Skopis", "Jonatas Freitas", "Jonathan-Ackerman", + "Jordi", "Jordi Villar", "Joris Giovannangeli", "Jose", @@ -489,6 +497,7 @@ const char * auto_contributors[] { "Jus", "Justin Hilliard", "Justin de Guzman", + "János Benjamin Antal", "Kang Liu", "Karl Pietrzak", "Keiji Yoshida", @@ -523,6 +532,7 @@ const char * auto_contributors[] { "Kruglov Pavel", "Krzysztof Góralski", "Kseniia Sumarokova", + "Kuba Kaflik", "Kunal Gurnani", "Kuz Le", "Ky Li", @@ -540,6 +550,7 @@ const char * auto_contributors[] { "Leopold Schabel", "Lev Borodin", "Lewinma", + "Li Shuai", "Li Yin", "Liu Cong", "LiuCong", @@ -549,12 +560,14 @@ const char * auto_contributors[] { "Lopatin Konstantin", "Lorenzo Mangani", "Loud_Scream", + "Lucas Chang", "Lucid Dreams", "Luck-Chang", "Luis Bosque", "Lv Feng", "Léo Ercolanelli", "M0r64n", + "M1eyu2018", "MEX7", "MaceWindu", "MagiaGroz", @@ -564,8 +577,10 @@ const char * auto_contributors[] { "Maksim Fedotov", "Maksim Kita", "Maksym Sobolyev", + "Mal Curtis", "Mallik Hassan", "Malte", + "Manas Alekar", "Manuel de la Peña", "Marat IDRISOV", "Marcelo Rodriguez", @@ -650,7 +665,9 @@ const char * auto_contributors[] { "Milad Arabi", "Mingliang Pan", "Misko Lee", + "Misz606", "Mohamad Fadhil", + "Mohammad Arab Anvari", "Mohammad Hossein Sekhavat", "Mojtaba Yaghoobzadeh", "Mostafa Dahab", @@ -764,6 +781,7 @@ const char * auto_contributors[] { "Rajkumar Varada", "Ramazan Polat", "Rami Dridi", + "Raqbit", "Ravengg", "Raúl Marín", "Realist007", @@ -791,6 +809,7 @@ const char * auto_contributors[] { "Roman Peshkurov", "Roman Tsisyk", "Roman Vasin", + "Roman Vlasenko", "Roman Zhukov", "Roy Bellingan", "Ruslan", @@ -825,6 +844,8 @@ const char * auto_contributors[] { "Sergey Demurin", "Sergey Elantsev", "Sergey Fedorov", + "Sergey Kazmin", + "Sergey Kislov", "Sergey Kononenko", "Sergey Lazarev", "Sergey Magidovich", @@ -840,6 +861,7 @@ const char * auto_contributors[] { "Sergio Tulentsev", "SevaCode", "Seyed Mehrshad Hosseini", + "Shane Andrade", "Sherry Wang", "Shoh Jahon", "Sichen Zhao", @@ -857,6 +879,8 @@ const char * auto_contributors[] { "SmitaRKulkarni", "Snow", "Sofia Antipushina", + "Sorck", + "Stanislav Dobrovolschii", "Stanislav Pavlovichev", "Stas Kelvich", "Stas Pavlovichev", @@ -922,6 +946,8 @@ const char * auto_contributors[] { "Vadym Chekan", "Vage Ogannisian", "Val", + "Val Doroshchuk", + "Valentin Alexeev", "Valera Ryaboshapko", "Varinara", "Vasily Kozhukhovskiy", @@ -935,8 +961,10 @@ const char * auto_contributors[] { "Veselkov Konstantin", "Viachaslau Boben", "Victor", + "Victor Krasnov", "Victor Tarnavsky", "Viktor Taranenko", + "Vincent", "Vincent Bernat", "Vitalii S", "Vitaliy", @@ -1018,12 +1046,14 @@ const char * auto_contributors[] { "Yuriy Korzhenevskiy", "Yury Karpovich", "Yury Stankevich", + "Yusuke Tanaka", "ZhiYong Wang", "Zhichang Yu", "Zhichun Wu", "Zhiguo Zhou", "Zhipeng", "Zijie Lu", + "Ziy1-Tan", "Zoran Pandovski", "[데이터플랫폼팀] 이호선", "a.palagashvili", @@ -1039,6 +1069,7 @@ const char * auto_contributors[] { "akazz", "akonyaev", "akuzm", + "alekar", "alekseik1", "alekseygolub", "alesapin", @@ -1072,6 +1103,7 @@ const char * auto_contributors[] { "asiana21", "atereh", "attack204", + "auxten", "avasiliev", "avogar", "avoiderboi", @@ -1094,6 +1126,7 @@ const char * auto_contributors[] { "caipengxiang", "candiduslynx", "canenoneko", + "cangyin", "caspian", "cekc", "centos7", @@ -1131,6 +1164,7 @@ const char * auto_contributors[] { "damozhaeva", "dankondr", "daoready", + "darkkeks", "dasmfm", "davydovska", "decaseal", @@ -1189,6 +1223,7 @@ const char * auto_contributors[] { "franklee", "fredchenbj", "freedomDR", + "frinkr", "fuqi", "fuwhu", "fuzhe1989", @@ -1236,6 +1271,7 @@ const char * auto_contributors[] { "ikopylov", "imgbot[bot]", "ip", + "ismailakpolat", "it1804", "ivan-klass", "ivan-kush", @@ -1255,6 +1291,7 @@ const char * auto_contributors[] { "jianmei zhang", "jinjunzh", "jkuklis", + "johanngan", "jthmath", "jun won", "jus1096", @@ -1280,6 +1317,7 @@ const char * auto_contributors[] { "kst-morozov", "l", "l1tsolaiki", + "laimuxi", "lalex", "lanfz", "larryluogit", @@ -1298,8 +1336,10 @@ const char * auto_contributors[] { "liang.huang", "liangqian", "libenwang", + "libin", "lichengxiang", "liding1992", + "lihaibo42", "linceyou", "lincion", "lingo-xp", @@ -1341,6 +1381,7 @@ const char * auto_contributors[] { "mastertheknife", "mateng0915", "mateng915", + "mauidude", "maxim", "maxim-babenko", "maxkuzn", @@ -1352,6 +1393,7 @@ const char * auto_contributors[] { "meo", "meoww-bot", "mergify[bot]", + "merlllle", "metahys", "mf5137", "mfridental", @@ -1398,6 +1440,7 @@ const char * auto_contributors[] { "olevino", "olevino999", "olgarev", + "ongkong", "orantius", "p0ny", "palasonicq", @@ -1473,6 +1516,7 @@ const char * auto_contributors[] { "spongedc", "spume", "spyros87", + "sslouis", "stan", "stavrolia", "stepenhu", @@ -1518,6 +1562,7 @@ const char * auto_contributors[] { "vicdashkov", "vicgao", "vinity", + "vitac", "vitstn", "vivarum", "vladimir golovchenko", @@ -1527,6 +1572,7 @@ const char * auto_contributors[] { "vzakaznikov", "wangchao", "wangdh15", + "wangxiaobo", "weeds085490", "whysage", "wineternity", @@ -1537,6 +1583,7 @@ const char * auto_contributors[] { "xieyichen", "xinhuitian", "xlwh", + "xmy", "yakkomajuri", "yakov-olkhovskiy", "yandd", @@ -1589,6 +1636,7 @@ const char * auto_contributors[] { "zvvr", "zxc111", "zxealous", + "zy-kkk", "zzsmdfj", "Šimon Podlipský", "Александр", @@ -1605,6 +1653,7 @@ const char * auto_contributors[] { "Сундуков Алексей", "万康", "何李夫", + "你不要过来啊", "凌涛", "刘浩林", "刘陶峰", From 2d220bd8411607521fb6a2767b568352a2cae127 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Jun 2023 11:38:50 +0000 Subject: [PATCH 1421/2223] Fix NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index f43d10dbc97..8f267da1a91 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit f43d10dbc977a63f11dfb3afdd010fcf7ad89950 +Subproject commit 8f267da1a91310bd152af755b0178cfd38c646c7 From 32d781c058321b3c2ee0b21b17a2cbb87a9a3e23 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 9 Jun 2023 14:51:26 +0300 Subject: [PATCH 1422/2223] Align the documentation with the new feature --- docs/en/sql-reference/functions/type-conversion-functions.md | 4 +++- docs/ru/sql-reference/functions/type-conversion-functions.md | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index f6c99b168ac..dad3cfb4cc5 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1343,6 +1343,7 @@ parseDateTimeBestEffort(time_string [, time_zone]) - A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `DD/MM/YYYY`, `DD-MM-YY` etc. - A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case `YYYY-MM` are substituted as `2000-01`. - A string that includes the date and time along with time zone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. +- A string that includes the date and time in a [syslog](https://datatracker.ietf.org/doc/html/rfc3164) datetime format. For example, `Jun 9 14:20:32`. For all of the formats with separator the function parses months names expressed by their full name or by the first three letters of a month name. Examples: `24/DEC/18`, `24-Dec-18`, `01-September-2018`. @@ -1428,10 +1429,11 @@ Result: **See Also** -- [RFC 1123](https://tools.ietf.org/html/rfc1123) +- [RFC 1123](https://datatracker.ietf.org/doc/html/rfc1123) - [toDate](#todate) - [toDateTime](#todatetime) - [ISO 8601 announcement by @xkcd](https://xkcd.com/1179/) +- [RFC 3164](https://datatracker.ietf.org/doc/html/rfc3164) ## parseDateTimeBestEffortUS diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 298b7bbc93e..03e3adfbdca 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -1022,6 +1022,7 @@ parseDateTimeBestEffort(time_string[, time_zone]) - Строка с датой, но без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `DD/MM/YYYY`, `DD-MM-YY` и т.д. - Строка с временем, и с днём: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` принимается равным `2000-01`. - Строка, содержащая дату и время вместе с информацией о часовом поясе: `YYYY-MM-DD hh:mm:ss ±h:mm`, и т.д. Например, `2020-12-12 17:36:00 -5:00`. +- Строка, содержащая дату и время в формате [syslog](https://datatracker.ietf.org/doc/html/rfc3164). Например, `Jun 9 14:20:32`. Для всех форматов с разделителями функция распознаёт названия месяцев, выраженных в виде полного англоязычного имени месяца или в виде первых трёх символов имени месяца. Примеры: `24/DEC/18`, `24-Dec-18`, `01-September-2018`. @@ -1108,9 +1109,10 @@ SELECT parseDateTimeBestEffort('10 20:19'); **Смотрите также** - [Информация о формате ISO 8601 от @xkcd](https://xkcd.com/1179/) -- [RFC 1123](https://tools.ietf.org/html/rfc1123) +- [RFC 1123](https://datatracker.ietf.org/doc/html/rfc1123) - [toDate](#todate) - [toDateTime](#todatetime) +- [RFC 3164](https://datatracker.ietf.org/doc/html/rfc3164) ## parseDateTimeBestEffortUS {#parsedatetimebesteffortUS} From 61a20468f7429866c3125bd7c55627072de7ea5d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 9 Jun 2023 11:53:29 +0000 Subject: [PATCH 1423/2223] Update version_date.tsv and changelogs after v23.5.1.3174-stable --- SECURITY.md | 3 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.5.1.3174-stable.md | 599 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 605 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v23.5.1.3174-stable.md diff --git a/SECURITY.md b/SECURITY.md index 75c1a9d7d6a..1864eb6e9e5 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -13,9 +13,10 @@ The following versions of ClickHouse server are currently being supported with s | Version | Supported | |:-|:-| +| 23.5 | ✔️ | | 23.4 | ✔️ | | 23.3 | ✔️ | -| 23.2 | ✔️ | +| 23.2 | ❌ | | 23.1 | ❌ | | 22.12 | ❌ | | 22.11 | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 73da4515ff4..7190ef4d649 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.4.2.11" +ARG VERSION="23.5.1.3174" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 1a5d2071f6b..ca966b16a2d 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.4.2.11" +ARG VERSION="23.5.1.3174" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 8792d419a16..c82ac592120 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.4.2.11" +ARG VERSION="23.5.1.3174" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.5.1.3174-stable.md b/docs/changelogs/v23.5.1.3174-stable.md new file mode 100644 index 00000000000..01e5425de71 --- /dev/null +++ b/docs/changelogs/v23.5.1.3174-stable.md @@ -0,0 +1,599 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.5.1.3174-stable (2fec796e73e) FIXME as compared to v23.4.1.1943-stable (3920eb987f7) + +#### Backward Incompatible Change +* Make local object storage work consistently with s3 object storage, fix problem with append (closes [#48465](https://github.com/ClickHouse/ClickHouse/issues/48465)), make it configurable as independent storage. The change is backward incompatible because cache on top of local object storage is not incompatible to previous versions. [#48791](https://github.com/ClickHouse/ClickHouse/pull/48791) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Date_trunc function to always return datetime type. [#48851](https://github.com/ClickHouse/ClickHouse/pull/48851) ([Shane Andrade](https://github.com/mauidude)). +* Remove the experimental feature "in-memory data parts". The data format is still supported, but the settings are no-op, and compact or wide parts will be used instead. This closes [#45409](https://github.com/ClickHouse/ClickHouse/issues/45409). [#49429](https://github.com/ClickHouse/ClickHouse/pull/49429) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Changed default values of settings parallelize_output_from_storages and input_format_parquet_preserve_order. This allows ClickHouse to reorder rows when reading from files (e.g. CSV or Parquet), greatly improving performance in many cases. To restore the old behavior of preserving order, use `parallelize_output_from_storages = 0`, `input_format_parquet_preserve_order = 1`. [#49479](https://github.com/ClickHouse/ClickHouse/pull/49479) ([Michael Kolupaev](https://github.com/al13n321)). +* Make projections production-ready. Add the `optimize_use_projections` setting to control whether the projections will be selected for SELECT queries. The setting `allow_experimental_projection_optimization` is obsolete and does nothing. [#49719](https://github.com/ClickHouse/ClickHouse/pull/49719) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Mark joinGet() as non deterministic (so as dictGet). [#49843](https://github.com/ClickHouse/ClickHouse/pull/49843) ([Azat Khuzhin](https://github.com/azat)). +* Revert "`groupArray` returns cannot be nullable" (due to binary compatibility breakage for `groupArray`/`groupArrayLast`/`groupArraySample` over `Nullable` types, which likely will lead to `TOO_LARGE_ARRAY_SIZE` or `CANNOT_READ_ALL_DATA`). [#49971](https://github.com/ClickHouse/ClickHouse/pull/49971) ([Azat Khuzhin](https://github.com/azat)). + +#### New Feature +* Password type in queries like `CREATE USER u IDENTIFIED BY 'p'` will be automatically set according to the setting `default_password_type` in the `config.xml` on the server. Closes [#42915](https://github.com/ClickHouse/ClickHouse/issues/42915). [#44674](https://github.com/ClickHouse/ClickHouse/pull/44674) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add bcrypt password authentication type. Closes [#34599](https://github.com/ClickHouse/ClickHouse/issues/34599). [#44905](https://github.com/ClickHouse/ClickHouse/pull/44905) ([Nikolay Degterinsky](https://github.com/evillique)). +* Added `system.zookeeper_connection` table that shows information about ZooKeeper connections. [#45245](https://github.com/ClickHouse/ClickHouse/pull/45245) ([mateng915](https://github.com/mateng0915)). +* Add urlCluster table function. Refactor all *Cluster table functions to reduce code duplication. Make schema inference work for all possible *Cluster function signatures and for named collections. Closes [#38499](https://github.com/ClickHouse/ClickHouse/issues/38499). [#45427](https://github.com/ClickHouse/ClickHouse/pull/45427) ([attack204](https://github.com/attack204)). +* Extend `first_value` and `last_value` to accept null. [#46467](https://github.com/ClickHouse/ClickHouse/pull/46467) ([lgbo](https://github.com/lgbo-ustc)). +* Add server and format settings `display_secrets_in_show_and_select` for displaying secrets of tables, databases, table functions, and dictionaries. Add privilege `displaySecretsInShowAndSelect` controlling which users can view secrets. [#46528](https://github.com/ClickHouse/ClickHouse/pull/46528) ([Mike Kot](https://github.com/myrrc)). +* Add new function `generateRandomStructure` that generates random table structure. It can be used in combination with table function `generateRandom`. [#47409](https://github.com/ClickHouse/ClickHouse/pull/47409) ([Kruglov Pavel](https://github.com/Avogar)). +* Added native ClickHouse Keeper CLI Client. [#47414](https://github.com/ClickHouse/ClickHouse/pull/47414) ([pufit](https://github.com/pufit)). +* The query cache can now be used for production workloads. [#47977](https://github.com/ClickHouse/ClickHouse/pull/47977) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix a bug that prevented the use of `CASE` without an `ELSE` branch and extended `transform` to deal with more types. Also fix some bugs that made transform() return incorrect results when decimal types were mixed with other numeric types. [#48300](https://github.com/ClickHouse/ClickHouse/pull/48300) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Added [server-side encryption using KMS keys](https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingKMSEncryption.html) with S3 tables, and the `header` setting with S3 disks. Closes [#48723](https://github.com/ClickHouse/ClickHouse/issues/48723). [#48724](https://github.com/ClickHouse/ClickHouse/pull/48724) ([Johann Gan](https://github.com/johanngan)). +* Add MemoryTracker for the background tasks (merges and mutation). Introduces `merges_mutations_memory_usage_soft_limit` and `merges_mutations_memory_usage_to_ram_ratio` settings that represent the soft memory limit for merges and mutations. If this limit is reached ClickHouse won't schedule new merge or mutation tasks. Also `MergesMutationsMemoryTracking` metric is introduced to allow observing current memory usage of background tasks. Resubmit [#46089](https://github.com/ClickHouse/ClickHouse/issues/46089). Closes [#48774](https://github.com/ClickHouse/ClickHouse/issues/48774). [#48787](https://github.com/ClickHouse/ClickHouse/pull/48787) ([Dmitry Novik](https://github.com/novikd)). +* Function `dotProduct` work for array. [#49050](https://github.com/ClickHouse/ClickHouse/pull/49050) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Support statement `SHOW INDEX` to improve compatibility with MySQL. [#49158](https://github.com/ClickHouse/ClickHouse/pull/49158) ([Robert Schulze](https://github.com/rschu1ze)). +* Add virtual column `_file` and `_path` support to table function `url`. - Impove error message for table function `url`. - resolves [#49231](https://github.com/ClickHouse/ClickHouse/issues/49231) - resolves [#49232](https://github.com/ClickHouse/ClickHouse/issues/49232). [#49356](https://github.com/ClickHouse/ClickHouse/pull/49356) ([Ziyi Tan](https://github.com/Ziy1-Tan)). +* Adding the `grants` field in the users.xml file, which allows specifying grants for users. [#49381](https://github.com/ClickHouse/ClickHouse/pull/49381) ([pufit](https://github.com/pufit)). +* Add alias `str_to_map` and `mapfromstring` for `extractkeyvaluepairs`. closes [#47185](https://github.com/ClickHouse/ClickHouse/issues/47185). [#49466](https://github.com/ClickHouse/ClickHouse/pull/49466) ([flynn](https://github.com/ucasfl)). +* Support full/right join by using grace hash join algorithm. [#49483](https://github.com/ClickHouse/ClickHouse/pull/49483) ([lgbo](https://github.com/lgbo-ustc)). +* `WITH FILL` modifier groups filling by sorting prefix. Controlled by `use_with_fill_by_sorting_prefix` setting (enabled by default). Related to [#33203](https://github.com/ClickHouse/ClickHouse/issues/33203)#issuecomment-1418736794. [#49503](https://github.com/ClickHouse/ClickHouse/pull/49503) ([Igor Nikonov](https://github.com/devcrafter)). +* Add SQL functions for entropy-learned hashing. [#49656](https://github.com/ClickHouse/ClickHouse/pull/49656) ([Robert Schulze](https://github.com/rschu1ze)). +* Clickhouse-client now accepts queries after "--multiquery" when "--query" (or "-q") is absent. example: clickhouse-client --multiquery "select 1; select 2;". [#49870](https://github.com/ClickHouse/ClickHouse/pull/49870) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Add separate `handshake_timeout` for receiving Hello packet from replica. Closes [#48854](https://github.com/ClickHouse/ClickHouse/issues/48854). [#49948](https://github.com/ClickHouse/ClickHouse/pull/49948) ([Kruglov Pavel](https://github.com/Avogar)). +* New setting s3_max_inflight_parts_for_one_file sets the limit of concurrently loaded parts with multipart upload request in scope of one file. [#49961](https://github.com/ClickHouse/ClickHouse/pull/49961) ([Sema Checherinda](https://github.com/CheSema)). +* Geographical data types (`Point`, `Ring`, `Polygon`, and `MultiPolygon`) are production-ready. [#50022](https://github.com/ClickHouse/ClickHouse/pull/50022) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Added a function "space()" which repeats a space as many times as specified. [#50103](https://github.com/ClickHouse/ClickHouse/pull/50103) ([Robert Schulze](https://github.com/rschu1ze)). +* Added --input_format_csv_trim_whitespaces option. [#50215](https://github.com/ClickHouse/ClickHouse/pull/50215) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Added the dictGetAll function for regexp tree dictionaries to return values from multiple matches as arrays. Closes [#50254](https://github.com/ClickHouse/ClickHouse/issues/50254). [#50255](https://github.com/ClickHouse/ClickHouse/pull/50255) ([Johann Gan](https://github.com/johanngan)). +* Added toLastDayOfWeek() function to round a date or a date with time up to the nearest Saturday or Sunday. [#50315](https://github.com/ClickHouse/ClickHouse/pull/50315) ([Victor Krasnov](https://github.com/sirvickr)). +* Ability to ignore a skip index by specifying `ignore_data_skipping_indices`. [#50329](https://github.com/ClickHouse/ClickHouse/pull/50329) ([Boris Kuschel](https://github.com/bkuschel)). +* Revert 'Add SQL functions for entropy-learned hashing'. [#50416](https://github.com/ClickHouse/ClickHouse/pull/50416) ([Robert Schulze](https://github.com/rschu1ze)). +* Add `system.user_processes` table and `SHOW USER PROCESSES` query to show memory info and ProfileEvents on user level. [#50492](https://github.com/ClickHouse/ClickHouse/pull/50492) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Added storage engine `AzureBlobStorage` and `azure_blob_storage` table function. The supported set of features is very similar to storage/table function `S3`. Implements [#19307](https://github.com/ClickHouse/ClickHouse/issues/19307). [#50604](https://github.com/ClickHouse/ClickHouse/pull/50604) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). + +#### Performance Improvement +* Compress marks and primary key by default. It significantly reduces the cold query time. Upgrade notes: the support for compressed marks and primary key has been added in version 22.9. If you turned on compressed marks or primary key or installed version 23.5 or newer, which has compressed marks or primary key on by default, you will not be able to downgrade to version 22.8 or earlier. You can also explicitly disable compressed marks or primary keys by specifying the `compress_marks` and `compress_primary_key` settings in the `` section of the server configuration file. **Upgrade notes:** If you upgrade from versions prior to 22.9, you should either upgrade all replicas at once or disable the compression before upgrade, or upgrade through an intermediate version, where the compressed marks are supported but not enabled by default, such as 23.3. [#42587](https://github.com/ClickHouse/ClickHouse/pull/42587) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* When reading from multiple files reduce parallel parsing threads for each file resolves [#42192](https://github.com/ClickHouse/ClickHouse/issues/42192). [#46661](https://github.com/ClickHouse/ClickHouse/pull/46661) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Do not store blocks in `ANY` hash join if nothing is inserted. [#48633](https://github.com/ClickHouse/ClickHouse/pull/48633) ([vdimir](https://github.com/vdimir)). +* Fixes aggregate combinator `-If` when JIT compiled. Closes [#48120](https://github.com/ClickHouse/ClickHouse/issues/48120). [#49083](https://github.com/ClickHouse/ClickHouse/pull/49083) ([Igor Nikonov](https://github.com/devcrafter)). +* For reading from remote tables we use smaller tasks (instead of reading the whole part) to make tasks stealing work * task size is determined by size of columns to read * always use 1mb buffers for reading from s3 * boundaries of cache segments aligned to 1mb so they have decent size even with small tasks. it also should prevent fragmentation. [#49287](https://github.com/ClickHouse/ClickHouse/pull/49287) ([Nikita Taranov](https://github.com/nickitat)). +* Default size of a read buffer for reading from local filesystem changed to a slightly better value. Also two new settings are introduced: `max_read_buffer_size_local_fs` and `max_read_buffer_size_remote_fs`. [#49321](https://github.com/ClickHouse/ClickHouse/pull/49321) ([Nikita Taranov](https://github.com/nickitat)). +* Improve memory usage and speed of `SPARSE_HASHED`/`HASHED` dictionaries (e.g. `SPARSE_HASHED` now eats 2.6x less memory, and is ~2x faster). [#49380](https://github.com/ClickHouse/ClickHouse/pull/49380) ([Azat Khuzhin](https://github.com/azat)). +* Use aggregate projection only if it reads fewer granules than normal reading. It should help in case if query hits the PK of the table, but not the projection. Fixes [#49150](https://github.com/ClickHouse/ClickHouse/issues/49150). [#49417](https://github.com/ClickHouse/ClickHouse/pull/49417) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Optimize PODArray::resize_fill() callers. [#49459](https://github.com/ClickHouse/ClickHouse/pull/49459) ([Azat Khuzhin](https://github.com/azat)). +* Optimize the system.query_log and system.query_thread_log tables by applying LowCardinality when appropriate. The queries over these tables will be faster. [#49530](https://github.com/ClickHouse/ClickHouse/pull/49530) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better performance when reading local Parquet files (through parallel reading). [#49539](https://github.com/ClickHouse/ClickHouse/pull/49539) ([Michael Kolupaev](https://github.com/al13n321)). +* Improve the performance of `RIGHT/FULL JOIN` by up to 2 times in certain scenarios, especially when joining a small left table with a large right table. [#49585](https://github.com/ClickHouse/ClickHouse/pull/49585) ([lgbo](https://github.com/lgbo-ustc)). +* Improve performance of BLAKE3 by 11% by enabling LTO for Rust. [#49600](https://github.com/ClickHouse/ClickHouse/pull/49600) ([Azat Khuzhin](https://github.com/azat)). +* Optimize the structure of the `system.opentelemetry_span_log`. Use `LowCardinality` where appropriate. Although this table is generally stupid (it is using the Map data type even for common attributes), it will be slightly better. [#49647](https://github.com/ClickHouse/ClickHouse/pull/49647) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Try to reserve hash table's size in `grace_hash` join. [#49816](https://github.com/ClickHouse/ClickHouse/pull/49816) ([lgbo](https://github.com/lgbo-ustc)). +* As is addresed in issue [#49748](https://github.com/ClickHouse/ClickHouse/issues/49748), the predicates with date converters, such as **toYear, toYYYYMM**, could be rewritten with the equivalent date (YYYY-MM-DD) comparisons at the AST level. And this transformation could bring performance improvement as it is free from the expensive date converter and the comparison between dates (or integers in the low level representation) is quite low-cost. The [prototype](https://github.com/ZhiguoZh/ClickHouse/commit/c7f1753f0c9363a19d95fa46f1cfed1d9f505ee0) shows that, with all identified date converters optimized, the overall QPS of the 13 queries is enhanced by **~11%** on the ICX server (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads). [#50062](https://github.com/ClickHouse/ClickHouse/pull/50062) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Parallel merge of `uniqExactIf` states. Closes [#49885](https://github.com/ClickHouse/ClickHouse/issues/49885). [#50285](https://github.com/ClickHouse/ClickHouse/pull/50285) ([flynn](https://github.com/ucasfl)). +* As is addresed in issue [#49748](https://github.com/ClickHouse/ClickHouse/issues/49748), the predicates with date converters, such as toYear, toYYYYMM, could be rewritten with the equivalent date (YYYY-MM-DD) comparisons at the AST level. And this transformation could bring performance improvement as it is free from the expensive date converter and the comparison between dates (or integers in the low level representation) is quite low-cost. [#50307](https://github.com/ClickHouse/ClickHouse/pull/50307) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Parallel merging supported for `uniqExact` with modifiers `-Array`, `-Merge`, `-OrNull`, `-State`. [#50413](https://github.com/ClickHouse/ClickHouse/pull/50413) ([flynn](https://github.com/ucasfl)). +* Enable LZ4_FAST_DEC_LOOP for Arm LZ4 to get 5% of decompression speed. [#50588](https://github.com/ClickHouse/ClickHouse/pull/50588) ([Daniel Kutenin](https://github.com/danlark1)). + +#### Improvement +* Add support for CGroup version 2 for asynchronous metrics about the memory usage and availability. This closes [#37983](https://github.com/ClickHouse/ClickHouse/issues/37983). [#45999](https://github.com/ClickHouse/ClickHouse/pull/45999) ([sichenzhao](https://github.com/sichenzhao)). +* Cluster table functions should always skip unavailable shards. close [#46314](https://github.com/ClickHouse/ClickHouse/issues/46314). [#46765](https://github.com/ClickHouse/ClickHouse/pull/46765) ([zk_kiger](https://github.com/zk-kiger)). +* When your csv file contains empty columns, like: ```. [#47496](https://github.com/ClickHouse/ClickHouse/pull/47496) ([你不要过来啊](https://github.com/iiiuwioajdks)). +* ROW POLICY for all tables that belong to a DATABASE. [#47640](https://github.com/ClickHouse/ClickHouse/pull/47640) ([Ilya Golshtein](https://github.com/ilejn)). +* Add Google Cloud Storage S3 compatible table function `gcs`. Like the `oss` and `cosn` functions, it is just an alias over the `s3` table function, and it does not bring any new features. [#47815](https://github.com/ClickHouse/ClickHouse/pull/47815) ([Kuba Kaflik](https://github.com/jkaflik)). +* Add ability to use strict parts size for S3 (compatibility with CloudFlare R2 S3 Storage). [#48492](https://github.com/ClickHouse/ClickHouse/pull/48492) ([Azat Khuzhin](https://github.com/azat)). +* Added new columns with info about `Replicated` database replicas to `system.clusters`: `database_shard_name`, `database_replica_name`, `is_active`. Added an optional `FROM SHARD` clause to `SYSTEM DROP DATABASE REPLICA` query. [#48548](https://github.com/ClickHouse/ClickHouse/pull/48548) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add a new column `zookeeper_name` in system.replicas, to indicate on which (auxiliary) zookeeper cluster the replicated table's metadata is stored. [#48549](https://github.com/ClickHouse/ClickHouse/pull/48549) ([cangyin](https://github.com/cangyin)). +* `IN` operator support compare `Date` and `Date32`. Closes [#48736](https://github.com/ClickHouse/ClickHouse/issues/48736). [#48806](https://github.com/ClickHouse/ClickHouse/pull/48806) ([flynn](https://github.com/ucasfl)). +* Support for erasure codes in HDFS, author: @M1eyu2018, @tomscut. [#48833](https://github.com/ClickHouse/ClickHouse/pull/48833) ([M1eyu](https://github.com/M1eyu2018)). +* The query cache can now supports queries with totals and extremes modifier. [#48853](https://github.com/ClickHouse/ClickHouse/pull/48853) ([Robert Schulze](https://github.com/rschu1ze)). +* Introduces new keyword `INTO OUTFILE 'file.txt' APPEND`. [#48880](https://github.com/ClickHouse/ClickHouse/pull/48880) ([alekar](https://github.com/alekar)). +* The `BACKUP` command will not decrypt data from encrypted disks while making a backup. Instead the data will be stored in a backup in encrypted form. Such backups can be restored only to an encrypted disk with the same (or extended) list of encryption keys. [#48896](https://github.com/ClickHouse/ClickHouse/pull/48896) ([Vitaly Baranov](https://github.com/vitlibar)). +* Keeper improvement: add `CheckNotExists` request to Keeper. [#48897](https://github.com/ClickHouse/ClickHouse/pull/48897) ([Antonio Andelic](https://github.com/antonio2368)). +* Implement SYSTEM DROP REPLICA from auxillary ZooKeeper clusters, may be close [#48931](https://github.com/ClickHouse/ClickHouse/issues/48931). [#48932](https://github.com/ClickHouse/ClickHouse/pull/48932) ([wangxiaobo](https://github.com/wzb5212)). +* Add Array data type to MongoDB. Closes [#48598](https://github.com/ClickHouse/ClickHouse/issues/48598). [#48983](https://github.com/ClickHouse/ClickHouse/pull/48983) ([Nikolay Degterinsky](https://github.com/evillique)). +* Keeper performance improvements: avoid serializing same request twice while processing. Cache deserialization results of large requests. Controlled by new coordination setting `min_request_size_for_cache`. [#49004](https://github.com/ClickHouse/ClickHouse/pull/49004) ([Antonio Andelic](https://github.com/antonio2368)). +* Support storing `Interval` data types in tables. [#49085](https://github.com/ClickHouse/ClickHouse/pull/49085) ([larryluogit](https://github.com/larryluogit)). +* Add support for size suffixes in quota creation statement parameters. [#49087](https://github.com/ClickHouse/ClickHouse/pull/49087) ([Eridanus](https://github.com/Eridanus117)). +* Allow using `ntile` window function without explicit window frame definition: `ntile(3) OVER (ORDER BY a)`, close [#46763](https://github.com/ClickHouse/ClickHouse/issues/46763). [#49093](https://github.com/ClickHouse/ClickHouse/pull/49093) ([vdimir](https://github.com/vdimir)). +* Added settings (`number_of_mutations_to_delay`, `number_of_mutations_to_throw`) to delay or throw `ALTER` queries that create mutations (`ALTER UPDATE`, `ALTER DELETE`, `ALTER MODIFY COLUMN`, ...) in case when table already has a lot of unfinished mutations. [#49117](https://github.com/ClickHouse/ClickHouse/pull/49117) ([Anton Popov](https://github.com/CurtizJ)). +* Added setting `async_insert` for `MergeTables`. It has the same meaning as query-level setting `async_insert` and enables asynchronous inserts for specific table. Note: it doesn't take effect for insert queries from `clickhouse-client`, use query-level setting in that case. [#49122](https://github.com/ClickHouse/ClickHouse/pull/49122) ([Anton Popov](https://github.com/CurtizJ)). +* Catch exception from `create_directories` in filesystem cache. [#49203](https://github.com/ClickHouse/ClickHouse/pull/49203) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Copies embedded examples to a new field `example` in `system.functions` to supplement the field `description`. [#49222](https://github.com/ClickHouse/ClickHouse/pull/49222) ([Dan Roscigno](https://github.com/DanRoscigno)). +* Enable connection options for the MongoDB dictionary. Example: ``` xml localhost 27017 test dictionary_source ssl=true ``` ### Documentation entry for user-facing changes. [#49225](https://github.com/ClickHouse/ClickHouse/pull/49225) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Added an alias `asymptotic` for `asymp` computational method for `kolmogorovSmirnovTest`. Improved documentation. [#49286](https://github.com/ClickHouse/ClickHouse/pull/49286) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Aggregation function groupBitAnd/Or/Xor now work on signed integer data. This makes them consistent with the behavior of scalar functions bitAnd/Or/Xor. [#49292](https://github.com/ClickHouse/ClickHouse/pull/49292) ([exmy](https://github.com/exmy)). +* Split function-documentation into more fine-granular fields. [#49300](https://github.com/ClickHouse/ClickHouse/pull/49300) ([Robert Schulze](https://github.com/rschu1ze)). +* Introduced settings: - `merge_max_block_size_bytes` to limit the amount of memory used for background operations. - `vertical_merge_algorithm_min_bytes_to_activate` to add another condition to activate vertical merges. [#49313](https://github.com/ClickHouse/ClickHouse/pull/49313) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Use multiple threads shared between all tables within a server to load outdated data parts. The the size of the pool and its queue is controlled by `max_outdated_parts_loading_thread_pool_size` and `outdated_part_loading_thread_pool_queue_size` settings. [#49317](https://github.com/ClickHouse/ClickHouse/pull/49317) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Don't overestimate the size of processed data for `LowCardinality` columns when they share dictionaries between blocks. This closes [#49322](https://github.com/ClickHouse/ClickHouse/issues/49322). See also [#48745](https://github.com/ClickHouse/ClickHouse/issues/48745). [#49323](https://github.com/ClickHouse/ClickHouse/pull/49323) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Parquet writer now uses reasonable row group size when invoked through OUTFILE. [#49325](https://github.com/ClickHouse/ClickHouse/pull/49325) ([Michael Kolupaev](https://github.com/al13n321)). +* Allow restricted keywords like `ARRAY` as an alias if the alias is quoted. Closes [#49324](https://github.com/ClickHouse/ClickHouse/issues/49324). [#49360](https://github.com/ClickHouse/ClickHouse/pull/49360) ([Nikolay Degterinsky](https://github.com/evillique)). +* Added possibility to use temporary tables in FROM part of ATTACH PARTITION FROM and REPLACE PARTITION FROM. [#49436](https://github.com/ClickHouse/ClickHouse/pull/49436) ([Roman Vasin](https://github.com/rvasin)). +* Data parts loading and deletion jobs were moved to shared server-wide pools instead of per-table pools. Pools sizes are controlled via settings `max_active_parts_loading_thread_pool_size`, `max_outdated_parts_loading_thread_pool_size` and `max_parts_cleaning_thread_pool_size` in top-level config. Table-level settings `max_part_loading_threads` and `max_part_removal_threads` became obsolete. [#49474](https://github.com/ClickHouse/ClickHouse/pull/49474) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Allow `?password=pass` in URL. Password is replaced in browser history. [#49505](https://github.com/ClickHouse/ClickHouse/pull/49505) ([Mike Kot](https://github.com/myrrc)). +* Allow zero objects in ReadBufferFromRemoteFSGather (because empty files are not backuped, so we might end up with zero blobs in metadata file). Closes [#49480](https://github.com/ClickHouse/ClickHouse/issues/49480). [#49519](https://github.com/ClickHouse/ClickHouse/pull/49519) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Attach thread MemoryTracker to `total_memory_tracker` after `ThreadGroup` detached. [#49527](https://github.com/ClickHouse/ClickHouse/pull/49527) ([Dmitry Novik](https://github.com/novikd)). +* Make `Pretty` formats prettier: squash blocks if not much time passed since the output of the previous block. This is controlled by a new setting `output_format_pretty_squash_ms` (100ms by default). This closes [#49153](https://github.com/ClickHouse/ClickHouse/issues/49153). [#49537](https://github.com/ClickHouse/ClickHouse/pull/49537) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add initial support to do JOINs with pure parallel replicas. [#49544](https://github.com/ClickHouse/ClickHouse/pull/49544) ([Raúl Marín](https://github.com/Algunenano)). +* Fix parameterized views when query parameter used multiple times in the query. [#49556](https://github.com/ClickHouse/ClickHouse/pull/49556) ([Azat Khuzhin](https://github.com/azat)). +* Release memory allocated for the last sent ProfileEvents snapshot in the context of a query. Followup [#47564](https://github.com/ClickHouse/ClickHouse/issues/47564). [#49561](https://github.com/ClickHouse/ClickHouse/pull/49561) ([Dmitry Novik](https://github.com/novikd)). +* Function "makeDate" now provides a MySQL-compatible overload (year & day of the year argument). [#49603](https://github.com/ClickHouse/ClickHouse/pull/49603) ([Robert Schulze](https://github.com/rschu1ze)). +* More parallelism on `Outdated` parts removal with "zero-copy replication". [#49630](https://github.com/ClickHouse/ClickHouse/pull/49630) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Reduced number of `List` ZooKeeper requests when selecting parts to merge and a lot of partitions do not have anything to merge. [#49637](https://github.com/ClickHouse/ClickHouse/pull/49637) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Support `dictionary` table function for `RegExpTreeDictionary`. [#49666](https://github.com/ClickHouse/ClickHouse/pull/49666) ([Han Fei](https://github.com/hanfei1991)). +* Added weighted fair IO scheduling policy. Added dynamic resource manager, which allows IO scheduling hierarchy to be updated in runtime w/o server restarts. [#49671](https://github.com/ClickHouse/ClickHouse/pull/49671) ([Sergei Trifonov](https://github.com/serxa)). +* Add compose request after multipart upload to GCS. This enables the usage of copy operation on objects uploaded with the multipart upload. It's recommended to set `s3_strict_upload_part_size` to some value because compose request can fail on objects created with parts of different sizes. [#49693](https://github.com/ClickHouse/ClickHouse/pull/49693) ([Antonio Andelic](https://github.com/antonio2368)). +* Improve the "best-effort" parsing logic to accept `key_value_delimiter` as a valid part of the value. This also simplifies branching and might even speed up things a bit. [#49760](https://github.com/ClickHouse/ClickHouse/pull/49760) ([Arthur Passos](https://github.com/arthurpassos)). +* Facilitate profile data association and aggregation for the same query. [#49777](https://github.com/ClickHouse/ClickHouse/pull/49777) ([helifu](https://github.com/helifu)). +* System log tables can now have custom sorting keys. [#49778](https://github.com/ClickHouse/ClickHouse/pull/49778) ([helifu](https://github.com/helifu)). +* A new field 'partitions' is used to indicate which partitions are participating in the calculation. [#49779](https://github.com/ClickHouse/ClickHouse/pull/49779) ([helifu](https://github.com/helifu)). +* Added `enable_the_endpoint_id_with_zookeeper_name_prefix` setting for `ReplicatedMergeTree` (disabled by default). When enabled, it adds ZooKeeper cluster name to table's interserver communication endpoint. It avoids `Duplicate interserver IO endpoint` errors when having replicated tables with the same path, but different auxiliary ZooKeepers. [#49780](https://github.com/ClickHouse/ClickHouse/pull/49780) ([helifu](https://github.com/helifu)). +* Add query parameters to clickhouse-local. Closes [#46561](https://github.com/ClickHouse/ClickHouse/issues/46561). [#49785](https://github.com/ClickHouse/ClickHouse/pull/49785) ([Nikolay Degterinsky](https://github.com/evillique)). +* Qpl_deflate codec lower the minimum simd version to sse 4.2. [doc change in qpl](https://github.com/intel/qpl/commit/3f8f5cea27739f5261e8fd577dc233ffe88bf679) - intel® qpl relies on a run-time kernels dispatcher and cpuid check to choose the best available implementation(sse/avx2/avx512) - restructured cmakefile for qpl build in clickhouse to align with latest upstream qpl. [#49811](https://github.com/ClickHouse/ClickHouse/pull/49811) ([jasperzhu](https://github.com/jinjunzh)). +* Allow loading dictionaries and functions from YAML by default. In previous versions, it required editing the `dictionaries_config` or `user_defined_executable_functions_config` in the configuration file, as they expected `*.xml` files. [#49812](https://github.com/ClickHouse/ClickHouse/pull/49812) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The Kafka table engine now allows to use alias columns. [#49824](https://github.com/ClickHouse/ClickHouse/pull/49824) ([Aleksandr Musorin](https://github.com/AVMusorin)). +* Add setting to limit the max number of pairs produced by extractKeyValuePairs, safeguard to avoid using way too much memory. [#49836](https://github.com/ClickHouse/ClickHouse/pull/49836) ([Arthur Passos](https://github.com/arthurpassos)). +* Add support for (an unusual) case where the arguments in the `IN` operator are single-element tuples. [#49844](https://github.com/ClickHouse/ClickHouse/pull/49844) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* `bitHammingDistance` function support `String` and `FixedString` data type. Closes [#48827](https://github.com/ClickHouse/ClickHouse/issues/48827). [#49858](https://github.com/ClickHouse/ClickHouse/pull/49858) ([flynn](https://github.com/ucasfl)). +* Fix timeout resetting errors in the client on OS X. [#49863](https://github.com/ClickHouse/ClickHouse/pull/49863) ([alekar](https://github.com/alekar)). +* Add support for big integers, such as UInt128, Int128, UInt256, and Int256 in the function `bitCount`. This enables Hamming distance over large bit masks for AI applications. [#49867](https://github.com/ClickHouse/ClickHouse/pull/49867) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This PR makes fingerprints to be used instead of key IDs in encrypted disks. [#49882](https://github.com/ClickHouse/ClickHouse/pull/49882) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add UUID data type to PostgreSQL. Closes [#49739](https://github.com/ClickHouse/ClickHouse/issues/49739). [#49894](https://github.com/ClickHouse/ClickHouse/pull/49894) ([Nikolay Degterinsky](https://github.com/evillique)). +* Make `allow_experimental_query_cache` setting as obsolete for backward-compatibility. It was removed in https://github.com/ClickHouse/ClickHouse/pull/47977. [#49934](https://github.com/ClickHouse/ClickHouse/pull/49934) ([Timur Solodovnikov](https://github.com/tsolodov)). +* Function toUnixTimestamp() now accepts Date and Date32 arguments. [#49989](https://github.com/ClickHouse/ClickHouse/pull/49989) ([Victor Krasnov](https://github.com/sirvickr)). +* Charge only server memory for dictionaries. [#49995](https://github.com/ClickHouse/ClickHouse/pull/49995) ([Azat Khuzhin](https://github.com/azat)). +* Add schema inference to PostgreSQL, MySQL, MeiliSearch, and SQLite table engines. Closes [#49972](https://github.com/ClickHouse/ClickHouse/issues/49972). [#50000](https://github.com/ClickHouse/ClickHouse/pull/50000) ([Nikolay Degterinsky](https://github.com/evillique)). +* The server will allow using the `SQL_*` settings such as `SQL_AUTO_IS_NULL` as no-ops for MySQL compatibility. This closes [#49927](https://github.com/ClickHouse/ClickHouse/issues/49927). [#50013](https://github.com/ClickHouse/ClickHouse/pull/50013) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Preserve initial_query_id for ON CLUSTER queries, which is useful for introspection (under `distributed_ddl_entry_format_version=5`). [#50015](https://github.com/ClickHouse/ClickHouse/pull/50015) ([Azat Khuzhin](https://github.com/azat)). +* Preserve backward incompatibility for renamed settings by using aliases (`allow_experimental_projection_optimization` for `optimize_use_projections`, `allow_experimental_lightweight_delete` for `enable_lightweight_delete`). [#50044](https://github.com/ClickHouse/ClickHouse/pull/50044) ([Azat Khuzhin](https://github.com/azat)). +* Support cross-replication in distributed queries using the new infrastructure. [#50097](https://github.com/ClickHouse/ClickHouse/pull/50097) ([Dmitry Novik](https://github.com/novikd)). +* Support passing fqdn through setting my_hostname to register cluster node in keeper. Add setting of invisible to support multi compute groups. A compute group as a cluster, is invisible to other compute groups. [#50186](https://github.com/ClickHouse/ClickHouse/pull/50186) ([Yangkuan Liu](https://github.com/LiuYangkuan)). +* Fix PostgreSQL reading all the data even though `LIMIT n` could be specified. [#50187](https://github.com/ClickHouse/ClickHouse/pull/50187) ([Kseniia Sumarokova](https://github.com/kssenii)). +* 1) Fixed an error `NOT_FOUND_COLUMN_IN_BLOCK` in case of using parallel replicas with non-replicated storage with disabled setting `parallel_replicas_for_non_replicated_merge_tree` 2) Now `allow_experimental_parallel_reading_from_replicas` have 3 possible values - 0, 1 and 2. 0 - disabled, 1 - enabled, silently disable them in case of failure (in case of FINAL or JOIN), 2 - enabled, throw an expection in case of failure. 3) If FINAL modifier is used in SELECT query and parallel replicas are enabled, ClickHouse will try to disable them if `allow_experimental_parallel_reading_from_replicas` is set to 1 and throw an exception otherwise. [#50195](https://github.com/ClickHouse/ClickHouse/pull/50195) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Don't send head request for all keys in Iceberg schema inference, only for keys that are used for reaing data. [#50203](https://github.com/ClickHouse/ClickHouse/pull/50203) ([Kruglov Pavel](https://github.com/Avogar)). +* Add new profile events for queries with subqueries (`QueriesWithSubqueries`/`SelectQueriesWithSubqueries`/`InsertQueriesWithSubqueries`). [#50204](https://github.com/ClickHouse/ClickHouse/pull/50204) ([Azat Khuzhin](https://github.com/azat)). +* Adding the roles field in the users.xml file, which allows specifying roles with grants via a config file. [#50278](https://github.com/ClickHouse/ClickHouse/pull/50278) ([pufit](https://github.com/pufit)). +* When parallel replicas are enabled they will always skip unavailable servers (the behavior is controlled by the setting `skip_unavailable_shards`, enabled by default and can be only disabled). This closes: [#48565](https://github.com/ClickHouse/ClickHouse/issues/48565). [#50293](https://github.com/ClickHouse/ClickHouse/pull/50293) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix a typo. [#50306](https://github.com/ClickHouse/ClickHouse/pull/50306) ([helifu](https://github.com/helifu)). +* Setting `enable_memory_bound_merging_of_aggregation_results` is enabled by default. If you update from version prior to 22.12, we recommend to set this flag to `false` until update is finished. [#50319](https://github.com/ClickHouse/ClickHouse/pull/50319) ([Nikita Taranov](https://github.com/nickitat)). +* Report `CGroupCpuCfsPeriod` and `CGroupCpuCfsQuota` in AsynchronousMetrics. - Respect cgroup v2 memory limits during server startup. [#50379](https://github.com/ClickHouse/ClickHouse/pull/50379) ([alekar](https://github.com/alekar)). +* Bump internal protobuf to v3.18 (fixes CVE-2022-1941). [#50400](https://github.com/ClickHouse/ClickHouse/pull/50400) ([Robert Schulze](https://github.com/rschu1ze)). +* Bump internal libxml2 to v2.10.4 (fixes CVE-2023-28484 and CVE-2023-29469). [#50402](https://github.com/ClickHouse/ClickHouse/pull/50402) ([Robert Schulze](https://github.com/rschu1ze)). +* Bump c-ares to v1.19.1 (CVE-2023-32067, CVE-2023-31130, CVE-2023-31147). [#50403](https://github.com/ClickHouse/ClickHouse/pull/50403) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix CVE-2022-2469 in libgsasl. [#50404](https://github.com/ClickHouse/ClickHouse/pull/50404) ([Robert Schulze](https://github.com/rschu1ze)). +* Make filter push down through cross join. [#50430](https://github.com/ClickHouse/ClickHouse/pull/50430) ([Han Fei](https://github.com/hanfei1991)). +* Add a signal handler for SIGQUIT to work the same way as SIGINT. Closes [#50298](https://github.com/ClickHouse/ClickHouse/issues/50298). [#50435](https://github.com/ClickHouse/ClickHouse/pull/50435) ([Nikolay Degterinsky](https://github.com/evillique)). +* In case JSON parse fails due to the large size of the object output the last position to allow debugging. [#50474](https://github.com/ClickHouse/ClickHouse/pull/50474) ([Valentin Alexeev](https://github.com/valentinalexeev)). +* Support decimals with not fixed size. Closes [#49130](https://github.com/ClickHouse/ClickHouse/issues/49130). [#50586](https://github.com/ClickHouse/ClickHouse/pull/50586) ([Kruglov Pavel](https://github.com/Avogar)). +* Disable pure parallel replicas if trivial count optimization is possible. [#50594](https://github.com/ClickHouse/ClickHouse/pull/50594) ([Raúl Marín](https://github.com/Algunenano)). +* Added support of TRUNCATE db.table additional to TRUNCATE TABLE db.table in MaterializedMySQL. [#50624](https://github.com/ClickHouse/ClickHouse/pull/50624) ([Val Doroshchuk](https://github.com/valbok)). +* Disable parallel replicas automatically when the estimated number of granules is less than threshold. The behavior is controlled by a setting `parallel_replicas_min_number_of_granules_to_enable`. [#50639](https://github.com/ClickHouse/ClickHouse/pull/50639) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* When creating skipping indexes via "ALTER TABLE table ADD INDEX", the "GRANULARITY" clause can now be omitted. In that case, GRANULARITY is assumed to be 1. [#50658](https://github.com/ClickHouse/ClickHouse/pull/50658) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix slow cache in presence of big inserts. [#50680](https://github.com/ClickHouse/ClickHouse/pull/50680) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Set default max_elements limit in filesystem cache to 10000000. [#50682](https://github.com/ClickHouse/ClickHouse/pull/50682) ([Kseniia Sumarokova](https://github.com/kssenii)). +* SHOW INDICES is now an alias of statement SHOW INDEX/INDEXES/KEYS. [#50713](https://github.com/ClickHouse/ClickHouse/pull/50713) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Build/Testing/Packaging Improvement +* New and improved keeper-bench. Everything can be customized from yaml/XML file: - request generator - each type of request generator can have a specific set of fields - multi requests can be generated just by doing the same under `multi` key - for each request or subrequest in multi a `weight` field can be defined to control distribution - define trees that need to be setup for a test run - hosts can be defined with all timeouts customizable and it's possible to control how many sessions to generate for each host - integers defined with `min_value` and `max_value` fields are random number generators. [#48547](https://github.com/ClickHouse/ClickHouse/pull/48547) ([Antonio Andelic](https://github.com/antonio2368)). +* ... Add a test to check max_rows_to_read_leaf behaviour. [#48950](https://github.com/ClickHouse/ClickHouse/pull/48950) ([Sean Haynes](https://github.com/seandhaynes)). +* Io_uring is not supported on macos, don't choose it when running tests on local to avoid occassional failures. [#49250](https://github.com/ClickHouse/ClickHouse/pull/49250) ([Frank Chen](https://github.com/FrankChen021)). +* Support named fault injection for testing. [#49361](https://github.com/ClickHouse/ClickHouse/pull/49361) ([Han Fei](https://github.com/hanfei1991)). +* Fix the 01193_metadata_loading test to match the query execution time specific to s390x. [#49455](https://github.com/ClickHouse/ClickHouse/pull/49455) ([MeenaRenganathan22](https://github.com/MeenaRenganathan22)). +* Use the RapidJSONParser library to parse the JSON float values in case of s390x. [#49457](https://github.com/ClickHouse/ClickHouse/pull/49457) ([MeenaRenganathan22](https://github.com/MeenaRenganathan22)). +* Allow running ClickHouse in the OS where the `prctl` (process control) syscall is not available, such as AWS Lambda. [#49538](https://github.com/ClickHouse/ClickHouse/pull/49538) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve CI check with an enabled analyzer. Now it should be green if only tests from `tests/broken_tests.txt` are broken. [#49562](https://github.com/ClickHouse/ClickHouse/pull/49562) ([Dmitry Novik](https://github.com/novikd)). +* Fixed the issue of build conflict between contrib/isa-l and isa-l in qpl [49296](https://github.com/ClickHouse/ClickHouse/issues/49296). [#49584](https://github.com/ClickHouse/ClickHouse/pull/49584) ([jasperzhu](https://github.com/jinjunzh)). +* Utilities are now only build if explicitly requested ("-DENABLE_UTILS=1") instead of by default, this reduces link times in typical development builds. [#49620](https://github.com/ClickHouse/ClickHouse/pull/49620) ([Robert Schulze](https://github.com/rschu1ze)). +* Pull build description of idxd-config into a separate CMake file to avoid accidental removal in future. [#49651](https://github.com/ClickHouse/ClickHouse/pull/49651) ([jasperzhu](https://github.com/jinjunzh)). +* Add CI check with an enabled analyzer in the master. Followup [#49562](https://github.com/ClickHouse/ClickHouse/issues/49562). [#49668](https://github.com/ClickHouse/ClickHouse/pull/49668) ([Dmitry Novik](https://github.com/novikd)). +* Switch to LLVM/clang 16. [#49678](https://github.com/ClickHouse/ClickHouse/pull/49678) ([Azat Khuzhin](https://github.com/azat)). +* Fixed DefaultHash64 for non-64 bit integers on s390x. [#49833](https://github.com/ClickHouse/ClickHouse/pull/49833) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Allow building ClickHouse with clang-17. [#49851](https://github.com/ClickHouse/ClickHouse/pull/49851) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ClickHouse is now easier to be integrated into other cmake projects. [#49991](https://github.com/ClickHouse/ClickHouse/pull/49991) ([Amos Bird](https://github.com/amosbird)). +* Link `boost::context` library to `clickhouse_common_io`. This closes: [#50381](https://github.com/ClickHouse/ClickHouse/issues/50381). [#50385](https://github.com/ClickHouse/ClickHouse/pull/50385) ([HaiBo Li](https://github.com/marising)). +* Add support for building with clang-17. [#50410](https://github.com/ClickHouse/ClickHouse/pull/50410) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix strange additional QEMU logging after [#47151](https://github.com/ClickHouse/ClickHouse/issues/47151), see https://s3.amazonaws.com/clickhouse-test-reports/50078/a4743996ee4f3583884d07bcd6501df0cfdaa346/stateless_tests__release__databasereplicated__[3_4].html. [#50442](https://github.com/ClickHouse/ClickHouse/pull/50442) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* ClickHouse can work on Linux RISC-V 6.1.22. This closes [#50456](https://github.com/ClickHouse/ClickHouse/issues/50456). [#50457](https://github.com/ClickHouse/ClickHouse/pull/50457) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* ActionsDAG: fix wrong optimization [#47584](https://github.com/ClickHouse/ClickHouse/pull/47584) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Correctly handle concurrent snapshots in Keeper [#48466](https://github.com/ClickHouse/ClickHouse/pull/48466) ([Antonio Andelic](https://github.com/antonio2368)). +* MergeTreeMarksLoader holds DataPart instead of DataPartStorage [#48515](https://github.com/ClickHouse/ClickHouse/pull/48515) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* sequence state fix [#48603](https://github.com/ClickHouse/ClickHouse/pull/48603) ([Ilya Golshtein](https://github.com/ilejn)). +* Back/Restore concurrency check on previous fails [#48726](https://github.com/ClickHouse/ClickHouse/pull/48726) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix Attaching a table with non-existent ZK path does not increase the ReadonlyReplica metric [#48954](https://github.com/ClickHouse/ClickHouse/pull/48954) ([wangxiaobo](https://github.com/wzb5212)). +* Fix possible terminate called for uncaught exception in some places [#49112](https://github.com/ClickHouse/ClickHouse/pull/49112) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix key not found error for queries with multiple StorageJoin [#49137](https://github.com/ClickHouse/ClickHouse/pull/49137) ([vdimir](https://github.com/vdimir)). +* Fix wrong query result when using nullable primary key [#49172](https://github.com/ClickHouse/ClickHouse/pull/49172) ([Duc Canh Le](https://github.com/canhld94)). +* Revert "Fix GCS native copy ([#48981](https://github.com/ClickHouse/ClickHouse/issues/48981))" [#49194](https://github.com/ClickHouse/ClickHouse/pull/49194) ([Raúl Marín](https://github.com/Algunenano)). +* Fix reinterpretAs*() on big endian machines [#49198](https://github.com/ClickHouse/ClickHouse/pull/49198) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Lock zero copy parts more atomically [#49211](https://github.com/ClickHouse/ClickHouse/pull/49211) ([alesapin](https://github.com/alesapin)). +* Fix race on Outdated parts loading [#49223](https://github.com/ClickHouse/ClickHouse/pull/49223) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix all key value is null and group use rollup return wrong answer [#49282](https://github.com/ClickHouse/ClickHouse/pull/49282) ([Shuai li](https://github.com/loneylee)). +* Fix calculating load_factor for HASHED dictionaries with SHARDS [#49319](https://github.com/ClickHouse/ClickHouse/pull/49319) ([Azat Khuzhin](https://github.com/azat)). +* Disallow configuring compression CODECs for alias columns [#49363](https://github.com/ClickHouse/ClickHouse/pull/49363) ([Timur Solodovnikov](https://github.com/tsolodov)). +* Fix bug in removal of existing part directory [#49365](https://github.com/ClickHouse/ClickHouse/pull/49365) ([alesapin](https://github.com/alesapin)). +* Properly fix GCS when HMAC is used [#49390](https://github.com/ClickHouse/ClickHouse/pull/49390) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix fuzz bug when subquery set is not built when reading from remote() [#49425](https://github.com/ClickHouse/ClickHouse/pull/49425) ([Alexander Gololobov](https://github.com/davenger)). +* Invert `shutdown_wait_unfinished_queries` [#49427](https://github.com/ClickHouse/ClickHouse/pull/49427) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Fix another zero copy bug [#49473](https://github.com/ClickHouse/ClickHouse/pull/49473) ([alesapin](https://github.com/alesapin)). +* Fix postgres database setting [#49481](https://github.com/ClickHouse/ClickHouse/pull/49481) ([Mal Curtis](https://github.com/snikch)). +* Correctly handle s3Cluster arguments [#49490](https://github.com/ClickHouse/ClickHouse/pull/49490) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix bug in TraceCollector destructor. [#49508](https://github.com/ClickHouse/ClickHouse/pull/49508) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix AsynchronousReadIndirectBufferFromRemoteFS breaking on short seeks [#49525](https://github.com/ClickHouse/ClickHouse/pull/49525) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix dictionaries loading order [#49560](https://github.com/ClickHouse/ClickHouse/pull/49560) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Forbid the change of data type of Object('json') column [#49563](https://github.com/ClickHouse/ClickHouse/pull/49563) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix stress test (Logical error: Expected 7134 >= 11030) [#49623](https://github.com/ClickHouse/ClickHouse/pull/49623) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix bug in DISTINCT [#49628](https://github.com/ClickHouse/ClickHouse/pull/49628) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix: DISTINCT in order with zero values in non-sorted columns [#49636](https://github.com/ClickHouse/ClickHouse/pull/49636) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix one-off error in big integers found by UBSan with fuzzer [#49645](https://github.com/ClickHouse/ClickHouse/pull/49645) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix reading from sparse columns after restart [#49660](https://github.com/ClickHouse/ClickHouse/pull/49660) ([Anton Popov](https://github.com/CurtizJ)). +* Fix assert in SpanHolder::finish() with fibers [#49673](https://github.com/ClickHouse/ClickHouse/pull/49673) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix short circuit functions and mutations with sparse arguments [#49716](https://github.com/ClickHouse/ClickHouse/pull/49716) ([Anton Popov](https://github.com/CurtizJ)). +* Fix writing appended files to incremental backups [#49725](https://github.com/ClickHouse/ClickHouse/pull/49725) ([Vitaly Baranov](https://github.com/vitlibar)). +* Ignore LWD column in checkPartDynamicColumns [#49737](https://github.com/ClickHouse/ClickHouse/pull/49737) ([Alexander Gololobov](https://github.com/davenger)). +* Fix msan issue in randomStringUTF8() [#49750](https://github.com/ClickHouse/ClickHouse/pull/49750) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix aggregate function kolmogorovSmirnovTest [#49768](https://github.com/ClickHouse/ClickHouse/pull/49768) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Fix settings aliases in native protocol [#49776](https://github.com/ClickHouse/ClickHouse/pull/49776) ([Azat Khuzhin](https://github.com/azat)). +* Fix `arrayMap` with array of tuples with single argument [#49789](https://github.com/ClickHouse/ClickHouse/pull/49789) ([Anton Popov](https://github.com/CurtizJ)). +* Fix per-query IO/BACKUPs throttling settings [#49797](https://github.com/ClickHouse/ClickHouse/pull/49797) ([Azat Khuzhin](https://github.com/azat)). +* Fix setting NULL in profile definition [#49831](https://github.com/ClickHouse/ClickHouse/pull/49831) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix a bug with projections and the aggregate_functions_null_for_empty setting (for query_plan_optimize_projection) [#49873](https://github.com/ClickHouse/ClickHouse/pull/49873) ([Amos Bird](https://github.com/amosbird)). +* Fix processing pending batch for Distributed async INSERT after restart [#49884](https://github.com/ClickHouse/ClickHouse/pull/49884) ([Azat Khuzhin](https://github.com/azat)). +* Fix assertion in CacheMetadata::doCleanup [#49914](https://github.com/ClickHouse/ClickHouse/pull/49914) ([Kseniia Sumarokova](https://github.com/kssenii)). +* fix `is_prefix` in OptimizeRegularExpression [#49919](https://github.com/ClickHouse/ClickHouse/pull/49919) ([Han Fei](https://github.com/hanfei1991)). +* Fix metrics `WriteBufferFromS3Bytes`, `WriteBufferFromS3Microseconds` and `WriteBufferFromS3RequestsErrors` [#49930](https://github.com/ClickHouse/ClickHouse/pull/49930) ([Aleksandr Musorin](https://github.com/AVMusorin)). +* Fix IPv6 encoding in protobuf [#49933](https://github.com/ClickHouse/ClickHouse/pull/49933) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix possible Logical error on bad Nullable parsing for text formats [#49960](https://github.com/ClickHouse/ClickHouse/pull/49960) ([Kruglov Pavel](https://github.com/Avogar)). +* Add setting output_format_parquet_compliant_nested_types to produce more compatible Parquet files [#50001](https://github.com/ClickHouse/ClickHouse/pull/50001) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix logical error in stress test "Not enough space to add ..." [#50021](https://github.com/ClickHouse/ClickHouse/pull/50021) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Avoid deadlock when starting table in attach thread of `ReplicatedMergeTree` [#50026](https://github.com/ClickHouse/ClickHouse/pull/50026) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix assert in SpanHolder::finish() with fibers attempt 2 [#50034](https://github.com/ClickHouse/ClickHouse/pull/50034) ([Kruglov Pavel](https://github.com/Avogar)). +* Add proper escaping for DDL OpenTelemetry context serialization [#50045](https://github.com/ClickHouse/ClickHouse/pull/50045) ([Azat Khuzhin](https://github.com/azat)). +* Fix reporting broken projection parts [#50052](https://github.com/ClickHouse/ClickHouse/pull/50052) ([Amos Bird](https://github.com/amosbird)). +* JIT compilation not equals NaN fix [#50056](https://github.com/ClickHouse/ClickHouse/pull/50056) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix crashing in case of Replicated database without arguments [#50058](https://github.com/ClickHouse/ClickHouse/pull/50058) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash with `multiIf` and constant condition and nullable arguments [#50123](https://github.com/ClickHouse/ClickHouse/pull/50123) ([Anton Popov](https://github.com/CurtizJ)). +* Fix invalid index analysis for date related keys [#50153](https://github.com/ClickHouse/ClickHouse/pull/50153) ([Amos Bird](https://github.com/amosbird)). +* do not allow modify order by when there are no order by cols [#50154](https://github.com/ClickHouse/ClickHouse/pull/50154) ([Han Fei](https://github.com/hanfei1991)). +* Fix broken index analysis when binary operator contains a null constant argument [#50177](https://github.com/ClickHouse/ClickHouse/pull/50177) ([Amos Bird](https://github.com/amosbird)). +* clickhouse-client: disallow usage of `--query` and `--queries-file` at the same time [#50210](https://github.com/ClickHouse/ClickHouse/pull/50210) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Fix UB for INTO OUTFILE extensions (APPEND / AND STDOUT) and WATCH EVENTS [#50216](https://github.com/ClickHouse/ClickHouse/pull/50216) ([Azat Khuzhin](https://github.com/azat)). +* Fix skipping spaces at end of row in CustomSeparatedIgnoreSpaces format [#50224](https://github.com/ClickHouse/ClickHouse/pull/50224) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix iceberg metadata parsing [#50232](https://github.com/ClickHouse/ClickHouse/pull/50232) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix nested distributed SELECT in WITH clause [#50234](https://github.com/ClickHouse/ClickHouse/pull/50234) ([Azat Khuzhin](https://github.com/azat)). +* Fix reconnecting of HTTPS session when target host IP was changed [#50240](https://github.com/ClickHouse/ClickHouse/pull/50240) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix msan issue in keyed siphash [#50245](https://github.com/ClickHouse/ClickHouse/pull/50245) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bugs in Poco sockets in non-blocking mode, use true non-blocking sockets [#50252](https://github.com/ClickHouse/ClickHouse/pull/50252) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix checksum calculation for backup entries [#50264](https://github.com/ClickHouse/ClickHouse/pull/50264) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed type conversion from Date/Date32 to DateTime64 when querying with DateTime64 index [#50280](https://github.com/ClickHouse/ClickHouse/pull/50280) ([Lucas Chang](https://github.com/lucas-tubi)). +* Comparison functions NaN fix [#50287](https://github.com/ClickHouse/ClickHouse/pull/50287) ([Maksim Kita](https://github.com/kitaisreal)). +* JIT aggregation nullable key fix [#50291](https://github.com/ClickHouse/ClickHouse/pull/50291) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix clickhouse-local crashing when writing empty Arrow or Parquet output [#50328](https://github.com/ClickHouse/ClickHouse/pull/50328) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix crash when Pool::Entry::disconnect() is called [#50334](https://github.com/ClickHouse/ClickHouse/pull/50334) ([Val Doroshchuk](https://github.com/valbok)). +* Improved fetch part by holding directory lock longer [#50339](https://github.com/ClickHouse/ClickHouse/pull/50339) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix bitShift* functions with both constant arguments [#50343](https://github.com/ClickHouse/ClickHouse/pull/50343) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix Keeper deadlock on exception when preprocessing requests. [#50387](https://github.com/ClickHouse/ClickHouse/pull/50387) ([frinkr](https://github.com/frinkr)). +* Fix hashing of const integer values [#50421](https://github.com/ClickHouse/ClickHouse/pull/50421) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix excessive memory usage for FINAL (due to too much streams usage) [#50429](https://github.com/ClickHouse/ClickHouse/pull/50429) ([Azat Khuzhin](https://github.com/azat)). +* Fix merge_tree_min_rows_for_seek/merge_tree_min_bytes_for_seek for data skipping indexes [#50432](https://github.com/ClickHouse/ClickHouse/pull/50432) ([Azat Khuzhin](https://github.com/azat)). +* Limit the number of in-flight tasks for loading outdated parts [#50450](https://github.com/ClickHouse/ClickHouse/pull/50450) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Keeper fix: apply uncommitted state after snapshot install [#50483](https://github.com/ClickHouse/ClickHouse/pull/50483) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix incorrect constant folding [#50536](https://github.com/ClickHouse/ClickHouse/pull/50536) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix logical error in stress test (Not enough space to add ...) [#50583](https://github.com/ClickHouse/ClickHouse/pull/50583) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix converting Null to LowCardinality(Nullable) in values table function [#50637](https://github.com/ClickHouse/ClickHouse/pull/50637) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix crash in anti/semi join [#50638](https://github.com/ClickHouse/ClickHouse/pull/50638) ([vdimir](https://github.com/vdimir)). +* Revert invalid RegExpTreeDictionary optimization [#50642](https://github.com/ClickHouse/ClickHouse/pull/50642) ([Johann Gan](https://github.com/johanngan)). +* Correctly disable async insert with deduplication when it's not needed [#50663](https://github.com/ClickHouse/ClickHouse/pull/50663) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Build Improvement + +* Fixed Functional Test 00870_t64_codec, 00871_t64_codec_signed, 00872_t64_bit_codec. [#49658](https://github.com/ClickHouse/ClickHouse/pull/49658) ([Sanjam Panda](https://github.com/saitama951)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Fix user MemoryTracker counter in async inserts'. [#47630](https://github.com/ClickHouse/ClickHouse/pull/47630) ([Dmitry Novik](https://github.com/novikd)). +* NO CL ENTRY: 'Revert "Make `Pretty` formats even prettier."'. [#49850](https://github.com/ClickHouse/ClickHouse/pull/49850) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Update first_value.md:remove redundant 's''. [#50331](https://github.com/ClickHouse/ClickHouse/pull/50331) ([sslouis](https://github.com/savezed)). +* NO CL ENTRY: 'Revert "less logs in WriteBufferFromS3"'. [#50390](https://github.com/ClickHouse/ClickHouse/pull/50390) ([Alexander Tokmakov](https://github.com/tavplubix)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Attempt to fix the "system.stack_trace" test [#44627](https://github.com/ClickHouse/ClickHouse/pull/44627) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* rework WriteBufferFromS3, add tests, add abortion [#44869](https://github.com/ClickHouse/ClickHouse/pull/44869) ([Sema Checherinda](https://github.com/CheSema)). +* Rework locking in fs cache [#44985](https://github.com/ClickHouse/ClickHouse/pull/44985) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update ubuntu_ami_for_ci.sh [#47151](https://github.com/ClickHouse/ClickHouse/pull/47151) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Implement status comment [#48468](https://github.com/ClickHouse/ClickHouse/pull/48468) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update curl to 8.0.1 (for CVEs) [#48765](https://github.com/ClickHouse/ClickHouse/pull/48765) ([Boris Kuschel](https://github.com/bkuschel)). +* Fix some tests [#48792](https://github.com/ClickHouse/ClickHouse/pull/48792) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Bug Fix for 02432_s3_parallel_parts_cleanup.sql with zero copy replication [#48865](https://github.com/ClickHouse/ClickHouse/pull/48865) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Add AsyncLoader with dependency tracking and runtime prioritization [#48923](https://github.com/ClickHouse/ClickHouse/pull/48923) ([Sergei Trifonov](https://github.com/serxa)). +* Fix incorrect createColumn call on join clause [#48998](https://github.com/ClickHouse/ClickHouse/pull/48998) ([Ongkong](https://github.com/ongkong)). +* Try fix flaky 01346_alter_enum_partition_key_replicated_zookeeper_long [#49099](https://github.com/ClickHouse/ClickHouse/pull/49099) ([Sergei Trifonov](https://github.com/serxa)). +* Fix possible logical error "Cannot cancel. Either no query sent or already cancelled" [#49106](https://github.com/ClickHouse/ClickHouse/pull/49106) ([Kruglov Pavel](https://github.com/Avogar)). +* Refactor ColumnLowCardinality::cutAndCompact [#49111](https://github.com/ClickHouse/ClickHouse/pull/49111) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix tests with enabled analyzer [#49116](https://github.com/ClickHouse/ClickHouse/pull/49116) ([Dmitry Novik](https://github.com/novikd)). +* Use `SharedMutex` instead of `UpgradableMutex` [#49139](https://github.com/ClickHouse/ClickHouse/pull/49139) ([Sergei Trifonov](https://github.com/serxa)). +* Don't add metadata_version file if it doesn't exist [#49146](https://github.com/ClickHouse/ClickHouse/pull/49146) ([alesapin](https://github.com/alesapin)). +* clearing s3 between tests in a robust way [#49157](https://github.com/ClickHouse/ClickHouse/pull/49157) ([Sema Checherinda](https://github.com/CheSema)). +* Align connect timeout with aws sdk default [#49161](https://github.com/ClickHouse/ClickHouse/pull/49161) ([Nikita Taranov](https://github.com/nickitat)). +* Fix test_encrypted_disk_replication [#49193](https://github.com/ClickHouse/ClickHouse/pull/49193) ([Vitaly Baranov](https://github.com/vitlibar)). +* Allow using function `concat` with `Map` type [#49200](https://github.com/ClickHouse/ClickHouse/pull/49200) ([Anton Popov](https://github.com/CurtizJ)). +* Slight improvements to coordinator logging [#49204](https://github.com/ClickHouse/ClickHouse/pull/49204) ([Raúl Marín](https://github.com/Algunenano)). +* Fix some typos in conversion functions [#49221](https://github.com/ClickHouse/ClickHouse/pull/49221) ([Raúl Marín](https://github.com/Algunenano)). +* CMake: Remove some GCC-specific code [#49224](https://github.com/ClickHouse/ClickHouse/pull/49224) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix oss-fuzz build errors [#49236](https://github.com/ClickHouse/ClickHouse/pull/49236) ([Nikita Taranov](https://github.com/nickitat)). +* Update version after release [#49237](https://github.com/ClickHouse/ClickHouse/pull/49237) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version_date.tsv and changelogs after v23.4.1.1943-stable [#49239](https://github.com/ClickHouse/ClickHouse/pull/49239) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Merge [#24050](https://github.com/ClickHouse/ClickHouse/issues/24050) [#49240](https://github.com/ClickHouse/ClickHouse/pull/49240) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add file name to exception raised during decompression [#49241](https://github.com/ClickHouse/ClickHouse/pull/49241) ([Nikolay Degterinsky](https://github.com/evillique)). +* Disable ISA-L on aarch64 architectures [#49256](https://github.com/ClickHouse/ClickHouse/pull/49256) ([Jordi Villar](https://github.com/jrdi)). +* Add a comment in FileCache.cpp [#49260](https://github.com/ClickHouse/ClickHouse/pull/49260) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix garbage [#48719](https://github.com/ClickHouse/ClickHouse/issues/48719) [#49263](https://github.com/ClickHouse/ClickHouse/pull/49263) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update build for nasm [#49288](https://github.com/ClickHouse/ClickHouse/pull/49288) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix race in `waitForProcessingQueue` [#49302](https://github.com/ClickHouse/ClickHouse/pull/49302) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix stress test [#49309](https://github.com/ClickHouse/ClickHouse/pull/49309) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix 02516_join_with_totals_and_subquery_bug with new analyzer [#49310](https://github.com/ClickHouse/ClickHouse/pull/49310) ([Dmitry Novik](https://github.com/novikd)). +* Fallback auth gh api [#49314](https://github.com/ClickHouse/ClickHouse/pull/49314) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Unpoison stack frame ptrs from libunwind for msan [#49316](https://github.com/ClickHouse/ClickHouse/pull/49316) ([Robert Schulze](https://github.com/rschu1ze)). +* Respect projections in 01600_parts [#49318](https://github.com/ClickHouse/ClickHouse/pull/49318) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* move pipe compute into initializePipeline [#49326](https://github.com/ClickHouse/ClickHouse/pull/49326) ([Konstantin Morozov](https://github.com/k-morozov)). +* Fix compiling average example (suppress -Wframe-larger-than) [#49358](https://github.com/ClickHouse/ClickHouse/pull/49358) ([Azat Khuzhin](https://github.com/azat)). +* Fix join_use_nulls in analyzer [#49359](https://github.com/ClickHouse/ClickHouse/pull/49359) ([vdimir](https://github.com/vdimir)). +* Fix 02680_mysql_ast_logical_err in analyzer [#49362](https://github.com/ClickHouse/ClickHouse/pull/49362) ([vdimir](https://github.com/vdimir)). +* Remove wrong assertion in cache [#49376](https://github.com/ClickHouse/ClickHouse/pull/49376) ([Kseniia Sumarokova](https://github.com/kssenii)). +* A better way of excluding ISA-L on non-x86 [#49378](https://github.com/ClickHouse/ClickHouse/pull/49378) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix decimal aggregates test for s390x [#49382](https://github.com/ClickHouse/ClickHouse/pull/49382) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Move logging one line higher [#49387](https://github.com/ClickHouse/ClickHouse/pull/49387) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Improve CI: status commit, auth for get_gh_api [#49388](https://github.com/ClickHouse/ClickHouse/pull/49388) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix printing hung queries in clickhouse-test. [#49389](https://github.com/ClickHouse/ClickHouse/pull/49389) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Correctly stop CNF convert for too many atomics in new analyzer [#49402](https://github.com/ClickHouse/ClickHouse/pull/49402) ([Antonio Andelic](https://github.com/antonio2368)). +* Remove 02707_complex_query_fails_analyzer test [#49403](https://github.com/ClickHouse/ClickHouse/pull/49403) ([Dmitry Novik](https://github.com/novikd)). +* Update FileSegment.cpp [#49411](https://github.com/ClickHouse/ClickHouse/pull/49411) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Switch Block::NameMap to google::dense_hash_map over HashMap [#49412](https://github.com/ClickHouse/ClickHouse/pull/49412) ([Azat Khuzhin](https://github.com/azat)). +* Slightly reduce inter-header dependencies [#49413](https://github.com/ClickHouse/ClickHouse/pull/49413) ([Azat Khuzhin](https://github.com/azat)). +* Update WithFileName.cpp [#49414](https://github.com/ClickHouse/ClickHouse/pull/49414) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix some assertions failing in stress test [#49415](https://github.com/ClickHouse/ClickHouse/pull/49415) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Correctly cleanup sequential node in ZooKeeperWithFaultInjection [#49418](https://github.com/ClickHouse/ClickHouse/pull/49418) ([vdimir](https://github.com/vdimir)). +* Throw an exception for non-parametric functions in new analyzer [#49419](https://github.com/ClickHouse/ClickHouse/pull/49419) ([Dmitry Novik](https://github.com/novikd)). +* Fix some bad error messages [#49420](https://github.com/ClickHouse/ClickHouse/pull/49420) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update version_date.tsv and changelogs after v23.4.2.11-stable [#49422](https://github.com/ClickHouse/ClickHouse/pull/49422) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Remove trash [#49423](https://github.com/ClickHouse/ClickHouse/pull/49423) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Whitespaces [#49424](https://github.com/ClickHouse/ClickHouse/pull/49424) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove dependency from DB::Context in remote/cache readers [#49426](https://github.com/ClickHouse/ClickHouse/pull/49426) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Merging [#49066](https://github.com/ClickHouse/ClickHouse/issues/49066) (Better error handling during loading of parts) [#49430](https://github.com/ClickHouse/ClickHouse/pull/49430) ([Anton Popov](https://github.com/CurtizJ)). +* all s3-blobs removed when merge aborted, remove part from failed fetch without unlock keper [#49432](https://github.com/ClickHouse/ClickHouse/pull/49432) ([Sema Checherinda](https://github.com/CheSema)). +* Make INSERT do more things in parallel to avoid getting bottlenecked on one thread [#49434](https://github.com/ClickHouse/ClickHouse/pull/49434) ([Michael Kolupaev](https://github.com/al13n321)). +* Make 'exceptions shorter than 30' test less noisy [#49435](https://github.com/ClickHouse/ClickHouse/pull/49435) ([Michael Kolupaev](https://github.com/al13n321)). +* Build fixes for ENABLE_LIBRARIES=OFF [#49437](https://github.com/ClickHouse/ClickHouse/pull/49437) ([Azat Khuzhin](https://github.com/azat)). +* Add image for docker-server jepsen [#49452](https://github.com/ClickHouse/ClickHouse/pull/49452) ([alesapin](https://github.com/alesapin)). +* Follow-up to [#48792](https://github.com/ClickHouse/ClickHouse/issues/48792) [#49458](https://github.com/ClickHouse/ClickHouse/pull/49458) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add method `getCurrentAvailabilityZone` to `AWSEC2MetadataClient` [#49464](https://github.com/ClickHouse/ClickHouse/pull/49464) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add an integration test for `shutdown_wait_unfinished_queries` [#49469](https://github.com/ClickHouse/ClickHouse/pull/49469) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Replace `NO DELAY` with `SYNC` in tests [#49470](https://github.com/ClickHouse/ClickHouse/pull/49470) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Check the PRs body directly in lambda, without rerun. Fix RCE in the CI [#49475](https://github.com/ClickHouse/ClickHouse/pull/49475) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Minor changes for setThreadName [#49476](https://github.com/ClickHouse/ClickHouse/pull/49476) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Static cast std::atomic to uint64_t to serialize. [#49482](https://github.com/ClickHouse/ClickHouse/pull/49482) ([alekar](https://github.com/alekar)). +* Fix logical error in stress test, add some logging [#49491](https://github.com/ClickHouse/ClickHouse/pull/49491) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixes in server jepsen image [#49492](https://github.com/ClickHouse/ClickHouse/pull/49492) ([alesapin](https://github.com/alesapin)). +* Fix UserTimeMicroseconds and SystemTimeMicroseconds descriptions [#49521](https://github.com/ClickHouse/ClickHouse/pull/49521) ([Sergei Trifonov](https://github.com/serxa)). +* Remove garbage from HDFS [#49531](https://github.com/ClickHouse/ClickHouse/pull/49531) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Split ReadWriteBufferFromHTTP.h into .h and .cpp file [#49533](https://github.com/ClickHouse/ClickHouse/pull/49533) ([Michael Kolupaev](https://github.com/al13n321)). +* Remove garbage from Pretty format [#49534](https://github.com/ClickHouse/ClickHouse/pull/49534) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make input_format_parquet_preserve_order imply !parallelize_output_from_storages [#49536](https://github.com/ClickHouse/ClickHouse/pull/49536) ([Michael Kolupaev](https://github.com/al13n321)). +* Remove extra semicolons [#49545](https://github.com/ClickHouse/ClickHouse/pull/49545) ([Bulat Gaifullin](https://github.com/bgaifullin)). +* Fix 00597_push_down_predicate_long for analyzer [#49551](https://github.com/ClickHouse/ClickHouse/pull/49551) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix stress test (assertion 'key_metadata.lock()') [#49554](https://github.com/ClickHouse/ClickHouse/pull/49554) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix writeAnyEscapedString if quote_character is a meta character [#49558](https://github.com/ClickHouse/ClickHouse/pull/49558) ([Robert Schulze](https://github.com/rschu1ze)). +* Add CMake option for BOOST_USE_UCONTEXT [#49564](https://github.com/ClickHouse/ClickHouse/pull/49564) ([ltrk2](https://github.com/ltrk2)). +* Fix 01655_plan_optimizations_optimize_read_in_window_order for analyzer [#49565](https://github.com/ClickHouse/ClickHouse/pull/49565) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix `ThreadPool::wait` [#49572](https://github.com/ClickHouse/ClickHouse/pull/49572) ([Anton Popov](https://github.com/CurtizJ)). +* Query cache: disable for internal queries [#49573](https://github.com/ClickHouse/ClickHouse/pull/49573) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove `test_merge_tree_s3_restore` [#49576](https://github.com/ClickHouse/ClickHouse/pull/49576) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test [#49578](https://github.com/ClickHouse/ClickHouse/pull/49578) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove obsolete test about deprecated feature [#49579](https://github.com/ClickHouse/ClickHouse/pull/49579) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Avoid error found by AST Fuzzer [#49580](https://github.com/ClickHouse/ClickHouse/pull/49580) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix wrong assert [#49581](https://github.com/ClickHouse/ClickHouse/pull/49581) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Flaky test 02723_zookeeper_name.sql [#49592](https://github.com/ClickHouse/ClickHouse/pull/49592) ([Sema Checherinda](https://github.com/CheSema)). +* Query Cache: Safeguard against empty chunks [#49593](https://github.com/ClickHouse/ClickHouse/pull/49593) ([Robert Schulze](https://github.com/rschu1ze)). +* 02723_zookeeper_name: Force a deterministic result order [#49594](https://github.com/ClickHouse/ClickHouse/pull/49594) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove dangerous code (stringstream) [#49595](https://github.com/ClickHouse/ClickHouse/pull/49595) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove some code [#49596](https://github.com/ClickHouse/ClickHouse/pull/49596) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove "locale" [#49597](https://github.com/ClickHouse/ClickHouse/pull/49597) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CMake: Cleanup utils build [#49598](https://github.com/ClickHouse/ClickHouse/pull/49598) ([Robert Schulze](https://github.com/rschu1ze)). +* Follow-up for [#49580](https://github.com/ClickHouse/ClickHouse/issues/49580) [#49604](https://github.com/ClickHouse/ClickHouse/pull/49604) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix typo [#49605](https://github.com/ClickHouse/ClickHouse/pull/49605) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test 01660_system_parts_smoke [#49611](https://github.com/ClickHouse/ClickHouse/pull/49611) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Minor changes [#49612](https://github.com/ClickHouse/ClickHouse/pull/49612) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Follow-up for [#49576](https://github.com/ClickHouse/ClickHouse/issues/49576) [#49615](https://github.com/ClickHouse/ClickHouse/pull/49615) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix error in [#48300](https://github.com/ClickHouse/ClickHouse/issues/48300) [#49616](https://github.com/ClickHouse/ClickHouse/pull/49616) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix typo: "as much slots" -> "as many slots" [#49617](https://github.com/ClickHouse/ClickHouse/pull/49617) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better concurrent parts removal with zero copy [#49619](https://github.com/ClickHouse/ClickHouse/pull/49619) ([alesapin](https://github.com/alesapin)). +* CMake: Remove legacy switch for ccache [#49627](https://github.com/ClickHouse/ClickHouse/pull/49627) ([Robert Schulze](https://github.com/rschu1ze)). +* Try to fix integration test 'test_ssl_cert_authentication' [#49632](https://github.com/ClickHouse/ClickHouse/pull/49632) ([Nikolay Degterinsky](https://github.com/evillique)). +* Unflake 01660_system_parts_smoke [#49633](https://github.com/ClickHouse/ClickHouse/pull/49633) ([Robert Schulze](https://github.com/rschu1ze)). +* Add trash [#49634](https://github.com/ClickHouse/ClickHouse/pull/49634) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove commented code [#49635](https://github.com/ClickHouse/ClickHouse/pull/49635) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add flaky test [#49646](https://github.com/ClickHouse/ClickHouse/pull/49646) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix race in `Context::createCopy` [#49663](https://github.com/ClickHouse/ClickHouse/pull/49663) ([Anton Popov](https://github.com/CurtizJ)). +* Disable 01710_projection_aggregation_in_order.sql [#49667](https://github.com/ClickHouse/ClickHouse/pull/49667) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix flaky 02684_bson.sql [#49674](https://github.com/ClickHouse/ClickHouse/pull/49674) ([Kruglov Pavel](https://github.com/Avogar)). +* Some cache cleanup after rework locking [#49675](https://github.com/ClickHouse/ClickHouse/pull/49675) ([Igor Nikonov](https://github.com/devcrafter)). +* Correctly update log pointer during database replica recovery [#49676](https://github.com/ClickHouse/ClickHouse/pull/49676) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Enable distinct in order after fix [#49636](https://github.com/ClickHouse/ClickHouse/issues/49636) [#49677](https://github.com/ClickHouse/ClickHouse/pull/49677) ([Igor Nikonov](https://github.com/devcrafter)). +* Build fixes for RISCV64 [#49688](https://github.com/ClickHouse/ClickHouse/pull/49688) ([Azat Khuzhin](https://github.com/azat)). +* Add some logging [#49690](https://github.com/ClickHouse/ClickHouse/pull/49690) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix a wrong built generator removal, use `depth=1` [#49692](https://github.com/ClickHouse/ClickHouse/pull/49692) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix member call on null pointer in AST fuzzer [#49696](https://github.com/ClickHouse/ClickHouse/pull/49696) ([Nikolay Degterinsky](https://github.com/evillique)). +* Improve woboq codebrowser pipeline [#49701](https://github.com/ClickHouse/ClickHouse/pull/49701) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Enable `do_not_evict_index_and_mark_files` by default [#49702](https://github.com/ClickHouse/ClickHouse/pull/49702) ([Nikita Taranov](https://github.com/nickitat)). +* Backport fix for UBSan error in musl/logf.c [#49705](https://github.com/ClickHouse/ClickHouse/pull/49705) ([Nikita Taranov](https://github.com/nickitat)). +* Fix flaky test for `kolmogorovSmirnovTest` function [#49710](https://github.com/ClickHouse/ClickHouse/pull/49710) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Update clickhouse-test [#49712](https://github.com/ClickHouse/ClickHouse/pull/49712) ([Alexander Tokmakov](https://github.com/tavplubix)). +* IBM s390x: ip encoding fix [#49713](https://github.com/ClickHouse/ClickHouse/pull/49713) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Remove not used ErrorCodes [#49715](https://github.com/ClickHouse/ClickHouse/pull/49715) ([Sergei Trifonov](https://github.com/serxa)). +* Disable mmap for StorageFile in clickhouse-server [#49717](https://github.com/ClickHouse/ClickHouse/pull/49717) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix typo [#49718](https://github.com/ClickHouse/ClickHouse/pull/49718) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Do not launch workflows for PRs w/o "can be tested" [#49726](https://github.com/ClickHouse/ClickHouse/pull/49726) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Move assertions after logging [#49729](https://github.com/ClickHouse/ClickHouse/pull/49729) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Docs: Fix sidebar label for dictionary table function [#49730](https://github.com/ClickHouse/ClickHouse/pull/49730) ([Robert Schulze](https://github.com/rschu1ze)). +* Do not allocate own buffer in CachedOnDiskReadBufferFromFile when `use_external_buffer == true` [#49733](https://github.com/ClickHouse/ClickHouse/pull/49733) ([Nikita Taranov](https://github.com/nickitat)). +* fix convertation [#49749](https://github.com/ClickHouse/ClickHouse/pull/49749) ([Sema Checherinda](https://github.com/CheSema)). +* fix flaky test 02504_regexp_dictionary_ua_parser [#49753](https://github.com/ClickHouse/ClickHouse/pull/49753) ([Han Fei](https://github.com/hanfei1991)). +* Fix unit test `ExceptionFromWait` [#49755](https://github.com/ClickHouse/ClickHouse/pull/49755) ([Anton Popov](https://github.com/CurtizJ)). +* Add forgotten lock (addition to [#49117](https://github.com/ClickHouse/ClickHouse/issues/49117)) [#49757](https://github.com/ClickHouse/ClickHouse/pull/49757) ([Anton Popov](https://github.com/CurtizJ)). +* Fix typo [#49762](https://github.com/ClickHouse/ClickHouse/pull/49762) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix build of `libfiu` on clang-16 [#49766](https://github.com/ClickHouse/ClickHouse/pull/49766) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update README.md [#49782](https://github.com/ClickHouse/ClickHouse/pull/49782) ([Tyler Hannan](https://github.com/tylerhannan)). +* Analyzer: fix column not found for optimized prewhere with sample by [#49784](https://github.com/ClickHouse/ClickHouse/pull/49784) ([vdimir](https://github.com/vdimir)). +* Typo: demange.cpp --> demangle.cpp [#49799](https://github.com/ClickHouse/ClickHouse/pull/49799) ([Robert Schulze](https://github.com/rschu1ze)). +* Analyzer: apply _CAST to constants only once [#49800](https://github.com/ClickHouse/ClickHouse/pull/49800) ([Dmitry Novik](https://github.com/novikd)). +* Use CLOCK_MONOTONIC_RAW over CLOCK_MONOTONIC on Linux (fixes non monotonic clock) [#49819](https://github.com/ClickHouse/ClickHouse/pull/49819) ([Azat Khuzhin](https://github.com/azat)). +* README.md: 4 --> 5 [#49822](https://github.com/ClickHouse/ClickHouse/pull/49822) ([Robert Schulze](https://github.com/rschu1ze)). +* Allow ASOF JOIN over nullable right column [#49826](https://github.com/ClickHouse/ClickHouse/pull/49826) ([vdimir](https://github.com/vdimir)). +* Make 01533_multiple_nested test more reliable [#49828](https://github.com/ClickHouse/ClickHouse/pull/49828) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* What happens if I remove everything in msan_suppressions? [#49829](https://github.com/ClickHouse/ClickHouse/pull/49829) ([Robert Schulze](https://github.com/rschu1ze)). +* Update README.md [#49832](https://github.com/ClickHouse/ClickHouse/pull/49832) ([AnneClickHouse](https://github.com/AnneClickHouse)). +* Randomize enable_multiple_prewhere_read_steps setting [#49834](https://github.com/ClickHouse/ClickHouse/pull/49834) ([Alexander Gololobov](https://github.com/davenger)). +* Analyzer: do not optimize GROUP BY keys with ROLLUP and CUBE [#49838](https://github.com/ClickHouse/ClickHouse/pull/49838) ([Dmitry Novik](https://github.com/novikd)). +* Clearable hash table and zero values [#49846](https://github.com/ClickHouse/ClickHouse/pull/49846) ([Igor Nikonov](https://github.com/devcrafter)). +* Reset vectorscan reference to an "official" repo [#49848](https://github.com/ClickHouse/ClickHouse/pull/49848) ([Robert Schulze](https://github.com/rschu1ze)). +* Enable few slow clang-tidy checks for clangd [#49855](https://github.com/ClickHouse/ClickHouse/pull/49855) ([Azat Khuzhin](https://github.com/azat)). +* Update QPL docs [#49857](https://github.com/ClickHouse/ClickHouse/pull/49857) ([Robert Schulze](https://github.com/rschu1ze)). +* Small-ish .clang-tidy update [#49859](https://github.com/ClickHouse/ClickHouse/pull/49859) ([Robert Schulze](https://github.com/rschu1ze)). +* Follow-up for clang-tidy [#49861](https://github.com/ClickHouse/ClickHouse/pull/49861) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix "reference to local binding" after fixes for clang-17 [#49874](https://github.com/ClickHouse/ClickHouse/pull/49874) ([Azat Khuzhin](https://github.com/azat)). +* fix typo [#49876](https://github.com/ClickHouse/ClickHouse/pull/49876) ([JackyWoo](https://github.com/JackyWoo)). +* Log with warning if the server was terminated forcefully [#49881](https://github.com/ClickHouse/ClickHouse/pull/49881) ([Azat Khuzhin](https://github.com/azat)). +* Fix some tests [#49889](https://github.com/ClickHouse/ClickHouse/pull/49889) ([Alexander Tokmakov](https://github.com/tavplubix)). +* use chassert in MergeTreeDeduplicationLog to have better log info [#49891](https://github.com/ClickHouse/ClickHouse/pull/49891) ([Han Fei](https://github.com/hanfei1991)). +* Multiple pools support for AsyncLoader [#49893](https://github.com/ClickHouse/ClickHouse/pull/49893) ([Sergei Trifonov](https://github.com/serxa)). +* Fix stack-use-after-scope in resource manager test [#49908](https://github.com/ClickHouse/ClickHouse/pull/49908) ([Sergei Trifonov](https://github.com/serxa)). +* Retry connection expired in test_rename_column/test.py [#49911](https://github.com/ClickHouse/ClickHouse/pull/49911) ([alesapin](https://github.com/alesapin)). +* Try to fix flaky test_distributed_load_balancing tests [#49912](https://github.com/ClickHouse/ClickHouse/pull/49912) ([Kruglov Pavel](https://github.com/Avogar)). +* Remove unused code [#49918](https://github.com/ClickHouse/ClickHouse/pull/49918) ([alesapin](https://github.com/alesapin)). +* Fix flakiness of test_distributed_load_balancing test [#49921](https://github.com/ClickHouse/ClickHouse/pull/49921) ([Azat Khuzhin](https://github.com/azat)). +* Add some logging [#49925](https://github.com/ClickHouse/ClickHouse/pull/49925) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support hardlinking parts transactionally [#49931](https://github.com/ClickHouse/ClickHouse/pull/49931) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix for analyzer: 02377_ optimize_sorting_by_input_stream_properties_e… [#49943](https://github.com/ClickHouse/ClickHouse/pull/49943) ([Igor Nikonov](https://github.com/devcrafter)). +* Follow up to [#49429](https://github.com/ClickHouse/ClickHouse/issues/49429) [#49964](https://github.com/ClickHouse/ClickHouse/pull/49964) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test_ssl_cert_authentication to use urllib3 [#49982](https://github.com/ClickHouse/ClickHouse/pull/49982) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix woboq codebrowser build with -Wno-poison-system-directories [#49992](https://github.com/ClickHouse/ClickHouse/pull/49992) ([Azat Khuzhin](https://github.com/azat)). +* test for [#46128](https://github.com/ClickHouse/ClickHouse/issues/46128) [#49993](https://github.com/ClickHouse/ClickHouse/pull/49993) ([Denny Crane](https://github.com/den-crane)). +* Fix test_insert_same_partition_and_merge failing if one Azure request attempt fails [#49996](https://github.com/ClickHouse/ClickHouse/pull/49996) ([Michael Kolupaev](https://github.com/al13n321)). +* Check return value of `ftruncate` in Keeper [#50020](https://github.com/ClickHouse/ClickHouse/pull/50020) ([Antonio Andelic](https://github.com/antonio2368)). +* Add some assertions [#50025](https://github.com/ClickHouse/ClickHouse/pull/50025) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update 02441_alter_delete_and_drop_column.sql [#50027](https://github.com/ClickHouse/ClickHouse/pull/50027) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Move some common code to common [#50028](https://github.com/ClickHouse/ClickHouse/pull/50028) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add method getCredentials() to S3::Client [#50030](https://github.com/ClickHouse/ClickHouse/pull/50030) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update query_log.md [#50032](https://github.com/ClickHouse/ClickHouse/pull/50032) ([Sergei Trifonov](https://github.com/serxa)). +* Get rid of indirect write buffer in object storages [#50033](https://github.com/ClickHouse/ClickHouse/pull/50033) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Load balancing bugfixes [#50036](https://github.com/ClickHouse/ClickHouse/pull/50036) ([Sergei Trifonov](https://github.com/serxa)). +* Update S3 sdk to v1.11.61 [#50037](https://github.com/ClickHouse/ClickHouse/pull/50037) ([Nikita Taranov](https://github.com/nickitat)). +* Fix 02735_system_zookeeper_connection for DatabaseReplicated [#50047](https://github.com/ClickHouse/ClickHouse/pull/50047) ([Azat Khuzhin](https://github.com/azat)). +* Add more profile events for distributed connections [#50051](https://github.com/ClickHouse/ClickHouse/pull/50051) ([Sergei Trifonov](https://github.com/serxa)). +* FileCache: simple tryReserve() cleanup [#50059](https://github.com/ClickHouse/ClickHouse/pull/50059) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix hashed/sparse_hashed dictionaries max_load_factor upper range [#50065](https://github.com/ClickHouse/ClickHouse/pull/50065) ([Azat Khuzhin](https://github.com/azat)). +* Clearer coordinator log [#50101](https://github.com/ClickHouse/ClickHouse/pull/50101) ([Raúl Marín](https://github.com/Algunenano)). +* Analyzer: Do not execute table functions multiple times [#50105](https://github.com/ClickHouse/ClickHouse/pull/50105) ([Dmitry Novik](https://github.com/novikd)). +* Update default settings for Replicated database [#50108](https://github.com/ClickHouse/ClickHouse/pull/50108) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Make async prefetched buffer work with arbitrary impl [#50109](https://github.com/ClickHouse/ClickHouse/pull/50109) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update github.com/distribution/distribution [#50114](https://github.com/ClickHouse/ClickHouse/pull/50114) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Docs: Update clickhouse-local arguments [#50138](https://github.com/ClickHouse/ClickHouse/pull/50138) ([Robert Schulze](https://github.com/rschu1ze)). +* Change fields destruction order in AsyncTaskExecutor [#50151](https://github.com/ClickHouse/ClickHouse/pull/50151) ([Kruglov Pavel](https://github.com/Avogar)). +* Follow-up to [#49889](https://github.com/ClickHouse/ClickHouse/issues/49889) [#50152](https://github.com/ClickHouse/ClickHouse/pull/50152) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Clarification comment on retries controller behavior [#50155](https://github.com/ClickHouse/ClickHouse/pull/50155) ([Igor Nikonov](https://github.com/devcrafter)). +* Switch to upstream repository of vectorscan [#50159](https://github.com/ClickHouse/ClickHouse/pull/50159) ([Azat Khuzhin](https://github.com/azat)). +* Refactor lambdas, prepare to prio runners [#50160](https://github.com/ClickHouse/ClickHouse/pull/50160) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Speed-up the shellcheck with parallel xargs [#50164](https://github.com/ClickHouse/ClickHouse/pull/50164) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update an exception message [#50180](https://github.com/ClickHouse/ClickHouse/pull/50180) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Upgrade boost submodule [#50188](https://github.com/ClickHouse/ClickHouse/pull/50188) ([ltrk2](https://github.com/ltrk2)). +* Implement a uniform way to query processor core IDs [#50190](https://github.com/ClickHouse/ClickHouse/pull/50190) ([ltrk2](https://github.com/ltrk2)). +* Don't replicate delete through DDL worker if there is just 1 shard [#50193](https://github.com/ClickHouse/ClickHouse/pull/50193) ([Alexander Gololobov](https://github.com/davenger)). +* Fix codebrowser by using clang-15 image [#50197](https://github.com/ClickHouse/ClickHouse/pull/50197) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add comments to build reports [#50200](https://github.com/ClickHouse/ClickHouse/pull/50200) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Automatic backports of important fixes to cloud-release [#50202](https://github.com/ClickHouse/ClickHouse/pull/50202) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Unify priorities: lower value means higher priority [#50205](https://github.com/ClickHouse/ClickHouse/pull/50205) ([Sergei Trifonov](https://github.com/serxa)). +* Use transactions for encrypted disks [#50206](https://github.com/ClickHouse/ClickHouse/pull/50206) ([alesapin](https://github.com/alesapin)). +* Get detailed error instead of unknown error for function test [#50207](https://github.com/ClickHouse/ClickHouse/pull/50207) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* README.md: Remove Berlin Meetup from upcoming events [#50218](https://github.com/ClickHouse/ClickHouse/pull/50218) ([Robert Schulze](https://github.com/rschu1ze)). +* Minor adjustment of clickhouse-client/local parameter docs [#50219](https://github.com/ClickHouse/ClickHouse/pull/50219) ([Robert Schulze](https://github.com/rschu1ze)). +* Unify priorities: rework IO scheduling subsystem [#50231](https://github.com/ClickHouse/ClickHouse/pull/50231) ([Sergei Trifonov](https://github.com/serxa)). +* Add new metrics BrokenDistributedBytesToInsert/DistributedBytesToInsert [#50238](https://github.com/ClickHouse/ClickHouse/pull/50238) ([Azat Khuzhin](https://github.com/azat)). +* Fix URL in backport comment [#50241](https://github.com/ClickHouse/ClickHouse/pull/50241) ([pufit](https://github.com/pufit)). +* Fix `02535_max_parallel_replicas_custom_key` [#50242](https://github.com/ClickHouse/ClickHouse/pull/50242) ([Antonio Andelic](https://github.com/antonio2368)). +* Fixes for MergeTree with readonly disks [#50244](https://github.com/ClickHouse/ClickHouse/pull/50244) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Yet another refactoring [#50257](https://github.com/ClickHouse/ClickHouse/pull/50257) ([Anton Popov](https://github.com/CurtizJ)). +* Unify priorities: rework AsyncLoader [#50272](https://github.com/ClickHouse/ClickHouse/pull/50272) ([Sergei Trifonov](https://github.com/serxa)). +* buffers d-tor finalize free [#50275](https://github.com/ClickHouse/ClickHouse/pull/50275) ([Sema Checherinda](https://github.com/CheSema)). +* Fix 02767_into_outfile_extensions_msan under analyzer [#50290](https://github.com/ClickHouse/ClickHouse/pull/50290) ([Azat Khuzhin](https://github.com/azat)). +* QPL: Add a comment about isal [#50308](https://github.com/ClickHouse/ClickHouse/pull/50308) ([Robert Schulze](https://github.com/rschu1ze)). +* Avoid clang 15 crash [#50310](https://github.com/ClickHouse/ClickHouse/pull/50310) ([Raúl Marín](https://github.com/Algunenano)). +* Cleanup Annoy index [#50312](https://github.com/ClickHouse/ClickHouse/pull/50312) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix flaky `AsyncLoader.StaticPriorities` unit test [#50313](https://github.com/ClickHouse/ClickHouse/pull/50313) ([Sergei Trifonov](https://github.com/serxa)). +* Update gtest_async_loader.cpp [#50317](https://github.com/ClickHouse/ClickHouse/pull/50317) ([Nikita Taranov](https://github.com/nickitat)). +* Fix IS (NOT) NULL operator priority [#50327](https://github.com/ClickHouse/ClickHouse/pull/50327) ([Nikolay Degterinsky](https://github.com/evillique)). +* Update README.md [#50340](https://github.com/ClickHouse/ClickHouse/pull/50340) ([Tyler Hannan](https://github.com/tylerhannan)). +* do not fix the event list in test [#50342](https://github.com/ClickHouse/ClickHouse/pull/50342) ([Sema Checherinda](https://github.com/CheSema)). +* less logs in WriteBufferFromS3 [#50347](https://github.com/ClickHouse/ClickHouse/pull/50347) ([Sema Checherinda](https://github.com/CheSema)). +* Remove legacy install scripts superseded by universal.sh [#50360](https://github.com/ClickHouse/ClickHouse/pull/50360) ([Robert Schulze](https://github.com/rschu1ze)). +* Fail perf tests when too many queries slowed down [#50361](https://github.com/ClickHouse/ClickHouse/pull/50361) ([Nikita Taranov](https://github.com/nickitat)). +* Fix after [#50109](https://github.com/ClickHouse/ClickHouse/issues/50109) [#50362](https://github.com/ClickHouse/ClickHouse/pull/50362) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix log message [#50363](https://github.com/ClickHouse/ClickHouse/pull/50363) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Compare functions NaN update test [#50366](https://github.com/ClickHouse/ClickHouse/pull/50366) ([Maksim Kita](https://github.com/kitaisreal)). +* Add re-creation for cherry-pick PRs [#50373](https://github.com/ClickHouse/ClickHouse/pull/50373) ([pufit](https://github.com/pufit)). +* Without applying `prepareRightBlock` will cause mismatch block structrue [#50383](https://github.com/ClickHouse/ClickHouse/pull/50383) ([lgbo](https://github.com/lgbo-ustc)). +* fix hung in unit tests [#50391](https://github.com/ClickHouse/ClickHouse/pull/50391) ([Sema Checherinda](https://github.com/CheSema)). +* Fix poll timeout in MaterializedMySQL [#50392](https://github.com/ClickHouse/ClickHouse/pull/50392) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Compile aggregate expressions enable by default [#50401](https://github.com/ClickHouse/ClickHouse/pull/50401) ([Maksim Kita](https://github.com/kitaisreal)). +* Update app.py [#50407](https://github.com/ClickHouse/ClickHouse/pull/50407) ([Nikita Taranov](https://github.com/nickitat)). +* reuse s3_mocks, rewrite test test_paranoid_check_in_logs [#50408](https://github.com/ClickHouse/ClickHouse/pull/50408) ([Sema Checherinda](https://github.com/CheSema)). +* test for [#42610](https://github.com/ClickHouse/ClickHouse/issues/42610) [#50409](https://github.com/ClickHouse/ClickHouse/pull/50409) ([Denny Crane](https://github.com/den-crane)). +* Remove something [#50411](https://github.com/ClickHouse/ClickHouse/pull/50411) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Mark the builds without results as pending [#50415](https://github.com/ClickHouse/ClickHouse/pull/50415) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Revert "Fix msan issue in keyed siphash" [#50426](https://github.com/ClickHouse/ClickHouse/pull/50426) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Revert "Revert "less logs in WriteBufferFromS3" ([#50390](https://github.com/ClickHouse/ClickHouse/issues/50390))" [#50444](https://github.com/ClickHouse/ClickHouse/pull/50444) ([Sema Checherinda](https://github.com/CheSema)). +* Paranoid fix for removing parts from ZooKeeper [#50448](https://github.com/ClickHouse/ClickHouse/pull/50448) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add timeout for unit tests [#50449](https://github.com/ClickHouse/ClickHouse/pull/50449) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Changes related to an internal feature [#50453](https://github.com/ClickHouse/ClickHouse/pull/50453) ([Michael Kolupaev](https://github.com/al13n321)). +* Don't crash if config doesn't have logger section [#50455](https://github.com/ClickHouse/ClickHouse/pull/50455) ([Michael Kolupaev](https://github.com/al13n321)). +* Update function docs [#50466](https://github.com/ClickHouse/ClickHouse/pull/50466) ([Robert Schulze](https://github.com/rschu1ze)). +* Revert "make filter push down through cross join" [#50467](https://github.com/ClickHouse/ClickHouse/pull/50467) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add some assertions [#50470](https://github.com/ClickHouse/ClickHouse/pull/50470) ([Kseniia Sumarokova](https://github.com/kssenii)). +* CI: Enable aspell on nested docs [#50476](https://github.com/ClickHouse/ClickHouse/pull/50476) ([Robert Schulze](https://github.com/rschu1ze)). +* Try fix flaky test test_async_query_sending [#50480](https://github.com/ClickHouse/ClickHouse/pull/50480) ([Kruglov Pavel](https://github.com/Avogar)). +* Disable 00534_functions_bad_arguments with msan [#50481](https://github.com/ClickHouse/ClickHouse/pull/50481) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Typos: Follow-up to [#50476](https://github.com/ClickHouse/ClickHouse/issues/50476) [#50482](https://github.com/ClickHouse/ClickHouse/pull/50482) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove unneeded Keeper test [#50485](https://github.com/ClickHouse/ClickHouse/pull/50485) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix KeyError in cherry-pick [#50493](https://github.com/ClickHouse/ClickHouse/pull/50493) ([pufit](https://github.com/pufit)). +* Make typeid_cast for pointers noexcept [#50495](https://github.com/ClickHouse/ClickHouse/pull/50495) ([Sergey Kazmin ](https://github.com/yerseg)). +* less traces in logs [#50518](https://github.com/ClickHouse/ClickHouse/pull/50518) ([Sema Checherinda](https://github.com/CheSema)). +* Implement endianness-independent serialization for UUID [#50519](https://github.com/ClickHouse/ClickHouse/pull/50519) ([ltrk2](https://github.com/ltrk2)). +* Remove strange object storage methods [#50521](https://github.com/ClickHouse/ClickHouse/pull/50521) ([alesapin](https://github.com/alesapin)). +* Fix low quality code around metadata in RocksDB (experimental feature never used in production) [#50527](https://github.com/ClickHouse/ClickHouse/pull/50527) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Function if constant folding [#50529](https://github.com/ClickHouse/ClickHouse/pull/50529) ([Maksim Kita](https://github.com/kitaisreal)). +* Add profile events for fs cache eviction [#50533](https://github.com/ClickHouse/ClickHouse/pull/50533) ([Kseniia Sumarokova](https://github.com/kssenii)). +* QueryNode small fix [#50535](https://github.com/ClickHouse/ClickHouse/pull/50535) ([Maksim Kita](https://github.com/kitaisreal)). +* Control memory usage in generateRandom [#50538](https://github.com/ClickHouse/ClickHouse/pull/50538) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable skim (Rust library) under memory sanitizer [#50539](https://github.com/ClickHouse/ClickHouse/pull/50539) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* MSan support for Rust [#50541](https://github.com/ClickHouse/ClickHouse/pull/50541) ([Azat Khuzhin](https://github.com/azat)). +* Make 01565_query_loop_after_client_error slightly more robust [#50542](https://github.com/ClickHouse/ClickHouse/pull/50542) ([Azat Khuzhin](https://github.com/azat)). +* Resize BufferFromVector underlying vector only pos_offset == vector.size() [#50546](https://github.com/ClickHouse/ClickHouse/pull/50546) ([auxten](https://github.com/auxten)). +* Add async iteration to object storage [#50548](https://github.com/ClickHouse/ClickHouse/pull/50548) ([alesapin](https://github.com/alesapin)). +* skip extracting darwin toolchain in builder when unncessary [#50550](https://github.com/ClickHouse/ClickHouse/pull/50550) ([SuperDJY](https://github.com/cmsxbc)). +* Remove flaky test [#50558](https://github.com/ClickHouse/ClickHouse/pull/50558) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Revert "Disable skim (Rust library) under memory sanitizer" [#50574](https://github.com/ClickHouse/ClickHouse/pull/50574) ([Azat Khuzhin](https://github.com/azat)). +* Analyzer: fix 01487_distributed_in_not_default_db [#50587](https://github.com/ClickHouse/ClickHouse/pull/50587) ([Dmitry Novik](https://github.com/novikd)). +* Fix commit for DiskObjectStorage [#50599](https://github.com/ClickHouse/ClickHouse/pull/50599) ([alesapin](https://github.com/alesapin)). +* Fix Jepsen runs in PRs [#50615](https://github.com/ClickHouse/ClickHouse/pull/50615) ([Antonio Andelic](https://github.com/antonio2368)). +* Revert incorrect optimizations [#50629](https://github.com/ClickHouse/ClickHouse/pull/50629) ([Raúl Marín](https://github.com/Algunenano)). +* Disable 01676_clickhouse_client_autocomplete under UBSan [#50636](https://github.com/ClickHouse/ClickHouse/pull/50636) ([Nikita Taranov](https://github.com/nickitat)). +* Merging [#50329](https://github.com/ClickHouse/ClickHouse/issues/50329) [#50660](https://github.com/ClickHouse/ClickHouse/pull/50660) ([Anton Popov](https://github.com/CurtizJ)). +* Revert "date_trunc function to always return DateTime type" [#50670](https://github.com/ClickHouse/ClickHouse/pull/50670) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix flaky test 02461_prewhere_row_level_policy_lightweight_delete [#50674](https://github.com/ClickHouse/ClickHouse/pull/50674) ([Alexander Gololobov](https://github.com/davenger)). +* Fix asan issue with analyzer and prewhere [#50685](https://github.com/ClickHouse/ClickHouse/pull/50685) ([Alexander Gololobov](https://github.com/davenger)). +* Catch issues with dockerd during the build [#50700](https://github.com/ClickHouse/ClickHouse/pull/50700) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Temporarily disable annoy index tests (flaky for analyzer) [#50714](https://github.com/ClickHouse/ClickHouse/pull/50714) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix assertion from stress test [#50718](https://github.com/ClickHouse/ClickHouse/pull/50718) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky unit test [#50719](https://github.com/ClickHouse/ClickHouse/pull/50719) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Show correct sharing state in system.query_cache [#50728](https://github.com/ClickHouse/ClickHouse/pull/50728) ([Robert Schulze](https://github.com/rschu1ze)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 653a0cd5388..9704c68be54 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.5.1.3174-stable 2023-06-09 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 v23.3.2.37-lts 2023-04-22 From b740a08b6e508ccee08efc7e2ca83e8d7192e3ef Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 9 Jun 2023 15:05:23 +0300 Subject: [PATCH 1424/2223] Fix the docs --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index dad3cfb4cc5..8e186844c93 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1343,7 +1343,7 @@ parseDateTimeBestEffort(time_string [, time_zone]) - A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `DD/MM/YYYY`, `DD-MM-YY` etc. - A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case `YYYY-MM` are substituted as `2000-01`. - A string that includes the date and time along with time zone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. -- A string that includes the date and time in a [syslog](https://datatracker.ietf.org/doc/html/rfc3164) datetime format. For example, `Jun 9 14:20:32`. +- A string that includes the date and time in the [syslog timestamp](https://datatracker.ietf.org/doc/html/rfc3164) format. For example, `Jun 9 14:20:32`. For all of the formats with separator the function parses months names expressed by their full name or by the first three letters of a month name. Examples: `24/DEC/18`, `24-Dec-18`, `01-September-2018`. diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 03e3adfbdca..93ca6b410c8 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -1022,7 +1022,7 @@ parseDateTimeBestEffort(time_string[, time_zone]) - Строка с датой, но без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `DD/MM/YYYY`, `DD-MM-YY` и т.д. - Строка с временем, и с днём: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` принимается равным `2000-01`. - Строка, содержащая дату и время вместе с информацией о часовом поясе: `YYYY-MM-DD hh:mm:ss ±h:mm`, и т.д. Например, `2020-12-12 17:36:00 -5:00`. -- Строка, содержащая дату и время в формате [syslog](https://datatracker.ietf.org/doc/html/rfc3164). Например, `Jun 9 14:20:32`. +- Строка, содержащая дату и время в формате [syslog timestamp](https://datatracker.ietf.org/doc/html/rfc3164). Например, `Jun 9 14:20:32`. Для всех форматов с разделителями функция распознаёт названия месяцев, выраженных в виде полного англоязычного имени месяца или в виде первых трёх символов имени месяца. Примеры: `24/DEC/18`, `24-Dec-18`, `01-September-2018`. From d0c2c1dbad9da978246fe6c9105a62a972041cd8 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 9 Jun 2023 12:06:43 +0000 Subject: [PATCH 1425/2223] Add test and reorder if's --- src/Functions/FunctionsConversion.h | 49 ++++++++++--------- .../01556_accurate_cast_or_null.reference | 1 + .../01556_accurate_cast_or_null.sql | 1 + 3 files changed, 27 insertions(+), 24 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index ea8efada21d..e44a3bdaa2e 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2866,36 +2866,37 @@ private: using LeftDataType = typename Types::LeftType; using RightDataType = typename Types::RightType; - if constexpr (IsDataTypeNumber && IsDataTypeNumber) + if constexpr (IsDataTypeNumber) { - if (wrapper_cast_type == CastType::accurate) + if constexpr (IsDataTypeNumber) { - result_column = ConvertImpl::execute( - arguments, result_type, input_rows_count, AccurateConvertStrategyAdditions()); - } - else - { - result_column = ConvertImpl::execute( - arguments, result_type, input_rows_count, AccurateOrNullConvertStrategyAdditions()); + if (wrapper_cast_type == CastType::accurate) + { + result_column = ConvertImpl::execute( + arguments, result_type, input_rows_count, AccurateConvertStrategyAdditions()); + } + else + { + result_column = ConvertImpl::execute( + arguments, result_type, input_rows_count, AccurateOrNullConvertStrategyAdditions()); + } + return true; } - return true; - } - - if constexpr (IsDataTypeNumber - && (std::is_same_v || std::is_same_v)) - { - if (wrapper_cast_type == CastType::accurate) + if constexpr (std::is_same_v || std::is_same_v) { - result_column = ConvertImpl::template execute( - arguments, result_type, input_rows_count); + if (wrapper_cast_type == CastType::accurate) + { + result_column = ConvertImpl::template execute( + arguments, result_type, input_rows_count); + } + else + { + result_column = ConvertImpl::template execute( + arguments, result_type, input_rows_count); + } + return true; } - else - { - result_column = ConvertImpl::template execute( - arguments, result_type, input_rows_count); - } - return true; } return false; diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference index 3bff125068a..31a9c37421e 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference @@ -40,4 +40,5 @@ \N \N 2023-05-30 +2149-06-06 1970-01-20 diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql index 3266198d930..f00f6ef837f 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql @@ -47,4 +47,5 @@ SELECT accurateCastOrNull(-1, 'Date'); SELECT accurateCastOrNull(5000000000, 'Date'); SELECT accurateCastOrNull('1xxx', 'Date'); SELECT accurateCastOrNull('2023-05-30', 'Date'); +SELECT accurateCastOrNull('2180-01-01', 'Date'); SELECT accurateCastOrNull(19, 'Date'); From a21bd4ec62825172d859424be0687e127d36b4c0 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 9 Jun 2023 15:33:51 +0300 Subject: [PATCH 1426/2223] Elucidate the syslog case in the documentation --- docs/en/sql-reference/functions/type-conversion-functions.md | 3 ++- docs/ru/sql-reference/functions/type-conversion-functions.md | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 8e186844c93..e62cf89a6b2 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1341,11 +1341,12 @@ parseDateTimeBestEffort(time_string [, time_zone]) - A string containing 9..10 digit [unix timestamp](https://en.wikipedia.org/wiki/Unix_time). - A string with a date and a time component: `YYYYMMDDhhmmss`, `DD/MM/YYYY hh:mm:ss`, `DD-MM-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. - A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `DD/MM/YYYY`, `DD-MM-YY` etc. -- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case `YYYY-MM` are substituted as `2000-01`. +- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case `MM` is substituted by `01`. - A string that includes the date and time along with time zone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. - A string that includes the date and time in the [syslog timestamp](https://datatracker.ietf.org/doc/html/rfc3164) format. For example, `Jun 9 14:20:32`. For all of the formats with separator the function parses months names expressed by their full name or by the first three letters of a month name. Examples: `24/DEC/18`, `24-Dec-18`, `01-September-2018`. +If the year is not specified, it is considered to be equal to the current year. If the resulting date and time happens to be ahead of the current moment even by a second, the current year is substituted by the previous one. **Returned value** diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 93ca6b410c8..6de55757b64 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -1020,11 +1020,12 @@ parseDateTimeBestEffort(time_string[, time_zone]) - [Unix timestamp](https://ru.wikipedia.org/wiki/Unix-время) в строковом представлении. 9 или 10 символов. - Строка с датой и временем: `YYYYMMDDhhmmss`, `DD/MM/YYYY hh:mm:ss`, `DD-MM-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. - Строка с датой, но без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `DD/MM/YYYY`, `DD-MM-YY` и т.д. -- Строка с временем, и с днём: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` принимается равным `2000-01`. +- Строка с временем, и с днём: `DD`, `DD hh`, `DD hh:mm`. В этом случае `MM` принимается равным `01`. - Строка, содержащая дату и время вместе с информацией о часовом поясе: `YYYY-MM-DD hh:mm:ss ±h:mm`, и т.д. Например, `2020-12-12 17:36:00 -5:00`. - Строка, содержащая дату и время в формате [syslog timestamp](https://datatracker.ietf.org/doc/html/rfc3164). Например, `Jun 9 14:20:32`. Для всех форматов с разделителями функция распознаёт названия месяцев, выраженных в виде полного англоязычного имени месяца или в виде первых трёх символов имени месяца. Примеры: `24/DEC/18`, `24-Dec-18`, `01-September-2018`. +Если год не указан, вместо него подставляется текущий год. Если в результате получается будущее время, хотя бы на одну секунду впереди, текущий год заменяется на прошлый. **Возвращаемое значение** From 4009b5fef1db916eb6baa265f305ef412cffc8e3 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Fri, 9 Jun 2023 14:34:05 +0200 Subject: [PATCH 1427/2223] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a5b6dd85941..09c832f0c04 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,7 +12,7 @@ #### Upgrade Notes * Compress marks and primary key by default. It significantly reduces the cold query time. Upgrade notes: the support for compressed marks and primary key has been added in version 22.9. If you turned on compressed marks or primary key or installed version 23.5 or newer, which has compressed marks or primary key on by default, you will not be able to downgrade to version 22.8 or earlier. You can also explicitly disable compressed marks or primary keys by specifying the `compress_marks` and `compress_primary_key` settings in the `` section of the server configuration file. **Upgrade notes:** If you upgrade from versions prior to 22.9, you should either upgrade all replicas at once or disable the compression before upgrade, or upgrade through an intermediate version, where the compressed marks are supported but not enabled by default, such as 23.3. [#42587](https://github.com/ClickHouse/ClickHouse/pull/42587) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Make local object storage work consistently with s3 object storage, fix problem with append (closes [#48465](https://github.com/ClickHouse/ClickHouse/issues/48465)), make it configurable as independent storage. The change is backward incompatible because the cache on top of local object storage is not incompatible to previous versions. [#48791](https://github.com/ClickHouse/ClickHouse/pull/48791) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Make local object storage work consistently with s3 object storage, fix problem with append (closes [#48465](https://github.com/ClickHouse/ClickHouse/issues/48465)), make it configurable as independent storage. The change is backward incompatible because the cache on top of local object storage is not compatible to previous versions. [#48791](https://github.com/ClickHouse/ClickHouse/pull/48791) ([Kseniia Sumarokova](https://github.com/kssenii)). * The experimental feature "in-memory data parts" is removed. The data format is still supported, but the settings are no-op, and compact or wide parts will be used instead. This closes [#45409](https://github.com/ClickHouse/ClickHouse/issues/45409). [#49429](https://github.com/ClickHouse/ClickHouse/pull/49429) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Changed default values of settings `parallelize_output_from_storages` and `input_format_parquet_preserve_order`. This allows ClickHouse to reorder rows when reading from files (e.g. CSV or Parquet), greatly improving performance in many cases. To restore the old behavior of preserving order, use `parallelize_output_from_storages = 0`, `input_format_parquet_preserve_order = 1`. [#49479](https://github.com/ClickHouse/ClickHouse/pull/49479) ([Michael Kolupaev](https://github.com/al13n321)). * Make projections production-ready. Add the `optimize_use_projections` setting to control whether the projections will be selected for SELECT queries. The setting `allow_experimental_projection_optimization` is obsolete and does nothing. [#49719](https://github.com/ClickHouse/ClickHouse/pull/49719) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From 056ca4f555fbbf4463de5be8642a2c01b6759192 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 24 May 2023 13:26:15 -0400 Subject: [PATCH 1428/2223] Add extensive testing cases for deflate qpl codec --- .../sql-reference/statements/create/table.md | 2 +- src/Client/Connection.cpp | 2 +- src/Compression/CompressionCodecDeflateQpl.h | 3 +- src/Compression/CompressionFactory.h | 4 +- .../CompressionFactoryAdditions.cpp | 14 ++-- src/Compression/ICompressionCodec.h | 3 + src/Core/Settings.h | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 3 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/AlterCommands.cpp | 8 +-- src/Storages/ColumnsDescription.cpp | 2 +- src/Storages/Distributed/DistributedSink.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- .../deflateqpl_compression_by_default.xml | 11 ++++ .../configs/enable_deflateqpl_codec.xml | 7 ++ .../test_non_default_compression/test.py | 65 ++++++++++++++++++- ...04_test_alter_compression_codecs.reference | 31 ++++++--- .../00804_test_alter_compression_codecs.sql | 28 +++++--- 18 files changed, 153 insertions(+), 37 deletions(-) create mode 100644 tests/integration/test_non_default_compression/configs/deflateqpl_compression_by_default.xml create mode 100644 tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index de44a001472..b0865ad2896 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -380,7 +380,7 @@ High compression levels are useful for asymmetric scenarios, like compress once, `DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library. Some limitations apply: -- DEFLATE_QPL is experimental and can only be used after setting configuration parameter `allow_experimental_codecs=1`. +- DEFLATE_QPL is disabled by default and can only be used after setting configuration parameter `enable_qpl_deflate=1`. - DEFLATE_QPL requires a ClickHouse build compiled with SSE 4.2 instructions (by default, this is the case). Refer to [Build Clickhouse with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Build-Clickhouse-with-DEFLATE_QPL) for more details. - DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [Benchmark with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Run-Benchmark-with-DEFLATE_QPL) for more details. - DEFLATE_QPL-compressed data can only be transferred between ClickHouse nodes compiled with SSE 4.2 enabled. diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 2350a5039ab..68bc3b39a56 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -588,7 +588,7 @@ void Connection::sendQuery( if (method == "ZSTD") level = settings->network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs); + CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs, settings->enable_qpl_deflate); compression_codec = CompressionCodecFactory::instance().get(method, level); } else diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 7a1a764295d..13aa8733b54 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -98,7 +98,8 @@ public: protected: bool isCompression() const override { return true; } bool isGenericCompression() const override { return true; } - bool isExperimental() const override { return true; } + bool isExperimental() const override { return false; } + bool isDeflateQplCompression() const override { return true; } UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index a4451f9ed2e..1fdaf4f1c71 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -40,10 +40,10 @@ public: CompressionCodecPtr getDefaultCodec() const; /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) - ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs) const; + ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const; /// Validate codecs AST specified by user - void validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const; + void validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const; /// Get codec by AST and possible column_type. Some codecs can use /// information about type to improve inner settings, but every codec should diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp index 978a0fe5069..2630326238a 100644 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes void CompressionCodecFactory::validateCodec( - const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const + const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const { if (family_name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Compression codec name cannot be empty"); @@ -43,13 +43,13 @@ void CompressionCodecFactory::validateCodec( { auto literal = std::make_shared(static_cast(*level)); validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), - {}, sanity_check, allow_experimental_codecs); + {}, sanity_check, allow_experimental_codecs, enable_qpl_deflate); } else { auto identifier = std::make_shared(Poco::toUpper(family_name)); validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), - {}, sanity_check, allow_experimental_codecs); + {}, sanity_check, allow_experimental_codecs, enable_qpl_deflate); } } @@ -77,7 +77,7 @@ bool innerDataTypeIsFloat(const DataTypePtr & type) } ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( - const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs) const + const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const { if (const auto * func = ast->as()) { @@ -159,6 +159,12 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( " You can enable it with the 'allow_experimental_codecs' setting.", codec_family_name); + if (!enable_qpl_deflate && result_codec->isDeflateQplCompression()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Codec {} is disabled by default." + " You can enable it with the 'enable_qpl_deflate' setting.", + codec_family_name); + codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); } diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index 44835ac19cb..d92ad3fc718 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -112,6 +112,9 @@ public: /// If it does nothing. virtual bool isNone() const { return false; } + /// This is a knob for Deflate QPL codec. + virtual bool isDeflateQplCompression() const { return false; } + protected: /// This is used for fuzz testing friend int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 464b9168a4c..c6a2069e6ae 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -319,6 +319,7 @@ class IColumn; M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ + M(Bool, enable_qpl_deflate, false, "If it is set to true, allow to use deflate_qpl for compression.", 0) \ M(UInt64, query_profiler_real_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, query_profiler_cpu_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ab9e1fb04d6..5c22b46b360 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -571,6 +571,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( bool sanity_check_compression_codecs = !attach && !context_->getSettingsRef().allow_suspicious_codecs; bool allow_experimental_codecs = attach || context_->getSettingsRef().allow_experimental_codecs; + bool enable_qpl_deflate = attach || context_->getSettingsRef().enable_qpl_deflate; ColumnsDescription res; auto name_type_it = column_names_and_types.begin(); @@ -631,7 +632,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.default_specifier == "ALIAS") throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot specify codec for column type ALIAS"); column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( - col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs); + col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_qpl_deflate); } if (col_decl.ttl) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1ded7d97248..96c585e7d16 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1775,7 +1775,7 @@ void TCPHandler::initBlockOutput(const Block & block) if (state.compression == Protocol::Compression::Enable) { - CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs); + CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs, query_settings.enable_qpl_deflate); state.maybe_compressed_out = std::make_shared( *out, CompressionCodecFactory::instance().get(method, level)); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 5fd823b9e01..ecbddfc3e2a 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -388,7 +388,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) column.comment = *comment; if (codec) - column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type, false, true); + column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type, false, true, true); column.ttl = ttl; @@ -429,7 +429,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) else { if (codec) - column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false, true); + column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false, true, true); if (comment) column.comment = *comment; @@ -1067,7 +1067,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const "this column name is reserved for lightweight delete feature", backQuote(column_name)); if (command.codec) - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate); all_columns.add(ColumnDescription(column_name, command.data_type)); } @@ -1093,7 +1093,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { if (all_columns.hasAlias(column_name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot specify codec for column type ALIAS"); - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate); } auto column_default = all_columns.getDefault(column_name); if (column_default) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 8eabae7929c..045afd7e6e6 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -130,7 +130,7 @@ void ColumnDescription::readText(ReadBuffer & buf) comment = col_ast->comment->as().value.get(); if (col_ast->codec) - codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false, true); + codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(col_ast->codec, type, false, true, true); if (col_ast->ttl) ttl = col_ast->ttl; diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 720a951299a..ce1dbde8eae 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -733,7 +733,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const if (compression_method == "ZSTD") compression_level = settings.network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs); + CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enable_qpl_deflate); CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level); /// tmp directory is used to ensure atomicity of transactions diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index e1a80800630..f5209cbdff6 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -285,7 +285,7 @@ TTLDescription TTLDescription::getTTLFromAST( { result.recompression_codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( - ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs); + ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate); } } diff --git a/tests/integration/test_non_default_compression/configs/deflateqpl_compression_by_default.xml b/tests/integration/test_non_default_compression/configs/deflateqpl_compression_by_default.xml new file mode 100644 index 00000000000..2ad6a0f1eff --- /dev/null +++ b/tests/integration/test_non_default_compression/configs/deflateqpl_compression_by_default.xml @@ -0,0 +1,11 @@ + + + + + 0 + 0 + + deflate_qpl + + + diff --git a/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml b/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml new file mode 100644 index 00000000000..46e9e43ca27 --- /dev/null +++ b/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_non_default_compression/test.py b/tests/integration/test_non_default_compression/test.py index e0a67a5db95..e69b32daae0 100644 --- a/tests/integration/test_non_default_compression/test.py +++ b/tests/integration/test_non_default_compression/test.py @@ -41,7 +41,14 @@ node6 = cluster.add_instance( main_configs=["configs/allow_experimental_codecs.xml"], user_configs=["configs/allow_suspicious_codecs.xml"], ) - +node7 = cluster.add_instance( + "node7", + main_configs=["configs/deflateqpl_compression_by_default.xml"], + user_configs=[ + "configs/enable_deflateqpl_codec.xml", + "configs/allow_suspicious_codecs.xml", + ], +) @pytest.fixture(scope="module") def start_cluster(): @@ -244,3 +251,59 @@ def test_uncompressed_cache_plus_zstd_codec(start_cluster): ) == "10000\n" ) + +def test_preconfigured_deflateqpl_codec(start_cluster): + node7.query( + """ + CREATE TABLE compression_codec_multiple_with_key ( + somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), DEFLATE_QPL), + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, DEFLATE_QPL), + data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4, DEFLATE_QPL), + somecolumn Float64 + ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; + """ + ) + node7.query( + "INSERT INTO compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), 100000, 'hello', 88.88), (toDate('2018-10-12'), 100002, 'world', 99.99), (toDate('2018-10-12'), 1111, '!', 777.777)" + ) + assert ( + node7.query( + "SELECT COUNT(*) FROM compression_codec_multiple_with_key WHERE id % 2 == 0" + ) + == "2\n" + ) + assert ( + node7.query( + "SELECT DISTINCT somecolumn FROM compression_codec_multiple_with_key ORDER BY id" + ) + == "777.777\n88.88\n99.99\n" + ) + assert ( + node7.query( + "SELECT data FROM compression_codec_multiple_with_key WHERE id >= 1112 AND somedate = toDate('2018-10-12') AND somecolumn <= 100" + ) + == "hello\nworld\n" + ) + + node7.query( + "INSERT INTO compression_codec_multiple_with_key SELECT toDate('2018-10-12'), number, toString(number), 1.0 FROM system.numbers LIMIT 10000" + ) + + assert ( + node7.query( + "SELECT COUNT(id) FROM compression_codec_multiple_with_key WHERE id % 10 == 0" + ) + == "1001\n" + ) + assert ( + node7.query( + "SELECT SUM(somecolumn) FROM compression_codec_multiple_with_key" + ) + == str(777.777 + 88.88 + 99.99 + 1.0 * 10000) + "\n" + ) + assert ( + node7.query( + "SELECT count(*) FROM compression_codec_multiple_with_key GROUP BY somedate" + ) + == "10003\n" + ) diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference b/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference index cfbfadf1e67..a6afe11126c 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference @@ -12,13 +12,7 @@ CODEC(NONE) 2018-01-01 4 4 2018-01-01 5 5 2018-01-01 6 6 -2018-01-01 1 default_value -2018-01-01 2 default_value -2018-01-01 3 3 -2018-01-01 4 4 -2018-01-01 5 5 -2018-01-01 6 6 -CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, NONE) +CODEC(DEFLATE_QPL) 2018-01-01 1 default_value 2018-01-01 2 default_value 2018-01-01 3 3 @@ -27,7 +21,26 @@ CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, NONE) 2018-01-01 6 6 2018-01-01 7 7 2018-01-01 8 8 -CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, NONE) -CODEC(NONE, LZ4, LZ4HC(0), ZSTD(1)) +2018-01-01 1 default_value +2018-01-01 2 default_value +2018-01-01 3 3 +2018-01-01 4 4 +2018-01-01 5 5 +2018-01-01 6 6 +2018-01-01 7 7 +2018-01-01 8 8 +CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, DEFLATE_QPL, NONE) +2018-01-01 1 default_value +2018-01-01 2 default_value +2018-01-01 3 3 +2018-01-01 4 4 +2018-01-01 5 5 +2018-01-01 6 6 +2018-01-01 7 7 +2018-01-01 8 8 +2018-01-01 9 9 +2018-01-01 10 10 +CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, DEFLATE_QPL, NONE) +CODEC(NONE, LZ4, LZ4HC(0), ZSTD(1), DEFLATE_QPL) 2 1 diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index 85e5f8b63ad..40a8bb4c7cb 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -25,15 +25,23 @@ INSERT INTO alter_compression_codec VALUES('2018-01-01', 5, '5'); INSERT INTO alter_compression_codec VALUES('2018-01-01', 6, '6'); SELECT * FROM alter_compression_codec ORDER BY id; -OPTIMIZE TABLE alter_compression_codec FINAL; -SELECT * FROM alter_compression_codec ORDER BY id; - -SET allow_suspicious_codecs = 1; -ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE); +SET enable_qpl_deflate = 1; +ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(DEFLATE_QPL); SELECT compression_codec FROM system.columns WHERE database = currentDatabase() AND table = 'alter_compression_codec' AND name = 'alter_column'; INSERT INTO alter_compression_codec VALUES('2018-01-01', 7, '7'); INSERT INTO alter_compression_codec VALUES('2018-01-01', 8, '8'); +SELECT * FROM alter_compression_codec ORDER BY id; + +OPTIMIZE TABLE alter_compression_codec FINAL; +SELECT * FROM alter_compression_codec ORDER BY id; + +SET allow_suspicious_codecs = 1; +ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, DEFLATE_QPL, NONE); +SELECT compression_codec FROM system.columns WHERE database = currentDatabase() AND table = 'alter_compression_codec' AND name = 'alter_column'; + +INSERT INTO alter_compression_codec VALUES('2018-01-01', 9, '9'); +INSERT INTO alter_compression_codec VALUES('2018-01-01', 10, '10'); OPTIMIZE TABLE alter_compression_codec FINAL; SELECT * FROM alter_compression_codec ORDER BY id; @@ -54,15 +62,17 @@ ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(ZSTD(100)); -- { serverError 433 } +ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(DEFLATE_QPL(100)); -- { serverError 378 } + DROP TABLE IF EXISTS alter_bad_codec; DROP TABLE IF EXISTS large_alter_table_00804; DROP TABLE IF EXISTS store_of_hash_00804; CREATE TABLE large_alter_table_00804 ( - somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)), - id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), - data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4) + somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), DEFLATE_QPL), + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, DEFLATE_QPL), + data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4, DEFLATE_QPL) ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi', min_bytes_for_wide_part = 0; INSERT INTO large_alter_table_00804 SELECT toDate('2019-01-01'), number, toString(number + rand()) FROM system.numbers LIMIT 300000; @@ -71,7 +81,7 @@ CREATE TABLE store_of_hash_00804 (hash UInt64) ENGINE = Memory(); INSERT INTO store_of_hash_00804 SELECT sum(cityHash64(*)) FROM large_alter_table_00804; -ALTER TABLE large_alter_table_00804 MODIFY COLUMN data CODEC(NONE, LZ4, LZ4HC, ZSTD); +ALTER TABLE large_alter_table_00804 MODIFY COLUMN data CODEC(NONE, LZ4, LZ4HC, ZSTD, DEFLATE_QPL); OPTIMIZE TABLE large_alter_table_00804; From d85bc02388317ed4b2743814bcc217baf1652971 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 24 May 2023 15:08:23 -0400 Subject: [PATCH 1429/2223] add function test for deflate_qpl --- ...4_test_custom_compression_codecs.reference | 8 ++-- .../00804_test_custom_compression_codecs.sql | 45 +++++++++++-------- ...m_compression_codes_log_storages.reference | 20 ++++----- ..._custom_compression_codes_log_storages.sql | 41 +++++++++-------- ...st_deflate_qpl_codec_compression.reference | 4 ++ ...804_test_deflate_qpl_codec_compression.sql | 32 +++++++++++++ ...804_test_delta_codec_compression.reference | 2 + .../00804_test_delta_codec_compression.sql | 38 ++++++++++++++++ 8 files changed, 140 insertions(+), 50 deletions(-) create mode 100644 tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference create mode 100644 tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference b/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference index 7bd91e5a69b..a9cbe3d32d3 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference @@ -1,6 +1,6 @@ -1 hello 2018-12-14 1.1 aaa 5 -2 world 2018-12-15 2.2 bbb 6 -3 ! 2018-12-16 3.3 ccc 7 +1 hello 2018-12-14 1.1 aaa 5 qpl11 11 +2 world 2018-12-15 2.2 bbb 6 qpl22 22 +3 ! 2018-12-16 3.3 ccc 7 qpl33 33 2 1 world 2018-10-05 1.1 2 hello 2018-10-01 2.2 @@ -9,7 +9,7 @@ 10003 274972506.6 9175437371954010821 -CREATE TABLE default.compression_codec_multiple_more_types\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.compression_codec_multiple_more_types\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0), DEFLATE_QPL),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), NONE, NONE, NONE, LZ4HC(0), DEFLATE_QPL),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8), DEFLATE_QPL),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8), DEFLATE_QPL)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1.5555555555555 hello world! [77] ['John'] 7.1 xxxxxxxxxxxx [127] ['Henry'] ! diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index c080c2fc98e..44a0daada27 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -1,5 +1,6 @@ SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; +SET enable_qpl_deflate = 1; DROP TABLE IF EXISTS compression_codec; @@ -9,18 +10,20 @@ CREATE TABLE compression_codec( ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), - othernum Int64 CODEC(Delta) + othernum Int64 CODEC(Delta), + qplstr String CODEC(DEFLATE_QPL), + qplnum UInt32 CODEC(DEFLATE_QPL), ) ENGINE = MergeTree() ORDER BY tuple(); -INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); -INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6); -INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7); +INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5, 'qpl11', 11); +INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6,'qpl22', 22); +INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7, 'qpl33', 33); SELECT * FROM compression_codec ORDER BY id; OPTIMIZE TABLE compression_codec FINAL; -INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8); +INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8, 'qpl44', 44); DETACH TABLE compression_codec; ATTACH TABLE compression_codec; @@ -31,25 +34,31 @@ DROP TABLE IF EXISTS compression_codec; DROP TABLE IF EXISTS bad_codec; DROP TABLE IF EXISTS params_when_no_params; +DROP TABLE IF EXISTS params_when_no_params2; DROP TABLE IF EXISTS too_many_params; DROP TABLE IF EXISTS codec_multiple_direct_specification_1; DROP TABLE IF EXISTS codec_multiple_direct_specification_2; +DROP TABLE IF EXISTS codec_multiple_direct_specification_3; DROP TABLE IF EXISTS delta_bad_params1; DROP TABLE IF EXISTS delta_bad_params2; CREATE TABLE bad_codec(id UInt64 CODEC(adssadads)) ENGINE = MergeTree() order by tuple(); -- { serverError 432 } CREATE TABLE too_many_params(id UInt64 CODEC(ZSTD(2,3,4,5))) ENGINE = MergeTree() order by tuple(); -- { serverError 431 } CREATE TABLE params_when_no_params(id UInt64 CODEC(LZ4(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 } +CREATE TABLE params_when_no_params2(id UInt64 CODEC(DEFLATE_QPL(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 } CREATE TABLE codec_multiple_direct_specification_1(id UInt64 CODEC(MULTIPLE(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 } CREATE TABLE codec_multiple_direct_specification_2(id UInt64 CODEC(multiple(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 } +CREATE TABLE codec_multiple_direct_specification_3(id UInt64 CODEC(multiple(LZ4, DEFLATE_QPL))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 } CREATE TABLE delta_bad_params1(id UInt64 CODEC(Delta(3))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 433 } CREATE TABLE delta_bad_params2(id UInt64 CODEC(Delta(16))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 433 } DROP TABLE IF EXISTS bad_codec; DROP TABLE IF EXISTS params_when_no_params; +DROP TABLE IF EXISTS params_when_no_params2; DROP TABLE IF EXISTS too_many_params; DROP TABLE IF EXISTS codec_multiple_direct_specification_1; DROP TABLE IF EXISTS codec_multiple_direct_specification_2; +DROP TABLE IF EXISTS codec_multiple_direct_specification_3; DROP TABLE IF EXISTS delta_bad_params1; DROP TABLE IF EXISTS delta_bad_params2; @@ -58,10 +67,10 @@ DROP TABLE IF EXISTS compression_codec_multiple; SET network_compression_method = 'lz4hc'; CREATE TABLE compression_codec_multiple ( - id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), - data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), - ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), - somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4), DEFLATE_QPL), + data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8), DEFLATE_QPL), + ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC, DEFLATE_QPL), + somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD, DEFLATE_QPL) ) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO compression_codec_multiple VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); @@ -85,15 +94,15 @@ SELECT sum(cityHash64(*)) FROM compression_codec_multiple; DROP TABLE IF EXISTS compression_codec_multiple_more_types; CREATE TABLE compression_codec_multiple_more_types ( - id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC), - data FixedString(12) CODEC(ZSTD, ZSTD, Delta, Delta, Delta, NONE, NONE, NONE, LZ4HC), - ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8)) + id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC, DEFLATE_QPL), + data FixedString(12) CODEC(ZSTD, ZSTD, Delta, Delta, Delta, NONE, NONE, NONE, LZ4HC, DEFLATE_QPL), + ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8), DEFLATE_QPL) ) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36 } CREATE TABLE compression_codec_multiple_more_types ( - id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC), - data FixedString(12) CODEC(ZSTD, ZSTD, NONE, NONE, NONE, LZ4HC), - ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8)) + id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC, DEFLATE_QPL), + data FixedString(12) CODEC(ZSTD, ZSTD, NONE, NONE, NONE, LZ4HC, DEFLATE_QPL), + ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8), DEFLATE_QPL) ) ENGINE = MergeTree() ORDER BY tuple(); SHOW CREATE TABLE compression_codec_multiple_more_types; @@ -109,9 +118,9 @@ SET network_compression_method = 'zstd'; SET network_zstd_compression_level = 5; CREATE TABLE compression_codec_multiple_with_key ( - somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta), - id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta), - data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4) + somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta, DEFLATE_QPL), + id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta, DEFLATE_QPL), + data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4, DEFLATE_QPL) ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference index 8145ca99829..d64b8a77eed 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference +++ b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference @@ -1,9 +1,9 @@ -CREATE TABLE default.compression_codec_log\n(\n `id` UInt64 CODEC(LZ4),\n `data` String CODEC(ZSTD(1)),\n `ddd` Date CODEC(NONE),\n `somenum` Float64 CODEC(ZSTD(2)),\n `somestr` FixedString(3) CODEC(LZ4HC(7)),\n `othernum` Int64 CODEC(Delta(8))\n)\nENGINE = Log -1 hello 2018-12-14 1.1 aaa 5 -2 world 2018-12-15 2.2 bbb 6 -3 ! 2018-12-16 3.3 ccc 7 +CREATE TABLE default.compression_codec_log\n(\n `id` UInt64 CODEC(LZ4),\n `data` String CODEC(ZSTD(1)),\n `ddd` Date CODEC(NONE),\n `somenum` Float64 CODEC(ZSTD(2)),\n `somestr` FixedString(3) CODEC(LZ4HC(7)),\n `othernum` Int64 CODEC(Delta(8)),\n `qplstr` String CODEC(DEFLATE_QPL),\n `qplnum` UInt32 CODEC(DEFLATE_QPL)\n)\nENGINE = Log +1 hello 2018-12-14 1.1 aaa 5 qpl11 11 +2 world 2018-12-15 2.2 bbb 6 qpl22 22 +3 ! 2018-12-16 3.3 ccc 7 qpl33 33 2 -CREATE TABLE default.compression_codec_multiple_log\n(\n `id` UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4)),\n `data` String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8)),\n `ddd` Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0)),\n `somenum` Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1))\n)\nENGINE = Log +CREATE TABLE default.compression_codec_multiple_log\n(\n `id` UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4), DEFLATE_QPL),\n `data` String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8), DEFLATE_QPL),\n `ddd` Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0), DEFLATE_QPL),\n `somenum` Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1), DEFLATE_QPL)\n)\nENGINE = Log 1 world 2018-10-05 1.1 2 hello 2018-10-01 2.2 3 buy 2018-10-11 3.3 @@ -11,12 +11,12 @@ CREATE TABLE default.compression_codec_multiple_log\n(\n `id` UInt64 CODEC(LZ 10003 274972506.6 9175437371954010821 -CREATE TABLE default.compression_codec_tiny_log\n(\n `id` UInt64 CODEC(LZ4),\n `data` String CODEC(ZSTD(1)),\n `ddd` Date CODEC(NONE),\n `somenum` Float64 CODEC(ZSTD(2)),\n `somestr` FixedString(3) CODEC(LZ4HC(7)),\n `othernum` Int64 CODEC(Delta(8))\n)\nENGINE = TinyLog -1 hello 2018-12-14 1.1 aaa 5 -2 world 2018-12-15 2.2 bbb 6 -3 ! 2018-12-16 3.3 ccc 7 +CREATE TABLE default.compression_codec_tiny_log\n(\n `id` UInt64 CODEC(LZ4),\n `data` String CODEC(ZSTD(1)),\n `ddd` Date CODEC(NONE),\n `somenum` Float64 CODEC(ZSTD(2)),\n `somestr` FixedString(3) CODEC(LZ4HC(7)),\n `othernum` Int64 CODEC(Delta(8)),\n `qplstr` String CODEC(DEFLATE_QPL),\n `qplnum` UInt32 CODEC(DEFLATE_QPL)\n)\nENGINE = TinyLog +1 hello 2018-12-14 1.1 aaa 5 qpl11 11 +2 world 2018-12-15 2.2 bbb 6 qpl22 22 +3 ! 2018-12-16 3.3 ccc 7 qpl33 33 2 -CREATE TABLE default.compression_codec_multiple_tiny_log\n(\n `id` UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4)),\n `data` String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8)),\n `ddd` Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0)),\n `somenum` Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1))\n)\nENGINE = TinyLog +CREATE TABLE default.compression_codec_multiple_tiny_log\n(\n `id` UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4), DEFLATE_QPL),\n `data` String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8), DEFLATE_QPL),\n `ddd` Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0), DEFLATE_QPL),\n `somenum` Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1), DEFLATE_QPL)\n)\nENGINE = TinyLog 1 world 2018-10-05 1.1 2 hello 2018-10-01 2.2 3 buy 2018-10-11 3.3 diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql index fba6a216762..113f26732e7 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql @@ -1,5 +1,6 @@ SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; +SET enable_qpl_deflate = 1; -- copy-paste for storage log @@ -11,18 +12,20 @@ CREATE TABLE compression_codec_log( ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), - othernum Int64 CODEC(Delta) + othernum Int64 CODEC(Delta), + qplstr String CODEC(DEFLATE_QPL), + qplnum UInt32 CODEC(DEFLATE_QPL), ) ENGINE = Log(); SHOW CREATE TABLE compression_codec_log; -INSERT INTO compression_codec_log VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); -INSERT INTO compression_codec_log VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6); -INSERT INTO compression_codec_log VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7); +INSERT INTO compression_codec_log VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5, 'qpl11', 11); +INSERT INTO compression_codec_log VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6,'qpl22', 22); +INSERT INTO compression_codec_log VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7, 'qpl33', 33); SELECT * FROM compression_codec_log ORDER BY id; -INSERT INTO compression_codec_log VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8); +INSERT INTO compression_codec_log VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8, 'qpl44', 44); DETACH TABLE compression_codec_log; ATTACH TABLE compression_codec_log; @@ -34,10 +37,10 @@ DROP TABLE IF EXISTS compression_codec_log; DROP TABLE IF EXISTS compression_codec_multiple_log; CREATE TABLE compression_codec_multiple_log ( - id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), - data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), - ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), - somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4), DEFLATE_QPL), + data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8), DEFLATE_QPL), + ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC, DEFLATE_QPL), + somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD, DEFLATE_QPL) ) ENGINE = Log(); SHOW CREATE TABLE compression_codec_multiple_log; @@ -69,18 +72,20 @@ CREATE TABLE compression_codec_tiny_log( ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), - othernum Int64 CODEC(Delta) + othernum Int64 CODEC(Delta), + qplstr String CODEC(DEFLATE_QPL), + qplnum UInt32 CODEC(DEFLATE_QPL), ) ENGINE = TinyLog(); SHOW CREATE TABLE compression_codec_tiny_log; -INSERT INTO compression_codec_tiny_log VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); -INSERT INTO compression_codec_tiny_log VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6); -INSERT INTO compression_codec_tiny_log VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7); +INSERT INTO compression_codec_tiny_log VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5, 'qpl11', 11); +INSERT INTO compression_codec_tiny_log VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6, 'qpl22', 22); +INSERT INTO compression_codec_tiny_log VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7, 'qpl33', 33); SELECT * FROM compression_codec_tiny_log ORDER BY id; -INSERT INTO compression_codec_tiny_log VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8); +INSERT INTO compression_codec_tiny_log VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8, 'qpl44', 44); DETACH TABLE compression_codec_tiny_log; ATTACH TABLE compression_codec_tiny_log; @@ -92,10 +97,10 @@ DROP TABLE IF EXISTS compression_codec_tiny_log; DROP TABLE IF EXISTS compression_codec_multiple_tiny_log; CREATE TABLE compression_codec_multiple_tiny_log ( - id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), - data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), - ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), - somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4), DEFLATE_QPL), + data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8), DEFLATE_QPL), + ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC, DEFLATE_QPL), + somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD, DEFLATE_QPL) ) ENGINE = TinyLog(); SHOW CREATE TABLE compression_codec_multiple_tiny_log; diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference new file mode 100644 index 00000000000..88d274d9cba --- /dev/null +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference @@ -0,0 +1,4 @@ +1 hello 2018-12-14 1.1 aaa 5 qpl11 11 +2 world 2018-12-15 2.2 bbb 6 qpl22 22 +3 ! 2018-12-16 3.3 ccc 7 qpl33 33 +2 diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql new file mode 100644 index 00000000000..fe23e49804d --- /dev/null +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql @@ -0,0 +1,32 @@ +SET send_logs_level = 'fatal'; +SET enable_qpl_deflate = 1; + +DROP TABLE IF EXISTS compression_codec; + +CREATE TABLE compression_codec( + id UInt64 CODEC(DEFLATE_QPL), + data String CODEC(DEFLATE_QPL), + ddd Date CODEC(DEFLATE_QPL), + somenum Float64 CODEC(DEFLATE_QPL), + somestr FixedString(3) CODEC(DEFLATE_QPL), + othernum Int64 CODEC(DEFLATE_QPL), + qplstr String CODEC(DEFLATE_QPL), + qplnum UInt32 CODEC(DEFLATE_QPL), +) ENGINE = MergeTree() ORDER BY tuple(); + +INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5, 'qpl11', 11); +INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6,'qpl22', 22); +INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7, 'qpl33', 33); + +SELECT * FROM compression_codec ORDER BY id; + +OPTIMIZE TABLE compression_codec FINAL; + +INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8, 'qpl44', 44); + +DETACH TABLE compression_codec; +ATTACH TABLE compression_codec; + +SELECT count(*) FROM compression_codec WHERE id = 2 GROUP BY id; + +DROP TABLE IF EXISTS compression_codec; diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference index 949d37ed27a..37f9d4901b3 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference @@ -4,3 +4,5 @@ 1 32 1 +17 +1 diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index 25988f6474b..f9805246662 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -115,3 +115,41 @@ USING(key); DROP TABLE IF EXISTS delta_codec_string; DROP TABLE IF EXISTS default_codec_string; + +SET enable_qpl_deflate = 1; +DROP TABLE IF EXISTS delta_codec_string_qpl; +DROP TABLE IF EXISTS default_codec_string_qpl; + +CREATE TABLE delta_codec_string_qpl +( + id Float64 Codec(Delta, DEFLATE_QPL) +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; + +CREATE TABLE default_codec_string_qpl +( + id Float64 Codec(DEFLATE_QPL) +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; + +INSERT INTO delta_codec_string_qpl SELECT concat(toString(number), toString(number % 100)) FROM numbers(1547510400, 500000); +INSERT INTO default_codec_string_qpl SELECT * from delta_codec_string_qpl; + +OPTIMIZE TABLE delta_codec_string_qpl FINAL; +OPTIMIZE TABLE default_codec_string_qpl FINAL; + +SELECT + floor(big_size / small_size) as ratio +FROM + (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_string_qpl' and active) +INNER JOIN + (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_string_qpl' and active) USING(key); + +SELECT + small_hash == big_hash +FROM + (SELECT 1 AS key, sum(cityHash64(*)) AS small_hash FROM delta_codec_string_qpl) +INNER JOIN + (SELECT 1 AS key, sum(cityHash64(*)) AS big_hash FROM default_codec_string_qpl) +USING(key); + +DROP TABLE IF EXISTS delta_codec_string_qpl; +DROP TABLE IF EXISTS default_codec_string_qpl; From 31173ab55b0926f634c2fbfc06f7d2f34410a4ff Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 24 May 2023 15:15:40 -0400 Subject: [PATCH 1430/2223] add sections of deflate_qpl for stress test and performance test --- tests/ci/stress.py | 1 + tests/performance/codecs_float_insert.xml | 2 ++ tests/performance/codecs_float_select.xml | 2 ++ tests/performance/codecs_int_insert.xml | 2 ++ tests/performance/codecs_int_select.xml | 2 ++ 5 files changed, 9 insertions(+) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index b9044874071..b95cac9044e 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -20,6 +20,7 @@ def get_options(i, upgrade_check): '''--db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i) ) client_options.append("allow_experimental_database_replicated=1") + client_options.append("enable_qpl_deflate=1") # If database name is not specified, new database is created for each functional test. # Run some threads with one database for all tests. diff --git a/tests/performance/codecs_float_insert.xml b/tests/performance/codecs_float_insert.xml index 64325d30189..25291f7f499 100644 --- a/tests/performance/codecs_float_insert.xml +++ b/tests/performance/codecs_float_insert.xml @@ -1,6 +1,7 @@ 1 + 1 @@ -10,6 +11,7 @@ NONE LZ4 ZSTD + DEFLATE_QPL DoubleDelta Gorilla FPC diff --git a/tests/performance/codecs_float_select.xml b/tests/performance/codecs_float_select.xml index 325c140d9a0..bb67987c75e 100644 --- a/tests/performance/codecs_float_select.xml +++ b/tests/performance/codecs_float_select.xml @@ -1,6 +1,7 @@ 1 + 1 @@ -10,6 +11,7 @@ NONE LZ4 ZSTD + DEFLATE_QPL DoubleDelta Gorilla FPC diff --git a/tests/performance/codecs_int_insert.xml b/tests/performance/codecs_int_insert.xml index 618e20160f8..1db9ee8f746 100644 --- a/tests/performance/codecs_int_insert.xml +++ b/tests/performance/codecs_int_insert.xml @@ -1,6 +1,7 @@ 1 + 1 @@ -10,6 +11,7 @@ NONE LZ4 ZSTD + DEFLATE_QPL Delta T64 DoubleDelta diff --git a/tests/performance/codecs_int_select.xml b/tests/performance/codecs_int_select.xml index 62c1ee16e7b..5dc7ab48704 100644 --- a/tests/performance/codecs_int_select.xml +++ b/tests/performance/codecs_int_select.xml @@ -1,6 +1,7 @@ 1 + 1 @@ -10,6 +11,7 @@ NONE LZ4 ZSTD + DEFLATE_QPL Delta T64 DoubleDelta From cbdb408ec8330c8ce469c68e979ca208c76d0629 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Fri, 26 May 2023 12:15:34 -0400 Subject: [PATCH 1431/2223] add USE_QPL for buildoptions --- src/Storages/System/StorageSystemBuildOptions.cpp.in | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index 3465e47449b..c2a188e7750 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -68,6 +68,7 @@ const char * auto_config_build[] "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(@GIT_BRANCH@)IRjaNsZIL9Yh7FQ4", "GIT_DATE", "@GIT_DATE@", "GIT_COMMIT_SUBJECT", R"Gi17KJMlbGCjErEN(@GIT_COMMIT_SUBJECT@)Gi17KJMlbGCjErEN", + "USE_QPL", "@ENABLE_QPL@", nullptr, nullptr }; From f1192d59afa7ee2271d7ee6b5cb9d98bb27254a0 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 1 Jun 2023 12:42:22 -0400 Subject: [PATCH 1432/2223] refine patch according to comments --- .../sql-reference/statements/create/table.md | 2 +- src/Client/Connection.cpp | 2 +- src/Compression/CompressionFactory.h | 4 +- .../CompressionFactoryAdditions.cpp | 12 +++--- src/Compression/ICompressionCodec.h | 6 +-- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/AlterCommands.cpp | 4 +- src/Storages/Distributed/DistributedSink.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- tests/ci/stress.py | 2 +- .../configs/enable_deflateqpl_codec.xml | 2 +- .../test_non_default_compression/test.py | 32 ++++++++-------- tests/performance/codecs_float_insert.xml | 3 +- tests/performance/codecs_float_select.xml | 3 +- tests/performance/codecs_int_insert.xml | 3 +- tests/performance/codecs_int_select.xml | 3 +- ...04_test_alter_compression_codecs.reference | 4 +- .../00804_test_alter_compression_codecs.sql | 10 ++--- ...4_test_custom_compression_codecs.reference | 6 +-- .../00804_test_custom_compression_codecs.sql | 13 +++---- ..._custom_compression_codes_log_storages.sql | 2 +- ...st_deflate_qpl_codec_compression.reference | 6 +-- ...804_test_deflate_qpl_codec_compression.sql | 16 ++++---- ...804_test_delta_codec_compression.reference | 2 - .../00804_test_delta_codec_compression.sql | 38 ------------------- 27 files changed, 71 insertions(+), 116 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index b0865ad2896..d0e17410791 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -380,7 +380,7 @@ High compression levels are useful for asymmetric scenarios, like compress once, `DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library. Some limitations apply: -- DEFLATE_QPL is disabled by default and can only be used after setting configuration parameter `enable_qpl_deflate=1`. +- DEFLATE_QPL is disabled by default and can only be used after setting configuration parameter `enable_qpl_deflate_codec=1`. - DEFLATE_QPL requires a ClickHouse build compiled with SSE 4.2 instructions (by default, this is the case). Refer to [Build Clickhouse with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Build-Clickhouse-with-DEFLATE_QPL) for more details. - DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [Benchmark with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Run-Benchmark-with-DEFLATE_QPL) for more details. - DEFLATE_QPL-compressed data can only be transferred between ClickHouse nodes compiled with SSE 4.2 enabled. diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 68bc3b39a56..ac8e6654e84 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -588,7 +588,7 @@ void Connection::sendQuery( if (method == "ZSTD") level = settings->network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs, settings->enable_qpl_deflate); + CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs, settings->enable_qpl_deflate_codec); compression_codec = CompressionCodecFactory::instance().get(method, level); } else diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index 1fdaf4f1c71..e020e51bb09 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -40,10 +40,10 @@ public: CompressionCodecPtr getDefaultCodec() const; /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) - ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const; + ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const; /// Validate codecs AST specified by user - void validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const; + void validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const; /// Get codec by AST and possible column_type. Some codecs can use /// information about type to improve inner settings, but every codec should diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp index 2630326238a..b4a2d96cf39 100644 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes void CompressionCodecFactory::validateCodec( - const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const + const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const { if (family_name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Compression codec name cannot be empty"); @@ -43,13 +43,13 @@ void CompressionCodecFactory::validateCodec( { auto literal = std::make_shared(static_cast(*level)); validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), - {}, sanity_check, allow_experimental_codecs, enable_qpl_deflate); + {}, sanity_check, allow_experimental_codecs, enable_qpl_deflate_codec); } else { auto identifier = std::make_shared(Poco::toUpper(family_name)); validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), - {}, sanity_check, allow_experimental_codecs, enable_qpl_deflate); + {}, sanity_check, allow_experimental_codecs, enable_qpl_deflate_codec); } } @@ -77,7 +77,7 @@ bool innerDataTypeIsFloat(const DataTypePtr & type) } ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( - const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate) const + const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const { if (const auto * func = ast->as()) { @@ -159,10 +159,10 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( " You can enable it with the 'allow_experimental_codecs' setting.", codec_family_name); - if (!enable_qpl_deflate && result_codec->isDeflateQplCompression()) + if (!enable_qpl_deflate_codec && result_codec->isDeflateQplCompression()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec {} is disabled by default." - " You can enable it with the 'enable_qpl_deflate' setting.", + " You can enable it with the 'enable_qpl_deflate_codec' setting.", codec_family_name); codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index d92ad3fc718..f7e8f4e43d2 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -109,12 +109,12 @@ public: /// It will not be allowed to use unless the user will turn off the safety switch. virtual bool isExperimental() const { return false; } - /// If it does nothing. - virtual bool isNone() const { return false; } - /// This is a knob for Deflate QPL codec. virtual bool isDeflateQplCompression() const { return false; } + /// If it does nothing. + virtual bool isNone() const { return false; } + protected: /// This is used for fuzz testing friend int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c6a2069e6ae..4aae8f5d572 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -319,7 +319,7 @@ class IColumn; M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ - M(Bool, enable_qpl_deflate, false, "If it is set to true, allow to use deflate_qpl for compression.", 0) \ + M(Bool, enable_qpl_deflate_codec, false, "If it is set to true, allow usage of the DEFLATE_QPL codec.", 0) \ M(UInt64, query_profiler_real_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, query_profiler_cpu_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 5c22b46b360..ddb53bbbfaa 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -571,7 +571,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( bool sanity_check_compression_codecs = !attach && !context_->getSettingsRef().allow_suspicious_codecs; bool allow_experimental_codecs = attach || context_->getSettingsRef().allow_experimental_codecs; - bool enable_qpl_deflate = attach || context_->getSettingsRef().enable_qpl_deflate; + bool enable_qpl_deflate_codec = attach || context_->getSettingsRef().enable_qpl_deflate_codec; ColumnsDescription res; auto name_type_it = column_names_and_types.begin(); @@ -632,7 +632,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.default_specifier == "ALIAS") throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot specify codec for column type ALIAS"); column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( - col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_qpl_deflate); + col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_qpl_deflate_codec); } if (col_decl.ttl) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 96c585e7d16..b43fef9dd54 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1775,7 +1775,7 @@ void TCPHandler::initBlockOutput(const Block & block) if (state.compression == Protocol::Compression::Enable) { - CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs, query_settings.enable_qpl_deflate); + CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs, query_settings.enable_qpl_deflate_codec); state.maybe_compressed_out = std::make_shared( *out, CompressionCodecFactory::instance().get(method, level)); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index ecbddfc3e2a..73d7be8dc56 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1067,7 +1067,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const "this column name is reserved for lightweight delete feature", backQuote(column_name)); if (command.codec) - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate_codec); all_columns.add(ColumnDescription(column_name, command.data_type)); } @@ -1093,7 +1093,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { if (all_columns.hasAlias(column_name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot specify codec for column type ALIAS"); - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate_codec); } auto column_default = all_columns.getDefault(column_name); if (column_default) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index ce1dbde8eae..e383890d1f7 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -733,7 +733,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const if (compression_method == "ZSTD") compression_level = settings.network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enable_qpl_deflate); + CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enable_qpl_deflate_codec); CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level); /// tmp directory is used to ensure atomicity of transactions diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index f5209cbdff6..a437465b3fe 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -285,7 +285,7 @@ TTLDescription TTLDescription::getTTLFromAST( { result.recompression_codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( - ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate); + ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate_codec); } } diff --git a/tests/ci/stress.py b/tests/ci/stress.py index b95cac9044e..e5ceb251d0f 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -20,7 +20,7 @@ def get_options(i, upgrade_check): '''--db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i) ) client_options.append("allow_experimental_database_replicated=1") - client_options.append("enable_qpl_deflate=1") + client_options.append("enable_qpl_deflate_codec=1") # If database name is not specified, new database is created for each functional test. # Run some threads with one database for all tests. diff --git a/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml b/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml index 46e9e43ca27..521b0fd663c 100644 --- a/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml +++ b/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml @@ -1,7 +1,7 @@ - 1 + 1 diff --git a/tests/integration/test_non_default_compression/test.py b/tests/integration/test_non_default_compression/test.py index e69b32daae0..e1a9c1ae540 100644 --- a/tests/integration/test_non_default_compression/test.py +++ b/tests/integration/test_non_default_compression/test.py @@ -38,16 +38,16 @@ node5 = cluster.add_instance( ) node6 = cluster.add_instance( "node6", - main_configs=["configs/allow_experimental_codecs.xml"], - user_configs=["configs/allow_suspicious_codecs.xml"], + main_configs=["configs/deflateqpl_compression_by_default.xml"], + user_configs=[ + "configs/allow_suspicious_codecs.xml", + "configs/enable_deflateqpl_codec.xml", + ], ) node7 = cluster.add_instance( "node7", - main_configs=["configs/deflateqpl_compression_by_default.xml"], - user_configs=[ - "configs/enable_deflateqpl_codec.xml", - "configs/allow_suspicious_codecs.xml", - ], + main_configs=["configs/allow_experimental_codecs.xml"], + user_configs=["configs/allow_suspicious_codecs.xml"], ) @pytest.fixture(scope="module") @@ -253,7 +253,7 @@ def test_uncompressed_cache_plus_zstd_codec(start_cluster): ) def test_preconfigured_deflateqpl_codec(start_cluster): - node7.query( + node6.query( """ CREATE TABLE compression_codec_multiple_with_key ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), DEFLATE_QPL), @@ -263,46 +263,46 @@ def test_preconfigured_deflateqpl_codec(start_cluster): ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; """ ) - node7.query( + node6.query( "INSERT INTO compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), 100000, 'hello', 88.88), (toDate('2018-10-12'), 100002, 'world', 99.99), (toDate('2018-10-12'), 1111, '!', 777.777)" ) assert ( - node7.query( + node6.query( "SELECT COUNT(*) FROM compression_codec_multiple_with_key WHERE id % 2 == 0" ) == "2\n" ) assert ( - node7.query( + node6.query( "SELECT DISTINCT somecolumn FROM compression_codec_multiple_with_key ORDER BY id" ) == "777.777\n88.88\n99.99\n" ) assert ( - node7.query( + node6.query( "SELECT data FROM compression_codec_multiple_with_key WHERE id >= 1112 AND somedate = toDate('2018-10-12') AND somecolumn <= 100" ) == "hello\nworld\n" ) - node7.query( + node6.query( "INSERT INTO compression_codec_multiple_with_key SELECT toDate('2018-10-12'), number, toString(number), 1.0 FROM system.numbers LIMIT 10000" ) assert ( - node7.query( + node6.query( "SELECT COUNT(id) FROM compression_codec_multiple_with_key WHERE id % 10 == 0" ) == "1001\n" ) assert ( - node7.query( + node6.query( "SELECT SUM(somecolumn) FROM compression_codec_multiple_with_key" ) == str(777.777 + 88.88 + 99.99 + 1.0 * 10000) + "\n" ) assert ( - node7.query( + node6.query( "SELECT count(*) FROM compression_codec_multiple_with_key GROUP BY somedate" ) == "10003\n" diff --git a/tests/performance/codecs_float_insert.xml b/tests/performance/codecs_float_insert.xml index 25291f7f499..be0935ad4cf 100644 --- a/tests/performance/codecs_float_insert.xml +++ b/tests/performance/codecs_float_insert.xml @@ -1,7 +1,7 @@ 1 - 1 + 1 @@ -11,7 +11,6 @@ NONE LZ4 ZSTD - DEFLATE_QPL DoubleDelta Gorilla FPC diff --git a/tests/performance/codecs_float_select.xml b/tests/performance/codecs_float_select.xml index bb67987c75e..844ab4508d8 100644 --- a/tests/performance/codecs_float_select.xml +++ b/tests/performance/codecs_float_select.xml @@ -1,7 +1,7 @@ 1 - 1 + 1 @@ -11,7 +11,6 @@ NONE LZ4 ZSTD - DEFLATE_QPL DoubleDelta Gorilla FPC diff --git a/tests/performance/codecs_int_insert.xml b/tests/performance/codecs_int_insert.xml index 1db9ee8f746..d5f12810118 100644 --- a/tests/performance/codecs_int_insert.xml +++ b/tests/performance/codecs_int_insert.xml @@ -1,7 +1,7 @@ 1 - 1 + 1 @@ -11,7 +11,6 @@ NONE LZ4 ZSTD - DEFLATE_QPL Delta T64 DoubleDelta diff --git a/tests/performance/codecs_int_select.xml b/tests/performance/codecs_int_select.xml index 5dc7ab48704..06b2c2a73f3 100644 --- a/tests/performance/codecs_int_select.xml +++ b/tests/performance/codecs_int_select.xml @@ -1,7 +1,7 @@ 1 - 1 + 1 @@ -11,7 +11,6 @@ NONE LZ4 ZSTD - DEFLATE_QPL Delta T64 DoubleDelta diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference b/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference index a6afe11126c..5c77a102740 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference @@ -12,15 +12,13 @@ CODEC(NONE) 2018-01-01 4 4 2018-01-01 5 5 2018-01-01 6 6 -CODEC(DEFLATE_QPL) 2018-01-01 1 default_value 2018-01-01 2 default_value 2018-01-01 3 3 2018-01-01 4 4 2018-01-01 5 5 2018-01-01 6 6 -2018-01-01 7 7 -2018-01-01 8 8 +CODEC(DEFLATE_QPL) 2018-01-01 1 default_value 2018-01-01 2 default_value 2018-01-01 3 3 diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index 40a8bb4c7cb..5b8b73270a2 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -25,7 +25,10 @@ INSERT INTO alter_compression_codec VALUES('2018-01-01', 5, '5'); INSERT INTO alter_compression_codec VALUES('2018-01-01', 6, '6'); SELECT * FROM alter_compression_codec ORDER BY id; -SET enable_qpl_deflate = 1; +OPTIMIZE TABLE alter_compression_codec FINAL; +SELECT * FROM alter_compression_codec ORDER BY id; + +SET enable_qpl_deflate_codec = 1; ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(DEFLATE_QPL); SELECT compression_codec FROM system.columns WHERE database = currentDatabase() AND table = 'alter_compression_codec' AND name = 'alter_column'; @@ -33,9 +36,6 @@ INSERT INTO alter_compression_codec VALUES('2018-01-01', 7, '7'); INSERT INTO alter_compression_codec VALUES('2018-01-01', 8, '8'); SELECT * FROM alter_compression_codec ORDER BY id; -OPTIMIZE TABLE alter_compression_codec FINAL; -SELECT * FROM alter_compression_codec ORDER BY id; - SET allow_suspicious_codecs = 1; ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, DEFLATE_QPL, NONE); SELECT compression_codec FROM system.columns WHERE database = currentDatabase() AND table = 'alter_compression_codec' AND name = 'alter_column'; @@ -62,7 +62,7 @@ ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(ZSTD(100)); -- { serverError 433 } -ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(DEFLATE_QPL(100)); -- { serverError 378 } +ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(DEFLATE_QPL(100)); -- { serverError DATA_TYPE_CANNOT_HAVE_ARGUMENTS } DROP TABLE IF EXISTS alter_bad_codec; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference b/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference index a9cbe3d32d3..8b51d65004a 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference @@ -1,6 +1,6 @@ -1 hello 2018-12-14 1.1 aaa 5 qpl11 11 -2 world 2018-12-15 2.2 bbb 6 qpl22 22 -3 ! 2018-12-16 3.3 ccc 7 qpl33 33 +1 hello 2018-12-14 2018-12-14 1.1 aaa 5 +2 world 2018-12-15 2018-12-15 2.2 bbb 6 +3 ! 2018-12-16 2018-12-16 3.3 ccc 7 2 1 world 2018-10-05 1.1 2 hello 2018-10-01 2.2 diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 44a0daada27..47ec268bfec 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -1,6 +1,6 @@ SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; -SET enable_qpl_deflate = 1; +SET enable_qpl_deflate_codec = 1; DROP TABLE IF EXISTS compression_codec; @@ -8,22 +8,21 @@ CREATE TABLE compression_codec( id UInt64 CODEC(LZ4), data String CODEC(ZSTD), ddd Date CODEC(NONE), + ddd32 Date32 CODEC(DEFLATE_QPL), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), othernum Int64 CODEC(Delta), - qplstr String CODEC(DEFLATE_QPL), - qplnum UInt32 CODEC(DEFLATE_QPL), ) ENGINE = MergeTree() ORDER BY tuple(); -INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5, 'qpl11', 11); -INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6,'qpl22', 22); -INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7, 'qpl33', 33); +INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), toDate32('2018-12-14'), 1.1, 'aaa', 5); +INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), toDate32('2018-12-15'), 2.2, 'bbb', 6); +INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), toDate32('2018-12-16'), 3.3, 'ccc', 7); SELECT * FROM compression_codec ORDER BY id; OPTIMIZE TABLE compression_codec FINAL; -INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8, 'qpl44', 44); +INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), toDate32('2018-12-13'), 4.4, 'ddd', 8); DETACH TABLE compression_codec; ATTACH TABLE compression_codec; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql index 113f26732e7..bcd09277824 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql @@ -1,6 +1,6 @@ SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; -SET enable_qpl_deflate = 1; +SET enable_qpl_deflate_codec = 1; -- copy-paste for storage log diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference index 88d274d9cba..276747f8233 100644 --- a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference @@ -1,4 +1,4 @@ -1 hello 2018-12-14 1.1 aaa 5 qpl11 11 -2 world 2018-12-15 2.2 bbb 6 qpl22 22 -3 ! 2018-12-16 3.3 ccc 7 qpl33 33 +1 hello 2018-12-14 2018-12-14 1.1 aaa 5 [1,2,3] {'k1':1,'k2':2} (1,2) +2 world 2018-12-15 2018-12-15 2.2 bbb 6 [4,5,6] {'k3':3,'k4':4} (3,4) +3 ! 2018-12-16 2018-12-16 3.3 ccc 7 [7,8,9] {'k5':5,'k6':6} (5,6) 2 diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql index fe23e49804d..64e66d47522 100644 --- a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql @@ -1,5 +1,5 @@ SET send_logs_level = 'fatal'; -SET enable_qpl_deflate = 1; +SET enable_qpl_deflate_codec = 1; DROP TABLE IF EXISTS compression_codec; @@ -7,22 +7,24 @@ CREATE TABLE compression_codec( id UInt64 CODEC(DEFLATE_QPL), data String CODEC(DEFLATE_QPL), ddd Date CODEC(DEFLATE_QPL), + ddd32 Date32 CODEC(DEFLATE_QPL), somenum Float64 CODEC(DEFLATE_QPL), somestr FixedString(3) CODEC(DEFLATE_QPL), othernum Int64 CODEC(DEFLATE_QPL), - qplstr String CODEC(DEFLATE_QPL), - qplnum UInt32 CODEC(DEFLATE_QPL), + somearray Array(UInt8) CODEC(DEFLATE_QPL), + somemap Map(String, UInt32) CODEC(DEFLATE_QPL), + sometuple Tuple(UInt16, UInt64) CODEC(DEFLATE_QPL), ) ENGINE = MergeTree() ORDER BY tuple(); -INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5, 'qpl11', 11); -INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6,'qpl22', 22); -INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7, 'qpl33', 33); +INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), toDate32('2018-12-14'), 1.1, 'aaa', 5, [1,2,3], map('k1',1,'k2',2), tuple(1,2)); +INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), toDate32('2018-12-15'), 2.2, 'bbb', 6, [4,5,6], map('k3',3,'k4',4), tuple(3,4)); +INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), toDate32('2018-12-16'), 3.3, 'ccc', 7, [7,8,9], map('k5',5,'k6',6), tuple(5,6)); SELECT * FROM compression_codec ORDER BY id; OPTIMIZE TABLE compression_codec FINAL; -INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8, 'qpl44', 44); +INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), toDate32('2018-12-13'), 4.4, 'ddd', 8, [10,11,12], map('k7',7,'k8',8), tuple(7,8)); DETACH TABLE compression_codec; ATTACH TABLE compression_codec; diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference index 37f9d4901b3..949d37ed27a 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference @@ -4,5 +4,3 @@ 1 32 1 -17 -1 diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index f9805246662..25988f6474b 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -115,41 +115,3 @@ USING(key); DROP TABLE IF EXISTS delta_codec_string; DROP TABLE IF EXISTS default_codec_string; - -SET enable_qpl_deflate = 1; -DROP TABLE IF EXISTS delta_codec_string_qpl; -DROP TABLE IF EXISTS default_codec_string_qpl; - -CREATE TABLE delta_codec_string_qpl -( - id Float64 Codec(Delta, DEFLATE_QPL) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; - -CREATE TABLE default_codec_string_qpl -( - id Float64 Codec(DEFLATE_QPL) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; - -INSERT INTO delta_codec_string_qpl SELECT concat(toString(number), toString(number % 100)) FROM numbers(1547510400, 500000); -INSERT INTO default_codec_string_qpl SELECT * from delta_codec_string_qpl; - -OPTIMIZE TABLE delta_codec_string_qpl FINAL; -OPTIMIZE TABLE default_codec_string_qpl FINAL; - -SELECT - floor(big_size / small_size) as ratio -FROM - (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_string_qpl' and active) -INNER JOIN - (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_string_qpl' and active) USING(key); - -SELECT - small_hash == big_hash -FROM - (SELECT 1 AS key, sum(cityHash64(*)) AS small_hash FROM delta_codec_string_qpl) -INNER JOIN - (SELECT 1 AS key, sum(cityHash64(*)) AS big_hash FROM default_codec_string_qpl) -USING(key); - -DROP TABLE IF EXISTS delta_codec_string_qpl; -DROP TABLE IF EXISTS default_codec_string_qpl; From aae281eb7df6ce8e00d872d3ef0d0558781a5f1a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 15:49:52 +0200 Subject: [PATCH 1433/2223] Update codecs_float_insert.xml --- tests/performance/codecs_float_insert.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/codecs_float_insert.xml b/tests/performance/codecs_float_insert.xml index be0935ad4cf..64325d30189 100644 --- a/tests/performance/codecs_float_insert.xml +++ b/tests/performance/codecs_float_insert.xml @@ -1,7 +1,6 @@ 1 - 1 From dc93b6324ee505228b96791db629b7437f6db931 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 15:50:28 +0200 Subject: [PATCH 1434/2223] Update codecs_float_select.xml --- tests/performance/codecs_float_select.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/codecs_float_select.xml b/tests/performance/codecs_float_select.xml index 844ab4508d8..325c140d9a0 100644 --- a/tests/performance/codecs_float_select.xml +++ b/tests/performance/codecs_float_select.xml @@ -1,7 +1,6 @@ 1 - 1 From 7043db669e4e445764d99cd749cfef99d3f437cf Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 15:50:40 +0200 Subject: [PATCH 1435/2223] Update codecs_int_insert.xml --- tests/performance/codecs_int_insert.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/codecs_int_insert.xml b/tests/performance/codecs_int_insert.xml index d5f12810118..618e20160f8 100644 --- a/tests/performance/codecs_int_insert.xml +++ b/tests/performance/codecs_int_insert.xml @@ -1,7 +1,6 @@ 1 - 1 From 4d7364af97893c4457a86a064628ff478d900c05 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 15:50:49 +0200 Subject: [PATCH 1436/2223] Update codecs_int_select.xml --- tests/performance/codecs_int_select.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/codecs_int_select.xml b/tests/performance/codecs_int_select.xml index 06b2c2a73f3..62c1ee16e7b 100644 --- a/tests/performance/codecs_int_select.xml +++ b/tests/performance/codecs_int_select.xml @@ -1,7 +1,6 @@ 1 - 1 From 1f928f2d3d0eea55ff1743cea386162fd87fed92 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 15:53:48 +0200 Subject: [PATCH 1437/2223] Update StorageSystemBuildOptions.cpp.in --- src/Storages/System/StorageSystemBuildOptions.cpp.in | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemBuildOptions.cpp.in b/src/Storages/System/StorageSystemBuildOptions.cpp.in index c2a188e7750..c2d35c96ce5 100644 --- a/src/Storages/System/StorageSystemBuildOptions.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.cpp.in @@ -64,11 +64,11 @@ const char * auto_config_build[] "USE_ARROW", "@USE_ARROW@", "USE_ORC", "@USE_ORC@", "USE_MSGPACK", "@USE_MSGPACK@", + "USE_QPL", "@ENABLE_QPL@", "GIT_HASH", "@GIT_HASH@", "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(@GIT_BRANCH@)IRjaNsZIL9Yh7FQ4", "GIT_DATE", "@GIT_DATE@", "GIT_COMMIT_SUBJECT", R"Gi17KJMlbGCjErEN(@GIT_COMMIT_SUBJECT@)Gi17KJMlbGCjErEN", - "USE_QPL", "@ENABLE_QPL@", nullptr, nullptr }; From 1aa158909e434438733504d2dbcd9ea9d113e41b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Jun 2023 12:38:38 +0000 Subject: [PATCH 1438/2223] enable_qpl_deflate_codec --> enable_deflate_qpl_codec --- docs/en/sql-reference/statements/create/table.md | 2 +- src/Client/Connection.cpp | 2 +- src/Compression/CompressionFactory.h | 4 ++-- src/Compression/CompressionFactoryAdditions.cpp | 12 ++++++------ src/Core/Settings.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++-- src/Server/TCPHandler.cpp | 2 +- src/Storages/AlterCommands.cpp | 4 ++-- src/Storages/Distributed/DistributedSink.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- tests/ci/stress.py | 2 +- .../configs/enable_deflateqpl_codec.xml | 2 +- .../00804_test_alter_compression_codecs.sql | 2 +- .../00804_test_custom_compression_codecs.sql | 2 +- ...04_test_custom_compression_codes_log_storages.sql | 2 +- .../00804_test_deflate_qpl_codec_compression.sql | 2 +- 16 files changed, 24 insertions(+), 24 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index d0e17410791..496ecdbda7b 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -380,7 +380,7 @@ High compression levels are useful for asymmetric scenarios, like compress once, `DEFLATE_QPL` — [Deflate compression algorithm](https://github.com/intel/qpl) implemented by Intel® Query Processing Library. Some limitations apply: -- DEFLATE_QPL is disabled by default and can only be used after setting configuration parameter `enable_qpl_deflate_codec=1`. +- DEFLATE_QPL is disabled by default and can only be used after setting configuration parameter `enable_deflate_qpl_codec = 1`. - DEFLATE_QPL requires a ClickHouse build compiled with SSE 4.2 instructions (by default, this is the case). Refer to [Build Clickhouse with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Build-Clickhouse-with-DEFLATE_QPL) for more details. - DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [Benchmark with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Run-Benchmark-with-DEFLATE_QPL) for more details. - DEFLATE_QPL-compressed data can only be transferred between ClickHouse nodes compiled with SSE 4.2 enabled. diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index ac8e6654e84..636532ade4b 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -588,7 +588,7 @@ void Connection::sendQuery( if (method == "ZSTD") level = settings->network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs, settings->enable_qpl_deflate_codec); + CompressionCodecFactory::instance().validateCodec(method, level, !settings->allow_suspicious_codecs, settings->allow_experimental_codecs, settings->enable_deflate_qpl_codec); compression_codec = CompressionCodecFactory::instance().get(method, level); } else diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index e020e51bb09..4f2627587a3 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -40,10 +40,10 @@ public: CompressionCodecPtr getDefaultCodec() const; /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) - ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const; + ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec) const; /// Validate codecs AST specified by user - void validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const; + void validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec) const; /// Get codec by AST and possible column_type. Some codecs can use /// information about type to improve inner settings, but every codec should diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp index b4a2d96cf39..46f7e2653c2 100644 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes void CompressionCodecFactory::validateCodec( - const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const + const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec) const { if (family_name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Compression codec name cannot be empty"); @@ -43,13 +43,13 @@ void CompressionCodecFactory::validateCodec( { auto literal = std::make_shared(static_cast(*level)); validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal)), - {}, sanity_check, allow_experimental_codecs, enable_qpl_deflate_codec); + {}, sanity_check, allow_experimental_codecs, enable_deflate_qpl_codec); } else { auto identifier = std::make_shared(Poco::toUpper(family_name)); validateCodecAndGetPreprocessedAST(makeASTFunction("CODEC", identifier), - {}, sanity_check, allow_experimental_codecs, enable_qpl_deflate_codec); + {}, sanity_check, allow_experimental_codecs, enable_deflate_qpl_codec); } } @@ -77,7 +77,7 @@ bool innerDataTypeIsFloat(const DataTypePtr & type) } ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( - const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_qpl_deflate_codec) const + const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs, bool enable_deflate_qpl_codec) const { if (const auto * func = ast->as()) { @@ -159,10 +159,10 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( " You can enable it with the 'allow_experimental_codecs' setting.", codec_family_name); - if (!enable_qpl_deflate_codec && result_codec->isDeflateQplCompression()) + if (!enable_deflate_qpl_codec && result_codec->isDeflateQplCompression()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec {} is disabled by default." - " You can enable it with the 'enable_qpl_deflate_codec' setting.", + " You can enable it with the 'enable_deflate_qpl_codec' setting.", codec_family_name); codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4aae8f5d572..e0034174597 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -319,7 +319,7 @@ class IColumn; M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ - M(Bool, enable_qpl_deflate_codec, false, "If it is set to true, allow usage of the DEFLATE_QPL codec.", 0) \ + M(Bool, enable_deflate_qpl_codec, false, "Enable/disable the DEFLATE_QPL codec.", 0) \ M(UInt64, query_profiler_real_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, query_profiler_cpu_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ddb53bbbfaa..d0bb3dd389f 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -571,7 +571,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( bool sanity_check_compression_codecs = !attach && !context_->getSettingsRef().allow_suspicious_codecs; bool allow_experimental_codecs = attach || context_->getSettingsRef().allow_experimental_codecs; - bool enable_qpl_deflate_codec = attach || context_->getSettingsRef().enable_qpl_deflate_codec; + bool enable_deflate_qpl_codec = attach || context_->getSettingsRef().enable_deflate_qpl_codec; ColumnsDescription res; auto name_type_it = column_names_and_types.begin(); @@ -632,7 +632,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.default_specifier == "ALIAS") throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot specify codec for column type ALIAS"); column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( - col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_qpl_deflate_codec); + col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec); } if (col_decl.ttl) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b43fef9dd54..50e9d50e2f6 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1775,7 +1775,7 @@ void TCPHandler::initBlockOutput(const Block & block) if (state.compression == Protocol::Compression::Enable) { - CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs, query_settings.enable_qpl_deflate_codec); + CompressionCodecFactory::instance().validateCodec(method, level, !query_settings.allow_suspicious_codecs, query_settings.allow_experimental_codecs, query_settings.enable_deflate_qpl_codec); state.maybe_compressed_out = std::make_shared( *out, CompressionCodecFactory::instance().get(method, level)); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 73d7be8dc56..a9247f9b898 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1067,7 +1067,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const "this column name is reserved for lightweight delete feature", backQuote(column_name)); if (command.codec) - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate_codec); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec); all_columns.add(ColumnDescription(column_name, command.data_type)); } @@ -1093,7 +1093,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { if (all_columns.hasAlias(column_name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot specify codec for column type ALIAS"); - CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate_codec); + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec); } auto column_default = all_columns.getDefault(column_name); if (column_default) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index e383890d1f7..1e1c911920e 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -733,7 +733,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const if (compression_method == "ZSTD") compression_level = settings.network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enable_qpl_deflate_codec); + CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enale_deflate_qpl_codec); CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level); /// tmp directory is used to ensure atomicity of transactions diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index a437465b3fe..f601fed06ac 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -285,7 +285,7 @@ TTLDescription TTLDescription::getTTLFromAST( { result.recompression_codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( - ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_qpl_deflate_codec); + ttl_element->recompression_codec, {}, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec); } } diff --git a/tests/ci/stress.py b/tests/ci/stress.py index e5ceb251d0f..6d17384c63f 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -20,7 +20,7 @@ def get_options(i, upgrade_check): '''--db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i) ) client_options.append("allow_experimental_database_replicated=1") - client_options.append("enable_qpl_deflate_codec=1") + client_options.append("enable_deflate_qpl_codec=1") # If database name is not specified, new database is created for each functional test. # Run some threads with one database for all tests. diff --git a/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml b/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml index 521b0fd663c..24e101e0e3f 100644 --- a/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml +++ b/tests/integration/test_non_default_compression/configs/enable_deflateqpl_codec.xml @@ -1,7 +1,7 @@ - 1 + 1 diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index 5b8b73270a2..fd9855e82d3 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -28,7 +28,7 @@ SELECT * FROM alter_compression_codec ORDER BY id; OPTIMIZE TABLE alter_compression_codec FINAL; SELECT * FROM alter_compression_codec ORDER BY id; -SET enable_qpl_deflate_codec = 1; +SET enable_deflate_qpl_codec = 1; ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(DEFLATE_QPL); SELECT compression_codec FROM system.columns WHERE database = currentDatabase() AND table = 'alter_compression_codec' AND name = 'alter_column'; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 47ec268bfec..89e77f758a7 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -1,6 +1,6 @@ SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; -SET enable_qpl_deflate_codec = 1; +SET enable_deflate_qpl_codec = 1; DROP TABLE IF EXISTS compression_codec; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql index bcd09277824..a629df2666d 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql @@ -1,6 +1,6 @@ SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; -SET enable_qpl_deflate_codec = 1; +SET enable_deflate_qpl_codec = 1; -- copy-paste for storage log diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql index 64e66d47522..5a56fc0d576 100644 --- a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql @@ -1,5 +1,5 @@ SET send_logs_level = 'fatal'; -SET enable_qpl_deflate_codec = 1; +SET enable_deflate_qpl_codec = 1; DROP TABLE IF EXISTS compression_codec; From 3c1b02a37bfa349fbf3af86277c9ad3ae0eadc1c Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 9 Jun 2023 15:43:36 +0300 Subject: [PATCH 1439/2223] Rectify the existing example of the year omission --- .../sql-reference/functions/type-conversion-functions.md | 8 ++++---- .../sql-reference/functions/type-conversion-functions.md | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index e62cf89a6b2..f1e2785285c 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1417,15 +1417,15 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffort('10 20:19'); +SELECT toYear(now()) as year, parseDateTimeBestEffort('10 20:19'); ``` Result: ```response -┌─parseDateTimeBestEffort('10 20:19')─┐ -│ 2000-01-10 20:19:00 │ -└─────────────────────────────────────┘ +┌─year─┬─parseDateTimeBestEffort('10 20:19')─┐ +│ 2023 │ 2023-01-10 20:19:00 │ +└──────┴─────────────────────────────────────┘ ``` **See Also** diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 6de55757b64..b763ee2b3ac 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -1096,15 +1096,15 @@ AS parseDateTimeBestEffort; Запрос: ``` sql -SELECT parseDateTimeBestEffort('10 20:19'); +SELECT toYear(now()) as year, parseDateTimeBestEffort('10 20:19'); ``` Результат: ``` text -┌─parseDateTimeBestEffort('10 20:19')─┐ -│ 2000-01-10 20:19:00 │ -└─────────────────────────────────────┘ +┌─year─┬─parseDateTimeBestEffort('10 20:19')─┐ +│ 2023 │ 2023-01-10 20:19:00 │ +└──────┴─────────────────────────────────────┘ ``` **Смотрите также** From 47b0c2a862c282b642a81ce0f0287c5059d717dc Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 9 Jun 2023 13:01:36 +0000 Subject: [PATCH 1440/2223] Make better --- src/Formats/CapnProtoSerializer.cpp | 117 +++++++++++++----- .../0_stateless/02030_capnp_format.reference | 11 ++ .../queries/0_stateless/02030_capnp_format.sh | 13 +- ...case_insensitive_names_matching.reference} | 0 ..._capnp_case_insensitive_names_matching.sh} | 0 5 files changed, 110 insertions(+), 31 deletions(-) rename tests/queries/0_stateless/{02735_capnp_case_insensitive_names_matcing.reference => 02735_capnp_case_insensitive_names_matching.reference} (100%) rename tests/queries/0_stateless/{02735_capnp_case_insensitive_names_matcing.sh => 02735_capnp_case_insensitive_names_matching.sh} (100%) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index e99db23bb5e..f51f8c4b933 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -110,7 +110,7 @@ namespace /// Write row as struct field. virtual void writeRow( const ColumnPtr & column, - std::unique_ptr & builder, + std::unique_ptr & builder, /// Maybe unused for simple types, needed to initialize structs and lists. capnp::DynamicStruct::Builder & parent_struct_builder, UInt32 slot_offset, size_t row_num) = 0; @@ -118,7 +118,7 @@ namespace /// Write row as list element. virtual void writeRow( const ColumnPtr & column, - std::unique_ptr & builder, + std::unique_ptr & builder, /// Maybe unused for simple types, needed to initialize structs and lists. capnp::DynamicList::Builder & parent_list_builder, UInt32 array_index, size_t row_num) = 0; @@ -262,54 +262,93 @@ namespace if (!capnp_type.isEnum()) throwCannotConvert(data_type, column_name, capnp_type); - bool to_lower = enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE; const auto * enum_type = assert_cast *>(data_type.get()); const auto & enum_values = dynamic_cast &>(*enum_type); enum_schema = capnp_type.asEnum(); auto enumerants = enum_schema.getEnumerants(); - constexpr auto max_value = std::is_same_v ? INT8_MAX : INT16_MAX; if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) { - /// In CapnProto Enum fields are numbered sequentially starting from zero. - if (enumerants.size() > max_value) - throw Exception( - ErrorCodes::CAPN_PROTO_BAD_CAST, - "Enum from CapnProto schema contains values that are out of range for Clickhouse enum type {}", - data_type->getName()); - - auto values = enum_values.getSetOfAllValues(); - std::unordered_set capn_enum_values; + auto ch_enum_values = enum_values.getSetOfAllValues(); + std::unordered_set capn_enum_values; for (auto enumerant : enumerants) - capn_enum_values.insert(EnumType(enumerant.getOrdinal())); - if (values != capn_enum_values) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "The set of values in Enum from CapnProto schema is different from the set of values in ClickHouse Enum"); + capn_enum_values.insert(enumerant.getOrdinal()); + + /// Check if ClickHouse values is a superset of CapnProto values. + ch_enum_is_superset = true; + /// In CapnProto Enum fields are numbered sequentially starting from zero. + /// Check if max CapnProto value exceeds max ClickHouse value. + constexpr auto max_value = std::is_same_v ? INT8_MAX : INT16_MAX; + if (enumerants.size() > max_value) + { + ch_enum_is_superset = false; + } + else + { + for (auto capnp_value : capn_enum_values) + { + if (!ch_enum_values.contains(static_cast(capnp_value))) + { + ch_enum_is_superset = false; + break; + } + } + } + + /// Check if CapnProto values is a superset of ClickHouse values. + capnp_enum_is_superset = true; + for (auto ch_value : ch_enum_values) + { + /// Capnp doesn't support negative enum values. + if (ch_value < 0 || !capn_enum_values.contains(static_cast(ch_value))) + { + capnp_enum_is_superset = false; + break; + } + } } else { - auto all_values = enum_values.getValues(); - if (all_values.size() != enumerants.size()) - throw Exception( - ErrorCodes::CAPN_PROTO_BAD_CAST, - "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"); + bool to_lower = enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE; + auto all_values = enum_values.getValues(); std::unordered_map ch_name_to_value; for (auto & [name, value] : all_values) ch_name_to_value[to_lower ? boost::algorithm::to_lower_copy(name) : name] = value; + std::unordered_map capnp_name_to_value; for (auto enumerant : enumerants) { String capnp_name = enumerant.getProto().getName(); - UInt16 capnp_value = enumerant.getOrdinal(); - auto it = ch_name_to_value.find(to_lower ? boost::algorithm::to_lower_copy(capnp_name) : capnp_name); - if (it == ch_name_to_value.end()) - throw Exception( - ErrorCodes::CAPN_PROTO_BAD_CAST, - "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"); + capnp_name_to_value[to_lower ? boost::algorithm::to_lower_copy(capnp_name) : capnp_name] = enumerant.getOrdinal(); + } - ch_to_capnp_values[it->second] = capnp_value; + /// Check if ClickHouse names is a superset of CapnProto names. + ch_enum_is_superset = true; + for (auto & [capnp_name, capnp_value] : capnp_name_to_value) + { + auto it = ch_name_to_value.find(capnp_name); + if (it == ch_name_to_value.end()) + { + ch_enum_is_superset = false; + break; + } capnp_to_ch_values[capnp_value] = it->second; } + + /// Check if CapnProto names is a superset of ClickHouse names. + capnp_enum_is_superset = true; + + for (auto & [ch_name, ch_value] : ch_name_to_value) + { + auto it = capnp_name_to_value.find(ch_name); + if (it == capnp_name_to_value.end()) + { + capnp_enum_is_superset = false; + break; + } + ch_to_capnp_values[ch_value] = it->second; + } } } @@ -336,23 +375,43 @@ namespace private: UInt16 getValue(const ColumnPtr & column, size_t row_num) { + if (!capnp_enum_is_superset) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Cannot convert ClickHouse enum to CapnProto enum: CapnProto enum values/names is not a superset of ClickHouse enum values/names"); + EnumType enum_value = assert_cast &>(*column).getElement(row_num); if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) return static_cast(enum_value); - return ch_to_capnp_values[enum_value]; + auto it = ch_to_capnp_values.find(enum_value); + if (it == ch_to_capnp_values.end()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected value {} in ClickHouse enum", enum_value); + + return it->second; } void insertValue(IColumn & column, UInt16 capnp_enum_value) { + if (!ch_enum_is_superset) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Cannot convert CapnProto enum to ClickHouse enum: ClickHouse enum values/names is not a superset of CapnProto enum values/names"); + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + { assert_cast &>(column).insertValue(static_cast(capnp_enum_value)); + } else + { + auto it = capnp_to_ch_values.find(capnp_enum_value); + if (it == capnp_to_ch_values.end()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected value {} in CapnProto enum", capnp_enum_value); + assert_cast &>(column).insertValue(capnp_to_ch_values[capnp_enum_value]); + } } DataTypePtr data_type; capnp::EnumSchema enum_schema; const FormatSettings::CapnProtoEnumComparingMode enum_comparing_mode; + bool ch_enum_is_superset; + bool capnp_enum_is_superset; std::unordered_map ch_to_capnp_values; std::unordered_map capnp_to_ch_values; }; diff --git a/tests/queries/0_stateless/02030_capnp_format.reference b/tests/queries/0_stateless/02030_capnp_format.reference index 2b2307bfc6a..e08b1eb1271 100644 --- a/tests/queries/0_stateless/02030_capnp_format.reference +++ b/tests/queries/0_stateless/02030_capnp_format.reference @@ -12,6 +12,9 @@ \N [NULL,NULL,42] (NULL) 1 [1,NULL,2] (1) \N [NULL,NULL,42] (NULL) +OK +OK +OK one two tHrEe @@ -21,6 +24,14 @@ threE first second third +first +second +third +OK +one +two +tHrEe +OK OK OK OK diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index 625104fb590..b4484ca3766 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -71,16 +71,25 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nullable" $CLICKHOUSE_CLIENT --query="SELECT CAST(number, 'Enum(\'one\' = 0, \'two\' = 1, \'tHrEe\' = 2)') AS value FROM numbers(3) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_enum:Message'" > $CAPN_PROTO_FILE +$CLICKHOUSE_CLIENT --query="SELECT CAST(number % 2, 'Enum(\'one\' = 0, \'two\' = 1, \'tHrEe\' = 2, \'four\' = 4)') AS value FROM numbers(3) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT CAST(number % 2, 'Enum(\'one\' = 0, \'two\' = 1, \'tHrEe\' = 2, \'four\' = 4)') AS value FROM numbers(3) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names_case_insensitive'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT CAST(number % 2, 'Enum(\'one\' = 0, \'two\' = 1, \'tHrEe\' = 2, \'four\' = 4)') AS value FROM numbers(3) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_values'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'one\' = 1, \'two\' = 2, \'tHrEe\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names'" $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'oNe\' = 1, \'tWo\' = 2, \'threE\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names_case_insensitive'" $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'first\' = 0, \'second\' = 1, \'third\' = 2)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_values'" - +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'first\' = 0, \'second\' = 1, \'third\' = 2)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_values'" $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'one\' = 0, \'two\' = 1, \'three\' = 2)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; -$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'one\' = 0, \'two\' = 1, \'tHrEe\' = 2, \'four\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'one\' = 0, \'two\' = 1, \'tHrEe\' = 2, \'four\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names'" $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'one\' = 1, \'two\' = 2, \'tHrEe\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_values'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'first\' = 1, \'two\' = 2, \'three\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names_case_insensitive'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT CAST(number % 2, 'Enum(\'one\' = 0, \'two\' = 1)') AS value FROM numbers(3) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_enum:Message'" > $CAPN_PROTO_FILE +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'first\' = 0, \'two\' = 1)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_values'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'first\' = 0, \'two\' = 1)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'first\' = 0, \'two\' = 1)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names_case_insensitive'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; + + $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_low_cardinality" $CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_low_cardinality (lc1 LowCardinality(String), lc2 LowCardinality(Nullable(String)), lc3 Array(LowCardinality(Nullable(String)))) ENGINE=Memory" $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_low_cardinality VALUES ('one', 'two', ['one', Null, 'two', Null]), ('two', Null, [Null])" diff --git a/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.reference b/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matching.reference similarity index 100% rename from tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.reference rename to tests/queries/0_stateless/02735_capnp_case_insensitive_names_matching.reference diff --git a/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.sh b/tests/queries/0_stateless/02735_capnp_case_insensitive_names_matching.sh similarity index 100% rename from tests/queries/0_stateless/02735_capnp_case_insensitive_names_matcing.sh rename to tests/queries/0_stateless/02735_capnp_case_insensitive_names_matching.sh From 9b70836b6c2b7f90ed04c41f90ad9ba3473dbe59 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 9 Jun 2023 16:07:07 +0300 Subject: [PATCH 1441/2223] Add a syslog format example to the documentation --- .../functions/type-conversion-functions.md | 22 +++++++++++++++++++ .../functions/type-conversion-functions.md | 22 +++++++++++++++++++ 2 files changed, 44 insertions(+) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index f1e2785285c..9c079bd9515 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1428,6 +1428,28 @@ Result: └──────┴─────────────────────────────────────┘ ``` +Query: + +``` sql +WITH + now() AS ts_now, + formatDateTime(ts_around, '%b %e %T') AS syslog_arg +SELECT + ts_now, + syslog_arg, + parseDateTimeBestEffort(syslog_arg) +FROM (SELECT arrayJoin([ts_now - 30, ts_now + 30]) AS ts_around); +``` + +Result: + +```response +┌──────────────ts_now─┬─syslog_arg──────┬─parseDateTimeBestEffort(syslog_arg)─┐ +│ 2023-06-09 16:04:30 │ Jun 9 16:04:00 │ 2023-06-09 16:04:00 │ +│ 2023-06-09 16:04:30 │ Jun 9 16:05:00 │ 2022-06-09 16:05:00 │ +└─────────────────────┴─────────────────┴─────────────────────────────────────┘ +``` + **See Also** - [RFC 1123](https://datatracker.ietf.org/doc/html/rfc1123) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index b763ee2b3ac..6e93a5e0acf 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -1107,6 +1107,28 @@ SELECT toYear(now()) as year, parseDateTimeBestEffort('10 20:19'); └──────┴─────────────────────────────────────┘ ``` +Запрос: + +``` sql +WITH + now() AS ts_now, + formatDateTime(ts_around, '%b %e %T') AS syslog_arg +SELECT + ts_now, + syslog_arg, + parseDateTimeBestEffort(syslog_arg) +FROM (SELECT arrayJoin([ts_now - 30, ts_now + 30]) AS ts_around); +``` + +Результат: + +``` text +┌──────────────ts_now─┬─syslog_arg──────┬─parseDateTimeBestEffort(syslog_arg)─┐ +│ 2023-06-09 16:04:30 │ Jun 9 16:04:00 │ 2023-06-09 16:04:00 │ +│ 2023-06-09 16:04:30 │ Jun 9 16:05:00 │ 2022-06-09 16:05:00 │ +└─────────────────────┴─────────────────┴─────────────────────────────────────┘ +``` + **Смотрите также** - [Информация о формате ISO 8601 от @xkcd](https://xkcd.com/1179/) From 8c4c82abd65753dcbe887c78baec2ab9c1f960d9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Jun 2023 15:11:07 +0200 Subject: [PATCH 1442/2223] Update lz4 to enable cutting-edge optimisations (#50621) --- contrib/lz4 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/lz4 b/contrib/lz4 index 4c9431e9af5..e82198428c8 160000 --- a/contrib/lz4 +++ b/contrib/lz4 @@ -1 +1 @@ -Subproject commit 4c9431e9af596af0556e5da0ae99305bafb2b10b +Subproject commit e82198428c8061372d5adef1f9bfff4203f6081e From 5af06f8c08a70298e217d3a8909a5ed8d412f474 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 9 Jun 2023 16:11:52 +0300 Subject: [PATCH 1443/2223] Amend the test infinitesimally --- .../0_stateless/02783_parseDateTimeBestEffort_syslog.reference | 2 +- .../0_stateless/02783_parseDateTimeBestEffort_syslog.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference index 63e7e367941..3ec93143e0e 100644 --- a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference +++ b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference @@ -1,5 +1,5 @@ parseDateTimeBestEffort - dt_ref res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc res_us res_us_sam res_us_auc res_us_null res_us_null_sam res_us_null_auc res_us_zero res_us_zero_sam res_us_zero_auc res64 res64_sam res64_auc res64_null res64_null_sam res64_null_auc res64_zero res64_zero_sam res64_zero_auc res64_us res64_us_sam res64_us_auc res64_us_null res64_us_null_sam res64_us_null_auc res64_us_zero res64_us_zero_sam res64_us_zero_auc + around_June_7 res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc res_us res_us_sam res_us_auc res_us_null res_us_null_sam res_us_null_auc res_us_zero res_us_zero_sam res_us_zero_auc res64 res64_sam res64_auc res64_null res64_null_sam res64_null_auc res64_zero res64_zero_sam res64_zero_auc res64_us res64_us_sam res64_us_auc res64_us_null res64_us_null_sam res64_us_null_auc res64_us_zero res64_us_zero_sam res64_us_zero_auc Jun 6 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 Jun 8 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 2022-06-08 00:00:00.000 diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql index 59211d3e6a0..52975cb5bbf 100644 --- a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql +++ b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql @@ -7,7 +7,7 @@ WITH dateDiff('second', toDateTime(ref_point), ts_now) AS impedimenta, formatDateTime(ts_around, '%b %e %T') AS dt_curr SELECT - formatDateTime(ts_around - impedimenta, '%b %e %H:%i:%s') AS dt_ref, + formatDateTime(ts_around - impedimenta, '%b %e %H:%i:%s') AS around_June_7, parseDateTimeBestEffort(dt_curr) - impedimenta AS res, parseDateTimeBestEffort(dt_curr, 'US/Samoa') - impedimenta AS res_sam, parseDateTimeBestEffort(dt_curr, 'Pacific/Auckland') - impedimenta AS res_auc, From ab0a01e4649d54bed7e3a5e1bc7ca22f97c9cec6 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 9 Jun 2023 09:14:50 -0400 Subject: [PATCH 1444/2223] close result block --- docs/en/operations/system-tables/asynchronous_insert_log.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/system-tables/asynchronous_insert_log.md b/docs/en/operations/system-tables/asynchronous_insert_log.md index 8b0509d7000..c3aaa8e6c41 100644 --- a/docs/en/operations/system-tables/asynchronous_insert_log.md +++ b/docs/en/operations/system-tables/asynchronous_insert_log.md @@ -56,6 +56,7 @@ status: Ok flush_time: 2023-06-08 10:08:55 flush_time_microseconds: 2023-06-08 10:08:55.139676 flush_query_id: cd2c1e43-83f5-49dc-92e4-2fbc7f8d3716 +``` **See Also** From 05cab78dd288c839d0d5bcafda070f8397c1fc53 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 9 Jun 2023 15:15:41 +0200 Subject: [PATCH 1445/2223] Fix bug in `uniqExact` parallel merging (#50590) * impl * disable test under sanitizers --- src/AggregateFunctions/UniqExactSet.h | 51 +++++++++++-------- ..._uniq_exact_parallel_merging_bug.reference | 0 .../02782_uniq_exact_parallel_merging_bug.sh | 21 ++++++++ 3 files changed, 51 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/02782_uniq_exact_parallel_merging_bug.reference create mode 100755 tests/queries/0_stateless/02782_uniq_exact_parallel_merging_bug.sh diff --git a/src/AggregateFunctions/UniqExactSet.h b/src/AggregateFunctions/UniqExactSet.h index 916dfe4a424..90cfe700179 100644 --- a/src/AggregateFunctions/UniqExactSet.h +++ b/src/AggregateFunctions/UniqExactSet.h @@ -1,10 +1,11 @@ #pragma once +#include #include #include #include -#include #include +#include namespace DB @@ -48,30 +49,38 @@ public: } else { - auto next_bucket_to_merge = std::make_shared(0); - - auto thread_func = [&lhs, &rhs, next_bucket_to_merge, thread_group = CurrentThread::getGroup()]() + try { - SCOPE_EXIT_SAFE( - if (thread_group) - CurrentThread::detachFromGroupIfNotDetached(); - ); - if (thread_group) - CurrentThread::attachToGroupIfDetached(thread_group); - setThreadName("UniqExactMerger"); + auto next_bucket_to_merge = std::make_shared(0); - while (true) + auto thread_func = [&lhs, &rhs, next_bucket_to_merge, thread_group = CurrentThread::getGroup()]() { - const auto bucket = next_bucket_to_merge->fetch_add(1); - if (bucket >= rhs.NUM_BUCKETS) - return; - lhs.impls[bucket].merge(rhs.impls[bucket]); - } - }; + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachFromGroupIfNotDetached(); + ); + if (thread_group) + CurrentThread::attachToGroupIfDetached(thread_group); + setThreadName("UniqExactMerger"); - for (size_t i = 0; i < std::min(thread_pool->getMaxThreads(), rhs.NUM_BUCKETS); ++i) - thread_pool->scheduleOrThrowOnError(thread_func); - thread_pool->wait(); + while (true) + { + const auto bucket = next_bucket_to_merge->fetch_add(1); + if (bucket >= rhs.NUM_BUCKETS) + return; + lhs.impls[bucket].merge(rhs.impls[bucket]); + } + }; + + for (size_t i = 0; i < std::min(thread_pool->getMaxThreads(), rhs.NUM_BUCKETS); ++i) + thread_pool->scheduleOrThrowOnError(thread_func); + thread_pool->wait(); + } + catch (...) + { + thread_pool->wait(); + throw; + } } } } diff --git a/tests/queries/0_stateless/02782_uniq_exact_parallel_merging_bug.reference b/tests/queries/0_stateless/02782_uniq_exact_parallel_merging_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02782_uniq_exact_parallel_merging_bug.sh b/tests/queries/0_stateless/02782_uniq_exact_parallel_merging_bug.sh new file mode 100755 index 00000000000..d84ffd21b87 --- /dev/null +++ b/tests/queries/0_stateless/02782_uniq_exact_parallel_merging_bug.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash +# Tags: long, no-random-settings, no-tsan, no-asan, no-ubsan, no-msan + +# shellcheck disable=SC2154 + +unset CLICKHOUSE_LOG_COMMENT + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +clickhouse-client -q " + CREATE TABLE ${CLICKHOUSE_DATABASE}.t(s String) + ENGINE = MergeTree + ORDER BY tuple(); +" + +clickhouse-client -q "insert into ${CLICKHOUSE_DATABASE}.t select number%10==0 ? toString(number) : '' from numbers_mt(1e7)" + +clickhouse-benchmark -q "select count(distinct s) from ${CLICKHOUSE_DATABASE}.t settings max_memory_usage = '50Mi'" --ignore-error -c 16 -i 1000 2>/dev/null From 07582d56f32d72a3f13d9e7303310ff1753c97f5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Jun 2023 13:18:13 +0000 Subject: [PATCH 1446/2223] Add config for pinning api version --- src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/KeeperConstants.h | 4 +- src/Coordination/KeeperContext.cpp | 37 ++++++++ src/Coordination/KeeperContext.h | 6 ++ src/Coordination/KeeperServer.cpp | 2 + src/Coordination/KeeperSnapshotManager.cpp | 2 +- src/Coordination/KeeperStorage.cpp | 2 +- src/Coordination/tests/gtest_coordination.cpp | 2 +- .../__init__.py | 0 .../configs/enable_keeper.xml | 31 +++++++ .../test_keeper_api_version_config/test.py | 87 +++++++++++++++++++ 11 files changed, 169 insertions(+), 6 deletions(-) create mode 100644 src/Coordination/KeeperContext.cpp create mode 100644 tests/integration/test_keeper_api_version_config/__init__.py create mode 100644 tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml create mode 100644 tests/integration/test_keeper_api_version_config/test.py diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 7077e792fd8..a64969e3d31 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -486,7 +486,7 @@ String RecoveryCommand::run() String ApiVersionCommand::run() { - return toString(static_cast(Coordination::current_keeper_api_version)); + return toString(static_cast(Coordination::latest_keeper_api_version)); } String CreateSnapshotCommand::run() diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 4b5a5b54be0..42161eee908 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -13,7 +13,7 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; -inline constexpr auto current_keeper_api_version = KeeperApiVersion::WITH_CHECK_NOT_EXISTS; +inline constexpr auto latest_keeper_api_version = KeeperApiVersion::WITH_CHECK_NOT_EXISTS; const std::string keeper_system_path = "/keeper"; const std::string keeper_api_version_path = keeper_system_path + "/api_version"; @@ -21,7 +21,7 @@ const std::string keeper_api_version_path = keeper_system_path + "/api_version"; using PathWithData = std::pair; const std::vector child_system_paths_with_data { - {keeper_api_version_path, toString(static_cast(current_keeper_api_version))} + {keeper_api_version_path, toString(static_cast(latest_keeper_api_version))} }; } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp new file mode 100644 index 00000000000..d420242e670 --- /dev/null +++ b/src/Coordination/KeeperContext.cpp @@ -0,0 +1,37 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +DECLARE_SETTING_ENUM(KeeperApiVersion); +IMPLEMENT_SETTING_ENUM(KeeperApiVersion, ErrorCodes::BAD_ARGUMENTS, + {{"ZOOKEEPER_COMPATIBLE", KeeperApiVersion::ZOOKEEPER_COMPATIBLE}, + {"WITH_FILTERED_LIST", KeeperApiVersion::WITH_FILTERED_LIST}, + {"WITH_MULTI_READ", KeeperApiVersion::WITH_MULTI_READ}, + {"WITH_CHECK_NOT_EXISTS", KeeperApiVersion::WITH_CHECK_NOT_EXISTS}}); + +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +{ + for (const auto & [path, data] : child_system_paths_with_data) + system_nodes_with_data[std::string{path}] = data; + + if (config.has("keeper_server.api_version")) + { + auto version_string = config.getString("keeper_server.api_version"); + auto api_version = SettingFieldKeeperApiVersionTraits::fromString(version_string); + LOG_INFO(&Poco::Logger::get("KeeperContext"), "API version override used: {}", version_string); + system_nodes_with_data[keeper_api_version_path] = toString(static_cast(api_version)); + } +} + +} diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 64fa8cea6ec..f26009c9af0 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,10 +1,14 @@ #pragma once +#include + namespace DB { struct KeeperContext { + void initialize(const Poco::Util::AbstractConfiguration & config); + enum class Phase : uint8_t { INIT, @@ -16,6 +20,8 @@ struct KeeperContext bool ignore_system_path_on_startup{false}; bool digest_enabled{true}; + + std::unordered_map system_nodes_with_data; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 45db9e85fa5..897d7e05671 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -119,6 +119,8 @@ KeeperServer::KeeperServer( if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); + keeper_context->initialize(config); + keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); keeper_context->ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 8b80db3e520..a2d9d8136cd 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -185,7 +185,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr } /// Serialize data tree - writeBinary(snapshot.snapshot_container_size - child_system_paths_with_data.size(), out); + writeBinary(snapshot.snapshot_container_size - keeper_context->system_nodes_with_data.size(), out); size_t counter = 0; for (auto it = snapshot.begin; counter < snapshot.snapshot_container_size; ++counter) { diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 66d6b0f5843..8abcd062b7f 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -283,7 +283,7 @@ void KeeperStorage::initializeSystemNodes() } // insert child system nodes - for (const auto & [path, data] : child_system_paths_with_data) + for (const auto & [path, data] : keeper_context->system_nodes_with_data) { assert(keeper_api_version_path.starts_with(keeper_system_path)); Node child_system_node; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 453fd0f2e60..005c67ad261 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2357,7 +2357,7 @@ TEST_P(CoordinationTest, TestCurrentApiVersion) uint8_t keeper_version{0}; DB::ReadBufferFromOwnString buf(get_response.data); DB::readIntText(keeper_version, buf); - EXPECT_EQ(keeper_version, static_cast(current_keeper_api_version)); + EXPECT_EQ(keeper_version, static_cast(latest_keeper_api_version)); } TEST_P(CoordinationTest, TestSystemNodeModify) diff --git a/tests/integration/test_keeper_api_version_config/__init__.py b/tests/integration/test_keeper_api_version_config/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml b/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml new file mode 100644 index 00000000000..c153d025598 --- /dev/null +++ b/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml @@ -0,0 +1,31 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 10 + 5 + 5000 + 10000 + trace + + + 0 + 0 + 0 + + + + + + + 1 + localhost + 9234 + + + + diff --git a/tests/integration/test_keeper_api_version_config/test.py b/tests/integration/test_keeper_api_version_config/test.py new file mode 100644 index 00000000000..34d3acc4f04 --- /dev/null +++ b/tests/integration/test_keeper_api_version_config/test.py @@ -0,0 +1,87 @@ +#!/usr/bin/env python3 + +import pytest +import os +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +from kazoo.client import KazooClient, KazooState + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +cluster = ClickHouseCluster(__file__) + +# clickhouse itself will use external zookeeper +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_connection_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def restart_clickhouse(api_version=None, expect_fail=True): + node.stop_clickhouse() + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), + "/etc/clickhouse-server/config.d/enable_keeper.xml", + ) + + if api_version: + node.replace_in_config( + "/etc/clickhouse-server/config.d/enable_keeper.xml", + "", + f"{api_version}<\\/api_version>", + ) + + node.start_clickhouse(retry_start=not expect_fail) + keeper_utils.wait_until_connected(cluster, node) + + +def test_keeper_api_version(started_cluster): + restart_clickhouse() + + def assert_version(string_version, version_number): + node.wait_for_log_line( + f"Detected server's API version: {string_version}", look_behind_lines=1000 + ) + + try: + node_zk = get_connection_zk(node.name) + assert node_zk.get("/keeper/api_version")[0] == str(version_number).encode() + finally: + if node_zk: + node_zk.stop() + node_zk.close() + + assert_version("WITH_CHECK_NOT_EXISTS", 3) + + for i, version in enumerate( + [ + "ZOOKEEPER_COMPATIBLE", + "WITH_FILTERED_LIST", + "WITH_MULTI_READ", + "WITH_CHECK_NOT_EXISTS", + ] + ): + restart_clickhouse(version) + assert_version(version, i) + + with pytest.raises(Exception): + restart_clickhouse("INVALID_VERSION", expect_fail=True) From b8b70d78e15948dd278d298d42bc1e4ef2d4fc0d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Jun 2023 13:34:27 +0000 Subject: [PATCH 1447/2223] Build fix --- contrib/azure-cmake/CMakeLists.txt | 2 +- programs/keeper/CMakeLists.txt | 1 + src/Coordination/KeeperContext.cpp | 7 +++++-- src/Coordination/KeeperContext.h | 2 ++ 4 files changed, 9 insertions(+), 3 deletions(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 9c361db47ca..23e38e6b63d 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD OR ARCH_PPC64LE) +if (NOT ENABLE_AZURE_BLOB_STORAGE OR OS_FREEBSD OR ARCH_PPC64LE) message(STATUS "Not using Azure blob storage") return() endif() diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 1f1138f49eb..4f74cc06801 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -43,6 +43,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperDispatcher.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperLogStore.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperServer.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperContext.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManagerS3.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index d420242e670..a750f2e1860 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -20,11 +20,14 @@ IMPLEMENT_SETTING_ENUM(KeeperApiVersion, ErrorCodes::BAD_ARGUMENTS, {"WITH_MULTI_READ", KeeperApiVersion::WITH_MULTI_READ}, {"WITH_CHECK_NOT_EXISTS", KeeperApiVersion::WITH_CHECK_NOT_EXISTS}}); -void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +KeeperContext::KeeperContext() { for (const auto & [path, data] : child_system_paths_with_data) system_nodes_with_data[std::string{path}] = data; - +} + +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +{ if (config.has("keeper_server.api_version")) { auto version_string = config.getString("keeper_server.api_version"); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index f26009c9af0..de502b6c566 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -7,6 +7,8 @@ namespace DB struct KeeperContext { + KeeperContext(); + void initialize(const Poco::Util::AbstractConfiguration & config); enum class Phase : uint8_t From f8791a0ea393120dbfba8eec8627edbc8d00deb8 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Fri, 9 Jun 2023 15:36:48 +0200 Subject: [PATCH 1448/2223] SummingMergeTree support for DateTime64 --- src/DataTypes/DataTypeDateTime64.h | 2 ++ .../02785_summing_merge_tree_datetime64.reference | 1 + .../02785_summing_merge_tree_datetime64.sql | 12 ++++++++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02785_summing_merge_tree_datetime64.reference create mode 100644 tests/queries/0_stateless/02785_summing_merge_tree_datetime64.sql diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index aaa99485040..64cedd798d1 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -37,6 +37,8 @@ public: bool canBeUsedAsVersion() const override { return true; } + bool isSummable() const override { return false; } + protected: SerializationPtr doGetDefaultSerialization() const override; }; diff --git a/tests/queries/0_stateless/02785_summing_merge_tree_datetime64.reference b/tests/queries/0_stateless/02785_summing_merge_tree_datetime64.reference new file mode 100644 index 00000000000..d395c4d6a0f --- /dev/null +++ b/tests/queries/0_stateless/02785_summing_merge_tree_datetime64.reference @@ -0,0 +1 @@ +1 2023-05-01 23:55:55.100 15 diff --git a/tests/queries/0_stateless/02785_summing_merge_tree_datetime64.sql b/tests/queries/0_stateless/02785_summing_merge_tree_datetime64.sql new file mode 100644 index 00000000000..1ed930ebbc7 --- /dev/null +++ b/tests/queries/0_stateless/02785_summing_merge_tree_datetime64.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS summing_merge_tree_datetime64; + +CREATE TABLE summing_merge_tree_datetime64 ( `pk` UInt64, `timestamp` DateTime64(3), `value` UInt64 ) +ENGINE = SummingMergeTree() ORDER BY pk; + +INSERT INTO summing_merge_tree_datetime64 SELECT 1 pk, '2023-05-01 23:55:55.100' timestamp, 1 value; +INSERT INTO summing_merge_tree_datetime64 SELECT 1 pk, '2023-05-01 23:55:55.100' timestamp, 2 value; +INSERT INTO summing_merge_tree_datetime64 SELECT 1 pk, '2023-05-01 23:55:55.100' timestamp, 3 value; +INSERT INTO summing_merge_tree_datetime64 SELECT 1 pk, '2023-05-01 23:55:55.100' timestamp, 4 value; +INSERT INTO summing_merge_tree_datetime64 SELECT 1 pk, '2023-05-01 23:55:55.100' timestamp, 5 value; + +SELECT * FROM summing_merge_tree_datetime64 FINAL; From af153399bf19d9dbb22253d6224fca37616401e3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 9 Jun 2023 15:26:13 +0000 Subject: [PATCH 1449/2223] Update version_date.tsv and changelogs after v23.2.7.32-stable --- docs/changelogs/v23.2.7.32-stable.md | 35 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 36 insertions(+) create mode 100644 docs/changelogs/v23.2.7.32-stable.md diff --git a/docs/changelogs/v23.2.7.32-stable.md b/docs/changelogs/v23.2.7.32-stable.md new file mode 100644 index 00000000000..db5e9e76311 --- /dev/null +++ b/docs/changelogs/v23.2.7.32-stable.md @@ -0,0 +1,35 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.2.7.32-stable (934f6a2aa0e) FIXME as compared to v23.2.6.34-stable (570190045b0) + +#### Performance Improvement +* Backported in [#49218](https://github.com/ClickHouse/ClickHouse/issues/49218): Fixed excessive reading in queries with `FINAL`. [#47801](https://github.com/ClickHouse/ClickHouse/pull/47801) ([Nikita Taranov](https://github.com/nickitat)). + +#### Build/Testing/Packaging Improvement +* Backported in [#49208](https://github.com/ClickHouse/ClickHouse/issues/49208): Fix glibc compatibility check: replace `preadv` from musl. [#49144](https://github.com/ClickHouse/ClickHouse/pull/49144) ([alesapin](https://github.com/alesapin)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix key not found error for queries with multiple StorageJoin [#49137](https://github.com/ClickHouse/ClickHouse/pull/49137) ([vdimir](https://github.com/vdimir)). +* Fix race on Outdated parts loading [#49223](https://github.com/ClickHouse/ClickHouse/pull/49223) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix bug in DISTINCT [#49628](https://github.com/ClickHouse/ClickHouse/pull/49628) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix msan issue in randomStringUTF8() [#49750](https://github.com/ClickHouse/ClickHouse/pull/49750) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix IPv6 encoding in protobuf [#49933](https://github.com/ClickHouse/ClickHouse/pull/49933) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Avoid deadlock when starting table in attach thread of `ReplicatedMergeTree` [#50026](https://github.com/ClickHouse/ClickHouse/pull/50026) ([Antonio Andelic](https://github.com/antonio2368)). +* JIT compilation not equals NaN fix [#50056](https://github.com/ClickHouse/ClickHouse/pull/50056) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix crash with `multiIf` and constant condition and nullable arguments [#50123](https://github.com/ClickHouse/ClickHouse/pull/50123) ([Anton Popov](https://github.com/CurtizJ)). +* Fix Keeper deadlock on exception when preprocessing requests. [#50387](https://github.com/ClickHouse/ClickHouse/pull/50387) ([frinkr](https://github.com/frinkr)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Improve test reports [#49151](https://github.com/ClickHouse/ClickHouse/pull/49151) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fallback auth gh api [#49314](https://github.com/ClickHouse/ClickHouse/pull/49314) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Improve CI: status commit, auth for get_gh_api [#49388](https://github.com/ClickHouse/ClickHouse/pull/49388) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update github.com/distribution/distribution [#50114](https://github.com/ClickHouse/ClickHouse/pull/50114) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Catch issues with dockerd during the build [#50700](https://github.com/ClickHouse/ClickHouse/pull/50700) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 9704c68be54..2aeeb5db35c 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -3,6 +3,7 @@ v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 v23.3.2.37-lts 2023-04-22 v23.3.1.2823-lts 2023-03-31 +v23.2.7.32-stable 2023-06-09 v23.2.6.34-stable 2023-04-23 v23.2.5.46-stable 2023-04-03 v23.2.4.12-stable 2023-03-10 From f8dc408ccbb600bd0c387feeedf899448501b8b1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Jun 2023 16:23:22 +0000 Subject: [PATCH 1450/2223] Desctructing --> Destructing --- src/Interpreters/Context.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8fb06e21d22..995e78d8f0b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -377,7 +377,7 @@ struct ContextSharedPart : boost::noncopyable { try { - LOG_DEBUG(log, "Desctructing remote fs threadpool reader"); + LOG_DEBUG(log, "Destructing remote fs threadpool reader"); asynchronous_remote_fs_reader->wait(); asynchronous_remote_fs_reader.reset(); } @@ -391,7 +391,7 @@ struct ContextSharedPart : boost::noncopyable { try { - LOG_DEBUG(log, "Desctructing local fs threadpool reader"); + LOG_DEBUG(log, "Destructing local fs threadpool reader"); asynchronous_local_fs_reader->wait(); asynchronous_local_fs_reader.reset(); } @@ -405,7 +405,7 @@ struct ContextSharedPart : boost::noncopyable { try { - LOG_DEBUG(log, "Desctructing local fs threadpool reader"); + LOG_DEBUG(log, "Destructing local fs threadpool reader"); synchronous_local_fs_reader->wait(); synchronous_local_fs_reader.reset(); } @@ -419,7 +419,7 @@ struct ContextSharedPart : boost::noncopyable { try { - LOG_DEBUG(log, "Desctructing threadpool writer"); + LOG_DEBUG(log, "Destructing threadpool writer"); threadpool_writer->wait(); threadpool_writer.reset(); } @@ -433,7 +433,7 @@ struct ContextSharedPart : boost::noncopyable { try { - LOG_DEBUG(log, "Desctructing marks loader"); + LOG_DEBUG(log, "Destructing marks loader"); load_marks_threadpool->wait(); load_marks_threadpool.reset(); } @@ -447,7 +447,7 @@ struct ContextSharedPart : boost::noncopyable { try { - LOG_DEBUG(log, "Desctructing prefetch threadpool"); + LOG_DEBUG(log, "Destructing prefetch threadpool"); prefetch_threadpool->wait(); prefetch_threadpool.reset(); } From 13798f8b07f2550ec73323ce8596a276155aa367 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 9 Jun 2023 19:52:49 +0300 Subject: [PATCH 1451/2223] Update MergeTreeData.cpp --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9cca471fddb..23351423d49 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1252,6 +1252,10 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( mark_broken(); return res; } + catch (const Poco::TimeoutException &) + { + throw; + } catch (...) { mark_broken(); From a8b579a85618f57f0fd6316d16d28677dfeb0d8b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 9 Jun 2023 19:28:06 +0200 Subject: [PATCH 1452/2223] Rename azure_blob_storage to azureBlobStorage --- .../table-functions/azure_blob_storage.md | 2 +- .../TableFunctionAzureBlobStorage.cpp | 2 +- .../TableFunctionAzureBlobStorage.h | 2 +- .../test_storage_azure_blob_storage/test.py | 22 +++++++++---------- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/table-functions/azure_blob_storage.md b/docs/en/sql-reference/table-functions/azure_blob_storage.md index f86307b3b85..6091aab5f9d 100644 --- a/docs/en/sql-reference/table-functions/azure_blob_storage.md +++ b/docs/en/sql-reference/table-functions/azure_blob_storage.md @@ -5,7 +5,7 @@ sidebar_label: azure_blob_storage keywords: [azure blob storage] --- -# azure\_blob\_storage Table Function +# azureBlobStorage Table Function Provides a table-like interface to select/insert files in [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs). This table function is similar to the [s3 function](../../sql-reference/table-functions/s3.md). diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 38d9362894a..d2a96173491 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -246,7 +246,7 @@ void registerTableFunctionAzureBlobStorage(TableFunctionFactory & factory) factory.registerFunction( {.documentation = {.description=R"(The table function can be used to read the data stored on Azure Blob Storage.)", - .examples{{"azure_blob_storage", "SELECT * FROM azure_blob_storage(connection, container, blob_path, format, structure)", ""}}}, + .examples{{"azureBlobStorage", "SELECT * FROM azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])", ""}}}, .allow_readonly = false}); } diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.h b/src/TableFunctions/TableFunctionAzureBlobStorage.h index 0bb872de3f3..0ac3f9771c7 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.h +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.h @@ -18,7 +18,7 @@ class Context; class TableFunctionAzureBlobStorage : public ITableFunction { public: - static constexpr auto name = "azure_blob_storage"; + static constexpr auto name = "azureBlobStorage"; static constexpr auto signature = "- connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]\n"; static size_t getMaxNumberOfArguments() { return 8; } diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f0934d3aa80..f9d337b6d86 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -466,7 +466,7 @@ def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] azure_query( node, - "INSERT INTO TABLE FUNCTION azure_blob_storage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", ) print(get_azure_file_content("test_simple_write_tf.csv")) assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' @@ -476,7 +476,7 @@ def test_simple_write_connection_string_table_function(cluster): node = cluster.instances["node"] azure_query( node, - "INSERT INTO TABLE FUNCTION azure_blob_storage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + "INSERT INTO TABLE FUNCTION azureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", ) print(get_azure_file_content("test_simple_write_connection_tf.csv")) assert get_azure_file_content("test_simple_write_connection_tf.csv") == '1,"a"\n' @@ -486,7 +486,7 @@ def test_simple_write_named_collection_1_table_function(cluster): node = cluster.instances["node"] azure_query( node, - "INSERT INTO TABLE FUNCTION azure_blob_storage(azure_conf1) VALUES (1, 'a')", + "INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf1) VALUES (1, 'a')", ) print(get_azure_file_content("test_simple_write_named.csv")) assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' @@ -507,7 +507,7 @@ def test_simple_write_named_collection_2_table_function(cluster): azure_query( node, - "INSERT INTO TABLE FUNCTION azure_blob_storage(azure_conf2, container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", + "INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", ) print(get_azure_file_content("test_simple_write_named_2_tf.csv")) assert get_azure_file_content("test_simple_write_named_2_tf.csv") == '1,"a"\n' @@ -529,9 +529,9 @@ def test_put_get_with_globs_tf(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azure_blob_storage(azure_conf2, container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", ) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azure_blob_storage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" + query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" assert azure_query(node, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path @@ -543,10 +543,10 @@ def test_schema_inference_no_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 String, column3 UInt32" - query = f"insert into table function azure_blob_storage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') SELECT number, toString(number), number * number FROM numbers(1000)" + query = f"insert into table function azureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') SELECT number, toString(number), number * number FROM numbers(1000)" azure_query(node, query) - query = "select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from azure_blob_storage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv')" + query = "select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv')" assert azure_query(node, query).splitlines() == [ "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] @@ -566,10 +566,10 @@ def test_schema_inference_from_globs_tf(cluster): max_path = max(path, max_path) values = f"({i},{j},{i + j})" - query = f"insert into table function azure_blob_storage(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" + query = f"insert into table function azureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" azure_query(node, query) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azure_blob_storage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" + query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" assert azure_query(node, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path @@ -586,7 +586,7 @@ def test_partition_by_tf(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azure_blob_storage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", ) assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv") From 9a4043a4b4c97bcfb7eb345e0753b27228c2f4f7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Jun 2023 17:51:59 +0000 Subject: [PATCH 1453/2223] Fixing more tests. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 1 - src/Interpreters/PreparedSets.h | 6 ++ src/Planner/CollectSets.cpp | 25 +++--- src/Planner/CollectSets.h | 2 +- src/Planner/Planner.cpp | 103 +++++++++++----------- src/Planner/Utils.cpp | 4 +- src/Storages/StorageDistributed.cpp | 11 ++- 7 files changed, 82 insertions(+), 70 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index b39aff86d32..bab64480901 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2333,7 +2333,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); - auto result = std::make_shared(std::move(storage), std::move(storage_lock), std::move(storage_snapshot)); if (is_temporary_table) result->setTemporaryTableName(table_name); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index fa7f7069994..a119c24ad10 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -29,6 +29,9 @@ class Set; using SetPtr = std::shared_ptr; class InterpreterSelectWithUnionQuery; +class IQueryTreeNode; +using QueryTreeNodePtr = std::shared_ptr; + /// Represents a set in a query that might be referenced at analysis time and built later during execution. /// Also it can represent a constant set that is ready to use. /// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there @@ -131,6 +134,7 @@ public: /// The source is obtained using the InterpreterSelectQuery subquery. std::unique_ptr source; + QueryTreeNodePtr query_tree; }; class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this @@ -153,6 +157,8 @@ public: // void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } + SubqueryForSet & getSubquery() { return subquery; } + private: //SetPtr set; SubqueryForSet subquery; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 5f44994c14b..b3e3f5f472a 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -28,9 +28,9 @@ namespace class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor { public: - explicit CollectSetsVisitor(PlannerContext & planner_context_, const SelectQueryOptions & select_query_options_) + explicit CollectSetsVisitor(PlannerContext & planner_context_) //, const SelectQueryOptions & select_query_options_) : planner_context(planner_context_) - , select_query_options(select_query_options_) + //, select_query_options(select_query_options_) {} void visitImpl(const QueryTreeNodePtr & node) @@ -95,12 +95,12 @@ public: if (sets.getFuture(set_key)) return; - auto subquery_options = select_query_options.subquery(); - Planner subquery_planner( - in_second_argument, - subquery_options, - planner_context.getGlobalPlannerContext()); - subquery_planner.buildQueryPlanIfNeeded(); + // auto subquery_options = select_query_options.subquery(); + // Planner subquery_planner( + // in_second_argument, + // subquery_options, + // planner_context.getGlobalPlannerContext()); + // subquery_planner.buildQueryPlanIfNeeded(); // const auto & settings = planner_context.getQueryContext()->getSettingsRef(); // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; @@ -109,7 +109,8 @@ public: SubqueryForSet subquery_for_set; subquery_for_set.key = planner_context.createSetKey(in_second_argument); - subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + subquery_for_set.query_tree = in_second_argument; + //subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); /// TODO sets.addFromSubquery(set_key, std::move(subquery_for_set), settings, nullptr); @@ -132,14 +133,14 @@ public: private: PlannerContext & planner_context; - const SelectQueryOptions & select_query_options; + //const SelectQueryOptions & select_query_options; }; } -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options) +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) //, const SelectQueryOptions & select_query_options) { - CollectSetsVisitor visitor(planner_context, select_query_options); + CollectSetsVisitor visitor(planner_context); //, select_query_options); visitor.visit(node); } diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index 57e662a392e..e0db802d3b4 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -12,6 +12,6 @@ struct SelectQueryOptions; /** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations. * Collected sets are registered in planner context. */ -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options); +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); //, const SelectQueryOptions & select_query_options); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 3184e229c15..30510d05840 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -893,50 +894,62 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana query_plan.addStep(std::move(offsets_step)); } -// void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, -// const SelectQueryOptions & select_query_options, -// const PlannerContextPtr & planner_context, -// const std::vector & result_actions_to_execute) -// { -// PreparedSets::SubqueriesForSets subqueries_for_sets; +void addBuildSubqueriesForSetsStepIfNeeded( + QueryPlan & query_plan, + const SelectQueryOptions & select_query_options, + const PlannerContextPtr & planner_context, + const std::vector & result_actions_to_execute) +{ + auto subqueries = planner_context->getPreparedSets().detachSubqueries(); + std::unordered_set useful_sets; -// for (const auto & actions_to_execute : result_actions_to_execute) -// { -// for (const auto & node : actions_to_execute->getNodes()) -// { -// const auto & set_key = node.result_name; -// auto * planner_set = planner_context->getSetOrNull(set_key); -// if (!planner_set) -// continue; + PreparedSets::SubqueriesForSets subqueries_for_sets; -// if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) -// continue; + for (const auto & actions_to_execute : result_actions_to_execute) + { + for (const auto & node : actions_to_execute->getNodes()) + { + if (node.column) + { + const IColumn * column = node.column.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); -// auto subquery_options = select_query_options.subquery(); -// Planner subquery_planner( -// planner_set->getSubqueryNode(), -// subquery_options, -// planner_context->getGlobalPlannerContext()); -// subquery_planner.buildQueryPlanIfNeeded(); + if (const auto * column_set = typeid_cast(column)) + useful_sets.insert(column_set->getData().get()); + } + } + } -// const auto & settings = planner_context->getQueryContext()->getSettingsRef(); -// SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; -// bool tranform_null_in = settings.transform_null_in; -// auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); + auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.set.get()); }; + auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); + subqueries.erase(it, subqueries.end()); -// SubqueryForSet subquery_for_set; -// subquery_for_set.key = set_key; -// subquery_for_set.set_in_progress = set; -// subquery_for_set.set = planner_set->getSet(); -// subquery_for_set.promise_to_fill_set = planner_set->extractPromiseToBuildSet(); -// subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + for (auto & subquery : subqueries) + { + auto & subquery_for_set = subquery.set->getSubquery(); + auto subquery_options = select_query_options.subquery(); + Planner subquery_planner( + subquery_for_set.query_tree, + subquery_options, + planner_context->getGlobalPlannerContext()); + subquery_planner.buildQueryPlanIfNeeded(); -// subqueries_for_sets.emplace(set_key, std::move(subquery_for_set)); -// } -// } + subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + } -// addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); -// } + //addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); + + if (!subqueries.empty()) + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + std::move(subqueries), + planner_context->getQueryContext()); + + query_plan.addStep(std::move(step)); + } +} /// Support for `additional_result_filter` setting void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, @@ -1197,7 +1210,7 @@ void Planner::buildPlanForQueryNode() } checkStoragesSupportTransactions(planner_context); - collectSets(query_tree, *planner_context, select_query_options); + collectSets(query_tree, *planner_context); //, select_query_options); collectTableExpressionData(query_tree, planner_context); const auto & settings = query_context->getSettingsRef(); @@ -1497,20 +1510,8 @@ void Planner::buildPlanForQueryNode() if (!select_query_options.only_analyze) { - auto subqueries = planner_context->getPreparedSets().detachSubqueries(); - - if (!subqueries.empty()) - { - auto step = std::make_unique( - query_plan.getCurrentDataStream(), - std::move(subqueries), - planner_context->getQueryContext()); - - query_plan.addStep(std::move(step)); - } - //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); - //addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); } } diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 2b4febf58ea..372bb15822a 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -426,7 +426,7 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options, + [[maybe_unused]] const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter) { const auto & query_context = planner_context->getQueryContext(); @@ -444,7 +444,7 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, } collectSourceColumns(filter_query_tree, planner_context); - collectSets(filter_query_tree, *planner_context, select_query_options); + collectSets(filter_query_tree, *planner_context); //, select_query_options); auto filter_actions_dag = std::make_shared(); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b0fb07d69ce..64a621e5710 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include @@ -898,9 +899,13 @@ QueryTreeNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, temporary_table_expression_node->setTemporaryTableName(temporary_table_name); auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context); - auto io = interpreter.execute(); - io.pipeline.complete(std::move(table_out)); - CompletedPipelineExecutor executor(io.pipeline); + + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(mutable_context); + auto build_pipeline_settings = BuildQueryPipelineSettings::fromContext(mutable_context); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline(optimization_settings, build_pipeline_settings))); + + pipeline.complete(std::move(table_out)); + CompletedPipelineExecutor executor(pipeline); executor.execute(); mutable_context->addExternalTable(temporary_table_name, std::move(external_storage_holder)); From e662fa01d0a1455aa2a625fd342ef8e7e998f34f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 9 Jun 2023 20:21:57 +0200 Subject: [PATCH 1454/2223] Added azureBlobStorage to aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d6cef1883f4..a01b67b26b1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1081,6 +1081,7 @@ avgweighted avro avx aws +azureBlobStorage backend backoff backticks From e094bf3247b82c94098567a4f8f172e57fcdc017 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Jun 2023 18:30:47 +0000 Subject: [PATCH 1455/2223] Resolving conflicts. --- src/Planner/CollectSets.cpp | 33 ++++++++++- .../QueryPlan/ReadFromMergeTree.cpp | 58 ++++++++++++++----- 2 files changed, 76 insertions(+), 15 deletions(-) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index be4cb0e2e2b..68ad1ab78d3 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -96,6 +97,36 @@ public: if (sets.getFuture(set_key)) return; + auto subquery_to_execute = in_second_argument; + + if (auto * table_node = in_second_argument->as()) + { + auto storage_snapshot = table_node->getStorageSnapshot(); + auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); + + size_t columns_to_select_size = columns_to_select.size(); + + auto column_nodes_to_select = std::make_shared(); + column_nodes_to_select->getNodes().reserve(columns_to_select_size); + + NamesAndTypes projection_columns; + projection_columns.reserve(columns_to_select_size); + + for (auto & column : columns_to_select) + { + column_nodes_to_select->getNodes().emplace_back(std::make_shared(column, subquery_to_execute)); + projection_columns.emplace_back(column.name, column.type); + } + + auto subquery_for_table = std::make_shared(Context::createCopy(planner_context.getQueryContext())); + subquery_for_table->setIsSubquery(true); + subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select); + subquery_for_table->getJoinTree() = std::move(subquery_to_execute); + subquery_for_table->resolveProjectionColumns(std::move(projection_columns)); + + subquery_to_execute = std::move(subquery_for_table); + } + // auto subquery_options = select_query_options.subquery(); // Planner subquery_planner( // in_second_argument, @@ -110,7 +141,7 @@ public: SubqueryForSet subquery_for_set; subquery_for_set.key = planner_context.createSetKey(in_second_argument); - subquery_for_set.query_tree = in_second_argument; + subquery_for_set.query_tree = std::move(subquery_to_execute); //subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); /// TODO diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 8483df797ef..8f610eb4380 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -37,6 +37,8 @@ #include #include #include +#include +#include #include #include @@ -100,6 +102,7 @@ namespace ErrorCodes extern const int INDEX_NOT_USED; extern const int LOGICAL_ERROR; extern const int TOO_MANY_ROWS; + extern const int CANNOT_PARSE_TEXT; } static MergeTreeReaderSettings getMergeTreeReaderSettings( @@ -1245,29 +1248,56 @@ static void buildIndexes( info = &*info_copy; } + std::unordered_set ignored_index_names; + + if (settings.ignore_data_skipping_indices.changed) + { + const auto & indices = settings.ignore_data_skipping_indices.toString(); + Tokens tokens(indices.data(), indices.data() + indices.size(), settings.max_query_size); + IParser::Pos pos(tokens, static_cast(settings.max_parser_depth)); + Expected expected; + + /// Use an unordered list rather than string vector + auto parse_single_id_or_literal = [&] + { + String str; + if (!parseIdentifierOrStringLiteral(pos, expected, str)) + return false; + + ignored_index_names.insert(std::move(str)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_single_id_or_literal, false)) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse ignore_data_skipping_indices ('{}')", indices); + } + UsefulSkipIndexes skip_indexes; using Key = std::pair; std::map merged; for (const auto & index : metadata_snapshot->getSecondaryIndices()) { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) + if (!ignored_index_names.contains(index.name)) { - auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); - if (inserted) + auto index_helper = MergeTreeIndexFactory::instance().get(index); + if (index_helper->isMergeable()) { - skip_indexes.merged_indices.emplace_back(); - skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot); - } + auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); + if (inserted) + { + skip_indexes.merged_indices.emplace_back(); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot); + } - skip_indexes.merged_indices[it->second].addIndex(index_helper); - } - else - { - auto condition = index_helper->createIndexCondition(*info, context); - if (!condition->alwaysUnknownOrTrue()) - skip_indexes.useful_indices.emplace_back(index_helper, condition); + skip_indexes.merged_indices[it->second].addIndex(index_helper); + } + else + { + auto condition = index_helper->createIndexCondition(*info, context); + if (!condition->alwaysUnknownOrTrue()) + skip_indexes.useful_indices.emplace_back(index_helper, condition); + } } } From f0d4ce4770a00a8c0cd9857a485fc8bbc124a95b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Jun 2023 22:05:21 +0300 Subject: [PATCH 1456/2223] Update 02785_summing_merge_tree_datetime64.sql --- .../queries/0_stateless/02785_summing_merge_tree_datetime64.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02785_summing_merge_tree_datetime64.sql b/tests/queries/0_stateless/02785_summing_merge_tree_datetime64.sql index 1ed930ebbc7..db00f189330 100644 --- a/tests/queries/0_stateless/02785_summing_merge_tree_datetime64.sql +++ b/tests/queries/0_stateless/02785_summing_merge_tree_datetime64.sql @@ -10,3 +10,4 @@ INSERT INTO summing_merge_tree_datetime64 SELECT 1 pk, '2023-05-01 23:55:55.100' INSERT INTO summing_merge_tree_datetime64 SELECT 1 pk, '2023-05-01 23:55:55.100' timestamp, 5 value; SELECT * FROM summing_merge_tree_datetime64 FINAL; +DROP TABLE summing_merge_tree_datetime64; From c538506f2e3ba0716dcc2f13f63bb4edc1f6f33e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 9 Jun 2023 20:50:17 +0000 Subject: [PATCH 1457/2223] More fixes --- docs/en/sql-reference/statements/create/table.md | 4 ++++ src/Compression/CompressionCodecDeflateQpl.h | 3 +-- src/Compression/CompressionFactoryAdditions.cpp | 2 +- src/Compression/ICompressionCodec.h | 4 ++-- src/Storages/Distributed/DistributedSink.cpp | 2 +- 5 files changed, 9 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 496ecdbda7b..1a72f89fb1f 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -385,6 +385,10 @@ High compression levels are useful for asymmetric scenarios, like compress once, - DEFLATE_QPL works best if the system has a Intel® IAA (In-Memory Analytics Accelerator) offloading device. Refer to [Accelerator Configuration](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#accelerator-configuration) and [Benchmark with DEFLATE_QPL](/docs/en/development/building_and_benchmarking_deflate_qpl.md/#Run-Benchmark-with-DEFLATE_QPL) for more details. - DEFLATE_QPL-compressed data can only be transferred between ClickHouse nodes compiled with SSE 4.2 enabled. +:::note +DEFLATE_QPL is not available in ClickHouse Cloud. +::: + ### Specialized Codecs These codecs are designed to make compression more effective by using specific features of data. Some of these codecs do not compress data themself. Instead, they prepare the data for a common purpose codec, which compresses it better than without this preparation. diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 13aa8733b54..8d73568707e 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -98,8 +98,7 @@ public: protected: bool isCompression() const override { return true; } bool isGenericCompression() const override { return true; } - bool isExperimental() const override { return false; } - bool isDeflateQplCompression() const override { return true; } + bool isDeflateQpl() const override { return true; } UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp index 46f7e2653c2..98e9e7480da 100644 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -159,7 +159,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( " You can enable it with the 'allow_experimental_codecs' setting.", codec_family_name); - if (!enable_deflate_qpl_codec && result_codec->isDeflateQplCompression()) + if (!enable_deflate_qpl_codec && result_codec->isDeflateQpl()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec {} is disabled by default." " You can enable it with the 'enable_deflate_qpl_codec' setting.", diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index f7e8f4e43d2..6630838fa64 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -109,8 +109,8 @@ public: /// It will not be allowed to use unless the user will turn off the safety switch. virtual bool isExperimental() const { return false; } - /// This is a knob for Deflate QPL codec. - virtual bool isDeflateQplCompression() const { return false; } + /// Is this the DEFLATE_QPL codec? + virtual bool isDeflateQpl() const { return false; } /// If it does nothing. virtual bool isNone() const { return false; } diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 1e1c911920e..875764f7633 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -733,7 +733,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const if (compression_method == "ZSTD") compression_level = settings.network_zstd_compression_level; - CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enale_deflate_qpl_codec); + CompressionCodecFactory::instance().validateCodec(compression_method, compression_level, !settings.allow_suspicious_codecs, settings.allow_experimental_codecs, settings.enable_deflate_qpl_codec); CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level); /// tmp directory is used to ensure atomicity of transactions From c71edb6c798163ad50a077a19a3bf74eb57ba212 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 9 Jun 2023 17:29:42 +0000 Subject: [PATCH 1458/2223] Fix style --- src/Processors/Sources/MongoDBSource.cpp | 10 +++++----- src/Processors/Sources/MongoDBSource.h | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Processors/Sources/MongoDBSource.cpp b/src/Processors/Sources/MongoDBSource.cpp index 74dfa13158c..cd4db416a29 100644 --- a/src/Processors/Sources/MongoDBSource.cpp +++ b/src/Processors/Sources/MongoDBSource.cpp @@ -371,8 +371,8 @@ bool isMongoDBWireProtocolOld(Poco::MongoDB::Connection & connection_) { Poco::MongoDB::Database db("config"); Poco::MongoDB::Document::Ptr doc = db.queryServerHello(connection_); - auto _wireVersion = doc->getInteger("maxWireVersion"); - return _wireVersion < Poco::MongoDB::Database::WireVersion::VER_36; + auto wire_version = doc->getInteger("maxWireVersion"); + return wire_version < Poco::MongoDB::Database::WireVersion::VER_36; } @@ -413,20 +413,20 @@ Poco::MongoDB::Document::Vector MongoDBCursor::nextDocuments(Poco::MongoDB::Conn if (is_wire_protocol_old) { auto response = old_cursor->next(connection); - cursorID_ = response.cursorID(); + cursor_id = response.cursorID(); return std::move(response.documents()); } else { auto response = new_cursor->next(connection); - cursorID_ = new_cursor->cursorID(); + cursor_id = new_cursor->cursorID(); return std::move(response.documents()); } } Int64 MongoDBCursor::cursorID() const { - return cursorID_; + return cursor_id; } diff --git a/src/Processors/Sources/MongoDBSource.h b/src/Processors/Sources/MongoDBSource.h index 2bc5481e20b..0e95d42c028 100644 --- a/src/Processors/Sources/MongoDBSource.h +++ b/src/Processors/Sources/MongoDBSource.h @@ -53,7 +53,7 @@ private: const bool is_wire_protocol_old; std::unique_ptr old_cursor; std::unique_ptr new_cursor; - Int64 cursorID_ = 0; + Int64 cursor_id = 0; }; /// Converts MongoDB Cursor to a stream of Blocks From 96d7b2efc9c0d4f40b919c5036fcfbe7445d10a1 Mon Sep 17 00:00:00 2001 From: tpanetti Date: Fri, 9 Jun 2023 13:50:30 -0700 Subject: [PATCH 1459/2223] Disable fasttest for MySQL Compatibility Type Conversion and refactor style for DataTypeNumberBase --- src/DataTypes/DataTypeNumberBase.cpp | 22 ------------------- .../02775_show_columns_mysql_compatibility.sh | 1 + 2 files changed, 1 insertion(+), 22 deletions(-) diff --git a/src/DataTypes/DataTypeNumberBase.cpp b/src/DataTypes/DataTypeNumberBase.cpp index e4c0fb96483..4cefc4945c6 100644 --- a/src/DataTypes/DataTypeNumberBase.cpp +++ b/src/DataTypes/DataTypeNumberBase.cpp @@ -15,50 +15,28 @@ template String DataTypeNumberBase::getSQLCompatibleName() const { if constexpr (std::is_same_v) - { return "TINYINT"; - } else if constexpr (std::is_same_v) - { return "SMALLINT"; - } else if constexpr (std::is_same_v) - { return "INTEGER"; - } else if constexpr (std::is_same_v) - { return "BIGINT"; - } else if constexpr (std::is_same_v) - { return "TINYINT UNSIGNED"; - } else if constexpr (std::is_same_v) - { return "SMALLINT UNSIGNED"; - } else if constexpr (std::is_same_v) - { return "INTEGER UNSIGNED"; - } else if constexpr (std::is_same_v) - { return "BIGINT UNSIGNED"; - } else if constexpr (std::is_same_v) - { return "FLOAT"; - } else if constexpr (std::is_same_v) - { return "DOUBLE"; - } /// Unsupported types are converted to TEXT else - { return "TEXT"; - } } template diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh index 938102cb5fc..6a546c47a38 100755 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 41c34aaf5ef747c98fb98fdb0c08c17bcb35bc78 Mon Sep 17 00:00:00 2001 From: Hongbin Ma Date: Mon, 22 May 2023 19:54:08 +0800 Subject: [PATCH 1460/2223] optimize parquet write performance for parallel threads fix CI fix review comments and CI --- .../Formats/Impl/ArrowBlockOutputFormat.cpp | 4 +- .../Formats/Impl/CHColumnToArrowColumn.cpp | 99 ++++++---- .../Formats/Impl/CHColumnToArrowColumn.h | 2 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 179 +++++++++--------- .../Formats/Impl/ParquetBlockOutputFormat.h | 9 +- 5 files changed, 154 insertions(+), 139 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index 1e72c949b09..8bd1cf3897d 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -58,7 +58,9 @@ void ArrowBlockOutputFormat::consume(Chunk chunk) format_settings.arrow.output_fixed_string_as_fixed_byte_array); } - ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num); + auto chunks = std::vector(); + chunks.push_back(std::move(chunk)); + ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunks, columns_num); if (!writer) prepareWriter(arrow_table->schema()); diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 19b2dcccf64..c3685e813d3 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -976,56 +976,75 @@ namespace DB void CHColumnToArrowColumn::chChunkToArrowTable( std::shared_ptr & res, - const Chunk & chunk, + const std::vector & chunks, size_t columns_num) { - /// For arrow::Schema and arrow::Table creation - std::vector> arrow_arrays; - arrow_arrays.reserve(columns_num); - for (size_t column_i = 0; column_i < columns_num; ++column_i) + std::shared_ptr arrow_schema; + std::vector table_data(columns_num); + + for (const auto & chunk : chunks) { - const ColumnWithTypeAndName & header_column = header_columns[column_i]; - auto column = chunk.getColumns()[column_i]; - - if (!low_cardinality_as_dictionary) - column = recursiveRemoveLowCardinality(column); - - if (!is_arrow_fields_initialized) + /// For arrow::Schema and arrow::Table creation + for (size_t column_i = 0; column_i < columns_num; ++column_i) { - bool is_column_nullable = false; - auto arrow_type = getArrowType(header_column.type, column, header_column.name, format_name, output_string_as_string, output_fixed_string_as_fixed_byte_array, &is_column_nullable); - arrow_fields.emplace_back(std::make_shared(header_column.name, arrow_type, is_column_nullable)); + const ColumnWithTypeAndName & header_column = header_columns[column_i]; + auto column = chunk.getColumns()[column_i]; + + if (!low_cardinality_as_dictionary) + column = recursiveRemoveLowCardinality(column); + + if (!is_arrow_fields_initialized) + { + bool is_column_nullable = false; + auto arrow_type = getArrowType( + header_column.type, + column, + header_column.name, + format_name, + output_string_as_string, + output_fixed_string_as_fixed_byte_array, + &is_column_nullable); + arrow_fields.emplace_back(std::make_shared(header_column.name, arrow_type, is_column_nullable)); + } + + arrow::MemoryPool * pool = arrow::default_memory_pool(); + std::unique_ptr array_builder; + arrow::Status status = MakeBuilder(pool, arrow_fields[column_i]->type(), &array_builder); + checkStatus(status, column->getName(), format_name); + + fillArrowArray( + header_column.name, + column, + header_column.type, + nullptr, + array_builder.get(), + format_name, + 0, + column->size(), + output_string_as_string, + output_fixed_string_as_fixed_byte_array, + dictionary_values); + + std::shared_ptr arrow_array; + status = array_builder->Finish(&arrow_array); + checkStatus(status, column->getName(), format_name); + + table_data.at(column_i).emplace_back(std::move(arrow_array)); } - arrow::MemoryPool* pool = arrow::default_memory_pool(); - std::unique_ptr array_builder; - arrow::Status status = MakeBuilder(pool, arrow_fields[column_i]->type(), &array_builder); - checkStatus(status, column->getName(), format_name); - - fillArrowArray( - header_column.name, - column, - header_column.type, - nullptr, - array_builder.get(), - format_name, - 0, - column->size(), - output_string_as_string, - output_fixed_string_as_fixed_byte_array, - dictionary_values); - - std::shared_ptr arrow_array; - status = array_builder->Finish(&arrow_array); - checkStatus(status, column->getName(), format_name); - arrow_arrays.emplace_back(std::move(arrow_array)); + is_arrow_fields_initialized = true; + if (!arrow_schema) + arrow_schema = std::make_shared(arrow_fields); } - std::shared_ptr arrow_schema = std::make_shared(arrow_fields); + std::vector> columns; + columns.reserve(columns_num); + for (size_t column_i = 0; column_i < columns_num; ++column_i) + columns.emplace_back(std::make_shared(table_data.at(column_i))); - res = arrow::Table::Make(arrow_schema, arrow_arrays); - is_arrow_fields_initialized = true; + res = arrow::Table::Make(arrow_schema, columns); } + } #endif diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h index 3649d0eed9b..02766e299a0 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h @@ -16,7 +16,7 @@ class CHColumnToArrowColumn public: CHColumnToArrowColumn(const Block & header, const std::string & format_name_, bool low_cardinality_as_dictionary_, bool output_string_as_string_, bool output_fixed_string_as_fixed_byte_array_); - void chChunkToArrowTable(std::shared_ptr & res, const Chunk & chunk, size_t columns_num); + void chChunkToArrowTable(std::shared_ptr & res, const std::vector & chunk, size_t columns_num); private: ColumnsWithTypeAndName header_columns; diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 031e499b545..91840cd2c50 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -20,47 +20,47 @@ namespace ErrorCodes namespace { -parquet::ParquetVersion::type getParquetVersion(const FormatSettings & settings) -{ - switch (settings.parquet.output_version) + parquet::ParquetVersion::type getParquetVersion(const FormatSettings & settings) { - case FormatSettings::ParquetVersion::V1_0: - return parquet::ParquetVersion::PARQUET_1_0; - case FormatSettings::ParquetVersion::V2_4: - return parquet::ParquetVersion::PARQUET_2_4; - case FormatSettings::ParquetVersion::V2_6: - return parquet::ParquetVersion::PARQUET_2_6; - case FormatSettings::ParquetVersion::V2_LATEST: - return parquet::ParquetVersion::PARQUET_2_LATEST; + switch (settings.parquet.output_version) + { + case FormatSettings::ParquetVersion::V1_0: + return parquet::ParquetVersion::PARQUET_1_0; + case FormatSettings::ParquetVersion::V2_4: + return parquet::ParquetVersion::PARQUET_2_4; + case FormatSettings::ParquetVersion::V2_6: + return parquet::ParquetVersion::PARQUET_2_6; + case FormatSettings::ParquetVersion::V2_LATEST: + return parquet::ParquetVersion::PARQUET_2_LATEST; + } } -} -parquet::Compression::type getParquetCompression(FormatSettings::ParquetCompression method) -{ - if (method == FormatSettings::ParquetCompression::NONE) - return parquet::Compression::type::UNCOMPRESSED; + parquet::Compression::type getParquetCompression(FormatSettings::ParquetCompression method) + { + if (method == FormatSettings::ParquetCompression::NONE) + return parquet::Compression::type::UNCOMPRESSED; #if USE_SNAPPY - if (method == FormatSettings::ParquetCompression::SNAPPY) - return parquet::Compression::type::SNAPPY; + if (method == FormatSettings::ParquetCompression::SNAPPY) + return parquet::Compression::type::SNAPPY; #endif #if USE_BROTLI - if (method == FormatSettings::ParquetCompression::BROTLI) - return parquet::Compression::type::BROTLI; + if (method == FormatSettings::ParquetCompression::BROTLI) + return parquet::Compression::type::BROTLI; #endif - if (method == FormatSettings::ParquetCompression::ZSTD) - return parquet::Compression::type::ZSTD; + if (method == FormatSettings::ParquetCompression::ZSTD) + return parquet::Compression::type::ZSTD; - if (method == FormatSettings::ParquetCompression::LZ4) - return parquet::Compression::type::LZ4; + if (method == FormatSettings::ParquetCompression::LZ4) + return parquet::Compression::type::LZ4; - if (method == FormatSettings::ParquetCompression::GZIP) - return parquet::Compression::type::GZIP; + if (method == FormatSettings::ParquetCompression::GZIP) + return parquet::Compression::type::GZIP; - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); -} + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); + } } @@ -69,70 +69,9 @@ ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Blo { } -void ParquetBlockOutputFormat::consume(Chunk chunk) +void ParquetBlockOutputFormat::consumeStaged() { - /// Do something like SquashingTransform to produce big enough row groups. - /// Because the real SquashingTransform is only used for INSERT, not for SELECT ... INTO OUTFILE. - /// The latter doesn't even have a pipeline where a transform could be inserted, so it's more - /// convenient to do the squashing here. - - appendToAccumulatedChunk(std::move(chunk)); - - if (!accumulated_chunk) - return; - - const size_t target_rows = std::max(static_cast(1), format_settings.parquet.row_group_rows); - - if (accumulated_chunk.getNumRows() < target_rows && - accumulated_chunk.bytes() < format_settings.parquet.row_group_bytes) - return; - - /// Increase row group size slightly (by < 2x) to avoid adding a small row groups for the - /// remainder of the new chunk. - /// E.g. suppose input chunks are 70K rows each, and max_rows = 1M. Then we'll have - /// getNumRows() = 1.05M. We want to write all 1.05M as one row group instead of 1M and 0.05M. - size_t num_row_groups = std::max(static_cast(1), accumulated_chunk.getNumRows() / target_rows); - size_t row_group_size = (accumulated_chunk.getNumRows() - 1) / num_row_groups + 1; // round up - - write(std::move(accumulated_chunk), row_group_size); - accumulated_chunk.clear(); -} - -void ParquetBlockOutputFormat::finalizeImpl() -{ - if (accumulated_chunk) - write(std::move(accumulated_chunk), format_settings.parquet.row_group_rows); - - if (!file_writer) - { - Block header = materializeBlock(getPort(PortKind::Main).getHeader()); - write(Chunk(header.getColumns(), 0), 1); - } - - auto status = file_writer->Close(); - if (!status.ok()) - throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while closing a table: {}", status.ToString()); -} - -void ParquetBlockOutputFormat::resetFormatterImpl() -{ - file_writer.reset(); -} - -void ParquetBlockOutputFormat::appendToAccumulatedChunk(Chunk chunk) -{ - if (!accumulated_chunk) - { - accumulated_chunk = std::move(chunk); - return; - } - chassert(accumulated_chunk.getNumColumns() == chunk.getNumColumns()); - accumulated_chunk.append(chunk); -} - -void ParquetBlockOutputFormat::write(Chunk chunk, size_t row_group_size) -{ - const size_t columns_num = chunk.getNumColumns(); + const size_t columns_num = staging_chunks.at(0).getNumColumns(); std::shared_ptr arrow_table; if (!ch_column_to_arrow_column) @@ -146,7 +85,7 @@ void ParquetBlockOutputFormat::write(Chunk chunk, size_t row_group_size) format_settings.parquet.output_fixed_string_as_fixed_byte_array); } - ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num); + ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, staging_chunks, columns_num); if (!file_writer) { @@ -173,12 +112,66 @@ void ParquetBlockOutputFormat::write(Chunk chunk, size_t row_group_size) file_writer = std::move(result.ValueOrDie()); } - auto status = file_writer->WriteTable(*arrow_table, row_group_size); + // TODO: calculate row_group_size depending on a number of rows and table size + + // allow slightly bigger than row_group_size to avoid a very small tail row group + auto status = file_writer->WriteTable(*arrow_table, std::max(format_settings.parquet.row_group_rows, staging_rows)); if (!status.ok()) throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while writing a table: {}", status.ToString()); } +void ParquetBlockOutputFormat::consume(Chunk chunk) +{ + /// Do something like SquashingTransform to produce big enough row groups. + /// Because the real SquashingTransform is only used for INSERT, not for SELECT ... INTO OUTFILE. + /// The latter doesn't even have a pipeline where a transform could be inserted, so it's more + /// convenient to do the squashing here. + staging_rows += chunk.getNumRows(); + staging_bytes += chunk.bytes(); + staging_chunks.push_back(std::move(chunk)); + chassert(staging_chunks.back().getNumColumns() == staging_chunks.front().getNumColumns()); + if (staging_rows < format_settings.parquet.row_group_rows && + staging_bytes < format_settings.parquet.row_group_bytes) + { + return; + } + else + { + consumeStaged(); + staging_chunks.clear(); + staging_rows = 0; + staging_bytes = 0; + } +} + +void ParquetBlockOutputFormat::finalizeImpl() +{ + if (!file_writer && staging_chunks.empty()) + { + Block header = materializeBlock(getPort(PortKind::Main).getHeader()); + + consume(Chunk(header.getColumns(), 0)); // this will make staging_chunks non-empty + } + + if (!staging_chunks.empty()) + { + consumeStaged(); + staging_chunks.clear(); + staging_rows = 0; + staging_bytes = 0; + } + + auto status = file_writer->Close(); + if (!status.ok()) + throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while closing a table: {}", status.ToString()); +} + +void ParquetBlockOutputFormat::resetFormatterImpl() +{ + file_writer.reset(); +} + void registerOutputFormatParquet(FormatFactory & factory) { factory.registerOutputFormat( diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index 0fb7013e6d6..482c778bc52 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -34,18 +34,19 @@ public: String getContentType() const override { return "application/octet-stream"; } private: + void consumeStaged(); void consume(Chunk) override; - void appendToAccumulatedChunk(Chunk chunk); - void write(Chunk chunk, size_t row_group_size); void finalizeImpl() override; void resetFormatterImpl() override; + std::vector staging_chunks; + size_t staging_rows = 0; + size_t staging_bytes = 0; + const FormatSettings format_settings; std::unique_ptr file_writer; std::unique_ptr ch_column_to_arrow_column; - - Chunk accumulated_chunk; }; } From 20b66689f3c912757540c4e91589b7ffd6fe3593 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 10 Jun 2023 13:34:51 +0200 Subject: [PATCH 1461/2223] Fix test --- docker/test/upgrade/run.sh | 6 ------ 1 file changed, 6 deletions(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index bd0c59a12cd..8353d03fc69 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -59,12 +59,6 @@ install_packages previous_release_package_folder # available for dump via clickhouse-local configure -# local_blob_storage disk type does not exist in older versions -sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ - | sed "s|local_blob_storage|local|" \ - > /etc/clickhouse-server/config.d/storage_conf.xml.tmp -sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml - # it contains some new settings, but we can safely remove it rm /etc/clickhouse-server/config.d/merge_tree.xml From ff96c4c0d8898c15e1aea876267c65ec8b0c69f0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 10 Jun 2023 12:09:47 +0000 Subject: [PATCH 1462/2223] Fix black --- tests/integration/test_non_default_compression/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_non_default_compression/test.py b/tests/integration/test_non_default_compression/test.py index e1a9c1ae540..18e2eb43813 100644 --- a/tests/integration/test_non_default_compression/test.py +++ b/tests/integration/test_non_default_compression/test.py @@ -50,6 +50,7 @@ node7 = cluster.add_instance( user_configs=["configs/allow_suspicious_codecs.xml"], ) + @pytest.fixture(scope="module") def start_cluster(): try: @@ -252,6 +253,7 @@ def test_uncompressed_cache_plus_zstd_codec(start_cluster): == "10000\n" ) + def test_preconfigured_deflateqpl_codec(start_cluster): node6.query( """ @@ -268,7 +270,7 @@ def test_preconfigured_deflateqpl_codec(start_cluster): ) assert ( node6.query( - "SELECT COUNT(*) FROM compression_codec_multiple_with_key WHERE id % 2 == 0" + "SELECT COUNT(*) FROM compression_codec_multiple_with_key WHERE id % 2 == 0" ) == "2\n" ) @@ -296,9 +298,7 @@ def test_preconfigured_deflateqpl_codec(start_cluster): == "1001\n" ) assert ( - node6.query( - "SELECT SUM(somecolumn) FROM compression_codec_multiple_with_key" - ) + node6.query("SELECT SUM(somecolumn) FROM compression_codec_multiple_with_key") == str(777.777 + 88.88 + 99.99 + 1.0 * 10000) + "\n" ) assert ( From 1f3e923528c25ddab273243fa80a6d0838c568c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Jun 2023 15:23:41 +0300 Subject: [PATCH 1463/2223] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index eb08a8ec100..7142ad26e15 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -31,7 +31,7 @@ * Introduces new keyword `INTO OUTFILE 'file.txt' APPEND`. [#48880](https://github.com/ClickHouse/ClickHouse/pull/48880) ([alekar](https://github.com/alekar)). * Added `system.zookeeper_connection` table that shows information about Keeper connections. [#45245](https://github.com/ClickHouse/ClickHouse/pull/45245) ([mateng915](https://github.com/mateng0915)). * Add new function `generateRandomStructure` that generates random table structure. It can be used in combination with table function `generateRandom`. [#47409](https://github.com/ClickHouse/ClickHouse/pull/47409) ([Kruglov Pavel](https://github.com/Avogar)). -* Allow the use of `CASE` without an `ELSE` branch and extended `transform` to deal with more types. Also fix some issues that made transform() return incorrect results when decimal types were mixed with other numeric types. [#48300](https://github.com/ClickHouse/ClickHouse/pull/48300) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Allow the use of `CASE` without an `ELSE` branch and extended `transform` to deal with more types. Also fix some issues that made transform() return incorrect results when decimal types were mixed with other numeric types. [#48300](https://github.com/ClickHouse/ClickHouse/pull/48300) ([Salvatore Mesoraca](https://github.com/aiven-sal)). This closes #2655. This closes #9596. This closes #38666. * Added [server-side encryption using KMS keys](https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingKMSEncryption.html) with S3 tables, and the `header` setting with S3 disks. Closes [#48723](https://github.com/ClickHouse/ClickHouse/issues/48723). [#48724](https://github.com/ClickHouse/ClickHouse/pull/48724) ([Johann Gan](https://github.com/johanngan)). * Add MemoryTracker for the background tasks (merges and mutation). Introduces `merges_mutations_memory_usage_soft_limit` and `merges_mutations_memory_usage_to_ram_ratio` settings that represent the soft memory limit for merges and mutations. If this limit is reached ClickHouse won't schedule new merge or mutation tasks. Also `MergesMutationsMemoryTracking` metric is introduced to allow observing current memory usage of background tasks. Resubmit [#46089](https://github.com/ClickHouse/ClickHouse/issues/46089). Closes [#48774](https://github.com/ClickHouse/ClickHouse/issues/48774). [#48787](https://github.com/ClickHouse/ClickHouse/pull/48787) ([Dmitry Novik](https://github.com/novikd)). * Function `dotProduct` work for array. [#49050](https://github.com/ClickHouse/ClickHouse/pull/49050) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). From 818e081162e77045468d8349ad5c438b261b02d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Jun 2023 14:30:36 +0200 Subject: [PATCH 1464/2223] Fill gaps on the dashboard --- programs/server/dashboard.html | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 97b35ec97c4..951b7db3aa3 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -449,7 +449,7 @@ let queries = [ FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "CPU Usage (cores)", @@ -457,7 +457,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Queries Running", @@ -465,7 +465,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Merges Running", @@ -473,7 +473,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Selected Bytes/second", @@ -481,7 +481,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "IO Wait", @@ -489,7 +489,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "CPU Wait", @@ -497,7 +497,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "OS CPU Usage (Userspace)", @@ -506,7 +506,7 @@ FROM system.asynchronous_metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSUserTimeNormalized' GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "OS CPU Usage (Kernel)", @@ -515,7 +515,7 @@ FROM system.asynchronous_metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSSystemTimeNormalized' GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Read From Disk", @@ -523,7 +523,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Read From Filesystem", @@ -531,7 +531,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Memory (tracked)", @@ -539,7 +539,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Load Average (15 minutes)", @@ -548,7 +548,7 @@ FROM system.asynchronous_metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'LoadAverage15' GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Selected Rows/second", @@ -556,7 +556,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Inserted Rows/second", @@ -564,7 +564,7 @@ ORDER BY t` FROM system.metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Total MergeTree Parts", @@ -573,7 +573,7 @@ FROM system.asynchronous_metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'TotalPartsOfMergeTreeTables' GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` }, { "title": "Max Parts For Partition", @@ -582,7 +582,7 @@ FROM system.asynchronous_metric_log WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'MaxPartCountForPartition' GROUP BY t -ORDER BY t` +ORDER BY t WITH FILL STEP {rounding:UInt32}` } ]; From dc4a2fb07d29a4fe5818a42aa6c17be9bd59ccb7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Jun 2023 14:47:24 +0200 Subject: [PATCH 1465/2223] Add tests for function "transform" --- .../02786_transform_float.reference | 10 +++++ .../0_stateless/02786_transform_float.sql | 3 ++ .../02787_transform_null.reference | 9 +++++ .../0_stateless/02787_transform_null.sql | 40 +++++++++++++++++++ 4 files changed, 62 insertions(+) create mode 100644 tests/queries/0_stateless/02786_transform_float.reference create mode 100644 tests/queries/0_stateless/02786_transform_float.sql create mode 100644 tests/queries/0_stateless/02787_transform_null.reference create mode 100644 tests/queries/0_stateless/02787_transform_null.sql diff --git a/tests/queries/0_stateless/02786_transform_float.reference b/tests/queries/0_stateless/02786_transform_float.reference new file mode 100644 index 00000000000..3fbb2492f2e --- /dev/null +++ b/tests/queries/0_stateless/02786_transform_float.reference @@ -0,0 +1,10 @@ +1 +1 +1 +--- +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02786_transform_float.sql b/tests/queries/0_stateless/02786_transform_float.sql new file mode 100644 index 00000000000..4229425b084 --- /dev/null +++ b/tests/queries/0_stateless/02786_transform_float.sql @@ -0,0 +1,3 @@ +select transform(number, [1], [toFloat32(1)], toFloat32(1)) from numbers(3); +SELECT '---'; +select transform(number, [3], [toFloat32(1)], toFloat32(1)) from numbers(6); diff --git a/tests/queries/0_stateless/02787_transform_null.reference b/tests/queries/0_stateless/02787_transform_null.reference new file mode 100644 index 00000000000..a650dbbd173 --- /dev/null +++ b/tests/queries/0_stateless/02787_transform_null.reference @@ -0,0 +1,9 @@ +ZERO +ZERO +ONE +ONE +a +a + \N 0 \N 0 \N +1 1 1 \N 1 1 +a \N 3 3 3 3 diff --git a/tests/queries/0_stateless/02787_transform_null.sql b/tests/queries/0_stateless/02787_transform_null.sql new file mode 100644 index 00000000000..64a771f0f4b --- /dev/null +++ b/tests/queries/0_stateless/02787_transform_null.sql @@ -0,0 +1,40 @@ +SELECT transform(0, [0, 1], ['ZERO', 'ONE'], 'DEFAULT') AS result; +SELECT transform(0, [0, 1], ['ZERO', 'ONE'], NULL) AS result; + +SELECT CASE 1 + WHEN 0 THEN 'ZERO' + WHEN 1 THEN 'ONE' + ELSE 'NONE' +END AS result; + +SELECT CASE 1 + WHEN 0 THEN NULL + WHEN 1 THEN 'ONE' + ELSE 'NONE' +END AS result; + +select + case 1 + when 1 then 'a' + else 'b' + end value; + +select + case 1 + when 1 then 'a' + end value; + +SELECT + d, + toInt16OrNull(d), + caseWithExpression(d, 'a', 3, toInt16OrZero(d)) AS case_zero, + caseWithExpression(d, 'a', 3, toInt16OrNull(d)) AS case_null, + if(d = 'a', 3, toInt16OrZero(d)) AS if_zero, + if(d = 'a', 3, toInt16OrNull(d)) AS if_null +FROM +( + SELECT arrayJoin(['', '1', 'a']) AS d +) +ORDER BY + case_zero ASC, + d ASC; From a614aa3b03b365c769f955f22b788181a8cf36ab Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 10 Jun 2023 16:06:37 +0200 Subject: [PATCH 1466/2223] More leftovers --- docker/test/upgrade/run.sh | 5 ----- 1 file changed, 5 deletions(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 8353d03fc69..6f7d3999f1d 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -86,11 +86,6 @@ export USE_S3_STORAGE_FOR_MERGE_TREE=1 export ZOOKEEPER_FAULT_INJECTION=0 configure -sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ - | sed "s|local_blob_storage|local|" \ - > /etc/clickhouse-server/config.d/storage_conf.xml.tmp -sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml - # it contains some new settings, but we can safely remove it rm /etc/clickhouse-server/config.d/merge_tree.xml From cb8c20722b8976fe0bc402498667b02c2585cc02 Mon Sep 17 00:00:00 2001 From: tpanetti Date: Sat, 10 Jun 2023 08:35:51 -0700 Subject: [PATCH 1467/2223] Rename setting and description for MySQL compatible types This renames the setting for MySQL compatible types from output_format_mysql_types to use_mysql_types_in_show_columns --- src/Core/Settings.h | 2 +- src/Storages/System/StorageSystemColumns.cpp | 4 ++-- .../0_stateless/02775_show_columns_mysql_compatibility.sh | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b72fc037fbb..d47015ebb39 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -196,7 +196,7 @@ class IColumn; M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ - M(Bool, output_format_mysql_types, false, "Use MySQL converted types when connected via MySQL compatibility", 0) \ + M(Bool, use_mysql_types_in_show_columns, false, "Use MySQL converted types when connected via MySQL compatibility for show columns query", 0) \ \ M(UInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \ \ diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 684c35709a4..e4ca6a15138 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -75,7 +75,7 @@ public: , columns_mask(std::move(columns_mask_)), max_block_size(max_block_size_) , databases(std::move(databases_)), tables(std::move(tables_)), storages(std::move(storages_)) , client_info_interface(context->getClientInfo().interface) - , use_mysql_types(context->getSettingsRef().output_format_mysql_types) + , use_mysql_types(context->getSettingsRef().use_mysql_types_in_show_columns) , total_tables(tables->size()), access(context->getAccess()) , query_id(context->getCurrentQueryId()), lock_acquire_timeout(context->getSettingsRef().lock_acquire_timeout) { @@ -133,7 +133,7 @@ protected: auto get_type_name = [this](const IDataType& type) -> std::string { - // Check if the output_format_mysql_types setting is enabled and client is connected via MySQL protocol + // Check if the use_mysql_types_in_show_columns setting is enabled and client is connected via MySQL protocol if (use_mysql_types && client_info_interface == DB::ClientInfo::Interface::MYSQL) { return type.getSQLCompatibleName(); diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh index 6a546c47a38..51c9da2a842 100755 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh @@ -127,7 +127,7 @@ TEMP_FILE=$(mktemp) cat < $TEMP_FILE SHOW COLUMNS FROM tab; -SET output_format_mysql_types=1; +SET use_mysql_types_in_show_columns=1; SHOW COLUMNS FROM tab; SHOW EXTENDED COLUMNS FROM tab; SHOW FULL COLUMNS FROM tab; From 7cb6f3c72279878db1ad65c5ea9670287cc42d16 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Jun 2023 17:43:42 +0200 Subject: [PATCH 1468/2223] Rename Cpu to CPU and Cfs to CFS --- src/Common/AsynchronousMetrics.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 6821647a180..531f0b04aa1 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -962,8 +962,8 @@ void AsynchronousMetrics::update(TimePoint update_time) period = std::stoull(field2); } - new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; - new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; + new_values["CGroupCPUCFSPeriod"] = { period, "The CFS period of CPU cgroup."}; + new_values["CGroupCPUCFSQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; } catch (...) { @@ -982,8 +982,8 @@ void AsynchronousMetrics::update(TimePoint update_time) tryReadText(quota, *cgroupcpu_cfs_quota); tryReadText(period, *cgroupcpu_cfs_period); - new_values["CGroupCpuCfsPeriod"] = { period, "The CFS period of CPU cgroup."}; - new_values["CGroupCpuCfsQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; + new_values["CGroupCPUCFSPeriod"] = { period, "The CFS period of CPU cgroup."}; + new_values["CGroupCPUCFSQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; } catch (...) { From e785c6796d8c1694e0f820e15772fa59bf31cbf0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Jun 2023 17:54:46 +0200 Subject: [PATCH 1469/2223] Replace CGroups CPU metrics to one --- src/Common/AsynchronousMetrics.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 531f0b04aa1..c610034a6b0 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -939,7 +939,8 @@ void AsynchronousMetrics::update(TimePoint update_time) if (cgroupcpu_max) { - try { + try + { cgroupcpu_max->rewind(); uint64_t quota = 0; @@ -962,8 +963,7 @@ void AsynchronousMetrics::update(TimePoint update_time) period = std::stoull(field2); } - new_values["CGroupCPUCFSPeriod"] = { period, "The CFS period of CPU cgroup."}; - new_values["CGroupCPUCFSQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; + new_values["CGroupMaxCPU"] = { static_cast(quota) / period, "The maximum number of CPU cores according to CGroups."}; } catch (...) { @@ -972,7 +972,8 @@ void AsynchronousMetrics::update(TimePoint update_time) } else if (cgroupcpu_cfs_quota && cgroupcpu_cfs_period) { - try { + try + { cgroupcpu_cfs_quota->rewind(); cgroupcpu_cfs_period->rewind(); @@ -982,8 +983,7 @@ void AsynchronousMetrics::update(TimePoint update_time) tryReadText(quota, *cgroupcpu_cfs_quota); tryReadText(period, *cgroupcpu_cfs_period); - new_values["CGroupCPUCFSPeriod"] = { period, "The CFS period of CPU cgroup."}; - new_values["CGroupCPUCFSQuota"] = { quota, "The CFS quota of CPU cgroup. If stated zero, the quota is max."}; + new_values["CGroupMaxCPU"] = { static_cast(quota) / period, "The maximum number of CPU cores according to CGroups."}; } catch (...) { From a91b84e60c1904bdd6fe7bdfd82869ad07e6bb94 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Jun 2023 18:00:47 +0200 Subject: [PATCH 1470/2223] Slightly better --- src/Common/AsynchronousMetrics.cpp | 116 +++++++++++++++-------------- 1 file changed, 62 insertions(+), 54 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index c610034a6b0..de9800aa896 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -727,6 +727,68 @@ void AsynchronousMetrics::update(TimePoint update_time) } } + Float64 max_cpu_cgroups = 0; + if (cgroupcpu_max) + { + try + { + cgroupcpu_max->rewind(); + + uint64_t quota = 0; + uint64_t period = 0; + + std::string line; + readText(line, *cgroupcpu_max); + + auto space = line.find(' '); + + if (line.rfind("max", space) == std::string::npos) + { + auto field1 = line.substr(0, space); + quota = std::stoull(field1); + } + + if (space != std::string::npos) + { + auto field2 = line.substr(space + 1); + period = std::stoull(field2); + } + + if (quota > 0 && period > 0) + max_cpu_cgroups = static_cast(quota) / period; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + else if (cgroupcpu_cfs_quota && cgroupcpu_cfs_period) + { + try + { + cgroupcpu_cfs_quota->rewind(); + cgroupcpu_cfs_period->rewind(); + + uint64_t quota = 0; + uint64_t period = 0; + + tryReadText(quota, *cgroupcpu_cfs_quota); + tryReadText(period, *cgroupcpu_cfs_period); + + if (quota > 0 && period > 0) + max_cpu_cgroups = static_cast(quota) / period; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + if (max_cpu_cgroups > 0) + { + new_values["CGroupMaxCPU"] = { max_cpu_cgroups, "The maximum number of CPU cores according to CGroups."}; + } + if (proc_stat) { try @@ -937,60 +999,6 @@ void AsynchronousMetrics::update(TimePoint update_time) } } - if (cgroupcpu_max) - { - try - { - cgroupcpu_max->rewind(); - - uint64_t quota = 0; - uint64_t period = 0; - - std::string line; - readText(line, *cgroupcpu_max); - - auto space = line.find(' '); - - if (line.rfind("max", space) == std::string::npos) - { - auto field1 = line.substr(0, space); - quota = std::stoull(field1); - } - - if (space != std::string::npos) - { - auto field2 = line.substr(space + 1); - period = std::stoull(field2); - } - - new_values["CGroupMaxCPU"] = { static_cast(quota) / period, "The maximum number of CPU cores according to CGroups."}; - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - else if (cgroupcpu_cfs_quota && cgroupcpu_cfs_period) - { - try - { - cgroupcpu_cfs_quota->rewind(); - cgroupcpu_cfs_period->rewind(); - - uint64_t quota = 0; - uint64_t period = 0; - - tryReadText(quota, *cgroupcpu_cfs_quota); - tryReadText(period, *cgroupcpu_cfs_period); - - new_values["CGroupMaxCPU"] = { static_cast(quota) / period, "The maximum number of CPU cores according to CGroups."}; - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - if (meminfo) { try From ddd2257cf51edf0cf0fb264b1d010e7436d7e94b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Jun 2023 18:03:17 +0200 Subject: [PATCH 1471/2223] Normalize with respect to CGroups --- src/Common/AsynchronousMetrics.cpp | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index de9800aa896..36c87010fa5 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -933,36 +933,38 @@ void AsynchronousMetrics::update(TimePoint update_time) /// Also write values normalized to 0..1 by diving to the number of CPUs. /// These values are good to be averaged across the cluster of non-uniform servers. - if (num_cpus) + Float64 num_cpus_to_normalize = max_cpu_cgroups > 0 ? max_cpu_cgroups : num_cpus; + + if (num_cpus_to_normalize > 0) { - new_values["OSUserTimeNormalized"] = { delta_values_all_cpus.user * multiplier / num_cpus, + new_values["OSUserTimeNormalized"] = { delta_values_all_cpus.user * multiplier / num_cpus_to_normalize, "The value is similar to `OSUserTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSNiceTimeNormalized"] = { delta_values_all_cpus.nice * multiplier / num_cpus, + new_values["OSNiceTimeNormalized"] = { delta_values_all_cpus.nice * multiplier / num_cpus_to_normalize, "The value is similar to `OSNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSSystemTimeNormalized"] = { delta_values_all_cpus.system * multiplier / num_cpus, + new_values["OSSystemTimeNormalized"] = { delta_values_all_cpus.system * multiplier / num_cpus_to_normalize, "The value is similar to `OSSystemTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSIdleTimeNormalized"] = { delta_values_all_cpus.idle * multiplier / num_cpus, + new_values["OSIdleTimeNormalized"] = { delta_values_all_cpus.idle * multiplier / num_cpus_to_normalize, "The value is similar to `OSIdleTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSIOWaitTimeNormalized"] = { delta_values_all_cpus.iowait * multiplier / num_cpus, + new_values["OSIOWaitTimeNormalized"] = { delta_values_all_cpus.iowait * multiplier / num_cpus_to_normalize, "The value is similar to `OSIOWaitTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSIrqTimeNormalized"] = { delta_values_all_cpus.irq * multiplier / num_cpus, + new_values["OSIrqTimeNormalized"] = { delta_values_all_cpus.irq * multiplier / num_cpus_to_normalize, "The value is similar to `OSIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSSoftIrqTimeNormalized"] = { delta_values_all_cpus.softirq * multiplier / num_cpus, + new_values["OSSoftIrqTimeNormalized"] = { delta_values_all_cpus.softirq * multiplier / num_cpus_to_normalize, "The value is similar to `OSSoftIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSStealTimeNormalized"] = { delta_values_all_cpus.steal * multiplier / num_cpus, + new_values["OSStealTimeNormalized"] = { delta_values_all_cpus.steal * multiplier / num_cpus_to_normalize, "The value is similar to `OSStealTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSGuestTimeNormalized"] = { delta_values_all_cpus.guest * multiplier / num_cpus, + new_values["OSGuestTimeNormalized"] = { delta_values_all_cpus.guest * multiplier / num_cpus_to_normalize, "The value is similar to `OSGuestTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; - new_values["OSGuestNiceTimeNormalized"] = { delta_values_all_cpus.guest_nice * multiplier / num_cpus, + new_values["OSGuestNiceTimeNormalized"] = { delta_values_all_cpus.guest_nice * multiplier / num_cpus_to_normalize, "The value is similar to `OSGuestNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; } From 72b9d75a84476d7ee402de8a160f8a22b9ccdb59 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 10 Jun 2023 12:53:02 +0000 Subject: [PATCH 1472/2223] Add compat setting for non-const timezones SQL function toTimezone() converts a Date or DateTime into another timezone. The problem is that the timezone is part of the Date / DateTime type but not part of the internal representation (value). This led to the fact that toTimeZone() wqith non-const timezones produced wrong and misleading results until #48471 (shipped with v23.4) enforced a const timezone. Unfortunately, this PR also broke existing table definitions with non-const timezones, e.g. in ALIAS expressions. So while #48471 addressed the issue appropriately, it is really backwards-incompatible. This PR adds a setting to toggle the behavior and makes it also part of the compatibility profile. --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + .../FunctionDateOrDateTimeAddInterval.h | 4 +- .../FunctionDateOrDateTimeToDateOrDate32.h | 2 +- ...tionDateOrDateTimeToDateTimeOrDateTime64.h | 2 +- .../FunctionDateOrDateTimeToSomething.h | 4 +- src/Functions/FunctionSnowflake.h | 17 ++++-- src/Functions/FunctionUnixTimestamp64.h | 13 +++-- src/Functions/FunctionsConversion.h | 10 ++-- src/Functions/FunctionsTimeWindow.cpp | 4 +- src/Functions/date_trunc.cpp | 2 +- .../extractTimeZoneFromFunctionArguments.cpp | 5 +- .../extractTimeZoneFromFunctionArguments.h | 10 +++- src/Functions/fromUnixTimestamp64Micro.cpp | 4 +- src/Functions/fromUnixTimestamp64Milli.cpp | 4 +- src/Functions/fromUnixTimestamp64Nano.cpp | 4 +- src/Functions/now.cpp | 12 +++-- src/Functions/now64.cpp | 10 +++- src/Functions/nowInBlock.cpp | 12 +++-- src/Functions/snowflake.cpp | 8 +-- src/Functions/timeSlots.cpp | 4 +- src/Functions/toStartOfInterval.cpp | 4 +- src/Functions/toTimezone.cpp | 11 +++- .../00515_enhanced_time_zones.reference | 7 +++ .../0_stateless/00515_enhanced_time_zones.sql | 54 ++++++++++++++----- 25 files changed, 147 insertions(+), 62 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 464b9168a4c..d2e6a470a91 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -147,6 +147,7 @@ class IColumn; M(Bool, enable_memory_bound_merging_of_aggregation_results, true, "Enable memory bound merging strategy for aggregation.", 0) \ M(Bool, enable_positional_arguments, true, "Enable positional arguments in ORDER BY, GROUP BY and LIMIT BY", 0) \ M(Bool, enable_extended_results_for_datetime_functions, false, "Enable date functions like toLastDayOfMonth return Date32 results (instead of Date results) for Date32/DateTime64 arguments.", 0) \ + M(Bool, allow_nonconst_timezone_arguments, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()", 0) \ \ M(Bool, group_by_use_nulls, false, "Treat columns mentioned in ROLLUP, CUBE or GROUPING SETS as Nullable", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index c0f10b13282..9fd45ac16d6 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,6 +85,7 @@ static std::map sett {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, + {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}}}, diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 507dc37e266..1546c24d30c 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -679,7 +679,7 @@ public: } else if constexpr (std::is_same_v) { - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); } else if constexpr (std::is_same_v) { @@ -696,7 +696,7 @@ public: return {}; }); - auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0); + auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false); if (const auto* datetime64_type = typeid_cast(arguments[0].type.get())) { const auto from_scale = datetime64_type->getScale(); diff --git a/src/Functions/FunctionDateOrDateTimeToDateOrDate32.h b/src/Functions/FunctionDateOrDateTimeToDateOrDate32.h index 8e006b93b98..6eb3e534b62 100644 --- a/src/Functions/FunctionDateOrDateTimeToDateOrDate32.h +++ b/src/Functions/FunctionDateOrDateTimeToDateOrDate32.h @@ -36,7 +36,7 @@ public: /// If the time zone is specified but empty, throw an exception. /// only validate the time_zone part if the number of arguments is 2. if ((which.isDateTime() || which.isDateTime64()) && arguments.size() == 2 - && extractTimeZoneNameFromFunctionArguments(arguments, 1, 0).empty()) + && extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, false).empty()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} supports a 2nd argument (optional) that must be a valid time zone", this->getName()); diff --git a/src/Functions/FunctionDateOrDateTimeToDateTimeOrDateTime64.h b/src/Functions/FunctionDateOrDateTimeToDateTimeOrDateTime64.h index 3d1f0f192cf..9f1066fd687 100644 --- a/src/Functions/FunctionDateOrDateTimeToDateTimeOrDateTime64.h +++ b/src/Functions/FunctionDateOrDateTimeToDateTimeOrDateTime64.h @@ -34,7 +34,7 @@ public: WhichDataType which(from_type); - std::string time_zone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + std::string time_zone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, false); /// If the time zone is specified but empty, throw an exception. /// only validate the time_zone part if the number of arguments is 2. diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index 47433d13e0b..82818cc3d2b 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -24,7 +24,7 @@ public: /// If the time zone is specified but empty, throw an exception. if constexpr (std::is_same_v) { - std::string time_zone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + std::string time_zone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, false); /// only validate the time_zone part if the number of arguments is 2. This is mainly /// to accommodate functions like toStartOfDay(today()), toStartOfDay(yesterday()) etc. if (arguments.size() == 2 && time_zone.empty()) @@ -53,7 +53,7 @@ public: scale = std::max(source_scale, static_cast(9)); } - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 1, 0)); + return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, false)); } else return std::make_shared(); diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index 998db98890a..ce3a48269b4 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -72,9 +73,13 @@ class FunctionSnowflakeToDateTime : public IFunction { private: const char * name; + const bool allow_nonconst_timezone_arguments; public: - explicit FunctionSnowflakeToDateTime(const char * name_) : name(name_) { } + explicit FunctionSnowflakeToDateTime(const char * name_, ContextPtr context) + : name(name_) + , allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + {} String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -92,7 +97,7 @@ public: std::string timezone; if (arguments.size() == 2) - timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, allow_nonconst_timezone_arguments); return std::make_shared(timezone); } @@ -162,9 +167,13 @@ class FunctionSnowflakeToDateTime64 : public IFunction { private: const char * name; + const bool allow_nonconst_timezone_arguments; public: - explicit FunctionSnowflakeToDateTime64(const char * name_) : name(name_) { } + explicit FunctionSnowflakeToDateTime64(const char * name_, ContextPtr context) + : name(name_) + , allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + {} String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -182,7 +191,7 @@ public: std::string timezone; if (arguments.size() == 2) - timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, allow_nonconst_timezone_arguments); return std::make_shared(3, timezone); } diff --git a/src/Functions/FunctionUnixTimestamp64.h b/src/Functions/FunctionUnixTimestamp64.h index 7519e46f9dc..58a23f7266e 100644 --- a/src/Functions/FunctionUnixTimestamp64.h +++ b/src/Functions/FunctionUnixTimestamp64.h @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -99,11 +100,13 @@ class FunctionFromUnixTimestamp64 : public IFunction private: size_t target_scale; const char * name; + const bool allow_nonconst_timezone_arguments; public: - FunctionFromUnixTimestamp64(size_t target_scale_, const char * name_) - : target_scale(target_scale_), name(name_) - { - } + FunctionFromUnixTimestamp64(size_t target_scale_, const char * name_, ContextPtr context) + : target_scale(target_scale_) + , name(name_) + , allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + {} String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -121,7 +124,7 @@ public: std::string timezone; if (arguments.size() == 2) - timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, allow_nonconst_timezone_arguments); return std::make_shared(target_scale, timezone); } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 940585d6d57..87229b8ad04 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1796,13 +1796,13 @@ public: if (to_datetime64 || scale != 0) /// toDateTime('xxxx-xx-xx xx:xx:xx', 0) return DateTime return std::make_shared(scale, - extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0, false)); - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0, false)); } if constexpr (std::is_same_v) - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0)); + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0, false)); else if constexpr (std::is_same_v) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected branch in code of conversion function: it is a bug."); else @@ -2067,7 +2067,7 @@ public: UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; if (arguments.size() > 1) scale = extractToDecimalScale(arguments[1]); - const auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0); + const auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false); res = scale == 0 ? res = std::make_shared(timezone) : std::make_shared(scale, timezone); } @@ -2117,7 +2117,7 @@ public: } if constexpr (std::is_same_v) - res = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0)); + res = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, false)); else if constexpr (std::is_same_v) throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: It is a bug."); else if constexpr (to_decimal) diff --git a/src/Functions/FunctionsTimeWindow.cpp b/src/Functions/FunctionsTimeWindow.cpp index 8a57a4da692..231e8b6fa77 100644 --- a/src/Functions/FunctionsTimeWindow.cpp +++ b/src/Functions/FunctionsTimeWindow.cpp @@ -138,7 +138,7 @@ struct TimeWindowImpl if (result_type_is_date) data_type = std::make_shared(); else - data_type = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); + data_type = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); return std::make_shared(DataTypes{data_type, data_type}); } @@ -322,7 +322,7 @@ struct TimeWindowImpl if (result_type_is_date) data_type = std::make_shared(); else - data_type = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0)); + data_type = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0, false)); return std::make_shared(DataTypes{data_type, data_type}); } diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index 016b8f4da5e..414512fc4f8 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -107,7 +107,7 @@ public: if (result_type_is_date) return std::make_shared(); else - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1)); + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false)); } bool useDefaultImplementationForConstants() const override { return true; } diff --git a/src/Functions/extractTimeZoneFromFunctionArguments.cpp b/src/Functions/extractTimeZoneFromFunctionArguments.cpp index 7ed240fdbcf..7168c68c9c9 100644 --- a/src/Functions/extractTimeZoneFromFunctionArguments.cpp +++ b/src/Functions/extractTimeZoneFromFunctionArguments.cpp @@ -30,10 +30,11 @@ std::string extractTimeZoneNameFromColumn(const IColumn * column, const String & } -std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num, size_t datetime_arg_num) +std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num, size_t datetime_arg_num, bool allow_nonconst_timezone_arguments) { /// Explicit time zone may be passed in last argument. - if (arguments.size() == time_zone_arg_num + 1) + if ((arguments.size() == time_zone_arg_num + 1) + && (!allow_nonconst_timezone_arguments || arguments[time_zone_arg_num].column)) { return extractTimeZoneNameFromColumn(arguments[time_zone_arg_num].column.get(), arguments[time_zone_arg_num].name); } diff --git a/src/Functions/extractTimeZoneFromFunctionArguments.h b/src/Functions/extractTimeZoneFromFunctionArguments.h index 858be40def7..3c012c64c49 100644 --- a/src/Functions/extractTimeZoneFromFunctionArguments.h +++ b/src/Functions/extractTimeZoneFromFunctionArguments.h @@ -16,8 +16,16 @@ std::string extractTimeZoneNameFromColumn(const IColumn * column, const String & /// Determine working timezone either from optional argument with time zone name or from time zone in DateTime type of argument. /// Returns empty string if default time zone should be used. +/// +/// Parameter allow_nonconst_timezone_arguments toggles if non-const timezone function arguments are accepted (legacy behavior) or not. The +/// problem with the old behavior is that the timezone is part of the type, and not part of the value. This lead to confusion and unexpected +/// results. +/// - For new functions, set allow_nonconst_timezone_arguments = false. +/// - For existing functions +/// - which disallow non-const timezone arguments anyways (e.g. getArgumentsThatAreAlwaysConstant()), set allow_nonconst_timezone_arguments = false, +/// - which allow non-const timezone arguments, set allow_nonconst_timezone_arguments according to the corresponding setting. std::string extractTimeZoneNameFromFunctionArguments( - const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num, size_t datetime_arg_num); + const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num, size_t datetime_arg_num, bool allow_nonconst_timezone_arguments); const DateLUTImpl & extractTimeZoneFromFunctionArguments( const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num, size_t datetime_arg_num); diff --git a/src/Functions/fromUnixTimestamp64Micro.cpp b/src/Functions/fromUnixTimestamp64Micro.cpp index 70dcbcd1d4b..191e2137a0d 100644 --- a/src/Functions/fromUnixTimestamp64Micro.cpp +++ b/src/Functions/fromUnixTimestamp64Micro.cpp @@ -7,8 +7,8 @@ namespace DB REGISTER_FUNCTION(FromUnixTimestamp64Micro) { factory.registerFunction("fromUnixTimestamp64Micro", - [](ContextPtr){ return std::make_unique( - std::make_shared(6, "fromUnixTimestamp64Micro")); }); + [](ContextPtr context){ return std::make_unique( + std::make_shared(6, "fromUnixTimestamp64Micro", context)); }); } } diff --git a/src/Functions/fromUnixTimestamp64Milli.cpp b/src/Functions/fromUnixTimestamp64Milli.cpp index 532013dfe5f..c6d4fcd30a2 100644 --- a/src/Functions/fromUnixTimestamp64Milli.cpp +++ b/src/Functions/fromUnixTimestamp64Milli.cpp @@ -7,8 +7,8 @@ namespace DB REGISTER_FUNCTION(FromUnixTimestamp64Milli) { factory.registerFunction("fromUnixTimestamp64Milli", - [](ContextPtr){ return std::make_unique( - std::make_shared(3, "fromUnixTimestamp64Milli")); }); + [](ContextPtr context){ return std::make_unique( + std::make_shared(3, "fromUnixTimestamp64Milli", context)); }); } } diff --git a/src/Functions/fromUnixTimestamp64Nano.cpp b/src/Functions/fromUnixTimestamp64Nano.cpp index 96afdda0fa8..2b5a7addbfc 100644 --- a/src/Functions/fromUnixTimestamp64Nano.cpp +++ b/src/Functions/fromUnixTimestamp64Nano.cpp @@ -7,8 +7,8 @@ namespace DB REGISTER_FUNCTION(FromUnixTimestamp64Nano) { factory.registerFunction("fromUnixTimestamp64Nano", - [](ContextPtr){ return std::make_unique( - std::make_shared(9, "fromUnixTimestamp64Nano")); }); + [](ContextPtr context){ return std::make_unique( + std::make_shared(9, "fromUnixTimestamp64Nano", context)); }); } } diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index 3c3bff1524f..d3a94379a61 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -87,7 +88,10 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } + explicit NowOverloadResolver(ContextPtr context) + : allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + {} DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -102,7 +106,7 @@ public: } if (arguments.size() == 1) { - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0)); + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0, allow_nonconst_timezone_arguments)); } return std::make_shared(); } @@ -121,10 +125,12 @@ public: if (arguments.size() == 1) return std::make_unique( time(nullptr), DataTypes{arguments.front().type}, - std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0))); + std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0, allow_nonconst_timezone_arguments))); return std::make_unique(time(nullptr), DataTypes(), std::make_shared()); } +private: + const bool allow_nonconst_timezone_arguments; }; } diff --git a/src/Functions/now64.cpp b/src/Functions/now64.cpp index f29b73061d9..349b8c71145 100644 --- a/src/Functions/now64.cpp +++ b/src/Functions/now64.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -115,7 +116,10 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } + explicit Now64OverloadResolver(ContextPtr context) + : allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + {} DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -137,7 +141,7 @@ public: } if (arguments.size() == 2) { - timezone_name = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + timezone_name = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, allow_nonconst_timezone_arguments); } return std::make_shared(scale, timezone_name); @@ -157,6 +161,8 @@ public: return std::make_unique(nowSubsecond(scale), std::move(arg_types), result_type); } +private: + const bool allow_nonconst_timezone_arguments; }; } diff --git a/src/Functions/nowInBlock.cpp b/src/Functions/nowInBlock.cpp index dfb3ed7c34a..0d5f9c45780 100644 --- a/src/Functions/nowInBlock.cpp +++ b/src/Functions/nowInBlock.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -25,10 +26,13 @@ class FunctionNowInBlock : public IFunction { public: static constexpr auto name = "nowInBlock"; - static FunctionPtr create(ContextPtr) + static FunctionPtr create(ContextPtr context) { - return std::make_shared(); + return std::make_shared(context); } + explicit FunctionNowInBlock(ContextPtr context) + : allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + {} String getName() const override { @@ -68,7 +72,7 @@ public: } if (arguments.size() == 1) { - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0)); + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0, allow_nonconst_timezone_arguments)); } return std::make_shared(); } @@ -77,6 +81,8 @@ public: { return ColumnDateTime::create(input_rows_count, static_cast(time(nullptr))); } +private: + const bool allow_nonconst_timezone_arguments; }; } diff --git a/src/Functions/snowflake.cpp b/src/Functions/snowflake.cpp index 4849d6512ca..ca78945acb9 100644 --- a/src/Functions/snowflake.cpp +++ b/src/Functions/snowflake.cpp @@ -21,14 +21,14 @@ REGISTER_FUNCTION(DateTime64ToSnowflake) REGISTER_FUNCTION(SnowflakeToDateTime) { factory.registerFunction("snowflakeToDateTime", - [](ContextPtr){ return std::make_unique( - std::make_shared("snowflakeToDateTime")); }); + [](ContextPtr context ){ return std::make_unique( + std::make_shared("snowflakeToDateTime", context)); }); } REGISTER_FUNCTION(SnowflakeToDateTime64) { factory.registerFunction("snowflakeToDateTime64", - [](ContextPtr){ return std::make_unique( - std::make_shared("snowflakeToDateTime64")); }); + [](ContextPtr context){ return std::make_unique( + std::make_shared("snowflakeToDateTime64", context)); }); } } diff --git a/src/Functions/timeSlots.cpp b/src/Functions/timeSlots.cpp index 568ab5e5a47..040495ab023 100644 --- a/src/Functions/timeSlots.cpp +++ b/src/Functions/timeSlots.cpp @@ -270,14 +270,14 @@ public: /// Note that there is no explicit time zone argument for this function (we specify 2 as an argument number with explicit time zone). if (WhichDataType(arguments[0].type).isDateTime()) { - return std::make_shared(std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0))); + return std::make_shared(std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0, false))); } else { auto start_time_scale = assert_cast(*arguments[0].type).getScale(); auto duration_scale = assert_cast(*arguments[1].type).getScale(); return std::make_shared( - std::make_shared(std::max(start_time_scale, duration_scale), extractTimeZoneNameFromFunctionArguments(arguments, 3, 0))); + std::make_shared(std::max(start_time_scale, duration_scale), extractTimeZoneNameFromFunctionArguments(arguments, 3, 0, false))); } } diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index c0220f1aed2..649242d0d86 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -384,7 +384,7 @@ public: if (result_type_is_date) return std::make_shared(); else if (result_type_is_datetime) - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); else { auto scale = 0; @@ -396,7 +396,7 @@ public: else if (interval_type->getKind() == IntervalKind::Millisecond) scale = 3; - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); + return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); } } diff --git a/src/Functions/toTimezone.cpp b/src/Functions/toTimezone.cpp index 0a54e5a86b7..a0d90351898 100644 --- a/src/Functions/toTimezone.cpp +++ b/src/Functions/toTimezone.cpp @@ -5,6 +5,8 @@ #include #include +#include + #include #include @@ -84,7 +86,10 @@ public: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } - static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } + explicit ToTimeZoneOverloadResolver(ContextPtr context) + : allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + {} DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -98,7 +103,7 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " "Should be DateTime or DateTime64", arguments[0].type->getName(), getName()); - String time_zone_name = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + String time_zone_name = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, allow_nonconst_timezone_arguments); if (which_type.isDateTime()) return std::make_shared(time_zone_name); @@ -119,6 +124,8 @@ public: return std::make_unique(is_constant_timezone, data_types, result_type); } +private: + const bool allow_nonconst_timezone_arguments; }; } diff --git a/tests/queries/0_stateless/00515_enhanced_time_zones.reference b/tests/queries/0_stateless/00515_enhanced_time_zones.reference index ad0dae35c45..2ee2c3eac81 100644 --- a/tests/queries/0_stateless/00515_enhanced_time_zones.reference +++ b/tests/queries/0_stateless/00515_enhanced_time_zones.reference @@ -16,3 +16,10 @@ 2017-11-05 08:07:47 2017-11-05 10:37:47 2017-11-05 10:37:47 +-- Test const/non-const timezone arguments -- +Asia/Kolkata 2017-11-05 08:07:47 +42 Asia/Kolkata 1970-01-01 00:00:00.042 +42 Asia/Kolkata 1970-01-01 00:00:00.000042 +42 Asia/Kolkata 1970-01-01 00:00:00.000000042 +42 Asia/Kolkata 2010-11-04 01:42:54 +42 Asia/Kolkata 2010-11-04 01:42:54.657 diff --git a/tests/queries/0_stateless/00515_enhanced_time_zones.sql b/tests/queries/0_stateless/00515_enhanced_time_zones.sql index f719ff70d7a..7659b6e4603 100644 --- a/tests/queries/0_stateless/00515_enhanced_time_zones.sql +++ b/tests/queries/0_stateless/00515_enhanced_time_zones.sql @@ -21,16 +21,46 @@ SELECT toString(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul')); SELECT toString(toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 'Asia/Kolkata')); SELECT toString(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 'Asia/Kolkata'); -SELECT toTimeZone(dt, tz) FROM ( - SELECT toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul') AS dt, arrayJoin(['Asia/Kolkata', 'UTC']) AS tz -); -- { serverError ILLEGAL_COLUMN } -SELECT materialize('Asia/Kolkata') t, toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), t); -- { serverError ILLEGAL_COLUMN } +SELECT '-- Test const/non-const timezone arguments --'; -CREATE TEMPORARY TABLE tmp AS SELECT arrayJoin(['Europe/Istanbul', 'Asia/Istanbul']); -SELECT toTimeZone(now(), (*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN } -SELECT now((*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN } -SELECT now64(1, (*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN } -SELECT toStartOfInterval(now(), INTERVAL 3 HOUR, (*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN } -SELECT snowflakeToDateTime(toInt64(123), (*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN } -SELECT toUnixTimestamp(now(), (*,).1) FROM tmp; -- { serverError ILLEGAL_COLUMN } -SELECT toDateTimeOrDefault('2023-04-12 16:43:32', (*,).1, now()) FROM tmp; -- { serverError ILLEGAL_COLUMN } +SELECT materialize('Asia/Kolkata') tz, toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT materialize('Asia/Kolkata') tz, toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), tz) SETTINGS allow_nonconst_timezone_arguments = 1; + +SELECT materialize('Asia/Kolkata') tz, now(tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +-- SELECT materialize('Asia/Kolkata') tz, now(tz) SETTINGS allow_nonconst_timezone_arguments = 1; + +SELECT materialize('Asia/Kolkata') tz, now64(9, tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +-- SELECT materialize('Asia/Kolkata') tz, now64(9, tz) SETTINGS allow_nonconst_timezone_arguments = 1; + +SELECT materialize('Asia/Kolkata') tz, nowInBlock(tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +-- SELECT materialize('Asia/Kolkata') tz, nowInBlock(tz) SETTINGS allow_nonconst_timezone_arguments = 1; + +SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Milli(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Milli(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 1; + +SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Micro(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Micro(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 1; + +SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Nano(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Nano(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 1; + +SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, snowflakeToDateTime(ts, tz) settings allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, snowflakeToDateTime(ts, tz) settings allow_nonconst_timezone_arguments = 1; + +SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, snowflakeToDateTime64(ts, tz) settings allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, snowflakeToDateTime64(ts, tz) settings allow_nonconst_timezone_arguments = 1; + +-- test for a related bug: + +DROP TABLE IF EXISTS tab; + +SET allow_nonconst_timezone_arguments = 1; + +CREATE TABLE tab (`country` LowCardinality(FixedString(7)) DEFAULT 'unknown', `city` LowCardinality(String) DEFAULT 'unknown', `region` LowCardinality(String) DEFAULT 'unknown', `continent` LowCardinality(FixedString(7)) DEFAULT 'unknown', `is_eu_country` Bool, `date` DateTime CODEC(DoubleDelta, LZ4), `viewer_date` DateTime ALIAS toTimezone(date, timezone), `device_browser` LowCardinality(String) DEFAULT 'unknown', `metro_code` LowCardinality(String) DEFAULT 'unknown', `domain` String DEFAULT 'unknown', `device_platform` LowCardinality(String) DEFAULT 'unknown', `device_type` LowCardinality(String) DEFAULT 'unknown', `device_vendor` LowCardinality(String) DEFAULT 'unknown', `ip` FixedString(39) DEFAULT 'unknown', `lat` Decimal(8, 6) CODEC(T64), `lng` Decimal(9, 6) CODEC(T64), `asset_id` String DEFAULT 'unknown', `is_personalized` Bool, `metric` String, `origin` String DEFAULT 'unknown', `product_id` UInt64 CODEC(T64), `referer` String DEFAULT 'unknown', `server_side` Int8 CODEC(T64), `third_party_id` String DEFAULT 'unknown', `partner_slug` LowCardinality(FixedString(10)) DEFAULT 'unknown', `user_agent` String DEFAULT 'unknown', `user_id` UUID, `zip` FixedString(10) DEFAULT 'unknown', `timezone` LowCardinality(String), `as_organization` LowCardinality(String) DEFAULT 'unknown', `content_cat` Array(String), `playback_method` LowCardinality(String) DEFAULT 'unknown', `store_id` LowCardinality(String) DEFAULT 'unknown', `store_url` String DEFAULT 'unknown', `timestamp` Nullable(DateTime), `ad_count` Int8 CODEC(T64), `ad_type` LowCardinality(FixedString(10)) DEFAULT 'unknown', `ad_categories` Array(FixedString(8)), `blocked_ad_categories` Array(FixedString(8)), `break_max_ad_length` Int8 CODEC(T64), `break_max_ads` Int8 CODEC(T64), `break_max_duration` Int8 CODEC(T64), `break_min_ad_length` Int8 CODEC(T64), `break_position` LowCardinality(FixedString(18)) DEFAULT 'unknown', `media_playhead` String DEFAULT 'unknown', `placement_type` Int8 CODEC(T64), `transaction_id` String, `universal_ad_id` Array(String), `client_ua` LowCardinality(String) DEFAULT 'unknown', `device_ip` FixedString(39) DEFAULT 'unknown', `device_ua` LowCardinality(String) DEFAULT 'unknown', `ifa` String, `ifa_type` LowCardinality(String) DEFAULT 'unknown', `vast_lat` Decimal(8, 6) CODEC(T64), `vast_long` Decimal(9, 6) CODEC(T64), `server_ua` String DEFAULT 'unknown', `app_bundle` String DEFAULT 'unknown', `page_url` String DEFAULT 'unknown', `api_framework` Array(UInt8), `click_type` LowCardinality(String), `extensions` Array(String), `media_mime` Array(String), `om_id_partner` LowCardinality(String) DEFAULT 'unknown', `player_capabilities` Array(FixedString(12)), `vast_versions` Array(UInt8), `verification_vendors` Array(String), `ad_play_head` String DEFAULT 'unknown', `ad_serving_id` String DEFAULT 'unknown', `asset_uri` String DEFAULT 'unknown', `content_id` String DEFAULT 'unknown', `content_uri` String DEFAULT 'unknown', `inventory_state` Array(FixedString(14)), `player_size` Array(UInt8), `player_state` Array(FixedString(12)), `pod_sequence` Int8 CODEC(T64), `click_position` Array(UInt32), `error_code` Int16 CODEC(T64), `error_reason` Int8 CODEC(T64), `gdpr_consent` String DEFAULT 'unknown', `limited_tracking` Bool, `regulations` String DEFAULT 'unknown', `content_category` Array(String), PROJECTION projection_TPAG_VAST_date (SELECT * ORDER BY toYYYYMMDD(date), metric, product_id, asset_id)) ENGINE = MergeTree ORDER BY (product_id, metric, asset_id, toYYYYMMDD(date)); + +DETACH TABLE tab; + +ATTACH TABLE tab SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +ATTACH TABLE tab SETTINGS allow_nonconst_timezone_arguments = 1; + +DROP TABLE tab; From ffb941624bc971886212e0745716e79688a154a1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 10 Jun 2023 17:01:44 +0000 Subject: [PATCH 1473/2223] Exclude some tests with QPL from fasttest --- .../0_stateless/00804_test_alter_compression_codecs.sql | 3 +++ .../0_stateless/00804_test_custom_compression_codecs.sql | 3 +++ .../00804_test_custom_compression_codes_log_storages.sql | 3 +++ .../0_stateless/00804_test_deflate_qpl_codec_compression.sql | 3 +++ 4 files changed, 12 insertions(+) diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index fd9855e82d3..eb1abda9a21 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -1,3 +1,6 @@ +--Tags: no-fasttest +-- no-fasttest because DEFLATE_QPL isn't available in fasttest + SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS alter_compression_codec; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 89e77f758a7..df74620a201 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -1,3 +1,6 @@ +--Tags: no-fasttest +-- no-fasttest because DEFLATE_QPL isn't available in fasttest + SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; SET enable_deflate_qpl_codec = 1; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql index a629df2666d..67c0074c58f 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql @@ -1,3 +1,6 @@ +--Tags: no-fasttest +-- no-fasttest because DEFLATE_QPL isn't available in fasttest + SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; SET enable_deflate_qpl_codec = 1; diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql index 5a56fc0d576..a46272112a9 100644 --- a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql @@ -1,3 +1,6 @@ +--Tags: no-fasttest +-- no-fasttest because DEFLATE_QPL isn't available in fasttest + SET send_logs_level = 'fatal'; SET enable_deflate_qpl_codec = 1; From a3da7c8ebe33977cacf6f67f5b1a75833de9aa64 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 10 Jun 2023 17:20:29 +0000 Subject: [PATCH 1474/2223] Merged NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 8f267da1a91..491eaf592d9 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 8f267da1a91310bd152af755b0178cfd38c646c7 +Subproject commit 491eaf592d950e0e37accbe8b3f217e068c9fecf From 296b11a1aeaa069b56fb2befd3b933c984b20a1b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 10 Jun 2023 20:56:54 +0000 Subject: [PATCH 1475/2223] Update version_date.tsv and changelogs after v23.5.2.7-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.5.2.7-stable.md | 18 ++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 5 files changed, 22 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.5.2.7-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 7190ef4d649..44967af4b32 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.5.1.3174" +ARG VERSION="23.5.2.7" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index ca966b16a2d..8ab9bf7b077 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.5.1.3174" +ARG VERSION="23.5.2.7" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index c82ac592120..b3b0cfe1510 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.5.1.3174" +ARG VERSION="23.5.2.7" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.5.2.7-stable.md b/docs/changelogs/v23.5.2.7-stable.md new file mode 100644 index 00000000000..2e4931c64e0 --- /dev/null +++ b/docs/changelogs/v23.5.2.7-stable.md @@ -0,0 +1,18 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.5.2.7-stable (5751aa1ab9f) FIXME as compared to v23.5.1.3174-stable (2fec796e73e) + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Do not read all the columns from right GLOBAL JOIN table. [#50721](https://github.com/ClickHouse/ClickHouse/pull/50721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix build for aarch64 (temporary disable azure) [#50770](https://github.com/ClickHouse/ClickHouse/pull/50770) ([alesapin](https://github.com/alesapin)). +* Rename azure_blob_storage to azureBlobStorage [#50812](https://github.com/ClickHouse/ClickHouse/pull/50812) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 2aeeb5db35c..4647bcb4af1 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.5.2.7-stable 2023-06-10 v23.5.1.3174-stable 2023-06-09 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 From 78c32a204ce656c278433fed92fb535584b8ee3b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 10 Jun 2023 23:00:59 +0200 Subject: [PATCH 1476/2223] Updated docs for azureBlobStorage Table function & engine --- .../integrations/azureBlobStorage.md | 28 ++++++++ .../table-functions/azureBlobStorage.md | 70 +++++++++++++++++++ .../table-functions/azure_blob_storage.md | 11 --- 3 files changed, 98 insertions(+), 11 deletions(-) create mode 100644 docs/en/engines/table-engines/integrations/azureBlobStorage.md create mode 100644 docs/en/sql-reference/table-functions/azureBlobStorage.md delete mode 100644 docs/en/sql-reference/table-functions/azure_blob_storage.md diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md new file mode 100644 index 00000000000..b1c6169592b --- /dev/null +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -0,0 +1,28 @@ +--- +slug: /en/engines/table-engines/integrations/azureBlobStorage +sidebar_position: 7 +sidebar_label: AzureBlobStorage +--- + +# AzureBlobStorage Table Engine + +This engine provides integration with [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs) ecosystem. + +## Create Table {#creating-a-table} + +``` sql +CREATE TABLE azure_blob_storage_table (name String, value UInt32) + ENGINE = AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) + [PARTITION BY expr] + [SETTINGS ...] +``` + +**Engine parameters** + +- `connection_string|storage_account_url` — connection_string includes account name & key ([Create connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string?toc=%2Fazure%2Fstorage%2Fblobs%2Ftoc.json&bc=%2Fazure%2Fstorage%2Fblobs%2Fbreadcrumb%2Ftoc.json#configure-a-connection-string-for-an-azure-storage-account)) or you could also provide the storage account url here and account name & account key as separate parameters (see parameters account_name & account_key) +- `container_name` - Container name +- `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. +- `account_name` - if storage_account_url is used, then account name can be specified here +- `account_key` - if storage_account_url is used, then account key can be specified here +- `format` — The [format](../../interfaces/formats.md#formats) of the file. +- `compression` — Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. By default, it will autodetect compression by file extension. (same as setting to `auto`). diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md new file mode 100644 index 00000000000..f8a9016bd15 --- /dev/null +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -0,0 +1,70 @@ +--- +slug: /en/sql-reference/table-functions/azure_blob_storage +sidebar_position: 45 +sidebar_label: azure_blob_storage +keywords: [azure blob storage] +--- + +# azure\_blob\_storage Table Function + +Provides a table-like interface to select/insert files in [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs). This table function is similar to the [s3 function](../../sql-reference/table-functions/s3.md). + +**Syntax** + +``` sql +azureBlobStorage(- connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) +``` + +**Arguments** + +- `connection_string|storage_account_url` — connection_string includes account name & key ([Create connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string?toc=%2Fazure%2Fstorage%2Fblobs%2Ftoc.json&bc=%2Fazure%2Fstorage%2Fblobs%2Fbreadcrumb%2Ftoc.json#configure-a-connection-string-for-an-azure-storage-account)) or you could also provide the storage account url here and account name & account key as separate parameters (see parameters account_name & account_key) +- `container_name` - Container name +- `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. +- `account_name` - if storage_account_url is used, then account name can be specified here +- `account_key` - if storage_account_url is used, then account key can be specified here +- `format` — The [format](../../interfaces/formats.md#formats) of the file. +- `compression` — Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. By default, it will autodetect compression by file extension. (same as setting to `auto`). +- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. + +**Returned value** + +A table with the specified structure for reading or writing data in the specified file. + +**Examples** + +Write data into azure blob storage using the following : + +```sql +INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', + 'test_container', 'test_{_partition_id}.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', + 'CSV', 'auto', 'column1 UInt32, column2 UInt32, column3 UInt32') PARTITION BY column3 VALUES (1, 2, 3), (3, 2, 1), (78, 43, 3); +``` + +And then it can be read using + +```sql +SELECT * FROM azureBlobStorage('http://azurite1:10000/devstoreaccount1', + 'test_container', 'test_1.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', + 'CSV', 'auto', 'column1 UInt32, column2 UInt32, column3 UInt32'); +``` + +```response +┌───column1─┬────column2─┬───column3─┐ +│ 3 │ 2 │ 1 │ +└───────────┴────────────┴───────────┘ +``` + +or with storage_account_url + +```sql +SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;EndPointSuffix=core.windows.net', + 'test_container', 'test_3.csv', 'CSV', 'auto' , 'column1 UInt32, column2 UInt32, column3 UInt32'); +``` + +``` text +┌─count()─┐ +│ 2 │ +└─────────┘ +``` + + \ No newline at end of file diff --git a/docs/en/sql-reference/table-functions/azure_blob_storage.md b/docs/en/sql-reference/table-functions/azure_blob_storage.md deleted file mode 100644 index 6091aab5f9d..00000000000 --- a/docs/en/sql-reference/table-functions/azure_blob_storage.md +++ /dev/null @@ -1,11 +0,0 @@ ---- -slug: /en/sql-reference/table-functions/azure_blob_storage -sidebar_position: 45 -sidebar_label: azure_blob_storage -keywords: [azure blob storage] ---- - -# azureBlobStorage Table Function - -Provides a table-like interface to select/insert files in [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs). This table function is similar to the [s3 function](../../sql-reference/table-functions/s3.md). - From 05d4baf1e7186c902c9e44fe0f16e9cbbc18e5c0 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Sat, 10 Jun 2023 18:20:39 -0400 Subject: [PATCH 1477/2223] edits --- .../integrations/azureBlobStorage.md | 15 +++++++------ .../table-engines/integrations/index.md | 21 +------------------ .../table-functions/azureBlobStorage.md | 5 +++-- 3 files changed, 13 insertions(+), 28 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index b1c6169592b..82b26e79579 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -1,14 +1,13 @@ --- slug: /en/engines/table-engines/integrations/azureBlobStorage -sidebar_position: 7 -sidebar_label: AzureBlobStorage +sidebar_label: Azure Blob Storage --- # AzureBlobStorage Table Engine -This engine provides integration with [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs) ecosystem. +This engine provides an integration with [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs) ecosystem. -## Create Table {#creating-a-table} +## Create Table ``` sql CREATE TABLE azure_blob_storage_table (name String, value UInt32) @@ -17,12 +16,16 @@ CREATE TABLE azure_blob_storage_table (name String, value UInt32) [SETTINGS ...] ``` -**Engine parameters** +### Engine parameters - `connection_string|storage_account_url` — connection_string includes account name & key ([Create connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string?toc=%2Fazure%2Fstorage%2Fblobs%2Ftoc.json&bc=%2Fazure%2Fstorage%2Fblobs%2Fbreadcrumb%2Ftoc.json#configure-a-connection-string-for-an-azure-storage-account)) or you could also provide the storage account url here and account name & account key as separate parameters (see parameters account_name & account_key) - `container_name` - Container name - `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. - `account_name` - if storage_account_url is used, then account name can be specified here - `account_key` - if storage_account_url is used, then account key can be specified here -- `format` — The [format](../../interfaces/formats.md#formats) of the file. +- `format` — The [format](/docs/en/interfaces/formats.md) of the file. - `compression` — Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. By default, it will autodetect compression by file extension. (same as setting to `auto`). + +## See also + +[Azure Blob Storage Table Function](/docs/en/sql-reference/table-functions/azureBlobStorage.md) diff --git a/docs/en/engines/table-engines/integrations/index.md b/docs/en/engines/table-engines/integrations/index.md index b321a644d32..b9171d9033b 100644 --- a/docs/en/engines/table-engines/integrations/index.md +++ b/docs/en/engines/table-engines/integrations/index.md @@ -6,24 +6,5 @@ sidebar_label: Integrations # Table Engines for Integrations -ClickHouse provides various means for integrating with external systems, including table engines. Like with all other table engines, the configuration is done using `CREATE TABLE` or `ALTER TABLE` queries. Then from a user perspective, the configured integration looks like a normal table, but queries to it are proxied to the external system. This transparent querying is one of the key advantages of this approach over alternative integration methods, like dictionaries or table functions, which require to use custom query methods on each use. +ClickHouse provides various means for integrating with external systems, including table engines. Like with all other table engines, the configuration is done using `CREATE TABLE` or `ALTER TABLE` queries. Then from a user perspective, the configured integration looks like a normal table, but queries to it are proxied to the external system. This transparent querying is one of the key advantages of this approach over alternative integration methods, like dictionaries or table functions, which require the use of custom query methods on each use. -List of supported integrations: - -- [ODBC](../../../engines/table-engines/integrations/odbc.md) -- [JDBC](../../../engines/table-engines/integrations/jdbc.md) -- [MySQL](../../../engines/table-engines/integrations/mysql.md) -- [MongoDB](../../../engines/table-engines/integrations/mongodb.md) -- [HDFS](../../../engines/table-engines/integrations/hdfs.md) -- [S3](../../../engines/table-engines/integrations/s3.md) -- [Kafka](../../../engines/table-engines/integrations/kafka.md) -- [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md) -- [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) -- [PostgreSQL](../../../engines/table-engines/integrations/postgresql.md) -- [SQLite](../../../engines/table-engines/integrations/sqlite.md) -- [Hive](../../../engines/table-engines/integrations/hive.md) -- [ExternalDistributed](../../../engines/table-engines/integrations/ExternalDistributed.md) -- [MaterializedPostgreSQL](../../../engines/table-engines/integrations/materialized-postgresql.md) -- [NATS](../../../engines/table-engines/integrations/nats.md) -- [DeltaLake](../../../engines/table-engines/integrations/deltalake.md) -- [Hudi](../../../engines/table-engines/integrations/hudi.md) diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index f8a9016bd15..b79137cb786 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -1,6 +1,5 @@ --- slug: /en/sql-reference/table-functions/azure_blob_storage -sidebar_position: 45 sidebar_label: azure_blob_storage keywords: [azure blob storage] --- @@ -67,4 +66,6 @@ SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountNam └─────────┘ ``` - \ No newline at end of file +**See Also** + +- [AzureBlogStorage Table Engine](/docs/en/engines/table-engines/integrations/azureBlobStorage.md) From 2146c356081fbe1b43da41ad4c739262f1db60c1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 11 Jun 2023 07:40:39 +0000 Subject: [PATCH 1478/2223] Fix style --- src/Functions/snowflake.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/snowflake.cpp b/src/Functions/snowflake.cpp index ca78945acb9..c18f1c03332 100644 --- a/src/Functions/snowflake.cpp +++ b/src/Functions/snowflake.cpp @@ -21,7 +21,7 @@ REGISTER_FUNCTION(DateTime64ToSnowflake) REGISTER_FUNCTION(SnowflakeToDateTime) { factory.registerFunction("snowflakeToDateTime", - [](ContextPtr context ){ return std::make_unique( + [](ContextPtr context){ return std::make_unique( std::make_shared("snowflakeToDateTime", context)); }); } REGISTER_FUNCTION(SnowflakeToDateTime64) From e6c2a6d13db4604442f84e8e2cacf6bc617fc42e Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 11 Jun 2023 09:55:20 +0200 Subject: [PATCH 1479/2223] Added example for table engine and fixed typos --- .../integrations/azureBlobStorage.md | 20 +++++++++++++++++++ .../table-functions/azureBlobStorage.md | 4 ++-- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index 82b26e79579..b8e621fd513 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -26,6 +26,26 @@ CREATE TABLE azure_blob_storage_table (name String, value UInt32) - `format` — The [format](/docs/en/interfaces/formats.md) of the file. - `compression` — Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. By default, it will autodetect compression by file extension. (same as setting to `auto`). +**Example** + +``` sql +CREATE TABLE test_table (key UInt64, data String) + ENGINE = AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', + 'test_container', 'test_table', 'CSV'); + +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); + +SELECT * FROM test_table; +``` + +```text +┌─key──┬─data──┐ +│ 1 │ a │ +│ 2 │ b │ +│ 3 │ c │ +└──────┴───────┘ +``` + ## See also [Azure Blob Storage Table Function](/docs/en/sql-reference/table-functions/azureBlobStorage.md) diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index b79137cb786..369bf7a964d 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -53,7 +53,7 @@ SELECT * FROM azureBlobStorage('http://azurite1:10000/devstoreaccount1', └───────────┴────────────┴───────────┘ ``` -or with storage_account_url +or using connection_string ```sql SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;EndPointSuffix=core.windows.net', @@ -68,4 +68,4 @@ SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountNam **See Also** -- [AzureBlogStorage Table Engine](/docs/en/engines/table-engines/integrations/azureBlobStorage.md) +- [AzureBlobStorage Table Engine](/docs/en/engines/table-engines/integrations/azureBlobStorage.md) From e9d539f4bd72d94cef27ed7f1a8a34cd6fa08322 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 11 Jun 2023 10:05:52 +0200 Subject: [PATCH 1480/2223] Updated changelog with azureBlobStorage table function & engine entry --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7142ad26e15..72372c8fac4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,6 +21,7 @@ * Setting `enable_memory_bound_merging_of_aggregation_results` is enabled by default. If you update from version prior to 22.12, we recommend to set this flag to `false` until update is finished. [#50319](https://github.com/ClickHouse/ClickHouse/pull/50319) ([Nikita Taranov](https://github.com/nickitat)). #### New Feature +* Added storage engine AzureBlobStorage and azureBlobStorage table function. The supported set of features is very similar to storage/table function S3 [#50604] (https://github.com/ClickHouse/ClickHouse/pull/50604) ([alesapin](https://github.com/alesapin)) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni). * Added native ClickHouse Keeper CLI Client, it is available as `clickhouse keeper-client` [#47414](https://github.com/ClickHouse/ClickHouse/pull/47414) ([pufit](https://github.com/pufit)). * Add `urlCluster` table function. Refactor all *Cluster table functions to reduce code duplication. Make schema inference work for all possible *Cluster function signatures and for named collections. Closes [#38499](https://github.com/ClickHouse/ClickHouse/issues/38499). [#45427](https://github.com/ClickHouse/ClickHouse/pull/45427) ([attack204](https://github.com/attack204)), Pavel Kruglov. * The query cache can now be used for production workloads. [#47977](https://github.com/ClickHouse/ClickHouse/pull/47977) ([Robert Schulze](https://github.com/rschu1ze)). The query cache can now support queries with totals and extremes modifier. [#48853](https://github.com/ClickHouse/ClickHouse/pull/48853) ([Robert Schulze](https://github.com/rschu1ze)). Make `allow_experimental_query_cache` setting as obsolete for backward-compatibility. It was removed in https://github.com/ClickHouse/ClickHouse/pull/47977. [#49934](https://github.com/ClickHouse/ClickHouse/pull/49934) ([Timur Solodovnikov](https://github.com/tsolodov)). From b4bc28b6de86c569bcdfe8d2de0e21ce1717a8c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Jun 2023 16:48:29 +0300 Subject: [PATCH 1481/2223] Update easy_tasks_sorted_ru.md --- tests/instructions/easy_tasks_sorted_ru.md | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/tests/instructions/easy_tasks_sorted_ru.md b/tests/instructions/easy_tasks_sorted_ru.md index a98a5766ffe..09ea48d0bd9 100644 --- a/tests/instructions/easy_tasks_sorted_ru.md +++ b/tests/instructions/easy_tasks_sorted_ru.md @@ -6,7 +6,7 @@ Делаем `chmod 000 /etc/clickhouse-client/config.xml` и смотрим, что получится. -## Уменьшать max_memory_usage, если на сервере мало оперативки. +## + Уменьшать max_memory_usage, если на сервере мало оперативки. Смотрим, сколько на сервере оперативки. Если `max_memory_usage`, `max_memory_usage_for_all_queries` ограничены, но больше 90% (настройка) от имеющейся оперативки, то уменьшать их и выводить предупреждение в лог.. @@ -42,10 +42,12 @@ void memoryBitAnd(const char * a, const char * b, char * result, size_t size); В ClickHouse есть возможность указать collation для сортировки строк. Это не работает для `Nullable(String)`. -## Запретить чтение значений типа AggregateFunction по-умолчанию и добавить настройку. +## + Запретить чтение значений типа AggregateFunction по-умолчанию и добавить настройку. Состояния агрегатных функций могут быть записаны в дамп и считаны из него. Но десериализация состояний агрегатных функций небезопасна. Аккуратно выбранные пользовательские данные могут привести к segfault или порче памяти. Поэтому нужно просто сделать настройку, которая запрещает читать AggregateFunction из пользовательских данных. +Upd: сделали по-другому: теперь всё безопасно. + ## + В статистику jemalloc добавить информацию по arenas. В `system.asynchronous_metrics` - суммарный размер арен. @@ -56,9 +58,9 @@ void memoryBitAnd(const char * a, const char * b, char * result, size_t size); Как cache, но без кэша — всегда прямой запрос в источник. -## Функции randomFixedString, randomBinaryString, fuzzBits, fuzzBytes. +## + Функции randomFixedString, randomBinaryString, fuzzBits, fuzzBytes. -## Агрегатные функции для статистических тестов (e.g. тест нормальности распределения) и статистик. +## + Агрегатные функции для статистических тестов (e.g. тест нормальности распределения) и статистик. ## + Функции создания и обновления состояния агрегатной функции по одному кортежу аргументов. @@ -119,11 +121,11 @@ position с конца строки. Добавляем счётчики всех ошибок (ErrorCodes) по аналогии с ProfileEvents. Кроме количества запоминаем также время последней ошибки, стек трейс, сообщение. Добавляем системную таблицу system.errors. Отправка в Graphite. -## Добавить Lizard, LZSSE и density в качестве вариантов алгоритмов сжатия. +## + Добавить Lizard, LZSSE и density в качестве вариантов алгоритмов сжатия. Экспериментальные алгоритмы сжатия. Сейчас ClickHouse поддерживает только lz4 и zstd. -## Запрос CREATE OR REPLACE TABLE +## + Запрос CREATE OR REPLACE TABLE Атомарно удаляет таблицу перед созданием новой, если такая была. @@ -149,12 +151,16 @@ https://clickhouse.com/docs/en/query_language/create/#create-table Запретить модификацию данных в партиции. На партицию ставится флаг, что она заблокирована. В неё нельзя делать INSERT и ALTER. С файлов снимается доступ на запись. +Upd: не нужно. + ## Настройка join_use_nulls: поддержка для LEFT ARRAY JOIN. -## Внешние словари из Aerospike/Couchbase/Cassandra (на выбор). +## + Внешние словари из Aerospike/Couchbase/Cassandra (на выбор). Подключить одну из key-value БД как источник. +Upd: сделали Redis, Cassandra, MongoDB. + ## + Движок таблиц Mongo, табличная функция mongo. Возможность легко импортировать данные из MongoDB. @@ -181,7 +187,7 @@ https://clickhouse.com/docs/en/operations/table_engines/external_data/ Не работает, если открыть clickhouse-client в интерактивном режиме и делать несколько запросов. -## Настройка для возможности получить частичный результат при cancel-е. +## + Настройка для возможности получить частичный результат при cancel-е. Хотим по Ctrl+C получить те данные, которые успели обработаться. From 6bdbcd3f436f7f55e0ab71e24d0c96df072d0003 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 11 Jun 2023 14:26:13 +0000 Subject: [PATCH 1482/2223] Stabilize tests --- .../00515_enhanced_time_zones.reference | 26 ++++++++--- .../0_stateless/00515_enhanced_time_zones.sql | 45 ++++++++++--------- 2 files changed, 44 insertions(+), 27 deletions(-) diff --git a/tests/queries/0_stateless/00515_enhanced_time_zones.reference b/tests/queries/0_stateless/00515_enhanced_time_zones.reference index 2ee2c3eac81..2555c885558 100644 --- a/tests/queries/0_stateless/00515_enhanced_time_zones.reference +++ b/tests/queries/0_stateless/00515_enhanced_time_zones.reference @@ -16,10 +16,22 @@ 2017-11-05 08:07:47 2017-11-05 10:37:47 2017-11-05 10:37:47 --- Test const/non-const timezone arguments -- -Asia/Kolkata 2017-11-05 08:07:47 -42 Asia/Kolkata 1970-01-01 00:00:00.042 -42 Asia/Kolkata 1970-01-01 00:00:00.000042 -42 Asia/Kolkata 1970-01-01 00:00:00.000000042 -42 Asia/Kolkata 2010-11-04 01:42:54 -42 Asia/Kolkata 2010-11-04 01:42:54.657 +-- Test const timezone arguments -- +42 +43 +42 +43 +42 +43 +42 +43 +42 +43 +42 +43 +42 +43 +42 +43 +42 +43 diff --git a/tests/queries/0_stateless/00515_enhanced_time_zones.sql b/tests/queries/0_stateless/00515_enhanced_time_zones.sql index 7659b6e4603..5f40cfb53c1 100644 --- a/tests/queries/0_stateless/00515_enhanced_time_zones.sql +++ b/tests/queries/0_stateless/00515_enhanced_time_zones.sql @@ -21,38 +21,43 @@ SELECT toString(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul')); SELECT toString(toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 'Asia/Kolkata')); SELECT toString(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 'Asia/Kolkata'); -SELECT '-- Test const/non-const timezone arguments --'; +SELECT '-- Test const timezone arguments --'; -SELECT materialize('Asia/Kolkata') tz, toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } -SELECT materialize('Asia/Kolkata') tz, toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), tz) SETTINGS allow_nonconst_timezone_arguments = 1; +DROP TABLE IF EXISTS tab; -SELECT materialize('Asia/Kolkata') tz, now(tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } --- SELECT materialize('Asia/Kolkata') tz, now(tz) SETTINGS allow_nonconst_timezone_arguments = 1; +CREATE TABLE tab (val Int64, tz String) engine=Log; +INSERT INTO tab VALUES (42, 'Asia/Singapore') (43, 'Asia/Tokyo'); -SELECT materialize('Asia/Kolkata') tz, now64(9, tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } --- SELECT materialize('Asia/Kolkata') tz, now64(9, tz) SETTINGS allow_nonconst_timezone_arguments = 1; +SELECT val FROM tab WHERE now(tz) != toDateTime('2000-01-01 00:00:00') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT val FROM tab WHERE now(tz) != toDateTime('2000-01-01 00:00:00') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1; -SELECT materialize('Asia/Kolkata') tz, nowInBlock(tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } --- SELECT materialize('Asia/Kolkata') tz, nowInBlock(tz) SETTINGS allow_nonconst_timezone_arguments = 1; +SELECT val FROM tab WHERE now64(9, tz) != toDateTime64('2000-01-01 00:00:00', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT val FROM tab WHERE now64(9, tz) != toDateTime64('2000-01-01 00:00:00', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1; -SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Milli(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } -SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Milli(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 1; +SELECT val FROM tab WHERE nowInBlock(tz) != toDateTime('2000-01-01 00:00:00') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT val FROM tab WHERE nowInBlock(tz) != toDateTime('2000-01-01 00:00:00') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1; -SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Micro(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } -SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Micro(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 1; +SELECT val FROM tab WHERE toTimeZone(toDateTime(val), tz) != toDateTime('2023-06-11 14:14:14') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT val FROM tab WHERE toTimeZone(toDateTime(val), tz) != toDateTime('2023-06-11 14:14:14') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1; -SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Nano(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } -SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, fromUnixTimestamp64Nano(ts, tz) SETTINGS allow_nonconst_timezone_arguments = 1; +SELECT val FROM tab WHERE fromUnixTimestamp64Milli(val, tz) != toDateTime64('2023-06-11 14:14:14', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT val FROM tab WHERE fromUnixTimestamp64Milli(val, tz) != toDateTime64('2023-06-11 14:14:14', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1; -SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, snowflakeToDateTime(ts, tz) settings allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } -SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, snowflakeToDateTime(ts, tz) settings allow_nonconst_timezone_arguments = 1; +SELECT val FROM tab WHERE fromUnixTimestamp64Micro(val, tz) != toDateTime64('2023-06-11 14:14:14', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT val FROM tab WHERE fromUnixTimestamp64Micro(val, tz) != toDateTime64('2023-06-11 14:14:14', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1; -SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, snowflakeToDateTime64(ts, tz) settings allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } -SELECT materialize(42::Int64) ts, materialize('Asia/Kolkata') tz, snowflakeToDateTime64(ts, tz) settings allow_nonconst_timezone_arguments = 1; +SELECT val FROM tab WHERE fromUnixTimestamp64Nano(val, tz) != toDateTime64('2023-06-11 14:14:14', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT val FROM tab WHERE fromUnixTimestamp64Nano(val, tz) != toDateTime64('2023-06-11 14:14:14', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1; + +SELECT val FROM tab WHERE snowflakeToDateTime(val, tz) != toDateTime('2023-06-11 14:14:14') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT val FROM tab WHERE snowflakeToDateTime(val, tz) != toDateTime('2023-06-11 14:14:14') ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1; + +SELECT val FROM tab WHERE snowflakeToDateTime64(val, tz) != toDateTime64('2023-06-11 14:14:14', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 0; -- { serverError ILLEGAL_COLUMN } +SELECT val FROM tab WHERE snowflakeToDateTime64(val, tz) != toDateTime64('2023-06-11 14:14:14', 6) ORDER BY val SETTINGS allow_nonconst_timezone_arguments = 1; -- test for a related bug: -DROP TABLE IF EXISTS tab; +DROP TABLE tab; SET allow_nonconst_timezone_arguments = 1; From 48e03ac92a457d612dd8b2e4838dce1e47e51109 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 11 Jun 2023 14:33:21 +0000 Subject: [PATCH 1483/2223] Revert "Exclude some tests with QPL from fasttest" This reverts commit ffb941624bc971886212e0745716e79688a154a1. --- .../0_stateless/00804_test_alter_compression_codecs.sql | 3 --- .../0_stateless/00804_test_custom_compression_codecs.sql | 3 --- .../00804_test_custom_compression_codes_log_storages.sql | 3 --- .../0_stateless/00804_test_deflate_qpl_codec_compression.sql | 3 --- 4 files changed, 12 deletions(-) diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index eb1abda9a21..fd9855e82d3 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -1,6 +1,3 @@ ---Tags: no-fasttest --- no-fasttest because DEFLATE_QPL isn't available in fasttest - SET send_logs_level = 'fatal'; DROP TABLE IF EXISTS alter_compression_codec; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index df74620a201..89e77f758a7 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -1,6 +1,3 @@ ---Tags: no-fasttest --- no-fasttest because DEFLATE_QPL isn't available in fasttest - SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; SET enable_deflate_qpl_codec = 1; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql index 67c0074c58f..a629df2666d 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql @@ -1,6 +1,3 @@ ---Tags: no-fasttest --- no-fasttest because DEFLATE_QPL isn't available in fasttest - SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; SET enable_deflate_qpl_codec = 1; diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql index a46272112a9..5a56fc0d576 100644 --- a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql @@ -1,6 +1,3 @@ ---Tags: no-fasttest --- no-fasttest because DEFLATE_QPL isn't available in fasttest - SET send_logs_level = 'fatal'; SET enable_deflate_qpl_codec = 1; From d228411f41eabf7e443fbbb2f4148880a3da78fa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 11 Jun 2023 14:39:15 +0000 Subject: [PATCH 1484/2223] Reset modified tests --- ...04_test_alter_compression_codecs.reference | 17 ++----- .../00804_test_alter_compression_codecs.sql | 22 +++------- ...4_test_custom_compression_codecs.reference | 8 ++-- .../00804_test_custom_compression_codecs.sql | 44 ++++++++----------- ...m_compression_codes_log_storages.reference | 20 ++++----- ..._custom_compression_codes_log_storages.sql | 41 ++++++++--------- ...804_test_deflate_qpl_codec_compression.sql | 4 ++ 7 files changed, 63 insertions(+), 93 deletions(-) diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference b/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference index 5c77a102740..cfbfadf1e67 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.reference @@ -18,7 +18,7 @@ CODEC(NONE) 2018-01-01 4 4 2018-01-01 5 5 2018-01-01 6 6 -CODEC(DEFLATE_QPL) +CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, NONE) 2018-01-01 1 default_value 2018-01-01 2 default_value 2018-01-01 3 3 @@ -27,18 +27,7 @@ CODEC(DEFLATE_QPL) 2018-01-01 6 6 2018-01-01 7 7 2018-01-01 8 8 -CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, DEFLATE_QPL, NONE) -2018-01-01 1 default_value -2018-01-01 2 default_value -2018-01-01 3 3 -2018-01-01 4 4 -2018-01-01 5 5 -2018-01-01 6 6 -2018-01-01 7 7 -2018-01-01 8 8 -2018-01-01 9 9 -2018-01-01 10 10 -CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, DEFLATE_QPL, NONE) -CODEC(NONE, LZ4, LZ4HC(0), ZSTD(1), DEFLATE_QPL) +CODEC(ZSTD(1), LZ4HC(0), LZ4, LZ4, NONE) +CODEC(NONE, LZ4, LZ4HC(0), ZSTD(1)) 2 1 diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index fd9855e82d3..85e5f8b63ad 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -28,20 +28,12 @@ SELECT * FROM alter_compression_codec ORDER BY id; OPTIMIZE TABLE alter_compression_codec FINAL; SELECT * FROM alter_compression_codec ORDER BY id; -SET enable_deflate_qpl_codec = 1; -ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(DEFLATE_QPL); +SET allow_suspicious_codecs = 1; +ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE); SELECT compression_codec FROM system.columns WHERE database = currentDatabase() AND table = 'alter_compression_codec' AND name = 'alter_column'; INSERT INTO alter_compression_codec VALUES('2018-01-01', 7, '7'); INSERT INTO alter_compression_codec VALUES('2018-01-01', 8, '8'); -SELECT * FROM alter_compression_codec ORDER BY id; - -SET allow_suspicious_codecs = 1; -ALTER TABLE alter_compression_codec MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, DEFLATE_QPL, NONE); -SELECT compression_codec FROM system.columns WHERE database = currentDatabase() AND table = 'alter_compression_codec' AND name = 'alter_column'; - -INSERT INTO alter_compression_codec VALUES('2018-01-01', 9, '9'); -INSERT INTO alter_compression_codec VALUES('2018-01-01', 10, '10'); OPTIMIZE TABLE alter_compression_codec FINAL; SELECT * FROM alter_compression_codec ORDER BY id; @@ -62,17 +54,15 @@ ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(ZSTD(100)); -- { serverError 433 } -ALTER TABLE alter_bad_codec ADD COLUMN alter_column DateTime DEFAULT '2019-01-01 00:00:00' CODEC(DEFLATE_QPL(100)); -- { serverError DATA_TYPE_CANNOT_HAVE_ARGUMENTS } - DROP TABLE IF EXISTS alter_bad_codec; DROP TABLE IF EXISTS large_alter_table_00804; DROP TABLE IF EXISTS store_of_hash_00804; CREATE TABLE large_alter_table_00804 ( - somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), DEFLATE_QPL), - id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, DEFLATE_QPL), - data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4, DEFLATE_QPL) + somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)), + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), + data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4) ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi', min_bytes_for_wide_part = 0; INSERT INTO large_alter_table_00804 SELECT toDate('2019-01-01'), number, toString(number + rand()) FROM system.numbers LIMIT 300000; @@ -81,7 +71,7 @@ CREATE TABLE store_of_hash_00804 (hash UInt64) ENGINE = Memory(); INSERT INTO store_of_hash_00804 SELECT sum(cityHash64(*)) FROM large_alter_table_00804; -ALTER TABLE large_alter_table_00804 MODIFY COLUMN data CODEC(NONE, LZ4, LZ4HC, ZSTD, DEFLATE_QPL); +ALTER TABLE large_alter_table_00804 MODIFY COLUMN data CODEC(NONE, LZ4, LZ4HC, ZSTD); OPTIMIZE TABLE large_alter_table_00804; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference b/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference index 8b51d65004a..7bd91e5a69b 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference @@ -1,6 +1,6 @@ -1 hello 2018-12-14 2018-12-14 1.1 aaa 5 -2 world 2018-12-15 2018-12-15 2.2 bbb 6 -3 ! 2018-12-16 2018-12-16 3.3 ccc 7 +1 hello 2018-12-14 1.1 aaa 5 +2 world 2018-12-15 2.2 bbb 6 +3 ! 2018-12-16 3.3 ccc 7 2 1 world 2018-10-05 1.1 2 hello 2018-10-01 2.2 @@ -9,7 +9,7 @@ 10003 274972506.6 9175437371954010821 -CREATE TABLE default.compression_codec_multiple_more_types\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0), DEFLATE_QPL),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), NONE, NONE, NONE, LZ4HC(0), DEFLATE_QPL),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8), DEFLATE_QPL),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8), DEFLATE_QPL)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.compression_codec_multiple_more_types\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1.5555555555555 hello world! [77] ['John'] 7.1 xxxxxxxxxxxx [127] ['Henry'] ! diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 89e77f758a7..c080c2fc98e 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -1,6 +1,5 @@ SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; -SET enable_deflate_qpl_codec = 1; DROP TABLE IF EXISTS compression_codec; @@ -8,21 +7,20 @@ CREATE TABLE compression_codec( id UInt64 CODEC(LZ4), data String CODEC(ZSTD), ddd Date CODEC(NONE), - ddd32 Date32 CODEC(DEFLATE_QPL), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), - othernum Int64 CODEC(Delta), + othernum Int64 CODEC(Delta) ) ENGINE = MergeTree() ORDER BY tuple(); -INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), toDate32('2018-12-14'), 1.1, 'aaa', 5); -INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), toDate32('2018-12-15'), 2.2, 'bbb', 6); -INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), toDate32('2018-12-16'), 3.3, 'ccc', 7); +INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); +INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6); +INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7); SELECT * FROM compression_codec ORDER BY id; OPTIMIZE TABLE compression_codec FINAL; -INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), toDate32('2018-12-13'), 4.4, 'ddd', 8); +INSERT INTO compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8); DETACH TABLE compression_codec; ATTACH TABLE compression_codec; @@ -33,31 +31,25 @@ DROP TABLE IF EXISTS compression_codec; DROP TABLE IF EXISTS bad_codec; DROP TABLE IF EXISTS params_when_no_params; -DROP TABLE IF EXISTS params_when_no_params2; DROP TABLE IF EXISTS too_many_params; DROP TABLE IF EXISTS codec_multiple_direct_specification_1; DROP TABLE IF EXISTS codec_multiple_direct_specification_2; -DROP TABLE IF EXISTS codec_multiple_direct_specification_3; DROP TABLE IF EXISTS delta_bad_params1; DROP TABLE IF EXISTS delta_bad_params2; CREATE TABLE bad_codec(id UInt64 CODEC(adssadads)) ENGINE = MergeTree() order by tuple(); -- { serverError 432 } CREATE TABLE too_many_params(id UInt64 CODEC(ZSTD(2,3,4,5))) ENGINE = MergeTree() order by tuple(); -- { serverError 431 } CREATE TABLE params_when_no_params(id UInt64 CODEC(LZ4(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 } -CREATE TABLE params_when_no_params2(id UInt64 CODEC(DEFLATE_QPL(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 } CREATE TABLE codec_multiple_direct_specification_1(id UInt64 CODEC(MULTIPLE(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 } CREATE TABLE codec_multiple_direct_specification_2(id UInt64 CODEC(multiple(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 } -CREATE TABLE codec_multiple_direct_specification_3(id UInt64 CODEC(multiple(LZ4, DEFLATE_QPL))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 } CREATE TABLE delta_bad_params1(id UInt64 CODEC(Delta(3))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 433 } CREATE TABLE delta_bad_params2(id UInt64 CODEC(Delta(16))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 433 } DROP TABLE IF EXISTS bad_codec; DROP TABLE IF EXISTS params_when_no_params; -DROP TABLE IF EXISTS params_when_no_params2; DROP TABLE IF EXISTS too_many_params; DROP TABLE IF EXISTS codec_multiple_direct_specification_1; DROP TABLE IF EXISTS codec_multiple_direct_specification_2; -DROP TABLE IF EXISTS codec_multiple_direct_specification_3; DROP TABLE IF EXISTS delta_bad_params1; DROP TABLE IF EXISTS delta_bad_params2; @@ -66,10 +58,10 @@ DROP TABLE IF EXISTS compression_codec_multiple; SET network_compression_method = 'lz4hc'; CREATE TABLE compression_codec_multiple ( - id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4), DEFLATE_QPL), - data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8), DEFLATE_QPL), - ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC, DEFLATE_QPL), - somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD, DEFLATE_QPL) + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), + data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), + ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), + somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) ) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO compression_codec_multiple VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); @@ -93,15 +85,15 @@ SELECT sum(cityHash64(*)) FROM compression_codec_multiple; DROP TABLE IF EXISTS compression_codec_multiple_more_types; CREATE TABLE compression_codec_multiple_more_types ( - id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC, DEFLATE_QPL), - data FixedString(12) CODEC(ZSTD, ZSTD, Delta, Delta, Delta, NONE, NONE, NONE, LZ4HC, DEFLATE_QPL), - ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8), DEFLATE_QPL) + id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC), + data FixedString(12) CODEC(ZSTD, ZSTD, Delta, Delta, Delta, NONE, NONE, NONE, LZ4HC), + ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8)) ) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36 } CREATE TABLE compression_codec_multiple_more_types ( - id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC, DEFLATE_QPL), - data FixedString(12) CODEC(ZSTD, ZSTD, NONE, NONE, NONE, LZ4HC, DEFLATE_QPL), - ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8), DEFLATE_QPL) + id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC), + data FixedString(12) CODEC(ZSTD, ZSTD, NONE, NONE, NONE, LZ4HC), + ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8)) ) ENGINE = MergeTree() ORDER BY tuple(); SHOW CREATE TABLE compression_codec_multiple_more_types; @@ -117,9 +109,9 @@ SET network_compression_method = 'zstd'; SET network_zstd_compression_level = 5; CREATE TABLE compression_codec_multiple_with_key ( - somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta, DEFLATE_QPL), - id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta, DEFLATE_QPL), - data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4, DEFLATE_QPL) + somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta), + id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta), + data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4) ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference index d64b8a77eed..8145ca99829 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference +++ b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference @@ -1,9 +1,9 @@ -CREATE TABLE default.compression_codec_log\n(\n `id` UInt64 CODEC(LZ4),\n `data` String CODEC(ZSTD(1)),\n `ddd` Date CODEC(NONE),\n `somenum` Float64 CODEC(ZSTD(2)),\n `somestr` FixedString(3) CODEC(LZ4HC(7)),\n `othernum` Int64 CODEC(Delta(8)),\n `qplstr` String CODEC(DEFLATE_QPL),\n `qplnum` UInt32 CODEC(DEFLATE_QPL)\n)\nENGINE = Log -1 hello 2018-12-14 1.1 aaa 5 qpl11 11 -2 world 2018-12-15 2.2 bbb 6 qpl22 22 -3 ! 2018-12-16 3.3 ccc 7 qpl33 33 +CREATE TABLE default.compression_codec_log\n(\n `id` UInt64 CODEC(LZ4),\n `data` String CODEC(ZSTD(1)),\n `ddd` Date CODEC(NONE),\n `somenum` Float64 CODEC(ZSTD(2)),\n `somestr` FixedString(3) CODEC(LZ4HC(7)),\n `othernum` Int64 CODEC(Delta(8))\n)\nENGINE = Log +1 hello 2018-12-14 1.1 aaa 5 +2 world 2018-12-15 2.2 bbb 6 +3 ! 2018-12-16 3.3 ccc 7 2 -CREATE TABLE default.compression_codec_multiple_log\n(\n `id` UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4), DEFLATE_QPL),\n `data` String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8), DEFLATE_QPL),\n `ddd` Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0), DEFLATE_QPL),\n `somenum` Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1), DEFLATE_QPL)\n)\nENGINE = Log +CREATE TABLE default.compression_codec_multiple_log\n(\n `id` UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4)),\n `data` String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8)),\n `ddd` Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0)),\n `somenum` Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1))\n)\nENGINE = Log 1 world 2018-10-05 1.1 2 hello 2018-10-01 2.2 3 buy 2018-10-11 3.3 @@ -11,12 +11,12 @@ CREATE TABLE default.compression_codec_multiple_log\n(\n `id` UInt64 CODEC(LZ 10003 274972506.6 9175437371954010821 -CREATE TABLE default.compression_codec_tiny_log\n(\n `id` UInt64 CODEC(LZ4),\n `data` String CODEC(ZSTD(1)),\n `ddd` Date CODEC(NONE),\n `somenum` Float64 CODEC(ZSTD(2)),\n `somestr` FixedString(3) CODEC(LZ4HC(7)),\n `othernum` Int64 CODEC(Delta(8)),\n `qplstr` String CODEC(DEFLATE_QPL),\n `qplnum` UInt32 CODEC(DEFLATE_QPL)\n)\nENGINE = TinyLog -1 hello 2018-12-14 1.1 aaa 5 qpl11 11 -2 world 2018-12-15 2.2 bbb 6 qpl22 22 -3 ! 2018-12-16 3.3 ccc 7 qpl33 33 +CREATE TABLE default.compression_codec_tiny_log\n(\n `id` UInt64 CODEC(LZ4),\n `data` String CODEC(ZSTD(1)),\n `ddd` Date CODEC(NONE),\n `somenum` Float64 CODEC(ZSTD(2)),\n `somestr` FixedString(3) CODEC(LZ4HC(7)),\n `othernum` Int64 CODEC(Delta(8))\n)\nENGINE = TinyLog +1 hello 2018-12-14 1.1 aaa 5 +2 world 2018-12-15 2.2 bbb 6 +3 ! 2018-12-16 3.3 ccc 7 2 -CREATE TABLE default.compression_codec_multiple_tiny_log\n(\n `id` UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4), DEFLATE_QPL),\n `data` String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8), DEFLATE_QPL),\n `ddd` Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0), DEFLATE_QPL),\n `somenum` Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1), DEFLATE_QPL)\n)\nENGINE = TinyLog +CREATE TABLE default.compression_codec_multiple_tiny_log\n(\n `id` UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4)),\n `data` String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8)),\n `ddd` Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0)),\n `somenum` Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1))\n)\nENGINE = TinyLog 1 world 2018-10-05 1.1 2 hello 2018-10-01 2.2 3 buy 2018-10-11 3.3 diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql index a629df2666d..fba6a216762 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql @@ -1,6 +1,5 @@ SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; -SET enable_deflate_qpl_codec = 1; -- copy-paste for storage log @@ -12,20 +11,18 @@ CREATE TABLE compression_codec_log( ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), - othernum Int64 CODEC(Delta), - qplstr String CODEC(DEFLATE_QPL), - qplnum UInt32 CODEC(DEFLATE_QPL), + othernum Int64 CODEC(Delta) ) ENGINE = Log(); SHOW CREATE TABLE compression_codec_log; -INSERT INTO compression_codec_log VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5, 'qpl11', 11); -INSERT INTO compression_codec_log VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6,'qpl22', 22); -INSERT INTO compression_codec_log VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7, 'qpl33', 33); +INSERT INTO compression_codec_log VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); +INSERT INTO compression_codec_log VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6); +INSERT INTO compression_codec_log VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7); SELECT * FROM compression_codec_log ORDER BY id; -INSERT INTO compression_codec_log VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8, 'qpl44', 44); +INSERT INTO compression_codec_log VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8); DETACH TABLE compression_codec_log; ATTACH TABLE compression_codec_log; @@ -37,10 +34,10 @@ DROP TABLE IF EXISTS compression_codec_log; DROP TABLE IF EXISTS compression_codec_multiple_log; CREATE TABLE compression_codec_multiple_log ( - id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4), DEFLATE_QPL), - data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8), DEFLATE_QPL), - ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC, DEFLATE_QPL), - somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD, DEFLATE_QPL) + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), + data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), + ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), + somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) ) ENGINE = Log(); SHOW CREATE TABLE compression_codec_multiple_log; @@ -72,20 +69,18 @@ CREATE TABLE compression_codec_tiny_log( ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), - othernum Int64 CODEC(Delta), - qplstr String CODEC(DEFLATE_QPL), - qplnum UInt32 CODEC(DEFLATE_QPL), + othernum Int64 CODEC(Delta) ) ENGINE = TinyLog(); SHOW CREATE TABLE compression_codec_tiny_log; -INSERT INTO compression_codec_tiny_log VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5, 'qpl11', 11); -INSERT INTO compression_codec_tiny_log VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6, 'qpl22', 22); -INSERT INTO compression_codec_tiny_log VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7, 'qpl33', 33); +INSERT INTO compression_codec_tiny_log VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); +INSERT INTO compression_codec_tiny_log VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6); +INSERT INTO compression_codec_tiny_log VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7); SELECT * FROM compression_codec_tiny_log ORDER BY id; -INSERT INTO compression_codec_tiny_log VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8, 'qpl44', 44); +INSERT INTO compression_codec_tiny_log VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8); DETACH TABLE compression_codec_tiny_log; ATTACH TABLE compression_codec_tiny_log; @@ -97,10 +92,10 @@ DROP TABLE IF EXISTS compression_codec_tiny_log; DROP TABLE IF EXISTS compression_codec_multiple_tiny_log; CREATE TABLE compression_codec_multiple_tiny_log ( - id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4), DEFLATE_QPL), - data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8), DEFLATE_QPL), - ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC, DEFLATE_QPL), - somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD, DEFLATE_QPL) + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), + data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), + ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), + somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) ) ENGINE = TinyLog(); SHOW CREATE TABLE compression_codec_multiple_tiny_log; diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql index 5a56fc0d576..78c57013eeb 100644 --- a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql @@ -1,3 +1,7 @@ +--Tags: no-fasttest, no-cpu-aarch64 +-- no-fasttest because DEFLATE_QPL isn't available in fasttest +-- no-cpu-aarch64 because DEFLATE_QPL is x86-only + SET send_logs_level = 'fatal'; SET enable_deflate_qpl_codec = 1; From d72751be27ba5f69337a0039f41e577c05a3ae7f Mon Sep 17 00:00:00 2001 From: alekseygolub Date: Sun, 11 Jun 2023 15:01:45 +0000 Subject: [PATCH 1485/2223] Added cache invalidation; Fix issues --- src/Databases/DatabaseFactory.cpp | 5 +-- src/Databases/DatabaseFilesystem.cpp | 40 ++++++++++++++----- src/Databases/DatabaseFilesystem.h | 8 +++- src/Databases/DatabaseHDFS.cpp | 6 +++ src/Databases/DatabaseHDFS.h | 5 ++- src/Databases/DatabaseS3.cpp | 14 +++---- src/Databases/DatabaseS3.h | 5 ++- .../0_stateless/02724_database_s3.reference | 4 -- .../queries/0_stateless/02724_database_s3.sh | 6 --- 9 files changed, 59 insertions(+), 34 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 26952cc574e..9d90c61bb41 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -15,10 +15,9 @@ #include #include #include -#include -#include -#include #include +#include +#include #include "config.h" diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 8de609f0ca2..cf45240a5f0 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -94,14 +94,32 @@ bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, Cont return true; } -bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr context_) const +StoragePtr DatabaseFilesystem::tryGetTableFromCache(const std::string & name) const { + StoragePtr table = nullptr; { std::lock_guard lock(mutex); - if (loaded_tables.find(name) != loaded_tables.end()) - return true; + auto it = loaded_tables.find(name); + if (it != loaded_tables.end()) + table = it->second; } + // invalidate cache if file no longer exists + if (table && !fs::exists(getTablePath(name))) + { + std::lock_guard lock(mutex); + loaded_tables.erase(name); + return nullptr; + } + + return table; +} + +bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr context_) const +{ + if (tryGetTableFromCache(name)) + return true; + fs::path table_file_path(getTablePath(name)); return checkTableFilePath(table_file_path, context_, false); @@ -109,13 +127,9 @@ bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr context_) StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_) const { - // Check if the table exists in the loaded tables map - { - std::lock_guard lock(mutex); - auto it = loaded_tables.find(name); - if (it != loaded_tables.end()) - return it->second; - } + // Check if table exists in loaded tables map + if (auto table = tryGetTableFromCache(name)) + return table; auto table_path = getTablePath(name); @@ -165,6 +179,12 @@ StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr conte } } +bool DatabaseFilesystem::empty() const +{ + std::lock_guard lock(mutex); + return loaded_tables.empty(); +} + ASTPtr DatabaseFilesystem::getCreateDatabaseQuery() const { auto settings = getContext()->getSettingsRef(); diff --git a/src/Databases/DatabaseFilesystem.h b/src/Databases/DatabaseFilesystem.h index 3d2ad695cc6..350ebfe37a3 100644 --- a/src/Databases/DatabaseFilesystem.h +++ b/src/Databases/DatabaseFilesystem.h @@ -31,7 +31,10 @@ public: StoragePtr tryGetTable(const String & name, ContextPtr context) const override; - bool empty() const override { return true; } + // Contains only temporary tables + bool shouldBeEmptyOnDetach() const override { return false; } + + bool empty() const override; bool isReadOnly() const override { return true; } @@ -45,13 +48,14 @@ public: protected: StoragePtr getTableImpl(const String & name, ContextPtr context) const; + StoragePtr tryGetTableFromCache(const std::string & name) const; + std::string getTablePath(const std::string & table_name) const; void addTable(const std::string & table_name, StoragePtr table_storage) const; bool checkTableFilePath(const std::string & table_path, ContextPtr context_, bool throw_on_error) const; - private: String path; mutable Tables loaded_tables TSA_GUARDED_BY(mutex); diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index 39c3f955bf5..34cb337cdbe 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -170,6 +170,12 @@ StoragePtr DatabaseHDFS::tryGetTable(const String & name, ContextPtr context_) c } } +bool DatabaseHDFS::empty() const +{ + std::lock_guard lock(mutex); + return loaded_tables.empty(); +} + ASTPtr DatabaseHDFS::getCreateDatabaseQuery() const { auto settings = getContext()->getSettingsRef(); diff --git a/src/Databases/DatabaseHDFS.h b/src/Databases/DatabaseHDFS.h index 9a506c5c8ac..c7071370b5e 100644 --- a/src/Databases/DatabaseHDFS.h +++ b/src/Databases/DatabaseHDFS.h @@ -33,7 +33,10 @@ public: StoragePtr tryGetTable(const String & name, ContextPtr context) const override; - bool empty() const override { return true; } + // Contains only temporary tables + bool shouldBeEmptyOnDetach() const override { return false; } + + bool empty() const override; bool isReadOnly() const override { return true; } diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index 96616426475..46f8a67687d 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -67,14 +67,8 @@ void DatabaseS3::addTable(const std::string & table_name, StoragePtr table_stora std::string DatabaseS3::getFullUrl(const std::string & name) const { - try - { - S3::URI uri(name); - } - catch (...) - { + if (!config.url_prefix.empty()) return (fs::path(config.url_prefix) / name).string(); - } return name; } @@ -181,6 +175,12 @@ StoragePtr DatabaseS3::tryGetTable(const String & name, ContextPtr context_) con } } +bool DatabaseS3::empty() const +{ + std::lock_guard lock(mutex); + return loaded_tables.empty(); +} + ASTPtr DatabaseS3::getCreateDatabaseQuery() const { auto settings = getContext()->getSettingsRef(); diff --git a/src/Databases/DatabaseS3.h b/src/Databases/DatabaseS3.h index 4e6910566df..f494925b09b 100644 --- a/src/Databases/DatabaseS3.h +++ b/src/Databases/DatabaseS3.h @@ -43,7 +43,10 @@ public: StoragePtr tryGetTable(const String & name, ContextPtr context) const override; - bool empty() const override { return true; } + // Contains only temporary tables + bool shouldBeEmptyOnDetach() const override { return false; } + + bool empty() const override; bool isReadOnly() const override { return true; } diff --git a/tests/queries/0_stateless/02724_database_s3.reference b/tests/queries/0_stateless/02724_database_s3.reference index 811e38b7f2b..425cca6a077 100644 --- a/tests/queries/0_stateless/02724_database_s3.reference +++ b/tests/queries/0_stateless/02724_database_s3.reference @@ -12,10 +12,6 @@ test1 13 14 15 16 17 18 0 0 0 -1 2 3 -4 5 6 -7 8 9 -0 0 0 10 11 12 13 14 15 16 17 18 diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index ac1b97beecf..79199b43571 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -32,12 +32,6 @@ USE test4; SELECT * FROM \"b.tsv\" """ -# check that database url_prefix is ignored if pass full url as table name -${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ -USE test4; -SELECT * FROM \"http://localhost:11111/test/a.tsv\" -""" - # Check named collection loading ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test5; From 2419a7b90fd1effd8ebf8b5b4741a0325447cdec Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 11 Jun 2023 15:16:52 +0000 Subject: [PATCH 1486/2223] Fix tests --- .../00804_test_deflate_qpl_codec_compression.reference | 2 ++ .../00804_test_deflate_qpl_codec_compression.sql | 9 ++++++++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference index 276747f8233..a2178f5eda7 100644 --- a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.reference @@ -1,4 +1,6 @@ +CREATE TABLE default.compression_codec\n(\n `id` UInt64 CODEC(DEFLATE_QPL),\n `data` String CODEC(DEFLATE_QPL),\n `ddd` Date CODEC(DEFLATE_QPL),\n `ddd32` Date32 CODEC(DEFLATE_QPL),\n `somenum` Float64 CODEC(DEFLATE_QPL),\n `somestr` FixedString(3) CODEC(DEFLATE_QPL),\n `othernum` Int64 CODEC(DEFLATE_QPL),\n `somearray` Array(UInt8) CODEC(DEFLATE_QPL),\n `somemap` Map(String, UInt32) CODEC(DEFLATE_QPL),\n `sometuple` Tuple(UInt16, UInt64) CODEC(DEFLATE_QPL)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1 hello 2018-12-14 2018-12-14 1.1 aaa 5 [1,2,3] {'k1':1,'k2':2} (1,2) 2 world 2018-12-15 2018-12-15 2.2 bbb 6 [4,5,6] {'k3':3,'k4':4} (3,4) 3 ! 2018-12-16 2018-12-16 3.3 ccc 7 [7,8,9] {'k5':5,'k6':6} (5,6) 2 +10001 diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql index 78c57013eeb..ff3c1812c86 100644 --- a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql @@ -2,7 +2,8 @@ -- no-fasttest because DEFLATE_QPL isn't available in fasttest -- no-cpu-aarch64 because DEFLATE_QPL is x86-only -SET send_logs_level = 'fatal'; +-- A bunch of random DDLs to test the DEFLATE_QPL codec. + SET enable_deflate_qpl_codec = 1; DROP TABLE IF EXISTS compression_codec; @@ -20,6 +21,8 @@ CREATE TABLE compression_codec( sometuple Tuple(UInt16, UInt64) CODEC(DEFLATE_QPL), ) ENGINE = MergeTree() ORDER BY tuple(); +SHOW CREATE TABLE compression_codec; + INSERT INTO compression_codec VALUES(1, 'hello', toDate('2018-12-14'), toDate32('2018-12-14'), 1.1, 'aaa', 5, [1,2,3], map('k1',1,'k2',2), tuple(1,2)); INSERT INTO compression_codec VALUES(2, 'world', toDate('2018-12-15'), toDate32('2018-12-15'), 2.2, 'bbb', 6, [4,5,6], map('k3',3,'k4',4), tuple(3,4)); INSERT INTO compression_codec VALUES(3, '!', toDate('2018-12-16'), toDate32('2018-12-16'), 3.3, 'ccc', 7, [7,8,9], map('k5',5,'k6',6), tuple(5,6)); @@ -35,4 +38,8 @@ ATTACH TABLE compression_codec; SELECT count(*) FROM compression_codec WHERE id = 2 GROUP BY id; +INSERT INTO compression_codec SELECT 3, '!', toDate('2018-12-16'), toDate32('2018-12-16'), 3.3, 'ccc', 7, [7,8,9], map('k5',5,'k6',6), tuple(5,6) FROM system.numbers LIMIT 10000; + +SELECT count(*) FROM compression_codec WHERE id = 3 GROUP BY id; + DROP TABLE IF EXISTS compression_codec; From 598501011f5cbedb42188b2f828c055d44a0fcd8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Jun 2023 17:51:54 +0200 Subject: [PATCH 1487/2223] Relax "too many parts" further --- programs/server/Server.cpp | 1 - src/Loggers/OwnPatternFormatter.cpp | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 6 +++--- src/Storages/MergeTree/MergeTreeSink.cpp | 9 +++++++-- src/Storages/MergeTree/MergeTreeSink.h | 3 ++- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 9 +++++++-- src/Storages/MergeTree/ReplicatedMergeTreeSink.h | 1 + 9 files changed, 26 insertions(+), 16 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d0fc8aca5e8..cfef7f0a94a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1705,7 +1705,6 @@ try #endif /// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread. - async_metrics.start(); { diff --git a/src/Loggers/OwnPatternFormatter.cpp b/src/Loggers/OwnPatternFormatter.cpp index ccf6c479b80..0c2256aaa1b 100644 --- a/src/Loggers/OwnPatternFormatter.cpp +++ b/src/Loggers/OwnPatternFormatter.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9cca471fddb..b42d130bf62 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4315,14 +4315,14 @@ std::optional MergeTreeData::getMinPartDataVersion() const } -void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const ContextPtr & query_context) const +void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const ContextPtr & query_context, bool allow_throw) const { const auto settings = getSettings(); const auto & query_settings = query_context->getSettingsRef(); const size_t parts_count_in_total = getActivePartsCount(); - /// check if have too many parts in total - if (parts_count_in_total >= settings->max_parts_in_total) + /// Check if we have too many parts in total + if (allow_throw && parts_count_in_total >= settings->max_parts_in_total) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception( @@ -4338,7 +4338,7 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const Contex if (settings->inactive_parts_to_throw_insert > 0 || settings->inactive_parts_to_delay_insert > 0) outdated_parts_count_in_partition = getMaxOutdatedPartsCountForPartition(); - if (settings->inactive_parts_to_throw_insert > 0 && outdated_parts_count_in_partition >= settings->inactive_parts_to_throw_insert) + if (allow_throw && settings->inactive_parts_to_throw_insert > 0 && outdated_parts_count_in_partition >= settings->inactive_parts_to_throw_insert) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception( @@ -4362,7 +4362,7 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const Contex bool parts_are_large_enough_in_average = settings->max_avg_part_size_for_too_many_parts && average_part_size > settings->max_avg_part_size_for_too_many_parts; - if (parts_count_in_partition >= active_parts_to_throw_insert && !parts_are_large_enough_in_average) + if (allow_throw && parts_count_in_partition >= active_parts_to_throw_insert && !parts_are_large_enough_in_average) { ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception( diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b1e1e43bd0b..ebda82eeaed 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -557,7 +557,7 @@ public: /// If the table contains too many active parts, sleep for a while to give them time to merge. /// If until is non-null, wake up from the sleep earlier if the event happened. /// The decision to delay or throw is made according to settings 'parts_to_delay_insert' and 'parts_to_throw_insert'. - void delayInsertOrThrowIfNeeded(Poco::Event * until, const ContextPtr & query_context) const; + void delayInsertOrThrowIfNeeded(Poco::Event * until, const ContextPtr & query_context, bool allow_throw) const; /// If the table contains too many unfinished mutations, sleep for a while to give them time to execute. /// If until is non-null, wake up from the sleep earlier if the event happened. diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 33aea358078..082b84be575 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -73,11 +73,11 @@ struct Settings; M(UInt64, max_delay_to_mutate_ms, 1000, "Max delay of mutating MergeTree table in milliseconds, if there are a lot of unfinished mutations", 0) \ \ /** Inserts settings. */ \ - M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ + M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ M(UInt64, inactive_parts_to_delay_insert, 0, "If table contains at least that many inactive parts in single partition, artificially slow down insert into table.", 0) \ - M(UInt64, parts_to_throw_insert, 300, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \ + M(UInt64, parts_to_throw_insert, 3000, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \ M(UInt64, inactive_parts_to_throw_insert, 0, "If more than this number inactive parts in single partition, throw 'Too many inactive parts ...' exception.", 0) \ - M(UInt64, max_avg_part_size_for_too_many_parts, 10ULL * 1024 * 1024 * 1024, "The 'too many parts' check according to 'parts_to_delay_insert' and 'parts_to_throw_insert' will be active only if the average part size (in the relevant partition) is not larger than the specified threshold. If it is larger than the specified threshold, the INSERTs will be neither delayed or rejected. This allows to have hundreds of terabytes in a single table on a single server if the parts are successfully merged to larger parts. This does not affect the thresholds on inactive parts or total parts.", 0) \ + M(UInt64, max_avg_part_size_for_too_many_parts, 1ULL * 1024 * 1024 * 1024, "The 'too many parts' check according to 'parts_to_delay_insert' and 'parts_to_throw_insert' will be active only if the average part size (in the relevant partition) is not larger than the specified threshold. If it is larger than the specified threshold, the INSERTs will be neither delayed or rejected. This allows to have hundreds of terabytes in a single table on a single server if the parts are successfully merged to larger parts. This does not affect the thresholds on inactive parts or total parts.", 0) \ M(UInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \ M(UInt64, min_delay_to_insert_ms, 10, "Min delay of inserting data into MergeTree table in milliseconds, if there are a lot of unmerged parts in single partition.", 0) \ M(UInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index d62fe5024f4..36816904a81 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -45,9 +45,9 @@ MergeTreeSink::MergeTreeSink( void MergeTreeSink::onStart() { - /// Only check "too many parts" before write, + /// It's only allowed to throw "too many parts" before write, /// because interrupting long-running INSERT query in the middle is not convenient for users. - storage.delayInsertOrThrowIfNeeded(nullptr, context); + storage.delayInsertOrThrowIfNeeded(nullptr, context, true); } void MergeTreeSink::onFinish() @@ -57,6 +57,9 @@ void MergeTreeSink::onFinish() void MergeTreeSink::consume(Chunk chunk) { + if (num_blocks_processed > 0) + storage.delayInsertOrThrowIfNeeded(nullptr, context, false); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); if (!storage_snapshot->object_columns.empty()) convertDynamicColumnsToTuples(block, storage_snapshot); @@ -136,6 +139,8 @@ void MergeTreeSink::consume(Chunk chunk) finishDelayedChunk(); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); + + ++num_blocks_processed; } void MergeTreeSink::finishDelayedChunk() diff --git a/src/Storages/MergeTree/MergeTreeSink.h b/src/Storages/MergeTree/MergeTreeSink.h index 68f11d86a25..07ab3850df2 100644 --- a/src/Storages/MergeTree/MergeTreeSink.h +++ b/src/Storages/MergeTree/MergeTreeSink.h @@ -35,7 +35,8 @@ private: size_t max_parts_per_block; ContextPtr context; StorageSnapshotPtr storage_snapshot; - uint64_t chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token + UInt64 chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token + UInt64 num_blocks_processed = 0; /// We can delay processing for previous chunk and start writing a new one. struct DelayedChunk; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 28dad454afe..5fbd72ccddc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -367,6 +367,9 @@ size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const template void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) { + if (num_blocks_processed > 0) + storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context, false); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); const auto & settings = context->getSettingsRef(); @@ -512,6 +515,8 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) /// TODO: we can also delay commit if there is no MVs. if (!settings.deduplicate_blocks_in_dependent_materialized_views) finishDelayedChunk(zookeeper); + + ++num_blocks_processed; } template<> @@ -1136,9 +1141,9 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: template void ReplicatedMergeTreeSinkImpl::onStart() { - /// Only check "too many parts" before write, + /// It's only allowed to throw "too many parts" before write, /// because interrupting long-running INSERT query in the middle is not convenient for users. - storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context); + storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context, true); } template diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 8d9e2e14129..868590efa25 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -123,6 +123,7 @@ private: bool quorum_parallel = false; const bool deduplicate = true; bool last_block_is_duplicate = false; + UInt64 num_blocks_processed = 0; using Logger = Poco::Logger; Poco::Logger * log; From 61fa6944145f851e156943db678e0657a0d6fb42 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Sun, 11 Jun 2023 19:03:57 +0000 Subject: [PATCH 1488/2223] Fix boundery -> boundary in docs --- .../sql-reference/functions/type-conversion-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 28db7e6e677..9258b6d6026 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -317,7 +317,7 @@ SELECT ## toDateOrZero -The same as [toDate](#todate) but returns lower boundery of [Date](/docs/en/sql-reference/data-types/date.md) if an invalid argument is received. Only [String](/docs/en/sql-reference/data-types/string.md) argument is supported. +The same as [toDate](#todate) but returns lower boundary of [Date](/docs/en/sql-reference/data-types/date.md) if an invalid argument is received. Only [String](/docs/en/sql-reference/data-types/string.md) argument is supported. **Example** @@ -359,7 +359,7 @@ Result: ## toDateOrDefault -Like [toDate](#todate) but if unsuccessful, returns a default value which is either the second argument (if specified), or otherwise the lower boundery of [Date](/docs/en/sql-reference/data-types/date.md). +Like [toDate](#todate) but if unsuccessful, returns a default value which is either the second argument (if specified), or otherwise the lower boundary of [Date](/docs/en/sql-reference/data-types/date.md). **Syntax** @@ -424,7 +424,7 @@ Result: ## toDateTimeOrZero -The same as [toDateTime](#todatetime) but returns lower boundery of [DateTime](/docs/en/sql-reference/data-types/datetime.md) if an invalid argument is received. Only [String](/docs/en/sql-reference/data-types/string.md) argument is supported. +The same as [toDateTime](#todatetime) but returns lower boundary of [DateTime](/docs/en/sql-reference/data-types/datetime.md) if an invalid argument is received. Only [String](/docs/en/sql-reference/data-types/string.md) argument is supported. **Example** @@ -466,7 +466,7 @@ Result: ## toDateTimeOrDefault -Like [toDateTime](#todatetime) but if unsuccessful, returns a default value which is either the third argument (if specified), or otherwise the lower boundery of [DateTime](/docs/en/sql-reference/data-types/datetime.md). +Like [toDateTime](#todatetime) but if unsuccessful, returns a default value which is either the third argument (if specified), or otherwise the lower boundary of [DateTime](/docs/en/sql-reference/data-types/datetime.md). **Syntax** From 3797a4202cb3da6bf20684149d24931e72fbd239 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 11 Jun 2023 23:38:31 +0300 Subject: [PATCH 1489/2223] Update MergeTreeData.cpp --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 23351423d49..8a69c8ff75c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1252,6 +1252,10 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( mark_broken(); return res; } + catch (const Poco::NetException &) + { + throw; + } catch (const Poco::TimeoutException &) { throw; From 7bcaf8b233e91a11fe0d82daaf265d20c8279906 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 12 Jun 2023 10:15:32 +0800 Subject: [PATCH 1490/2223] fix build error --- src/Storages/StorageRedis.cpp | 6 +++++- src/Storages/StorageRedis.h | 3 ++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 97f1dbce6da..68c71cac508 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -461,7 +461,11 @@ Block StorageRedis::getSampleBlock(const Names &) const return getInMemoryMetadataPtr()->getSampleBlock(); } -SinkToStoragePtr StorageRedis::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) +SinkToStoragePtr StorageRedis::write( + const ASTPtr & /*query*/, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr /*context*/, + bool /*async_insert*/) { return std::make_shared(*this, metadata_snapshot); } diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h index a4ab9a6aa4e..a525a4ed7de 100644 --- a/src/Storages/StorageRedis.h +++ b/src/Storages/StorageRedis.h @@ -36,7 +36,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context) override; + ContextPtr context, + bool /*async_insert*/) override; void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, From a9d4d5194972daa1dcecf80b388c6ccb127f92d0 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 12 Jun 2023 10:16:02 +0800 Subject: [PATCH 1491/2223] add word redis to aspell-dict --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a01b67b26b1..e594962ec44 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2044,6 +2044,7 @@ reconnection recurse redash reddit +redis redisstreams refcounter regexpExtract From ef40c029a5617b9abfee6fb4525de9aeca2fcf73 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 12 Jun 2023 11:54:42 +0800 Subject: [PATCH 1492/2223] fix style --- src/Storages/StorageRedis.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 68c71cac508..71c84443d8e 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -462,7 +462,7 @@ Block StorageRedis::getSampleBlock(const Names &) const } SinkToStoragePtr StorageRedis::write( - const ASTPtr & /*query*/, + const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool /*async_insert*/) From 911f8ad8dc68d126cedebd3e990ea185ed3c41b1 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 12 Jun 2023 11:57:52 +0800 Subject: [PATCH 1493/2223] use whitespace or tab as field delimiter --- docs/en/interfaces/formats.md | 1 + .../operations/settings/settings-formats.md | 32 +++++++++++++++++ src/Core/Settings.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- .../Formats/Impl/CSVRowInputFormat.cpp | 34 +++++++++---------- ...h_whitespace_tab_field_delimiter.reference | 4 +-- ...ext_with_whitespace_tab_field_delimiter.sh | 4 +-- 8 files changed, 57 insertions(+), 24 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index f19fd94dcd8..57962c1d730 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -468,6 +468,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_skip_first_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_first_lines) - skip the specified number of lines at the beginning of data. Default value - `0`. - [input_format_csv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_csv_detect_header) - automatically detect header with names and types in CSV format. Default value - `true`. - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. +- [input_format_csv_use_whitespace_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_use_whitespace_tab_as_delimiter) - use whitespace or tab as field delimiter in CSV strings. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 025e9f889f3..0e30c8f319e 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -914,6 +914,38 @@ Result " string " ``` +### input_format_csv_use_whitespace_tab_as_delimiter {#input_format_csv_use_whitespace_tab_as_delimiter} + +Use whitespace or tab as field delimiter in CSV strings. + +Default value: `false`. + +**Examples** + +Query + +```bash +echo 'a b' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_use_whitespace_tab_as_delimiter=true --format_csv_delimiter=' ' +``` + +Result + +```text +a b +``` + +Query + +```bash +echo 'a b' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_use_whitespace_tab_as_delimiter=true --format_csv_delimiter='\t' +``` + +Result + +```text +a b +``` + ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 45641e76689..4306ac855a3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -850,7 +850,7 @@ class IColumn; M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ - M(Bool, input_format_csv_skip_whitespaces_tabs, true, "Skips spaces and tabs(\\t) characters in the CSV strings", 0) \ + M(Bool, input_format_csv_use_whitespace_tab_as_delimiter, false, "Use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 73a7d4f73f2..33ecddfc223 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -70,7 +70,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.skip_first_lines = settings.input_format_csv_skip_first_lines; format_settings.csv.try_detect_header = settings.input_format_csv_detect_header; format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; - format_settings.csv.skip_whitespaces_tabs = settings.input_format_csv_skip_whitespaces_tabs; + format_settings.csv.use_whitespace_tab_as_delimiter = settings.input_format_csv_use_whitespace_tab_as_delimiter; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 434389e31a1..72d60e8423e 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -137,7 +137,7 @@ struct FormatSettings String custom_delimiter; bool try_detect_header = true; bool trim_whitespaces = true; - bool skip_whitespaces_tabs = true; + bool use_whitespace_tab_as_delimiter = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 4094285e1ad..b8d3413f863 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -25,10 +25,10 @@ namespace ErrorCodes namespace { - void checkBadDelimiter(char delimiter, bool skip_whitespaces_tabs) + void checkBadDelimiter(char delimiter, bool use_whitespace_tab_as_delimiter) { constexpr std::string_view bad_delimiters = " \t\"'.UL"; - if (bad_delimiters.find(delimiter) != std::string_view::npos && skip_whitespaces_tabs) + if (bad_delimiters.find(delimiter) != std::string_view::npos && !use_whitespace_tab_as_delimiter) throw Exception( ErrorCodes::BAD_ARGUMENTS, "CSV format may not work correctly with delimiter '{}'. Try use CustomSeparated format instead", @@ -68,7 +68,7 @@ CSVRowInputFormat::CSVRowInputFormat( format_settings_.csv.try_detect_header), buf(std::move(in_)) { - checkBadDelimiter(format_settings_.csv.delimiter, format_settings_.csv.skip_whitespaces_tabs); + checkBadDelimiter(format_settings_.csv.delimiter, format_settings_.csv.use_whitespace_tab_as_delimiter); } CSVRowInputFormat::CSVRowInputFormat( @@ -90,7 +90,7 @@ CSVRowInputFormat::CSVRowInputFormat( format_settings_.csv.try_detect_header), buf(std::move(in_)) { - checkBadDelimiter(format_settings_.csv.delimiter, format_settings_.csv.skip_whitespaces_tabs); + checkBadDelimiter(format_settings_.csv.delimiter, format_settings_.csv.use_whitespace_tab_as_delimiter); } void CSVRowInputFormat::syncAfterError() @@ -134,9 +134,9 @@ static void skipEndOfLine(ReadBuffer & in) } /// Skip `whitespace` symbols allowed in CSV. -static inline void skipWhitespacesAndTabs(ReadBuffer & in, const bool & skip_whitespaces_tabs) +static inline void skipWhitespacesAndTabs(ReadBuffer & in, const bool & use_whitespace_tab_as_delimiter) { - if (!skip_whitespaces_tabs) + if (use_whitespace_tab_as_delimiter) { return; } @@ -150,7 +150,7 @@ CSVFormatReader::CSVFormatReader(PeekableReadBuffer & buf_, const FormatSettings void CSVFormatReader::skipFieldDelimiter() { - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); assertChar(format_settings.csv.delimiter, *buf); } @@ -158,7 +158,7 @@ template String CSVFormatReader::readCSVFieldIntoString() { if (format_settings.csv.trim_whitespaces) [[likely]] - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); String field; if constexpr (read_string) @@ -170,14 +170,14 @@ String CSVFormatReader::readCSVFieldIntoString() void CSVFormatReader::skipField() { - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); NullOutput out; readCSVStringInto(out, *buf, format_settings.csv); } void CSVFormatReader::skipRowEndDelimiter() { - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); if (buf->eof()) return; @@ -186,7 +186,7 @@ void CSVFormatReader::skipRowEndDelimiter() if (*buf->position() == format_settings.csv.delimiter) ++buf->position(); - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); if (buf->eof()) return; @@ -198,7 +198,7 @@ void CSVFormatReader::skipHeaderRow() do { skipField(); - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); } while (checkChar(format_settings.csv.delimiter, *buf)); skipRowEndDelimiter(); @@ -211,7 +211,7 @@ std::vector CSVFormatReader::readRowImpl() do { fields.push_back(readCSVFieldIntoString()); - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); } while (checkChar(format_settings.csv.delimiter, *buf)); skipRowEndDelimiter(); @@ -224,7 +224,7 @@ bool CSVFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) try { - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); assertChar(delimiter, *buf); } catch (const DB::Exception &) @@ -250,7 +250,7 @@ bool CSVFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) { - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); if (buf->eof()) return true; @@ -259,7 +259,7 @@ bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) if (*buf->position() == format_settings.csv.delimiter) { ++buf->position(); - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); if (buf->eof()) return true; } @@ -287,7 +287,7 @@ bool CSVFormatReader::readField( const String & /*column_name*/) { if (format_settings.csv.trim_whitespaces || !isStringOrFixedString(removeNullable(type))) [[likely]] - skipWhitespacesAndTabs(*buf, format_settings.csv.skip_whitespaces_tabs); + skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; const bool at_last_column_line_end = is_last_file_column && (buf->eof() || *buf->position() == '\n' || *buf->position() == '\r'); diff --git a/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.reference b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.reference index 531391394a7..228436130dc 100644 --- a/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.reference +++ b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.reference @@ -1,2 +1,2 @@ -1 a b -2 c d +1 a b +2 c d diff --git a/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh index e3f61262674..deb6e317aac 100755 --- a/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh +++ b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh @@ -10,8 +10,8 @@ $CLICKHOUSE_CLIENT -q "drop table if exists test_whitespace" $CLICKHOUSE_CLIENT -q "drop table if exists test_tab" $CLICKHOUSE_CLIENT -q "create table test_whitespace (x UInt32, y String, z String) engine=MergeTree order by x" $CLICKHOUSE_CLIENT -q "create table test_tab (x UInt32, y String, z String) engine=MergeTree order by x" -cat $CURDIR/data_csv/csv_with_space_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_whitespace SETTINGS format_csv_delimiter=' ', input_format_csv_skip_whitespaces_tabs=false FORMAT CSV" -cat $CURDIR/data_csv/csv_with_tab_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tab SETTINGS format_csv_delimiter='\t', input_format_csv_skip_whitespaces_tabs=false FORMAT CSV" +cat $CURDIR/data_csv/csv_with_space_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_whitespace SETTINGS format_csv_delimiter=' ', input_format_csv_use_whitespace_tab_as_delimiter=true FORMAT CSV" +cat $CURDIR/data_csv/csv_with_tab_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tab SETTINGS format_csv_delimiter='\t', input_format_csv_use_whitespace_tab_as_delimiter=true FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_whitespace" $CLICKHOUSE_CLIENT -q "select * from test_tab" $CLICKHOUSE_CLIENT -q "drop table test_whitespace" From 1af062a53214168345838f79cba53ecb11cbc41e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Jun 2023 08:04:55 +0000 Subject: [PATCH 1494/2223] Un-flake 00804_test_deflate_qpl_codec_compression --- .../0_stateless/00804_test_deflate_qpl_codec_compression.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql index ff3c1812c86..8a256567e80 100644 --- a/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_deflate_qpl_codec_compression.sql @@ -6,6 +6,10 @@ SET enable_deflate_qpl_codec = 1; +-- Suppress test failures because stderr contains warning "Initialization of hardware-assisted DeflateQpl failed, falling +-- back to software DeflateQpl coded." +SET send_logs_level = 'fatal'; + DROP TABLE IF EXISTS compression_codec; CREATE TABLE compression_codec( From c378c3fcbbb678e96f5bc11958295f4dd1b4b6ba Mon Sep 17 00:00:00 2001 From: Julian Maicher Date: Mon, 12 Jun 2023 10:29:46 +0200 Subject: [PATCH 1495/2223] Fix type of LDAP server params hash in cache entry In 1ed7ad57d91db198ca94085a2c56372fb813543a, we switched from (`size_t`, usually 64bit) to SipHash (128bit) and forgot to change the type of the cache entry. This broke the caching of successful LDAP authentication requests (verification cooldown). Fixes #50864 --- src/Access/ExternalAuthenticators.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Access/ExternalAuthenticators.h b/src/Access/ExternalAuthenticators.h index bf928c18d5b..7b47c9351fd 100644 --- a/src/Access/ExternalAuthenticators.h +++ b/src/Access/ExternalAuthenticators.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -42,7 +43,7 @@ public: private: struct LDAPCacheEntry { - std::size_t last_successful_params_hash = 0; + UInt128 last_successful_params_hash = 0; std::chrono::steady_clock::time_point last_successful_authentication_timestamp; LDAPClient::SearchResultsList last_successful_role_search_results; }; From 6b2c33b1e478f175bec55e41a7ca8054807bb4fa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Jun 2023 09:16:22 +0000 Subject: [PATCH 1496/2223] Document x86 / ARM prerequisites for Docker image --- docker/server/README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/server/README.md b/docker/server/README.md index e6c9ee51fa7..46d30f252b4 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -16,6 +16,10 @@ For more information and documentation see https://clickhouse.com/. - The tag `head` is built from the latest commit to the default branch. - Each tag has optional `-alpine` suffix to reflect that it's built on top of `alpine`. +Compatibility +- The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. +- The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A). Most ARM CPUs after 2017 support ARMv8.2-A. A notable exception is Raspberry Pi 4 from 2019 whose CPU only supports ARMv8.0-A. + ## How to use this image ### start server instance From a7408170c8bce6be3d4849fc3614834d8f646298 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 12 Jun 2023 11:21:43 +0200 Subject: [PATCH 1497/2223] Use H3 --- docker/server/README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/server/README.md b/docker/server/README.md index 46d30f252b4..67646a262f5 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -16,7 +16,8 @@ For more information and documentation see https://clickhouse.com/. - The tag `head` is built from the latest commit to the default branch. - Each tag has optional `-alpine` suffix to reflect that it's built on top of `alpine`. -Compatibility +### Compatibility + - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. - The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A). Most ARM CPUs after 2017 support ARMv8.2-A. A notable exception is Raspberry Pi 4 from 2019 whose CPU only supports ARMv8.0-A. From 676ba2fbde78ec9ada09a45c0453e0cd96a3ab01 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 12 Jun 2023 12:30:38 +0300 Subject: [PATCH 1498/2223] Update MergeTreeData.cpp --- src/Storages/MergeTree/MergeTreeData.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8a69c8ff75c..ee06056985a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -99,6 +99,7 @@ #include #include +#include template <> struct fmt::formatter : fmt::formatter From c85344f83b8c1568e67ef45fdcb55b0ec0c07a8b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 12 Jun 2023 10:02:44 +0000 Subject: [PATCH 1499/2223] Try to fix test (becouse timezone randomization) --- .../0_stateless/01746_convert_type_with_default.reference | 2 +- tests/queries/0_stateless/01746_convert_type_with_default.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.reference b/tests/queries/0_stateless/01746_convert_type_with_default.reference index 541580d67f5..e5aa42e6116 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.reference +++ b/tests/queries/0_stateless/01746_convert_type_with_default.reference @@ -40,7 +40,7 @@ 1970-01-20 1970-01-20 2149-06-06 -1970-01-01 +1970-01-02 2023-05-30 2023-05-30 2023-05-30 14:38:20 diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index 2620780cfb9..e6e420ae4c0 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -54,7 +54,7 @@ select toDateOrDefault(cast(19 as Int256)); select toDateOrDefault(cast(19 as UInt256)); select toDateOrDefault(65535); -select toDateOrDefault(65536); +select toDateOrDefault(122400); select toDateOrDefault(19507, '2000-01-01'::Date); select toDateOrDefault(-1, '2023-05-30'::Date); From b634012feb40445079145639d23c44967c00547e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Jun 2023 10:57:03 +0000 Subject: [PATCH 1500/2223] Add feature flags for Keeper API --- src/Common/ZooKeeper/IKeeper.h | 4 +- src/Common/ZooKeeper/TestKeeper.h | 5 +- src/Common/ZooKeeper/ZooKeeper.cpp | 4 +- src/Common/ZooKeeper/ZooKeeper.h | 7 +- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 1 + src/Common/ZooKeeper/ZooKeeperImpl.cpp | 60 +++++++---- src/Common/ZooKeeper/ZooKeeperImpl.h | 7 +- .../ZooKeeper/ZooKeeperWithFaultInjection.h | 4 +- src/Coordination/CoordinationSettings.cpp | 2 +- src/Coordination/FourLetterCommand.cpp | 27 ++++- src/Coordination/FourLetterCommand.h | 12 +++ src/Coordination/KeeperConstants.h | 9 +- src/Coordination/KeeperContext.cpp | 46 ++++---- src/Coordination/KeeperContext.h | 3 + src/Coordination/KeeperDispatcher.cpp | 47 ++++---- src/Coordination/KeeperDispatcher.h | 8 ++ src/Coordination/KeeperFeatureFlags.cpp | 102 ++++++++++++++++++ src/Coordination/KeeperFeatureFlags.h | 49 +++++++++ src/Coordination/KeeperServer.cpp | 8 +- src/Coordination/KeeperServer.h | 1 + src/Coordination/KeeperStorage.cpp | 4 +- src/Coordination/tests/gtest_coordination.cpp | 15 +-- .../StorageSystemZooKeeperConnection.cpp | 4 +- .../test_keeper_api_version_config/test.py | 87 --------------- .../__init__.py | 0 .../configs/enable_keeper.xml | 2 +- .../test_keeper_feature_flags_config/test.py | 92 ++++++++++++++++ 27 files changed, 422 insertions(+), 188 deletions(-) create mode 100644 src/Coordination/KeeperFeatureFlags.cpp create mode 100644 src/Coordination/KeeperFeatureFlags.h delete mode 100644 tests/integration/test_keeper_api_version_config/test.py rename tests/integration/{test_keeper_api_version_config => test_keeper_feature_flags_config}/__init__.py (100%) rename tests/integration/{test_keeper_api_version_config => test_keeper_feature_flags_config}/configs/enable_keeper.xml (97%) create mode 100644 tests/integration/test_keeper_feature_flags_config/test.py diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 3eb5819df90..369aacf16c7 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include @@ -530,7 +530,7 @@ public: const Requests & requests, MultiCallback callback) = 0; - virtual DB::KeeperApiVersion getApiVersion() const = 0; + virtual bool isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const = 0; /// Expire session and finish all pending requests virtual void finalize(const String & reason) = 0; diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 4bffa4e1d4f..9bbd018cfb1 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace Coordination @@ -92,9 +93,9 @@ public: void finalize(const String & reason) override; - DB::KeeperApiVersion getApiVersion() const override + bool isFeatureEnabled(DB::KeeperFeatureFlag) const override { - return KeeperApiVersion::ZOOKEEPER_COMPATIBLE; + return false; } struct Node diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index a587ad6caf4..826032fc56b 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -865,9 +865,9 @@ bool ZooKeeper::expired() return impl->isExpired(); } -DB::KeeperApiVersion ZooKeeper::getApiVersion() const +bool ZooKeeper::isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const { - return impl->getApiVersion(); + return impl->isFeatureEnabled(feature_flag); } Int64 ZooKeeper::getClientID() diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 96f9914b597..a4a631b9d5a 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -215,7 +216,7 @@ public: /// Returns true, if the session has expired. bool expired(); - DB::KeeperApiVersion getApiVersion() const; + bool isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const; /// Create a znode. /// Throw an exception if something went wrong. @@ -553,7 +554,7 @@ private: template MultiReadResponses multiRead(TIter start, TIter end, RequestFactory request_factory, AsyncFunction async_fun) { - if (getApiVersion() >= DB::KeeperApiVersion::WITH_MULTI_READ) + if (isFeatureEnabled(DB::KeeperFeatureFlag::MULTI_READ)) { Coordination::Requests requests; for (auto it = start; it != end; ++it) @@ -685,7 +686,7 @@ String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config); template void addCheckNotExistsRequest(Coordination::Requests & requests, const Client & client, const std::string & path) { - if (client.getApiVersion() >= DB::KeeperApiVersion::WITH_CHECK_NOT_EXISTS) + if (client.isFeatureEnabled(DB::KeeperFeatureFlag::CHECK_NOT_EXISTS)) { auto request = std::make_shared(); request->path = path; diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 0ebc7f667cb..e99285b0056 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 7f23ac00efe..bf0d1871244 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -354,7 +354,8 @@ ZooKeeper::ZooKeeper( send_thread = ThreadFromGlobalPool([this] { sendThread(); }); receive_thread = ThreadFromGlobalPool([this] { receiveThread(); }); - initApiVersion(); + initFeatureFlags(); + keeper_feature_flags.logFlags(log); ProfileEvents::increment(ProfileEvents::ZooKeeperInit); } @@ -1089,41 +1090,58 @@ void ZooKeeper::pushRequest(RequestInfo && info) ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); } -KeeperApiVersion ZooKeeper::getApiVersion() const +bool ZooKeeper::isFeatureEnabled(KeeperFeatureFlag feature_flag) const { - return keeper_api_version; + return keeper_feature_flags.isEnabled(feature_flag); } -void ZooKeeper::initApiVersion() +void ZooKeeper::initFeatureFlags() { - auto promise = std::make_shared>(); - auto future = promise->get_future(); - - auto callback = [promise](const Coordination::GetResponse & response) mutable + const auto try_get = [&](const std::string & path, const std::string & description) -> std::optional { - promise->set_value(response); + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise](const Coordination::GetResponse & response) mutable + { + promise->set_value(response); + }; + + get(path, std::move(callback), {}); + if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) + { + LOG_TRACE(log, "Failed to get {}: timeout", description); + return std::nullopt; + } + + auto response = future.get(); + + if (response.error != Coordination::Error::ZOK) + { + LOG_TRACE(log, "Failed to get {}", description); + return std::nullopt; + } + + return std::move(response.data); }; - get(keeper_api_version_path, std::move(callback), {}); - if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) + if (auto feature_flags = try_get(keeper_api_feature_flags_path, "feature flags"); feature_flags.has_value()) { - LOG_TRACE(log, "Failed to get API version: timeout"); + keeper_feature_flags.setFeatureFlags(std::move(*feature_flags)); return; } - auto response = future.get(); + auto keeper_api_version_string = try_get(keeper_api_version_path, "API version"); - if (response.error != Coordination::Error::ZOK) - { - LOG_TRACE(log, "Failed to get API version"); + if (!keeper_api_version_string.has_value()) return; - } + DB::ReadBufferFromOwnString buf(*keeper_api_version_string); uint8_t keeper_version{0}; - DB::ReadBufferFromOwnString buf(response.data); DB::readIntText(keeper_version, buf); - keeper_api_version = static_cast(keeper_version); + auto keeper_api_version = static_cast(keeper_version); LOG_TRACE(log, "Detected server's API version: {}", keeper_api_version); + keeper_feature_flags.fromApiVersion(keeper_api_version); } @@ -1243,7 +1261,7 @@ void ZooKeeper::list( WatchCallback watch) { std::shared_ptr request{nullptr}; - if (keeper_api_version < Coordination::KeeperApiVersion::WITH_FILTERED_LIST) + if (!isFeatureEnabled(KeeperFeatureFlag::FILTERED_LIST)) { if (list_request_type != ListRequestType::ALL) throw Exception(Error::ZBADARGUMENTS, "Filtered list request type cannot be used because it's not supported by the server"); @@ -1308,7 +1326,7 @@ void ZooKeeper::multi( { ZooKeeperMultiRequest request(requests, default_acls); - if (request.getOpNum() == OpNum::MultiRead && keeper_api_version < Coordination::KeeperApiVersion::WITH_MULTI_READ) + if (request.getOpNum() == OpNum::MultiRead && !isFeatureEnabled(KeeperFeatureFlag::MULTI_READ)) throw Exception(Error::ZBADARGUMENTS, "MultiRead request type cannot be used because it's not supported by the server"); RequestInfo request_info; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 944c5032fac..ae6bef067e3 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -181,7 +182,7 @@ public: const Requests & requests, MultiCallback callback) override; - DB::KeeperApiVersion getApiVersion() const override; + bool isFeatureEnabled(KeeperFeatureFlag feature_flag) const override; /// Without forcefully invalidating (finalizing) ZooKeeper session before /// establishing a new one, there was a possibility that server is using @@ -312,12 +313,12 @@ private: void logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response = nullptr, bool finalize = false, UInt64 elapsed_ms = 0); - void initApiVersion(); + void initFeatureFlags(); CurrentMetrics::Increment active_session_metric_increment{CurrentMetrics::ZooKeeperSession}; std::shared_ptr zk_log; - DB::KeeperApiVersion keeper_api_version{DB::KeeperApiVersion::ZOOKEEPER_COMPATIBLE}; + DB::KeeperFeatureFlags keeper_feature_flags; }; } diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h index bf99cb76798..9d02d674010 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h @@ -402,9 +402,9 @@ public: ephemeral_nodes.clear(); } - KeeperApiVersion getApiVersion() const + bool isFeatureEnabled(KeeperFeatureFlag feature_flag) const { - return keeper->getApiVersion(); + return keeper->isFeatureEnabled(feature_flag); } private: diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 7a66134f43f..edbdfd4f9fc 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -36,7 +36,7 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco } -const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc,clrs"; +const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc,clrs,ftfl"; KeeperConfigurationAndSettings::KeeperConfigurationAndSettings() : server_id(NOT_EXIST) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index a64969e3d31..55120e70d99 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -9,6 +9,7 @@ #include #include #include +#include "Coordination/KeeperFeatureFlags.h" #include #include #include @@ -153,6 +154,9 @@ void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispat FourLetterCommandPtr clean_resources_command = std::make_shared(keeper_dispatcher); factory.registerCommand(clean_resources_command); + FourLetterCommandPtr feature_flags_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(feature_flags_command); + factory.initializeAllowList(keeper_dispatcher); factory.setInitialize(true); } @@ -486,7 +490,7 @@ String RecoveryCommand::run() String ApiVersionCommand::run() { - return toString(static_cast(Coordination::latest_keeper_api_version)); + return toString(0); } String CreateSnapshotCommand::run() @@ -535,4 +539,25 @@ String CleanResourcesCommand::run() return "ok"; } +String FeatureFlagsCommand::run() +{ + const auto & feature_flags = keeper_dispatcher.getKeeperContext()->feature_flags; + + StringBuffer ret; + + auto append = [&ret] (String key, uint8_t value) -> void + { + writeText(key, ret); + writeText('\t', ret); + writeText(std::to_string(value), ret); + writeText('\n', ret); + }; + + for (const auto feature : all_keeper_feature_flags) + append(SettingFieldKeeperFeatureFlagTraits::toString(feature), feature_flags.isEnabled(feature)); + + return ret.str(); + +} + } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index c1a91303c05..eb2cf9419ae 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -401,4 +401,16 @@ struct CleanResourcesCommand : public IFourLetterCommand ~CleanResourcesCommand() override = default; }; +struct FeatureFlagsCommand : public IFourLetterCommand +{ + explicit FeatureFlagsCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } + + String name() override { return "ftfl"; } + String run() override; + ~FeatureFlagsCommand() override = default; +}; + } diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 42161eee908..6d9c03ca8a9 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -5,6 +5,7 @@ namespace DB { +/// left for backwards compatibility enum class KeeperApiVersion : uint8_t { ZOOKEEPER_COMPATIBLE = 0, @@ -13,15 +14,9 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; -inline constexpr auto latest_keeper_api_version = KeeperApiVersion::WITH_CHECK_NOT_EXISTS; const std::string keeper_system_path = "/keeper"; const std::string keeper_api_version_path = keeper_system_path + "/api_version"; - -using PathWithData = std::pair; -const std::vector child_system_paths_with_data -{ - {keeper_api_version_path, toString(static_cast(latest_keeper_api_version))} -}; +const std::string keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index a750f2e1860..dc408afc19a 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -1,40 +1,42 @@ #include #include -#include -#include #include -#include namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -DECLARE_SETTING_ENUM(KeeperApiVersion); -IMPLEMENT_SETTING_ENUM(KeeperApiVersion, ErrorCodes::BAD_ARGUMENTS, - {{"ZOOKEEPER_COMPATIBLE", KeeperApiVersion::ZOOKEEPER_COMPATIBLE}, - {"WITH_FILTERED_LIST", KeeperApiVersion::WITH_FILTERED_LIST}, - {"WITH_MULTI_READ", KeeperApiVersion::WITH_MULTI_READ}, - {"WITH_CHECK_NOT_EXISTS", KeeperApiVersion::WITH_CHECK_NOT_EXISTS}}); - KeeperContext::KeeperContext() { - for (const auto & [path, data] : child_system_paths_with_data) - system_nodes_with_data[std::string{path}] = data; + /// enable by default some feature flags + feature_flags.enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST); + feature_flags.enableFeatureFlag(KeeperFeatureFlag::MULTI_READ); + system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); } void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) { - if (config.has("keeper_server.api_version")) + digest_enabled = config.getBool("keeper_server.digest_enabled", false); + ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); + + static const std::string feature_flags_key = "keeper_server.feature_flags"; + if (config.has(feature_flags_key)) { - auto version_string = config.getString("keeper_server.api_version"); - auto api_version = SettingFieldKeeperApiVersionTraits::fromString(version_string); - LOG_INFO(&Poco::Logger::get("KeeperContext"), "API version override used: {}", version_string); - system_nodes_with_data[keeper_api_version_path] = toString(static_cast(api_version)); + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(feature_flags_key, keys); + for (const auto & key : keys) + { + auto feature_flag = SettingFieldKeeperFeatureFlagTraits::fromString(key); + auto is_enabled = config.getBool(feature_flags_key + "." + key); + if (is_enabled) + feature_flags.enableFeatureFlag(feature_flag); + else + feature_flags.disableFeatureFlag(feature_flag); + } + + system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); } + + feature_flags.logFlags(&Poco::Logger::get("KeeperContext")); } } diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index de502b6c566..9b7000fa726 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -24,6 +25,8 @@ struct KeeperContext bool digest_enabled{true}; std::unordered_map system_nodes_with_data; + + KeeperFeatureFlags feature_flags; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index d64134f3024..7318a492b35 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -336,28 +336,39 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); - server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, snapshot_s3, [this](const KeeperStorage::RequestForSession & request_for_session) - { - /// check if we have queue of read requests depending on this request to be committed - std::lock_guard lock(read_request_queue_mutex); - if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) + keeper_context = std::make_shared(); + keeper_context->initialize(config); + + server = std::make_unique( + configuration_and_settings, + config, + responses_queue, + snapshots_queue, + keeper_context, + snapshot_s3, + [this](const KeeperStorage::RequestForSession & request_for_session) { - auto & xid_to_request_queue = it->second; - - if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); request_queue_it != xid_to_request_queue.end()) + /// check if we have queue of read requests depending on this request to be committed + std::lock_guard lock(read_request_queue_mutex); + if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) { - for (const auto & read_request : request_queue_it->second) - { - if (server->isLeaderAlive()) - server->putLocalReadRequest(read_request); - else - addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); - } + auto & xid_to_request_queue = it->second; - xid_to_request_queue.erase(request_queue_it); + if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); + request_queue_it != xid_to_request_queue.end()) + { + for (const auto & read_request : request_queue_it->second) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } + + xid_to_request_queue.erase(request_queue_it); + } } - } - }); + }); try { diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 4b8b134cf8f..1759f55d981 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -81,6 +81,8 @@ private: KeeperSnapshotManagerS3 snapshot_s3; + KeeperContextPtr keeper_context; + /// Thread put requests to raft void requestThread(); /// Thread put responses for subscribed sessions @@ -198,6 +200,12 @@ public: return configuration_and_settings; } + const KeeperContextPtr & getKeeperContext() const + { + return keeper_context; + } + + void incrementPacketsSent() { keeper_stats.incrementPacketsSent(); diff --git a/src/Coordination/KeeperFeatureFlags.cpp b/src/Coordination/KeeperFeatureFlags.cpp new file mode 100644 index 00000000000..216dca014d4 --- /dev/null +++ b/src/Coordination/KeeperFeatureFlags.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +std::pair getByteAndBitIndex(size_t num) +{ + size_t byte_idx = num / 8; + auto bit_idx = (7 - num % 8); + return {byte_idx, bit_idx}; +} + +} + +IMPLEMENT_SETTING_ENUM(KeeperFeatureFlag, ErrorCodes::BAD_ARGUMENTS, + {{"filtered_list", KeeperFeatureFlag::FILTERED_LIST}, + {"multi_read", KeeperFeatureFlag::MULTI_READ}, + {"check_not_exists", KeeperFeatureFlag::CHECK_NOT_EXISTS}}); + +KeeperFeatureFlags::KeeperFeatureFlags() +{ + /// get byte idx of largest value + auto [byte_idx, _] = getByteAndBitIndex(all_keeper_feature_flags.size() - 1); + feature_flags = std::string(byte_idx + 1, 0); +} + +KeeperFeatureFlags::KeeperFeatureFlags(std::string feature_flags_) + : feature_flags(std::move(feature_flags_)) +{} + +void KeeperFeatureFlags::fromApiVersion(KeeperApiVersion keeper_api_version) +{ + if (keeper_api_version == KeeperApiVersion::ZOOKEEPER_COMPATIBLE) + return; + + if (keeper_api_version >= KeeperApiVersion::WITH_FILTERED_LIST) + enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST); + + if (keeper_api_version >= KeeperApiVersion::WITH_MULTI_READ) + enableFeatureFlag(KeeperFeatureFlag::MULTI_READ); + + if (keeper_api_version >= KeeperApiVersion::WITH_CHECK_NOT_EXISTS) + enableFeatureFlag(KeeperFeatureFlag::CHECK_NOT_EXISTS); +} + +bool KeeperFeatureFlags::isEnabled(KeeperFeatureFlag feature_flag) const +{ + auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + + if (byte_idx > feature_flags.size()) + return false; + + return feature_flags[byte_idx] & (1 << bit_idx); +} + +void KeeperFeatureFlags::setFeatureFlags(std::string feature_flags_) +{ + feature_flags = std::move(feature_flags_); +} + +void KeeperFeatureFlags::enableFeatureFlag(KeeperFeatureFlag feature_flag) +{ + auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + chassert(byte_idx < feature_flags.size()); + + feature_flags[byte_idx] |= (1 << bit_idx); +} + +void KeeperFeatureFlags::disableFeatureFlag(KeeperFeatureFlag feature_flag) +{ + auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + chassert(byte_idx < feature_flags.size()); + + feature_flags[byte_idx] &= ~(1 << bit_idx); +} + +const std::string & KeeperFeatureFlags::getFeatureFlags() const +{ + return feature_flags; +} + +void KeeperFeatureFlags::logFlags(Poco::Logger * log) const +{ + for (const auto & feature_flag : all_keeper_feature_flags) + { + auto is_enabled = isEnabled(feature_flag); + LOG_INFO(log, "Keeper feature flag {}: {}", SettingFieldKeeperFeatureFlagTraits::toString(feature_flag), is_enabled ? "enabled" : "disabled"); + } +} + +} diff --git a/src/Coordination/KeeperFeatureFlags.h b/src/Coordination/KeeperFeatureFlags.h new file mode 100644 index 00000000000..cdd4704a7ca --- /dev/null +++ b/src/Coordination/KeeperFeatureFlags.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +enum KeeperFeatureFlag +{ + FILTERED_LIST = 0, + MULTI_READ, + CHECK_NOT_EXISTS, +}; + +static inline constexpr std::array all_keeper_feature_flags +{ + KeeperFeatureFlag::FILTERED_LIST, + KeeperFeatureFlag::MULTI_READ, + KeeperFeatureFlag::CHECK_NOT_EXISTS, +}; + +DECLARE_SETTING_ENUM(KeeperFeatureFlag); + +class KeeperFeatureFlags +{ +public: + KeeperFeatureFlags(); + + explicit KeeperFeatureFlags(std::string feature_flags_); + + /// backwards compatibility + void fromApiVersion(KeeperApiVersion keeper_api_version); + + bool isEnabled(KeeperFeatureFlag feature) const; + + void setFeatureFlags(std::string feature_flags_); + const std::string & getFeatureFlags() const; + + void enableFeatureFlag(KeeperFeatureFlag feature); + void disableFeatureFlag(KeeperFeatureFlag feature); + + void logFlags(Poco::Logger * log) const; +private: + std::string feature_flags; +}; + +} diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b3150f11cfb..43719e5a69e 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -108,23 +108,19 @@ KeeperServer::KeeperServer( const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, + KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) , is_recovering(config.getBool("keeper_server.force_recovery", false)) - , keeper_context{std::make_shared()} + , keeper_context{std::move(keeper_context_)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); - keeper_context->initialize(config); - - keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); - keeper_context->ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); - state_machine = nuraft::cs_new( responses_queue_, snapshots_queue_, diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index db4e9c1962e..8f416b1f48c 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -72,6 +72,7 @@ public: const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, + KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 8abcd062b7f..ecf795f4068 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -285,7 +285,7 @@ void KeeperStorage::initializeSystemNodes() // insert child system nodes for (const auto & [path, data] : keeper_context->system_nodes_with_data) { - assert(keeper_api_version_path.starts_with(keeper_system_path)); + assert(path.starts_with(keeper_system_path)); Node child_system_node; child_system_node.setData(data); auto [map_key, _] = container.insert(std::string{path}, child_system_node); @@ -1060,7 +1060,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce ProfileEvents::increment(ProfileEvents::KeeperGetRequest); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); - if (request.path == Coordination::keeper_api_version_path) + if (request.path == Coordination::keeper_api_feature_flags_path) return {}; if (!storage.uncommitted_state.getNode(request.path)) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 005c67ad261..2793b23c572 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2,7 +2,9 @@ #include #include "Common/ZooKeeper/IKeeper.h" +#include "Coordination/KeeperConstants.h" #include "Coordination/KeeperContext.h" +#include "Coordination/KeeperFeatureFlags.h" #include "Coordination/KeeperStorage.h" #include "Core/Defines.h" #include "IO/WriteHelpers.h" @@ -2346,18 +2348,19 @@ TEST_P(CoordinationTest, TestDurableState) } } -TEST_P(CoordinationTest, TestCurrentApiVersion) +TEST_P(CoordinationTest, TestFeatureFlags) { using namespace Coordination; KeeperStorage storage{500, "", keeper_context}; auto request = std::make_shared(); - request->path = DB::keeper_api_version_path; + request->path = DB::keeper_api_feature_flags_path; auto responses = storage.processRequest(request, 0, std::nullopt, true, true); const auto & get_response = getSingleResponse(responses); - uint8_t keeper_version{0}; - DB::ReadBufferFromOwnString buf(get_response.data); - DB::readIntText(keeper_version, buf); - EXPECT_EQ(keeper_version, static_cast(latest_keeper_api_version)); + DB::KeeperFeatureFlags feature_flags; + feature_flags.setFeatureFlags(get_response.data); + 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)); } TEST_P(CoordinationTest, TestSystemNodeModify) diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 559e12ad5ee..9a6a592f2c4 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -31,7 +31,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); res_columns[4]->insert(context->getZooKeeperSessionUptime()); res_columns[5]->insert(context->getZooKeeper()->expired()); - res_columns[6]->insert(context->getZooKeeper()->getApiVersion()); + res_columns[6]->insert(0); res_columns[7]->insert(context->getZooKeeper()->getClientID()); for (const auto & elem : context->getAuxiliaryZooKeepers()) @@ -42,7 +42,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(elem.second->getApiVersion()); + res_columns[6]->insert(0); res_columns[7]->insert(elem.second->getClientID()); } diff --git a/tests/integration/test_keeper_api_version_config/test.py b/tests/integration/test_keeper_api_version_config/test.py deleted file mode 100644 index 34d3acc4f04..00000000000 --- a/tests/integration/test_keeper_api_version_config/test.py +++ /dev/null @@ -1,87 +0,0 @@ -#!/usr/bin/env python3 - -import pytest -import os -from helpers.cluster import ClickHouseCluster -import helpers.keeper_utils as keeper_utils -from kazoo.client import KazooClient, KazooState - -CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) -cluster = ClickHouseCluster(__file__) - -# clickhouse itself will use external zookeeper -node = cluster.add_instance( - "node", - main_configs=["configs/enable_keeper.xml"], - stay_alive=True, -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - - finally: - cluster.shutdown() - - -def get_connection_zk(nodename, timeout=30.0): - _fake_zk_instance = KazooClient( - hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout - ) - _fake_zk_instance.start() - return _fake_zk_instance - - -def restart_clickhouse(api_version=None, expect_fail=True): - node.stop_clickhouse() - node.copy_file_to_container( - os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), - "/etc/clickhouse-server/config.d/enable_keeper.xml", - ) - - if api_version: - node.replace_in_config( - "/etc/clickhouse-server/config.d/enable_keeper.xml", - "", - f"{api_version}<\\/api_version>", - ) - - node.start_clickhouse(retry_start=not expect_fail) - keeper_utils.wait_until_connected(cluster, node) - - -def test_keeper_api_version(started_cluster): - restart_clickhouse() - - def assert_version(string_version, version_number): - node.wait_for_log_line( - f"Detected server's API version: {string_version}", look_behind_lines=1000 - ) - - try: - node_zk = get_connection_zk(node.name) - assert node_zk.get("/keeper/api_version")[0] == str(version_number).encode() - finally: - if node_zk: - node_zk.stop() - node_zk.close() - - assert_version("WITH_CHECK_NOT_EXISTS", 3) - - for i, version in enumerate( - [ - "ZOOKEEPER_COMPATIBLE", - "WITH_FILTERED_LIST", - "WITH_MULTI_READ", - "WITH_CHECK_NOT_EXISTS", - ] - ): - restart_clickhouse(version) - assert_version(version, i) - - with pytest.raises(Exception): - restart_clickhouse("INVALID_VERSION", expect_fail=True) diff --git a/tests/integration/test_keeper_api_version_config/__init__.py b/tests/integration/test_keeper_feature_flags_config/__init__.py similarity index 100% rename from tests/integration/test_keeper_api_version_config/__init__.py rename to tests/integration/test_keeper_feature_flags_config/__init__.py diff --git a/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml b/tests/integration/test_keeper_feature_flags_config/configs/enable_keeper.xml similarity index 97% rename from tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml rename to tests/integration/test_keeper_feature_flags_config/configs/enable_keeper.xml index c153d025598..53a169c4c3c 100644 --- a/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_feature_flags_config/configs/enable_keeper.xml @@ -18,7 +18,7 @@ 0
- + diff --git a/tests/integration/test_keeper_feature_flags_config/test.py b/tests/integration/test_keeper_feature_flags_config/test.py new file mode 100644 index 00000000000..432c413cbe8 --- /dev/null +++ b/tests/integration/test_keeper_feature_flags_config/test.py @@ -0,0 +1,92 @@ +#!/usr/bin/env python3 + +import pytest +import os +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +from kazoo.client import KazooClient, KazooState + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +cluster = ClickHouseCluster(__file__) + +# clickhouse itself will use external zookeeper +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_connection_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def restart_clickhouse(feature_flags=[], expect_fail=True): + node.stop_clickhouse() + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), + "/etc/clickhouse-server/config.d/enable_keeper.xml", + ) + + if len(feature_flags) > 0: + feature_flags_config = "" + + for feature, is_enabled in feature_flags: + feature_flags_config += f"<{feature}>{is_enabled}<\\/{feature}>" + + feature_flags_config += "<\\/feature_flags>" + + node.replace_in_config( + "/etc/clickhouse-server/config.d/enable_keeper.xml", + "", + feature_flags_config + ) + + node.start_clickhouse(retry_start=not expect_fail) + keeper_utils.wait_until_connected(cluster, node) + + +def test_keeper_feature_flags(started_cluster): + restart_clickhouse() + + def assert_feature_flags(feature_flags): + res = keeper_utils.send_4lw_cmd(started_cluster, node, "ftfl") + + for feature, is_enabled in feature_flags: + node.wait_for_log_line( + f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + ) + + node.wait_for_log_line( + f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + ) + + assert f"{feature}\t{1 if is_enabled else 0}" in res + + assert_feature_flags([("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)]) + + feature_flags = [("multi_read", 0), ("check_not_exists", 1)] + restart_clickhouse(feature_flags) + assert_feature_flags(feature_flags + [("filtered_list", 1)]) + + feature_flags = [("multi_read", 0), ("check_not_exists", 0), ("filtered_list", 0)] + restart_clickhouse(feature_flags) + assert_feature_flags(feature_flags) + + with pytest.raises(Exception): + restart_clickhouse([("invalid_feature", 1)], expect_fail=True) From eb16745033fd34b5c9c32124b6bb870f7c795f9c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Jun 2023 11:04:07 +0000 Subject: [PATCH 1501/2223] Collect sets from indexHint actions as well. --- src/Planner/Planner.cpp | 44 ++++++++++++++++++++++++++++------------- 1 file changed, 30 insertions(+), 14 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index c1f472bb5a8..4ac81e28f92 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -895,6 +896,34 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana query_plan.addStep(std::move(offsets_step)); } +void collectSetsFromActionsDAG(const ActionsDAGPtr & dag, std::unordered_set & useful_sets) +{ + for (const auto & node : dag->getNodes()) + { + if (node.column) + { + const IColumn * column = node.column.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); + + if (const auto * column_set = typeid_cast(column)) + useful_sets.insert(column_set->getData().get()); + } + + if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint") + { + ActionsDAG::NodeRawConstPtrs children; + if (const auto * adaptor = typeid_cast(node.function_base.get())) + { + if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) + { + collectSetsFromActionsDAG(index_hint->getActions(), useful_sets); + } + } + } + } +} + void addBuildSubqueriesForSetsStepIfNeeded( QueryPlan & query_plan, const SelectQueryOptions & select_query_options, @@ -907,20 +936,7 @@ void addBuildSubqueriesForSetsStepIfNeeded( PreparedSets::SubqueriesForSets subqueries_for_sets; for (const auto & actions_to_execute : result_actions_to_execute) - { - for (const auto & node : actions_to_execute->getNodes()) - { - if (node.column) - { - const IColumn * column = node.column.get(); - if (const auto * column_const = typeid_cast(column)) - column = &column_const->getDataColumn(); - - if (const auto * column_set = typeid_cast(column)) - useful_sets.insert(column_set->getData().get()); - } - } - } + collectSetsFromActionsDAG(actions_to_execute, useful_sets); auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.set.get()); }; auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); From 708a99a6ea63409ce33f83d450592eaa42411ebb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 12 Jun 2023 11:19:38 +0000 Subject: [PATCH 1502/2223] Automatic style fix --- .../test_keeper_feature_flags_config/test.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_keeper_feature_flags_config/test.py b/tests/integration/test_keeper_feature_flags_config/test.py index 432c413cbe8..bb7252e9ec8 100644 --- a/tests/integration/test_keeper_feature_flags_config/test.py +++ b/tests/integration/test_keeper_feature_flags_config/test.py @@ -45,7 +45,7 @@ def restart_clickhouse(feature_flags=[], expect_fail=True): if len(feature_flags) > 0: feature_flags_config = "" - + for feature, is_enabled in feature_flags: feature_flags_config += f"<{feature}>{is_enabled}<\\/{feature}>" @@ -54,7 +54,7 @@ def restart_clickhouse(feature_flags=[], expect_fail=True): node.replace_in_config( "/etc/clickhouse-server/config.d/enable_keeper.xml", "", - feature_flags_config + feature_flags_config, ) node.start_clickhouse(retry_start=not expect_fail) @@ -69,16 +69,20 @@ def test_keeper_feature_flags(started_cluster): for feature, is_enabled in feature_flags: node.wait_for_log_line( - f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", + look_behind_lines=1000, ) node.wait_for_log_line( - f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", + look_behind_lines=1000, ) assert f"{feature}\t{1 if is_enabled else 0}" in res - assert_feature_flags([("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)]) + assert_feature_flags( + [("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)] + ) feature_flags = [("multi_read", 0), ("check_not_exists", 1)] restart_clickhouse(feature_flags) From 5cec4c3161b84e32341ef723dc8cea2b38343b69 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 12 Jun 2023 11:34:40 +0000 Subject: [PATCH 1503/2223] Fallback to parsing big integer from String instead of exception in Parquet format --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 9 +++------ .../02786_parquet_big_integer_compatibility.reference | 1 + .../02786_parquet_big_integer_compatibility.sh | 9 +++++++++ 3 files changed, 13 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02786_parquet_big_integer_compatibility.reference create mode 100755 tests/queries/0_stateless/02786_parquet_big_integer_compatibility.sh diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 0b4700c9d4c..5a7306111a5 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -202,13 +202,10 @@ static ColumnWithTypeAndName readColumnWithBigNumberFromBinaryData(std::shared_p for (size_t i = 0; i != chunk_length; ++i) { + /// If at least one value size is not equal to the size if big integer, fallback to reading String column and further cast to result type. if (!chunk.IsNull(i) && chunk.value_length(i) != sizeof(ValueType)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot insert data into {} column from binary value, expected data with size {}, got {}", - column_type->getName(), - sizeof(ValueType), - chunk.value_length(i)); + return readColumnWithStringData(arrow_column, column_name); + total_size += chunk_length; } } diff --git a/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.reference b/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.reference new file mode 100644 index 00000000000..7764974255b --- /dev/null +++ b/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.reference @@ -0,0 +1 @@ +424242424242424242424242424242424242424242424242424242 diff --git a/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.sh b/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.sh new file mode 100755 index 00000000000..8865b2e7aab --- /dev/null +++ b/tests/queries/0_stateless/02786_parquet_big_integer_compatibility.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select toString(424242424242424242424242424242424242424242424242424242::UInt256) as x format Parquet" | $CLICKHOUSE_LOCAL --input-format=Parquet --structure='x UInt256' -q "select * from table" + From 24d70a2afd70a10a709fffe942b4e759d406f93b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 12 Jun 2023 13:37:59 +0200 Subject: [PATCH 1504/2223] Fix --- src/Formats/CapnProtoSerializer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index f51f8c4b933..b306cca4f94 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -403,7 +403,7 @@ namespace if (it == capnp_to_ch_values.end()) throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected value {} in CapnProto enum", capnp_enum_value); - assert_cast &>(column).insertValue(capnp_to_ch_values[capnp_enum_value]); + assert_cast &>(column).insertValue(it->second); } } From 42393b51ee1747e838c71196a29eb305fca6257c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 12 Jun 2023 13:45:00 +0200 Subject: [PATCH 1505/2223] Fix style --- tests/clickhouse-test | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 56cf2f0ce0f..9242ca8a0b0 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -349,10 +349,11 @@ def kill_gdb_if_any(): for i in range(5): code = subprocess.call("kill -TERM $(pidof gdb)", shell=True, timeout=30) if code != 0: - time.sleep(i) + sleep(i) else: break + # collect server stacktraces using gdb def get_stacktraces_from_gdb(server_pid): try: From 17a6512cdc80c0a14f4570c12df520deff05550c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 12 Jun 2023 13:53:20 +0200 Subject: [PATCH 1506/2223] Delete bad test --- tests/queries/0_stateless/02782_bitmap_overflow.sql | 4 ---- 1 file changed, 4 deletions(-) delete mode 100644 tests/queries/0_stateless/02782_bitmap_overflow.sql diff --git a/tests/queries/0_stateless/02782_bitmap_overflow.sql b/tests/queries/0_stateless/02782_bitmap_overflow.sql deleted file mode 100644 index 71ddce5c3b9..00000000000 --- a/tests/queries/0_stateless/02782_bitmap_overflow.sql +++ /dev/null @@ -1,4 +0,0 @@ --- Tags: no-msan, no-asan - -select unhex('0181808080908380808000')::AggregateFunction(groupBitmap, UInt64); -- {serverError TOO_LARGE_ARRAY_SIZE} - From d100a2031cd51f51e7320d62f683e7bf8520083c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 12 Jun 2023 13:53:44 +0200 Subject: [PATCH 1507/2223] Delete bad test --- tests/queries/0_stateless/02782_bitmap_overflow.reference | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 tests/queries/0_stateless/02782_bitmap_overflow.reference diff --git a/tests/queries/0_stateless/02782_bitmap_overflow.reference b/tests/queries/0_stateless/02782_bitmap_overflow.reference deleted file mode 100644 index e69de29bb2d..00000000000 From 1c8371e1db7c1bfbc9a98d2cf33b1450e5c3547a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 12 Jun 2023 12:13:10 +0000 Subject: [PATCH 1508/2223] Update version_date.tsv and changelogs after v22.8.18.31-lts --- docs/changelogs/v22.8.18.31-lts.md | 32 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 33 insertions(+) create mode 100644 docs/changelogs/v22.8.18.31-lts.md diff --git a/docs/changelogs/v22.8.18.31-lts.md b/docs/changelogs/v22.8.18.31-lts.md new file mode 100644 index 00000000000..709bb926f8a --- /dev/null +++ b/docs/changelogs/v22.8.18.31-lts.md @@ -0,0 +1,32 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v22.8.18.31-lts (4de7a95a544) FIXME as compared to v22.8.17.17-lts (df7f2ef0b41) + +#### Performance Improvement +* Backported in [#49214](https://github.com/ClickHouse/ClickHouse/issues/49214): Fixed excessive reading in queries with `FINAL`. [#47801](https://github.com/ClickHouse/ClickHouse/pull/47801) ([Nikita Taranov](https://github.com/nickitat)). + +#### Build/Testing/Packaging Improvement +* Backported in [#49079](https://github.com/ClickHouse/ClickHouse/issues/49079): Update time zones. The following were updated: Africa/Cairo, Africa/Casablanca, Africa/El_Aaiun, America/Bogota, America/Cambridge_Bay, America/Ciudad_Juarez, America/Godthab, America/Inuvik, America/Iqaluit, America/Nuuk, America/Ojinaga, America/Pangnirtung, America/Rankin_Inlet, America/Resolute, America/Whitehorse, America/Yellowknife, Asia/Gaza, Asia/Hebron, Asia/Kuala_Lumpur, Asia/Singapore, Canada/Yukon, Egypt, Europe/Kirov, Europe/Volgograd, Singapore. [#48572](https://github.com/ClickHouse/ClickHouse/pull/48572) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix bad cast from LowCardinality column when using short circuit function execution [#43311](https://github.com/ClickHouse/ClickHouse/pull/43311) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix msan issue in randomStringUTF8() [#49750](https://github.com/ClickHouse/ClickHouse/pull/49750) ([Robert Schulze](https://github.com/rschu1ze)). +* JIT compilation not equals NaN fix [#50056](https://github.com/ClickHouse/ClickHouse/pull/50056) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix crash with `multiIf` and constant condition and nullable arguments [#50123](https://github.com/ClickHouse/ClickHouse/pull/50123) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed type conversion from Date/Date32 to DateTime64 when querying with DateTime64 index [#50280](https://github.com/ClickHouse/ClickHouse/pull/50280) ([Lucas Chang](https://github.com/lucas-tubi)). +* Fix Keeper deadlock on exception when preprocessing requests. [#50387](https://github.com/ClickHouse/ClickHouse/pull/50387) ([frinkr](https://github.com/frinkr)). +* Fix Log family table return wrong rows count after truncate [#50585](https://github.com/ClickHouse/ClickHouse/pull/50585) ([flynn](https://github.com/ucasfl)). +* Do not read all the columns from right GLOBAL JOIN table. [#50721](https://github.com/ClickHouse/ClickHouse/pull/50721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Improve test reports [#49151](https://github.com/ClickHouse/ClickHouse/pull/49151) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update github.com/distribution/distribution [#50114](https://github.com/ClickHouse/ClickHouse/pull/50114) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Catch issues with dockerd during the build [#50700](https://github.com/ClickHouse/ClickHouse/pull/50700) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4647bcb4af1..dce6aadbff4 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -44,6 +44,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.18.31-lts 2023-06-12 v22.8.17.17-lts 2023-04-22 v22.8.16.32-lts 2023-04-04 v22.8.15.23-lts 2023-03-10 From d177cfceca3af796f7e5cab3adb421212d9856f0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 12 Jun 2023 12:17:11 +0000 Subject: [PATCH 1509/2223] Update version_date.tsv and changelogs after v23.3.3.52-lts --- docs/changelogs/v23.3.3.52-lts.md | 45 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 47 insertions(+) create mode 100644 docs/changelogs/v23.3.3.52-lts.md diff --git a/docs/changelogs/v23.3.3.52-lts.md b/docs/changelogs/v23.3.3.52-lts.md new file mode 100644 index 00000000000..f845e14eb78 --- /dev/null +++ b/docs/changelogs/v23.3.3.52-lts.md @@ -0,0 +1,45 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.3.3.52-lts (cb963c474db) FIXME as compared to v23.3.2.37-lts (1b144bcd101) + +#### Improvement +* Backported in [#49954](https://github.com/ClickHouse/ClickHouse/issues/49954): Add support for (an unusual) case where the arguments in the `IN` operator are single-element tuples. [#49844](https://github.com/ClickHouse/ClickHouse/pull/49844) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). + +#### Build/Testing/Packaging Improvement +* Backported in [#49210](https://github.com/ClickHouse/ClickHouse/issues/49210): Fix glibc compatibility check: replace `preadv` from musl. [#49144](https://github.com/ClickHouse/ClickHouse/pull/49144) ([alesapin](https://github.com/alesapin)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix aggregate empty string error [#48999](https://github.com/ClickHouse/ClickHouse/pull/48999) ([LiuNeng](https://github.com/liuneng1994)). +* Fix key not found error for queries with multiple StorageJoin [#49137](https://github.com/ClickHouse/ClickHouse/pull/49137) ([vdimir](https://github.com/vdimir)). +* Fix race on Outdated parts loading [#49223](https://github.com/ClickHouse/ClickHouse/pull/49223) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix bug in DISTINCT [#49628](https://github.com/ClickHouse/ClickHouse/pull/49628) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix msan issue in randomStringUTF8() [#49750](https://github.com/ClickHouse/ClickHouse/pull/49750) ([Robert Schulze](https://github.com/rschu1ze)). +* fix `is_prefix` in OptimizeRegularExpression [#49919](https://github.com/ClickHouse/ClickHouse/pull/49919) ([Han Fei](https://github.com/hanfei1991)). +* Fix IPv6 encoding in protobuf [#49933](https://github.com/ClickHouse/ClickHouse/pull/49933) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Avoid deadlock when starting table in attach thread of `ReplicatedMergeTree` [#50026](https://github.com/ClickHouse/ClickHouse/pull/50026) ([Antonio Andelic](https://github.com/antonio2368)). +* JIT compilation not equals NaN fix [#50056](https://github.com/ClickHouse/ClickHouse/pull/50056) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix crash with `multiIf` and constant condition and nullable arguments [#50123](https://github.com/ClickHouse/ClickHouse/pull/50123) ([Anton Popov](https://github.com/CurtizJ)). +* Fix reconnecting of HTTPS session when target host IP was changed [#50240](https://github.com/ClickHouse/ClickHouse/pull/50240) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fixed type conversion from Date/Date32 to DateTime64 when querying with DateTime64 index [#50280](https://github.com/ClickHouse/ClickHouse/pull/50280) ([Lucas Chang](https://github.com/lucas-tubi)). +* Fix Keeper deadlock on exception when preprocessing requests. [#50387](https://github.com/ClickHouse/ClickHouse/pull/50387) ([frinkr](https://github.com/frinkr)). +* Fix incorrect constant folding [#50536](https://github.com/ClickHouse/ClickHouse/pull/50536) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix Log family table return wrong rows count after truncate [#50585](https://github.com/ClickHouse/ClickHouse/pull/50585) ([flynn](https://github.com/ucasfl)). +* Fix bug in `uniqExact` parallel merging [#50590](https://github.com/ClickHouse/ClickHouse/pull/50590) ([Nikita Taranov](https://github.com/nickitat)). +* Do not read all the columns from right GLOBAL JOIN table. [#50721](https://github.com/ClickHouse/ClickHouse/pull/50721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Implement status comment [#48468](https://github.com/ClickHouse/ClickHouse/pull/48468) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update curl to 8.0.1 (for CVEs) [#48765](https://github.com/ClickHouse/ClickHouse/pull/48765) ([Boris Kuschel](https://github.com/bkuschel)). +* Improve test reports [#49151](https://github.com/ClickHouse/ClickHouse/pull/49151) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fallback auth gh api [#49314](https://github.com/ClickHouse/ClickHouse/pull/49314) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Improve CI: status commit, auth for get_gh_api [#49388](https://github.com/ClickHouse/ClickHouse/pull/49388) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update github.com/distribution/distribution [#50114](https://github.com/ClickHouse/ClickHouse/pull/50114) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Catch issues with dockerd during the build [#50700](https://github.com/ClickHouse/ClickHouse/pull/50700) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4647bcb4af1..411dcd81957 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,6 +2,7 @@ v23.5.2.7-stable 2023-06-10 v23.5.1.3174-stable 2023-06-09 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.3.52-lts 2023-06-12 v23.3.2.37-lts 2023-04-22 v23.3.1.2823-lts 2023-03-31 v23.2.7.32-stable 2023-06-09 @@ -44,6 +45,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.18.31-lts 2023-06-12 v22.8.17.17-lts 2023-04-22 v22.8.16.32-lts 2023-04-04 v22.8.15.23-lts 2023-03-10 From 11fbc01de5bc5840f6d91c1e9b0d10bf25387bd9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 12 Jun 2023 12:37:47 +0000 Subject: [PATCH 1510/2223] Update version_date.tsv and changelogs after v23.4.3.48-stable --- docs/changelogs/v23.4.3.48-stable.md | 42 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 ++ 2 files changed, 45 insertions(+) create mode 100644 docs/changelogs/v23.4.3.48-stable.md diff --git a/docs/changelogs/v23.4.3.48-stable.md b/docs/changelogs/v23.4.3.48-stable.md new file mode 100644 index 00000000000..8bafd22bfbd --- /dev/null +++ b/docs/changelogs/v23.4.3.48-stable.md @@ -0,0 +1,42 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.4.3.48-stable (d9199f8d3cc) FIXME as compared to v23.4.2.11-stable (b6442320f9d) + +#### Backward Incompatible Change +* Backported in [#49981](https://github.com/ClickHouse/ClickHouse/issues/49981): Revert "`groupArray` returns cannot be nullable" (due to binary compatibility breakage for `groupArray`/`groupArrayLast`/`groupArraySample` over `Nullable` types, which likely will lead to `TOO_LARGE_ARRAY_SIZE` or `CANNOT_READ_ALL_DATA`). [#49971](https://github.com/ClickHouse/ClickHouse/pull/49971) ([Azat Khuzhin](https://github.com/azat)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix key not found error for queries with multiple StorageJoin [#49137](https://github.com/ClickHouse/ClickHouse/pull/49137) ([vdimir](https://github.com/vdimir)). +* Fix fuzz bug when subquery set is not built when reading from remote() [#49425](https://github.com/ClickHouse/ClickHouse/pull/49425) ([Alexander Gololobov](https://github.com/davenger)). +* Fix postgres database setting [#49481](https://github.com/ClickHouse/ClickHouse/pull/49481) ([Mal Curtis](https://github.com/snikch)). +* Fix AsynchronousReadIndirectBufferFromRemoteFS breaking on short seeks [#49525](https://github.com/ClickHouse/ClickHouse/pull/49525) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix bug in DISTINCT [#49628](https://github.com/ClickHouse/ClickHouse/pull/49628) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix assert in SpanHolder::finish() with fibers [#49673](https://github.com/ClickHouse/ClickHouse/pull/49673) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix msan issue in randomStringUTF8() [#49750](https://github.com/ClickHouse/ClickHouse/pull/49750) ([Robert Schulze](https://github.com/rschu1ze)). +* fix `is_prefix` in OptimizeRegularExpression [#49919](https://github.com/ClickHouse/ClickHouse/pull/49919) ([Han Fei](https://github.com/hanfei1991)). +* Fix IPv6 encoding in protobuf [#49933](https://github.com/ClickHouse/ClickHouse/pull/49933) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Avoid deadlock when starting table in attach thread of `ReplicatedMergeTree` [#50026](https://github.com/ClickHouse/ClickHouse/pull/50026) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix assert in SpanHolder::finish() with fibers attempt 2 [#50034](https://github.com/ClickHouse/ClickHouse/pull/50034) ([Kruglov Pavel](https://github.com/Avogar)). +* JIT compilation not equals NaN fix [#50056](https://github.com/ClickHouse/ClickHouse/pull/50056) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix crashing in case of Replicated database without arguments [#50058](https://github.com/ClickHouse/ClickHouse/pull/50058) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash with `multiIf` and constant condition and nullable arguments [#50123](https://github.com/ClickHouse/ClickHouse/pull/50123) ([Anton Popov](https://github.com/CurtizJ)). +* Fix iceberg metadata parsing [#50232](https://github.com/ClickHouse/ClickHouse/pull/50232) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix bugs in Poco sockets in non-blocking mode, use true non-blocking sockets [#50252](https://github.com/ClickHouse/ClickHouse/pull/50252) ([Kruglov Pavel](https://github.com/Avogar)). +* Fixed type conversion from Date/Date32 to DateTime64 when querying with DateTime64 index [#50280](https://github.com/ClickHouse/ClickHouse/pull/50280) ([Lucas Chang](https://github.com/lucas-tubi)). +* Fix Keeper deadlock on exception when preprocessing requests. [#50387](https://github.com/ClickHouse/ClickHouse/pull/50387) ([frinkr](https://github.com/frinkr)). +* Fix Log family table return wrong rows count after truncate [#50585](https://github.com/ClickHouse/ClickHouse/pull/50585) ([flynn](https://github.com/ucasfl)). +* Fix bug in `uniqExact` parallel merging [#50590](https://github.com/ClickHouse/ClickHouse/pull/50590) ([Nikita Taranov](https://github.com/nickitat)). +* Do not read all the columns from right GLOBAL JOIN table. [#50721](https://github.com/ClickHouse/ClickHouse/pull/50721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Improve CI: status commit, auth for get_gh_api [#49388](https://github.com/ClickHouse/ClickHouse/pull/49388) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update github.com/distribution/distribution [#50114](https://github.com/ClickHouse/ClickHouse/pull/50114) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Catch issues with dockerd during the build [#50700](https://github.com/ClickHouse/ClickHouse/pull/50700) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4647bcb4af1..6afce99612f 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,7 +1,9 @@ v23.5.2.7-stable 2023-06-10 v23.5.1.3174-stable 2023-06-09 +v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.3.52-lts 2023-06-12 v23.3.2.37-lts 2023-04-22 v23.3.1.2823-lts 2023-03-31 v23.2.7.32-stable 2023-06-09 @@ -44,6 +46,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.18.31-lts 2023-06-12 v22.8.17.17-lts 2023-04-22 v22.8.16.32-lts 2023-04-04 v22.8.15.23-lts 2023-03-10 From a7e6264d567ffa7456d00df017675f0acf4ca90f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Jun 2023 13:05:44 +0000 Subject: [PATCH 1511/2223] Add backward compatibility --- programs/keeper/CMakeLists.txt | 1 + src/Common/ZooKeeper/CMakeLists.txt | 2 ++ src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/KeeperConstants.h | 1 - src/Coordination/KeeperContext.cpp | 4 ++++ src/Storages/System/StorageSystemZooKeeperConnection.cpp | 4 ++-- 6 files changed, 10 insertions(+), 4 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 4f74cc06801..555c6431865 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -44,6 +44,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperLogStore.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperContext.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperFeatureFlags.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManagerS3.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index a9a335d1461..3f7e87ff4a7 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -2,6 +2,8 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") add_headers_and_sources(clickhouse_common_zookeeper .) +list(APPEND clickhouse_common_zookeeper_sources ${CMAKE_CURRENT_SOURCE_DIR}/../../../src/Coordination/KeeperFeatureFlags.cpp) + # for clickhouse server add_library(clickhouse_common_zookeeper ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 55120e70d99..10d13657fb0 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -490,7 +490,7 @@ String RecoveryCommand::run() String ApiVersionCommand::run() { - return toString(0); + return toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } String CreateSnapshotCommand::run() diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 6d9c03ca8a9..84cbb0ab7c5 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -14,7 +14,6 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; - const std::string keeper_system_path = "/keeper"; const std::string keeper_api_version_path = keeper_system_path + "/api_version"; const std::string keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index dc408afc19a..e6f30c81310 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -11,6 +11,10 @@ KeeperContext::KeeperContext() feature_flags.enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST); feature_flags.enableFeatureFlag(KeeperFeatureFlag::MULTI_READ); system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); + + + /// for older clients, the default is equivalent to WITH_MULTI_READ version + system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 9a6a592f2c4..99872be6ba0 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -31,7 +31,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); res_columns[4]->insert(context->getZooKeeperSessionUptime()); res_columns[5]->insert(context->getZooKeeper()->expired()); - res_columns[6]->insert(0); + res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); res_columns[7]->insert(context->getZooKeeper()->getClientID()); for (const auto & elem : context->getAuxiliaryZooKeepers()) @@ -42,7 +42,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(0); + res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); res_columns[7]->insert(elem.second->getClientID()); } From 5db3b393d825e5597f204b9ff2ca67abf89e4045 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 12 Jun 2023 16:22:33 +0300 Subject: [PATCH 1512/2223] Update MergeTreeData.cpp --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee06056985a..5833955726f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1253,7 +1253,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( mark_broken(); return res; } - catch (const Poco::NetException &) + catch (const Poco::Net::NetException &) { throw; } From 25f08f8d194f77d0ee56e7c5132b9d5c4244a30f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Jun 2023 15:35:44 +0200 Subject: [PATCH 1513/2223] Remove duplicate include --- src/Coordination/Changelog.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index cc2ea491e17..ffa22a6b888 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -14,7 +14,6 @@ #include #include #include #include From b5b8c7086b43fbf3de9293196bfb7097e3888b58 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 12 Jun 2023 13:43:53 +0000 Subject: [PATCH 1514/2223] Update broken tests list --- tests/broken_tests.txt | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index b1fa18c44dd..d49b4f391e5 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -11,7 +11,6 @@ 00927_asof_joins 00940_order_by_read_in_order_query_plan 00945_bloom_filter_index -00952_input_function 00979_set_index_not 00981_in_subquery_with_tuple 01049_join_low_card_bug_long @@ -21,14 +20,12 @@ 01072_optimize_skip_unused_shards_const_expr_eval 01083_expressions_in_engine_arguments 01086_odbc_roundtrip -01152_cross_replication 01155_rename_move_materialized_view 01173_transaction_control_queries 01211_optimize_skip_unused_shards_type_mismatch 01213_optimize_skip_unused_shards_DISTINCT 01214_test_storage_merge_aliases_with_where 01231_distributed_aggregation_memory_efficient_mix_levels -01232_extremes 01244_optimize_distributed_group_by_sharding_key 01247_optimize_distributed_group_by_sharding_key_dist_on_dist 01268_mv_scalars @@ -50,7 +47,6 @@ 01585_use_index_for_global_in 01585_use_index_for_global_in_with_null 01586_columns_pruning -01615_random_one_shard_insertion 01624_soft_constraints 01651_bugs_from_15889 01655_plan_optimizations @@ -79,7 +75,6 @@ 01952_optimize_distributed_group_by_sharding_key 02000_join_on_const 02001_shard_num_shard_count -02024_join_on_or_long 02131_used_row_policies_in_query_log 02139_MV_with_scalar_subquery 02174_cte_scalar_cache_mv @@ -88,14 +83,11 @@ 02302_s3_file_pruning 02317_distinct_in_order_optimization_explain 02341_global_join_cte -02343_aggregation_pipeline 02345_implicit_transaction -02346_additional_filters_distr 02352_grouby_shadows_arg 02354_annoy 02366_union_decimal_conversion 02375_rocksdb_with_filters -02377_optimize_sorting_by_input_stream_properties_explain 02382_join_and_filtering_set 02402_merge_engine_with_view 02404_memory_bound_merging @@ -112,7 +104,6 @@ 02575_merge_prewhere_different_default_kind 02713_array_low_cardinality_string 02707_skip_index_with_in -02324_map_combinator_bug 02241_join_rocksdb_bs 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET 01115_join_with_dictionary @@ -120,7 +111,6 @@ 00917_multiple_joins_denny_crane 00725_join_on_bug_1 00636_partition_key_parts_pruning -00261_storage_aliases_and_array_join 01825_type_json_multiple_files 01281_group_by_limit_memory_tracking 02723_zookeeper_name From 01c7d2fe719f9b9ed59fce58d5e9dec44167e42f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 12 Jun 2023 16:53:26 +0300 Subject: [PATCH 1515/2223] Prostpone check of outdated parts (#50676) * prostpone check of outdated parts * Update ReplicatedMergeTreePartCheckThread.cpp --- .../ReplicatedMergeTreePartCheckThread.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 0882ff5a0bc..7bb8d9d758e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -344,6 +344,22 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na LOG_TRACE(log, "Part {} in zookeeper: {}, locally: {}", part_name, exists_in_zookeeper, part != nullptr); + if (exists_in_zookeeper && !part) + { + auto outdated = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting}); + if (outdated) + { + /// We cannot rely on exists_in_zookeeper, because the cleanup thread is probably going to remove it from ZooKeeper + /// Also, it will avoid "Cannot commit empty part: Part ... (state Outdated) already exists, but it will be deleted soon" + LOG_WARNING(log, "Part {} is Outdated, will wait for cleanup thread to handle it and check again later", part_name); + time_t lifetime = time(nullptr) - outdated->remove_time; + time_t max_lifetime = storage.getSettings()->old_parts_lifetime.totalSeconds(); + time_t delay = lifetime >= max_lifetime ? 0 : max_lifetime - lifetime; + enqueuePart(part_name, delay + 30); + return {part_name, true, "Part is Outdated, will recheck later"}; + } + } + /// We do not have this or a covering part. if (!part) { From d45f07743c3f27276740b3bac7200f7cad90292e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 12 Jun 2023 13:54:07 +0000 Subject: [PATCH 1516/2223] fix getting number of mutations --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a2a46229660..233b37c74a9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1358,7 +1358,7 @@ size_t StorageMergeTree::getNumberOfUnfinishedMutations() const size_t count = 0; for (const auto & [version, _] : current_mutations_by_version | std::views::reverse) { - auto status = getIncompleteMutationsStatusUnlocked(version, lock); + auto status = getIncompleteMutationsStatusUnlocked(version, lock, nullptr, true); if (!status) continue; From 26c9bda14410e6f589c843a664635db9ab02b15e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 12 Jun 2023 13:51:46 +0000 Subject: [PATCH 1517/2223] Add a comment --- src/Interpreters/InterpreterSelectQueryAnalyzer.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 4f2f05dc7eb..8db1d27c073 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -135,6 +135,8 @@ QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query, QueryTreePassManager query_tree_pass_manager(context); addQueryTreePasses(query_tree_pass_manager); + /// We should not apply any query tree level optimizations on shards + /// because it can lead to a changed header. if (select_query_options.ignore_ast_optimizations || context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) query_tree_pass_manager.run(query_tree, 1 /*up_to_pass_index*/); From 41ece306cf02519e97697d1a65bd3003d8cbe898 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 12 Jun 2023 13:53:46 +0000 Subject: [PATCH 1518/2223] Update broken_tests.txt --- tests/broken_tests.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index b1fa18c44dd..3b888223b78 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -106,7 +106,6 @@ 02458_use_structure_from_insertion_table 02479_race_condition_between_insert_and_droppin_mv 02493_inconsistent_hex_and_binary_number -02494_optimize_group_by_function_keys_and_alias_columns 02521_aggregation_by_partitions 02554_fix_grouping_sets_predicate_push_down 02575_merge_prewhere_different_default_kind From 07eb7b7d664778342e0b44049041c160ac868d94 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 12 Jun 2023 11:03:50 -0300 Subject: [PATCH 1519/2223] Update settings.md --- docs/en/operations/settings/settings.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2c9679c940d..8104478deff 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1957,6 +1957,10 @@ Default value: empty string (disabled) For the replicated tables by default the only 100 of the most recent inserts for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md/#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). For not replicated tables see [non_replicated_deduplication_window](merge-tree-settings.md/#non-replicated-deduplication-window). +:::note +`insert_deduplication_token` works on a partition level (the same as `insert_deduplication` checksum). Multiple partitions can have the same `insert_deduplication_token`. +::: + Example: ```sql From 252a10c670977c93b8808d8b98a8679714d6e9a3 Mon Sep 17 00:00:00 2001 From: tpanetti Date: Mon, 12 Jun 2023 08:19:06 -0700 Subject: [PATCH 1520/2223] Add "no-parallel" tag to MySQL Compatible Types test to fix test issue --- .../0_stateless/02775_show_columns_mysql_compatibility.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh index 51c9da2a842..e324926e2e7 100755 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 5aa05667677669a17d6356fd884c7da35478d280 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 12 Jun 2023 17:24:34 +0200 Subject: [PATCH 1521/2223] Fix checking the lock file too often while writing a backup. --- src/Backups/BackupImpl.cpp | 6 ++---- src/Backups/BackupImpl.h | 1 + 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 306236534b6..82793f44739 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -144,6 +144,7 @@ void BackupImpl::open(const ContextPtr & context) if (!uuid) uuid = UUIDHelpers::generateV4(); lock_file_name = use_archive ? (archive_params.archive_name + ".lock") : ".lock"; + lock_file_before_first_file_checked = false; writing_finalized = false; /// Check that we can write a backup there and create the lock file to own this destination. @@ -833,13 +834,10 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) if (writing_finalized) throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup is already finalized"); - bool should_check_lock_file = false; { std::lock_guard lock{mutex}; ++num_files; total_size += info.size; - if (!num_entries) - should_check_lock_file = true; } auto src_disk = entry->getDisk(); @@ -859,7 +857,7 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) return; } - if (!should_check_lock_file) + if (!lock_file_before_first_file_checked.exchange(true)) checkLockFile(true); /// NOTE: `mutex` must be unlocked during copying otherwise writing will be in one thread maximum and hence slow. diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index 7e95d156162..3ab11228892 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -141,6 +141,7 @@ private: std::shared_ptr archive_reader; std::shared_ptr archive_writer; String lock_file_name; + std::atomic lock_file_before_first_file_checked = false; bool writing_finalized = false; bool deduplicate_files = true; From 65d83e45cb177cc3abfec088e31da44fca357c95 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Jun 2023 16:21:28 +0000 Subject: [PATCH 1522/2223] Fix crash in snowflakeToDateTime(), follow-up to #50834 --- src/Functions/FunctionSnowflake.h | 18 +++++++++++++----- .../01942_snowflakeToDateTime.reference | 1 + .../0_stateless/01942_snowflakeToDateTime.sql | 4 +++- 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index ce3a48269b4..0a47534c47d 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -110,12 +111,19 @@ public: auto res_column = ColumnUInt32::create(input_rows_count); auto & result_data = res_column->getData(); - const auto & source_data = typeid_cast(col).getData(); - - for (size_t i = 0; i < input_rows_count; ++i) + if (const auto * src_non_const = typeid_cast(&col)) { - result_data[i] = static_cast( - ((source_data[i] >> time_shift) + snowflake_epoch) / 1000); + const auto & source_data = src_non_const->getData(); + for (size_t i = 0; i < input_rows_count; ++i) + result_data[i] = static_cast( + ((source_data[i] >> time_shift) + snowflake_epoch) / 1000); + } + else if (const auto * src_const = typeid_cast(&col)) + { + Int64 src_val = src_const->getValue(); + for (size_t i = 0; i < input_rows_count; ++i) + result_data[i] = static_cast( + ((src_val >> time_shift) + snowflake_epoch) / 1000); } return res_column; } diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.reference b/tests/queries/0_stateless/01942_snowflakeToDateTime.reference index bed18023f6a..fa00a22bc63 100644 --- a/tests/queries/0_stateless/01942_snowflakeToDateTime.reference +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.reference @@ -1,3 +1,4 @@ const column UTC 1426860704886947840 2021-08-15 10:57:56 DateTime(\'UTC\') 2021-08-15 10:57:56.492 DateTime64(3, \'UTC\') Asia/Shanghai 1426860704886947840 2021-08-15 18:57:56 DateTime(\'Asia/Shanghai\') 2021-08-15 18:57:56.492 DateTime64(3, \'Asia/Shanghai\') +Asia/Singapore 2010-11-04 01:42:54 diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql index f6f171afabf..3efccdddb2d 100644 --- a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql @@ -29,4 +29,6 @@ SELECT snowflakeToDateTime(i64, tz) as dt, toTypeName(dt), snowflakeToDateTime64(i64, tz) as dt64, - toTypeName(dt64); \ No newline at end of file + toTypeName(dt64); + +SELECT materialize('Asia/Singapore') a, snowflakeToDateTime(649::Int64, a) settings allow_nonconst_timezone_arguments = 1 From 326a3a3e8d719aebdc9ef9ee79f8b5fc8645183e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 12 Jun 2023 16:46:10 +0000 Subject: [PATCH 1523/2223] Use query tree to rewrite the query --- src/Storages/StorageDistributed.cpp | 335 +--------------- src/Storages/StorageReplicatedMergeTree.cpp | 22 +- src/Storages/buildQueryTreeForShard.cpp | 372 ++++++++++++++++++ src/Storages/buildQueryTreeForShard.h | 15 + ...02771_parallel_replicas_analyzer.reference | 2 +- 5 files changed, 406 insertions(+), 340 deletions(-) create mode 100644 src/Storages/buildQueryTreeForShard.cpp create mode 100644 src/Storages/buildQueryTreeForShard.h diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b91ad0b963a..1ec45ce3d57 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -81,6 +81,7 @@ #include #include +#include #include #include @@ -650,264 +651,6 @@ StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( namespace { -/// Visitor that collect column source to columns mapping from query and all subqueries -class CollectColumnSourceToColumnsVisitor : public InDepthQueryTreeVisitor -{ -public: - struct Columns - { - NameSet column_names; - NamesAndTypes columns; - - void addColumn(NameAndTypePair column) - { - if (column_names.contains(column.name)) - return; - - column_names.insert(column.name); - columns.push_back(std::move(column)); - } - }; - - const std::unordered_map & getColumnSourceToColumns() const - { - return column_source_to_columns; - } - - void visitImpl(QueryTreeNodePtr & node) - { - auto * column_node = node->as(); - if (!column_node) - return; - - auto column_source = column_node->getColumnSourceOrNull(); - if (!column_source) - return; - - auto it = column_source_to_columns.find(column_source); - if (it == column_source_to_columns.end()) - { - auto [insert_it, _] = column_source_to_columns.emplace(column_source, Columns()); - it = insert_it; - } - - it->second.addColumn(column_node->getColumn()); - } - -private: - std::unordered_map column_source_to_columns; -}; - -/** Visitor that rewrites IN and JOINs in query and all subqueries according to distributed_product_mode and - * prefer_global_in_and_join settings. - * - * Additionally collects GLOBAL JOIN and GLOBAL IN query nodes. - * - * If distributed_product_mode = deny, then visitor throws exception if there are multiple distributed tables. - * If distributed_product_mode = local, then visitor collects replacement map for tables that must be replaced - * with local tables. - * If distributed_product_mode = global or prefer_global_in_and_join setting is true, then visitor rewrites JOINs and IN functions that - * contain distributed tables to GLOBAL JOINs and GLOBAL IN functions. - * If distributed_product_mode = allow, then visitor does not rewrite query if there are multiple distributed tables. - */ -class DistributedProductModeRewriteInJoinVisitor : public InDepthQueryTreeVisitorWithContext -{ -public: - using Base = InDepthQueryTreeVisitorWithContext; - using Base::Base; - - explicit DistributedProductModeRewriteInJoinVisitor(const ContextPtr & context_) - : Base(context_) - {} - - struct InFunctionOrJoin - { - QueryTreeNodePtr query_node; - size_t subquery_depth = 0; - }; - - const std::unordered_map & getReplacementMap() const - { - return replacement_map; - } - - const std::vector & getGlobalInOrJoinNodes() const - { - return global_in_or_join_nodes; - } - - static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child) - { - auto * function_node = parent->as(); - if (function_node && isNameOfGlobalInFunction(function_node->getFunctionName())) - return false; - - auto * join_node = parent->as(); - if (join_node && join_node->getLocality() == JoinLocality::Global && join_node->getRightTableExpression() == child) - return false; - - return true; - } - - void visitImpl(QueryTreeNodePtr & node) - { - auto * function_node = node->as(); - auto * join_node = node->as(); - - if ((function_node && isNameOfGlobalInFunction(function_node->getFunctionName())) || - (join_node && join_node->getLocality() == JoinLocality::Global)) - { - InFunctionOrJoin in_function_or_join_entry; - in_function_or_join_entry.query_node = node; - in_function_or_join_entry.subquery_depth = getSubqueryDepth(); - global_in_or_join_nodes.push_back(std::move(in_function_or_join_entry)); - return; - } - - if ((function_node && isNameOfLocalInFunction(function_node->getFunctionName())) || - (join_node && join_node->getLocality() != JoinLocality::Global)) - { - InFunctionOrJoin in_function_or_join_entry; - in_function_or_join_entry.query_node = node; - in_function_or_join_entry.subquery_depth = getSubqueryDepth(); - in_function_or_join_stack.push_back(in_function_or_join_entry); - return; - } - - if (node->getNodeType() == QueryTreeNodeType::TABLE) - tryRewriteTableNodeIfNeeded(node); - } - - void leaveImpl(QueryTreeNodePtr & node) - { - if (!in_function_or_join_stack.empty() && node.get() == in_function_or_join_stack.back().query_node.get()) - in_function_or_join_stack.pop_back(); - } - -private: - void tryRewriteTableNodeIfNeeded(const QueryTreeNodePtr & table_node) - { - const auto & table_node_typed = table_node->as(); - const auto * distributed_storage = typeid_cast(table_node_typed.getStorage().get()); - if (!distributed_storage) - return; - - bool distributed_valid_for_rewrite = distributed_storage->getShardCount() >= 2; - if (!distributed_valid_for_rewrite) - return; - - auto distributed_product_mode = getSettings().distributed_product_mode; - - if (distributed_product_mode == DistributedProductMode::LOCAL) - { - StorageID remote_storage_id = StorageID{distributed_storage->getRemoteDatabaseName(), - distributed_storage->getRemoteTableName()}; - auto resolved_remote_storage_id = getContext()->resolveStorageID(remote_storage_id); - const auto & distributed_storage_columns = table_node_typed.getStorageSnapshot()->metadata->getColumns(); - auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_columns); - auto replacement_table_expression = std::make_shared(std::move(storage), getContext()); - replacement_map.emplace(table_node.get(), std::move(replacement_table_expression)); - } - else if ((distributed_product_mode == DistributedProductMode::GLOBAL || getSettings().prefer_global_in_and_join) && - !in_function_or_join_stack.empty()) - { - auto * in_or_join_node_to_modify = in_function_or_join_stack.back().query_node.get(); - - if (auto * in_function_to_modify = in_or_join_node_to_modify->as()) - { - auto global_in_function_name = getGlobalInFunctionNameForLocalInFunctionName(in_function_to_modify->getFunctionName()); - auto global_in_function_resolver = FunctionFactory::instance().get(global_in_function_name, getContext()); - in_function_to_modify->resolveAsFunction(global_in_function_resolver->build(in_function_to_modify->getArgumentColumns())); - } - else if (auto * join_node_to_modify = in_or_join_node_to_modify->as()) - { - join_node_to_modify->setLocality(JoinLocality::Global); - } - - global_in_or_join_nodes.push_back(in_function_or_join_stack.back()); - } - else if (distributed_product_mode == DistributedProductMode::ALLOW) - { - return; - } - else if (distributed_product_mode == DistributedProductMode::DENY) - { - throw Exception(ErrorCodes::DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED, - "Double-distributed IN/JOIN subqueries is denied (distributed_product_mode = 'deny'). " - "You may rewrite query to use local tables " - "in subqueries, or use GLOBAL keyword, or set distributed_product_mode to suitable value."); - } - } - - std::vector in_function_or_join_stack; - std::unordered_map replacement_map; - std::vector global_in_or_join_nodes; -}; - -/** Execute subquery node and put result in mutable context temporary table. - * Returns table node that is initialized with temporary table storage. - */ -TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, - ContextMutablePtr & mutable_context, - size_t subquery_depth) -{ - auto subquery_hash = subquery_node->getTreeHash(); - String temporary_table_name = fmt::format("_data_{}_{}", subquery_hash.first, subquery_hash.second); - - const auto & external_tables = mutable_context->getExternalTables(); - auto external_table_it = external_tables.find(temporary_table_name); - if (external_table_it != external_tables.end()) - { - auto temporary_table_expression_node = std::make_shared(external_table_it->second, mutable_context); - temporary_table_expression_node->setTemporaryTableName(temporary_table_name); - return temporary_table_expression_node; - } - - auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth, true /*is_subquery*/); - auto context_copy = Context::createCopy(mutable_context); - updateContextForSubqueryExecution(context_copy); - - InterpreterSelectQueryAnalyzer interpreter(subquery_node, context_copy, subquery_options); - auto & query_plan = interpreter.getQueryPlan(); - - auto sample_block_with_unique_names = query_plan.getCurrentDataStream().header; - makeUniqueColumnNamesInBlock(sample_block_with_unique_names); - - if (!blocksHaveEqualStructure(sample_block_with_unique_names, query_plan.getCurrentDataStream().header)) - { - auto actions_dag = ActionsDAG::makeConvertingActions( - query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), - sample_block_with_unique_names.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Position); - auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(actions_dag)); - query_plan.addStep(std::move(converting_step)); - } - - Block sample = interpreter.getSampleBlock(); - NamesAndTypesList columns = sample.getNamesAndTypesList(); - - auto external_storage_holder = TemporaryTableHolder( - mutable_context, - ColumnsDescription{columns}, - ConstraintsDescription{}, - nullptr /*query*/, - true /*create_for_global_subquery*/); - - StoragePtr external_storage = external_storage_holder.getTable(); - auto temporary_table_expression_node = std::make_shared(external_storage, mutable_context); - temporary_table_expression_node->setTemporaryTableName(temporary_table_name); - - auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context, /*async_insert=*/false); - auto io = interpreter.execute(); - io.pipeline.complete(std::move(table_out)); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); - - mutable_context->addExternalTable(temporary_table_name, std::move(external_storage_holder)); - - return temporary_table_expression_node; -} - QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const StorageSnapshotPtr & distributed_storage_snapshot, const StorageID & remote_storage_id, @@ -963,81 +706,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression)); - CollectColumnSourceToColumnsVisitor collect_column_source_to_columns_visitor; - collect_column_source_to_columns_visitor.visit(query_tree_to_modify); - - const auto & column_source_to_columns = collect_column_source_to_columns_visitor.getColumnSourceToColumns(); - - DistributedProductModeRewriteInJoinVisitor visitor(query_info.planner_context->getQueryContext()); - visitor.visit(query_tree_to_modify); - - auto replacement_map = visitor.getReplacementMap(); - const auto & global_in_or_join_nodes = visitor.getGlobalInOrJoinNodes(); - - for (const auto & global_in_or_join_node : global_in_or_join_nodes) - { - if (auto * join_node = global_in_or_join_node.query_node->as()) - { - auto join_right_table_expression = join_node->getRightTableExpression(); - auto join_right_table_expression_node_type = join_right_table_expression->getNodeType(); - - QueryTreeNodePtr subquery_node; - - if (join_right_table_expression_node_type == QueryTreeNodeType::QUERY || - join_right_table_expression_node_type == QueryTreeNodeType::UNION) - { - subquery_node = join_right_table_expression; - } - else if (join_right_table_expression_node_type == QueryTreeNodeType::TABLE || - join_right_table_expression_node_type == QueryTreeNodeType::TABLE_FUNCTION) - { - const auto & columns = column_source_to_columns.at(join_right_table_expression).columns; - subquery_node = buildSubqueryToReadColumnsFromTableExpression(columns, - join_right_table_expression, - planner_context->getQueryContext()); - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected JOIN right table expression to be table, table function, query or union node. Actual {}", - join_right_table_expression->formatASTForErrorMessage()); - } - - auto temporary_table_expression_node = executeSubqueryNode(subquery_node, - planner_context->getMutableQueryContext(), - global_in_or_join_node.subquery_depth); - temporary_table_expression_node->setAlias(join_right_table_expression->getAlias()); - - replacement_map.emplace(join_right_table_expression.get(), std::move(temporary_table_expression_node)); - continue; - } - else if (auto * in_function_node = global_in_or_join_node.query_node->as()) - { - auto & in_function_subquery_node = in_function_node->getArguments().getNodes().at(1); - auto in_function_node_type = in_function_subquery_node->getNodeType(); - if (in_function_node_type != QueryTreeNodeType::QUERY && in_function_node_type != QueryTreeNodeType::UNION) - continue; - - auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, - planner_context->getMutableQueryContext(), - global_in_or_join_node.subquery_depth); - - in_function_subquery_node = std::move(temporary_table_expression_node); - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected global IN or JOIN query node. Actual {}", - global_in_or_join_node.query_node->formatASTForErrorMessage()); - } - } - - if (!replacement_map.empty()) - query_tree_to_modify = query_tree_to_modify->cloneAndReplace(replacement_map); - - removeGroupingFunctionSpecializations(query_tree_to_modify); - - return query_tree_to_modify; + return buildQueryTreeForShard(query_info, query_tree_to_modify); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 61d1442df92..fafb3b124f2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -21,6 +21,7 @@ #include +#include #include #include #include @@ -74,6 +75,8 @@ #include #include +#include + #include #include #include @@ -4734,20 +4737,27 @@ void StorageReplicatedMergeTree::read( { auto table_id = getStorageID(); - const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery( - local_context, query_info.query, - table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - - auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); + ASTPtr modified_query_ast; Block header; if (local_context->getSettingsRef().allow_experimental_analyzer) + { + auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree); + header = InterpreterSelectQueryAnalyzer::getSampleBlock( - modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()); + modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); + modified_query_ast = queryNodeToSelectQuery(modified_query_tree); + } else + { header = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, + table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); + } + + auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory( diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp new file mode 100644 index 00000000000..a42d67d9aa7 --- /dev/null +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -0,0 +1,372 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED; +} + +namespace +{ + +/// Visitor that collect column source to columns mapping from query and all subqueries +class CollectColumnSourceToColumnsVisitor : public InDepthQueryTreeVisitor +{ +public: + struct Columns + { + NameSet column_names; + NamesAndTypes columns; + + void addColumn(NameAndTypePair column) + { + if (column_names.contains(column.name)) + return; + + column_names.insert(column.name); + columns.push_back(std::move(column)); + } + }; + + const std::unordered_map & getColumnSourceToColumns() const + { + return column_source_to_columns; + } + + void visitImpl(QueryTreeNodePtr & node) + { + auto * column_node = node->as(); + if (!column_node) + return; + + auto column_source = column_node->getColumnSourceOrNull(); + if (!column_source) + return; + + auto it = column_source_to_columns.find(column_source); + if (it == column_source_to_columns.end()) + { + auto [insert_it, _] = column_source_to_columns.emplace(column_source, Columns()); + it = insert_it; + } + + it->second.addColumn(column_node->getColumn()); + } + +private: + std::unordered_map column_source_to_columns; +}; + +/** Visitor that rewrites IN and JOINs in query and all subqueries according to distributed_product_mode and + * prefer_global_in_and_join settings. + * + * Additionally collects GLOBAL JOIN and GLOBAL IN query nodes. + * + * If distributed_product_mode = deny, then visitor throws exception if there are multiple distributed tables. + * If distributed_product_mode = local, then visitor collects replacement map for tables that must be replaced + * with local tables. + * If distributed_product_mode = global or prefer_global_in_and_join setting is true, then visitor rewrites JOINs and IN functions that + * contain distributed tables to GLOBAL JOINs and GLOBAL IN functions. + * If distributed_product_mode = allow, then visitor does not rewrite query if there are multiple distributed tables. + */ +class DistributedProductModeRewriteInJoinVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + explicit DistributedProductModeRewriteInJoinVisitor(const ContextPtr & context_) + : Base(context_) + {} + + struct InFunctionOrJoin + { + QueryTreeNodePtr query_node; + size_t subquery_depth = 0; + }; + + const std::unordered_map & getReplacementMap() const + { + return replacement_map; + } + + const std::vector & getGlobalInOrJoinNodes() const + { + return global_in_or_join_nodes; + } + + static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child) + { + auto * function_node = parent->as(); + if (function_node && isNameOfGlobalInFunction(function_node->getFunctionName())) + return false; + + auto * join_node = parent->as(); + if (join_node && join_node->getLocality() == JoinLocality::Global && join_node->getRightTableExpression() == child) + return false; + + return true; + } + + void visitImpl(QueryTreeNodePtr & node) + { + auto * function_node = node->as(); + auto * join_node = node->as(); + + if ((function_node && isNameOfGlobalInFunction(function_node->getFunctionName())) || + (join_node && join_node->getLocality() == JoinLocality::Global)) + { + InFunctionOrJoin in_function_or_join_entry; + in_function_or_join_entry.query_node = node; + in_function_or_join_entry.subquery_depth = getSubqueryDepth(); + global_in_or_join_nodes.push_back(std::move(in_function_or_join_entry)); + return; + } + + if ((function_node && isNameOfLocalInFunction(function_node->getFunctionName())) || + (join_node && join_node->getLocality() != JoinLocality::Global)) + { + InFunctionOrJoin in_function_or_join_entry; + in_function_or_join_entry.query_node = node; + in_function_or_join_entry.subquery_depth = getSubqueryDepth(); + in_function_or_join_stack.push_back(in_function_or_join_entry); + return; + } + + if (node->getNodeType() == QueryTreeNodeType::TABLE) + tryRewriteTableNodeIfNeeded(node); + } + + void leaveImpl(QueryTreeNodePtr & node) + { + if (!in_function_or_join_stack.empty() && node.get() == in_function_or_join_stack.back().query_node.get()) + in_function_or_join_stack.pop_back(); + } + +private: + void tryRewriteTableNodeIfNeeded(const QueryTreeNodePtr & table_node) + { + const auto & table_node_typed = table_node->as(); + const auto * distributed_storage = typeid_cast(table_node_typed.getStorage().get()); + if (!distributed_storage) + return; + + bool distributed_valid_for_rewrite = distributed_storage->getShardCount() >= 2; + if (!distributed_valid_for_rewrite) + return; + + auto distributed_product_mode = getSettings().distributed_product_mode; + + if (distributed_product_mode == DistributedProductMode::LOCAL) + { + StorageID remote_storage_id = StorageID{distributed_storage->getRemoteDatabaseName(), + distributed_storage->getRemoteTableName()}; + auto resolved_remote_storage_id = getContext()->resolveStorageID(remote_storage_id); + const auto & distributed_storage_columns = table_node_typed.getStorageSnapshot()->metadata->getColumns(); + auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_columns); + auto replacement_table_expression = std::make_shared(std::move(storage), getContext()); + replacement_map.emplace(table_node.get(), std::move(replacement_table_expression)); + } + else if ((distributed_product_mode == DistributedProductMode::GLOBAL || getSettings().prefer_global_in_and_join) && + !in_function_or_join_stack.empty()) + { + auto * in_or_join_node_to_modify = in_function_or_join_stack.back().query_node.get(); + + if (auto * in_function_to_modify = in_or_join_node_to_modify->as()) + { + auto global_in_function_name = getGlobalInFunctionNameForLocalInFunctionName(in_function_to_modify->getFunctionName()); + auto global_in_function_resolver = FunctionFactory::instance().get(global_in_function_name, getContext()); + in_function_to_modify->resolveAsFunction(global_in_function_resolver->build(in_function_to_modify->getArgumentColumns())); + } + else if (auto * join_node_to_modify = in_or_join_node_to_modify->as()) + { + join_node_to_modify->setLocality(JoinLocality::Global); + } + + global_in_or_join_nodes.push_back(in_function_or_join_stack.back()); + } + else if (distributed_product_mode == DistributedProductMode::ALLOW) + { + return; + } + else if (distributed_product_mode == DistributedProductMode::DENY) + { + throw Exception(ErrorCodes::DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED, + "Double-distributed IN/JOIN subqueries is denied (distributed_product_mode = 'deny'). " + "You may rewrite query to use local tables " + "in subqueries, or use GLOBAL keyword, or set distributed_product_mode to suitable value."); + } + } + + std::vector in_function_or_join_stack; + std::unordered_map replacement_map; + std::vector global_in_or_join_nodes; +}; + +/** Execute subquery node and put result in mutable context temporary table. + * Returns table node that is initialized with temporary table storage. + */ +TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, + ContextMutablePtr & mutable_context, + size_t subquery_depth) +{ + auto subquery_hash = subquery_node->getTreeHash(); + String temporary_table_name = fmt::format("_data_{}_{}", subquery_hash.first, subquery_hash.second); + + const auto & external_tables = mutable_context->getExternalTables(); + auto external_table_it = external_tables.find(temporary_table_name); + if (external_table_it != external_tables.end()) + { + auto temporary_table_expression_node = std::make_shared(external_table_it->second, mutable_context); + temporary_table_expression_node->setTemporaryTableName(temporary_table_name); + return temporary_table_expression_node; + } + + auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth, true /*is_subquery*/); + auto context_copy = Context::createCopy(mutable_context); + updateContextForSubqueryExecution(context_copy); + + InterpreterSelectQueryAnalyzer interpreter(subquery_node, context_copy, subquery_options); + auto & query_plan = interpreter.getQueryPlan(); + + auto sample_block_with_unique_names = query_plan.getCurrentDataStream().header; + makeUniqueColumnNamesInBlock(sample_block_with_unique_names); + + if (!blocksHaveEqualStructure(sample_block_with_unique_names, query_plan.getCurrentDataStream().header)) + { + auto actions_dag = ActionsDAG::makeConvertingActions( + query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), + sample_block_with_unique_names.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(actions_dag)); + query_plan.addStep(std::move(converting_step)); + } + + Block sample = interpreter.getSampleBlock(); + NamesAndTypesList columns = sample.getNamesAndTypesList(); + + auto external_storage_holder = TemporaryTableHolder( + mutable_context, + ColumnsDescription{columns}, + ConstraintsDescription{}, + nullptr /*query*/, + true /*create_for_global_subquery*/); + + StoragePtr external_storage = external_storage_holder.getTable(); + auto temporary_table_expression_node = std::make_shared(external_storage, mutable_context); + temporary_table_expression_node->setTemporaryTableName(temporary_table_name); + + auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context, /*async_insert=*/false); + auto io = interpreter.execute(); + io.pipeline.complete(std::move(table_out)); + CompletedPipelineExecutor executor(io.pipeline); + executor.execute(); + + mutable_context->addExternalTable(temporary_table_name, std::move(external_storage_holder)); + + return temporary_table_expression_node; +} + +} + +QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeNodePtr query_tree_to_modify) +{ + auto & planner_context = query_info.planner_context; + const auto & query_context = planner_context->getQueryContext(); + + CollectColumnSourceToColumnsVisitor collect_column_source_to_columns_visitor; + collect_column_source_to_columns_visitor.visit(query_tree_to_modify); + + const auto & column_source_to_columns = collect_column_source_to_columns_visitor.getColumnSourceToColumns(); + + DistributedProductModeRewriteInJoinVisitor visitor(query_info.planner_context->getQueryContext()); + visitor.visit(query_tree_to_modify); + + auto replacement_map = visitor.getReplacementMap(); + const auto & global_in_or_join_nodes = visitor.getGlobalInOrJoinNodes(); + + for (const auto & global_in_or_join_node : global_in_or_join_nodes) + { + if (auto * join_node = global_in_or_join_node.query_node->as()) + { + auto join_right_table_expression = join_node->getRightTableExpression(); + auto join_right_table_expression_node_type = join_right_table_expression->getNodeType(); + + QueryTreeNodePtr subquery_node; + + if (join_right_table_expression_node_type == QueryTreeNodeType::QUERY || + join_right_table_expression_node_type == QueryTreeNodeType::UNION) + { + subquery_node = join_right_table_expression; + } + else if (join_right_table_expression_node_type == QueryTreeNodeType::TABLE || + join_right_table_expression_node_type == QueryTreeNodeType::TABLE_FUNCTION) + { + const auto & columns = column_source_to_columns.at(join_right_table_expression).columns; + subquery_node = buildSubqueryToReadColumnsFromTableExpression(columns, + join_right_table_expression, + planner_context->getQueryContext()); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected JOIN right table expression to be table, table function, query or union node. Actual {}", + join_right_table_expression->formatASTForErrorMessage()); + } + + auto temporary_table_expression_node = executeSubqueryNode(subquery_node, + planner_context->getMutableQueryContext(), + global_in_or_join_node.subquery_depth); + temporary_table_expression_node->setAlias(join_right_table_expression->getAlias()); + + replacement_map.emplace(join_right_table_expression.get(), std::move(temporary_table_expression_node)); + continue; + } + else if (auto * in_function_node = global_in_or_join_node.query_node->as()) + { + auto & in_function_subquery_node = in_function_node->getArguments().getNodes().at(1); + auto in_function_node_type = in_function_subquery_node->getNodeType(); + if (in_function_node_type != QueryTreeNodeType::QUERY && in_function_node_type != QueryTreeNodeType::UNION) + continue; + + auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, + planner_context->getMutableQueryContext(), + global_in_or_join_node.subquery_depth); + + in_function_subquery_node = std::move(temporary_table_expression_node); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected global IN or JOIN query node. Actual {}", + global_in_or_join_node.query_node->formatASTForErrorMessage()); + } + } + + if (!replacement_map.empty()) + query_tree_to_modify = query_tree_to_modify->cloneAndReplace(replacement_map); + + removeGroupingFunctionSpecializations(query_tree_to_modify); + + return query_tree_to_modify; +} + +} diff --git a/src/Storages/buildQueryTreeForShard.h b/src/Storages/buildQueryTreeForShard.h new file mode 100644 index 00000000000..05d63faeb9f --- /dev/null +++ b/src/Storages/buildQueryTreeForShard.h @@ -0,0 +1,15 @@ +#pragma once + +#include + +namespace DB +{ + +struct SelectQueryInfo; + +class IQueryTreeNode; +using QueryTreeNodePtr = std::shared_ptr; + +QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeNodePtr query_tree_to_modify); + +} diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference index 4e93c530f7b..f688db940d9 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 allow_experimental_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,\n use_hedged_requests = 0; -0 2 SELECT `default`.`join_inner_table__fuzz_146_replicated`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` +0 2 SELECT `join_inner_table__fuzz_146_replicated`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` SETTINGS allow_experimental_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\', allow_experimental_parallel_reading_from_replicas = 1, use_hedged_requests = 0 From d05f89f8f5ec3793256cae1557e2af60650290cf Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 12 Jun 2023 17:33:15 +0000 Subject: [PATCH 1524/2223] Fix style --- src/Storages/StorageDistributed.cpp | 1 - src/Storages/buildQueryTreeForShard.cpp | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1ec45ce3d57..0472ce6f832 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -154,7 +154,6 @@ namespace ErrorCodes extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; extern const int TOO_LARGE_DISTRIBUTED_DEPTH; - extern const int DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED; } namespace ActionLocks diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index a42d67d9aa7..fa4730cbe84 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -21,6 +21,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED; } From a4285d56b22aafe453309ce728c7380666626576 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 31 May 2023 12:37:46 -0700 Subject: [PATCH 1525/2223] Fix compilation error on big-endian platforms --- src/Functions/FunctionsCodingIP.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsCodingIP.cpp b/src/Functions/FunctionsCodingIP.cpp index 2671418fc7b..897b24d90e0 100644 --- a/src/Functions/FunctionsCodingIP.cpp +++ b/src/Functions/FunctionsCodingIP.cpp @@ -580,7 +580,7 @@ private: #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ unalignedStoreLittleEndian(buf + 8, 0x00000000FFFF0000ull | (static_cast(ntohl(in)) << 32)); #else - unalignedStoreLittleEndian(buf + 8, 0x00000000FFFF0000ull | (static_cast(__builtin_bswap32(in))) << 32)); + unalignedStoreLittleEndian(buf + 8, 0x00000000FFFF0000ull | (static_cast(std::byteswap(in))) << 32); #endif } }; From edb4a644b1ef9fdcba7f53c60ed37438d610ae9a Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 8 Jun 2023 10:21:24 -0400 Subject: [PATCH 1526/2223] Update FunctionsCodingIP.cpp --- src/Functions/FunctionsCodingIP.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsCodingIP.cpp b/src/Functions/FunctionsCodingIP.cpp index 897b24d90e0..7bdbac6531d 100644 --- a/src/Functions/FunctionsCodingIP.cpp +++ b/src/Functions/FunctionsCodingIP.cpp @@ -580,7 +580,7 @@ private: #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ unalignedStoreLittleEndian(buf + 8, 0x00000000FFFF0000ull | (static_cast(ntohl(in)) << 32)); #else - unalignedStoreLittleEndian(buf + 8, 0x00000000FFFF0000ull | (static_cast(std::byteswap(in))) << 32); + unalignedStoreLittleEndian(buf + 8, 0x00000000FFFF0000ull | (static_cast(std::byteswap(in)) << 32)); #endif } }; From 70252321750b6a8d3ab6c41f658b76705a2e55b9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Jun 2023 18:59:07 +0000 Subject: [PATCH 1527/2223] Fixing cache for sets. --- src/Processors/Transforms/CreatingSetsTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index a1e43525ab1..3139fa5ed98 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -86,7 +86,7 @@ void CreatingSetsTransform::startSubquery() } } - if (subquery.set) + if (subquery.set && !set_from_cache) LOG_TRACE(log, "Creating set, key: {}", subquery.key); if (subquery.table) LOG_TRACE(log, "Filling temporary table."); @@ -97,7 +97,7 @@ void CreatingSetsTransform::startSubquery() /// TODO: make via port table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), nullptr, /*async_insert=*/false)); - done_with_set = !subquery.set; + done_with_set = !subquery.set || set_from_cache; done_with_table = !subquery.table; if ((done_with_set && !set_from_cache) /*&& done_with_join*/ && done_with_table) @@ -175,10 +175,10 @@ void CreatingSetsTransform::consume(Chunk chunk) Chunk CreatingSetsTransform::generate() { - if (subquery.set) + if (subquery.set && !set_from_cache) { subquery.set->finishInsert(); - subquery.promise_to_fill_set.set_value(subquery.set); + //subquery.promise_to_fill_set.set_value(subquery.set); if (promise_to_build) promise_to_build->set_value(subquery.set); } From e6e8576864421bd2db043d269dfe65d1bf4f85aa Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 12 Jun 2023 17:04:33 -0300 Subject: [PATCH 1528/2223] Update mergetree.md --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 61276110138..dbde1a90f67 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1138,7 +1138,7 @@ These parameters define the cache layer: Cache parameters: - `path` — The path where metadata for the cache is stored. -- `max_size` — The size (amount of memory) that the cache can grow to. +- `max_size` — The size (amount of disk space) that the cache can grow to. :::tip There are several other cache parameters that you can use to tune your storage, see [using local cache](/docs/en/operations/storing-data.md/#using-local-cache) for the details. From 4f39ee51ae867b219735290125f8dc91d461abf6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 12 Jun 2023 20:06:57 +0000 Subject: [PATCH 1529/2223] Update Annoy docs --- .../mergetree-family/annindexes.md | 20 +++++++++---------- src/Parsers/ASTIndexDeclaration.h | 3 +++ src/Parsers/ParserCreateIndexQuery.cpp | 4 ++-- src/Parsers/ParserCreateQuery.cpp | 4 ++-- 4 files changed, 17 insertions(+), 14 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 2b0b77a0735..16e244077a7 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -100,7 +100,7 @@ ANN indexes support two types of queries: :::tip To avoid writing out large vectors, you can use [query -parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. +parameters](/docs/en/interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. ```bash clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(vectors, {vec: Array(Float32)}) < 1.0" @@ -128,14 +128,14 @@ granularity of granules, sub-indexes extrapolate matching rows to granule granul skip data at the granularity of index blocks. The `GRANULARITY` parameter determines how many ANN sub-indexes are created. Bigger `GRANULARITY` values mean fewer but larger ANN -sub-indexes, up to the point where a column (or a column part) has only a single sub-index. In that case, the sub-index has a "global" view of -all column rows and can directly return all granules of the column (part) with relevant rows (there are at at most `LIMIT `-many -such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a brute-force distance -calculation over all rows of the granules. With a small `GRANULARITY` value, each of the sub-indexes returns up to `LIMIT N`-many granules. -As a result, more granules need to be loaded and post-filtered. Note that the search accuracy is with both cases equally good, only the -processing performance differs. It is generally recommended to use a large `GRANULARITY` for ANN indexes and fall back to a smaller -`GRANULARITY` values only in case of problems like excessive memory consumption of the ANN structures. If no `GRANULARITY` was specified for -ANN indexes, the default value is 100 million. +sub-indexes, up to the point where a column (or a column's data part) has only a single sub-index. In that case, the sub-index has a +"global" view of all column rows and can directly return all granules of the column (part) with relevant rows (there are at most `LIMIT +`-many such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a +brute-force distance calculation over all rows of the granules. With a small `GRANULARITY` value, each of the sub-indexes returns up to +`LIMIT N`-many granules. As a result, more granules need to be loaded and post-filtered. Note that the search accuracy is with both cases +equally good, only the processing performance differs. It is generally recommended to use a large `GRANULARITY` for ANN indexes and fall +back to a smaller `GRANULARITY` values only in case of problems like excessive memory consumption of the ANN structures. If no `GRANULARITY` +was specified for ANN indexes, the default value is 100 million. # Available ANN Indexes @@ -204,7 +204,7 @@ values mean more accurate results at the cost of longer query runtime: ``` sql SELECT * -FROM table_name [WHERE ...] +FROM table_name ORDER BY L2Distance(vectors, Point) LIMIT N SETTINGS annoy_index_search_k_nodes=100 diff --git a/src/Parsers/ASTIndexDeclaration.h b/src/Parsers/ASTIndexDeclaration.h index bd52a611f3f..6ed241f75ab 100644 --- a/src/Parsers/ASTIndexDeclaration.h +++ b/src/Parsers/ASTIndexDeclaration.h @@ -12,6 +12,9 @@ class ASTFunction; class ASTIndexDeclaration : public IAST { public: + static const auto DEFAULT_INDEX_GRANULARITY = 1uz; + static const auto DEFAULT_ANNOY_INDEX_GRANULARITY = 100'000'000uz; + String name; IAST * expr; ASTFunction * type; diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 57afd3fb99e..f231573b920 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -52,9 +52,9 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected else { if (index->type->name == "annoy") - index->granularity = 100'000'000; + index->granularity = ASTIndexDeclaration::DEFAULT_ANNOY_INDEX_GRANULARITY; else - index->granularity = 1; + index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY; } node = index; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index c6273f369b1..adf3513ba40 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -147,9 +147,9 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe else { if (index->type->name == "annoy") - index->granularity = 100'000'000; + index->granularity = ASTIndexDeclaration::DEFAULT_ANNOY_INDEX_GRANULARITY; else - index->granularity = 1; + index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY; } node = index; From 002c15823c26c3e0c577a4dd8ec8f319b3120a78 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 12 Jun 2023 16:44:46 -0700 Subject: [PATCH 1530/2223] Perform in-place endianness transform because of padding --- src/AggregateFunctions/ReservoirSamplerDeterministic.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/src/AggregateFunctions/ReservoirSamplerDeterministic.h index b1a39a5dcc5..25d3b182654 100644 --- a/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -190,12 +190,12 @@ public: /// Here we ensure that padding is zero without changing the protocol. /// TODO: After implementation of "versioning aggregate function state", /// change the serialization format. - Element elem; memset(&elem, 0, sizeof(elem)); elem = samples[i]; - writeBinaryLittleEndian(elem, buf); + DB::transformEndianness(elem); + DB::writeString(reinterpret_cast(&elem), sizeof(elem), buf); } } From 959fde4491e33586916efcf689ba1a4b361e6865 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 13 Jun 2023 09:33:38 +0800 Subject: [PATCH 1531/2223] add notifications in docs --- docs/en/engines/table-engines/integrations/redis.md | 9 ++++++++- src/Storages/StorageRedis.cpp | 7 +++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/redis.md b/docs/en/engines/table-engines/integrations/redis.md index 6cfc60c836c..68235a89d33 100644 --- a/docs/en/engines/table-engines/integrations/redis.md +++ b/docs/en/engines/table-engines/integrations/redis.md @@ -6,7 +6,7 @@ sidebar_label: Redis # Redis -This engine allows integrating ClickHouse with [Redis](https://redis.io/). +This engine allows integrating ClickHouse with [Redis](https://redis.io/). For Redis takes kv model, we strongly recommend you only query it in a point way, such as `where k=xx` or `where k in (xx, xx)`. ## Creating a Table {#creating-a-table} @@ -110,3 +110,10 @@ Flush Redis db asynchronously. Also `Truncate` support SYNC mode. ```sql TRUNCATE TABLE redis_table SYNC; ``` + + +## Limitations {#limitations} + +Redis engine also support scanning query, such as `where k > xx`, but it has some limitations: +1. Scanning query may produce some duplicated keys in a very rare case when it is rehashing, details see [Redis Scan](https://github.com/redis/redis/blob/e4d183afd33e0b2e6e8d1c79a832f678a04a7886/src/dict.c#L1186-L1269) +2. During the scanning keys could be created and deleted, so the resulting dataset can not represent a valid point in time. diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 71c84443d8e..ddb1b62c7b0 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -3,12 +3,12 @@ #include #include #include +#include #include #include #include #include #include -#include #include #include @@ -19,6 +19,7 @@ #include #include +#include #include #include @@ -87,9 +88,11 @@ public: return storage.getBySerializedKeys(raw_keys, nullptr); } - /// TODO scan may get duplicated keys + /// TODO scan may get duplicated keys when Redis is rehashing, it is a very rare case. Chunk generateFullScan() { + checkStackSize(); + /// redis scan ending if (iterator == 0) return {}; From 2395b25f9e8828ea1adbf6d303832a1ea7ee97a8 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 13 Jun 2023 01:55:34 +0000 Subject: [PATCH 1532/2223] Changes after review --- docs/en/interfaces/cli.md | 18 +++++++++++------- docs/ru/interfaces/cli.md | 14 +++++++++----- src/Client/ConnectionString.cpp | 6 +++--- .../02784_connection_string.reference | 1 + .../0_stateless/02784_connection_string.sh | 3 +++ 5 files changed, 27 insertions(+), 15 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index fc24bdcad68..6736d05e65f 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -202,15 +202,16 @@ Instead of --host, --port, --user and --password options, ClickHouse client also clickhouse-client alternatively supports connecting to clickhouse server using a connection string similar to [MongoDB](https://www.mongodb.com/docs/manual/reference/connection-string/), [PostgreSQL](https://www.postgresql.org/docs/current/libpq-connect.html#LIBPQ-CONNSTRING), [MySQL](https://dev.mysql.com/doc/refman/8.0/en/connecting-using-uri-or-key-value-pairs.html#connecting-using-uri). It has the following syntax: ```text -clickhouse:[//[user_info@][hosts_and_ports]][/database][?query_parameters] +clickhouse:[//[user[:password]@][hosts_and_ports]][/database][?query_parameters] ``` Where -- `user_spec` - (optional) is a user and an optional password, -- `hostspec` - (optional) is a list of hosts and optional ports `host[:port] [, host:[port]], ...`, +- `user` - (optional) is a user name, +- `password` - (optional) is a user password. If `:` is specified and the password is blank, the client will prompt for the user's password. +- `hosts_and_ports` - (optional) is a list of hosts and optional ports `host[:port] [, host:[port]], ...`, - `database` - (optional) is the database name, -- `paramspec` - (optional) is a list of key-value pairs `param1=value1[,¶m2=value2], ...`. For some parameters, no value is required. Parameter names and values are case-sensitive. +- `query_parameters` - (optional) is a list of key-value pairs `param1=value1[,¶m2=value2], ...`. For some parameters, no value is required. Parameter names and values are case-sensitive. @@ -239,7 +240,7 @@ URI allows multiple hosts to be connected to. Connection strings can contain mul ### Percent encoding {#connection_string_uri_percent_encoding} -Non-US ASCII characters in the user name, password, hosts, database or query parameters must be [percent-encoded](https://en.wikipedia.org/wiki/URL_encoding). +Non-US ASCII, spaces and special characters, and special characters in the `user`, `password`, `hosts`, `database` and `query parameters` must be [percent-encoded](https://en.wikipedia.org/wiki/URL_encoding). ### Examples {#connection_string_examples} @@ -306,10 +307,13 @@ Connect to default host using default port, default user, and default database. clickhouse-client clickhouse: ``` -Connect to the default host using the default port, using user user_name and no password. +Connect to the default host using the default port, using user `my_user` and no password. ``` bash -clickhouse-client clickhouse://user_name@ +clickhouse-client clickhouse://my_user@ + +# Using a blank password between : and @ means to asking user to enter the password before starting the connection. +clickhouse-client clickhouse://my_user:@ ``` Connect to localhost using email as the user name. `@` symbol is percent encoded to `%40`. diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index ee29b122afb..794ac60ec83 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -147,12 +147,13 @@ $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="numbe clickhouse-client также поддерживает подключение к серверу clickhouse с помощью строки подключения, аналогичной [MongoDB](https://www.mongodb.com/docs/manual/reference/connection-string/), [PostgreSQL](https://www.postgresql.org/docs/current/libpq-connect.html#LIBPQ-CONNSTRING), [MySQL](https://dev.mysql.com/doc/refman/8.0/en/connecting-using-uri-or-key-value-pairs.html#connecting-using-uri). Она имеет следующий синтаксис: ```text -clickhouse:[//[user_info@][hosts_and_ports]][/database][?query_parameters] +clickhouse:[//[user[:password]@][hosts_and_ports]][/database][?query_parameters] ``` Где -- `user_spec` - (необязательно) - это пользователь и необязательный пароль, +- `user` - (необязательно) - это имя пользователя, +- `password` - (необязательно) - Пароль пользователя. Если символ `:` укаказан, и пароль пуст, то клиент запросит ввести пользователя пароль. - `hostspec` - (необязательно) - список хостов и необязательных портов. `host[:port] [, host:[port]], ...`, - `database` - (необязательно) - это имя базы данных, - `paramspec` - (опционально) список пар ключ-значение `param1=value1[,¶m2=value2], ...`. Для некоторых параметров значение не требуется. Имена и значения параметров чувствительны к регистру. @@ -182,7 +183,7 @@ URI позволяет подключаться к нескольким хост ### Кодирование URI {#connection_string_uri_percent_encoding} -Не US ASCII символы в имени пользователя, пароле, хостах, базе данных или параметрах запроса должны быть [закодированы](https://ru.wikipedia.org/wiki/URL#%D0%9A%D0%BE%D0%B4%D0%B8%D1%80%D0%BE%D0%B2%D0%B0%D0%BD%D0%B8%D0%B5_URL). +Не US ASCII и специальные символы в имени пользователя, пароле, хостах, базе данных и параметрах запроса должны быть [закодированы](https://ru.wikipedia.org/wiki/URL#%D0%9A%D0%BE%D0%B4%D0%B8%D1%80%D0%BE%D0%B2%D0%B0%D0%BD%D0%B8%D0%B5_URL). ### Примеры {#connection_string_examples} @@ -248,10 +249,13 @@ clickhouse-client clickhouse://localhost/my_database -s clickhouse-client clickhouse: ``` -Подключиться к хосту по умолчанию через порт по умолчанию, используя имя пользователя user_name без пароля. +Подключиться к хосту по умолчанию через порт по умолчанию, используя имя пользователя `my_user` без пароля. ``` bash -clickhouse-client clickhouse://user_name@ +clickhouse-client clickhouse://my_user@ + +# Использование пустого пароля между : и @ означает, что пользователь должен ввести пароль перед началом соединения. +clickhouse-client clickhouse://my_user:@ ``` Подключиться к localhost, используя электронную почту, как имя пользователя. Символ `@` закодирован как `%40`. diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp index e1f39369b2a..8f0a0980f51 100644 --- a/src/Client/ConnectionString.cpp +++ b/src/Client/ConnectionString.cpp @@ -142,11 +142,11 @@ bool tryParseConnectionString( try { /** Poco::URI doesn't support several hosts in URI. - * Split string clickhouse:[user_info]host1:port1, ... , hostN:portN[database]?[query_parameters] + * Split string clickhouse:[user[:password]@]host1:port1, ... , hostN:portN[database]?[query_parameters] * into multiple string for each host: - * clickhouse:[user_info]host1:port1[database]?[query_parameters] + * clickhouse:[user[:password]@]host1:port1[database]?[query_parameters] * ... - * clickhouse:[user_info]hostN:portN[database]?[query_parameters] + * clickhouse:[user[:password]@]hostN:portN[database]?[query_parameters] */ Poco::URI uri; const auto * last_host_begin = connection_string.begin() + offset; diff --git a/tests/queries/0_stateless/02784_connection_string.reference b/tests/queries/0_stateless/02784_connection_string.reference index 6a36abae8e0..9d58d485a14 100644 --- a/tests/queries/0_stateless/02784_connection_string.reference +++ b/tests/queries/0_stateless/02784_connection_string.reference @@ -121,5 +121,6 @@ BAD_ARGUMENTS BAD_ARGUMENTS BAD_ARGUMENTS BAD_ARGUMENTS +BAD_ARGUMENTS Authentication failed Authentication failed diff --git a/tests/queries/0_stateless/02784_connection_string.sh b/tests/queries/0_stateless/02784_connection_string.sh index fce93fdad74..042f5b2108d 100755 --- a/tests/queries/0_stateless/02784_connection_string.sh +++ b/tests/queries/0_stateless/02784_connection_string.sh @@ -116,6 +116,9 @@ runClient "clickhouse:///?" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HO runClient "clickhouse://:/?" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse:" --database "$CLICKHOUSE_DATABASE" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' +# Using clickhouse-client and connection is prohibited +runClient "clickhouse:" --connection "connection" 2>&1 | grep -o 'BAD_ARGUMENTS' + # Space is used in connection string (This is prohibited). runClient " clickhouse:" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse: " 2>&1 | grep -o 'BAD_ARGUMENTS' From 6839a1318c8a656c20e7a1ed8e256fd51408820e Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 13 Jun 2023 04:03:30 +0000 Subject: [PATCH 1533/2223] minor changes in docs --- docs/en/interfaces/cli.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 6736d05e65f..b5134ea30c0 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -240,7 +240,7 @@ URI allows multiple hosts to be connected to. Connection strings can contain mul ### Percent encoding {#connection_string_uri_percent_encoding} -Non-US ASCII, spaces and special characters, and special characters in the `user`, `password`, `hosts`, `database` and `query parameters` must be [percent-encoded](https://en.wikipedia.org/wiki/URL_encoding). +Non-US ASCII, spaces and special characters in the `user`, `password`, `hosts`, `database` and `query parameters` must be [percent-encoded](https://en.wikipedia.org/wiki/URL_encoding). ### Examples {#connection_string_examples} From a3ff5df205ae8353395023ee4ef0bf83bee31458 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 13 Jun 2023 09:16:06 +0300 Subject: [PATCH 1534/2223] Remove reduntant header SELECT from the test --- .../0_stateless/02783_parseDateTimeBestEffort_syslog.reference | 1 - .../0_stateless/02783_parseDateTimeBestEffort_syslog.sql | 2 -- 2 files changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference index 3ec93143e0e..ee75d68bff4 100644 --- a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference +++ b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference @@ -1,4 +1,3 @@ -parseDateTimeBestEffort around_June_7 res res_sam res_auc res_null res_null_sam res_null_auc res_zero res_zero_sam res_zero_auc res_us res_us_sam res_us_auc res_us_null res_us_null_sam res_us_null_auc res_us_zero res_us_zero_sam res_us_zero_auc res64 res64_sam res64_auc res64_null res64_null_sam res64_null_auc res64_zero res64_zero_sam res64_zero_auc res64_us res64_us_sam res64_us_auc res64_us_null res64_us_null_sam res64_us_null_auc res64_us_zero res64_us_zero_sam res64_us_zero_auc Jun 6 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 2023-06-06 00:00:00.000 diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql index 52975cb5bbf..742ae03ddab 100644 --- a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql +++ b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql @@ -1,5 +1,3 @@ -SELECT 'parseDateTimeBestEffort'; - WITH 86400 AS secs_in_day, now() AS ts_now, From 879db5098a80d8c8c391296b672332a3367f6ac9 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Mon, 12 Jun 2023 13:47:38 +0200 Subject: [PATCH 1535/2223] MaterializedMySQL: Add test_named_collections --- .../configs/users.xml | 1 + .../materialize_with_ddl.py | 31 +++++++++++++++++++ .../test_materialized_mysql_database/test.py | 6 ++++ 3 files changed, 38 insertions(+) diff --git a/tests/integration/test_materialized_mysql_database/configs/users.xml b/tests/integration/test_materialized_mysql_database/configs/users.xml index 3669fbb46ba..7a7529c94bb 100644 --- a/tests/integration/test_materialized_mysql_database/configs/users.xml +++ b/tests/integration/test_materialized_mysql_database/configs/users.xml @@ -14,6 +14,7 @@ ::/0 default + 1 diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 2bbbe9a3f13..7fdb73ea1f3 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -2265,3 +2265,34 @@ def dropddl(clickhouse_node, mysql_node, mysql_host): ) mysql_node.query(f"DROP DATABASE {db}") clickhouse_node.query(f"DROP DATABASE {db}") + + +def named_collections(clickhouse_node, mysql_node, service_name): + db = "named_collections" + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db}") + mysql_node.query( + f"CREATE TABLE {db}.t1 (id INT PRIMARY KEY, name VARCHAR(64), val INT)" + ) + mysql_node.query( + f"INSERT INTO {db}.t1 (id, name, val) VALUES (1, 'a', 1), (2, 'b', 2)" + ) + + clickhouse_node.query( + f"""CREATE NAMED COLLECTION {db} AS + user = 'root', + password = 'clickhouse', + host = '{service_name}', + port = 3306, + database = '{db}' + """ + ) + clickhouse_node.query(f"CREATE DATABASE {db} ENGINE = MaterializedMySQL({db})") + check_query( + clickhouse_node, + f"/* expect: (1, 'a', 1), (2, 'b', 2) */ SELECT * FROM {db}.t1", + "1\ta\t1\n2\tb\t2\n", + ) + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index a22d73061ae..5272fb2ff8c 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -523,3 +523,9 @@ def test_materialized_database_mysql_drop_ddl( ): materialize_with_ddl.dropddl(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.dropddl(clickhouse_node, started_mysql_5_7, "mysql57") + + +def test_named_collections(started_cluster, started_mysql_8_0, clickhouse_node): + materialize_with_ddl.named_collections( + clickhouse_node, started_mysql_8_0, "mysql80" + ) From 6f1c4865372b408d0f773a8dd9b1db8e63b5e4bb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Jun 2023 10:25:53 +0200 Subject: [PATCH 1536/2223] Fix build --- src/Coordination/Changelog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index ffa22a6b888..94062140bac 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -13,7 +13,7 @@ #include #include #include -#include #include From bf69755adab474fbd166209ab7675537d1a9aeeb Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 13 Jun 2023 01:18:36 -0700 Subject: [PATCH 1537/2223] Address some usability issues with INTO OUTFILE usage. --- src/Client/ClientBase.cpp | 20 ++++++++++++++++++++ src/Parsers/ASTQueryWithOutput.cpp | 2 ++ src/Parsers/ASTQueryWithOutput.h | 1 + src/Parsers/ParserQueryWithOutput.cpp | 6 ++++++ 4 files changed, 29 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6f295c11070..fc108b8f57d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -568,6 +568,13 @@ try CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); UInt64 compression_level = 3; + if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); + } + if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) { throw Exception( @@ -589,9 +596,22 @@ try range.second); } + std::error_code ec; + if (std::filesystem::is_regular_file(out_file, ec)) + { + if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::CANNOT_OPEN_FILE, + "File {} exists, consider using 'INTO OUTFILE ... APPEND' or 'INTO OUTFILE ... TRUNCATE' if appropriate.", + out_file); + } + } auto flags = O_WRONLY | O_EXCL; if (query_with_output->is_outfile_append) flags |= O_APPEND; + else if (query_with_output->is_outfile_truncate) + flags |= O_TRUNC; else flags |= O_CREAT; diff --git a/src/Parsers/ASTQueryWithOutput.cpp b/src/Parsers/ASTQueryWithOutput.cpp index 5f717715a69..4bf1e6cb231 100644 --- a/src/Parsers/ASTQueryWithOutput.cpp +++ b/src/Parsers/ASTQueryWithOutput.cpp @@ -39,6 +39,8 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat s.ostr << (s.hilite ? hilite_keyword : ""); if (is_outfile_append) s.ostr << " APPEND"; + if (is_outfile_truncate) + s.ostr << " TRUNCATE"; if (is_into_outfile_with_stdout) s.ostr << " AND STDOUT"; s.ostr << (s.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index 7db021405e7..6f9cafc89a9 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -17,6 +17,7 @@ public: ASTPtr out_file; bool is_into_outfile_with_stdout = false; bool is_outfile_append = false; + bool is_outfile_truncate = false; ASTPtr format; ASTPtr settings_ast; ASTPtr compression; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 6796f4528c4..2bfe7353be4 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -109,6 +109,12 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query_with_output.is_outfile_append = true; } + ParserKeyword s_truncate("TRUNCATE"); + if (s_truncate.ignore(pos, expected)) + { + query_with_output.is_outfile_truncate = true; + } + ParserKeyword s_stdout("AND STDOUT"); if (s_stdout.ignore(pos, expected)) { From 2c018f5261553dd6106639f22c148fbdd61d8fc4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 3 Jun 2023 20:59:04 +0200 Subject: [PATCH 1538/2223] do not call finalize after exception --- src/Storages/StorageS3.cpp | 11 +- tests/integration/helpers/client.py | 2 + tests/integration/helpers/cluster.py | 2 + .../integration/helpers/s3_mocks/broken_s3.py | 241 +++++++++++++++--- .../test_checking_s3_blobs_paranoid/test.py | 98 ++++++- tests/integration/test_merge_tree_s3/test.py | 222 +++++++++++++++- 6 files changed, 530 insertions(+), 46 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index f1a7bcb71a2..dfa5ea2667a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -794,7 +794,7 @@ public: void onException() override { std::lock_guard lock(cancel_mutex); - finalize(); + release(); } void onFinish() override @@ -824,6 +824,15 @@ private: } } + void release() + { + if (!writer) + return; + + writer.reset(); + write_buf.reset(); + } + Block sample_block; std::optional format_settings; std::unique_ptr write_buf; diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index c2676ac08a6..fdeedb9a80d 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -121,6 +121,7 @@ class Client: user=None, password=None, database=None, + query_id=None, ): return self.get_query_request( sql, @@ -130,6 +131,7 @@ class Client: user=user, password=password, database=database, + query_id=query_id, ).get_error() @stacktraces_on_timeout_decorator diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f57ebf40e54..c77e67062a1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3376,6 +3376,7 @@ class ClickHouseInstance: user=None, password=None, database=None, + query_id=None, ): logging.debug(f"Executing query {sql} on {self.name}") return self.client.query_and_get_error( @@ -3386,6 +3387,7 @@ class ClickHouseInstance: user=user, password=password, database=database, + query_id=query_id, ) def query_and_get_error_with_retry( diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 026a3c6f515..8ff4f9e9203 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -6,10 +6,10 @@ import time import urllib.parse import http.server import socketserver +import string -UPSTREAM_HOST = "minio1" -UPSTREAM_PORT = 9001 +INF_COUNT = 100000000 class MockControl: @@ -28,31 +28,88 @@ class MockControl: ], nothrow=True, ) - assert response == "OK" + assert response == "OK", response + + def setup_error_at_object_upload(self, count=None, after=None): + url = f"http://localhost:{self._port}/mock_settings/error_at_object_upload?nothing=1" + + if count is not None: + url += f"&count={count}" + + if after is not None: + url += f"&after={after}" - def setup_fail_upload(self, part_length): response = self._cluster.exec_in_container( self._cluster.get_container_id(self._container), [ "curl", "-s", - f"http://localhost:{self._port}/mock_settings/error_at_put?when_length_bigger={part_length}", + url, ], nothrow=True, ) - assert response == "OK" + assert response == "OK", response + + def setup_error_at_part_upload(self, count=None, after=None): + url = f"http://localhost:{self._port}/mock_settings/error_at_part_upload?nothing=1" + + if count is not None: + url += f"&count={count}" + + if after is not None: + url += f"&after={after}" - def setup_fake_upload(self, part_length): response = self._cluster.exec_in_container( self._cluster.get_container_id(self._container), [ "curl", "-s", - f"http://localhost:{self._port}/mock_settings/fake_put?when_length_bigger={part_length}", + url, ], nothrow=True, ) - assert response == "OK" + assert response == "OK", response + + def setup_error_at_create_multi_part_upload(self, count=None): + url = f"http://localhost:{self._port}/mock_settings/error_at_create_multi_part_upload" + + if count is not None: + url += f"?count={count}" + + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + url, + ], + nothrow=True, + ) + assert response == "OK", response + + def setup_fake_puts(self, part_length): + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + f"http://localhost:{self._port}/mock_settings/fake_puts?when_length_bigger={part_length}", + ], + nothrow=True, + ) + assert response == "OK", response + + def setup_fake_multpartuploads(self): + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + f"http://localhost:{self._port}/mock_settings/setup_fake_multpartuploads?", + ], + nothrow=True, + ) + assert response == "OK", response def setup_slow_answers( self, minimal_length=0, timeout=None, probability=None, count=None @@ -77,7 +134,7 @@ class MockControl: ["curl", "-s", url], nothrow=True, ) - assert response == "OK" + assert response == "OK", response class _ServerRuntime: @@ -88,7 +145,7 @@ class _ServerRuntime: self.probability = probability_ if probability_ is not None else 1 self.timeout = timeout_ if timeout_ is not None else 0.1 self.minimal_length = minimal_length_ if minimal_length_ is not None else 0 - self.count = count_ if count_ is not None else 2**32 + self.count = count_ if count_ is not None else INF_COUNT def __str__(self): return ( @@ -109,12 +166,32 @@ class _ServerRuntime: return _runtime.slow_put.timeout return None + class CountAfter: + def __init__(self, count_=None, after_=None): + self.count = count_ if count_ is not None else INF_COUNT + self.after = after_ if after_ is not None else 0 + + def __str__(self): + return f"count:{self.count} after:{self.after}" + + def has_effect(self): + if self.after: + self.after -= 1 + if self.after == 0: + if self.count: + self.count -= 1 + return True + return False + def __init__(self): self.lock = threading.Lock() - self.error_at_put_when_length_bigger = None + self.error_at_part_upload = None + self.error_at_object_upload = None self.fake_put_when_length_bigger = None self.fake_uploads = dict() self.slow_put = None + self.fake_multipart_upload = None + self.error_at_create_multi_part_upload = None def register_fake_upload(self, upload_id, key): with self.lock: @@ -127,10 +204,14 @@ class _ServerRuntime: return False def reset(self): - self.error_at_put_when_length_bigger = None - self.fake_put_when_length_bigger = None - self.fake_uploads = dict() - self.slow_put = None + with self.lock: + self.error_at_part_upload = None + self.error_at_object_upload = None + self.fake_put_when_length_bigger = None + self.fake_uploads = dict() + self.slow_put = None + self.fake_multipart_upload = None + self.error_at_create_multi_part_upload = None _runtime = _ServerRuntime() @@ -141,6 +222,13 @@ def _and_then(value, func): return None if value is None else func(value) +def get_random_string(length): + # choose from all lowercase letter + letters = string.ascii_lowercase + result_str = "".join(random.choice(letters) for i in range(length)) + return result_str + + class RequestHandler(http.server.BaseHTTPRequestHandler): def _ok(self): self.send_response(200) @@ -166,19 +254,30 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self._read_out() self.send_response(307) - url = f"http://{UPSTREAM_HOST}:{UPSTREAM_PORT}{self.path}" + url = ( + f"http://{self.server.upstream_host}:{self.server.upstream_port}{self.path}" + ) self.send_header("Location", url) self.end_headers() self.wfile.write(b"Redirected") def _error(self, data): self._read_out() - self.send_response(500) self.send_header("Content-Type", "text/xml") self.end_headers() self.wfile.write(bytes(data, "UTF-8")) + def _error_expected_500(self): + self._error( + '' + "" + "ExpectedError" + "mock s3 injected error" + "txfbd566d03042474888193-00608d7537" + "" + ) + def _fake_put_ok(self): self._read_out() @@ -188,6 +287,28 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.send_header("Content-Length", 0) self.end_headers() + def _fake_uploads(self, path, upload_id): + self._read_out() + + parts = [x for x in path.split("/") if x] + bucket = parts[0] + key = "/".join(parts[1:]) + data = ( + '\n' + "\n" + f"{bucket}" + f"{key}" + f"{upload_id}" + "" + ) + + self.send_response(200) + self.send_header("Content-Type", "text/xml") + self.send_header("Content-Length", len(data)) + self.end_headers() + + self.wfile.write(bytes(data, "UTF-8")) + def _fake_post_ok(self, path): self._read_out() @@ -219,18 +340,29 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if len(path) < 2: return self._error("_mock_settings: wrong command") - if path[1] == "error_at_put": + if path[1] == "error_at_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.error_at_put_when_length_bigger = int( - params.get("when_length_bigger", [1024 * 1024])[0] + _runtime.error_at_part_upload = _ServerRuntime.CountAfter( + count_=_and_then(params.get("count", [None])[0], int), + after_=_and_then(params.get("after", [None])[0], int), ) return self._ok() - if path[1] == "fake_put": + + if path[1] == "error_at_object_upload": + params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) + _runtime.error_at_object_upload = _ServerRuntime.CountAfter( + count_=_and_then(params.get("count", [None])[0], int), + after_=_and_then(params.get("after", [None])[0], int), + ) + return self._ok() + + if path[1] == "fake_puts": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) _runtime.fake_put_when_length_bigger = int( params.get("when_length_bigger", [1024 * 1024])[0] ) return self._ok() + if path[1] == "slow_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) _runtime.slow_put = _ServerRuntime.SlowPut( @@ -241,6 +373,18 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): ) self.log_message("set slow put %s", _runtime.slow_put) return self._ok() + + if path[1] == "setup_fake_multpartuploads": + _runtime.fake_multipart_upload = True + return self._ok() + + if path[1] == "error_at_create_multi_part_upload": + params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) + _runtime.error_at_create_multi_part_upload = int( + params.get("count", [INF_COUNT])[0] + ) + return self._ok() + if path[1] == "reset": _runtime.reset() return self._ok() @@ -265,33 +409,42 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.log_message("slow put %s", timeout) time.sleep(timeout) - if _runtime.error_at_put_when_length_bigger is not None: - if content_length > _runtime.error_at_put_when_length_bigger: - return self._error( - '' - "" - "ExpectedError" - "mock s3 injected error" - "txfbd566d03042474888193-00608d7537" - "" - ) - parts = urllib.parse.urlsplit(self.path) params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) upload_id = params.get("uploadId", [None])[0] - if _runtime.fake_put_when_length_bigger is not None: - if content_length > _runtime.fake_put_when_length_bigger: - if upload_id is not None: - _runtime.register_fake_upload(upload_id, parts.path) - return self._fake_put_ok() + + if upload_id is not None: + if _runtime.error_at_part_upload is not None: + if _runtime.error_at_part_upload.has_effect(): + return self._error_expected_500() + if _runtime.fake_multipart_upload: + if _runtime.is_fake_upload(upload_id, parts.path): + return self._fake_put_ok() + else: + if _runtime.error_at_object_upload is not None: + if _runtime.error_at_object_upload.has_effect(): + return self._error_expected_500() + if _runtime.fake_put_when_length_bigger is not None: + if content_length > _runtime.fake_put_when_length_bigger: + return self._fake_put_ok() return self._redirect() def do_POST(self): parts = urllib.parse.urlsplit(self.path) - params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - upload_id = params.get("uploadId", [None])[0] + params = urllib.parse.parse_qs(parts.query, keep_blank_values=True) + uploads = params.get("uploads", [None])[0] + if uploads is not None: + if _runtime.error_at_create_multi_part_upload: + _runtime.error_at_create_multi_part_upload -= 1 + return self._error_expected_500() + if _runtime.fake_multipart_upload: + upload_id = get_random_string(5) + _runtime.register_fake_upload(upload_id, parts.path) + return self._fake_uploads(parts.path, upload_id) + + upload_id = params.get("uploadId", [None])[0] if _runtime.is_fake_upload(upload_id, parts.path): return self._fake_post_ok(parts.path) @@ -307,7 +460,15 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): class _ThreadedHTTPServer(socketserver.ThreadingMixIn, http.server.HTTPServer): """Handle requests in a separate thread.""" + def set_upstream(self, upstream_host, upstream_port): + self.upstream_host = upstream_host + self.upstream_port = upstream_port + if __name__ == "__main__": httpd = _ThreadedHTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) + if len(sys.argv) == 4: + httpd.set_upstream(sys.argv[2], sys.argv[3]) + else: + httpd.set_upstream("minio1", 9001) httpd.serve_forever() diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 042d57a0c43..c0f184815c9 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -54,7 +54,7 @@ def test_upload_after_check_works(cluster, broken_s3): """ ) - broken_s3.setup_fake_upload(1) + broken_s3.setup_fake_puts(1) error = node.query_and_get_error( "INSERT INTO s3_upload_after_check_works VALUES (1, 'Hello')" @@ -63,3 +63,99 @@ def test_upload_after_check_works(cluster, broken_s3): assert "Code: 499" in error, error assert "Immediately after upload" in error, error assert "suddenly disappeared" in error, error + + +def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): + node.query("SYSTEM FLUSH LOGS") + return [ + int(x) + for x in node.query( + f""" + SELECT + ProfileEvents['S3CreateMultipartUpload'], + ProfileEvents['S3UploadPart'], + ProfileEvents['S3WriteRequestsErrors'] + FROM system.query_log + WHERE query_id='{query_id}' + AND type='{log_type}' + """ + ).split() + if x + ] + + +def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3): + node = cluster.instances["node"] + + broken_s3.setup_error_at_create_multi_part_upload() + + insert_query_id = "INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU" + error = node.query_and_get_error( + """ + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_upload_s3_fail_create_multi_part_upload', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 100000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100 + """, + query_id=insert_query_id, + ) + + assert "Code: 499" in error, error + assert "mock s3 injected error" in error, error + assert "DB::WriteBufferFromS3::createMultipartUpload()" in error, error + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id + ) + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 0 + assert count_s3_errors == 1 + + +def test_upload_s3_fail_upload_part_when_multi_part_upload(cluster, broken_s3): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_error_at_part_upload(count=1, after=2) + + insert_query_id = "INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART" + error = node.query_and_get_error( + """ + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_upload_s3_fail_upload_part_when_multi_part_upload', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 100000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100 + """, + query_id=insert_query_id, + ) + + assert "Code: 499" in error, error + assert "mock s3 injected error" in error, error + assert "DB::WriteBufferFromS3::writePart" in error, error + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id + ) + assert count_create_multi_part_uploads == 1 + assert count_upload_parts >= 2 + assert ( + count_s3_errors == 2 + ) # the second is cancel multipart upload, s3_mock just redirects this request diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 2ccd517923a..f3f44f1452c 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -862,7 +862,9 @@ def test_merge_canceled_by_s3_errors(cluster, broken_s3, node_name, storage_poli min_key = node.query("SELECT min(key) FROM test_merge_canceled_by_s3_errors") assert int(min_key) == 0, min_key - broken_s3.setup_fail_upload(50000) + broken_s3.setup_error_at_object_upload() + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_error_at_part_upload() node.query("SYSTEM START MERGES test_merge_canceled_by_s3_errors") @@ -905,7 +907,7 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, broken_s3, node_name): settings={"materialize_ttl_after_modify": 0}, ) - broken_s3.setup_fail_upload(10000) + broken_s3.setup_error_at_object_upload(count=1, after=1) node.query("SYSTEM START MERGES merge_canceled_by_s3_errors_when_move") @@ -941,7 +943,7 @@ def test_s3_engine_heavy_write_check_mem( " ENGINE S3('http://resolver:8083/root/data/test-upload.csv', 'minio', 'minio123', 'CSV')", ) - broken_s3.setup_fake_upload(1000) + broken_s3.setup_fake_multpartuploads() broken_s3.setup_slow_answers(10 * 1024 * 1024, timeout=15, count=10) query_id = f"INSERT_INTO_S3_ENGINE_QUERY_ID_{in_flight}" @@ -987,7 +989,7 @@ def test_s3_disk_heavy_write_check_mem(cluster, broken_s3, node_name): ) node.query("SYSTEM STOP MERGES s3_test") - broken_s3.setup_fake_upload(1000) + broken_s3.setup_fake_multpartuploads() broken_s3.setup_slow_answers(10 * 1024 * 1024, timeout=10, count=50) query_id = f"INSERT_INTO_S3_DISK_QUERY_ID" @@ -1013,3 +1015,215 @@ def test_s3_disk_heavy_write_check_mem(cluster, broken_s3, node_name): assert int(result) > 0.8 * memory check_no_objects_after_drop(cluster, node_name=node_name) + + +def get_memory_usage(node, query_id): + node.query("SYSTEM FLUSH LOGS") + memory_usage = node.query( + "SELECT memory_usage" + " FROM system.query_log" + f" WHERE query_id='{query_id}'" + " AND type='QueryFinish'" + ) + return int(memory_usage) + + +def get_memory_usages(node, query_ids): + node.query("SYSTEM FLUSH LOGS") + result = [] + for query_id in query_ids: + memory_usage = node.query( + "SELECT memory_usage" + " FROM system.query_log" + f" WHERE query_id='{query_id}'" + " AND type='QueryFinish'" + ) + result.append(int(memory_usage)) + return result + + +@pytest.mark.parametrize("node_name", ["node"]) +def test_heavy_insert_select_check_memory(cluster, broken_s3, node_name): + node = cluster.instances[node_name] + + node.query( + """ + CREATE TABLE central_query_log + ( + control_plane_id UUID, + pod_id LowCardinality(String), + scrape_ts_microseconds DateTime64(6) CODEC(Delta(8), LZ4), + event_date Date, + event_time DateTime, + payload Array(String), + payload_01 String, + payload_02 String, + payload_03 String, + payload_04 String, + payload_05 String, + payload_06 String, + payload_07 String, + payload_08 String, + payload_09 String, + payload_10 String, + payload_11 String, + payload_12 String, + payload_13 String, + payload_14 String, + payload_15 String, + payload_16 String, + payload_17 String, + payload_18 String, + payload_19 String + ) + ENGINE=MergeTree() + PARTITION BY toYYYYMM(event_date) + ORDER BY (control_plane_id, event_date, pod_id) + SETTINGS + storage_policy='s3' + """ + ) + + node.query("SYSTEM STOP MERGES central_query_log") + + write_count = 2 + write_query_ids = [] + for x in range(write_count): + query_id = f"INSERT_INTO_TABLE_RANDOM_DATA_QUERY_ID_{x}" + write_query_ids.append(query_id) + node.query( + """ + INSERT INTO central_query_log + SELECT + control_plane_id, + pod_id, + toStartOfHour(event_time) + toIntervalSecond(randUniform(0,60)) as scrape_ts_microseconds, + toDate(event_time) as event_date, + event_time, + payload, + payload[1] as payload_01, + payload[2] as payload_02, + payload[3] as payload_03, + payload[4] as payload_04, + payload[5] as payload_05, + payload[6] as payload_06, + payload[7] as payload_07, + payload[8] as payload_08, + payload[9] as payload_09, + payload[10] as payload_10, + payload[11] as payload_11, + payload[12] as payload_12, + payload[13] as payload_13, + payload[14] as payload_14, + payload[15] as payload_15, + payload[16] as payload_16, + payload[17] as payload_17, + payload[18] as payload_18, + payload[19] as payload_19 + FROM + ( + SELECT + control_plane_id, + substring(payload[1], 1, 5) as pod_id, + toDateTime('2022-12-12 00:00:00') + + toIntervalDay(floor(randUniform(0,3))) + + toIntervalHour(floor(randUniform(0,24))) + + toIntervalSecond(floor(randUniform(0,60))) + as event_time, + payload + FROM + generateRandom( + 'control_plane_id UUID, payload Array(String)', + NULL, + 100, + 100 + ) + LIMIT 10000 + ) + SETTINGS + max_insert_block_size=256000000, + min_insert_block_size_rows=1000000, + min_insert_block_size_bytes=256000000 + """, + query_id=query_id, + ) + + memory = 845346116 + for memory_usage, query_id in zip( + get_memory_usages(node, write_query_ids), write_query_ids + ): + assert int(memory_usage) < 1.2 * memory, f"{memory_usage} : {query_id}" + assert int(memory_usage) > 0.8 * memory, f"{memory_usage} : {query_id}" + + broken_s3.setup_slow_answers(minimal_length=1000, timeout=5, count=20) + broken_s3.setup_fake_multpartuploads() + + insert_query_id = f"INSERT_INTO_S3_FUNCTION_QUERY_ID" + node.query( + """ + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test-upload_{_partition_id}.csv.gz', + 'minio', 'minio123', + 'CSV', auto, 'gzip' + ) + PARTITION BY formatDateTime(subtractHours(toDateTime('2022-12-13 00:00:00'), 1),'%Y-%m-%d_%H:00') + WITH toDateTime('2022-12-13 00:00:00') as time_point + SELECT + * + FROM central_query_log + WHERE + event_date >= subtractDays(toDate(time_point), 1) + AND scrape_ts_microseconds >= subtractHours(toStartOfHour(time_point), 12) + AND scrape_ts_microseconds < toStartOfDay(time_point) + SETTINGS + s3_max_inflight_parts_for_one_file=1 + """, + query_id=insert_query_id, + ) + + query_id = f"SELECT_QUERY_ID" + total = node.query( + """ + SELECT + count() + FROM central_query_log + """, + query_id=query_id, + ) + assert int(total) == 10000 * write_count + + query_id = f"SELECT_WHERE_QUERY_ID" + selected = node.query( + """ + WITH toDateTime('2022-12-13 00:00:00') as time_point + SELECT + count() + FROM central_query_log + WHERE + event_date >= subtractDays(toDate(time_point), 1) + AND scrape_ts_microseconds >= subtractHours(toStartOfHour(time_point), 12) + AND scrape_ts_microseconds < toStartOfDay(time_point) + """, + query_id=query_id, + ) + assert int(selected) < 4500, selected + assert int(selected) > 2500, selected + + node.query("SYSTEM FLUSH LOGS") + profile_events = node.query( + f""" + SELECT ProfileEvents + FROM system.query_log + WHERE query_id='{insert_query_id}' + AND type='QueryFinish' + """ + ) + + memory_usage = get_memory_usage(node, insert_query_id) + memory = 123507857 + assert int(memory_usage) < 1.2 * memory, f"{memory_usage} {profile_events}" + assert int(memory_usage) > 0.8 * memory, f"{memory_usage} {profile_events}" + + node.query(f"DROP TABLE IF EXISTS central_query_log SYNC") + remove_all_s3_objects(cluster) From 9c939b2f3db3b47116d739a3b81ab7c353e6e0bf Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 10:54:54 +0200 Subject: [PATCH 1539/2223] Fix heading and sidebar for azureBlobStorage table function --- docs/en/sql-reference/table-functions/azureBlobStorage.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 369bf7a964d..7a362710b9c 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -1,10 +1,10 @@ --- slug: /en/sql-reference/table-functions/azure_blob_storage -sidebar_label: azure_blob_storage +sidebar_label: azureBlobStorage keywords: [azure blob storage] --- -# azure\_blob\_storage Table Function +# azureBlobStorage Table Function Provides a table-like interface to select/insert files in [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs). This table function is similar to the [s3 function](../../sql-reference/table-functions/s3.md). From 79bc8847333f6e8e3653e63b1fed6a063bfb6302 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Jun 2023 08:54:25 +0000 Subject: [PATCH 1540/2223] Stabilize tests --- src/Functions/FunctionSnowflake.h | 58 ++++++++++--------- .../01942_snowflakeToDateTime.reference | 3 +- .../0_stateless/01942_snowflakeToDateTime.sql | 10 +++- 3 files changed, 41 insertions(+), 30 deletions(-) diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index 0a47534c47d..ace2fc54f09 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -54,22 +54,19 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const auto & src = arguments[0]; - const auto & col = *src.column; + const auto & src_column = *src.column; auto res_column = ColumnInt64::create(input_rows_count); - auto & result_data = res_column->getData(); + auto & res_data = res_column->getData(); - const auto & source_data = typeid_cast(col).getData(); + const auto & src_data = typeid_cast &>(src_column).getData(); for (size_t i = 0; i < input_rows_count; ++i) - { - result_data[i] = (Int64(source_data[i]) * 1000 - snowflake_epoch) << time_shift; - } + res_data[i] = (UInt32(src_data[i]) * 1000 - snowflake_epoch) << time_shift; return res_column; } }; - class FunctionSnowflakeToDateTime : public IFunction { private: @@ -106,23 +103,23 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const auto & src = arguments[0]; - const auto & col = *src.column; + const auto & src_column = *src.column; auto res_column = ColumnUInt32::create(input_rows_count); - auto & result_data = res_column->getData(); + auto & res_data = res_column->getData(); - if (const auto * src_non_const = typeid_cast(&col)) + if (const auto * src_column_non_const = typeid_cast(&src_column)) { - const auto & source_data = src_non_const->getData(); + const auto & src_data = src_column_non_const->getData(); for (size_t i = 0; i < input_rows_count; ++i) - result_data[i] = static_cast( - ((source_data[i] >> time_shift) + snowflake_epoch) / 1000); + res_data[i] = static_cast( + ((src_data[i] >> time_shift) + snowflake_epoch) / 1000); } - else if (const auto * src_const = typeid_cast(&col)) + else if (const auto * src_column_const = typeid_cast(&src_column)) { - Int64 src_val = src_const->getValue(); + Int64 src_val = src_column_const->getValue(); for (size_t i = 0; i < input_rows_count; ++i) - result_data[i] = static_cast( + res_data[i] = static_cast( ((src_val >> time_shift) + snowflake_epoch) / 1000); } return res_column; @@ -155,16 +152,14 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const auto & src = arguments[0]; - const auto & col = *src.column; + const auto & src_column = *src.column; auto res_column = ColumnInt64::create(input_rows_count); - auto & result_data = res_column->getData(); + auto & res_data = res_column->getData(); - const auto & source_data = typeid_cast &>(col).getData(); + const auto & src_data = typeid_cast &>(src_column).getData(); for (size_t i = 0; i < input_rows_count; ++i) - { - result_data[i] = (source_data[i] - snowflake_epoch) << time_shift; - } + res_data[i] = (src_data[i] - snowflake_epoch) << time_shift; return res_column; } @@ -207,17 +202,24 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const auto & src = arguments[0]; - const auto & col = *src.column; + const auto & src_column = *src.column; auto res_column = ColumnDecimal::create(input_rows_count, 3); - auto & result_data = res_column->getData(); + auto & res_data = res_column->getData(); - const auto & source_data = typeid_cast(col).getData(); - - for (size_t i = 0; i < input_rows_count; ++i) + if (const auto * src_column_non_const = typeid_cast(&src_column)) { - result_data[i] = (source_data[i] >> time_shift) + snowflake_epoch; + const auto & src_data = src_column_non_const->getData(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (src_data[i] >> time_shift) + snowflake_epoch; } + else if (const auto * src_column_const = typeid_cast(&src_column)) + { + Int64 src_val = src_column_const->getValue(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (src_val >> time_shift) + snowflake_epoch; + } + return res_column; } }; diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.reference b/tests/queries/0_stateless/01942_snowflakeToDateTime.reference index fa00a22bc63..e1d141fe450 100644 --- a/tests/queries/0_stateless/01942_snowflakeToDateTime.reference +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.reference @@ -1,4 +1,5 @@ const column UTC 1426860704886947840 2021-08-15 10:57:56 DateTime(\'UTC\') 2021-08-15 10:57:56.492 DateTime64(3, \'UTC\') Asia/Shanghai 1426860704886947840 2021-08-15 18:57:56 DateTime(\'Asia/Shanghai\') 2021-08-15 18:57:56.492 DateTime64(3, \'Asia/Shanghai\') -Asia/Singapore 2010-11-04 01:42:54 +Asia/Singapore 42 +Asia/Singapore 42 diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql index 3efccdddb2d..2ad03f2a4f5 100644 --- a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql @@ -31,4 +31,12 @@ SELECT snowflakeToDateTime64(i64, tz) as dt64, toTypeName(dt64); -SELECT materialize('Asia/Singapore') a, snowflakeToDateTime(649::Int64, a) settings allow_nonconst_timezone_arguments = 1 + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab(tz String, val Int64) engine=Log; +INSERT INTO tab VALUES ('Asia/Singapore', 42); + +SELECT * FROM tab WHERE snowflakeToDateTime(42::Int64, tz) != now() SETTINGS allow_nonconst_timezone_arguments = 1; +SELECT * FROM tab WHERE snowflakeToDateTime64(42::Int64, tz) != now() SETTINGS allow_nonconst_timezone_arguments = 1; + +DROP TABLE tab; From 2d0dc2c8f5c329a4da12ccb1db601d5edf2044cd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Jun 2023 08:59:34 +0000 Subject: [PATCH 1541/2223] Minor: Switch column order --- .../0_stateless/01942_snowflakeToDateTime.reference | 4 ++-- tests/queries/0_stateless/01942_snowflakeToDateTime.sql | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.reference b/tests/queries/0_stateless/01942_snowflakeToDateTime.reference index e1d141fe450..83fae3ef809 100644 --- a/tests/queries/0_stateless/01942_snowflakeToDateTime.reference +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.reference @@ -1,5 +1,5 @@ const column UTC 1426860704886947840 2021-08-15 10:57:56 DateTime(\'UTC\') 2021-08-15 10:57:56.492 DateTime64(3, \'UTC\') Asia/Shanghai 1426860704886947840 2021-08-15 18:57:56 DateTime(\'Asia/Shanghai\') 2021-08-15 18:57:56.492 DateTime64(3, \'Asia/Shanghai\') -Asia/Singapore 42 -Asia/Singapore 42 +1 +1 diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql index 2ad03f2a4f5..0092eca848c 100644 --- a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql @@ -33,10 +33,10 @@ SELECT DROP TABLE IF EXISTS tab; -CREATE TABLE tab(tz String, val Int64) engine=Log; -INSERT INTO tab VALUES ('Asia/Singapore', 42); +CREATE TABLE tab(val Int64, tz String) engine=Log; +INSERT INTO tab VALUES (42, 'Asia/Singapore'); -SELECT * FROM tab WHERE snowflakeToDateTime(42::Int64, tz) != now() SETTINGS allow_nonconst_timezone_arguments = 1; -SELECT * FROM tab WHERE snowflakeToDateTime64(42::Int64, tz) != now() SETTINGS allow_nonconst_timezone_arguments = 1; +SELECT 1 FROM tab WHERE snowflakeToDateTime(42::Int64, tz) != now() SETTINGS allow_nonconst_timezone_arguments = 1; +SELECT 1 FROM tab WHERE snowflakeToDateTime64(42::Int64, tz) != now() SETTINGS allow_nonconst_timezone_arguments = 1; DROP TABLE tab; From 72f28321295187ddf40d02a887be3106f6ec4ac3 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 13 Jun 2023 02:07:05 -0700 Subject: [PATCH 1542/2223] Slightly more information in error message about cached disk (#50897) --- src/Disks/ObjectStorages/Cached/registerDiskCache.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp index 779ec6120f8..2b40fa9c21b 100644 --- a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp +++ b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp @@ -48,7 +48,9 @@ void registerDiskCache(DiskFactory & factory, bool /* global_skip_access_check * auto cache = FileCacheFactory::instance().getOrCreate(name, file_cache_settings); auto disk = disk_it->second; if (!dynamic_cast(disk.get())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cached disk is allowed only on top of object storage"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot wrap disk `{}` with cache layer `{}`: cached disk is allowed only on top of object storage", + disk_name, name); auto disk_object_storage = disk->createDiskObjectStorage(); From 0ab3dc92618f8a7d1accd8f2e1cc21f851dead80 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 13 Jun 2023 11:25:13 +0200 Subject: [PATCH 1543/2223] A bit safer UserDefinedSQLFunctionVisitor (#50913) * Update UserDefinedSQLFunctionVisitor.cpp * Update UserDefinedSQLFunctionVisitor.cpp --------- Co-authored-by: Nikita Mikhaylov --- src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp index 57cc45cc75d..597e4efe35e 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp @@ -25,6 +25,12 @@ namespace ErrorCodes void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast) { + if (!ast) + { + chassert(false); + return; + } + const auto visit_child_with_shared_ptr = [&](ASTPtr & child) { if (!child) From 4ac090b12212567f7b9d30cb27132f566ae438c4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Jun 2023 11:01:44 +0000 Subject: [PATCH 1544/2223] Fix tests --- src/Coordination/tests/gtest_coordination.cpp | 22 +++++++++---------- .../StorageSystemZooKeeperConnection.cpp | 4 ++-- tests/config/config.d/keeper_port.xml | 4 ++++ .../test_keeper_four_word_command/test.py | 8 +++---- ...2735_system_zookeeper_connection.reference | 2 +- 5 files changed, 22 insertions(+), 18 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 2793b23c572..b29d5bca43d 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1193,7 +1193,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2); EXPECT_EQ(snapshot.session_id, 7); - EXPECT_EQ(snapshot.snapshot_container_size, 5); + EXPECT_EQ(snapshot.snapshot_container_size, 6); EXPECT_EQ(snapshot.session_and_timeout.size(), 2); auto buf = manager.serializeSnapshotToBuffer(snapshot); @@ -1205,7 +1205,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf); - EXPECT_EQ(restored_storage->container.size(), 5); + EXPECT_EQ(restored_storage->container.size(), 6); EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 2); EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1); EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0); @@ -1237,14 +1237,14 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) DB::KeeperStorageSnapshot snapshot(&storage, 50); EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50); - EXPECT_EQ(snapshot.snapshot_container_size, 53); + EXPECT_EQ(snapshot.snapshot_container_size, 54); for (size_t i = 50; i < 100; ++i) { addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } - EXPECT_EQ(storage.container.size(), 103); + EXPECT_EQ(storage.container.size(), 104); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); @@ -1254,7 +1254,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) auto debuf = manager.deserializeSnapshotBufferFromDisk(50); auto [restored_storage, meta, _] = manager.deserializeSnapshotFromBuffer(debuf); - EXPECT_EQ(restored_storage->container.size(), 53); + EXPECT_EQ(restored_storage->container.size(), 54); for (size_t i = 0; i < 50; ++i) { EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).getData(), "world_" + std::to_string(i)); @@ -1293,7 +1293,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) auto [restored_storage, meta, _] = manager.restoreFromLatestSnapshot(); - EXPECT_EQ(restored_storage->container.size(), 253); + EXPECT_EQ(restored_storage->container.size(), 254); for (size_t i = 0; i < 250; ++i) { @@ -1327,16 +1327,16 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) if (i % 2 == 0) storage.container.erase("/hello_" + std::to_string(i)); } - EXPECT_EQ(storage.container.size(), 28); - EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 104); + EXPECT_EQ(storage.container.size(), 29); + EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 105); EXPECT_EQ(storage.container.snapshotSizeWithVersion().second, 1); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); } EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); - EXPECT_EQ(storage.container.size(), 28); + EXPECT_EQ(storage.container.size(), 29); storage.clearGarbageAfterSnapshot(); - EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 28); + EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 29); for (size_t i = 0; i < 50; ++i) { if (i % 2 != 0) @@ -1865,7 +1865,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) auto [restored_storage, snapshot_meta, _] = new_manager.deserializeSnapshotFromBuffer(debuf); - EXPECT_EQ(restored_storage->container.size(), 5); + EXPECT_EQ(restored_storage->container.size(), 6); EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 2); EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1); EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0); diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 99872be6ba0..9a6a592f2c4 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -31,7 +31,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); res_columns[4]->insert(context->getZooKeeperSessionUptime()); res_columns[5]->insert(context->getZooKeeper()->expired()); - res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); + res_columns[6]->insert(0); res_columns[7]->insert(context->getZooKeeper()->getClientID()); for (const auto & elem : context->getAuxiliaryZooKeepers()) @@ -42,7 +42,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); + res_columns[6]->insert(0); res_columns[7]->insert(elem.second->getClientID()); } diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index cffd325e968..7db174c5419 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -28,5 +28,9 @@ 9234 + + + 1 +
diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 2098daea5fe..adc29abb584 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -183,8 +183,8 @@ def test_cmd_mntr(started_cluster): # contains: # 10 nodes created by test # 3 nodes created by clickhouse "/clickhouse/task_queue/ddl" - # 1 root node, 2 keeper system nodes - assert int(result["zk_znode_count"]) == 13 + # 1 root node, 3 keeper system nodes + assert int(result["zk_znode_count"]) == 14 assert int(result["zk_watch_count"]) == 2 assert int(result["zk_ephemerals_count"]) == 2 assert int(result["zk_approximate_data_size"]) > 0 @@ -329,7 +329,7 @@ def test_cmd_srvr(started_cluster): assert int(result["Connections"]) == 1 assert int(result["Zxid"]) > 10 assert result["Mode"] == "leader" - assert result["Node count"] == "13" + assert result["Node count"] == "14" finally: destroy_zk_client(zk) @@ -369,7 +369,7 @@ def test_cmd_stat(started_cluster): assert int(result["Connections"]) == 1 assert int(result["Zxid"]) >= 10 assert result["Mode"] == "leader" - assert result["Node count"] == "13" + assert result["Node count"] == "14" # filter connection statistics cons = [n for n in data.split("\n") if "=" in n] diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference index 1deabd88b88..55b3579f0dd 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference @@ -1,2 +1,2 @@ -default ::1 9181 0 0 3 +default ::1 9181 0 0 0 zookeeper2 ::1 9181 0 0 0 From e5de6cde244f530c0f7d3ec1acad462025430e58 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Jun 2023 11:37:24 +0000 Subject: [PATCH 1545/2223] Update after #50097 --- tests/queries/0_stateless/01655_plan_optimizations.reference | 4 ++-- tests/queries/0_stateless/01655_plan_optimizations.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 34ea2bc20a3..be42a656c66 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -172,7 +172,7 @@ Filter column: notEquals(number, 1) Join > (analyzer) one condition of filter is pushed down before LEFT JOIN Join -Filter column: notEquals(l.number_0, 1_UInt8) +Filter column: notEquals(number_0, 1_UInt8) 0 0 3 3 > one condition of filter is pushed down before INNER JOIN @@ -181,7 +181,7 @@ Filter column: notEquals(number, 1) Join > (analyzer) one condition of filter is pushed down before INNER JOIN Join -Filter column: notEquals(l.number_0, 1_UInt8) +Filter column: notEquals(number_0, 1_UInt8) 3 3 > filter is pushed down before UNION Union diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index d68c2c8b414..a765a6ea4fa 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -236,7 +236,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | - grep -o "Join\|Filter column: notEquals(l.number_0, 1_UInt8)" + grep -o "Join\|Filter column: notEquals(number_0, 1_UInt8)" $CLICKHOUSE_CLIENT -q " select number as a, r.b from numbers(4) as l any left join ( select number + 2 as b from numbers(3) @@ -255,7 +255,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " select number as a, r.b from numbers(4) as l any inner join ( select number + 2 as b from numbers(3) ) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" | - grep -o "Join\|Filter column: notEquals(l.number_0, 1_UInt8)" + grep -o "Join\|Filter column: notEquals(number_0, 1_UInt8)" $CLICKHOUSE_CLIENT -q " select number as a, r.b from numbers(4) as l any inner join ( select number + 2 as b from numbers(3) From a8b68a877aae945dee4e37b28e320a967a38f9f2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 13 Jun 2023 14:37:33 +0300 Subject: [PATCH 1546/2223] Rename the 'time shift' variable in the test to make it more clear --- .../02783_parseDateTimeBestEffort_syslog.sql | 76 +++++++++---------- 1 file changed, 38 insertions(+), 38 deletions(-) diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql index 742ae03ddab..f3ca78e8310 100644 --- a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql +++ b/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql @@ -2,45 +2,45 @@ WITH 86400 AS secs_in_day, now() AS ts_now, '2023-06-07' AS ref_point, - dateDiff('second', toDateTime(ref_point), ts_now) AS impedimenta, + dateDiff('second', toDateTime(ref_point), ts_now) AS time_shift, formatDateTime(ts_around, '%b %e %T') AS dt_curr SELECT - formatDateTime(ts_around - impedimenta, '%b %e %H:%i:%s') AS around_June_7, - parseDateTimeBestEffort(dt_curr) - impedimenta AS res, - parseDateTimeBestEffort(dt_curr, 'US/Samoa') - impedimenta AS res_sam, - parseDateTimeBestEffort(dt_curr, 'Pacific/Auckland') - impedimenta AS res_auc, - parseDateTimeBestEffortOrNull(dt_curr) - impedimenta AS res_null, - parseDateTimeBestEffortOrNull(dt_curr, 'US/Samoa') - impedimenta AS res_null_sam, - parseDateTimeBestEffortOrNull(dt_curr, 'Pacific/Auckland') - impedimenta AS res_null_auc, - parseDateTimeBestEffortOrZero(dt_curr) - impedimenta AS res_zero, - parseDateTimeBestEffortOrZero(dt_curr, 'US/Samoa') - impedimenta AS res_zero_sam, - parseDateTimeBestEffortOrZero(dt_curr, 'Pacific/Auckland') - impedimenta AS res_zero_auc, - parseDateTimeBestEffortUS(dt_curr) - impedimenta AS res_us, - parseDateTimeBestEffortUS(dt_curr, 'US/Samoa') - impedimenta AS res_us_sam, - parseDateTimeBestEffortUS(dt_curr, 'Pacific/Auckland') - impedimenta AS res_us_auc, - parseDateTimeBestEffortUSOrNull(dt_curr) - impedimenta AS res_us_null, - parseDateTimeBestEffortUSOrNull(dt_curr, 'US/Samoa') - impedimenta AS res_us_null_sam, - parseDateTimeBestEffortUSOrNull(dt_curr, 'Pacific/Auckland') - impedimenta AS res_us_null_auc, - parseDateTimeBestEffortUSOrZero(dt_curr) - impedimenta AS res_us_zero, - parseDateTimeBestEffortUSOrZero(dt_curr, 'US/Samoa') - impedimenta AS res_us_zero_sam, - parseDateTimeBestEffortUSOrZero(dt_curr, 'Pacific/Auckland') - impedimenta AS res_us_zero_auc, - parseDateTime64BestEffort(dt_curr) - impedimenta AS res64, - parseDateTime64BestEffort(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_sam, - parseDateTime64BestEffort(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_auc, - parseDateTime64BestEffortOrNull(dt_curr) - impedimenta AS res64_null, - parseDateTime64BestEffortOrNull(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_null_sam, - parseDateTime64BestEffortOrNull(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_null_auc, - parseDateTime64BestEffortOrZero(dt_curr) - impedimenta AS res64_zero, - parseDateTime64BestEffortOrZero(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_zero_sam, - parseDateTime64BestEffortOrZero(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_zero_auc, - parseDateTime64BestEffortUS(dt_curr) - impedimenta AS res64_us, - parseDateTime64BestEffortUS(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_us_sam, - parseDateTime64BestEffortUS(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_us_auc, - parseDateTime64BestEffortUSOrNull(dt_curr) - impedimenta AS res64_us_null, - parseDateTime64BestEffortUSOrNull(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_us_null_sam, - parseDateTime64BestEffortUSOrNull(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_us_null_auc, - parseDateTime64BestEffortUSOrZero(dt_curr) - impedimenta AS res64_us_zero, - parseDateTime64BestEffortUSOrZero(dt_curr, 3, 'US/Samoa') - impedimenta AS res64_us_zero_sam, - parseDateTime64BestEffortUSOrZero(dt_curr, 3, 'Pacific/Auckland') - impedimenta AS res64_us_zero_auc + formatDateTime(ts_around - time_shift, '%b %e %H:%i:%s') AS around_June_7, + parseDateTimeBestEffort(dt_curr) - time_shift AS res, + parseDateTimeBestEffort(dt_curr, 'US/Samoa') - time_shift AS res_sam, + parseDateTimeBestEffort(dt_curr, 'Pacific/Auckland') - time_shift AS res_auc, + parseDateTimeBestEffortOrNull(dt_curr) - time_shift AS res_null, + parseDateTimeBestEffortOrNull(dt_curr, 'US/Samoa') - time_shift AS res_null_sam, + parseDateTimeBestEffortOrNull(dt_curr, 'Pacific/Auckland') - time_shift AS res_null_auc, + parseDateTimeBestEffortOrZero(dt_curr) - time_shift AS res_zero, + parseDateTimeBestEffortOrZero(dt_curr, 'US/Samoa') - time_shift AS res_zero_sam, + parseDateTimeBestEffortOrZero(dt_curr, 'Pacific/Auckland') - time_shift AS res_zero_auc, + parseDateTimeBestEffortUS(dt_curr) - time_shift AS res_us, + parseDateTimeBestEffortUS(dt_curr, 'US/Samoa') - time_shift AS res_us_sam, + parseDateTimeBestEffortUS(dt_curr, 'Pacific/Auckland') - time_shift AS res_us_auc, + parseDateTimeBestEffortUSOrNull(dt_curr) - time_shift AS res_us_null, + parseDateTimeBestEffortUSOrNull(dt_curr, 'US/Samoa') - time_shift AS res_us_null_sam, + parseDateTimeBestEffortUSOrNull(dt_curr, 'Pacific/Auckland') - time_shift AS res_us_null_auc, + parseDateTimeBestEffortUSOrZero(dt_curr) - time_shift AS res_us_zero, + parseDateTimeBestEffortUSOrZero(dt_curr, 'US/Samoa') - time_shift AS res_us_zero_sam, + parseDateTimeBestEffortUSOrZero(dt_curr, 'Pacific/Auckland') - time_shift AS res_us_zero_auc, + parseDateTime64BestEffort(dt_curr) - time_shift AS res64, + parseDateTime64BestEffort(dt_curr, 3, 'US/Samoa') - time_shift AS res64_sam, + parseDateTime64BestEffort(dt_curr, 3, 'Pacific/Auckland') - time_shift AS res64_auc, + parseDateTime64BestEffortOrNull(dt_curr) - time_shift AS res64_null, + parseDateTime64BestEffortOrNull(dt_curr, 3, 'US/Samoa') - time_shift AS res64_null_sam, + parseDateTime64BestEffortOrNull(dt_curr, 3, 'Pacific/Auckland') - time_shift AS res64_null_auc, + parseDateTime64BestEffortOrZero(dt_curr) - time_shift AS res64_zero, + parseDateTime64BestEffortOrZero(dt_curr, 3, 'US/Samoa') - time_shift AS res64_zero_sam, + parseDateTime64BestEffortOrZero(dt_curr, 3, 'Pacific/Auckland') - time_shift AS res64_zero_auc, + parseDateTime64BestEffortUS(dt_curr) - time_shift AS res64_us, + parseDateTime64BestEffortUS(dt_curr, 3, 'US/Samoa') - time_shift AS res64_us_sam, + parseDateTime64BestEffortUS(dt_curr, 3, 'Pacific/Auckland') - time_shift AS res64_us_auc, + parseDateTime64BestEffortUSOrNull(dt_curr) - time_shift AS res64_us_null, + parseDateTime64BestEffortUSOrNull(dt_curr, 3, 'US/Samoa') - time_shift AS res64_us_null_sam, + parseDateTime64BestEffortUSOrNull(dt_curr, 3, 'Pacific/Auckland') - time_shift AS res64_us_null_auc, + parseDateTime64BestEffortUSOrZero(dt_curr) - time_shift AS res64_us_zero, + parseDateTime64BestEffortUSOrZero(dt_curr, 3, 'US/Samoa') - time_shift AS res64_us_zero_sam, + parseDateTime64BestEffortUSOrZero(dt_curr, 3, 'Pacific/Auckland') - time_shift AS res64_us_zero_auc FROM (SELECT arrayJoin([ts_now - secs_in_day, ts_now + secs_in_day]) AS ts_around) FORMAT PrettySpaceNoEscapes; From 38151f9c767d12b18e26ccc236244bc929c326bb Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 13 Jun 2023 14:59:38 +0300 Subject: [PATCH 1547/2223] Rename the test to the snake_case after the Team Lead's review --- ...g.reference => 02783_parsedatetimebesteffort_syslog.reference} | 0 ...Effort_syslog.sql => 02783_parsedatetimebesteffort_syslog.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02783_parseDateTimeBestEffort_syslog.reference => 02783_parsedatetimebesteffort_syslog.reference} (100%) rename tests/queries/0_stateless/{02783_parseDateTimeBestEffort_syslog.sql => 02783_parsedatetimebesteffort_syslog.sql} (100%) diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference b/tests/queries/0_stateless/02783_parsedatetimebesteffort_syslog.reference similarity index 100% rename from tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.reference rename to tests/queries/0_stateless/02783_parsedatetimebesteffort_syslog.reference diff --git a/tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql b/tests/queries/0_stateless/02783_parsedatetimebesteffort_syslog.sql similarity index 100% rename from tests/queries/0_stateless/02783_parseDateTimeBestEffort_syslog.sql rename to tests/queries/0_stateless/02783_parsedatetimebesteffort_syslog.sql From 3e3b8ff5f6b78c6ddd202d154ea9101625c561f1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Jun 2023 09:14:15 +0000 Subject: [PATCH 1548/2223] More robustness --- src/Functions/FunctionSnowflake.h | 40 ++++++++++++++++++++++++++----- 1 file changed, 34 insertions(+), 6 deletions(-) diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index ace2fc54f09..c7ec6dca27f 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -59,9 +60,20 @@ public: auto res_column = ColumnInt64::create(input_rows_count); auto & res_data = res_column->getData(); - const auto & src_data = typeid_cast &>(src_column).getData(); - for (size_t i = 0; i < input_rows_count; ++i) - res_data[i] = (UInt32(src_data[i]) * 1000 - snowflake_epoch) << time_shift; + if (const auto * src_column_non_const = typeid_cast(&src_column)) + { + const auto & src_data = src_column_non_const->getData(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (UInt32(src_data[i]) * 1000 - snowflake_epoch) << time_shift; + } + else if (const auto * src_column_const = typeid_cast(&src_column)) + { + UInt32 src_val = src_column_const->getValue(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (src_val * 1000 - snowflake_epoch) << time_shift; + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for function {}", name); return res_column; } @@ -122,6 +134,9 @@ public: res_data[i] = static_cast( ((src_val >> time_shift) + snowflake_epoch) / 1000); } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for function {}", name); + return res_column; } }; @@ -157,9 +172,20 @@ public: auto res_column = ColumnInt64::create(input_rows_count); auto & res_data = res_column->getData(); - const auto & src_data = typeid_cast &>(src_column).getData(); - for (size_t i = 0; i < input_rows_count; ++i) - res_data[i] = (src_data[i] - snowflake_epoch) << time_shift; + if (const auto * src_column_non_const = typeid_cast(&src_column)) + { + const auto & src_data = src_column_non_const->getData(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (UInt32(src_data[i]) * 1000 - snowflake_epoch) << time_shift; + } + else if (const auto * src_column_const = typeid_cast(&src_column)) + { + UInt32 src_val = src_column_const->getValue(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (src_val * 1000 - snowflake_epoch) << time_shift; + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for function {}", name); return res_column; } @@ -219,6 +245,8 @@ public: for (size_t i = 0; i < input_rows_count; ++i) res_data[i] = (src_val >> time_shift) + snowflake_epoch; } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for function {}", name); return res_column; } From eddd932636fdb16802ec0b541a7cb927abcc05ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 13 Jun 2023 12:34:26 +0000 Subject: [PATCH 1549/2223] Do not apply projection if read-in-order was enabled. --- .../Optimizations/projectionsCommon.cpp | 3 ++ .../QueryPlan/ReadFromMergeTree.cpp | 5 ++ src/Processors/QueryPlan/ReadFromMergeTree.h | 1 + ...84_projections_read_in_order_bug.reference | 0 .../02784_projections_read_in_order_bug.sql | 48 +++++++++++++++++++ 5 files changed, 57 insertions(+) create mode 100644 tests/queries/0_stateless/02784_projections_read_in_order_bug.reference create mode 100644 tests/queries/0_stateless/02784_projections_read_in_order_bug.sql diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 2f73e14b2a0..cb76ffa84ba 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -38,6 +38,9 @@ bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) if (reading->isParallelReadingEnabled()) return false; + if (reading->readsInOrder()) + return false; + // Currently projection don't support deduplication when moving parts between shards. if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) return false; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 2415507a6eb..3c38ecbbd3f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1427,6 +1427,11 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, return true; } +bool ReadFromMergeTree::readsInOrder() const +{ + return reader_settings.read_in_order; +} + void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value) { query_info.prewhere_info = prewhere_info_value; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 45beaaaf013..99cbe9d9e50 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -161,6 +161,7 @@ public: /// Returns `false` if requested reading cannot be performed. bool requestReadingInOrder(size_t prefix_size, int direction, size_t limit); + bool readsInOrder() const; void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value); diff --git a/tests/queries/0_stateless/02784_projections_read_in_order_bug.reference b/tests/queries/0_stateless/02784_projections_read_in_order_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql b/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql new file mode 100644 index 00000000000..52a3a6127ac --- /dev/null +++ b/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql @@ -0,0 +1,48 @@ +create table events ( + `organisation_id` UUID, + `session_id` UUID, + `id` UUID DEFAULT generateUUIDv4(), + `timestamp` UInt64, + `payload` String, + `customer_id` UUID, + `call_id` String, + PROJECTION events_by_session_and_org + ( + SELECT * + ORDER BY + organisation_id, + session_id, + timestamp + ), + PROJECTION events_by_session + ( + SELECT * + ORDER BY + session_id, + timestamp + ), + PROJECTION events_by_session_and_customer + ( + SELECT * + ORDER BY + customer_id, + session_id, + timestamp + ), + PROJECTION events_by_call_id + ( + SELECT * + ORDER BY + call_id, + timestamp + )) engine = MergeTree order by (organisation_id, session_id, timestamp) settings index_granularity = 3; + + +#insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02'), toString(0), reinterpretAsUUID(0), toString(0)); +#insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02'), toString(0), reinterpretAsUUID(0), toString(0)); + +insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)); +insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)); + +set read_in_order_two_level_merge_threshold=1; +SELECT id, timestamp, payload FROM events WHERE (organisation_id = reinterpretAsUUID(1)) AND (session_id = reinterpretAsUUID(0)) ORDER BY timestamp, payload, id ASC; From 1a4b7e8ebec920943a39c484576f147c130b00ec Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Tue, 13 Jun 2023 14:36:31 +0200 Subject: [PATCH 1550/2223] MaterializedMySQL: Add missing DROP DATABASE for tests --- .../materialize_with_ddl.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 2bbbe9a3f13..7efb9ac54a9 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -1476,6 +1476,9 @@ def utf8mb4_test(clickhouse_node, mysql_node, service_name): "1\t\U0001F984\n2\t\u2601\n", ) + clickhouse_node.query("DROP DATABASE utf8mb4_test") + mysql_node.query("DROP DATABASE utf8mb4_test") + def system_parts_test(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS system_parts_test") @@ -1504,6 +1507,9 @@ def system_parts_test(clickhouse_node, mysql_node, service_name): clickhouse_node.query("OPTIMIZE TABLE system_parts_test.test") check_active_parts(1) + clickhouse_node.query("DROP DATABASE system_parts_test") + mysql_node.query("DROP DATABASE system_parts_test") + def multi_table_update_test(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS multi_table_update") @@ -1529,6 +1535,8 @@ def multi_table_update_test(clickhouse_node, mysql_node, service_name): check_query(clickhouse_node, "SELECT * FROM multi_table_update.a", "1\tbaz\n") check_query(clickhouse_node, "SELECT * FROM multi_table_update.b", "1\tquux\n") + clickhouse_node.query("DROP DATABASE multi_table_update") + mysql_node.query("DROP DATABASE multi_table_update") def system_tables_test(clickhouse_node, mysql_node, service_name): @@ -1549,6 +1557,9 @@ def system_tables_test(clickhouse_node, mysql_node, service_name): "intDiv(id, 4294967)\tid\tid\n", ) + clickhouse_node.query("DROP DATABASE system_tables_test") + mysql_node.query("DROP DATABASE system_tables_test") + def materialize_with_column_comments_test(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS materialize_with_column_comments_test") From f4ed10c0a28b52f140f542c7ab0b21e1edf9a0c0 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 13 Jun 2023 14:44:39 +0200 Subject: [PATCH 1551/2223] Update src/Storages/StorageReplicatedMergeTree.cpp --- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fafb3b124f2..84eae32495d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4751,10 +4751,10 @@ void StorageReplicatedMergeTree::read( } else { - header - = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); + header + = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); } auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); From 46c23b3f8d185bf79cb819c2beff0216ca73c4bd Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 15:46:54 +0200 Subject: [PATCH 1552/2223] Fixed docs check fails --- docs/en/sql-reference/table-functions/azureBlobStorage.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 7a362710b9c..8587d9839b8 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -1,5 +1,5 @@ --- -slug: /en/sql-reference/table-functions/azure_blob_storage +slug: /en/sql-reference/table-functions/azureBlobStorageg sidebar_label: azureBlobStorage keywords: [azure blob storage] --- From ab020f9311eea0f657f57493e14191e75e51f8af Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 15:48:42 +0200 Subject: [PATCH 1553/2223] Fixed typo --- docs/en/sql-reference/table-functions/azureBlobStorage.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 8587d9839b8..5175aabd5d1 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -1,5 +1,5 @@ --- -slug: /en/sql-reference/table-functions/azureBlobStorageg +slug: /en/sql-reference/table-functions/azureBlobStorage sidebar_label: azureBlobStorage keywords: [azure blob storage] --- From 3a2fa65075ab32a04a377cef632dd1679dea02b0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 13 Jun 2023 16:02:54 +0200 Subject: [PATCH 1554/2223] fix 'Illegal column timezone' in stress tests --- docker/test/upgrade/run.sh | 6 ++---- tests/config/install.sh | 1 + tests/config/users.d/nonconst_timezone.xml | 7 +++++++ 3 files changed, 10 insertions(+), 4 deletions(-) create mode 100644 tests/config/users.d/nonconst_timezone.xml diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 6f7d3999f1d..951c443c30d 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -59,8 +59,7 @@ install_packages previous_release_package_folder # available for dump via clickhouse-local configure -# it contains some new settings, but we can safely remove it -rm /etc/clickhouse-server/config.d/merge_tree.xml +rm /etc/clickhouse-server/users.d/nonconst_timezone.xml start stop @@ -86,8 +85,7 @@ export USE_S3_STORAGE_FOR_MERGE_TREE=1 export ZOOKEEPER_FAULT_INJECTION=0 configure -# it contains some new settings, but we can safely remove it -rm /etc/clickhouse-server/config.d/merge_tree.xml +rm /etc/clickhouse-server/users.d/nonconst_timezone.xml start diff --git a/tests/config/install.sh b/tests/config/install.sh index efa5a9c086e..b2153db1b2c 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -78,6 +78,7 @@ ln -sf $SRC_PATH/users.d/enable_blobs_check.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/marks.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/insert_keeper_retries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/prefetch_settings.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/nonconst_timezone.xml $DEST_SERVER_PATH/users.d/ if [[ -n "$USE_NEW_ANALYZER" ]] && [[ "$USE_NEW_ANALYZER" -eq 1 ]]; then ln -sf $SRC_PATH/users.d/analyzer.xml $DEST_SERVER_PATH/users.d/ diff --git a/tests/config/users.d/nonconst_timezone.xml b/tests/config/users.d/nonconst_timezone.xml new file mode 100644 index 00000000000..c7e9de5ab69 --- /dev/null +++ b/tests/config/users.d/nonconst_timezone.xml @@ -0,0 +1,7 @@ + + + + 1 + + + From 8ea7560d898879e74887e042aca0a6c60031191b Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Tue, 13 Jun 2023 16:28:53 +0200 Subject: [PATCH 1555/2223] MaterializedMySQL: Add additional test case to insert_with_modify_binlog_checksum (#50884) --- .../materialize_with_ddl.py | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 2bbbe9a3f13..f5c28832f79 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -1050,6 +1050,8 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): def insert_with_modify_binlog_checksum(clickhouse_node, mysql_node, service_name): + clickhouse_node.query("DROP DATABASE IF EXISTS test_checksum") + mysql_node.query("DROP DATABASE IF EXISTS test_checksum") mysql_node.query("CREATE DATABASE test_checksum") mysql_node.query("CREATE TABLE test_checksum.t (a INT PRIMARY KEY, b varchar(200))") clickhouse_node.query( @@ -1081,6 +1083,21 @@ def insert_with_modify_binlog_checksum(clickhouse_node, mysql_node, service_name "1\t1111\n2\t2222\n3\t3333\n", ) + clickhouse_node.query("DROP DATABASE test_checksum") + mysql_node.query("SET GLOBAL binlog_checksum=NONE") + clickhouse_node.query( + "CREATE DATABASE test_checksum ENGINE = MaterializeMySQL('{}:3306', 'test_checksum', 'root', 'clickhouse')".format( + service_name + ) + ) + check_query(clickhouse_node, "SHOW TABLES FROM test_checksum FORMAT TSV", "t\n") + mysql_node.query("INSERT INTO test_checksum.t VALUES(4, '4444')") + check_query( + clickhouse_node, + "SELECT * FROM test_checksum.t ORDER BY a FORMAT TSV", + "1\t1111\n2\t2222\n3\t3333\n4\t4444\n", + ) + clickhouse_node.query("DROP DATABASE test_checksum") mysql_node.query("DROP DATABASE test_checksum") From c253c70510008eda1fc3aadb72cf5c8a92e875bb Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 16:33:36 +0200 Subject: [PATCH 1556/2223] Fix for MDXContent --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 16e244077a7..6b9c3d6157f 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -202,10 +202,10 @@ CHECK length(vectors) = 256`. Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. Larger values mean more accurate results at the cost of longer query runtime: -``` sql +```sql SELECT * FROM table_name ORDER BY L2Distance(vectors, Point) LIMIT N -SETTINGS annoy_index_search_k_nodes=100 +SETTINGS annoy_index_search_k_nodes=100; ``` From 263be33297a2ada5e5c5281924b56e5ffaa3f80f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 13 Jun 2023 16:37:52 +0200 Subject: [PATCH 1557/2223] Fix tests for throttling by allowing more margin of error for trottling event Right now 02703_max_local_write_bandwidth is flaky, and the reason I believe is that the server spent spent sometime somewhere else, which means that the throttler will sleep less. But what is important here is that the overall query duration time matches the expectation, so it is OK to match the LocalWriteThrottlerSleepMicroseconds/LocalReadThrottlerSleepMicroseconds with some error rate. Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02703_max_local_read_bandwidth.sh | 2 +- tests/queries/0_stateless/02703_max_local_write_bandwidth.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh index d47e2f363bd..c78cd202f1b 100755 --- a/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_read_bandwidth.sh @@ -32,7 +32,7 @@ for read_method in "${read_methods[@]}"; do query_duration_ms >= 7e3, ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] > 8e6, ProfileEvents['LocalReadThrottlerBytes'] > 8e6, - ProfileEvents['LocalReadThrottlerSleepMicroseconds'] > 7e6*0.9 + ProfileEvents['LocalReadThrottlerSleepMicroseconds'] > 7e6*0.5 FROM system.query_log WHERE current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' AND type != 'QueryStart' " diff --git a/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh b/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh index 41165d35d37..ccde0903278 100755 --- a/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh +++ b/tests/queries/0_stateless/02703_max_local_write_bandwidth.sh @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -nm -q " query_duration_ms >= 7e3, ProfileEvents['WriteBufferFromFileDescriptorWriteBytes'] > 8e6, ProfileEvents['LocalWriteThrottlerBytes'] > 8e6, - ProfileEvents['LocalWriteThrottlerSleepMicroseconds'] > 7e6*0.9 + ProfileEvents['LocalWriteThrottlerSleepMicroseconds'] > 7e6*0.5 FROM system.query_log WHERE current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' AND type != 'QueryStart' " From 0f4e3a34e846c3a635456dbc8cafa3c12c91155b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 13 Jun 2023 16:42:30 +0200 Subject: [PATCH 1558/2223] Update 02784_projections_read_in_order_bug.sql --- .../0_stateless/02784_projections_read_in_order_bug.sql | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql b/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql index 52a3a6127ac..9595fc9ae08 100644 --- a/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql +++ b/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql @@ -37,10 +37,6 @@ create table events ( timestamp )) engine = MergeTree order by (organisation_id, session_id, timestamp) settings index_granularity = 3; - -#insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02'), toString(0), reinterpretAsUUID(0), toString(0)); -#insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), now(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02'), toString(0), reinterpretAsUUID(0), toString(0)); - insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)); insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)); From 52a460df67f38f92e67316115fde6139cb1c7937 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Jun 2023 16:43:35 +0200 Subject: [PATCH 1559/2223] Tests with parallel replicas are no more "always green" (#50896) --- src/Interpreters/InterpreterSelectQuery.cpp | 6 +-- tests/ci/functional_test_check.py | 42 +++++-------------- .../1_stateful/00013_sorting_of_nested.sql | 3 -- .../1_stateful/00022_merge_prewhere.sql | 2 - .../1_stateful/00042_any_left_join.sql | 2 - .../1_stateful/00043_any_left_join.sql | 2 - .../1_stateful/00044_any_left_join_string.sql | 2 - .../1_stateful/00063_loyalty_joins.sql | 2 - .../00065_loyalty_with_storage_join.sql | 2 - tests/queries/1_stateful/00074_full_join.sql | 2 - .../1_stateful/00075_left_array_join.sql | 2 - ...0079_array_join_not_used_joined_column.sql | 2 - .../1_stateful/00080_array_join_and_union.sql | 2 - .../1_stateful/00084_external_aggregation.sql | 2 - tests/queries/1_stateful/00092_obfuscator.sh | 3 +- .../1_stateful/00096_obfuscator_save_load.sh | 2 - .../00146_aggregate_function_uniq.sql | 2 - .../00149_quantiles_timing_distributed.sql | 2 +- .../00152_insert_different_granularity.sql | 2 +- ...00156_max_execution_speed_sample_merge.sql | 3 -- .../1_stateful/00166_explain_estimate.sql | 2 +- tests/queries/1_stateful/00170_s3_cache.sql | 2 +- ...0171_grouping_aggregated_transform_bug.sql | 2 +- 23 files changed, 19 insertions(+), 74 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e84a400a220..1f95b1ebf9f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -477,7 +477,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Check support for JOIN for parallel replicas with custom key if (joined_tables.tablesCount() > 1 && !settings.parallel_replicas_custom_key.value.empty()) { - LOG_WARNING(log, "JOINs are not supported with parallel_replicas_custom_key. Query will be executed without using them."); + LOG_DEBUG(log, "JOINs are not supported with parallel_replicas_custom_key. Query will be executed without using them."); context->setSetting("parallel_replicas_custom_key", String{""}); } @@ -487,7 +487,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( { if (settings.allow_experimental_parallel_reading_from_replicas == 1) { - LOG_WARNING(log, "FINAL modifier is not supported with parallel replicas. Query will be executed without using them."); + LOG_DEBUG(log, "FINAL modifier is not supported with parallel replicas. Query will be executed without using them."); context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); context->setSetting("parallel_replicas_custom_key", String{""}); } @@ -503,7 +503,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( { if (settings.allow_experimental_parallel_reading_from_replicas == 1) { - LOG_WARNING(log, "To use parallel replicas with plain MergeTree tables please enable setting `parallel_replicas_for_non_replicated_merge_tree`. For now query will be executed without using them."); + LOG_DEBUG(log, "To use parallel replicas with plain MergeTree tables please enable setting `parallel_replicas_for_non_replicated_merge_tree`. For now query will be executed without using them."); context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); } else if (settings.allow_experimental_parallel_reading_from_replicas == 2) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 037bb13f1f8..864c3a81acf 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -378,34 +378,16 @@ def main(): print(f"::notice:: {check_name} Report url: {report_url}") if args.post_commit_status == "commit_status": - if "parallelreplicas" in check_name.lower(): - post_commit_status( - commit, - "success", - report_url, - description, - check_name_with_group, - pr_info, - ) - else: - post_commit_status( - commit, state, report_url, description, check_name_with_group, pr_info - ) + post_commit_status( + commit, state, report_url, description, check_name_with_group, pr_info + ) elif args.post_commit_status == "file": - if "parallelreplicas" in check_name.lower(): - post_commit_status_to_file( - post_commit_path, - description, - "success", - report_url, - ) - else: - post_commit_status_to_file( - post_commit_path, - description, - state, - report_url, - ) + post_commit_status_to_file( + post_commit_path, + description, + state, + report_url, + ) else: raise Exception( f'Unknown post_commit_status option "{args.post_commit_status}"' @@ -423,11 +405,7 @@ def main(): ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) if state != "success": - # Parallel replicas are always green for now - if ( - FORCE_TESTS_LABEL in pr_info.labels - or "parallelreplicas" in check_name.lower() - ): + if FORCE_TESTS_LABEL in pr_info.labels: print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") else: sys.exit(1) diff --git a/tests/queries/1_stateful/00013_sorting_of_nested.sql b/tests/queries/1_stateful/00013_sorting_of_nested.sql index f97120e2b98..7f4a5002a7b 100644 --- a/tests/queries/1_stateful/00013_sorting_of_nested.sql +++ b/tests/queries/1_stateful/00013_sorting_of_nested.sql @@ -1,4 +1 @@ --- Tags: no-parallel-replicas - SELECT ParsedParams.Key1 FROM test.visits FINAL WHERE VisitID != 0 AND notEmpty(ParsedParams.Key1) ORDER BY VisitID LIMIT 10 - diff --git a/tests/queries/1_stateful/00022_merge_prewhere.sql b/tests/queries/1_stateful/00022_merge_prewhere.sql index 400a896d5a8..74a3677b68e 100644 --- a/tests/queries/1_stateful/00022_merge_prewhere.sql +++ b/tests/queries/1_stateful/00022_merge_prewhere.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel-replicas - DROP TABLE IF EXISTS test.merge_hits; CREATE TABLE IF NOT EXISTS test.merge_hits AS test.hits ENGINE = Merge(test, '^hits$'); SELECT count() FROM test.merge_hits WHERE AdvEngineID = 2; diff --git a/tests/queries/1_stateful/00042_any_left_join.sql b/tests/queries/1_stateful/00042_any_left_join.sql index c7c0f0f987a..b87cf88f007 100644 --- a/tests/queries/1_stateful/00042_any_left_join.sql +++ b/tests/queries/1_stateful/00042_any_left_join.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel-replicas - SELECT EventDate, hits, diff --git a/tests/queries/1_stateful/00043_any_left_join.sql b/tests/queries/1_stateful/00043_any_left_join.sql index 6b8cce54051..704d38f727a 100644 --- a/tests/queries/1_stateful/00043_any_left_join.sql +++ b/tests/queries/1_stateful/00043_any_left_join.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel-replicas - SELECT EventDate, count() AS hits, diff --git a/tests/queries/1_stateful/00044_any_left_join_string.sql b/tests/queries/1_stateful/00044_any_left_join_string.sql index ceb7a1c1783..a4f2e9e1b96 100644 --- a/tests/queries/1_stateful/00044_any_left_join_string.sql +++ b/tests/queries/1_stateful/00044_any_left_join_string.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel-replicas - SELECT domain, hits, diff --git a/tests/queries/1_stateful/00063_loyalty_joins.sql b/tests/queries/1_stateful/00063_loyalty_joins.sql index 44f0767a87a..44b575cab85 100644 --- a/tests/queries/1_stateful/00063_loyalty_joins.sql +++ b/tests/queries/1_stateful/00063_loyalty_joins.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel-replicas - SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; diff --git a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index 35f0c7b60b9..a0f41f8aa8d 100644 --- a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel-replicas - USE test; DROP TABLE IF EXISTS join; diff --git a/tests/queries/1_stateful/00074_full_join.sql b/tests/queries/1_stateful/00074_full_join.sql index c1d9e4be1a4..f049be2a74d 100644 --- a/tests/queries/1_stateful/00074_full_join.sql +++ b/tests/queries/1_stateful/00074_full_join.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel-replicas - set any_join_distinct_right_table_keys = 1; set joined_subquery_requires_alias = 0; diff --git a/tests/queries/1_stateful/00075_left_array_join.sql b/tests/queries/1_stateful/00075_left_array_join.sql index 3540d791157..1fd045a26bf 100644 --- a/tests/queries/1_stateful/00075_left_array_join.sql +++ b/tests/queries/1_stateful/00075_left_array_join.sql @@ -1,4 +1,2 @@ --- Tags: no-parallel-replicas - SELECT UserID, EventTime::DateTime('Asia/Dubai'), pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100; SELECT UserID, EventTime::DateTime('Asia/Dubai'), pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits LEFT ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100; diff --git a/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql b/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql index 9431e1cf596..8e6742bb1e1 100644 --- a/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql +++ b/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel-replicas - SELECT PP.Key1 AS `ym:s:paramsLevel1`, sum(arrayAll(`x_1` -> `x_1`= '', ParsedParams.Key2)) AS `ym:s:visits` FROM test.hits ARRAY JOIN ParsedParams AS `PP` WHERE CounterID = 1704509 GROUP BY `ym:s:paramsLevel1` ORDER BY PP.Key1, `ym:s:visits` LIMIT 0, 100; SELECT PP.Key1 AS x1, ParsedParams.Key2 AS x2 FROM test.hits ARRAY JOIN ParsedParams AS PP WHERE CounterID = 1704509 ORDER BY x1, x2 LIMIT 10; SELECT ParsedParams.Key2 AS x FROM test.hits ARRAY JOIN ParsedParams AS PP ORDER BY x DESC LIMIT 10; diff --git a/tests/queries/1_stateful/00080_array_join_and_union.sql b/tests/queries/1_stateful/00080_array_join_and_union.sql index 2f2e5e9324f..d9aa1cc17cc 100644 --- a/tests/queries/1_stateful/00080_array_join_and_union.sql +++ b/tests/queries/1_stateful/00080_array_join_and_union.sql @@ -1,3 +1 @@ --- Tags: no-parallel-replicas - SELECT count() FROM (SELECT Goals.ID FROM test.visits ARRAY JOIN Goals WHERE CounterID = 842440 LIMIT 10 UNION ALL SELECT Goals.ID FROM test.visits ARRAY JOIN Goals WHERE CounterID = 842440 LIMIT 10); diff --git a/tests/queries/1_stateful/00084_external_aggregation.sql b/tests/queries/1_stateful/00084_external_aggregation.sql index 330aa158cf7..b3922eae049 100644 --- a/tests/queries/1_stateful/00084_external_aggregation.sql +++ b/tests/queries/1_stateful/00084_external_aggregation.sql @@ -1,5 +1,3 @@ --- Tags: no-random-settings, no-parallel-replicas - SET max_bytes_before_external_group_by = 200000000; SET max_memory_usage = 1500000000; diff --git a/tests/queries/1_stateful/00092_obfuscator.sh b/tests/queries/1_stateful/00092_obfuscator.sh index f19473f01ac..f9e0098a46c 100755 --- a/tests/queries/1_stateful/00092_obfuscator.sh +++ b/tests/queries/1_stateful/00092_obfuscator.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel-replicas -# clickhouse-local may not work with parallel replicas + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/1_stateful/00096_obfuscator_save_load.sh b/tests/queries/1_stateful/00096_obfuscator_save_load.sh index 1bb212e1bba..a88dfcdb9b9 100755 --- a/tests/queries/1_stateful/00096_obfuscator_save_load.sh +++ b/tests/queries/1_stateful/00096_obfuscator_save_load.sh @@ -1,6 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel-replicas -# clickhouse-local may not work with parallel replicas CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/1_stateful/00146_aggregate_function_uniq.sql b/tests/queries/1_stateful/00146_aggregate_function_uniq.sql index 2cab6e70d22..fd3fde7636d 100644 --- a/tests/queries/1_stateful/00146_aggregate_function_uniq.sql +++ b/tests/queries/1_stateful/00146_aggregate_function_uniq.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel-replicas - SELECT RegionID, uniqHLL12(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; SELECT RegionID, uniqCombined(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; SELECT abs(uniq(WatchID) - uniqExact(WatchID)) FROM test.hits; diff --git a/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql b/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql index 5d2476226ba..6f910646fb7 100644 --- a/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql +++ b/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql @@ -1,4 +1,4 @@ --- Tags: distributed, no-parallel-replicas +-- Tags: distributed SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID); SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID) SETTINGS optimize_aggregation_in_order = 1; diff --git a/tests/queries/1_stateful/00152_insert_different_granularity.sql b/tests/queries/1_stateful/00152_insert_different_granularity.sql index 35483149498..294d71b384b 100644 --- a/tests/queries/1_stateful/00152_insert_different_granularity.sql +++ b/tests/queries/1_stateful/00152_insert_different_granularity.sql @@ -1,4 +1,4 @@ --- Tags: no-tsan, no-replicated-database, no-parallel, no-parallel-replicas +-- Tags: no-tsan, no-replicated-database, no-parallel -- Tag no-replicated-database: Fails due to additional replicas or shards DROP TABLE IF EXISTS fixed_granularity_table; diff --git a/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql index 32079111f6c..e325c18200b 100644 --- a/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql +++ b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql @@ -1,6 +1,3 @@ --- Tags: no-parallel-replicas --- Merge tables doesn't work with parallel replicas currently - SET max_execution_speed = 4000000, timeout_before_checking_execution_speed = 0; CREATE TEMPORARY TABLE times (t DateTime); diff --git a/tests/queries/1_stateful/00166_explain_estimate.sql b/tests/queries/1_stateful/00166_explain_estimate.sql index abac92ecb2e..c4071271736 100644 --- a/tests/queries/1_stateful/00166_explain_estimate.sql +++ b/tests/queries/1_stateful/00166_explain_estimate.sql @@ -1,4 +1,4 @@ --- Tags: no-replicated-database, no-parallel-replicas +-- Tags: no-replicated-database -- Tag no-replicated-database: Requires investigation EXPLAIN ESTIMATE SELECT count() FROM test.hits WHERE CounterID = 29103473; diff --git a/tests/queries/1_stateful/00170_s3_cache.sql b/tests/queries/1_stateful/00170_s3_cache.sql index 43e85af0bc3..23663a1844d 100644 --- a/tests/queries/1_stateful/00170_s3_cache.sql +++ b/tests/queries/1_stateful/00170_s3_cache.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-random-settings, no-parallel-replicas +-- Tags: no-parallel, no-random-settings -- { echo } diff --git a/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql b/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql index 07788af927e..7068780a1b1 100644 --- a/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql +++ b/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql @@ -1,4 +1,4 @@ --- Tags: distributed, no-parallel-replicas +-- Tags: distributed SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID) SETTINGS max_block_size = 63169; SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID) SETTINGS optimize_aggregation_in_order = 1, max_block_size = 63169; From 2e1f56ae336e198d8f388ce815292ec049a7fdc5 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 13 Jun 2023 14:43:50 +0000 Subject: [PATCH 1560/2223] Address comments --- docs/en/engines/table-engines/special/file.md | 2 +- docs/en/operations/settings/settings.md | 6 +- docs/en/sql-reference/table-functions/file.md | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 90 ++++++++-------- src/Storages/StorageFile.cpp | 29 ++--- src/Storages/StorageS3.cpp | 92 +++++++++------- src/Storages/StorageURL.cpp | 101 ++++++++++-------- src/Storages/StorageURL.h | 2 +- 8 files changed, 177 insertions(+), 147 deletions(-) diff --git a/docs/en/engines/table-engines/special/file.md b/docs/en/engines/table-engines/special/file.md index cf325961b6a..27945b30c03 100644 --- a/docs/en/engines/table-engines/special/file.md +++ b/docs/en/engines/table-engines/special/file.md @@ -99,4 +99,4 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da - [engine_file_truncate_on_insert](/docs/en/operations/settings/settings.md#engine-file-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. - [engine_file_allow_create_multiple_files](/docs/en/operations/settings/settings.md#engine_file_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. - [engine_file_skip_empty_files](/docs/en/operations/settings/settings.md#engine_file_skip_empty_files) - allows to skip empty files while reading. Disabled by default. -- [storage_file_read_method](/docs/en/operations/settings/settings.md#engine-file-emptyif-not-exists) - method of reading data from storage file, one of: read, pread, mmap (only for clickhouse-local). Default value: `pread` for clickhouse-server, `mmap` for clickhouse-local. +- [storage_file_read_method](/docs/en/operations/settings/settings.md#engine-file-emptyif-not-exists) - method of reading data from storage file, one of: `read`, `pread`, `mmap`. The mmap method does not apply to clickhouse-server (it's intended for clickhouse-local). Default value: `pread` for clickhouse-server, `mmap` for clickhouse-local. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index a138f8f5515..7a28e33bf90 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3332,7 +3332,7 @@ Enables or disables creating a new file on each insert in file engine tables if Possible values: - 0 — `INSERT` query appends new data to the end of the file. -- 1 — `INSERT` query replaces existing content of the file with the new data. +- 1 — `INSERT` query creates a new file. Default value: `0`. @@ -3370,7 +3370,7 @@ initial: `data.Parquet.gz` -> `data.1.Parquet.gz` -> `data.2.Parquet.gz`, etc. Possible values: - 0 — `INSERT` query appends new data to the end of the file. -- 1 — `INSERT` query replaces existing content of the file with the new data. +- 1 — `INSERT` query creates a new file. Default value: `0`. @@ -3402,7 +3402,7 @@ initial: `data.Parquet.gz` -> `data.1.Parquet.gz` -> `data.2.Parquet.gz`, etc. Possible values: - 0 — `INSERT` query appends new data to the end of the file. -- 1 — `INSERT` query replaces existing content of the file with the new data. +- 1 — `INSERT` query creates a new file. Default value: `0`. diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 749aafb6d00..f25da96fddb 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -18,7 +18,7 @@ file(path [,format] [,structure] [,compression]) **Parameters** -- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-user_files_path). Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. - `format` — The [format](/docs/en/interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format: `'column1_name column1_type, column2_name column2_type, ...'`. - `compression` — The existing compression type when used in a `SELECT` query, or the desired compression type when used in an `INSERT` query. The supported compression types are `gz`, `br`, `xz`, `zst`, `lz4`, and `bz2`. diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 9ea1d805db5..dd3e8fecfaa 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -140,14 +140,6 @@ namespace return LSWithRegexpMatching("/", fs, path_from_uri); } - - size_t getFileSize(const String & path_from_uri, const String & uri_without_path, ContextPtr context) - { - HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); - HDFSFSPtr fs = createHDFSFS(builder.get()); - auto * info = hdfsGetPathInfo(fs.get(), path_from_uri.data()); - return info->mSize; - } } StorageHDFS::StorageHDFS( @@ -218,26 +210,36 @@ ColumnsDescription StorageHDFS::getTableStructureFromData( ReadBufferIterator read_buffer_iterator = [&, my_uri_without_path = uri_without_path, it = paths_with_info.begin(), first = true]( - ColumnsDescription & columns) mutable -> std::unique_ptr + ColumnsDescription &) mutable -> std::unique_ptr { - if (it == paths_with_info.end()) + PathWithInfo path_with_info; + std::unique_ptr buf; + while (true) { - if (first) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file, because all files are empty. " - "You must specify table structure manually", format); - return nullptr; + if (it == paths_with_info.end()) + { + if (first) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because all files are empty. " + "You must specify table structure manually", format); + return nullptr; + } + + path_with_info = *it++; + if (ctx->getSettingsRef().hdfs_skip_empty_files && path_with_info.info && path_with_info.info->size == 0) + continue; + + auto compression = chooseCompressionMethod(path_with_info.path, compression_method); + auto impl = std::make_unique(my_uri_without_path, path_with_info.path, ctx->getGlobalContext()->getConfigRef(), ctx->getReadSettings()); + const Int64 zstd_window_log_max = ctx->getSettingsRef().zstd_window_log_max; + buf = wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); + + if (!ctx->getSettingsRef().hdfs_skip_empty_files || !buf->eof()) + { + first = false; + return buf; + } } - - auto path_with_info = *it++; - if (ctx->getSettingsRef().hdfs_skip_empty_files && path_with_info.info && path_with_info.info->size == 0) - return read_buffer_iterator(columns); - - first = false; - auto compression = chooseCompressionMethod(path_with_info.path, compression_method); - auto impl = std::make_unique(my_uri_without_path, path_with_info.path, ctx->getGlobalContext()->getConfigRef(), ctx->getReadSettings()); - const Int64 zstd_window_log_max = ctx->getSettingsRef().zstd_window_log_max; - return wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); }; ColumnsDescription columns; @@ -362,26 +364,28 @@ HDFSSource::HDFSSource( bool HDFSSource::initialize() { - auto path_with_info = (*file_iterator)(); - if (path_with_info.path.empty()) - return false; - - current_path = path_with_info.path; - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_path); - - if (getContext()->getSettingsRef().hdfs_skip_empty_files) + StorageHDFS::PathWithInfo path_with_info; + bool skip_empty_files = getContext()->getSettingsRef().hdfs_skip_empty_files; + while (true) { - auto file_size = path_with_info.info ? path_with_info.info->size : getFileSize(path_from_uri, uri_without_path, getContext()); - /// If file is empty and hdfs_skip_empty_files=1, skip it and go to the next file. - if (file_size == 0) - return initialize(); - } + path_with_info = (*file_iterator)(); + if (path_with_info.path.empty()) + return false; - auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); - auto impl = std::make_unique( - uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); - const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - read_buf = wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); + if (path_with_info.info && skip_empty_files && path_with_info.info->size == 0) + continue; + + current_path = path_with_info.path; + const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_path); + + auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); + auto impl = std::make_unique( + uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); + const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; + read_buf = wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); + if (!skip_empty_files || !read_buf->eof()) + break; + } auto input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7fc143a6122..06f9d071706 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -404,21 +404,26 @@ ColumnsDescription StorageFile::getTableStructureFromFile( if (context->getSettingsRef().schema_inference_use_cache_for_file) columns_from_cache = tryGetColumnsFromCache(paths, format, format_settings, context); - ReadBufferIterator read_buffer_iterator = [&, it = paths.begin(), first = true](ColumnsDescription & columns) mutable -> std::unique_ptr + ReadBufferIterator read_buffer_iterator = [&, it = paths.begin(), first = true](ColumnsDescription &) mutable -> std::unique_ptr { - if (it == paths.end()) + String path; + struct stat file_stat; + do { - if (first) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file, because all files are empty. You must specify table structure manually", - format); - return nullptr; - } + if (it == paths.end()) + { + if (first) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because all files are empty. You must specify table structure manually", + format); + return nullptr; + } - auto path = *it++; - auto file_stat = getFileStat(path, false, -1, "File"); - if (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) - return read_buffer_iterator(columns); + path = *it++; + file_stat = getFileStat(path, false, -1, "File"); + } + while (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0); first = false; return createReadBuffer(path, file_stat, false, -1, compression_method, context); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index c30973d99e1..5c5895744ac 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -575,19 +575,21 @@ StorageS3Source::StorageS3Source( StorageS3Source::ReaderHolder StorageS3Source::createReader() { - auto [current_key, info] = (*file_iterator)(); - if (current_key.empty()) - return {}; + KeyWithInfo key_with_info; + size_t object_size; + do + { + key_with_info = (*file_iterator)(); + if (key_with_info.key.empty()) + return {}; - size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); + object_size = key_with_info.info ? key_with_info.info->size : S3::getObjectSize(*client, bucket, key_with_info.key, version_id, request_settings); + } + while (getContext()->getSettingsRef().s3_skip_empty_files && object_size == 0); - /// If object is empty and s3_skip_empty_files=1, skip it and go to the next key. - if (getContext()->getSettingsRef().s3_skip_empty_files && object_size == 0) - return createReader(); + auto compression_method = chooseCompressionMethod(key_with_info.key, compression_hint); - auto compression_method = chooseCompressionMethod(current_key, compression_hint); - - auto read_buf = createS3ReadBuffer(current_key, object_size); + auto read_buf = createS3ReadBuffer(key_with_info.key, object_size); auto input_format = FormatFactory::instance().getInput( format, *read_buf, sample_block, getContext(), max_block_size, format_settings, std::nullopt, std::nullopt, @@ -606,7 +608,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(bucket) / current_key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{fs::path(bucket) / key_with_info.key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() @@ -1451,41 +1453,53 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( ReadBufferIterator read_buffer_iterator = [&, first = true](ColumnsDescription & cached_columns) mutable -> std::unique_ptr { - auto [key, info] = (*file_iterator)(); + StorageS3Source::KeyWithInfo key_with_info; + std::unique_ptr buf; - if (key.empty()) + while (true) { - if (first) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file, because there are no files with provided path " - "in S3 or all files are empty. You must specify table structure manually", configuration.format); + key_with_info = (*file_iterator)(); - return nullptr; - } - - if (ctx->getSettingsRef().s3_skip_empty_files && info->size == 0) - return read_buffer_iterator(cached_columns); - - /// S3 file iterator could get new keys after new iteration, check them in schema cache. - if (ctx->getSettingsRef().schema_inference_use_cache_for_s3 && read_keys.size() > prev_read_keys_size) - { - columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), configuration, format_settings, ctx); - prev_read_keys_size = read_keys.size(); - if (columns_from_cache) + if (key_with_info.key.empty()) { - cached_columns = *columns_from_cache; + if (first) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because there are no files with provided path " + "in S3 or all files are empty. You must specify table structure manually", + configuration.format); + return nullptr; } - } - first = false; - int zstd_window_log_max = static_cast(ctx->getSettingsRef().zstd_window_log_max); - return wrapReadBufferWithCompressionMethod( - std::make_unique( - configuration.client, configuration.url.bucket, key, configuration.url.version_id, configuration.request_settings, ctx->getReadSettings()), - chooseCompressionMethod(key, configuration.compression_method), - zstd_window_log_max); + /// S3 file iterator could get new keys after new iteration, check them in schema cache. + if (ctx->getSettingsRef().schema_inference_use_cache_for_s3 && read_keys.size() > prev_read_keys_size) + { + columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), configuration, format_settings, ctx); + prev_read_keys_size = read_keys.size(); + if (columns_from_cache) + { + cached_columns = *columns_from_cache; + return nullptr; + } + } + + if (ctx->getSettingsRef().s3_skip_empty_files && key_with_info.info && key_with_info.info->size == 0) + continue; + + int zstd_window_log_max = static_cast(ctx->getSettingsRef().zstd_window_log_max); + buf = wrapReadBufferWithCompressionMethod( + std::make_unique( + configuration.client, configuration.url.bucket, key_with_info.key, configuration.url.version_id, configuration.request_settings, ctx->getReadSettings()), + chooseCompressionMethod(key_with_info.key, configuration.compression_method), + zstd_window_log_max); + + if (!ctx->getSettingsRef().s3_skip_empty_files || !buf->eof()) + { + first = false; + return buf; + } + } }; ColumnsDescription columns; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index fc5525b42d2..4e75a4d54cb 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -241,30 +241,34 @@ StorageURLSource::StorageURLSource( /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. initialize = [=, this]() { - const auto current_uri_options = (*uri_iterator)(); - if (current_uri_options.empty()) - return false; + std::vector current_uri_options; + std::pair> uri_and_buf; + do + { + current_uri_options = (*uri_iterator)(); + if (current_uri_options.empty()) + return false; - auto first_option = uri_options.begin(); - auto [actual_uri, buf] = getFirstAvailableURIAndReadBuffer( - first_option, - current_uri_options.end(), - context, - params, - http_method, - callback, - timeouts, - credentials, - headers, - glob_url, - current_uri_options.size() == 1); + auto first_option = current_uri_options.cbegin(); + uri_and_buf = getFirstAvailableURIAndReadBuffer( + first_option, + current_uri_options.end(), + context, + params, + http_method, + callback, + timeouts, + credentials, + headers, + glob_url, + current_uri_options.size() == 1); - /// If file is empty and engine_url_skip_empty_files=1, skip it and go to the next file. - if (context->getSettingsRef().engine_url_skip_empty_files && getFileSizeFromReadBuffer(*buf) == 0) - return initialize(); + /// If file is empty and engine_url_skip_empty_files=1, skip it and go to the next file. + } + while (context->getSettingsRef().engine_url_skip_empty_files && uri_and_buf.second->eof()); - curr_uri = actual_uri; - read_buf = std::move(buf); + curr_uri = uri_and_buf.first; + read_buf = std::move(uri_and_buf.second); try { @@ -347,7 +351,7 @@ Chunk StorageURLSource::generate() return {}; } -std::tuple> StorageURLSource::getFirstAvailableURIAndReadBuffer( +std::pair> StorageURLSource::getFirstAvailableURIAndReadBuffer( std::vector::const_iterator & option, const std::vector::const_iterator & end, ContextPtr context, @@ -590,38 +594,41 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( if (context->getSettingsRef().schema_inference_use_cache_for_url) columns_from_cache = tryGetColumnsFromCache(urls_to_check, headers, credentials, format, format_settings, context); - ReadBufferIterator read_buffer_iterator = [&, it = urls_to_check.cbegin(), first = true](ColumnsDescription & columns) mutable -> std::unique_ptr + ReadBufferIterator read_buffer_iterator = [&, it = urls_to_check.cbegin(), first = true](ColumnsDescription &) mutable -> std::unique_ptr { - if (it == urls_to_check.cend()) + std::pair> uri_and_buf; + do { - if (first) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file, because all files are empty. " - "You must specify table structure manually", format); - return nullptr; - } + if (it == urls_to_check.cend()) + { + if (first) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because all files are empty. " + "You must specify table structure manually", + format); + return nullptr; + } - auto [_, buf] = StorageURLSource::getFirstAvailableURIAndReadBuffer( - it, - urls_to_check.cend(), - context, - {}, - Poco::Net::HTTPRequest::HTTP_GET, - {}, - getHTTPTimeouts(context), - credentials, - headers, - false, - false); + uri_and_buf = StorageURLSource::getFirstAvailableURIAndReadBuffer( + it, + urls_to_check.cend(), + context, + {}, + Poco::Net::HTTPRequest::HTTP_GET, + {}, + getHTTPTimeouts(context), + credentials, + headers, + false, + false); - ++it; - - if (context->getSettingsRef().engine_url_skip_empty_files && buf_factory->getFileSize() == 0) - return read_buffer_iterator(columns); + ++it; + } while (context->getSettingsRef().engine_url_skip_empty_files && uri_and_buf.second->eof()); first = false; return wrapReadBufferWithCompressionMethod( - std::move(buf), + std::move(uri_and_buf.second), compression_method, static_cast(context->getSettingsRef().zstd_window_log_max)); }; diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 50928ed6962..a5c1174377b 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -183,7 +183,7 @@ public: static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); - static std::tuple> getFirstAvailableURIAndReadBuffer( + static std::pair> getFirstAvailableURIAndReadBuffer( std::vector::const_iterator & option, const std::vector::const_iterator & end, ContextPtr context, From 7057e0e25fd55f8a9cb9708da223883aaa8fe902 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Jun 2023 14:46:26 +0000 Subject: [PATCH 1561/2223] fix test --- .../02751_ip_types_aggregate_functions_states.sql.j2 | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 index 708eeab7724..7daff5a690f 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -3,8 +3,8 @@ {# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} -{% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,50) order by ip" %} -{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 50) order by ip" %} +{% set ip4_generator = "select num::UInt32::IPv4 ip from (select arrayJoin(range(999999999, number)) as num from numbers(999999999,50)) order by ip" %} +{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(num)||reinterpretAsFixedString(num), 16))) AS ip FROM (select arrayJoin(range(1010011101, number)) as num from numbers(1010011101,50)) order by ip" %} {% set ip_generators = {'ip4': ip4_generator, 'ip6': ip6_generator} %} From 332893344d3cbca205b0d99671cd4c8ba26ec2da Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 16:50:10 +0200 Subject: [PATCH 1562/2223] Updated lock for accessing azure blob storage iterator --- src/Storages/StorageAzureBlob.cpp | 6 +----- tests/integration/test_storage_azure_blob_storage/test.py | 1 - 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 3ee176a68b7..b9d59f04001 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -882,6 +882,7 @@ StorageAzureBlobSource::Iterator::Iterator( RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { + std::lock_guard lock(next_mutex); if (is_finished) return {}; @@ -900,7 +901,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { bool need_new_batch = false; { - std::lock_guard lock(next_mutex); need_new_batch = !blobs_with_metadata || index >= blobs_with_metadata->size(); } @@ -945,7 +945,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); const auto & idxs = typeid_cast(*block.getByName("_idx").column); - std::lock_guard lock(next_mutex); blob_path_with_globs.reset(); blob_path_with_globs.emplace(); for (UInt64 idx : idxs.getData()) @@ -961,7 +960,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() if (outer_blobs) outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); - std::lock_guard lock(next_mutex); blobs_with_metadata = std::move(new_batch); for (const auto & [_, info] : *blobs_with_metadata) total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); @@ -969,8 +967,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() } size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - - std::lock_guard lock(next_mutex); return (*blobs_with_metadata)[current_index]; } } diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f9d337b6d86..bb25ac4b029 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -551,7 +551,6 @@ def test_schema_inference_no_globs_tf(cluster): "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] - def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] unique_prefix = random.randint(1, 10000) From 3fd9911efe37500094532d07a1f6e0eaddaca6dd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Jun 2023 14:58:55 +0000 Subject: [PATCH 1563/2223] fix test reference --- ...es_aggregate_functions_states.reference.j2 | 164 +++++++++--------- 1 file changed, 82 insertions(+), 82 deletions(-) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 index 481dd723b66..90f98cf63fd 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 @@ -2,18 +2,18 @@ Row 1: ────── minState: 12535288824949910799 -maxState: 18210943739258811465 +maxState: 15790547582231549775 first_valueState: 12535288824949910799 -last_valueState: 18210943739258811465 -topKState: 1594227852744382511 -groupArrayState: 8025417272361615478 -groupUniqArrayState: 919082878249747568 -uniqState: 14828781561416784358 -uniqExactState: 11391659146320471795 -uniqCombinedState: 9631896280254268221 -uniqCombined64State: 5156097536649078816 -uniqHLL12State: 9696624347265201099 -uniqThetaState: 10464560810701154023 +last_valueState: 15790547582231549775 +topKState: 4906125994014190470 +groupArrayState: 9875990674330641453 +groupUniqArrayState: 15788623094139372318 +uniqState: 13857072740564896081 +uniqExactState: 2764760571052675772 +uniqCombinedState: 927481020821507998 +uniqCombined64State: 11588178464185397904 +uniqHLL12State: 592773541433144605 +uniqThetaState: 12573391720108828030 ----- hash / State / ip6 ----- Row 1: ────── @@ -21,31 +21,31 @@ minState: 9428555662807296659 maxState: 18253481702148601156 first_valueState: 9428555662807296659 last_valueState: 18253481702148601156 -topKState: 8045294331733869941 -groupArrayState: 10451014709837753966 -groupUniqArrayState: 1954028114836070615 -uniqState: 14986562136250471284 -uniqExactState: 10032843621916709112 -uniqCombinedState: 6379274083567016598 -uniqCombined64State: 6379274083567016598 -uniqHLL12State: 9181286681186915812 -uniqThetaState: 2415188383468008881 +topKState: 4649040466816645853 +groupArrayState: 15334593495826890008 +groupUniqArrayState: 18179202420787216155 +uniqState: 1113473461736161202 +uniqExactState: 17291302630176581193 +uniqCombinedState: 7689865507370303115 +uniqCombined64State: 7689865507370303115 +uniqHLL12State: 12630756003012135681 +uniqThetaState: 11768246996604802350 ----- finalizeAggregation / State / ip4 ----- Row 1: ────── min: 59.154.201.255 -max: 59.154.202.48 +max: 59.154.202.47 first_value: 59.154.201.255 -last_value: 59.154.202.48 -topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] -groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] -groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 49 -uniqTheta: 50 +last_value: 59.154.202.47 +topK: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8'] +groupArray: ['59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.45','59.154.202.45','59.154.202.45','59.154.202.46','59.154.202.46','59.154.202.47'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 48 +uniqTheta: 49 ----- finalizeAggregation / State / ip6 ----- Row 1: ────── @@ -53,31 +53,31 @@ min: 8c:333c::8c:333c:0:0 max: ff8b:333c::ff8b:333c:0:0 first_value: 8c:333c::8c:333c:0:0 last_value: ff8b:333c::ff8b:333c:0:0 -topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] -groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 50 -uniqTheta: 50 +topK: ['dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 49 +uniqTheta: 49 ----- hash / IfState / ip4 ----- Row 1: ────── minIfState: 12535288824949910799 -maxIfState: 18210943739258811465 +maxIfState: 15790547582231549775 first_valueIfState: 12535288824949910799 -last_valueIfState: 18210943739258811465 -topKIfState: 1594227852744382511 -groupArrayIfState: 8025417272361615478 -groupUniqArrayIfState: 919082878249747568 -uniqIfState: 14828781561416784358 -uniqExactIfState: 11391659146320471795 -uniqCombinedIfState: 9631896280254268221 -uniqCombined64IfState: 5156097536649078816 -uniqHLL12IfState: 9696624347265201099 -uniqThetaIfState: 10464560810701154023 +last_valueIfState: 15790547582231549775 +topKIfState: 4906125994014190470 +groupArrayIfState: 9875990674330641453 +groupUniqArrayIfState: 15788623094139372318 +uniqIfState: 13857072740564896081 +uniqExactIfState: 2764760571052675772 +uniqCombinedIfState: 927481020821507998 +uniqCombined64IfState: 11588178464185397904 +uniqHLL12IfState: 592773541433144605 +uniqThetaIfState: 12573391720108828030 ----- hash / IfState / ip6 ----- Row 1: ────── @@ -85,31 +85,31 @@ minIfState: 9428555662807296659 maxIfState: 18253481702148601156 first_valueIfState: 9428555662807296659 last_valueIfState: 18253481702148601156 -topKIfState: 8045294331733869941 -groupArrayIfState: 10451014709837753966 -groupUniqArrayIfState: 1954028114836070615 -uniqIfState: 14986562136250471284 -uniqExactIfState: 10032843621916709112 -uniqCombinedIfState: 6379274083567016598 -uniqCombined64IfState: 6379274083567016598 -uniqHLL12IfState: 9181286681186915812 -uniqThetaIfState: 2415188383468008881 +topKIfState: 4649040466816645853 +groupArrayIfState: 15334593495826890008 +groupUniqArrayIfState: 18179202420787216155 +uniqIfState: 1113473461736161202 +uniqExactIfState: 17291302630176581193 +uniqCombinedIfState: 7689865507370303115 +uniqCombined64IfState: 7689865507370303115 +uniqHLL12IfState: 12630756003012135681 +uniqThetaIfState: 11768246996604802350 ----- finalizeAggregation / IfState / ip4 ----- Row 1: ────── min: 59.154.201.255 -max: 59.154.202.48 +max: 59.154.202.47 first_value: 59.154.201.255 -last_value: 59.154.202.48 -topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] -groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] -groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 49 -uniqTheta: 50 +last_value: 59.154.202.47 +topK: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8'] +groupArray: ['59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.45','59.154.202.45','59.154.202.45','59.154.202.46','59.154.202.46','59.154.202.47'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 48 +uniqTheta: 49 ----- finalizeAggregation / IfState / ip6 ----- Row 1: ────── @@ -117,20 +117,20 @@ min: 8c:333c::8c:333c:0:0 max: ff8b:333c::ff8b:333c:0:0 first_value: 8c:333c::8c:333c:0:0 last_value: ff8b:333c::ff8b:333c:0:0 -topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] -groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 50 -uniqTheta: 50 +topK: ['dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 49 +uniqTheta: 49 ----- Arg / hash / State / ip4 ----- Row 1: ────── argMinState: 13774589216353164344 -argMaxState: 9177365218111013695 +argMaxState: 1131101663917664667 ----- Arg / hash / State / ip6 ----- Row 1: ────── @@ -140,7 +140,7 @@ argMaxState: 16598449636475438091 Row 1: ────── argMinState: 59.154.201.255 -argMaxState: 59.154.202.48 +argMaxState: 59.154.202.47 ----- Arg / finalizeAggregation / State / ip6 ----- Row 1: ────── From 46fbe7fb26d06abdb8f5e21f00d5dd215a084b9b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Jun 2023 15:01:51 +0000 Subject: [PATCH 1564/2223] 01746_convert_type_with_default: Temporarily disable flaky test --- .../0_stateless/01746_convert_type_with_default.reference | 1 - tests/queries/0_stateless/01746_convert_type_with_default.sql | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.reference b/tests/queries/0_stateless/01746_convert_type_with_default.reference index e5aa42e6116..0edea4de31e 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.reference +++ b/tests/queries/0_stateless/01746_convert_type_with_default.reference @@ -40,7 +40,6 @@ 1970-01-20 1970-01-20 2149-06-06 -1970-01-02 2023-05-30 2023-05-30 2023-05-30 14:38:20 diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index e6e420ae4c0..c74b185f7fd 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -54,7 +54,7 @@ select toDateOrDefault(cast(19 as Int256)); select toDateOrDefault(cast(19 as UInt256)); select toDateOrDefault(65535); -select toDateOrDefault(122400); +-- select toDateOrDefault(122400); select toDateOrDefault(19507, '2000-01-01'::Date); select toDateOrDefault(-1, '2023-05-30'::Date); @@ -80,4 +80,4 @@ select toDateTimeOrDefault(cast(19 as Int128), 'UTC'); select toDateTimeOrDefault(cast(19 as UInt128), 'UTC'); select toDateTimeOrDefault(cast(19 as Int256), 'UTC'); -select toDateTimeOrDefault(cast(19 as UInt256), 'UTC'); \ No newline at end of file +select toDateTimeOrDefault(cast(19 as UInt256), 'UTC'); From 478bad32376ff2787e83107c8274f4b743c569ac Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Jun 2023 15:04:11 +0000 Subject: [PATCH 1565/2223] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index bb25ac4b029..f9d337b6d86 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -551,6 +551,7 @@ def test_schema_inference_no_globs_tf(cluster): "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] + def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] unique_prefix = random.randint(1, 10000) From bc4724490239ea34b4924da17ddabbb1f90e2bee Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 17:06:40 +0200 Subject: [PATCH 1566/2223] Updated tests for CI checks --- tests/integration/test_storage_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f9d337b6d86..8ab5d416b03 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -558,6 +558,7 @@ def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" + for i in range(10): for j in range(10): path = "{}/{}_{}/{}.csv".format( From 57cdd3a25d25ca8274e3b68cf75cbaa9bf94daa7 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Tue, 13 Jun 2023 09:13:13 -0600 Subject: [PATCH 1567/2223] Update annindexes.md --- .../table-engines/mergetree-family/annindexes.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 16e244077a7..fe971571419 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -54,7 +54,7 @@ CREATE TABLE table ( `id` Int64, `vectors` Array(Float32), - INDEX vectors TYPE () [GRANULARITY ] + INDEX [ann_index_name vectors TYPE [ann_index_type]([ann_index_parameters]) [GRANULARITY [N]] ) ENGINE = MergeTree ORDER BY id; @@ -67,7 +67,7 @@ CREATE TABLE table ( `id` Int64, `vectors` Tuple(Float32[, Float32[, ...]]), - INDEX vectors TYPE () [GRANULARITY ] + INDEX [ann_index_name] vectors TYPE [ann_index_type]([ann_index_parameters]) [GRANULARITY [N]] ) ENGINE = MergeTree ORDER BY id; @@ -114,7 +114,7 @@ without `LIMIT` clause cannot utilize ANN indexes. Also ANN indexes are only use approximate neighbor search. **Differences to Skip Indexes** Similar to regular [skip indexes](https://clickhouse.com/docs/en/optimize/skipping-indexes), ANN indexes are -constructed over granules and each indexed block consists of `GRANULARITY = `-many granules (`` = 1 by default for normal skip +constructed over granules and each indexed block consists of `GRANULARITY = [N]`-many granules (`[N]` = 1 by default for normal skip indexes). For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`, then each indexed block will contain 16384 rows. However, data structures and algorithms for approximate neighborhood search (usually provided by external libraries) are inherently row-oriented. They store a compact representation of a set of rows and also return rows for @@ -130,7 +130,7 @@ skip data at the granularity of index blocks. The `GRANULARITY` parameter determines how many ANN sub-indexes are created. Bigger `GRANULARITY` values mean fewer but larger ANN sub-indexes, up to the point where a column (or a column's data part) has only a single sub-index. In that case, the sub-index has a "global" view of all column rows and can directly return all granules of the column (part) with relevant rows (there are at most `LIMIT -`-many such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a +[N]`-many such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a brute-force distance calculation over all rows of the granules. With a small `GRANULARITY` value, each of the sub-indexes returns up to `LIMIT N`-many granules. As a result, more granules need to be loaded and post-filtered. Note that the search accuracy is with both cases equally good, only the processing performance differs. It is generally recommended to use a large `GRANULARITY` for ANN indexes and fall @@ -169,7 +169,7 @@ CREATE TABLE table ( id Int64, vectors Array(Float32), - INDEX vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N] + INDEX [ann_index_name] vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N] ) ENGINE = MergeTree ORDER BY id; @@ -182,7 +182,7 @@ CREATE TABLE table ( id Int64, vectors Tuple(Float32[, Float32[, ...]]), - INDEX vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N] + INDEX [ann_index_name] vectors TYPE annoy([Distance[, NumTrees]]) [GRANULARITY N] ) ENGINE = MergeTree ORDER BY id; From 20ea87e527eb76bb296a46f0deab59abdd4a4325 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Tue, 13 Jun 2023 11:17:33 -0400 Subject: [PATCH 1568/2223] Update annindexes.md Don't break code snippets across lines. --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 16e244077a7..20e49f1c34c 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -129,8 +129,8 @@ skip data at the granularity of index blocks. The `GRANULARITY` parameter determines how many ANN sub-indexes are created. Bigger `GRANULARITY` values mean fewer but larger ANN sub-indexes, up to the point where a column (or a column's data part) has only a single sub-index. In that case, the sub-index has a -"global" view of all column rows and can directly return all granules of the column (part) with relevant rows (there are at most `LIMIT -`-many such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a +"global" view of all column rows and can directly return all granules of the column (part) with relevant rows (there are at most +`LIMIT `-many such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a brute-force distance calculation over all rows of the granules. With a small `GRANULARITY` value, each of the sub-indexes returns up to `LIMIT N`-many granules. As a result, more granules need to be loaded and post-filtered. Note that the search accuracy is with both cases equally good, only the processing performance differs. It is generally recommended to use a large `GRANULARITY` for ANN indexes and fall From 8dde50eb3aeef2bd83a1c7c0ec244dfe1c66bf32 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Jun 2023 15:19:16 +0000 Subject: [PATCH 1569/2223] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 8ab5d416b03..3d9c751be3c 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -558,7 +558,7 @@ def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" - + for i in range(10): for j in range(10): path = "{}/{}_{}/{}.csv".format( From b850f1b9995d981d7ec047df8261fec302d3020a Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 13 Jun 2023 11:26:12 -0400 Subject: [PATCH 1570/2223] fix broken line --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index fe971571419..71e7e008bf2 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -129,8 +129,7 @@ skip data at the granularity of index blocks. The `GRANULARITY` parameter determines how many ANN sub-indexes are created. Bigger `GRANULARITY` values mean fewer but larger ANN sub-indexes, up to the point where a column (or a column's data part) has only a single sub-index. In that case, the sub-index has a -"global" view of all column rows and can directly return all granules of the column (part) with relevant rows (there are at most `LIMIT -[N]`-many such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a +"global" view of all column rows and can directly return all granules of the column (part) with relevant rows (there are at most `LIMIT [N]`-many such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a brute-force distance calculation over all rows of the granules. With a small `GRANULARITY` value, each of the sub-indexes returns up to `LIMIT N`-many granules. As a result, more granules need to be loaded and post-filtered. Note that the search accuracy is with both cases equally good, only the processing performance differs. It is generally recommended to use a large `GRANULARITY` for ANN indexes and fall From 76f69f2b44ba23dbb0afd50f26dd3fd62352a381 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 13 Jun 2023 15:52:06 +0000 Subject: [PATCH 1571/2223] Revert overengineering --- src/Functions/FunctionSnowflake.h | 34 ++++++------------------------- 1 file changed, 6 insertions(+), 28 deletions(-) diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index c7ec6dca27f..b57e21e9a50 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -60,20 +60,9 @@ public: auto res_column = ColumnInt64::create(input_rows_count); auto & res_data = res_column->getData(); - if (const auto * src_column_non_const = typeid_cast(&src_column)) - { - const auto & src_data = src_column_non_const->getData(); - for (size_t i = 0; i < input_rows_count; ++i) - res_data[i] = (UInt32(src_data[i]) * 1000 - snowflake_epoch) << time_shift; - } - else if (const auto * src_column_const = typeid_cast(&src_column)) - { - UInt32 src_val = src_column_const->getValue(); - for (size_t i = 0; i < input_rows_count; ++i) - res_data[i] = (src_val * 1000 - snowflake_epoch) << time_shift; - } - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for function {}", name); + const auto & src_data = typeid_cast(src_column).getData(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (Int64(src_data[i]) * 1000 - snowflake_epoch) << time_shift; return res_column; } @@ -172,20 +161,9 @@ public: auto res_column = ColumnInt64::create(input_rows_count); auto & res_data = res_column->getData(); - if (const auto * src_column_non_const = typeid_cast(&src_column)) - { - const auto & src_data = src_column_non_const->getData(); - for (size_t i = 0; i < input_rows_count; ++i) - res_data[i] = (UInt32(src_data[i]) * 1000 - snowflake_epoch) << time_shift; - } - else if (const auto * src_column_const = typeid_cast(&src_column)) - { - UInt32 src_val = src_column_const->getValue(); - for (size_t i = 0; i < input_rows_count; ++i) - res_data[i] = (src_val * 1000 - snowflake_epoch) << time_shift; - } - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for function {}", name); + const auto & src_data = typeid_cast &>(src_column).getData(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (src_data[i] - snowflake_epoch) << time_shift; return res_column; } From 0bc624dc0291896001d45408e5316d23e28b3cc1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 13 Jun 2023 17:53:19 +0200 Subject: [PATCH 1572/2223] Fix the statless tests image for old commits --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index c0acb0291a4..21cb3168083 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -16,7 +16,7 @@ dpkg -i package_folder/clickhouse-client_*.deb ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test # shellcheck disable=SC1091 -source /usr/share/clickhouse-test/ci/attach_gdb.lib +source /usr/share/clickhouse-test/ci/attach_gdb.lib || true # FIXME: to not break old builds, clean on 2023-09-01 # install test configs /usr/share/clickhouse-test/config/install.sh @@ -88,7 +88,7 @@ fi sleep 5 -attach_gdb_to_clickhouse +attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 function run_tests() { From 9a5c5c7e931c41f406e316cb20cc7659c40fccc5 Mon Sep 17 00:00:00 2001 From: Yuriy Chernyshov Date: Tue, 13 Jun 2023 19:29:41 +0300 Subject: [PATCH 1573/2223] Update contrib/re2 to 2023-06-02 --- contrib/re2 | 2 +- src/Common/OptimizedRegularExpression.cpp | 12 ++++++------ src/Common/parseGlobs.cpp | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/contrib/re2 b/contrib/re2 index 13ebb377c6a..03da4fc0857 160000 --- a/contrib/re2 +++ b/contrib/re2 @@ -1 +1 @@ -Subproject commit 13ebb377c6ad763ca61d12dd6f88b1126bd0b911 +Subproject commit 03da4fc0857c285e3a26782f6bc8931c4c950df4 diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index f2fe922ef19..5df9ce76098 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -669,16 +669,16 @@ unsigned OptimizedRegularExpressionImpl::match(const char * subject matches.resize(limit); for (size_t i = 0; i < limit; ++i) { - if (pieces[i] != nullptr) - { - matches[i].offset = pieces[i].data() - subject; - matches[i].length = pieces[i].length(); - } - else + if (pieces[i].empty()) { matches[i].offset = std::string::npos; matches[i].length = 0; } + else + { + matches[i].offset = pieces[i].data() - subject; + matches[i].length = pieces[i].length(); + } } return limit; } diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index f8d331c2b76..07cce38afff 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -40,7 +40,7 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob size_t current_index = 0; while (RE2::FindAndConsume(&input, enum_or_range, &matched)) { - std::string buffer = matched.ToString(); + std::string buffer{matched}; oss_for_replacing << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '('; if (buffer.find(',') == std::string::npos) From a01056f67c787e069ca173cb63fafbfc5c6e5c96 Mon Sep 17 00:00:00 2001 From: santrancisco Date: Wed, 14 Jun 2023 02:33:48 +1000 Subject: [PATCH 1574/2223] Update orc submodule --- contrib/orc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/orc b/contrib/orc index c5d7755ba0b..f2c191f9653 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit c5d7755ba0b9a95631c8daea4d094101f26ec761 +Subproject commit f2c191f9653a5ddbca016e024ca0fb61508f5eeb From 945981e5f69145908aef64819da58725ca8e67e4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 13 Jun 2023 16:12:33 +0200 Subject: [PATCH 1575/2223] Fix tests sanity checks In #43147 the "tests" had been added to EXCLUDE_DIRS, and the reason for this is that there was some C++ code to ignore [1], however it also ignores snaity check for query_log. [1]: https://s3.amazonaws.com/clickhouse-test-reports/43147/63de577172ee024a08e76db69f5000568673db48/style_check.html v2: check-style: ignore $EXCLUDE_DIRS for some other sanity checks of tests Signed-off-by: Azat Khuzhin --- utils/check-style/check-style | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index afaf2ee6d48..bd3ee8e02d6 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -13,7 +13,7 @@ # and then to run formatter only for the specified files. ROOT_PATH=$(git rev-parse --show-toplevel) -EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|poco/|memcpy/|consistent-hashing|benchmark|tests/|utils/keeper-bench/example.yaml' +EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|poco/|memcpy/|consistent-hashing|benchmark|tests/.*.cpp|utils/keeper-bench/example.yaml' # From [1]: # But since array_to_string_internal() in array.c still loops over array @@ -163,14 +163,12 @@ find $ROOT_PATH -not -path $ROOT_PATH'/contrib*' \( -name '*.yaml' -or -name '*. # Tests should not be named with "fail" in their names. It makes looking at the results less convenient. find $ROOT_PATH/tests/queries -iname '*fail*' | - grep -vP $EXCLUDE_DIRS | grep . && echo 'Tests should not be named with "fail" in their names. It makes looking at the results less convenient when you search for "fail" substring in browser.' # Queries to system.query_log/system.query_thread_log should have current_database = currentDatabase() condition # NOTE: it is not that accurate, but at least something. tests_with_query_log=( $( find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | - grep -vP $EXCLUDE_DIRS | xargs grep --with-filename -e system.query_log -e system.query_thread_log | cut -d: -f1 | sort -u ) ) for test_case in "${tests_with_query_log[@]}"; do @@ -205,7 +203,6 @@ tables_with_database_column=( # NOTE: it is not that accuate, but at least something. tests_with_database_column=( $( find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | - grep -vP $EXCLUDE_DIRS | xargs grep --with-filename $(printf -- "-e %s " "${tables_with_database_column[@]}") | grep -v -e ':--' -e ':#' | cut -d: -f1 | sort -u @@ -225,7 +222,6 @@ done # NOTE: it is not that accuate, but at least something. tests_with_replicated_merge_tree=( $( find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | - grep -vP $EXCLUDE_DIRS | xargs grep --with-filename -e "Replicated.*MergeTree[ ]*(.*" | cut -d: -f1 | sort -u ) ) for test_case in "${tests_with_replicated_merge_tree[@]}"; do From dc6810601a65cf15a87459cdc72e5258d69949d2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 13 Jun 2023 16:15:30 +0200 Subject: [PATCH 1576/2223] Remove DROP TABEL system.query_log from tests This is a very ugly hack that breaks artifacts, since after this query_log is incomplete in artifacts. Signed-off-by: Azat Khuzhin --- .../02494_query_cache_case_agnostic_matching.sql | 4 ++-- .../0_stateless/02494_query_cache_events.reference | 1 + tests/queries/0_stateless/02494_query_cache_events.sql | 9 +++++---- .../0_stateless/02494_query_cache_normalize_ast.sql | 4 ++-- .../02494_query_cache_passive_usage.reference | 1 + .../0_stateless/02494_query_cache_passive_usage.sql | 10 +++++----- 6 files changed, 16 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql index 9f26d7759de..7dbd79059af 100644 --- a/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql +++ b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql @@ -1,9 +1,8 @@ -- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Start with empty query cache (QC) and query log +-- Start with empty query cache (QC) SYSTEM DROP QUERY CACHE; -DROP TABLE system.query_log SYNC; -- Insert an entry into the query cache. SELECT 1 SETTINGS use_query_cache = true; @@ -22,6 +21,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses'] FROM system.query_log WHERE type = 'QueryFinish' + AND current_database = currentDatabase() AND query = 'select 1 SETTINGS use_query_cache = true;'; SYSTEM DROP QUERY CACHE; diff --git a/tests/queries/0_stateless/02494_query_cache_events.reference b/tests/queries/0_stateless/02494_query_cache_events.reference index db60d3699e0..9bcd2820f27 100644 --- a/tests/queries/0_stateless/02494_query_cache_events.reference +++ b/tests/queries/0_stateless/02494_query_cache_events.reference @@ -3,4 +3,5 @@ 0 1 --- 1 +0 1 1 0 diff --git a/tests/queries/0_stateless/02494_query_cache_events.sql b/tests/queries/0_stateless/02494_query_cache_events.sql index 900b68f5eb2..05c0acad4b8 100644 --- a/tests/queries/0_stateless/02494_query_cache_events.sql +++ b/tests/queries/0_stateless/02494_query_cache_events.sql @@ -1,9 +1,8 @@ -- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Start with empty query cache QC and query log +-- Start with empty query cache QC SYSTEM DROP QUERY CACHE; -DROP TABLE system.query_log SYNC; -- Run a query with QC on. The first execution is a QC miss. SELECT '---'; @@ -13,6 +12,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses'] FROM system.query_log WHERE type = 'QueryFinish' + AND current_database = currentDatabase() AND query = 'SELECT 1 SETTINGS use_query_cache = true;'; @@ -20,11 +20,12 @@ WHERE type = 'QueryFinish' SELECT '---'; SELECT 1 SETTINGS use_query_cache = true; -DROP TABLE system.query_log SYNC; SYSTEM FLUSH LOGS; SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses'] FROM system.query_log WHERE type = 'QueryFinish' - AND query = 'SELECT 1 SETTINGS use_query_cache = true;'; + AND current_database = currentDatabase() + AND query = 'SELECT 1 SETTINGS use_query_cache = true;' +ORDER BY event_time_microseconds; SYSTEM DROP QUERY CACHE; diff --git a/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql b/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql index 5fd09eb935b..1dbb3ef8158 100644 --- a/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql +++ b/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql @@ -1,9 +1,8 @@ -- Tags: no-parallel -- Tag no-parallel: Messes with internal cache --- Start with empty query cache (QC) and query log. +-- Start with empty query cache (QC) SYSTEM DROP QUERY CACHE; -DROP TABLE system.query_log SYNC; -- Run query whose result gets cached in the query cache. -- Besides "use_query_cache", pass two more knobs (one QC-specific knob and one non-QC-specific knob). We just care @@ -24,6 +23,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses'] FROM system.query_log WHERE type = 'QueryFinish' + AND current_database = currentDatabase() AND query = 'SELECT 1 SETTINGS use_query_cache = true, enable_writes_to_query_cache = false, max_threads = 16;'; SYSTEM DROP QUERY CACHE; diff --git a/tests/queries/0_stateless/02494_query_cache_passive_usage.reference b/tests/queries/0_stateless/02494_query_cache_passive_usage.reference index edff09773d1..8b73647196e 100644 --- a/tests/queries/0_stateless/02494_query_cache_passive_usage.reference +++ b/tests/queries/0_stateless/02494_query_cache_passive_usage.reference @@ -9,4 +9,5 @@ ----- 1 1 +0 1 1 0 diff --git a/tests/queries/0_stateless/02494_query_cache_passive_usage.sql b/tests/queries/0_stateless/02494_query_cache_passive_usage.sql index 6143b5f7083..f0d2f6398a8 100644 --- a/tests/queries/0_stateless/02494_query_cache_passive_usage.sql +++ b/tests/queries/0_stateless/02494_query_cache_passive_usage.sql @@ -22,10 +22,7 @@ SELECT COUNT(*) FROM system.query_cache; SELECT '-----'; --- Run same query with passive mode again. There must still be one entry in the QC and we must have a QC hit. - --- Get rid of log of previous SELECT -DROP TABLE system.query_log SYNC; +/* Run same query with passive mode again. There must still be one entry in the QC and we must have a QC hit. */ SELECT 1 SETTINGS use_query_cache = true, enable_writes_to_query_cache = false; SELECT COUNT(*) FROM system.query_cache; @@ -34,6 +31,9 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses'] FROM system.query_log WHERE type = 'QueryFinish' - AND query = 'SELECT 1 SETTINGS use_query_cache = true, enable_writes_to_query_cache = false;'; + AND current_database = currentDatabase() + /* NOTE: client incorrectly join comments from the previous line into query, hence LIKE */ + AND query LIKE '%\nSELECT 1 SETTINGS use_query_cache = true, enable_writes_to_query_cache = false;' +ORDER BY event_time_microseconds; SYSTEM DROP QUERY CACHE; From 35825bc7d672974fc9219ca7e3608a32d0cd73bc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 13 Jun 2023 16:48:28 +0200 Subject: [PATCH 1577/2223] Increase line-length limit for yamlllint CI reports [1]: /ClickHouse/tests/queries/0_stateless/data_ua_parser/browser.yaml 713:301 warning line too long (328 > 300 characters) (line-length) /ClickHouse/tests/queries/0_stateless/data_ua_parser/device.yaml 2606:301 warning line too long (529 > 300 characters) (line-length) 2616:301 warning line too long (348 > 300 characters) (line-length) 2630:301 warning line too long (377 > 300 characters) (line-length) 2637:301 warning line too long (447 > 300 characters) (line-length) [1]: https://s3.amazonaws.com/clickhouse-test-reports/50934/be4555c3226298d956ff650fab477d67bf73ba83/style_check/style_output.txt Signed-off-by: Azat Khuzhin --- .yamllint | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/.yamllint b/.yamllint index fe161e71849..9d6550ac960 100644 --- a/.yamllint +++ b/.yamllint @@ -6,8 +6,10 @@ rules: level: warning indent-sequences: consistent line-length: - # there are some bash -c "", so this is OK - max: 300 + # there are: + # - bash -c "", so this is OK + # - yaml in tests + max: 1000 level: warning comments: min-spaces-from-content: 1 From 77775c6074601e49d44349f3d397229354571592 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 13 Jun 2023 16:49:45 +0200 Subject: [PATCH 1578/2223] Rename 02701_fail_on_invalid_having to 02701_invalid_having_NOT_AN_AGGREGATE To remove "fail" from the test name, which is prohibited Signed-off-by: Azat Khuzhin --- ....reference => 02701_invalid_having_NOT_AN_AGGREGATE.reference} | 0 ...valid_having.sql => 02701_invalid_having_NOT_AN_AGGREGATE.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02701_fail_on_invalid_having.reference => 02701_invalid_having_NOT_AN_AGGREGATE.reference} (100%) rename tests/queries/0_stateless/{02701_fail_on_invalid_having.sql => 02701_invalid_having_NOT_AN_AGGREGATE.sql} (100%) diff --git a/tests/queries/0_stateless/02701_fail_on_invalid_having.reference b/tests/queries/0_stateless/02701_invalid_having_NOT_AN_AGGREGATE.reference similarity index 100% rename from tests/queries/0_stateless/02701_fail_on_invalid_having.reference rename to tests/queries/0_stateless/02701_invalid_having_NOT_AN_AGGREGATE.reference diff --git a/tests/queries/0_stateless/02701_fail_on_invalid_having.sql b/tests/queries/0_stateless/02701_invalid_having_NOT_AN_AGGREGATE.sql similarity index 100% rename from tests/queries/0_stateless/02701_fail_on_invalid_having.sql rename to tests/queries/0_stateless/02701_invalid_having_NOT_AN_AGGREGATE.sql From 0444aa2fda48f59be08bef6482b06754f1cb2c0b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 13 Jun 2023 16:50:43 +0200 Subject: [PATCH 1579/2223] tests: add missing current_database condition for query_log Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02483_elapsed_time.sh | 2 +- .../queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh | 2 +- .../0_stateless/02521_incorrect_dealy_for_insert_bug_44902.sh | 2 +- .../queries/0_stateless/02578_parameterized_rename_queries.sql | 1 + tests/queries/0_stateless/02585_query_status_deadlock.sh | 2 +- tests/queries/0_stateless/02681_undrop_query.sql | 2 +- tests/queries/0_stateless/02761_ddl_initial_query_id.sh | 2 ++ .../02783_parallel_replicas_trivial_count_optimization.sh | 2 +- tests/queries/1_stateful/00177_memory_bound_merging.sh | 2 ++ 9 files changed, 11 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02483_elapsed_time.sh b/tests/queries/0_stateless/02483_elapsed_time.sh index 608299eb01b..e3b983129fb 100755 --- a/tests/queries/0_stateless/02483_elapsed_time.sh +++ b/tests/queries/0_stateless/02483_elapsed_time.sh @@ -25,7 +25,7 @@ QUERY_ID="${CLICKHOUSE_DATABASE}_$(date +%s)_02883_q1" ${CLICKHOUSE_CLIENT} -m --query "$EXCEPTION_BEFORE_START_QUERY" --query_id="$QUERY_ID" >/dev/null 2>&1 ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} --query "SELECT type == 'ExceptionBeforeStart' as expected_type, query_duration_ms <= 1000 as elapsed_more_than_one_second FROM system.query_log WHERE query_id='$QUERY_ID'" +${CLICKHOUSE_CLIENT} --query "SELECT type == 'ExceptionBeforeStart' as expected_type, query_duration_ms <= 1000 as elapsed_more_than_one_second FROM system.query_log WHERE current_database = '$CLICKHOUSE_DATABASE' AND query_id='$QUERY_ID'" # Now we test with a query that will take 1+ seconds. The CLI should show that as part of the output format OK_QUERY_JSON=" diff --git a/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh b/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh index 1223d7957b5..5d787aa0d8e 100755 --- a/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh +++ b/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh @@ -18,5 +18,5 @@ query_id="${CLICKHOUSE_DATABASE}_02499_$RANDOM$RANDOM" $CLICKHOUSE_CLIENT --query_id="$query_id" -q "select ts from t order by toUnixTimestamp64Nano(ts) limit 10 format Null settings max_block_size = $max_block_size, optimize_read_in_order = 1;" $CLICKHOUSE_CLIENT -q "system flush logs;" -$CLICKHOUSE_CLIENT --param_query_id="$query_id" -q "select read_rows <= $max_block_size from system.query_log where event_date >= yesterday() and query_id = {query_id:String} and type = 'QueryFinish';" +$CLICKHOUSE_CLIENT --param_query_id="$query_id" -q "select read_rows <= $max_block_size from system.query_log where event_date >= yesterday() and current_database = '$CLICKHOUSE_DATABASE' and query_id = {query_id:String} and type = 'QueryFinish';" diff --git a/tests/queries/0_stateless/02521_incorrect_dealy_for_insert_bug_44902.sh b/tests/queries/0_stateless/02521_incorrect_dealy_for_insert_bug_44902.sh index 5f91ef19a5a..0ae44ec0c01 100755 --- a/tests/queries/0_stateless/02521_incorrect_dealy_for_insert_bug_44902.sh +++ b/tests/queries/0_stateless/02521_incorrect_dealy_for_insert_bug_44902.sh @@ -16,7 +16,7 @@ do query_id="${CLICKHOUSE_DATABASE}_02521_${i}_$RANDOM$RANDOM" $CLICKHOUSE_CLIENT --query_id="$query_id" -q "INSERT INTO test_02521_insert_delay SELECT number, toString(number) FROM numbers(${i}, 1)" $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" - $CLICKHOUSE_CLIENT --param_query_id="$query_id" -q "select ProfileEvents['DelayedInsertsMilliseconds'] as delay from system.query_log where event_date >= yesterday() and query_id = {query_id:String} order by delay desc limit 1" + $CLICKHOUSE_CLIENT --param_query_id="$query_id" -q "select ProfileEvents['DelayedInsertsMilliseconds'] as delay from system.query_log where event_date >= yesterday() and current_database = '$CLICKHOUSE_DATABASE' and query_id = {query_id:String} order by delay desc limit 1" done $CLICKHOUSE_CLIENT -q "INSERT INTO test_02521_insert_delay VALUES(0, 'This query throws error')" 2>&1 | grep -o 'TOO_MANY_PARTS' | head -n 1 diff --git a/tests/queries/0_stateless/02578_parameterized_rename_queries.sql b/tests/queries/0_stateless/02578_parameterized_rename_queries.sql index eecb282083f..de36f8ae3b5 100644 --- a/tests/queries/0_stateless/02578_parameterized_rename_queries.sql +++ b/tests/queries/0_stateless/02578_parameterized_rename_queries.sql @@ -24,6 +24,7 @@ SET param_new_tbl_name = 02661_t1; CREATE TABLE {new_db_name:Identifier}.{old_tbl_name:Identifier} (a UInt64) ENGINE = MergeTree ORDER BY tuple(); RENAME TABLE {new_db_name:Identifier}.{old_tbl_name:Identifier} TO {new_db_name:Identifier}.{new_tbl_name:Identifier}; +-- NOTE: no 'database = currentDatabase()' on purpose SELECT name FROM system.tables WHERE name = {new_tbl_name:String}; -- Case 3: RENAME DICTIONARY diff --git a/tests/queries/0_stateless/02585_query_status_deadlock.sh b/tests/queries/0_stateless/02585_query_status_deadlock.sh index 227ecb1c1b2..9eb6eff8cd0 100755 --- a/tests/queries/0_stateless/02585_query_status_deadlock.sh +++ b/tests/queries/0_stateless/02585_query_status_deadlock.sh @@ -14,7 +14,7 @@ $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" while true do - res=$($CLICKHOUSE_CLIENT -q "select query, event_time from system.query_log where query_id = '$QUERY_ID' and query like 'select%' limit 1") + res=$($CLICKHOUSE_CLIENT -q "select query, event_time from system.query_log where query_id = '$QUERY_ID' and current_database = '$CLICKHOUSE_DATABASE' and query like 'select%' limit 1") if [ -n "$res" ]; then break fi diff --git a/tests/queries/0_stateless/02681_undrop_query.sql b/tests/queries/0_stateless/02681_undrop_query.sql index ead1a8bb305..39ca1548d53 100644 --- a/tests/queries/0_stateless/02681_undrop_query.sql +++ b/tests/queries/0_stateless/02681_undrop_query.sql @@ -21,7 +21,7 @@ detach table 02681_undrop_detach; undrop table 02681_undrop_detach; -- { serverError 57 } attach table 02681_undrop_detach; alter table 02681_undrop_detach update num = 2 where id = 1; -select command from system.mutations where table='02681_undrop_detach' limit 1; +select command from system.mutations where table='02681_undrop_detach' and database=currentDatabase() limit 1; drop table 02681_undrop_detach sync; select 'test MergeTree with cluster'; diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.sh b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh index e9a315b812b..b8b35ef01f7 100755 --- a/tests/queries/0_stateless/02761_ddl_initial_query_id.sh +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh @@ -21,4 +21,6 @@ $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" # - replace() is required to avoid non deterministic behaviour of # normalizeQuery() that replaces the identifier with "?" only if it has more # then two numbers. +# +# NOTE: no current_database = '$CLICKHOUSE_DATABASE' filter on purpose (since ON CLUSTER queries does not have current_database passed) $CLICKHOUSE_CLIENT -q "SELECT normalizeQuery(replace(query, currentDatabase(), 'default')) FROM system.query_log WHERE initial_query_id = '$query_id' AND type != 'QueryStart' ORDER BY event_time_microseconds" diff --git a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh index 4c29e513183..6210ef2e8b6 100755 --- a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh +++ b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh @@ -12,7 +12,7 @@ function has_used_parallel_replicas () { sumIf(read_rows, is_initial_query) as read_rows, sumIf(read_bytes, is_initial_query) as read_bytes FROM system.query_log - WHERE event_date >= yesterday() and initial_query_id LIKE '$1%' + WHERE event_date >= yesterday() and initial_query_id LIKE '$1%' AND current_database = '$CLICKHOUSE_DATABASE' GROUP BY initial_query_id ORDER BY min(event_time_microseconds) ASC FORMAT TSV" diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.sh b/tests/queries/1_stateful/00177_memory_bound_merging.sh index 008422be108..774f005b8eb 100755 --- a/tests/queries/1_stateful/00177_memory_bound_merging.sh +++ b/tests/queries/1_stateful/00177_memory_bound_merging.sh @@ -12,6 +12,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) check_replicas_read_in_order() { # to check this we actually look for at least one log message from MergeTreeInOrderSelectProcessor. # hopefully logger's names are a bit more stable than log messages itself + # + # NOTE: lack of "current_database = '$CLICKHOUSE_DATABASE'" filter is made on purpose $CLICKHOUSE_CLIENT -nq " SYSTEM FLUSH LOGS; From bb971fd7b7fd5bc400bbe28d16867c9dc337fb17 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 13 Jun 2023 17:02:24 +0200 Subject: [PATCH 1580/2223] check-style: allow {database} for ReplicatedMergeTree as well CLICKHOUSE_TEST_ZOOKEEPER_PREFIX is a {test_name}_{database}, but actually {database} should be enough, since it is uniq for each test run. Signed-off-by: Azat Khuzhin --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index bd3ee8e02d6..e7c06fefee2 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -229,7 +229,7 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do *.gen.*) ;; *.sh) - test_case_zk_prefix="\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" + test_case_zk_prefix="\(\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX\|{database}\)" grep -q -e "Replicated.*MergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "Replicated.*MergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)" ;; *.sql|*.sql.j2) From 1bc5598aa77bb5a7dcffc26b090bb0f45cb83abb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 13 Jun 2023 18:02:25 +0200 Subject: [PATCH 1581/2223] impl --- src/Disks/IO/AsynchronousBoundedReadBuffer.cpp | 5 ++--- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 ++ src/IO/SeekableReadBuffer.h | 4 ++++ 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index d75ec9f09e0..f9bd68222ae 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -301,9 +301,8 @@ off_t AsynchronousBoundedReadBuffer::seek(off_t offset, int whence) * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. * Note: we read in range [file_offset_of_buffer_end, read_until_position). */ - if (file_offset_of_buffer_end && read_until_position && new_pos < *read_until_position - && new_pos > file_offset_of_buffer_end - && new_pos < file_offset_of_buffer_end + read_settings.remote_read_min_bytes_for_seek) + if (!impl->seekIsCheap() && file_offset_of_buffer_end && read_until_position && new_pos < *read_until_position + && new_pos > file_offset_of_buffer_end && new_pos < file_offset_of_buffer_end + read_settings.remote_read_min_bytes_for_seek) { ProfileEvents::increment(ProfileEvents::RemoteFSLazySeeks); bytes_to_ignore = new_pos - file_offset_of_buffer_end; diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index cb98ac6d9f3..272ed2b3ac1 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -50,6 +50,8 @@ public: off_t getPosition() override { return file_offset_of_buffer_end - available() + bytes_to_ignore; } + bool seekIsCheap() override { return !current_buf; } + private: SeekableReadBufferPtr createImplementationBuffer(const StoredObject & object); diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index 8ced9d752de..5770948be20 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -83,6 +83,10 @@ public: /// Checks if readBigAt() is allowed. May be slow, may throw (e.g. it may do an HTTP request or an fstat). virtual bool supportsReadAt() { return false; } + + /// We do some tricks to avoid seek cost. E.g we read more data and than ignore it (see remote_read_min_bytes_for_seek). + /// Sometimes however seek is basically free because underlying read buffer wasn't yet initialised (or re-initialised after reset). + virtual bool seekIsCheap() { return false; } }; From 994228ab209d3dae408c43e135e2391e4a6b5112 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 13 Jun 2023 20:54:02 +0000 Subject: [PATCH 1582/2223] Uncomment flaky test --- .../0_stateless/01746_convert_type_with_default.reference | 1 + tests/queries/0_stateless/01746_convert_type_with_default.sql | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.reference b/tests/queries/0_stateless/01746_convert_type_with_default.reference index 0edea4de31e..e00156cd3c5 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.reference +++ b/tests/queries/0_stateless/01746_convert_type_with_default.reference @@ -40,6 +40,7 @@ 1970-01-20 1970-01-20 2149-06-06 +1 2023-05-30 2023-05-30 2023-05-30 14:38:20 diff --git a/tests/queries/0_stateless/01746_convert_type_with_default.sql b/tests/queries/0_stateless/01746_convert_type_with_default.sql index c74b185f7fd..5ef7718784d 100644 --- a/tests/queries/0_stateless/01746_convert_type_with_default.sql +++ b/tests/queries/0_stateless/01746_convert_type_with_default.sql @@ -54,7 +54,7 @@ select toDateOrDefault(cast(19 as Int256)); select toDateOrDefault(cast(19 as UInt256)); select toDateOrDefault(65535); --- select toDateOrDefault(122400); +select toDateOrDefault(65536) in ('1970-01-01', '1970-01-02'); select toDateOrDefault(19507, '2000-01-01'::Date); select toDateOrDefault(-1, '2023-05-30'::Date); From e7d2cc4a1a44e9c52d16ede99612e28342735c15 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Jun 2023 22:11:23 +0000 Subject: [PATCH 1583/2223] add map functions tests --- ...es_aggregate_functions_states.reference.j2 | 48 ++++++++++++++ ...ip_types_aggregate_functions_states.sql.j2 | 64 ++++++++++++++++--- 2 files changed, 102 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 index 90f98cf63fd..03b0e065151 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 @@ -170,3 +170,51 @@ Row 1: any: dd8b:333c::dd8b:333c:0:0 anyHeavy: dd8b:333c::dd8b:333c:0:0 anyLast: dd8b:333c::dd8b:333c:0:0 +----- Map/Map hash / State / ip4 ----- +Row 1: +────── +sumMapState: 9327034461443333306 +minMapState: 17403430892851901033 +maxMapState: 17403430892851901033 +----- Map/Map hash / State / ip6 ----- +Row 1: +────── +sumMapState: 5204122423200337352 +minMapState: 14534921151579960284 +maxMapState: 14534921151579960284 +----- Map/Map finalizeAggregation / State / ip4 ----- +Row 1: +────── +sumMap: {'59.154.201.255':1} +minMap: {'59.154.201.255':1} +maxMap: {'59.154.201.255':1} +----- Map/Map finalizeAggregation / State / ip6 ----- +Row 1: +────── +sumMap: {'dd8b:333c::dd8b:333c:0:0':1} +minMap: {'dd8b:333c::dd8b:333c:0:0':1} +maxMap: {'dd8b:333c::dd8b:333c:0:0':1} +----- Map/Array hash / State / ip4 ----- +Row 1: +────── +sumMapState: 9327034461443333306 +minMapState: 9327034461443333306 +maxMapState: 9327034461443333306 +----- Map/Array hash / State / ip6 ----- +Row 1: +────── +sumMapState: 15555709096566410627 +minMapState: 15555709096566410627 +maxMapState: 15555709096566410627 +----- Map/Array finalizeAggregation / State / ip4 ----- +Row 1: +────── +sumMap: (['59.154.201.255'],[1]) +minMap: (['59.154.201.255'],[1]) +maxMap: (['59.154.201.255'],[1]) +----- Map/Array finalizeAggregation / State / ip6 ----- +Row 1: +────── +sumMap: (['dd8b:333c::dd8b:333c:0:0'],[1]) +minMap: (['dd8b:333c::dd8b:333c:0:0'],[1]) +maxMap: (['dd8b:333c::dd8b:333c:0:0'],[1]) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 index 7daff5a690f..7d030d4be2d 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -1,5 +1,4 @@ --- Tags: no-fasttest --- no-fasttest because uniqTheta +-- Tags: no-parallel, no-fasttest {# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} @@ -22,7 +21,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- finalizeAggregation / State / {{ generator_name }} -----'; @@ -35,7 +33,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- hash / IfState / {{ generator_name }} -----'; @@ -48,7 +45,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- finalizeAggregation / IfState / {{ generator_name }} -----'; @@ -61,8 +57,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - - {% set agg_func_list = [ "argMin", "argMax" ] %} {% for generator_name, ip_generator in ip_generators.items() %} @@ -77,8 +71,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - - {% for generator_name, ip_generator in ip_generators.items() %} select '----- Arg / finalizeAggregation / State / {{ generator_name }} -----'; @@ -91,6 +83,8 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} + + {# let's test functions with not deterministic result against 1 row, to make it deterministic #} {% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,1) order by ip" %} {% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 1) order by ip" %} @@ -112,7 +106,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- finalizeAggregation / State / {{ generator_name }} -----'; @@ -124,3 +117,54 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} + + +{% set agg_func_list = [ "sumMap", "minMap", "maxMap" ] %} + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Map hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(map(ip, 1::Int64)) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Map finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(map(ip, 1::Int64)) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Array hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State([ip], [1::Int64]) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Array finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State([ip], [1::Int64]) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} From a570b00bdf76118e53de34a3337a66683b55fcb7 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 13 Jun 2023 19:19:34 -0300 Subject: [PATCH 1584/2223] Update README.md --- docker/server/README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/server/README.md b/docker/server/README.md index 67646a262f5..6aec001064e 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -20,6 +20,7 @@ For more information and documentation see https://clickhouse.com/. - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. - The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A). Most ARM CPUs after 2017 support ARMv8.2-A. A notable exception is Raspberry Pi 4 from 2019 whose CPU only supports ARMv8.0-A. +- Since Clickhouse 23.3 Ubuntu image started to use `ubuntu:22.04` as a base image it requiers docker version >= `20.10.10`, overwise use `docker run --privileged`. ## How to use this image From d8d570081031f4851c33737618d907d69a3b14e8 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 13 Jun 2023 19:24:36 -0300 Subject: [PATCH 1585/2223] Update README.md --- docker/server/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/README.md b/docker/server/README.md index 6aec001064e..0c89f834fcd 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -20,7 +20,7 @@ For more information and documentation see https://clickhouse.com/. - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. - The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A). Most ARM CPUs after 2017 support ARMv8.2-A. A notable exception is Raspberry Pi 4 from 2019 whose CPU only supports ARMv8.0-A. -- Since Clickhouse 23.3 Ubuntu image started to use `ubuntu:22.04` as a base image it requiers docker version >= `20.10.10`, overwise use `docker run --privileged`. +- Since Clickhouse 23.3 Ubuntu image started to use `ubuntu:22.04` as a base image it requiers docker version >= `20.10.10`, overwise use `docker run --privileged`. Alternativly try Clickhouse Alpine image. ## How to use this image From 404a52432c59a4b833d093e2e344916f4cb62de5 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 13 Jun 2023 19:39:23 -0300 Subject: [PATCH 1586/2223] Update README.md --- docker/server/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/README.md b/docker/server/README.md index 0c89f834fcd..18dce492123 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -20,7 +20,7 @@ For more information and documentation see https://clickhouse.com/. - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. - The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A). Most ARM CPUs after 2017 support ARMv8.2-A. A notable exception is Raspberry Pi 4 from 2019 whose CPU only supports ARMv8.0-A. -- Since Clickhouse 23.3 Ubuntu image started to use `ubuntu:22.04` as a base image it requiers docker version >= `20.10.10`, overwise use `docker run --privileged`. Alternativly try Clickhouse Alpine image. +- Since the Clickhouse 23.3 Ubuntu image started using `ubuntu:22.04` as its base image, it requires docker version >= `20.10.10`, or use `docker run -- privileged` instead. Alternatively, try the Clickhouse Alpine image. ## How to use this image From 86075dbae49f8acda2819fc6b8640631ccf1f3fe Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Tue, 13 Jun 2023 16:40:14 -0600 Subject: [PATCH 1587/2223] Update azureBlobStorage.md --- docs/en/engines/table-engines/integrations/azureBlobStorage.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index b8e621fd513..14fbf0c068e 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -48,4 +48,4 @@ SELECT * FROM test_table; ## See also -[Azure Blob Storage Table Function](/docs/en/sql-reference/table-functions/azureBlobStorage.md) +[Azure Blob Storage Table Function](/docs/en/sql-reference/table-functions/azureBlobStorage) From 49082bfe8919a4f7113e96eccde94c4fa2d74017 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 14 Jun 2023 09:00:50 +0800 Subject: [PATCH 1588/2223] fix typos in redis.md --- docs/en/engines/table-engines/integrations/redis.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/redis.md b/docs/en/engines/table-engines/integrations/redis.md index 68235a89d33..a78942ab7bb 100644 --- a/docs/en/engines/table-engines/integrations/redis.md +++ b/docs/en/engines/table-engines/integrations/redis.md @@ -114,6 +114,6 @@ TRUNCATE TABLE redis_table SYNC; ## Limitations {#limitations} -Redis engine also support scanning query, such as `where k > xx`, but it has some limitations: -1. Scanning query may produce some duplicated keys in a very rare case when it is rehashing, details see [Redis Scan](https://github.com/redis/redis/blob/e4d183afd33e0b2e6e8d1c79a832f678a04a7886/src/dict.c#L1186-L1269) -2. During the scanning keys could be created and deleted, so the resulting dataset can not represent a valid point in time. +Redis engine also supports scanning queries, such as `where k > xx`, but it has some limitations: +1. Scanning query may produce some duplicated keys in a very rare case when it is rehashing. See details in [Redis Scan](https://github.com/redis/redis/blob/e4d183afd33e0b2e6e8d1c79a832f678a04a7886/src/dict.c#L1186-L1269) +2. During the scanning, keys could be created and deleted, so the resulting dataset can not represent a valid point in time. From c05bcf56058597e180bfb65532e76dbe6c1639da Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 13 Jun 2023 21:09:30 -0400 Subject: [PATCH 1589/2223] Fix keeper-client help message --- programs/keeper-client/KeeperClient.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index f38da1b72aa..f41dca1e27a 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -127,42 +127,42 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) options.addOption( Poco::Util::Option("host", "h", "server hostname. default `localhost`") - .argument("host") + .argument("") .binding("host")); options.addOption( Poco::Util::Option("port", "p", "server port. default `2181`") - .argument("port") + .argument("") .binding("port")); options.addOption( Poco::Util::Option("query", "q", "will execute given query, then exit.") - .argument("query") + .argument("") .binding("query")); options.addOption( Poco::Util::Option("connection-timeout", "", "set connection timeout in seconds. default 10s.") - .argument("connection-timeout") + .argument("") .binding("connection-timeout")); options.addOption( Poco::Util::Option("session-timeout", "", "set session timeout in seconds. default 10s.") - .argument("session-timeout") + .argument("") .binding("session-timeout")); options.addOption( Poco::Util::Option("operation-timeout", "", "set operation timeout in seconds. default 10s.") - .argument("operation-timeout") + .argument("") .binding("operation-timeout")); options.addOption( Poco::Util::Option("history-file", "", "set path of history file. default `~/.keeper-client-history`") - .argument("history-file") + .argument("") .binding("history-file")); options.addOption( Poco::Util::Option("log-level", "", "set log level") - .argument("log-level") + .argument("") .binding("log-level")); } From f1b5d47ce27c053e76722d7827c49fb4aa78ac0e Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 14 Jun 2023 01:15:45 +0000 Subject: [PATCH 1590/2223] corrections after second review iteration --- docs/en/interfaces/cli.md | 37 ++++++++----------- docs/ru/interfaces/cli.md | 32 +++++++--------- programs/client/Client.cpp | 2 +- src/Client/ConnectionString.cpp | 31 ++++++++-------- src/Client/ConnectionString.h | 12 +++--- .../0_stateless/02784_connection_string.sh | 6 +-- 6 files changed, 54 insertions(+), 66 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index b5134ea30c0..e2c7dc1e608 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -194,7 +194,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--print-profile-events` – Print `ProfileEvents` packets. - `--profile-events-delay-ms` – Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet). -Instead of --host, --port, --user and --password options, ClickHouse client also supports connection strings. +Instead of `--host`, `--port`, `--user` and `--password` options, ClickHouse client also supports connection strings (see next section). ## Connection string {#connection_string} @@ -213,28 +213,27 @@ Where - `database` - (optional) is the database name, - `query_parameters` - (optional) is a list of key-value pairs `param1=value1[,¶m2=value2], ...`. For some parameters, no value is required. Parameter names and values are case-sensitive. +If no user is specified, `default` user without password will be used. +If no host is specified, the `localhost` will be used (localhost). +If no port is specified is not specified, `9000` will be used as port. +If no database is specified, the `default` database will be used. +If the user name, password or database was specified in the connection string, it cannot be specified using `--user`, `--password` or `--database` (and vice versa). -The host component can either be an IP address or a host name. Put an IPv6 address in square brackets to specify it: +The host component can either be an a host name and IP address. Put an IPv6 address in square brackets to specify it: ```text clickhouse://[2001:db8::1234] ``` -If user is not specified, `default` user without password will be used. -If host is not specified, the `localhost` will be used (localhost). -If port is not specified, `9000` will be used as port. -If database is not specified, the `default` database will be used. - -If the user name, password or database was specified in the connection string, it cannot be specified using `--user`, `--password` or `--database` (and vice versa). - -The connection string must be specified in the first argument of clickhouse-client. The connection string can be combined with other [command-line-options](#command-line-options) except `--host(h)` and `--port`. - -### Multiple hosts {#connection_string_multiple_hosts} - URI allows multiple hosts to be connected to. Connection strings can contain multiple hosts. ClickHouse-client will try to connect to these hosts in order (i.e. from left to right). After the connection is established, no attempt to connect to the remaining hosts is made. -### Allowed query_parameters keys {#connection_string_query_parameters} + + + +The connection string must be specified as the first argument of clickhouse-client. The connection string can be combined with arbitrary other [command-line-options](#command-line-options) except `--host/-h` and `--port`. + +The following keys are allowed for component `query_parameter`: - `secure` or shorthanded `s` - no value. If specified, client will connect to the server over a secure connection (TLS). See `secure` in [command-line-options](#command-line-options) @@ -244,7 +243,7 @@ Non-US ASCII, spaces and special characters in the `user`, `password`, `hosts`, ### Examples {#connection_string_examples} -Connect to localhost using port 9000 and execute the query "SELECT 1". +Connect to localhost using port 9000 and execute the query `SELECT 1`. ``` bash clickhouse-client clickhouse://localhost:9000 --query "SELECT 1" @@ -262,12 +261,6 @@ Connect to localhost using default user, host with IPV6 address `[::1]` and port clickhouse-client clickhouse://[::1]:9000 ``` -Connect to localhost using default user, host with IPV6 address `[2001:db8:3333:4444:5555:6666:7777:8888]` and port `9000`. - -``` bash -clickhouse-client clickhouse://[2001:db8:3333:4444:5555:6666:7777:8888]:9000 -``` - Connect to localhost using port 9000 in multiline mode. ``` bash @@ -277,7 +270,7 @@ clickhouse-client clickhouse://localhost:9000 '-m' Connect to localhost using port 9000 with the user `default`. ``` bash -clickhouse-client clickhouse://default@localhost:9000 --user default +clickhouse-client clickhouse://default@localhost:9000 # equivalent to: clickhouse-client clickhouse://localhost:9000 --user default diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 794ac60ec83..aa6ae3629e8 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -142,6 +142,8 @@ $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="numbe - `--history_file` - путь к файлу с историей команд. - `--param_` — значение параметра для [запроса с параметрами](#cli-queries-with-parameters). +Вместо параметров `--host`, `--port`, `--user` и `--password` клиент ClickHouse также поддерживает строки подключения (смотри следующий раздел). + ## Строка подключения {#connection_string} clickhouse-client также поддерживает подключение к серверу clickhouse с помощью строки подключения, аналогичной [MongoDB](https://www.mongodb.com/docs/manual/reference/connection-string/), [PostgreSQL](https://www.postgresql.org/docs/current/libpq-connect.html#LIBPQ-CONNSTRING), [MySQL](https://dev.mysql.com/doc/refman/8.0/en/connecting-using-uri-or-key-value-pairs.html#connecting-using-uri). Она имеет следующий синтаксис: @@ -154,15 +156,9 @@ clickhouse:[//[user[:password]@][hosts_and_ports]][/database][?query_parameters] - `user` - (необязательно) - это имя пользователя, - `password` - (необязательно) - Пароль пользователя. Если символ `:` укаказан, и пароль пуст, то клиент запросит ввести пользователя пароль. -- `hostspec` - (необязательно) - список хостов и необязательных портов. `host[:port] [, host:[port]], ...`, +- `hosts_and_ports` - (необязательно) - список хостов и необязательных портов. `host[:port] [, host:[port]], ...`, - `database` - (необязательно) - это имя базы данных, -- `paramspec` - (опционально) список пар ключ-значение `param1=value1[,¶m2=value2], ...`. Для некоторых параметров значение не требуется. Имена и значения параметров чувствительны к регистру. - -Параметр host может быть либо IP-адресом, либо именем хоста. Для указания IPv6-адреса поместите его в квадратные скобки: - -```text -clickhouse://[2001:db8::1234] -``` +- `query_parameters` - (опционально) список пар ключ-значение `param1=value1[,¶m2=value2], ...`. Для некоторых параметров значение не требуется. Имена и значения параметров чувствительны к регистру. Если user не указан, будут использоваться имя пользователя `default`. Если host не указан, будет использован хост `localhost`. @@ -171,13 +167,19 @@ clickhouse://[2001:db8::1234] Если имя пользователя, пароль или база данных были указаны в строке подключения, их нельзя указать с помощью `--user`, `--password` или `--database` (и наоборот). -Строка подключения должна быть указана в первом аргументе clickhouse-client. Строка подключения может комбинироваться с другими [параметрами командной строки] (#command-line-options) кроме `--host (h)` и `--port`. +Параметр host может быть либо именем хоста, либо IP-адресом. Для указания IPv6-адреса поместите его в квадратные скобки: -### Несколько хостов {#connection_string_multiple_hosts} +```text +clickhouse://[2001:db8::1234] +``` URI позволяет подключаться к нескольким хостам. Строки подключения могут содержать несколько хостов. ClickHouse-client будет пытаться подключиться к этим хостам по порядку (т.е. слева направо). После установления соединения попытки подключения к оставшимся хостам не предпринимаются. -### Допустимые ключи query_parameters {#connection_string_query_parameters} + + +Строка подключения должна быть указана в первом аргументе clickhouse-client. Строка подключения может комбинироваться с другими [параметрами командной строки] (#command-line-options) кроме `--host/-h` и `--port`. + +Для компонента `query_parameter` разрешены следующие ключи: - `secure` или сокращенно `s` - без значение. Если параметр указан, то соединение с сервером будет осуществляться по защищенному каналу (TLS). См. `secure` в [command-line-options](#command-line-options). @@ -187,7 +189,7 @@ URI позволяет подключаться к нескольким хост ### Примеры {#connection_string_examples} -Подключиться к localhost через порт 9000 и выполнить запрос "SELECT 1" +Подключиться к localhost через порт 9000 и выполнить запрос `SELECT 1` ``` bash clickhouse-client clickhouse://localhost:9000 --query "SELECT 1" @@ -204,12 +206,6 @@ clickhouse-client clickhouse://john:secret@127.0.0.1:9000 clickhouse-client clickhouse://[::1]:9000 ``` -Подключиться к localhost, используя пользователя по умолчанию, хост с IPV6 адресом `[2001:db8:3333:4444:5555:6666:7777:8888]` и портом `9000`. - -`` bash -clickhouse-client clickhouse://[2001:db8:3333:4444:5555:6666:7777:8888]:9000 -``` - Подключиться к localhost через порт 9000 многострочном режиме. ``` bash diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index a49447dff69..6c3df3520e9 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1268,7 +1268,7 @@ void Client::readArguments( std::string_view arg = argv[arg_num]; if (has_connection_string) - validateConnectionStringClientOption(arg); + checkIfCmdLineOptionCanBeUsedWithConnectionString(arg); if (arg == "--external") { diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp index 8f0a0980f51..62090487490 100644 --- a/src/Client/ConnectionString.cpp +++ b/src/Client/ConnectionString.cpp @@ -45,26 +45,26 @@ std::string uriDecode(const std::string & uri_encoded_string, bool plus_as_space void getHostAndPort(const Poco::URI & uri, std::vector> & hosts_and_ports_arguments) { std::vector host_and_port; - auto host = uri.getHost(); + const auto& host = uri.getHost(); if (!host.empty()) { - host_and_port.push_back("--host="s + uriDecode(host, false)); + host_and_port.push_back("--host=" + uriDecode(host, false)); } // Port can be written without host (":9000"). Empty host name equals to default host. auto port = uri.getPort(); if (port != 0) - host_and_port.push_back("--port="s + std::to_string(port)); + host_and_port.push_back("--port=" + std::to_string(port)); if (!host_and_port.empty()) hosts_and_ports_arguments.push_back(std::move(host_and_port)); } void buildConnectionString( - Poco::URI & uri, - std::vector> & hosts_and_ports_arguments, std::string_view host_and_port, - std::string_view right_part) + std::string_view right_part, + Poco::URI & uri, + std::vector> & hosts_and_ports_arguments) { // User info does not matter in sub URI auto uri_string = std::string(CONNECTION_URI_SCHEME); @@ -154,7 +154,7 @@ bool tryParseConnectionString( { if (*it == ',') { - buildConnectionString(uri, hosts_and_ports_arguments, {last_host_begin, it}, {hosts_end, connection_string.end()}); + buildConnectionString({last_host_begin, it}, {hosts_end, connection_string.end()}, uri, hosts_and_ports_arguments); last_host_begin = it + 1; } } @@ -166,7 +166,7 @@ bool tryParseConnectionString( getHostAndPort(uri, hosts_and_ports_arguments); } else - buildConnectionString(uri, hosts_and_ports_arguments, {last_host_begin, hosts_end}, {hosts_end, connection_string.end()}); + buildConnectionString({last_host_begin, hosts_end}, {hosts_end, connection_string.end()}, uri, hosts_and_ports_arguments); Poco::URI::QueryParameters params = uri.getQueryParameters(); for (const auto & param : params) @@ -174,12 +174,12 @@ bool tryParseConnectionString( if (param.first == "secure" || param.first == "s") { if (!param.second.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "secure URI query parameter does not require value"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "secure URI query parameter does not allow value"); common_arguments.push_back(makeArgument(param.first)); } else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "URI query parameter {} is unknown", param.first); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "URI query parameter {} is not supported", param.first); } auto user_info = uri.getUserInfo(); @@ -188,7 +188,7 @@ bool tryParseConnectionString( // Poco::URI doesn't decode user name/password by default. // But ClickHouse allows to have users with email user name like: 'john@some_mail.com' // john@some_mail.com should be percent-encoded: 'john%40some_mail.com' - std::string::size_type pos = user_info.find(':'); + size_t pos = user_info.find(':'); if (pos != std::string::npos) { common_arguments.push_back("--user"); @@ -229,12 +229,11 @@ bool tryParseConnectionString( return true; } -void validateConnectionStringClientOption(std::string_view command_line_option) +void checkIfCmdLineOptionCanBeUsedWithConnectionString(std::string_view command_line_option) { - const auto prohibited_option_iter = PROHIBITED_CLIENT_OPTIONS.find(command_line_option); - if (prohibited_option_iter != PROHIBITED_CLIENT_OPTIONS.end()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Mixing a connection string and {} option is prohibited", prohibited_option_iter->second); + if (PROHIBITED_CLIENT_OPTIONS.contains(command_line_option)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Mixing a connection string and {} option is prohibited", PROHIBITED_CLIENT_OPTIONS.at(command_line_option)); } } diff --git a/src/Client/ConnectionString.h b/src/Client/ConnectionString.h index ce72de9edf6..ad63e9cda3d 100644 --- a/src/Client/ConnectionString.h +++ b/src/Client/ConnectionString.h @@ -9,19 +9,19 @@ namespace DB /** Tries to parse ClickHouse connection string. * if @connection_string starts with 'clickhouse:' then connection string will be parsed * and converted into a set of arguments for the client. - * Connection string format is similar to URI "clickhouse:[//[user_info@][hosts_and_ports]][/dbname][?query_parameters]" + * Connection string format is similar to URI "clickhouse:[//[user[:password]@][hosts_and_ports]][/dbname][?query_parameters]" * with the difference that hosts_and_ports can contain multiple hosts separated by ','. * example: clickhouse://user@host1:port1,host2:port2 - * @return returns true if there is a URI, false otherwise. - * @exception throws DB::Exception if URI has valid scheme (clickhouse:), but invalid internals. + * @return Returns false if no connection string was specified. If a connection string was specified, returns true if it is valid, and throws an exception if it is invalid. + * @exception Throws DB::Exception if URI has valid scheme (clickhouse:), but invalid internals. */ bool tryParseConnectionString( std::string_view connection_string, std::vector & common_arguments, std::vector> & hosts_and_ports_arguments); -// throws DB::Exception with BAD_ARGUMENTS if the given command line argument is allowed -// to be used with the connection string -void validateConnectionStringClientOption(std::string_view command_line_option); +// Throws DB::Exception with BAD_ARGUMENTS if the given command line argument +// is not allowed to be used with a connection string. +void checkIfCmdLineOptionCanBeUsedWithConnectionString(std::string_view command_line_option); } diff --git a/tests/queries/0_stateless/02784_connection_string.sh b/tests/queries/0_stateless/02784_connection_string.sh index 042f5b2108d..8353ac5b1e4 100755 --- a/tests/queries/0_stateless/02784_connection_string.sh +++ b/tests/queries/0_stateless/02784_connection_string.sh @@ -99,7 +99,6 @@ runClient "click_house:" 2>&1 | grep -o 'BAD_ARGUMENTS' TEST_INDEX=1000087 # Using connection string prohibits to use --host and --port options runClient "clickhouse://default:@$CLICKHOUSE_HOST/" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' -runClient "clickhouse://default:@$CLICKHOUSE_HOST/" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://default:@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://default:@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" --port "$CLICKHOUSE_PORT_TCP" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://default:@$CLICKHOUSE_HOST:$CLICKHOUSE_PORT_TCP/" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' @@ -109,6 +108,7 @@ runClient "clickhouse://default:@$CLICKHOUSE_HOST/" --port "$CLICKHOUSE_PORT_TCP runClient "clickhouse://$CLICKHOUSE_HOST/" --port "$CLICKHOUSE_PORT_TCP" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://:@$CLICKHOUSE_HOST/" --port "$CLICKHOUSE_PORT_TCP" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://$CLICKHOUSE_HOST/" --port "$CLICKHOUSE_PORT_TCP" 2>&1 | grep -o 'BAD_ARGUMENTS' +runClient "clickhouse://" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse:" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse:///" --port "$CLICKHOUSE_PORT_TCP" --host "$CLICKHOUSE_HOST" 2>&1 | grep -o 'BAD_ARGUMENTS' @@ -130,9 +130,9 @@ runClient "clickhouse://host1/ database:" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://user :password@host1" 2>&1 | grep -o 'BAD_ARGUMENTS' runClient "clickhouse://user: password@host1" 2>&1 | grep -o 'BAD_ARGUMENTS' -# Query is not first argument +# Connection string is not first argument runClient --multiline "clickhouse://default:@$CLICKHOUSE_HOST/" 2>&1 | grep -o 'BAD_ARGUMENTS' -# Query used as the first and the second argument of client +# Connection string used as the first and the second argument of client runClient "clickhouse://default:@$CLICKHOUSE_HOST/" "clickhouse://default:@$CLICKHOUSE_HOST/" 2>&1 | grep -o 'BAD_ARGUMENTS' # Invalid hosts From 29b9cba75c18e23f9ee2eb589e5a69e7f46a5054 Mon Sep 17 00:00:00 2001 From: santrancisco Date: Wed, 14 Jun 2023 11:31:09 +1000 Subject: [PATCH 1591/2223] Update CMakeLists.txt with help from Nikita --- contrib/arrow-cmake/CMakeLists.txt | 68 +++++++++++++++++++++++------- 1 file changed, 52 insertions(+), 16 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 16198887075..5fe942d1cd0 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -116,43 +116,79 @@ configure_file("${ORC_SOURCE_SRC_DIR}/Adaptor.hh.in" "${ORC_BUILD_INCLUDE_DIR}/A # ARROW_ORC + adapters/orc/CMakefiles set(ORC_SRCS "${CMAKE_CURRENT_BINARY_DIR}/orc_proto.pb.h" - "${ORC_SOURCE_SRC_DIR}/sargs/ExpressionTree.cc" - "${ORC_SOURCE_SRC_DIR}/sargs/Literal.cc" - "${ORC_SOURCE_SRC_DIR}/sargs/PredicateLeaf.cc" - "${ORC_SOURCE_SRC_DIR}/sargs/SargsApplier.cc" - "${ORC_SOURCE_SRC_DIR}/sargs/SearchArgument.cc" - "${ORC_SOURCE_SRC_DIR}/sargs/TruthValue.cc" - "${ORC_SOURCE_SRC_DIR}/Exceptions.cc" - "${ORC_SOURCE_SRC_DIR}/OrcFile.cc" - "${ORC_SOURCE_SRC_DIR}/Reader.cc" + "${ORC_ADDITION_SOURCE_DIR}/orc_proto.pb.cc" + "${ORC_SOURCE_SRC_DIR}/Adaptor.cc" + "${ORC_SOURCE_SRC_DIR}/Adaptor.hh.in" + "${ORC_SOURCE_SRC_DIR}/BlockBuffer.cc" + "${ORC_SOURCE_SRC_DIR}/BlockBuffer.hh" + "${ORC_SOURCE_SRC_DIR}/BloomFilter.cc" + "${ORC_SOURCE_SRC_DIR}/BloomFilter.hh" + "${ORC_SOURCE_SRC_DIR}/Bpacking.hh" + "${ORC_SOURCE_SRC_DIR}/BpackingDefault.cc" + "${ORC_SOURCE_SRC_DIR}/BpackingDefault.hh" "${ORC_SOURCE_SRC_DIR}/ByteRLE.cc" + "${ORC_SOURCE_SRC_DIR}/ByteRLE.hh" + "${ORC_SOURCE_SRC_DIR}/CMakeLists.txt" "${ORC_SOURCE_SRC_DIR}/ColumnPrinter.cc" "${ORC_SOURCE_SRC_DIR}/ColumnReader.cc" + "${ORC_SOURCE_SRC_DIR}/ColumnReader.hh" "${ORC_SOURCE_SRC_DIR}/ColumnWriter.cc" + "${ORC_SOURCE_SRC_DIR}/ColumnWriter.hh" "${ORC_SOURCE_SRC_DIR}/Common.cc" "${ORC_SOURCE_SRC_DIR}/Compression.cc" + "${ORC_SOURCE_SRC_DIR}/Compression.hh" + "${ORC_SOURCE_SRC_DIR}/ConvertColumnReader.cc" + "${ORC_SOURCE_SRC_DIR}/ConvertColumnReader.hh" + "${ORC_SOURCE_SRC_DIR}/CpuInfoUtil.cc" + "${ORC_SOURCE_SRC_DIR}/CpuInfoUtil.hh" + "${ORC_SOURCE_SRC_DIR}/Dispatch.hh" + "${ORC_SOURCE_SRC_DIR}/Exceptions.cc" "${ORC_SOURCE_SRC_DIR}/Int128.cc" "${ORC_SOURCE_SRC_DIR}/LzoDecompressor.cc" + "${ORC_SOURCE_SRC_DIR}/LzoDecompressor.hh" "${ORC_SOURCE_SRC_DIR}/MemoryPool.cc" + "${ORC_SOURCE_SRC_DIR}/Murmur3.cc" + "${ORC_SOURCE_SRC_DIR}/Murmur3.hh" + "${ORC_SOURCE_SRC_DIR}/Options.hh" + "${ORC_SOURCE_SRC_DIR}/OrcFile.cc" "${ORC_SOURCE_SRC_DIR}/RLE.cc" + "${ORC_SOURCE_SRC_DIR}/RLE.hh" + "${ORC_SOURCE_SRC_DIR}/RLEV2Util.cc" + "${ORC_SOURCE_SRC_DIR}/RLEV2Util.hh" "${ORC_SOURCE_SRC_DIR}/RLEv1.cc" + "${ORC_SOURCE_SRC_DIR}/RLEv1.hh" + "${ORC_SOURCE_SRC_DIR}/RLEv2.hh" + "${ORC_SOURCE_SRC_DIR}/Reader.cc" + "${ORC_SOURCE_SRC_DIR}/Reader.hh" "${ORC_SOURCE_SRC_DIR}/RleDecoderV2.cc" "${ORC_SOURCE_SRC_DIR}/RleEncoderV2.cc" - "${ORC_SOURCE_SRC_DIR}/RLEV2Util.cc" + "${ORC_SOURCE_SRC_DIR}/SchemaEvolution.cc" + "${ORC_SOURCE_SRC_DIR}/SchemaEvolution.hh" "${ORC_SOURCE_SRC_DIR}/Statistics.cc" + "${ORC_SOURCE_SRC_DIR}/Statistics.hh" "${ORC_SOURCE_SRC_DIR}/StripeStream.cc" + "${ORC_SOURCE_SRC_DIR}/StripeStream.hh" "${ORC_SOURCE_SRC_DIR}/Timezone.cc" + "${ORC_SOURCE_SRC_DIR}/Timezone.hh" "${ORC_SOURCE_SRC_DIR}/TypeImpl.cc" + "${ORC_SOURCE_SRC_DIR}/TypeImpl.hh" + "${ORC_SOURCE_SRC_DIR}/Utils.hh" "${ORC_SOURCE_SRC_DIR}/Vector.cc" "${ORC_SOURCE_SRC_DIR}/Writer.cc" - "${ORC_SOURCE_SRC_DIR}/Adaptor.cc" - "${ORC_SOURCE_SRC_DIR}/BloomFilter.cc" - "${ORC_SOURCE_SRC_DIR}/Murmur3.cc" - "${ORC_SOURCE_SRC_DIR}/BlockBuffer.cc" - "${ORC_SOURCE_SRC_DIR}/wrap/orc-proto-wrapper.cc" "${ORC_SOURCE_SRC_DIR}/io/InputStream.cc" + "${ORC_SOURCE_SRC_DIR}/io/InputStream.hh" "${ORC_SOURCE_SRC_DIR}/io/OutputStream.cc" - "${ORC_ADDITION_SOURCE_DIR}/orc_proto.pb.cc" + "${ORC_SOURCE_SRC_DIR}/io/OutputStream.hh" + "${ORC_SOURCE_SRC_DIR}/sargs/ExpressionTree.cc" + "${ORC_SOURCE_SRC_DIR}/sargs/ExpressionTree.hh" + "${ORC_SOURCE_SRC_DIR}/sargs/Literal.cc" + "${ORC_SOURCE_SRC_DIR}/sargs/PredicateLeaf.cc" + "${ORC_SOURCE_SRC_DIR}/sargs/PredicateLeaf.hh" + "${ORC_SOURCE_SRC_DIR}/sargs/SargsApplier.cc" + "${ORC_SOURCE_SRC_DIR}/sargs/SargsApplier.hh" + "${ORC_SOURCE_SRC_DIR}/sargs/SearchArgument.cc" + "${ORC_SOURCE_SRC_DIR}/sargs/SearchArgument.hh" + "${ORC_SOURCE_SRC_DIR}/sargs/TruthValue.cc" ) add_library(_orc ${ORC_SRCS}) From 1f76d0874398f4985bb00dd16caf68a772c95502 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 14 Jun 2023 01:37:58 +0000 Subject: [PATCH 1592/2223] fixed --password "" --password issue --- programs/client/Client.cpp | 14 +++++++------- .../01317_no_password_in_command_line.reference | 1 + .../01317_no_password_in_command_line.sh | 2 ++ 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 32a07284d26..1e2696b4910 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -977,13 +977,7 @@ void Client::addOptions(OptionsDescription & options_description) ("connection", po::value(), "connection to use (from the client config), by default connection name is hostname") ("secure,s", "Use TLS connection") ("user,u", po::value()->default_value("default"), "user") - /** If "--password [value]" is used but the value is omitted, the bad argument exception will be thrown. - * implicit_value is used to avoid this exception (to allow user to type just "--password") - * Since currently boost provides no way to check if a value has been set implicitly for an option, - * the "\n" is used to distinguish this case because there is hardly a chance a user would use "\n" - * as the password. - */ - ("password", po::value()->implicit_value("\n", ""), "password") + ("password", po::value(), "password") ("ask-password", "ask-password") ("quota_key", po::value(), "A string to differentiate quotas when the user have keyed quotas configured on server") @@ -1391,6 +1385,12 @@ void Client::readArguments( arg = argv[arg_num]; addMultiquery(arg, common_arguments); } + else if (arg == "--password" && ((arg_num + 1) >= argc || std::string_view(argv[arg_num + 1]).starts_with('-'))) + { + common_arguments.emplace_back(arg); + // Add implicit value to the password. '\n' means client should ask user for password. + common_arguments.emplace_back("\n"); + } else common_arguments.emplace_back(arg); } diff --git a/tests/queries/0_stateless/01317_no_password_in_command_line.reference b/tests/queries/0_stateless/01317_no_password_in_command_line.reference index e69de29bb2d..8f2f637d5e3 100644 --- a/tests/queries/0_stateless/01317_no_password_in_command_line.reference +++ b/tests/queries/0_stateless/01317_no_password_in_command_line.reference @@ -0,0 +1 @@ +Bad arguments diff --git a/tests/queries/0_stateless/01317_no_password_in_command_line.sh b/tests/queries/0_stateless/01317_no_password_in_command_line.sh index 7f2e91201a3..fc5b8997636 100755 --- a/tests/queries/0_stateless/01317_no_password_in_command_line.sh +++ b/tests/queries/0_stateless/01317_no_password_in_command_line.sh @@ -45,3 +45,5 @@ ps u --no-header $bg_query | grep -F -- '--password' | grep -F hello ||: grep -F -- '--password' < "/proc/$bg_query/comm" | grep -F hello ||: $CLICKHOUSE_CLIENT --format Null --param_query_id "$query_id" -q "KILL QUERY WHERE query_id = {query_id:String} SYNC" wait + +$CLICKHOUSE_CLIENT --user "$user" --password=hello --password -q 'select currentUser()' 2>&1 | grep -o 'Bad arguments' From f3b99156aca66e5df89181d224ae05a28f80f257 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 14 Jun 2023 10:48:21 +0800 Subject: [PATCH 1593/2223] review fix --- docs/en/interfaces/formats.md | 2 +- .../operations/settings/settings-formats.md | 8 ++-- src/Core/Settings.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- .../Formats/Impl/CSVRowInputFormat.cpp | 38 ++++++++++--------- ...ext_with_whitespace_tab_field_delimiter.sh | 4 +- 7 files changed, 31 insertions(+), 27 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 57962c1d730..da1ba17cbb7 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -468,7 +468,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_skip_first_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_first_lines) - skip the specified number of lines at the beginning of data. Default value - `0`. - [input_format_csv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_csv_detect_header) - automatically detect header with names and types in CSV format. Default value - `true`. - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. -- [input_format_csv_use_whitespace_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_use_whitespace_tab_as_delimiter) - use whitespace or tab as field delimiter in CSV strings. Default value - `false`. +- [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 0e30c8f319e..daf27622d3a 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -914,9 +914,9 @@ Result " string " ``` -### input_format_csv_use_whitespace_tab_as_delimiter {#input_format_csv_use_whitespace_tab_as_delimiter} +### input_format_csv_allow_whitespace_or_tab_as_delimiter {#input_format_csv_allow_whitespace_or_tab_as_delimiter} -Use whitespace or tab as field delimiter in CSV strings. +Allow to use whitespace or tab as field delimiter in CSV strings. Default value: `false`. @@ -925,7 +925,7 @@ Default value: `false`. Query ```bash -echo 'a b' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_use_whitespace_tab_as_delimiter=true --format_csv_delimiter=' ' +echo 'a b' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_allow_whitespace_or_tab_as_delimiter=true --format_csv_delimiter=' ' ``` Result @@ -937,7 +937,7 @@ a b Query ```bash -echo 'a b' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_use_whitespace_tab_as_delimiter=true --format_csv_delimiter='\t' +echo 'a b' | ./clickhouse local -q "select * from table FORMAT CSV" --input-format="CSV" --input_format_csv_allow_whitespace_or_tab_as_delimiter=true --format_csv_delimiter='\t' ``` Result diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4306ac855a3..1d889d8b0c3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -850,7 +850,7 @@ class IColumn; M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ - M(Bool, input_format_csv_use_whitespace_tab_as_delimiter, false, "Use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ + M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 33ecddfc223..81528937b13 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -70,7 +70,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.skip_first_lines = settings.input_format_csv_skip_first_lines; format_settings.csv.try_detect_header = settings.input_format_csv_detect_header; format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; - format_settings.csv.use_whitespace_tab_as_delimiter = settings.input_format_csv_use_whitespace_tab_as_delimiter; + format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 72d60e8423e..5dc1a14a12c 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -137,7 +137,7 @@ struct FormatSettings String custom_delimiter; bool try_detect_header = true; bool trim_whitespaces = true; - bool use_whitespace_tab_as_delimiter = false; + bool allow_whitespace_or_tab_as_delimiter = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index b8d3413f863..181949b3bb7 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -25,10 +25,14 @@ namespace ErrorCodes namespace { - void checkBadDelimiter(char delimiter, bool use_whitespace_tab_as_delimiter) + void checkBadDelimiter(char delimiter, bool allow_whitespace_or_tab_as_delimiter) { + if ((delimiter == ' ' || delimiter == '\t') && allow_whitespace_or_tab_as_delimiter) + { + return; + } constexpr std::string_view bad_delimiters = " \t\"'.UL"; - if (bad_delimiters.find(delimiter) != std::string_view::npos && !use_whitespace_tab_as_delimiter) + if (bad_delimiters.find(delimiter) != std::string_view::npos) throw Exception( ErrorCodes::BAD_ARGUMENTS, "CSV format may not work correctly with delimiter '{}'. Try use CustomSeparated format instead", @@ -68,7 +72,7 @@ CSVRowInputFormat::CSVRowInputFormat( format_settings_.csv.try_detect_header), buf(std::move(in_)) { - checkBadDelimiter(format_settings_.csv.delimiter, format_settings_.csv.use_whitespace_tab_as_delimiter); + checkBadDelimiter(format_settings_.csv.delimiter, format_settings_.csv.allow_whitespace_or_tab_as_delimiter); } CSVRowInputFormat::CSVRowInputFormat( @@ -90,7 +94,7 @@ CSVRowInputFormat::CSVRowInputFormat( format_settings_.csv.try_detect_header), buf(std::move(in_)) { - checkBadDelimiter(format_settings_.csv.delimiter, format_settings_.csv.use_whitespace_tab_as_delimiter); + checkBadDelimiter(format_settings_.csv.delimiter, format_settings_.csv.allow_whitespace_or_tab_as_delimiter); } void CSVRowInputFormat::syncAfterError() @@ -134,9 +138,9 @@ static void skipEndOfLine(ReadBuffer & in) } /// Skip `whitespace` symbols allowed in CSV. -static inline void skipWhitespacesAndTabs(ReadBuffer & in, const bool & use_whitespace_tab_as_delimiter) +static inline void skipWhitespacesAndTabs(ReadBuffer & in, const bool & allow_whitespace_or_tab_as_delimiter) { - if (use_whitespace_tab_as_delimiter) + if (allow_whitespace_or_tab_as_delimiter) { return; } @@ -150,7 +154,7 @@ CSVFormatReader::CSVFormatReader(PeekableReadBuffer & buf_, const FormatSettings void CSVFormatReader::skipFieldDelimiter() { - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); assertChar(format_settings.csv.delimiter, *buf); } @@ -158,7 +162,7 @@ template String CSVFormatReader::readCSVFieldIntoString() { if (format_settings.csv.trim_whitespaces) [[likely]] - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); String field; if constexpr (read_string) @@ -170,14 +174,14 @@ String CSVFormatReader::readCSVFieldIntoString() void CSVFormatReader::skipField() { - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); NullOutput out; readCSVStringInto(out, *buf, format_settings.csv); } void CSVFormatReader::skipRowEndDelimiter() { - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); if (buf->eof()) return; @@ -186,7 +190,7 @@ void CSVFormatReader::skipRowEndDelimiter() if (*buf->position() == format_settings.csv.delimiter) ++buf->position(); - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); if (buf->eof()) return; @@ -198,7 +202,7 @@ void CSVFormatReader::skipHeaderRow() do { skipField(); - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); } while (checkChar(format_settings.csv.delimiter, *buf)); skipRowEndDelimiter(); @@ -211,7 +215,7 @@ std::vector CSVFormatReader::readRowImpl() do { fields.push_back(readCSVFieldIntoString()); - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); } while (checkChar(format_settings.csv.delimiter, *buf)); skipRowEndDelimiter(); @@ -224,7 +228,7 @@ bool CSVFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) try { - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); assertChar(delimiter, *buf); } catch (const DB::Exception &) @@ -250,7 +254,7 @@ bool CSVFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) { - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); if (buf->eof()) return true; @@ -259,7 +263,7 @@ bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) if (*buf->position() == format_settings.csv.delimiter) { ++buf->position(); - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); if (buf->eof()) return true; } @@ -287,7 +291,7 @@ bool CSVFormatReader::readField( const String & /*column_name*/) { if (format_settings.csv.trim_whitespaces || !isStringOrFixedString(removeNullable(type))) [[likely]] - skipWhitespacesAndTabs(*buf, format_settings.csv.use_whitespace_tab_as_delimiter); + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; const bool at_last_column_line_end = is_last_file_column && (buf->eof() || *buf->position() == '\n' || *buf->position() == '\r'); diff --git a/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh index deb6e317aac..6fca95cb839 100755 --- a/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh +++ b/tests/queries/0_stateless/02785_text_with_whitespace_tab_field_delimiter.sh @@ -10,8 +10,8 @@ $CLICKHOUSE_CLIENT -q "drop table if exists test_whitespace" $CLICKHOUSE_CLIENT -q "drop table if exists test_tab" $CLICKHOUSE_CLIENT -q "create table test_whitespace (x UInt32, y String, z String) engine=MergeTree order by x" $CLICKHOUSE_CLIENT -q "create table test_tab (x UInt32, y String, z String) engine=MergeTree order by x" -cat $CURDIR/data_csv/csv_with_space_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_whitespace SETTINGS format_csv_delimiter=' ', input_format_csv_use_whitespace_tab_as_delimiter=true FORMAT CSV" -cat $CURDIR/data_csv/csv_with_tab_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tab SETTINGS format_csv_delimiter='\t', input_format_csv_use_whitespace_tab_as_delimiter=true FORMAT CSV" +cat $CURDIR/data_csv/csv_with_space_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_whitespace SETTINGS format_csv_delimiter=' ', input_format_csv_allow_whitespace_or_tab_as_delimiter=true FORMAT CSV" +cat $CURDIR/data_csv/csv_with_tab_delimiter.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tab SETTINGS format_csv_delimiter='\t', input_format_csv_allow_whitespace_or_tab_as_delimiter=true FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_whitespace" $CLICKHOUSE_CLIENT -q "select * from test_tab" $CLICKHOUSE_CLIENT -q "drop table test_whitespace" From 6ffdfb8b6b8656dfb2ef004349a3cad82dd03e1f Mon Sep 17 00:00:00 2001 From: santrancisco Date: Wed, 14 Jun 2023 13:29:05 +1000 Subject: [PATCH 1594/2223] test removing CpuInfoUtil.cc and see if build breaks :p --- contrib/arrow-cmake/CMakeLists.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 5fe942d1cd0..01e9fc5fca9 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -139,8 +139,6 @@ set(ORC_SRCS "${ORC_SOURCE_SRC_DIR}/Compression.hh" "${ORC_SOURCE_SRC_DIR}/ConvertColumnReader.cc" "${ORC_SOURCE_SRC_DIR}/ConvertColumnReader.hh" - "${ORC_SOURCE_SRC_DIR}/CpuInfoUtil.cc" - "${ORC_SOURCE_SRC_DIR}/CpuInfoUtil.hh" "${ORC_SOURCE_SRC_DIR}/Dispatch.hh" "${ORC_SOURCE_SRC_DIR}/Exceptions.cc" "${ORC_SOURCE_SRC_DIR}/Int128.cc" From 868c3bd45d8585b03a9afce76e0e7466e675c420 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 14 Jun 2023 04:29:08 +0000 Subject: [PATCH 1595/2223] minor change --- src/Client/ConnectionString.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp index 62090487490..f4a4e73c198 100644 --- a/src/Client/ConnectionString.cpp +++ b/src/Client/ConnectionString.cpp @@ -45,7 +45,7 @@ std::string uriDecode(const std::string & uri_encoded_string, bool plus_as_space void getHostAndPort(const Poco::URI & uri, std::vector> & hosts_and_ports_arguments) { std::vector host_and_port; - const auto& host = uri.getHost(); + const std::string & host = uri.getHost(); if (!host.empty()) { host_and_port.push_back("--host=" + uriDecode(host, false)); From e281026e0085048e8d35d8cca78cc346501ee974 Mon Sep 17 00:00:00 2001 From: Chang Chen Date: Wed, 14 Jun 2023 12:29:55 +0800 Subject: [PATCH 1596/2223] fix build issue on clang 15 --- src/Formats/CapnProtoSerializer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index b306cca4f94..8373c95599f 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -751,7 +751,7 @@ namespace private: using Reader = typename CapnpType::Reader; - CapnpType::Reader getData(const ColumnPtr & column, size_t row_num) + typename CapnpType::Reader getData(const ColumnPtr & column, size_t row_num) { auto data = column->getDataAt(row_num); if constexpr (std::is_same_v) @@ -801,7 +801,7 @@ namespace private: using Reader = typename CapnpType::Reader; - CapnpType::Reader getData(const ColumnPtr & column, size_t row_num) + typename CapnpType::Reader getData(const ColumnPtr & column, size_t row_num) { auto data = column->getDataAt(row_num); if constexpr (std::is_same_v) From 4db8fa39c7904661a7aac6aa62ee4f0e44092369 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 14 Jun 2023 04:38:46 +0000 Subject: [PATCH 1597/2223] Removed extra lines --- docs/en/interfaces/cli.md | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index e2c7dc1e608..8779dd1a544 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -228,9 +228,6 @@ clickhouse://[2001:db8::1234] URI allows multiple hosts to be connected to. Connection strings can contain multiple hosts. ClickHouse-client will try to connect to these hosts in order (i.e. from left to right). After the connection is established, no attempt to connect to the remaining hosts is made. - - - The connection string must be specified as the first argument of clickhouse-client. The connection string can be combined with arbitrary other [command-line-options](#command-line-options) except `--host/-h` and `--port`. The following keys are allowed for component `query_parameter`: From 08cd94e826ce6af55135517d8abcbc72f4b270fb Mon Sep 17 00:00:00 2001 From: Derek Chia Date: Wed, 14 Jun 2023 12:57:50 +0800 Subject: [PATCH 1598/2223] Update delete.md LWD is not supported in table with projection --- docs/en/sql-reference/statements/delete.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index fa9f08e225f..5522c50d624 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -55,6 +55,9 @@ With the described implementation now we can see what can negatively affect 'DEL - Table having a very large number of data parts - Having a lot of data in Compact parts—in a Compact part, all columns are stored in one file. +:::note +Lightweight delete does not work for tables with projection as rows in projection may be affected and require the projection to be rebuilt. Rebuilding projection makes the deletion not lightweight, so this is not supported. +::: ## Related content From f74c585426f0f6dd7f2ce440193cda2c43a9f9d9 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 14 Jun 2023 07:46:11 +0200 Subject: [PATCH 1599/2223] Typos --- .../0_stateless/02784_projections_read_in_order_bug.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql b/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql index 9595fc9ae08..6bf287a3d77 100644 --- a/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql +++ b/tests/queries/0_stateless/02784_projections_read_in_order_bug.sql @@ -37,8 +37,8 @@ create table events ( timestamp )) engine = MergeTree order by (organisation_id, session_id, timestamp) settings index_granularity = 3; -insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)); -insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC')(), toString(0), reinterpretAsUUID(0), toString(0)); +insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)); +insert into events values (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(0), reinterpretAsUUID(1), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(1), reinterpretAsUUID(0), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)), (reinterpretAsUUID(3), reinterpretAsUUID(2), reinterpretAsUUID(0), toDateTime('2022-02-02', 'UTC'), toString(0), reinterpretAsUUID(0), toString(0)); set read_in_order_two_level_merge_threshold=1; SELECT id, timestamp, payload FROM events WHERE (organisation_id = reinterpretAsUUID(1)) AND (session_id = reinterpretAsUUID(0)) ORDER BY timestamp, payload, id ASC; From 0a1d0c4abd0f8ece5af6f3a3d5ccc5207dfff0f2 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 13 Jun 2023 23:36:14 -0700 Subject: [PATCH 1600/2223] Fix one stateless test. --- src/Client/ClientBase.cpp | 5 ++--- tests/queries/0_stateless/02346_into_outfile_and_stdout.sh | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index fc108b8f57d..b746d46148c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -596,14 +596,13 @@ try range.second); } - std::error_code ec; - if (std::filesystem::is_regular_file(out_file, ec)) + if (fs::exists(out_file)) { if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) { throw Exception( ErrorCodes::CANNOT_OPEN_FILE, - "File {} exists, consider using 'INTO OUTFILE ... APPEND' or 'INTO OUTFILE ... TRUNCATE' if appropriate.", + "File {} exists, consider using APPEND or TRUNCATE if appropriate.", out_file); } } diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh index 021dc9125d4..8ec086c97ef 100755 --- a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh @@ -66,7 +66,7 @@ performBadQuery "bad_query_incorrect_usage" "SELECT 1, 2, 3 INTO OUTFILE AND STD performBadQuery "bad_query_no_into_outfile" "SELECT 1, 2, 3 AND STDOUT'" "SYNTAX_ERROR" -performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File exists. (CANNOT_OPEN_FILE)" +performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File ${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists exists, consider using APPEND or TRUNCATE if appropriate. (CANNOT_OPEN_FILE)" performCompression "compression" "SELECT * FROM (SELECT 'Hello, World! From clickhouse.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_compression.gz' AND STDOUT COMPRESSION 'GZ' FORMAT TabSeparated" From 86694847c66a170781e5a561940c05e0221966d0 Mon Sep 17 00:00:00 2001 From: Chang Chen Date: Wed, 14 Jun 2023 15:22:32 +0800 Subject: [PATCH 1601/2223] using Reader instead of typename CapnpType::Reader --- src/Formats/CapnProtoSerializer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp index 8373c95599f..6f7254ab2aa 100644 --- a/src/Formats/CapnProtoSerializer.cpp +++ b/src/Formats/CapnProtoSerializer.cpp @@ -751,7 +751,7 @@ namespace private: using Reader = typename CapnpType::Reader; - typename CapnpType::Reader getData(const ColumnPtr & column, size_t row_num) + Reader getData(const ColumnPtr & column, size_t row_num) { auto data = column->getDataAt(row_num); if constexpr (std::is_same_v) @@ -801,7 +801,7 @@ namespace private: using Reader = typename CapnpType::Reader; - typename CapnpType::Reader getData(const ColumnPtr & column, size_t row_num) + Reader getData(const ColumnPtr & column, size_t row_num) { auto data = column->getDataAt(row_num); if constexpr (std::is_same_v) From b1f0a91b788e2ef89cd5848168e8357d6da13f0a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 14 Jun 2023 07:48:08 +0000 Subject: [PATCH 1602/2223] Docs: Fix embedded video link --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index d8a0193ff66..80e47e76ce0 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -152,7 +152,7 @@ This type of ANN index implements [the Annoy algorithm](https://github.com/spoti space in random linear surfaces (lines in 2D, planes in 3D etc.).
-